embulk-output-s3_parquet 0.0.3 → 0.1.0

Sign up to get free protection for your applications and to get access to all the features.
@@ -3,7 +3,7 @@ package org.embulk.output.s3_parquet.aws
3
3
 
4
4
  import java.util.Optional
5
5
 
6
- import com.amazonaws.auth.{AnonymousAWSCredentials, AWSCredentialsProvider, AWSStaticCredentialsProvider, BasicAWSCredentials, BasicSessionCredentials, DefaultAWSCredentialsProviderChain, EC2ContainerCredentialsProviderWrapper, EnvironmentVariableCredentialsProvider, STSAssumeRoleSessionCredentialsProvider, SystemPropertiesCredentialsProvider}
6
+ import com.amazonaws.auth.{AnonymousAWSCredentials, AWSCredentialsProvider, AWSStaticCredentialsProvider, BasicAWSCredentials, BasicSessionCredentials, DefaultAWSCredentialsProviderChain, EC2ContainerCredentialsProviderWrapper, EnvironmentVariableCredentialsProvider, STSAssumeRoleSessionCredentialsProvider, SystemPropertiesCredentialsProvider, WebIdentityTokenCredentialsProvider}
7
7
  import com.amazonaws.auth.profile.{ProfileCredentialsProvider, ProfilesConfigFile}
8
8
  import org.embulk.config.{Config, ConfigDefault, ConfigException}
9
9
  import org.embulk.output.s3_parquet.aws.AwsCredentials.Task
@@ -60,6 +60,9 @@ object AwsCredentials
60
60
  @ConfigDefault("null")
61
61
  def getScopeDownPolicy: Optional[String]
62
62
 
63
+ @Config("web_identity_token_file")
64
+ @ConfigDefault("null")
65
+ def getWebIdentityTokenFile: Optional[String]
63
66
  }
64
67
 
65
68
  def apply(task: Task): AwsCredentials =
@@ -119,6 +122,13 @@ class AwsCredentials(task: Task)
119
122
 
120
123
  builder.build()
121
124
 
125
+ case "web_identity_token" =>
126
+ WebIdentityTokenCredentialsProvider.builder()
127
+ .roleArn(getRequiredOption(task.getRoleArn, "role_arn"))
128
+ .roleSessionName(getRequiredOption(task.getRoleSessionName, "role_session_name"))
129
+ .webIdentityTokenFile(getRequiredOption(task.getWebIdentityTokenFile, "web_identity_token_file"))
130
+ .build()
131
+
122
132
  case "default" =>
123
133
  new DefaultAWSCredentialsProviderChain
124
134
 
@@ -16,30 +16,36 @@ object EmbulkMessageType
16
16
  }
17
17
 
18
18
  case class Builder(name: String = "embulk",
19
- schema: Schema = Schema.builder().build())
19
+ schema: Schema = Schema.builder().build(),
20
+ logicalTypeHandlers: LogicalTypeHandlerStore = LogicalTypeHandlerStore.empty)
20
21
  {
21
22
 
22
23
  def withName(name: String): Builder =
23
24
  {
24
- Builder(name = name, schema = schema)
25
+ Builder(name = name, schema = schema, logicalTypeHandlers = logicalTypeHandlers)
25
26
  }
26
27
 
27
28
  def withSchema(schema: Schema): Builder =
28
29
  {
29
- Builder(name = name, schema = schema)
30
+ Builder(name = name, schema = schema, logicalTypeHandlers = logicalTypeHandlers)
31
+ }
32
+
33
+ def withLogicalTypeHandlers(logicalTypeHandlers: LogicalTypeHandlerStore): Builder =
34
+ {
35
+ Builder(name = name, schema = schema, logicalTypeHandlers = logicalTypeHandlers)
30
36
  }
31
37
 
32
38
  def build(): MessageType =
33
39
  {
34
40
  val builder: ImmutableList.Builder[Type] = ImmutableList.builder[Type]()
35
- schema.visitColumns(EmbulkMessageTypeColumnVisitor(builder))
41
+ schema.visitColumns(EmbulkMessageTypeColumnVisitor(builder, logicalTypeHandlers))
36
42
  new MessageType("embulk", builder.build())
37
-
38
43
  }
39
44
 
40
45
  }
41
46
 
42
- private case class EmbulkMessageTypeColumnVisitor(builder: ImmutableList.Builder[Type])
47
+ private case class EmbulkMessageTypeColumnVisitor(builder: ImmutableList.Builder[Type],
48
+ logicalTypeHandlers: LogicalTypeHandlerStore = LogicalTypeHandlerStore.empty)
43
49
  extends ColumnVisitor
44
50
  {
45
51
 
@@ -50,7 +56,15 @@ object EmbulkMessageType
50
56
 
51
57
  override def longColumn(column: Column): Unit =
52
58
  {
53
- builder.add(new PrimitiveType(Type.Repetition.OPTIONAL, PrimitiveTypeName.INT64, column.getName))
59
+ val name = column.getName
60
+ val et = column.getType
61
+
62
+ val t = logicalTypeHandlers.get(name, et) match {
63
+ case Some(h) if h.isConvertible(et) => h.newSchemaFieldType(name)
64
+ case _ => new PrimitiveType(Type.Repetition.OPTIONAL, PrimitiveTypeName.INT64, column.getName)
65
+ }
66
+
67
+ builder.add(t)
54
68
  }
55
69
 
56
70
  override def doubleColumn(column: Column): Unit =
@@ -65,14 +79,28 @@ object EmbulkMessageType
65
79
 
66
80
  override def timestampColumn(column: Column): Unit =
67
81
  {
68
- // TODO: Support OriginalType.TIME* ?
69
- builder.add(new PrimitiveType(Type.Repetition.OPTIONAL, PrimitiveTypeName.BINARY, column.getName, OriginalType.UTF8))
82
+ val name = column.getName
83
+ val et = column.getType
84
+
85
+ val t = logicalTypeHandlers.get(name, et) match {
86
+ case Some(h) if h.isConvertible(et) => h.newSchemaFieldType(name)
87
+ case _ => new PrimitiveType(Type.Repetition.OPTIONAL, PrimitiveTypeName.BINARY, name, OriginalType.UTF8)
88
+ }
89
+
90
+ builder.add(t)
70
91
  }
71
92
 
72
93
  override def jsonColumn(column: Column): Unit =
73
94
  {
74
- // TODO: does this work?
75
- builder.add(new PrimitiveType(Type.Repetition.OPTIONAL, PrimitiveTypeName.BINARY, column.getName, OriginalType.UTF8))
95
+ val name = column.getName
96
+ val et = column.getType
97
+
98
+ val t = logicalTypeHandlers.get(name, et) match {
99
+ case Some(h) if h.isConvertible(et) => h.newSchemaFieldType(name)
100
+ case _ => new PrimitiveType(Type.Repetition.OPTIONAL, PrimitiveTypeName.BINARY, name, OriginalType.UTF8)
101
+ }
102
+
103
+ builder.add(t)
76
104
  }
77
105
  }
78
106
 
@@ -0,0 +1,145 @@
1
+ package org.embulk.output.s3_parquet.parquet
2
+
3
+
4
+ import org.apache.parquet.io.api.{Binary, RecordConsumer}
5
+ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName
6
+ import org.apache.parquet.schema.{Type => PType}
7
+ import org.apache.parquet.schema.{OriginalType, PrimitiveType}
8
+ import org.embulk.spi.DataException
9
+ import org.embulk.spi.`type`.{Type => EType}
10
+ import org.embulk.spi.`type`.Types
11
+ import org.embulk.spi.time.Timestamp
12
+ import org.msgpack.value.Value
13
+
14
+
15
+ /**
16
+ * Handle Apache Parquet 'Logical Types' on schema/value conversion.
17
+ * ref. https://github.com/apache/parquet-format/blob/master/LogicalTypes.md
18
+ *
19
+ * It focuses on only older representation because newer supported since 1.11 is not used actually yet.
20
+ * TODO Support both of older and newer representation after 1.11+ is published and other middleware supports it.
21
+ *
22
+ */
23
+ sealed trait LogicalTypeHandler
24
+ {
25
+ def isConvertible(t: EType): Boolean
26
+
27
+ def newSchemaFieldType(name: String): PrimitiveType
28
+
29
+ def consume(orig: Any,
30
+ recordConsumer: RecordConsumer): Unit
31
+ }
32
+
33
+ abstract class IntLogicalTypeHandler(ot: OriginalType)
34
+ extends LogicalTypeHandler
35
+ {
36
+ override def isConvertible(t: EType): Boolean =
37
+ {
38
+ t == Types.LONG
39
+ }
40
+
41
+ override def newSchemaFieldType(name: String): PrimitiveType =
42
+ {
43
+ new PrimitiveType(PType.Repetition.OPTIONAL, PrimitiveTypeName.INT64, name, ot)
44
+ }
45
+
46
+ override def consume(orig: Any,
47
+ recordConsumer: RecordConsumer): Unit =
48
+ {
49
+ orig match {
50
+ case v: Long => recordConsumer.addLong(v)
51
+ case _ => throw new DataException("given mismatched type value; expected type is long")
52
+ }
53
+ }
54
+ }
55
+
56
+ object TimestampMillisLogicalTypeHandler
57
+ extends LogicalTypeHandler
58
+ {
59
+ override def isConvertible(t: EType): Boolean =
60
+ {
61
+ t == Types.TIMESTAMP
62
+ }
63
+
64
+ override def newSchemaFieldType(name: String): PrimitiveType =
65
+ {
66
+ new PrimitiveType(PType.Repetition.OPTIONAL, PrimitiveTypeName.INT64, name, OriginalType.TIMESTAMP_MILLIS)
67
+ }
68
+
69
+ override def consume(orig: Any,
70
+ recordConsumer: RecordConsumer): Unit =
71
+ {
72
+ orig match {
73
+ case ts: Timestamp => recordConsumer.addLong(ts.toEpochMilli)
74
+ case _ => throw new DataException("given mismatched type value; expected type is timestamp")
75
+ }
76
+ }
77
+ }
78
+
79
+ object TimestampMicrosLogicalTypeHandler
80
+ extends LogicalTypeHandler
81
+ {
82
+ override def isConvertible(t: EType): Boolean =
83
+ {
84
+ t == Types.TIMESTAMP
85
+ }
86
+
87
+ override def newSchemaFieldType(name: String): PrimitiveType =
88
+ {
89
+ new PrimitiveType(PType.Repetition.OPTIONAL, PrimitiveTypeName.INT64, name, OriginalType.TIMESTAMP_MICROS)
90
+ }
91
+
92
+ override def consume(orig: Any,
93
+ recordConsumer: RecordConsumer): Unit =
94
+ {
95
+ orig match {
96
+ case ts: Timestamp =>
97
+ val v = (ts.getEpochSecond * 1_000_000L) + (ts.getNano.asInstanceOf[Long] / 1_000L)
98
+ recordConsumer.addLong(v)
99
+ case _ => throw new DataException("given mismatched type value; expected type is timestamp")
100
+ }
101
+ }
102
+ }
103
+
104
+ object Int8LogicalTypeHandler
105
+ extends IntLogicalTypeHandler(OriginalType.INT_8)
106
+ object Int16LogicalTypeHandler
107
+ extends IntLogicalTypeHandler(OriginalType.INT_16)
108
+ object Int32LogicalTypeHandler
109
+ extends IntLogicalTypeHandler(OriginalType.INT_32)
110
+ object Int64LogicalTypeHandler
111
+ extends IntLogicalTypeHandler(OriginalType.INT_64)
112
+
113
+ object Uint8LogicalTypeHandler
114
+ extends IntLogicalTypeHandler(OriginalType.UINT_8)
115
+ object Uint16LogicalTypeHandler
116
+ extends IntLogicalTypeHandler(OriginalType.UINT_16)
117
+ object Uint32LogicalTypeHandler
118
+ extends IntLogicalTypeHandler(OriginalType.UINT_32)
119
+ object Uint64LogicalTypeHandler
120
+ extends IntLogicalTypeHandler(OriginalType.UINT_64)
121
+
122
+ object JsonLogicalTypeHandler
123
+ extends LogicalTypeHandler
124
+ {
125
+ override def isConvertible(t: EType): Boolean =
126
+ {
127
+ t == Types.JSON
128
+ }
129
+
130
+ override def newSchemaFieldType(name: String): PrimitiveType =
131
+ {
132
+ new PrimitiveType(PType.Repetition.OPTIONAL, PrimitiveTypeName.BINARY, name, OriginalType.JSON)
133
+ }
134
+
135
+ override def consume(orig: Any,
136
+ recordConsumer: RecordConsumer): Unit =
137
+ {
138
+ orig match {
139
+ case msgPack: Value =>
140
+ val bin = Binary.fromString(msgPack.toJson)
141
+ recordConsumer.addBinary(bin)
142
+ case _ => throw new DataException("given mismatched type value; expected type is json")
143
+ }
144
+ }
145
+ }
@@ -0,0 +1,107 @@
1
+ package org.embulk.output.s3_parquet.parquet
2
+
3
+
4
+ import org.embulk.spi.`type`.{Type, Types}
5
+ import java.util.{Map => JMap}
6
+
7
+ import org.embulk.config.ConfigException
8
+ import org.embulk.output.s3_parquet.S3ParquetOutputPlugin.{ColumnOptionTask, TypeOptionTask}
9
+
10
+ import scala.jdk.CollectionConverters._
11
+
12
+
13
+ /**
14
+ * A storage has mapping from logical type query (column name, type) to handler.
15
+ *
16
+ * @param fromEmbulkType
17
+ * @param fromColumnName
18
+ */
19
+ case class LogicalTypeHandlerStore private(fromEmbulkType: Map[Type, LogicalTypeHandler],
20
+ fromColumnName: Map[String, LogicalTypeHandler])
21
+ {
22
+
23
+ // Try column name lookup, then column type
24
+ def get(n: String,
25
+ t: Type): Option[LogicalTypeHandler] =
26
+ {
27
+ get(n) match {
28
+ case Some(h) => Some(h)
29
+ case _ =>
30
+ get(t) match {
31
+ case Some(h) => Some(h)
32
+ case _ => None
33
+ }
34
+ }
35
+ }
36
+
37
+ def get(t: Type): Option[LogicalTypeHandler] =
38
+ {
39
+ fromEmbulkType.get(t)
40
+ }
41
+
42
+ def get(n: String): Option[LogicalTypeHandler] =
43
+ {
44
+ fromColumnName.get(n)
45
+ }
46
+ }
47
+
48
+ object LogicalTypeHandlerStore
49
+ {
50
+ private val STRING_TO_EMBULK_TYPE = Map[String, Type](
51
+ "boolean" -> Types.BOOLEAN,
52
+ "long" -> Types.LONG,
53
+ "double" -> Types.DOUBLE,
54
+ "string" -> Types.STRING,
55
+ "timestamp" -> Types.TIMESTAMP,
56
+ "json" -> Types.JSON
57
+ )
58
+
59
+ // Listed only older logical types that we can convert from embulk type
60
+ private val STRING_TO_LOGICAL_TYPE = Map[String, LogicalTypeHandler](
61
+ "timestamp-millis" -> TimestampMillisLogicalTypeHandler,
62
+ "timestamp-micros" -> TimestampMicrosLogicalTypeHandler,
63
+ "int8" -> Int8LogicalTypeHandler,
64
+ "int16" -> Int16LogicalTypeHandler,
65
+ "int32" -> Int32LogicalTypeHandler,
66
+ "int64" -> Int64LogicalTypeHandler,
67
+ "uint8" -> Uint8LogicalTypeHandler,
68
+ "uint16" -> Uint16LogicalTypeHandler,
69
+ "uint32" -> Uint32LogicalTypeHandler,
70
+ "uint64" -> Uint64LogicalTypeHandler,
71
+ "json" -> JsonLogicalTypeHandler
72
+ )
73
+
74
+ def empty: LogicalTypeHandlerStore =
75
+ {
76
+ LogicalTypeHandlerStore(Map.empty[Type, LogicalTypeHandler], Map.empty[String, LogicalTypeHandler])
77
+ }
78
+
79
+ def fromEmbulkOptions(typeOpts: JMap[String, TypeOptionTask],
80
+ columnOpts: JMap[String, ColumnOptionTask]): LogicalTypeHandlerStore =
81
+ {
82
+ val fromEmbulkType = typeOpts.asScala
83
+ .filter(_._2.getLogicalType.isPresent)
84
+ .map[Type, LogicalTypeHandler] { case (k, v) =>
85
+ val t = STRING_TO_EMBULK_TYPE.get(k)
86
+ val h = STRING_TO_LOGICAL_TYPE.get(v.getLogicalType.get)
87
+ (t, h) match {
88
+ case (Some(tt), Some(hh)) => (tt, hh)
89
+ case _ => throw new ConfigException("invalid logical types in type_options")
90
+ }
91
+ }
92
+ .toMap
93
+
94
+ val fromColumnName = columnOpts.asScala
95
+ .filter(_._2.getLogicalType.isPresent)
96
+ .map[String, LogicalTypeHandler] { case (k, v) =>
97
+ val h = STRING_TO_LOGICAL_TYPE.get(v.getLogicalType.get)
98
+ h match {
99
+ case Some(hh) => (k, hh)
100
+ case _ => throw new ConfigException("invalid logical types in column_options")
101
+ }
102
+ }
103
+ .toMap
104
+
105
+ LogicalTypeHandlerStore(fromEmbulkType, fromColumnName)
106
+ }
107
+ }
@@ -13,7 +13,8 @@ import scala.jdk.CollectionConverters._
13
13
 
14
14
 
15
15
  private[parquet] case class ParquetFileWriteSupport(schema: Schema,
16
- timestampFormatters: Seq[TimestampFormatter])
16
+ timestampFormatters: Seq[TimestampFormatter],
17
+ logicalTypeHandlers: LogicalTypeHandlerStore = LogicalTypeHandlerStore.empty)
17
18
  extends WriteSupport[PageReader]
18
19
  {
19
20
 
@@ -23,6 +24,7 @@ private[parquet] case class ParquetFileWriteSupport(schema: Schema,
23
24
  {
24
25
  val messageType: MessageType = EmbulkMessageType.builder()
25
26
  .withSchema(schema)
27
+ .withLogicalTypeHandlers(logicalTypeHandlers)
26
28
  .build()
27
29
  val metadata: Map[String, String] = Map.empty // NOTE: When is this used?
28
30
  new WriteContext(messageType, metadata.asJava)
@@ -30,7 +32,7 @@ private[parquet] case class ParquetFileWriteSupport(schema: Schema,
30
32
 
31
33
  override def prepareForWrite(recordConsumer: RecordConsumer): Unit =
32
34
  {
33
- currentParquetFileWriter = ParquetFileWriter(recordConsumer, schema, timestampFormatters)
35
+ currentParquetFileWriter = ParquetFileWriter(recordConsumer, schema, timestampFormatters, logicalTypeHandlers)
34
36
  }
35
37
 
36
38
  override def write(record: PageReader): Unit =
@@ -15,52 +15,59 @@ object ParquetFileWriter
15
15
 
16
16
  case class Builder(path: Path = null,
17
17
  schema: Schema = null,
18
- timestampFormatters: Seq[TimestampFormatter] = null)
18
+ timestampFormatters: Seq[TimestampFormatter] = null,
19
+ logicalTypeHandlers: LogicalTypeHandlerStore = LogicalTypeHandlerStore.empty)
19
20
  extends ParquetWriter.Builder[PageReader, Builder](path)
20
21
  {
21
22
 
22
- def withPath(path: Path): Builder =
23
- {
24
- copy(path = path)
25
- }
23
+ def withPath(path: Path): Builder =
24
+ {
25
+ copy(path = path)
26
+ }
26
27
 
27
- def withPath(pathString: String): Builder =
28
- {
29
- copy(path = new Path(pathString))
30
- }
28
+ def withPath(pathString: String): Builder =
29
+ {
30
+ copy(path = new Path(pathString))
31
+ }
31
32
 
32
- def withSchema(schema: Schema): Builder =
33
- {
34
- copy(schema = schema)
35
- }
33
+ def withSchema(schema: Schema): Builder =
34
+ {
35
+ copy(schema = schema)
36
+ }
36
37
 
37
- def withTimestampFormatters(timestampFormatters: Seq[TimestampFormatter]): Builder =
38
- {
39
- copy(timestampFormatters = timestampFormatters)
40
- }
38
+ def withTimestampFormatters(timestampFormatters: Seq[TimestampFormatter]): Builder =
39
+ {
40
+ copy(timestampFormatters = timestampFormatters)
41
+ }
41
42
 
42
- override def self(): Builder =
43
- {
44
- this
45
- }
43
+ def withLogicalTypeHandlers(logicalTypeHandlers: LogicalTypeHandlerStore): Builder =
44
+ {
45
+ copy(logicalTypeHandlers = logicalTypeHandlers)
46
+ }
47
+
48
+ override def self(): Builder =
49
+ {
50
+ this
51
+ }
46
52
 
47
53
  override def getWriteSupport(conf: Configuration): WriteSupport[PageReader] =
48
54
  {
49
- ParquetFileWriteSupport(schema, timestampFormatters)
55
+ ParquetFileWriteSupport(schema, timestampFormatters, logicalTypeHandlers)
50
56
  }
51
57
  }
52
58
 
53
- def builder(): Builder =
54
- {
55
- Builder()
56
- }
59
+ def builder(): Builder =
60
+ {
61
+ Builder()
62
+ }
57
63
 
58
64
  }
59
65
 
60
66
 
61
67
  private[parquet] case class ParquetFileWriter(recordConsumer: RecordConsumer,
62
68
  schema: Schema,
63
- timestampFormatters: Seq[TimestampFormatter])
69
+ timestampFormatters: Seq[TimestampFormatter],
70
+ logicalTypeHandlers: LogicalTypeHandlerStore = LogicalTypeHandlerStore.empty)
64
71
  {
65
72
 
66
73
  def write(record: PageReader): Unit =
@@ -117,11 +124,16 @@ private[parquet] case class ParquetFileWriter(recordConsumer: RecordConsumer,
117
124
  {
118
125
  nullOr(column, {
119
126
  withWriteFieldContext(column, {
120
- // TODO: is a correct way to convert for parquet ?
121
127
  val t = record.getTimestamp(column)
122
- val ft = timestampFormatters(column.getIndex).format(t)
123
- val bin = Binary.fromString(ft)
124
- recordConsumer.addBinary(bin)
128
+
129
+ logicalTypeHandlers.get(column.getName, column.getType) match {
130
+ case Some(h) =>
131
+ h.consume(t, recordConsumer)
132
+ case _ =>
133
+ val ft = timestampFormatters(column.getIndex).format(t)
134
+ val bin = Binary.fromString(ft)
135
+ recordConsumer.addBinary(bin)
136
+ }
125
137
  })
126
138
  })
127
139
  }
@@ -130,10 +142,15 @@ private[parquet] case class ParquetFileWriter(recordConsumer: RecordConsumer,
130
142
  {
131
143
  nullOr(column, {
132
144
  withWriteFieldContext(column, {
133
- // TODO: is a correct way to convert for parquet ?
134
145
  val msgPack = record.getJson(column)
135
- val bin = Binary.fromString(msgPack.toJson)
136
- recordConsumer.addBinary(bin)
146
+
147
+ logicalTypeHandlers.get(column.getName, column.getType) match {
148
+ case Some(h) =>
149
+ h.consume(msgPack, recordConsumer)
150
+ case _ =>
151
+ val bin = Binary.fromString(msgPack.toJson)
152
+ recordConsumer.addBinary(bin)
153
+ }
137
154
  })
138
155
  })
139
156
  }