embulk-output-s3_parquet 0.0.2

Sign up to get free protection for your applications and to get access to all the features.
Files changed (30) hide show
  1. checksums.yaml +7 -0
  2. data/.gitignore +12 -0
  3. data/.scalafmt.conf +9 -0
  4. data/CHANGELOG.md +9 -0
  5. data/LICENSE.txt +21 -0
  6. data/README.md +122 -0
  7. data/build.gradle +101 -0
  8. data/example/config.yml +25 -0
  9. data/example/data.tsv +5 -0
  10. data/gradle/wrapper/gradle-wrapper.jar +0 -0
  11. data/gradle/wrapper/gradle-wrapper.properties +5 -0
  12. data/gradlew +172 -0
  13. data/gradlew.bat +84 -0
  14. data/lib/embulk/output/s3_parquet.rb +3 -0
  15. data/settings.gradle +1 -0
  16. data/src/main/scala/org/embulk/output/s3_parquet/S3ParquetOutputPlugin.scala +199 -0
  17. data/src/main/scala/org/embulk/output/s3_parquet/S3ParquetPageOutput.scala +65 -0
  18. data/src/main/scala/org/embulk/output/s3_parquet/aws/Aws.scala +45 -0
  19. data/src/main/scala/org/embulk/output/s3_parquet/aws/AwsClientConfiguration.scala +34 -0
  20. data/src/main/scala/org/embulk/output/s3_parquet/aws/AwsCredentials.scala +128 -0
  21. data/src/main/scala/org/embulk/output/s3_parquet/aws/AwsEndpointConfiguration.scala +49 -0
  22. data/src/main/scala/org/embulk/output/s3_parquet/aws/AwsS3Configuration.scala +56 -0
  23. data/src/main/scala/org/embulk/output/s3_parquet/aws/HttpProxy.scala +56 -0
  24. data/src/main/scala/org/embulk/output/s3_parquet/parquet/EmbulkMessageType.scala +59 -0
  25. data/src/main/scala/org/embulk/output/s3_parquet/parquet/ParquetFileWriteSupport.scala +33 -0
  26. data/src/main/scala/org/embulk/output/s3_parquet/parquet/ParquetFileWriter.scala +125 -0
  27. data/src/test/resources/org/embulk/output/s3_parquet/in1.csv +6 -0
  28. data/src/test/resources/org/embulk/output/s3_parquet/out1.tsv +5 -0
  29. data/src/test/scala/org/embulk/output/s3_parquet/TestS3ParquetOutputPlugin.scala +140 -0
  30. metadata +184 -0
@@ -0,0 +1,49 @@
1
+ package org.embulk.output.s3_parquet.aws
2
+
3
+
4
+ import java.util.Optional
5
+
6
+ import com.amazonaws.client.builder.AwsClientBuilder
7
+ import com.amazonaws.client.builder.AwsClientBuilder.EndpointConfiguration
8
+ import com.amazonaws.regions.{DefaultAwsRegionProviderChain, Regions}
9
+ import org.embulk.config.{Config, ConfigDefault}
10
+ import org.embulk.output.s3_parquet.aws.AwsEndpointConfiguration.Task
11
+
12
+ import scala.util.Try
13
+
14
+ object AwsEndpointConfiguration {
15
+
16
+ trait Task {
17
+
18
+ @Config("endpoint")
19
+ @ConfigDefault("null")
20
+ def getEndpoint: Optional[String]
21
+
22
+ @Config("region")
23
+ @ConfigDefault("null")
24
+ def getRegion: Optional[String]
25
+
26
+ }
27
+
28
+ def apply(task: Task): AwsEndpointConfiguration = new AwsEndpointConfiguration(task)
29
+ }
30
+
31
+ class AwsEndpointConfiguration(task: Task) {
32
+
33
+ def configureAwsClientBuilder[S <: AwsClientBuilder[S, T], T](builder: AwsClientBuilder[S, T]): Unit = {
34
+ if (task.getRegion.isPresent && task.getEndpoint.isPresent) {
35
+ val ec = new EndpointConfiguration(task.getEndpoint.get, task.getRegion.get)
36
+ builder.setEndpointConfiguration(ec)
37
+ }
38
+ else if (task.getRegion.isPresent && !task.getEndpoint.isPresent) {
39
+ builder.setRegion(task.getRegion.get)
40
+ }
41
+ else if (!task.getRegion.isPresent && task.getEndpoint.isPresent) {
42
+ val r: String = Try(new DefaultAwsRegionProviderChain().getRegion).getOrElse(Regions.DEFAULT_REGION.getName)
43
+ val e: String = task.getEndpoint.get
44
+ val ec = new EndpointConfiguration(e, r)
45
+ builder.setEndpointConfiguration(ec)
46
+ }
47
+ }
48
+
49
+ }
@@ -0,0 +1,56 @@
1
+ package org.embulk.output.s3_parquet.aws
2
+
3
+
4
+ import java.util.Optional
5
+
6
+ import com.amazonaws.services.s3.AmazonS3ClientBuilder
7
+ import org.embulk.config.{Config, ConfigDefault}
8
+ import org.embulk.output.s3_parquet.aws.AwsS3Configuration.Task
9
+
10
+ /*
11
+ * These are advanced settings, so write no documentation.
12
+ */
13
+ object AwsS3Configuration {
14
+ trait Task {
15
+
16
+ @Config("accelerate_mode_enabled")
17
+ @ConfigDefault("null")
18
+ def getAccelerateModeEnabled: Optional[Boolean]
19
+
20
+ @Config("chunked_encoding_disabled")
21
+ @ConfigDefault("null")
22
+ def getChunkedEncodingDisabled: Optional[Boolean]
23
+
24
+ @Config("dualstack_enabled")
25
+ @ConfigDefault("null")
26
+ def getDualstackEnabled: Optional[Boolean]
27
+
28
+ @Config("force_global_bucket_access_enabled")
29
+ @ConfigDefault("null")
30
+ def getForceGlobalBucketAccessEnabled: Optional[Boolean]
31
+
32
+ @Config("path_style_access_enabled")
33
+ @ConfigDefault("null")
34
+ def getPathStyleAccessEnabled: Optional[Boolean]
35
+
36
+ @Config("payload_signing_enabled")
37
+ @ConfigDefault("null")
38
+ def getPayloadSigningEnabled: Optional[Boolean]
39
+
40
+ }
41
+
42
+ def apply(task: Task): AwsS3Configuration = new AwsS3Configuration(task)
43
+ }
44
+
45
+ class AwsS3Configuration(task: Task) {
46
+
47
+ def configureAmazonS3ClientBuilder(builder: AmazonS3ClientBuilder): Unit = {
48
+ task.getAccelerateModeEnabled.ifPresent(v => builder.setAccelerateModeEnabled(v))
49
+ task.getChunkedEncodingDisabled.ifPresent(v => builder.setChunkedEncodingDisabled(v))
50
+ task.getDualstackEnabled.ifPresent(v => builder.setDualstackEnabled(v))
51
+ task.getForceGlobalBucketAccessEnabled.ifPresent(v => builder.setForceGlobalBucketAccessEnabled(v))
52
+ task.getPathStyleAccessEnabled.ifPresent(v => builder.setPathStyleAccessEnabled(v))
53
+ task.getPayloadSigningEnabled.ifPresent(v => builder.setPayloadSigningEnabled(v))
54
+ }
55
+
56
+ }
@@ -0,0 +1,56 @@
1
+ package org.embulk.output.s3_parquet.aws
2
+
3
+
4
+ import java.util.Optional
5
+
6
+ import com.amazonaws.{ClientConfiguration, Protocol}
7
+ import org.embulk.config.{Config, ConfigDefault, ConfigException}
8
+ import org.embulk.output.s3_parquet.aws.HttpProxy.Task
9
+
10
+ object HttpProxy {
11
+
12
+ trait Task {
13
+
14
+ @Config("host")
15
+ @ConfigDefault("null")
16
+ def getHost: Optional[String]
17
+
18
+ @Config("port")
19
+ @ConfigDefault("null")
20
+ def getPort: Optional[Int]
21
+
22
+ @Config("protocol")
23
+ @ConfigDefault("\"https\"")
24
+ def getProtocol: String
25
+
26
+ @Config("user")
27
+ @ConfigDefault("null")
28
+ def getUser: Optional[String]
29
+
30
+ @Config("password")
31
+ @ConfigDefault("null")
32
+ def getPassword: Optional[String]
33
+
34
+ }
35
+
36
+ def apply(task: Task): HttpProxy = new HttpProxy(task)
37
+
38
+ }
39
+
40
+ class HttpProxy(task: Task) {
41
+
42
+ def configureClientConfiguration(cc: ClientConfiguration): Unit = {
43
+ task.getHost.ifPresent(v => cc.setProxyHost(v))
44
+ task.getPort.ifPresent(v => cc.setProxyPort(v))
45
+
46
+ Protocol.values.find(p => p.name().equals(task.getProtocol)) match {
47
+ case Some(v) =>
48
+ cc.setProtocol(v)
49
+ case None =>
50
+ throw new ConfigException(s"'${task.getProtocol}' is unsupported: `protocol` must be one of [${Protocol.values.map(v => s"'$v'").mkString(", ")}].")
51
+ }
52
+
53
+ task.getUser.ifPresent(v => cc.setProxyUsername(v))
54
+ task.getPassword.ifPresent(v => cc.setProxyPassword(v))
55
+ }
56
+ }
@@ -0,0 +1,59 @@
1
+ package org.embulk.output.s3_parquet.parquet
2
+
3
+
4
+ import com.google.common.collect.ImmutableList
5
+ import org.apache.parquet.schema.{MessageType, OriginalType, PrimitiveType, Type}
6
+ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName
7
+ import org.embulk.spi.{Column, ColumnVisitor, Schema}
8
+
9
+ object EmbulkMessageType {
10
+
11
+ def builder(): Builder = Builder()
12
+
13
+ case class Builder(name: String = "embulk",
14
+ schema: Schema = Schema.builder().build()) {
15
+
16
+ def withName(name: String): Builder = Builder(name = name, schema = schema)
17
+
18
+ def withSchema(schema: Schema): Builder = Builder(name = name, schema = schema)
19
+
20
+ def build(): MessageType = {
21
+ val builder: ImmutableList.Builder[Type] = ImmutableList.builder[Type]()
22
+ schema.visitColumns(EmbulkMessageTypeColumnVisitor(builder))
23
+ new MessageType("embulk", builder.build())
24
+
25
+ }
26
+
27
+ }
28
+
29
+ private case class EmbulkMessageTypeColumnVisitor(builder: ImmutableList.Builder[Type])
30
+ extends ColumnVisitor {
31
+
32
+ override def booleanColumn(column: Column): Unit = {
33
+ builder.add(new PrimitiveType(Type.Repetition.OPTIONAL, PrimitiveTypeName.BOOLEAN, column.getName))
34
+ }
35
+
36
+ override def longColumn(column: Column): Unit = {
37
+ builder.add(new PrimitiveType(Type.Repetition.OPTIONAL, PrimitiveTypeName.INT64, column.getName))
38
+ }
39
+
40
+ override def doubleColumn(column: Column): Unit = {
41
+ builder.add(new PrimitiveType(Type.Repetition.OPTIONAL, PrimitiveTypeName.DOUBLE, column.getName))
42
+ }
43
+
44
+ override def stringColumn(column: Column): Unit = {
45
+ builder.add(new PrimitiveType(Type.Repetition.OPTIONAL, PrimitiveTypeName.BINARY, column.getName, OriginalType.UTF8))
46
+ }
47
+
48
+ override def timestampColumn(column: Column): Unit = {
49
+ // TODO: Support OriginalType.TIME* ?
50
+ builder.add(new PrimitiveType(Type.Repetition.OPTIONAL, PrimitiveTypeName.BINARY, column.getName, OriginalType.UTF8))
51
+ }
52
+
53
+ override def jsonColumn(column: Column): Unit = {
54
+ // TODO: does this work?
55
+ builder.add(new PrimitiveType(Type.Repetition.OPTIONAL, PrimitiveTypeName.BINARY, column.getName, OriginalType.UTF8))
56
+ }
57
+ }
58
+
59
+ }
@@ -0,0 +1,33 @@
1
+ package org.embulk.output.s3_parquet.parquet
2
+
3
+
4
+ import org.apache.hadoop.conf.Configuration
5
+ import org.apache.parquet.hadoop.api.WriteSupport
6
+ import org.apache.parquet.hadoop.api.WriteSupport.WriteContext
7
+ import org.apache.parquet.io.api.RecordConsumer
8
+ import org.apache.parquet.schema.MessageType
9
+ import org.embulk.spi.{PageReader, Schema}
10
+ import org.embulk.spi.time.TimestampFormatter
11
+
12
+ import scala.collection.JavaConverters._
13
+
14
+ private[parquet] case class ParquetFileWriteSupport(schema: Schema,
15
+ timestampFormatters: Seq[TimestampFormatter])
16
+ extends WriteSupport[PageReader] {
17
+
18
+ private var currentParquetFileWriter: ParquetFileWriter = _
19
+
20
+ override def init(configuration: Configuration): WriteContext = {
21
+ val messageType: MessageType = EmbulkMessageType.builder()
22
+ .withSchema(schema)
23
+ .build()
24
+ val metadata: Map[String, String] = Map.empty // NOTE: When is this used?
25
+ new WriteContext(messageType, metadata.asJava)
26
+ }
27
+
28
+ override def prepareForWrite(recordConsumer: RecordConsumer): Unit = {
29
+ currentParquetFileWriter = ParquetFileWriter(recordConsumer, schema, timestampFormatters)
30
+ }
31
+
32
+ override def write(record: PageReader): Unit = currentParquetFileWriter.write(record)
33
+ }
@@ -0,0 +1,125 @@
1
+ package org.embulk.output.s3_parquet.parquet
2
+
3
+
4
+ import org.apache.hadoop.conf.Configuration
5
+ import org.apache.hadoop.fs.Path
6
+ import org.apache.parquet.hadoop.ParquetWriter
7
+ import org.apache.parquet.hadoop.api.WriteSupport
8
+ import org.apache.parquet.io.api.{Binary, RecordConsumer}
9
+ import org.embulk.spi.{Column, ColumnVisitor, PageReader, Schema}
10
+ import org.embulk.spi.time.TimestampFormatter
11
+
12
+ object ParquetFileWriter {
13
+
14
+ case class Builder(path: Path = null,
15
+ schema: Schema = null,
16
+ timestampFormatters: Seq[TimestampFormatter] = null)
17
+ extends ParquetWriter.Builder[PageReader, Builder](path) {
18
+
19
+ def withPath(path: Path): Builder = copy(path = path)
20
+
21
+ def withPath(pathString: String): Builder = copy(path = new Path(pathString))
22
+
23
+ def withSchema(schema: Schema): Builder = copy(schema = schema)
24
+
25
+ def withTimestampFormatters(timestampFormatters: Seq[TimestampFormatter]): Builder = copy(timestampFormatters = timestampFormatters)
26
+
27
+ override def self(): Builder = this
28
+
29
+ override def getWriteSupport(conf: Configuration): WriteSupport[PageReader] = {
30
+ ParquetFileWriteSupport(schema, timestampFormatters)
31
+ }
32
+ }
33
+
34
+ def builder(): Builder = Builder()
35
+
36
+ }
37
+
38
+
39
+ private[parquet] case class ParquetFileWriter(recordConsumer: RecordConsumer,
40
+ schema: Schema,
41
+ timestampFormatters: Seq[TimestampFormatter]) {
42
+
43
+ def write(record: PageReader): Unit = {
44
+ recordConsumer.startMessage()
45
+ writeRecord(record)
46
+ recordConsumer.endMessage()
47
+ }
48
+
49
+ private def writeRecord(record: PageReader): Unit = {
50
+
51
+ schema.visitColumns(new ColumnVisitor() {
52
+
53
+ override def booleanColumn(column: Column): Unit = {
54
+ nullOr(column, {
55
+ withWriteFieldContext(column, {
56
+ recordConsumer.addBoolean(record.getBoolean(column))
57
+ })
58
+ })
59
+ }
60
+
61
+ override def longColumn(column: Column): Unit = {
62
+ nullOr(column, {
63
+ withWriteFieldContext(column, {
64
+ recordConsumer.addLong(record.getLong(column))
65
+ })
66
+ })
67
+ }
68
+
69
+ override def doubleColumn(column: Column): Unit = {
70
+ nullOr(column, {
71
+ withWriteFieldContext(column, {
72
+ recordConsumer.addDouble(record.getDouble(column))
73
+ })
74
+ })
75
+ }
76
+
77
+ override def stringColumn(column: Column): Unit = {
78
+ nullOr(column, {
79
+ withWriteFieldContext(column, {
80
+ val bin = Binary.fromString(record.getString(column))
81
+ recordConsumer.addBinary(bin)
82
+ })
83
+ })
84
+ }
85
+
86
+ override def timestampColumn(column: Column): Unit = {
87
+ nullOr(column, {
88
+ withWriteFieldContext(column, {
89
+ // TODO: is a correct way to convert for parquet ?
90
+ val t = record.getTimestamp(column)
91
+ val ft = timestampFormatters(column.getIndex).format(t)
92
+ val bin = Binary.fromString(ft)
93
+ recordConsumer.addBinary(bin)
94
+ })
95
+ })
96
+ }
97
+
98
+ override def jsonColumn(column: Column): Unit = {
99
+ nullOr(column, {
100
+ withWriteFieldContext(column, {
101
+ // TODO: is a correct way to convert for parquet ?
102
+ val msgPack = record.getJson(column)
103
+ val bin = Binary.fromString(msgPack.toJson)
104
+ recordConsumer.addBinary(bin)
105
+ })
106
+ })
107
+ }
108
+
109
+ private def nullOr(column: Column,
110
+ f: => Unit): Unit = {
111
+ if (!record.isNull(column)) f
112
+ }
113
+
114
+ private def withWriteFieldContext(column: Column,
115
+ f: => Unit): Unit = {
116
+ recordConsumer.startField(column.getName, column.getIndex)
117
+ f
118
+ recordConsumer.endField(column.getName, column.getIndex)
119
+ }
120
+
121
+ })
122
+
123
+ }
124
+
125
+ }
@@ -0,0 +1,6 @@
1
+ c0:boolean,c1:long,c2:double,c3:string,c4:timestamp,c5:json
2
+ true,0,0.0,c212c89f91,2017-10-22 19:53:31.000000 +0900,"{""a"":0,""b"":""99""}"
3
+ false,1,-0.5,aaaaa,2017-10-22 19:53:31.000000 +0900,"{""a"":1,""b"":""a9""}"
4
+ false,2,1.5,90823c6a1f,2017-10-23 23:42:43.000000 +0900,"{""a"":2,""b"":""96""}"
5
+ true,3,0.44,,2017-10-22 06:12:13.000000 +0900,"{""a"":3,""b"":""86""}"
6
+ false,9999,10000.33333,e56a40571c,2017-10-23 04:59:16.000000 +0900,"{""a"":4,""b"":""d2""}"
@@ -0,0 +1,5 @@
1
+ true 0 0.0 c212c89f91 2017-10-22 19:53:31.000000 +0900 {"a":0,"b":"99"}
2
+ false 1 -0.5 aaaaa 2017-10-22 19:53:31.000000 +0900 {"a":1,"b":"a9"}
3
+ false 2 1.5 90823c6a1f 2017-10-23 23:42:43.000000 +0900 {"a":2,"b":"96"}
4
+ true 3 0.44 2017-10-22 06:12:13.000000 +0900 {"a":3,"b":"86"}
5
+ false 9999 10000.33333 e56a40571c 2017-10-23 04:59:16.000000 +0900 {"a":4,"b":"d2"}
@@ -0,0 +1,140 @@
1
+ package org.embulk.output.s3_parquet
2
+
3
+
4
+ import java.io.{File, PrintWriter}
5
+ import java.nio.file.{FileSystems, Path}
6
+
7
+ import cloud.localstack.{DockerTestUtils, Localstack, TestUtils}
8
+ import cloud.localstack.docker.LocalstackDocker
9
+ import cloud.localstack.docker.annotation.LocalstackDockerConfiguration
10
+ import com.amazonaws.services.s3.transfer.TransferManagerBuilder
11
+ import com.google.common.io.Resources
12
+ import org.apache.hadoop.fs.{Path => HadoopPath}
13
+ import org.apache.parquet.hadoop.ParquetReader
14
+ import org.apache.parquet.tools.read.{SimpleReadSupport, SimpleRecord}
15
+ import org.embulk.config.ConfigSource
16
+ import org.embulk.spi.OutputPlugin
17
+ import org.embulk.test.{EmbulkTests, TestingEmbulk}
18
+ import org.junit.Rule
19
+ import org.junit.runner.RunWith
20
+ import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, DiagrammedAssertions, FunSuite}
21
+ import org.scalatest.junit.JUnitRunner
22
+
23
+ import scala.annotation.meta.getter
24
+ import scala.collection.JavaConverters._
25
+
26
+ @RunWith(classOf[JUnitRunner])
27
+ class TestS3ParquetOutputPlugin
28
+ extends FunSuite
29
+ with BeforeAndAfter
30
+ with BeforeAndAfterAll
31
+ with DiagrammedAssertions {
32
+
33
+ val RESOURCE_NAME_PREFIX: String = "org/embulk/output/s3_parquet/"
34
+ val BUCKET_NAME: String = "my-bucket"
35
+
36
+ val LOCALSTACK_DOCKER: LocalstackDocker = LocalstackDocker.INSTANCE
37
+
38
+ override protected def beforeAll(): Unit = {
39
+ Localstack.teardownInfrastructure()
40
+ LOCALSTACK_DOCKER.startup(LocalstackDockerConfiguration.DEFAULT)
41
+ super.beforeAll()
42
+ }
43
+
44
+ override protected def afterAll(): Unit = {
45
+ LOCALSTACK_DOCKER.stop()
46
+ super.afterAll()
47
+ }
48
+
49
+ @(Rule@getter)
50
+ val embulk: TestingEmbulk = TestingEmbulk.builder()
51
+ .registerPlugin(classOf[OutputPlugin], "s3_parquet", classOf[S3ParquetOutputPlugin])
52
+ .build()
53
+
54
+ before {
55
+ DockerTestUtils.getClientS3.createBucket(BUCKET_NAME)
56
+ }
57
+
58
+ def defaultOutConfig(): ConfigSource = {
59
+ embulk.newConfig()
60
+ .set("type", "s3_parquet")
61
+ .set("endpoint", "http://localhost:4572") // See https://github.com/localstack/localstack#overview
62
+ .set("bucket", BUCKET_NAME)
63
+ .set("path_prefix", "path/to/p")
64
+ .set("auth_method", "basic")
65
+ .set("access_key_id", TestUtils.TEST_ACCESS_KEY)
66
+ .set("secret_access_key", TestUtils.TEST_SECRET_KEY)
67
+ .set("path_style_access_enabled", true)
68
+ .set("default_timezone", "Asia/Tokyo")
69
+ }
70
+
71
+
72
+ test("first test") {
73
+ val inPath = toPath("in1.csv")
74
+ val outConfig = defaultOutConfig()
75
+
76
+ val result: TestingEmbulk.RunResult = embulk.runOutput(outConfig, inPath)
77
+
78
+
79
+ val outRecords: Seq[Map[String, String]] = result.getOutputTaskReports.asScala.map { tr =>
80
+ val b = tr.get(classOf[String], "bucket")
81
+ val k = tr.get(classOf[String], "key")
82
+ readParquetFile(b, k)
83
+ }.foldLeft(Seq[Map[String, String]]()) { (merged,
84
+ records) =>
85
+ merged ++ records
86
+ }
87
+
88
+ val inRecords: Seq[Seq[String]] = EmbulkTests.readResource(RESOURCE_NAME_PREFIX + "out1.tsv")
89
+ .stripLineEnd
90
+ .split("\n")
91
+ .map(record => record.split("\t").toSeq)
92
+
93
+ inRecords.zipWithIndex.foreach {
94
+ case (record, recordIndex) =>
95
+ 0.to(5).foreach { columnIndex =>
96
+ val columnName = s"c$columnIndex"
97
+ val inData: String = inRecords(recordIndex)(columnIndex)
98
+ val outData: String = outRecords(recordIndex).getOrElse(columnName, "")
99
+
100
+ assert(outData === inData, s"record: $recordIndex, column: $columnName")
101
+ }
102
+ }
103
+ }
104
+
105
+ def readParquetFile(bucket: String,
106
+ key: String): Seq[Map[String, String]] = {
107
+ val xfer = TransferManagerBuilder.standard()
108
+ .withS3Client(DockerTestUtils.getClientS3)
109
+ .build()
110
+ val createdParquetFile = embulk.createTempFile("in")
111
+ try xfer.download(bucket, key, createdParquetFile.toFile).waitForCompletion()
112
+ finally xfer.shutdownNow()
113
+
114
+ val reader: ParquetReader[SimpleRecord] = ParquetReader
115
+ .builder(new SimpleReadSupport(), new HadoopPath(createdParquetFile.toString))
116
+ .build()
117
+
118
+ def read(reader: ParquetReader[SimpleRecord],
119
+ records: Seq[Map[String, String]] = Seq()): Seq[Map[String, String]] = {
120
+ val simpleRecord: SimpleRecord = reader.read()
121
+ if (simpleRecord != null) {
122
+ val r: Map[String, String] = simpleRecord.getValues.asScala.map(v => v.getName -> v.getValue.toString).toMap
123
+ return read(reader, records :+ r)
124
+ }
125
+ records
126
+ }
127
+
128
+ try read(reader)
129
+ finally {
130
+ reader.close()
131
+
132
+ }
133
+ }
134
+
135
+ private def toPath(fileName: String) = {
136
+ val url = Resources.getResource(RESOURCE_NAME_PREFIX + fileName)
137
+ FileSystems.getDefault.getPath(new File(url.toURI).getAbsolutePath)
138
+ }
139
+
140
+ }