├── .gitignore ├── .travis.yml ├── LICENSE ├── README.md ├── build.sbt ├── lib └── sis-jhdf5-batteries_included.jar ├── project ├── plugins.sbt └── scalastyle-config.xml └── src ├── main ├── resources │ └── META-INF │ │ └── services │ │ └── org.apache.spark.sql.sources.DataSourceRegister └── scala │ └── gov │ └── llnl │ └── spark │ └── hdf │ ├── DefaultSource.scala │ ├── HDF5Relation.scala │ ├── ScanExecutor.scala │ ├── SchemaConverter.scala │ ├── package.scala │ └── reader │ ├── DatasetReader.scala │ ├── HDF5Reader.scala │ └── HDF5Schema.scala └── test ├── resources ├── gov │ └── llnl │ │ └── spark │ │ └── hdf │ │ ├── test1.h5 │ │ ├── test2.h5 │ │ └── test3.h5 └── log4j.properties └── scala └── gov └── llnl └── spark └── hdf ├── FunTestSuite.scala └── HDF5QuerySuite.scala /.gitignore: -------------------------------------------------------------------------------- 1 | # Scala ignores 2 | *.class 3 | *.log 4 | 5 | # sbt specific 6 | .cache 7 | .history 8 | .lib/ 9 | dist/* 10 | target/ 11 | lib_managed/ 12 | src_managed/ 13 | project/boot/ 14 | project/plugins/project/ 15 | 16 | !lib/sis-jhdf5-batteries_included.jar 17 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: scala 2 | scala: 3 | - 2.10.6 4 | 5 | cache: 6 | directories: 7 | - $HOME/.ivy2/cache 8 | - $HOME/.sbt/boot/ 9 | 10 | before_cache: 11 | # Tricks to avoid unnecessary cache updates 12 | - find $HOME/.ivy2 -name "ivydata-*.properties" -delete 13 | - find $HOME/.sbt -name "*.lock" -delete 14 | -------------------------------------------------------------------------------- /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. -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Spark-HDF5 [![Build Status](https://travis-ci.org/LLNL/spark-hdf5.svg?branch=master)](https://travis-ci.org/LLNL/spark-hdf5) 2 | 3 | ## Progress 4 | The plugin can read single-dimensional arrays from HDF5 files. 5 | 6 | The following types are supported: 7 | * Int8 8 | * UInt8 9 | * Int16 10 | * UInt16 11 | * Int32 12 | * Int64 13 | * Float32 14 | * Float64 15 | * Fixed length strings 16 | 17 | ## Setup 18 | If you are using the sbt-spark-package, the easiest way to use the package is by requiring it from the [spark packages website](https://spark-packages.org/package/LLNL/spark-hdf5): 19 | ``` 20 | spDependencies += "LLNL/spark-hdf5:0.0.4" 21 | ``` 22 | Otherwise, download the latest release jar and include it on your classpath. 23 | 24 | ## Usage 25 | ```scala 26 | import gov.llnl.spark.hdf._ 27 | 28 | val df = sqlContext.read.hdf5("path/to/file.h5", "/dataset") 29 | df.show 30 | ``` 31 | 32 | You can start a spark repl with the console target: 33 | ``` 34 | sbt console 35 | ``` 36 | This will fetch all of the dependencies, set up a local Spark instance, and start a Spark repl with the plugin loaded. 37 | 38 | ## Options 39 | The following options can be set: 40 | 41 | Key | Default | Description 42 | -------------|---------|------------ 43 | `extension` | `h5` | The file extension of data 44 | `chunk size` | `10000` | The maximum number of elements to be read in a single scan 45 | 46 | ## Testing 47 | The plugin includes a test suite which can be run through SBT 48 | ``` 49 | sbt test 50 | ``` 51 | 52 | ## Roadmap 53 | * Use the [hdf-obj package](https://www.hdfgroup.org/products/java/hdf-object/) rather than the [sis-jhdf5 wrapper](https://wiki-bsse.ethz.ch/pages/viewpage.action?pageId=26609113) 54 | * Support for multi-dimensional arrays 55 | * Support for compound datasets 56 | * Additional testing 57 | * [Partition discovery][2] (data inference based on location) 58 | 59 | [1]: https://github.com/paulp/sbt-extras 60 | [2]: http://spark.apache.org/docs/latest/sql-programming-guide.html#partition-discovery 61 | 62 | ## Release 63 | This code was developed at the Lawrence Livermore National Lab (LLNL) and is available under the [Apache 2.0 license](LICENSE) (`LLNL-CODE-699384`) 64 | -------------------------------------------------------------------------------- /build.sbt: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright (c) 2016, Lawrence Livermore National Security, LLC. 4 | * Produced at the Lawrence Livermore National Laboratory 5 | * 6 | * Written by Joshua Asplund 7 | * LLNL-CODE-699384 8 | * 9 | * All rights reserved. 10 | * 11 | * This file is part of spark-hdf5. 12 | * For details, see https://github.com/LLNL/spark-hdf5 13 | * 14 | * Licensed under the Apache License, Version 2.0 (the "License"); 15 | * you may not use this file except in compliance with the License. 16 | * You may obtain a copy of the License at 17 | * 18 | * http://www.apache.org/licenses/LICENSE-2.0 19 | * 20 | * Unless required by applicable law or agreed to in writing, software 21 | * distributed under the License is distributed on an "AS IS" BASIS, 22 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 23 | * See the License for the specific language governing permissions and 24 | * limitations under the License. 25 | * 26 | */ 27 | 28 | // Project metadata 29 | organization := "LLNL" 30 | name := "spark-hdf5" 31 | version := "0.0.4" 32 | 33 | scalaVersion := "2.11.8" 34 | scalacOptions := Seq( "deprecation", "-feature" ) 35 | 36 | // Spark specific information 37 | sparkVersion := "2.0.0" 38 | sparkComponents ++= Seq("core", "sql") 39 | 40 | licenses += "Apache-2.0" -> url("http://opensource.org/licenses/Apache-2.0") 41 | spIncludeMaven := false 42 | 43 | credentials += Credentials(Path.userHome / ".ivy2" / ".sbtcredentials") 44 | 45 | // Test dependencies 46 | val deps = Seq( 47 | "org.slf4j" % "slf4j-api" % "1.7.5" % "provided", 48 | "org.scalatest" %% "scalatest" % "2.2.1" % "test", 49 | "com.novocode" % "junit-interface" % "0.9" % "test" 50 | ) 51 | 52 | lazy val shaded = Project("shaded", file(".")).settings( 53 | libraryDependencies ++= (deps.map(_ % "provided")), 54 | unmanagedJars += file("lib/sis-jhdf5-batteries_included.jar"), 55 | target := target.value / "shaded" 56 | ) 57 | 58 | lazy val distribute = Project("distribution", file(".")).settings( 59 | spName := "LLNL/spark-hdf5", 60 | spShortDescription := "A plugin to enable Apache Spark to read HDF5 files", 61 | spDescription := "Integrates HDF5 into Spark", 62 | 63 | target := target.value / "distribution", 64 | spShade := true, 65 | assemblyOption in assembly := (assemblyOption in assembly).value.copy(includeScala = false), 66 | libraryDependencies ++= deps 67 | ) 68 | 69 | scalastyleConfig := baseDirectory.value / "project/scalastyle-config.xml" 70 | 71 | lazy val testScalastyle = taskKey[Unit]("testScalastyle") 72 | 73 | testScalastyle := org.scalastyle.sbt.ScalastylePlugin.scalastyle.in(Test).toTask("").value 74 | 75 | (test in Test) <<= (test in Test) dependsOn testScalastyle 76 | 77 | -------------------------------------------------------------------------------- /lib/sis-jhdf5-batteries_included.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/LLNL/spark-hdf5/2fcc3b77e07035f5fbbe3addd267af1ac7149b7e/lib/sis-jhdf5-batteries_included.jar -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Lawrence Livermore National Lab 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | resolvers += "bintray-spark-packages" at "https://dl.bintray.com/spark-packages/maven/" 18 | 19 | addSbtPlugin("org.spark-packages" % "sbt-spark-package" % "0.2.5") 20 | 21 | addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.7.0") 22 | -------------------------------------------------------------------------------- /project/scalastyle-config.xml: -------------------------------------------------------------------------------- 1 | 17 | 43 | 44 | 45 | Scalastyle standard configuration 46 | 47 | 48 | 49 | 50 | 51 | 52 | 53 | 54 | 55 | 60 | * LLNL-CODE-699384 61 | * 62 | * All rights reserved. 63 | * 64 | * This file is part of spark-hdf5. 65 | * For details, see https://github.com/LLNL/spark-hdf5 66 | * 67 | * Licensed under the Apache License, Version 2.0 (the "License"); 68 | * you may not use this file except in compliance with the License. 69 | * You may obtain a copy of the License at 70 | * 71 | * http://www.apache.org/licenses/LICENSE-2.0 72 | * 73 | * Unless required by applicable law or agreed to in writing, software 74 | * distributed under the License is distributed on an "AS IS" BASIS, 75 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 76 | * See the License for the specific language governing permissions and 77 | * limitations under the License. 78 | */]]> 79 | 80 | 81 | 82 | 83 | 84 | 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | true 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 | ARROW, EQUALS, ELSE, TRY, CATCH, FINALLY, LARROW, RARROW 138 | 139 | 140 | 141 | 142 | 143 | ARROW, EQUALS, COMMA, COLON, IF, ELSE, DO, WHILE, FOR, MATCH, TRY, CATCH, FINALLY, LARROW, RARROW 144 | 145 | 146 | 147 | 148 | 149 | 150 | 151 | 152 | ^FunSuite[A-Za-z]*$ 153 | Tests must extend org.apache.spark.SparkFunSuite instead. 154 | 155 | 156 | 157 | 158 | ^println$ 159 | 163 | 164 | 165 | 166 | @VisibleForTesting 167 | 170 | 171 | 172 | 173 | Runtime\.getRuntime\.addShutdownHook 174 | 182 | 183 | 184 | 185 | mutable\.SynchronizedBuffer 186 | 194 | 195 | 196 | 197 | Class\.forName 198 | 205 | 206 | 207 | 208 | Await\.result 209 | 216 | 217 | 218 | 219 | 220 | JavaConversions 221 | Instead of importing implicits in scala.collection.JavaConversions._, import 222 | scala.collection.JavaConverters._ and use .asScala / .asJava methods 223 | 224 | 225 | 226 | 227 | java,scala,3rdParty,spark 228 | javax?\..* 229 | scala\..* 230 | (?!org\.apache\.spark\.).* 231 | org\.apache\.spark\..* 232 | 233 | 234 | 235 | 236 | 237 | COMMA 238 | 239 | 240 | 241 | 242 | 243 | \)\{ 244 | 247 | 248 | 249 | 250 | (?m)^(\s*)/[*][*].*$(\r|)\n^\1 [*] 251 | Use Javadoc style indentation for multiline comments 252 | 253 | 254 | 255 | case[^\n>]*=>\s*\{ 256 | Omit braces in case clauses. 257 | 258 | 259 | 260 | 261 | 262 | 263 | 264 | 265 | 266 | 267 | 268 | 269 | 270 | 271 | 272 | 273 | 274 | 275 | 276 | 277 | 278 | 279 | 280 | 281 | 282 | 283 | 284 | 285 | 286 | 287 | 288 | 289 | 290 | 291 | 292 | 293 | 294 | 295 | 296 | 297 | 298 | 299 | 300 | 301 | 302 | 303 | 304 | 800> 305 | 306 | 307 | 308 | 309 | 30 310 | 311 | 312 | 313 | 314 | 10 315 | 316 | 317 | 318 | 319 | 50 320 | 321 | 322 | 323 | 324 | 325 | 326 | 327 | 328 | 329 | 330 | -1,0,1,2,3 331 | 332 | 333 | 334 | -------------------------------------------------------------------------------- /src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister: -------------------------------------------------------------------------------- 1 | gov.llnl.spark.hdf.DefaultSource15 -------------------------------------------------------------------------------- /src/main/scala/gov/llnl/spark/hdf/DefaultSource.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Lawrence Livermore National Security, LLC. 3 | * Produced at the Lawrence Livermore National Laboratory 4 | * 5 | * Written by Joshua Asplund 6 | * LLNL-CODE-699384 7 | * 8 | * All rights reserved. 9 | * 10 | * This file is part of spark-hdf5. 11 | * For details, see https://github.com/LLNL/spark-hdf5 12 | * 13 | * Licensed under the Apache License, Version 2.0 (the "License"); 14 | * you may not use this file except in compliance with the License. 15 | * You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, software 20 | * distributed under the License is distributed on an "AS IS" BASIS, 21 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 22 | * See the License for the specific language governing permissions and 23 | * limitations under the License. 24 | */ 25 | package gov.llnl.spark.hdf 26 | 27 | import org.apache.spark.SparkException 28 | import org.apache.spark.sql.SQLContext 29 | import org.apache.spark.sql.sources._ 30 | 31 | class DefaultSource extends RelationProvider { 32 | 33 | override def createRelation(sqlContext: SQLContext, 34 | parameters: Map[String, String]): BaseRelation = { 35 | 36 | val paths = parameters.get("path") match { 37 | case Some(x) => x.split(",").map(_.trim) 38 | case None => sys.error("'path' must be specified.") 39 | } 40 | 41 | val dataset = parameters.get("dataset") match { 42 | case Some(x) => x 43 | case None => throw new SparkException("You must provide a path to the dataset") 44 | } 45 | 46 | val extensions = parameters.getOrElse("extension", "h5").split(",").map(_.trim) 47 | val chunkSize = parameters.getOrElse("chunk size", "10000").toInt 48 | 49 | new HDF5Relation(paths, dataset, extensions, chunkSize)(sqlContext) 50 | } 51 | } 52 | 53 | class DefaultSource15 extends DefaultSource with DataSourceRegister { 54 | 55 | /* Extension of spark.hdf5.DefaultSource (which is Spark 1.3 and 1.4 compatible) for Spark 1.5. 56 | * Since the class is loaded through META-INF/services we can decouple the two to have 57 | * Spark 1.5 byte-code loaded lazily. 58 | * 59 | * This trick is adapted from spark elasticsearch-hadoop data source: 60 | * 61 | */ 62 | override def shortName(): String = "hdf5" 63 | 64 | } 65 | -------------------------------------------------------------------------------- /src/main/scala/gov/llnl/spark/hdf/HDF5Relation.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Lawrence Livermore National Security, LLC. 3 | * Produced at the Lawrence Livermore National Laboratory 4 | * 5 | * Written by Joshua Asplund 6 | * LLNL-CODE-699384 7 | * 8 | * All rights reserved. 9 | * 10 | * This file is part of spark-hdf5. 11 | * For details, see https://github.com/LLNL/spark-hdf5 12 | * 13 | * Licensed under the Apache License, Version 2.0 (the "License"); 14 | * you may not use this file except in compliance with the License. 15 | * You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, software 20 | * distributed under the License is distributed on an "AS IS" BASIS, 21 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 22 | * See the License for the specific language governing permissions and 23 | * limitations under the License. 24 | */ 25 | package gov.llnl.spark.hdf 26 | 27 | import java.net.URI 28 | 29 | import gov.llnl.spark.hdf.ScanExecutor.{BoundedScan, UnboundedScan} 30 | import gov.llnl.spark.hdf.reader.HDF5Schema.{Dataset} 31 | import org.apache.commons.io.FilenameUtils 32 | import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} 33 | import org.apache.spark.rdd.RDD 34 | import org.apache.spark.sql.sources.{BaseRelation, TableScan} 35 | import org.apache.spark.sql.types.StructType 36 | import org.apache.spark.sql.{Row, SQLContext} 37 | 38 | class HDF5Relation( val paths: Array[String] 39 | , val dataset: String 40 | , val fileExtension: Array[String] 41 | , val chunkSize: Int) 42 | (@transient val sqlContext: SQLContext) 43 | extends BaseRelation with TableScan { 44 | 45 | val hadoopConfiguration = sqlContext.sparkContext.hadoopConfiguration 46 | val fileSystem = FileSystem.get(hadoopConfiguration) 47 | 48 | lazy val files: Array[URI] = { 49 | val roots = paths.map{ path => 50 | fileSystem.getFileStatus(new Path(path)) }.toSeq 51 | 52 | val leaves = roots.flatMap{ 53 | case status if status.isFile => Set(status) 54 | case directory if directory.isDirectory => 55 | val it = fileSystem.listFiles(directory.getPath, true) 56 | var children: Set[FileStatus] = Set() 57 | while (it.hasNext) { 58 | children += it.next() 59 | } 60 | children 61 | } 62 | 63 | leaves.filter(status => status.isFile) 64 | .map(_.getPath) 65 | .filter(path => fileExtension.contains(FilenameUtils.getExtension(path.toString))) 66 | .map(org.apache.hadoop.fs.Path.getPathWithoutSchemeAndAuthority(_).toUri) 67 | .toArray 68 | } 69 | 70 | private lazy val datasets: Array[Dataset[_]] = files.flatMap { 71 | file => new ScanExecutor(file.toString).openReader(_.getObject(dataset)) 72 | }.collect { case y: Dataset[_] => y } 73 | 74 | private lazy val hdf5Schema: Dataset[_] = datasets match { 75 | case Array(head: Dataset[_], _*) => head 76 | case _ => throw new java.io.FileNotFoundException("No files") 77 | } 78 | 79 | override def schema: StructType = SchemaConverter.convertSchema(hdf5Schema) 80 | 81 | override def buildScan(): RDD[Row] = { 82 | val scans = datasets.map{ UnboundedScan(_, chunkSize) } 83 | val splitScans = scans.flatMap{ 84 | case UnboundedScan(ds, size) if ds.size > size => 85 | (0L until Math.ceil(ds.size.toFloat / size).toLong).map(x => BoundedScan(ds, size, x)) 86 | case x: UnboundedScan => Seq(x) 87 | } 88 | sqlContext.sparkContext.parallelize(splitScans).flatMap{ item => 89 | new ScanExecutor(item.dataset.file).execQuery(item) 90 | } 91 | } 92 | 93 | } 94 | -------------------------------------------------------------------------------- /src/main/scala/gov/llnl/spark/hdf/ScanExecutor.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Lawrence Livermore National Security, LLC. 3 | * Produced at the Lawrence Livermore National Laboratory 4 | * 5 | * Written by Joshua Asplund 6 | * LLNL-CODE-699384 7 | * 8 | * All rights reserved. 9 | * 10 | * This file is part of spark-hdf5. 11 | * For details, see https://github.com/LLNL/spark-hdf5 12 | * 13 | * Licensed under the Apache License, Version 2.0 (the "License"); 14 | * you may not use this file except in compliance with the License. 15 | * You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, software 20 | * distributed under the License is distributed on an "AS IS" BASIS, 21 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 22 | * See the License for the specific language governing permissions and 23 | * limitations under the License. 24 | */ 25 | package gov.llnl.spark.hdf 26 | 27 | import java.io.File 28 | import java.net.URI 29 | 30 | import gov.llnl.spark.hdf.ScanExecutor.{BoundedScan, ScanItem, UnboundedScan} 31 | import gov.llnl.spark.hdf.reader.{DatasetReader, HDF5Reader} 32 | import gov.llnl.spark.hdf.reader.HDF5Schema.Dataset 33 | import org.apache.spark.sql.Row 34 | 35 | import scala.language.existentials 36 | 37 | object ScanExecutor { 38 | sealed trait ScanItem { 39 | val dataset: Dataset[_] 40 | val chunkSize: Int 41 | } 42 | case class UnboundedScan(dataset: Dataset[_], chunkSize: Int) extends ScanItem 43 | case class BoundedScan(dataset: Dataset[_] 44 | , chunkSize: Int 45 | , chunkNumber: Long = 0) extends ScanItem 46 | } 47 | 48 | class ScanExecutor(filePath: String) extends Serializable { 49 | 50 | def execQuery[T](scanItem: ScanItem): Seq[Row] = scanItem match { 51 | case UnboundedScan(dataset, _) => 52 | val dataReader = newDatasetReader(dataset)(_.readDataset()) 53 | dataReader.zipWithIndex.map { case (x, index) => Row(index.toLong, x) } 54 | case BoundedScan(dataset, size, number) => 55 | val dataReader = newDatasetReader(dataset)(_.readDataset(size, number)) 56 | dataReader.zipWithIndex.map { case (x, index) => Row((size * number) + index.toLong, x) } 57 | } 58 | 59 | def openReader[T](fun: HDF5Reader => T): T = { 60 | val file = new File(filePath) 61 | val reader = new HDF5Reader(file) 62 | val result = fun(reader) 63 | reader.close() 64 | result 65 | } 66 | 67 | def newDatasetReader[S, T](node: Dataset[T])(fun: DatasetReader[T] => S): S = { 68 | openReader(reader => reader.getDataset(node)(fun)) 69 | } 70 | 71 | } 72 | -------------------------------------------------------------------------------- /src/main/scala/gov/llnl/spark/hdf/SchemaConverter.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Lawrence Livermore National Security, LLC. 3 | * Produced at the Lawrence Livermore National Laboratory 4 | * 5 | * Written by Joshua Asplund 6 | * LLNL-CODE-699384 7 | * 8 | * All rights reserved. 9 | * 10 | * This file is part of spark-hdf5. 11 | * For details, see https://github.com/LLNL/spark-hdf5 12 | * 13 | * Licensed under the Apache License, Version 2.0 (the "License"); 14 | * you may not use this file except in compliance with the License. 15 | * You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, software 20 | * distributed under the License is distributed on an "AS IS" BASIS, 21 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 22 | * See the License for the specific language governing permissions and 23 | * limitations under the License. 24 | */ 25 | package gov.llnl.spark.hdf 26 | 27 | import gov.llnl.spark.hdf.reader.HDF5Schema._ 28 | import org.apache.spark.sql.types._ 29 | 30 | object SchemaConverter { 31 | 32 | def convertSchema(dataset: Dataset[_]): StructType = { 33 | val columns = dataset.dimension.indices.map { 34 | index => "index" + index 35 | }.map { 36 | name => StructField(name, LongType, nullable = false) 37 | } 38 | StructType(columns :+ StructField("value", extractTypes(dataset.contains))) 39 | } 40 | 41 | def extractTypes(datatype: HDF5Type[_]): DataType = datatype match { 42 | case Int8(_, _) => ByteType 43 | case UInt8(_, _) => ShortType 44 | case Int16(_, _) => ShortType 45 | case UInt16(_, _) => IntegerType 46 | case Int32(_, _) => IntegerType 47 | case UInt32(_, _) => LongType 48 | case Int64(_, _) => LongType 49 | case Float32(_, _) => FloatType 50 | case Float64(_, _) => DoubleType 51 | case FLString(_, _) => StringType 52 | } 53 | 54 | } 55 | -------------------------------------------------------------------------------- /src/main/scala/gov/llnl/spark/hdf/package.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Lawrence Livermore National Security, LLC. 3 | * Produced at the Lawrence Livermore National Laboratory 4 | * 5 | * Written by Joshua Asplund 6 | * LLNL-CODE-699384 7 | * 8 | * All rights reserved. 9 | * 10 | * This file is part of spark-hdf5. 11 | * For details, see https://github.com/LLNL/spark-hdf5 12 | * 13 | * Licensed under the Apache License, Version 2.0 (the "License"); 14 | * you may not use this file except in compliance with the License. 15 | * You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, software 20 | * distributed under the License is distributed on an "AS IS" BASIS, 21 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 22 | * See the License for the specific language governing permissions and 23 | * limitations under the License. 24 | */ 25 | package gov.llnl.spark 26 | 27 | import org.apache.spark.sql.{DataFrame, DataFrameReader} 28 | 29 | package object hdf { 30 | 31 | /* 32 | * Adds a method, `hdf5`, to DataFrameReader 33 | */ 34 | implicit class HDF5DataFrameReader(reader: DataFrameReader) { 35 | 36 | def hdf5(file: String, dataset: String): DataFrame = 37 | reader.format("gov.llnl.spark.hdf").option("dataset", dataset).load(file) 38 | 39 | } 40 | 41 | } 42 | -------------------------------------------------------------------------------- /src/main/scala/gov/llnl/spark/hdf/reader/DatasetReader.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Lawrence Livermore National Security, LLC. 3 | * Produced at the Lawrence Livermore National Laboratory 4 | * 5 | * Written by Joshua Asplund 6 | * LLNL-CODE-699384 7 | * 8 | * All rights reserved. 9 | * 10 | * This file is part of spark-hdf5. 11 | * For details, see https://github.com/LLNL/spark-hdf5 12 | * 13 | * Licensed under the Apache License, Version 2.0 (the "License"); 14 | * you may not use this file except in compliance with the License. 15 | * You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, software 20 | * distributed under the License is distributed on an "AS IS" BASIS, 21 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 22 | * See the License for the specific language governing permissions and 23 | * limitations under the License. 24 | */ 25 | package gov.llnl.spark.hdf.reader 26 | 27 | import ch.systemsx.cisd.hdf5.IHDF5Reader 28 | import gov.llnl.spark.hdf.reader.HDF5Schema.Dataset 29 | 30 | class DatasetReader[T](val reader: IHDF5Reader, val node: Dataset[T]) extends Serializable { 31 | 32 | def readDataset(): Array[T] = 33 | node.contains.readArray(reader) 34 | 35 | def readDataset(blockSize: Int, blockNumber: Long): Array[T] = 36 | node.contains.readArrayBlock(reader, blockSize, blockNumber) 37 | 38 | } 39 | -------------------------------------------------------------------------------- /src/main/scala/gov/llnl/spark/hdf/reader/HDF5Reader.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Lawrence Livermore National Security, LLC. 3 | * Produced at the Lawrence Livermore National Laboratory 4 | * 5 | * Written by Joshua Asplund 6 | * LLNL-CODE-699384 7 | * 8 | * All rights reserved. 9 | * 10 | * This file is part of spark-hdf5. 11 | * For details, see https://github.com/LLNL/spark-hdf5 12 | * 13 | * Licensed under the Apache License, Version 2.0 (the "License"); 14 | * you may not use this file except in compliance with the License. 15 | * You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, software 20 | * distributed under the License is distributed on an "AS IS" BASIS, 21 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 22 | * See the License for the specific language governing permissions and 23 | * limitations under the License. 24 | */ 25 | package gov.llnl.spark.hdf.reader 26 | 27 | import java.io.{Closeable, File} 28 | 29 | import ch.systemsx.cisd.hdf5.{HDF5DataClass, HDF5DataTypeInformation, HDF5FactoryProvider, IHDF5Reader} 30 | import gov.llnl.spark.hdf.reader.HDF5Schema._ 31 | 32 | import scala.collection.JavaConverters._ 33 | 34 | class HDF5Reader(val input: File) extends Closeable with Serializable { 35 | lazy val path: String = input.getPath 36 | 37 | val reader = HDF5FactoryProvider.get().openForReading(input) 38 | 39 | def getSchema: HDF5Node = listMembers() 40 | 41 | private lazy val objects = getSchema.flatten().map { 42 | case node@Dataset(_, name, _, _, _) => (name, node) 43 | case node@Group(_, name, _) => (name, node) 44 | case node@GenericNode(_, _) => (node.path, node) 45 | }.toMap 46 | 47 | def getObject(path: String): Option[HDF5Node] = objects.get(path) 48 | 49 | override def close(): Unit = reader.close() 50 | 51 | def isDataset(path: String): Boolean = !reader.isGroup(path) 52 | 53 | def getDataset[S, T](dataset: Dataset[T])(fun: DatasetReader[T] => S): S = 54 | fun(new DatasetReader[T](reader, dataset)) 55 | 56 | def listMembers(name: String = "/"): HDF5Node = { 57 | reader.isGroup(name) match { 58 | case true => 59 | val children = reader.getGroupMembers(name).asScala 60 | name match { 61 | case "/" => Group(path, name, children.map { x => listMembers("/" + x) }) 62 | case _ => Group(path, name, children.map { x => listMembers(name + "/" + x) }) 63 | } 64 | case false => 65 | val info = reader.getDataSetInformation(name) 66 | val hdfType = infoToType(name, info.getTypeInformation) 67 | Dataset(path, name, hdfType, info.getDimensions, info.getNumberOfElements) 68 | } 69 | } 70 | 71 | def infoToType(name: String, info: HDF5DataTypeInformation): HDF5Type[_] = { 72 | (info.getDataClass, info.isSigned, info.getElementSize) match { 73 | case (HDF5DataClass.INTEGER, true, 1) => HDF5Schema.Int8(path, name) 74 | case (HDF5DataClass.INTEGER, false, 1) => HDF5Schema.UInt8(path, name) 75 | case (HDF5DataClass.INTEGER, true, 2) => HDF5Schema.Int16(path, name) 76 | case (HDF5DataClass.INTEGER, false, 2) => HDF5Schema.UInt16(path, name) 77 | case (HDF5DataClass.INTEGER, true, 4) => HDF5Schema.Int32(path, name) 78 | case (HDF5DataClass.INTEGER, false, 4) => HDF5Schema.UInt32(path, name) 79 | case (HDF5DataClass.INTEGER, true, 8) => HDF5Schema.Int64(path, name) 80 | case (HDF5DataClass.FLOAT, true, 4) => HDF5Schema.Float32(path, name) 81 | case (HDF5DataClass.FLOAT, true, 8) => HDF5Schema.Float64(path, name) 82 | case (HDF5DataClass.STRING, signed, size) => HDF5Schema.FLString(path, name) 83 | case _ => throw new NotImplementedError("Type not supported") 84 | } 85 | } 86 | 87 | } 88 | -------------------------------------------------------------------------------- /src/main/scala/gov/llnl/spark/hdf/reader/HDF5Schema.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Lawrence Livermore National Security, LLC. 3 | * Produced at the Lawrence Livermore National Laboratory 4 | * 5 | * Written by Joshua Asplund 6 | * LLNL-CODE-699384 7 | * 8 | * All rights reserved. 9 | * 10 | * This file is part of spark-hdf5. 11 | * For details, see https://github.com/LLNL/spark-hdf5 12 | * 13 | * Licensed under the Apache License, Version 2.0 (the "License"); 14 | * you may not use this file except in compliance with the License. 15 | * You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, software 20 | * distributed under the License is distributed on an "AS IS" BASIS, 21 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 22 | * See the License for the specific language governing permissions and 23 | * limitations under the License. 24 | */ 25 | package gov.llnl.spark.hdf.reader 26 | 27 | import ch.systemsx.cisd.hdf5._ 28 | 29 | object HDF5Schema { 30 | 31 | // TODO: Needs reference, time, unsigned, compound, enumeration 32 | // case COMPOUND 33 | sealed trait HDF5Type[T] { 34 | def readArray(reader: IHDF5Reader): Array[T] 35 | def readArrayBlock(reader: IHDF5Reader 36 | , blockSize: Int 37 | , blockNumber: Long): Array[T] 38 | } 39 | 40 | case class Int8(file: String, name: String) extends HDF5Type[Byte] { 41 | override def readArray(reader: IHDF5Reader): Array[Byte] = 42 | reader.int8.readArray(name) 43 | 44 | override def readArrayBlock(reader: IHDF5Reader 45 | , blockSize: Int 46 | , blockNumber: Long): Array[Byte] = 47 | reader.int8.readArrayBlock(name, blockSize, blockNumber) 48 | } 49 | 50 | case class UInt8(file: String, name: String) extends HDF5Type[Short] { 51 | override def readArray(reader: IHDF5Reader): Array[Short] = 52 | reader.uint8.readArray(name).map(UnsignedIntUtils.toUint8) 53 | 54 | override def readArrayBlock(reader: IHDF5Reader 55 | , blockSize: Int 56 | , blockNumber: Long): Array[Short] = 57 | reader.uint8.readArrayBlock(name, blockSize, blockNumber).map(UnsignedIntUtils.toUint8) 58 | } 59 | 60 | case class Int16(file: String, name: String) extends HDF5Type[Short] { 61 | override def readArray(reader: IHDF5Reader): Array[Short] = 62 | reader.int16.readArray(name) 63 | 64 | override def readArrayBlock(reader: IHDF5Reader 65 | , blockSize: Int 66 | , blockNumber: Long): Array[Short] = 67 | reader.int16.readArrayBlock(name, blockSize, blockNumber) 68 | } 69 | 70 | case class UInt16(file: String, name: String) extends HDF5Type[Int] { 71 | override def readArray(reader: IHDF5Reader): Array[Int] = 72 | reader.uint16.readArray(name).map(UnsignedIntUtils.toUint16) 73 | 74 | override def readArrayBlock(reader: IHDF5Reader 75 | , blockSize: Int 76 | , blockNumber: Long): Array[Int] = 77 | reader.uint16.readArrayBlock(name, blockSize, blockNumber).map(UnsignedIntUtils.toUint16) 78 | } 79 | 80 | case class Int32(file: String, name: String) extends HDF5Type[Int] { 81 | override def readArray(reader: IHDF5Reader): Array[Int] = 82 | reader.int32.readArray(name) 83 | 84 | override def readArrayBlock(reader: IHDF5Reader 85 | , blockSize: Int 86 | , blockNumber: Long): Array[Int] = 87 | reader.int32.readArrayBlock(name, blockSize, blockNumber) 88 | } 89 | 90 | case class UInt32(file: String, name: String) extends HDF5Type[Long] { 91 | override def readArray(reader: IHDF5Reader): Array[Long] = 92 | reader.uint32.readArray(name).map(UnsignedIntUtils.toUint32) 93 | 94 | override def readArrayBlock(reader: IHDF5Reader 95 | , blockSize: Int 96 | , blockNumber: Long): Array[Long] = 97 | reader.uint32.readArrayBlock(name, blockSize, blockNumber).map(UnsignedIntUtils.toUint32) 98 | } 99 | 100 | case class Int64(file: String, name: String) extends HDF5Type[Long] { 101 | override def readArray(reader: IHDF5Reader): Array[Long] = 102 | reader.int64.readArray(name) 103 | 104 | override def readArrayBlock(reader: IHDF5Reader 105 | , blockSize: Int 106 | , blockNumber: Long): Array[Long] = 107 | reader.int64.readArrayBlock(name, blockSize, blockNumber) 108 | } 109 | 110 | case class Float32(file: String, name: String) extends HDF5Type[Float] { 111 | override def readArray(reader: IHDF5Reader): Array[Float] = 112 | reader.float32.readArray(name) 113 | 114 | override def readArrayBlock(reader: IHDF5Reader 115 | , blockSize: Int 116 | , blockNumber: Long): Array[Float] = 117 | reader.float32.readArrayBlock(name, blockSize, blockNumber) 118 | } 119 | 120 | case class Float64(file: String, name: String) extends HDF5Type[Double] { 121 | override def readArray(reader: IHDF5Reader): Array[Double] = 122 | reader.float64.readArray(name) 123 | 124 | override def readArrayBlock(reader: IHDF5Reader 125 | , blockSize: Int 126 | , blockNumber: Long): Array[Double] = 127 | reader.float64.readArrayBlock(name, blockSize, blockNumber) 128 | } 129 | 130 | case class FLString(file: String, name: String) extends HDF5Type[String] { 131 | override def readArray(reader: IHDF5Reader): Array[String] = 132 | reader.string.readArray(name) 133 | 134 | override def readArrayBlock(reader: IHDF5Reader 135 | , blockSize: Int 136 | , blockNumber: Long): Array[String] = 137 | reader.string.readArrayBlock(name, blockSize, blockNumber) 138 | } 139 | 140 | sealed trait HDF5Node { 141 | val file: String 142 | val path: String 143 | 144 | def flatten(): Seq[HDF5Node] 145 | } 146 | 147 | case class Dataset[T]( file: String 148 | , path: String 149 | , contains: HDF5Type[T] 150 | , dimension: Array[Long] 151 | , size: Long) extends HDF5Node with Serializable { 152 | def flatten(): Seq[HDF5Node] = Seq(this) 153 | } 154 | 155 | case class Group(file: String, path: String, children: Seq[HDF5Node]) extends HDF5Node { 156 | def flatten(): Seq[HDF5Node] = this +: children.flatMap(x => x.flatten()) 157 | } 158 | 159 | case class GenericNode(file: String, path: String) extends HDF5Node { 160 | def flatten(): Seq[HDF5Node] = Seq(this) 161 | } 162 | 163 | } 164 | -------------------------------------------------------------------------------- /src/test/resources/gov/llnl/spark/hdf/test1.h5: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/LLNL/spark-hdf5/2fcc3b77e07035f5fbbe3addd267af1ac7149b7e/src/test/resources/gov/llnl/spark/hdf/test1.h5 -------------------------------------------------------------------------------- /src/test/resources/gov/llnl/spark/hdf/test2.h5: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/LLNL/spark-hdf5/2fcc3b77e07035f5fbbe3addd267af1ac7149b7e/src/test/resources/gov/llnl/spark/hdf/test2.h5 -------------------------------------------------------------------------------- /src/test/resources/gov/llnl/spark/hdf/test3.h5: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/LLNL/spark-hdf5/2fcc3b77e07035f5fbbe3addd267af1ac7149b7e/src/test/resources/gov/llnl/spark/hdf/test3.h5 -------------------------------------------------------------------------------- /src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Copyright (c) 2016, Lawrence Livermore National Security, LLC. 2 | # Produced at the Lawrence Livermore National Laboratory 3 | # 4 | # Written by Joshua Asplund 5 | # LLNL-CODE-699384 6 | # 7 | # All rights reserved. 8 | # 9 | # This file is part of spark-hdf5. 10 | # For details, see https://github.com/LLNL/spark-hdf5 11 | # 12 | # Licensed under the Apache License, Version 2.0 (the "License"); 13 | # you may not use this file except in compliance with the License. 14 | # You may obtain a copy of the License at 15 | # 16 | # http://www.apache.org/licenses/LICENSE-2.0 17 | # 18 | # Unless required by applicable law or agreed to in writing, software 19 | # distributed under the License is distributed on an "AS IS" BASIS, 20 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 21 | # See the License for the specific language governing permissions and 22 | # limitations under the License. 23 | 24 | test.appender=file 25 | log4j.rootCategory=INFO, ${test.appender} 26 | log4j.appender.file=org.apache.log4j.FileAppender 27 | log4j.appender.file.append=true 28 | log4j.appender.file.file=target/unit-tests.log 29 | log4j.appender.file.layout=org.apache.log4j.PatternLayout 30 | log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n 31 | -------------------------------------------------------------------------------- /src/test/scala/gov/llnl/spark/hdf/FunTestSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Lawrence Livermore National Security, LLC. 3 | * Produced at the Lawrence Livermore National Laboratory 4 | * 5 | * Written by Joshua Asplund 6 | * LLNL-CODE-699384 7 | * 8 | * All rights reserved. 9 | * 10 | * This file is part of spark-hdf5. 11 | * For details, see https://github.com/LLNL/spark-hdf5 12 | * 13 | * Licensed under the Apache License, Version 2.0 (the "License"); 14 | * you may not use this file except in compliance with the License. 15 | * You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, software 20 | * distributed under the License is distributed on an "AS IS" BASIS, 21 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 22 | * See the License for the specific language governing permissions and 23 | * limitations under the License. 24 | */ 25 | package gov.llnl.spark.hdf 26 | 27 | import org.apache.spark.sql._ 28 | import org.apache.spark.{SparkConf, SparkContext} 29 | import org.scalatest.{BeforeAndAfterAll, FunSuite} 30 | 31 | /* 32 | * Base abstract class for all unit tests in Spark for handling common functionality. 33 | */ 34 | abstract class FunTestSuite extends FunSuite with BeforeAndAfterAll { 35 | 36 | private val sparkConf = new SparkConf() 37 | 38 | protected var sqlContext: SQLContext = _ 39 | 40 | override protected def beforeAll(): Unit = { 41 | super.beforeAll() 42 | sqlContext = new SQLContext(new SparkContext("local[2]", "HDF5Suite", sparkConf)) 43 | } 44 | 45 | override protected def afterAll(): Unit = { 46 | try { 47 | sqlContext.sparkContext.stop() 48 | } finally { 49 | super.afterAll() 50 | } 51 | } 52 | 53 | def checkEqual(df: DataFrame, expected: Seq[Row]): Unit = { 54 | assert(df.collect.toSet === expected.toSet) 55 | } 56 | 57 | } 58 | -------------------------------------------------------------------------------- /src/test/scala/gov/llnl/spark/hdf/HDF5QuerySuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Lawrence Livermore National Security, LLC. 3 | * Produced at the Lawrence Livermore National Laboratory 4 | * 5 | * Written by Joshua Asplund 6 | * LLNL-CODE-699384 7 | * 8 | * All rights reserved. 9 | * 10 | * This file is part of spark-hdf5. 11 | * For details, see https://github.com/LLNL/spark-hdf5 12 | * 13 | * Licensed under the Apache License, Version 2.0 (the "License"); 14 | * you may not use this file except in compliance with the License. 15 | * You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, software 20 | * distributed under the License is distributed on an "AS IS" BASIS, 21 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 22 | * See the License for the specific language governing permissions and 23 | * limitations under the License. 24 | */ 25 | package gov.llnl.spark.hdf 26 | 27 | import org.apache.commons.io.FilenameUtils 28 | import org.apache.spark.sql.Row 29 | import org.apache.spark.sql.types._ 30 | 31 | class HDF5QuerySuite extends FunTestSuite { 32 | 33 | val h5file = getClass.getResource("test1.h5").toString 34 | val h5dir = FilenameUtils.getFullPathNoEndSeparator(getClass.getResource("test1.h5").getPath) 35 | 36 | val multiDataset = "/multi" 37 | val int8test = "/datatypes/int8" 38 | val int16test = "/datatypes/int16" 39 | val int32test = "/datatypes/int32" 40 | val int64test = "/datatypes/int64" 41 | val uint8test = "/datatypes/uint8" 42 | val uint16test = "/datatypes/uint16" 43 | val uint32test = "/datatypes/uint32" 44 | val float32test = "/datatypes/float32" 45 | val float64test = "/datatypes/float64" 46 | 47 | test("Reading multiple files") { 48 | val df = sqlContext.read.hdf5(h5dir, multiDataset) 49 | val expected = (0 until 30).map{ x => Row(x % 10, x) } 50 | 51 | checkEqual(df, expected) 52 | } 53 | 54 | test("Read files in chunks") { 55 | val evenchunkeddf = sqlContext.read 56 | .option("chunk size", 5.toString) 57 | .hdf5(h5file, int8test) 58 | 59 | val oddchunkeddf = sqlContext.read 60 | .option("chunk size", 3.toString) 61 | .hdf5(h5file, int8test) 62 | 63 | val expected = Row(0L, Byte.MinValue) +: 64 | (1L until 9L).map { x => Row(x, (x - 2).toByte) } :+ 65 | Row(9L, Byte.MaxValue) 66 | 67 | checkEqual(evenchunkeddf, expected) 68 | checkEqual(oddchunkeddf, expected) 69 | } 70 | 71 | // Signed integer tests 72 | 73 | test("Reading int8") { 74 | val df = sqlContext.read.hdf5(h5file, int8test) 75 | 76 | val expectedSchema = StructType(Seq( 77 | StructField("index0", LongType, nullable = false), 78 | StructField("value", ByteType))) 79 | assert(df.schema === expectedSchema) 80 | 81 | val expected = Row(0L, Byte.MinValue) +: 82 | (1L until 9L).map { x => Row(x, (x - 2).toByte) } :+ 83 | Row(9L, Byte.MaxValue) 84 | 85 | checkEqual(df, expected) 86 | } 87 | 88 | test("Reading int16") { 89 | val df = sqlContext.read.hdf5(h5file, int16test) 90 | 91 | val expectedSchema = StructType(Seq( 92 | StructField("index0", LongType, nullable = false), 93 | StructField("value", ShortType))) 94 | assert(df.schema === expectedSchema) 95 | 96 | val expected = Row(0L, Short.MinValue) +: 97 | (1L until 9L).map { x => Row(x, (x - 2).toShort) } :+ 98 | Row(9L, Short.MaxValue) 99 | 100 | checkEqual(df, expected) 101 | } 102 | 103 | test("Reading int32") { 104 | val df = sqlContext.read.hdf5(h5file, int32test) 105 | 106 | val expectedSchema = StructType(Seq( 107 | StructField("index0", LongType, nullable = false), 108 | StructField("value", IntegerType))) 109 | assert(df.schema === expectedSchema) 110 | 111 | val expected = Row(0L, Int.MinValue) +: 112 | (1L until 9L).map { x => Row(x, (x - 2).toInt) } :+ 113 | Row(9L, Int.MaxValue) 114 | 115 | checkEqual(df, expected) 116 | } 117 | 118 | test("Reading int64") { 119 | val df = sqlContext.read.hdf5(h5file, int64test) 120 | 121 | val expectedSchema = StructType(Seq( 122 | StructField("index0", LongType, nullable = false), 123 | StructField("value", LongType))) 124 | assert(df.schema === expectedSchema) 125 | 126 | val expected = Row(0L, Long.MinValue) +: 127 | (1L until 9L).map { x => Row(x, x - 2) } :+ 128 | Row(9L, Long.MaxValue) 129 | 130 | checkEqual(df, expected) 131 | } 132 | 133 | // Unsigned integer tests 134 | 135 | test("Reading uint8") { 136 | val df = sqlContext.read.hdf5(h5file, uint8test) 137 | 138 | val expectedSchema = StructType(Seq( 139 | StructField("index0", LongType, nullable = false), 140 | StructField("value", ShortType))) 141 | assert(df.schema === expectedSchema) 142 | 143 | val expected = (0L until 9L).map { x => Row(x, x.toShort) } :+ Row(9L, 255) 144 | 145 | checkEqual(df, expected) 146 | } 147 | 148 | test("Reading uint16") { 149 | val df = sqlContext.read.hdf5(h5file, uint16test) 150 | 151 | val expectedSchema = StructType(Seq( 152 | StructField("index0", LongType, nullable = false), 153 | StructField("value", IntegerType))) 154 | assert(df.schema === expectedSchema) 155 | 156 | val expected = (0L until 9L).map { x => Row(x, x.toInt) } :+ Row(9L, 65535) 157 | 158 | checkEqual(df, expected) 159 | } 160 | 161 | test("Reading uint32") { 162 | val df = sqlContext.read.hdf5(h5file, uint32test) 163 | 164 | val expectedSchema = StructType(Seq( 165 | StructField("index0", LongType, nullable = false), 166 | StructField("value", LongType))) 167 | assert(df.schema === expectedSchema) 168 | 169 | val expected = (0L until 9L).map { x => Row(x, x) } :+ Row(9L, 4294967295L) 170 | 171 | checkEqual(df, expected) 172 | } 173 | 174 | // Float tests 175 | 176 | test("Reading float32") { 177 | val df = sqlContext.read.hdf5(h5file, float32test) 178 | 179 | val expectedSchema = StructType(Seq( 180 | StructField("index0", LongType, nullable = false), 181 | StructField("value", FloatType))) 182 | assert(df.schema === expectedSchema) 183 | 184 | val expected = (0 until 10).map(x => x % 2 match { 185 | case 0 => Row(x, (0.2 * x).toFloat) 186 | case 1 => Row(x, (-0.2 * x).toFloat) 187 | }) 188 | 189 | checkEqual(df, expected) 190 | } 191 | 192 | test("Reading float64") { 193 | val df = sqlContext.read.hdf5(h5file, float64test) 194 | 195 | val expectedSchema = StructType(Seq( 196 | StructField("index0", LongType, nullable = false), 197 | StructField("value", DoubleType))) 198 | assert(df.schema === expectedSchema) 199 | 200 | val expected = (0 until 10).map(x => x % 2 match { 201 | case 0 => Row(x, (2 * x).toDouble / 10) 202 | case 1 => Row(x, (-2 * x).toDouble / 10) 203 | }) 204 | 205 | checkEqual(df, expected) 206 | } 207 | 208 | test("Reading fixed length strings") { 209 | val dataset = "/datatypes/string" 210 | val alpha = "abcdefghijklmnopqrstuvwxyz" 211 | val df = sqlContext.read.hdf5(h5file, dataset) 212 | 213 | val expectedSchema = StructType(Seq( 214 | StructField("index0", LongType, nullable = false), 215 | StructField("value", StringType))) 216 | assert(df.schema === expectedSchema) 217 | 218 | val expected = (0 until 10).map{x => Row(x, alpha.substring(0, 0 + x))} 219 | 220 | checkEqual(df, expected) 221 | } 222 | } 223 | --------------------------------------------------------------------------------