Skip to content

Commit addacc3

Browse files
shnapzlabianchin
andauthored
Propagate array item types from SQL to Avro schema (#931)
* Propagate array item types from SQL to Avro schema * Add tests * Fix checkstyle errors * Update e2e/e2e.sh Co-authored-by: Luís Bianchin <[email protected]> * Update e2e/e2e.sh Co-authored-by: Luís Bianchin <[email protected]> * fix e2e * Update docs --------- Co-authored-by: Luís Bianchin <[email protected]>
1 parent 9c72eac commit addacc3

File tree

7 files changed

+149
-62
lines changed

7 files changed

+149
-62
lines changed

dbeam-core/src/main/java/com/spotify/dbeam/avro/JdbcAvroRecord.java

+2-1
Original file line numberDiff line numberDiff line change
@@ -112,10 +112,11 @@ static SqlFunction<ResultSet, Object> computeMapping(
112112
} else {
113113
return resultSet -> nullableBytes(resultSet.getBytes(column));
114114
}
115+
case ARRAY:
116+
return resultSet -> resultSet.getArray(column);
115117
case BINARY:
116118
case VARBINARY:
117119
case LONGVARBINARY:
118-
case ARRAY:
119120
case BLOB:
120121
return resultSet -> nullableBytes(resultSet.getBytes(column));
121122
case DOUBLE:

dbeam-core/src/main/java/com/spotify/dbeam/avro/JdbcAvroRecordConverter.java

+30-15
Original file line numberDiff line numberDiff line change
@@ -97,24 +97,39 @@ public ByteBuffer convertResultSetIntoAvroBytes() throws SQLException, IOExcepti
9797
binaryEncoder.writeNull();
9898
} else {
9999
binaryEncoder.writeIndex(1);
100-
if (value instanceof String) {
101-
binaryEncoder.writeString((String) value);
102-
} else if (value instanceof Long) {
103-
binaryEncoder.writeLong((Long) value);
104-
} else if (value instanceof Integer) {
105-
binaryEncoder.writeInt((Integer) value);
106-
} else if (value instanceof Boolean) {
107-
binaryEncoder.writeBoolean((Boolean) value);
108-
} else if (value instanceof ByteBuffer) {
109-
binaryEncoder.writeBytes((ByteBuffer) value);
110-
} else if (value instanceof Double) {
111-
binaryEncoder.writeDouble((Double) value);
112-
} else if (value instanceof Float) {
113-
binaryEncoder.writeFloat((Float) value);
114-
}
100+
writeValue(value, binaryEncoder);
115101
}
116102
}
117103
binaryEncoder.flush();
118104
return ByteBuffer.wrap(out.getBufffer(), 0, out.size());
119105
}
106+
107+
private void writeValue(Object value, BinaryEncoder binaryEncoder)
108+
throws SQLException, IOException {
109+
if (value instanceof String) {
110+
binaryEncoder.writeString((String) value);
111+
} else if (value instanceof Long) {
112+
binaryEncoder.writeLong((Long) value);
113+
} else if (value instanceof Integer) {
114+
binaryEncoder.writeInt((Integer) value);
115+
} else if (value instanceof Boolean) {
116+
binaryEncoder.writeBoolean((Boolean) value);
117+
} else if (value instanceof ByteBuffer) {
118+
binaryEncoder.writeBytes((ByteBuffer) value);
119+
} else if (value instanceof Double) {
120+
binaryEncoder.writeDouble((Double) value);
121+
} else if (value instanceof Float) {
122+
binaryEncoder.writeFloat((Float) value);
123+
} else if (value instanceof java.sql.Array) {
124+
binaryEncoder.writeArrayStart();
125+
Object[] array = (Object[]) ((java.sql.Array) value).getArray();
126+
binaryEncoder.setItemCount(array.length);
127+
for (Object arrayItem : array) {
128+
binaryEncoder.startItem();
129+
writeValue(arrayItem, binaryEncoder);
130+
}
131+
132+
binaryEncoder.writeArrayEnd();
133+
}
134+
}
120135
}

dbeam-core/src/main/java/com/spotify/dbeam/avro/JdbcAvroSchema.java

+21-5
Original file line numberDiff line numberDiff line change
@@ -74,6 +74,8 @@ public static Schema createSchemaByReadingOneRow(
7474
try (Statement statement = connection.createStatement()) {
7575
final ResultSet resultSet = statement.executeQuery(queryBuilderArgs.sqlQueryWithLimitOne());
7676

77+
resultSet.next();
78+
7779
final Schema schema =
7880
createAvroSchema(
7981
resultSet,
@@ -107,7 +109,7 @@ public static Schema createAvroSchema(
107109
.prop("tableName", tableName)
108110
.prop("connectionUrl", connectionUrl)
109111
.fields();
110-
return createAvroFields(meta, builder, useLogicalTypes).endRecord();
112+
return createAvroFields(resultSet, builder, useLogicalTypes).endRecord();
111113
}
112114

113115
static String getDatabaseTableName(final ResultSetMetaData meta) throws SQLException {
@@ -123,11 +125,13 @@ static String getDatabaseTableName(final ResultSetMetaData meta) throws SQLExcep
123125
}
124126

125127
private static SchemaBuilder.FieldAssembler<Schema> createAvroFields(
126-
final ResultSetMetaData meta,
127-
final SchemaBuilder.FieldAssembler<Schema> builder,
128+
final ResultSet resultSet,
129+
final SchemaBuilder.FieldAssembler<Schema> builder,
128130
final boolean useLogicalTypes)
129131
throws SQLException {
130132

133+
ResultSetMetaData meta = resultSet.getMetaData();
134+
131135
for (int i = 1; i <= meta.getColumnCount(); i++) {
132136

133137
final String columnName;
@@ -140,7 +144,8 @@ private static SchemaBuilder.FieldAssembler<Schema> createAvroFields(
140144
final int columnType = meta.getColumnType(i);
141145
final String typeName = JDBCType.valueOf(columnType).getName();
142146
final String columnClassName = meta.getColumnClassName(i);
143-
final SchemaBuilder.FieldBuilder<Schema> field =
147+
final String columnTypeName = meta.getColumnTypeName(i);
148+
SchemaBuilder.FieldBuilder<Schema> field =
144149
builder
145150
.name(normalizeForAvro(columnName))
146151
.doc(String.format("From sqlType %d %s (%s)", columnType, typeName, columnClassName))
@@ -149,13 +154,21 @@ private static SchemaBuilder.FieldAssembler<Schema> createAvroFields(
149154
.prop("typeName", typeName)
150155
.prop("columnClassName", columnClassName);
151156

157+
if (columnTypeName != null) {
158+
field = field.prop("columnTypeName", columnTypeName);
159+
}
160+
152161
final SchemaBuilder.BaseTypeBuilder<
153162
SchemaBuilder.UnionAccumulator<SchemaBuilder.NullDefault<Schema>>>
154163
fieldSchemaBuilder = field.type().unionOf().nullBuilder().endNull().and();
155164

165+
Integer arrayItemType = resultSet.isFirst() && columnType == ARRAY
166+
? resultSet.getArray(i).getBaseType() : null;
167+
156168
final SchemaBuilder.UnionAccumulator<SchemaBuilder.NullDefault<Schema>> schemaFieldAssembler =
157169
setAvroColumnType(
158170
columnType,
171+
arrayItemType,
159172
meta.getPrecision(i),
160173
columnClassName,
161174
useLogicalTypes,
@@ -181,6 +194,7 @@ private static SchemaBuilder.FieldAssembler<Schema> createAvroFields(
181194
private static SchemaBuilder.UnionAccumulator<SchemaBuilder.NullDefault<Schema>>
182195
setAvroColumnType(
183196
final int columnType,
197+
final Integer arrayItemType,
184198
final int precision,
185199
final String columnClassName,
186200
final boolean useLogicalTypes,
@@ -225,10 +239,12 @@ private static SchemaBuilder.FieldAssembler<Schema> createAvroFields(
225239
} else {
226240
return field.bytesType();
227241
}
242+
case ARRAY:
243+
return setAvroColumnType(arrayItemType, null, precision, columnClassName,
244+
useLogicalTypes, field.array().items());
228245
case BINARY:
229246
case VARBINARY:
230247
case LONGVARBINARY:
231-
case ARRAY:
232248
case BLOB:
233249
return field.bytesType();
234250
case DOUBLE:

dbeam-core/src/test/java/com/spotify/dbeam/Coffee.java

+46-7
Original file line numberDiff line numberDiff line change
@@ -22,9 +22,12 @@
2222

2323
import com.google.auto.value.AutoValue;
2424
import java.math.BigDecimal;
25+
import java.util.ArrayList;
26+
import java.util.List;
2527
import java.util.Locale;
2628
import java.util.Optional;
2729
import java.util.UUID;
30+
import java.util.stream.Collectors;
2831

2932
// A fictitious DB model to test different SQL types
3033
@AutoValue
@@ -42,7 +45,9 @@ public static Coffee create(
4245
final java.sql.Timestamp created,
4346
final Optional<java.sql.Timestamp> updated,
4447
final UUID uid,
45-
final Long rownum) {
48+
final Long rownum,
49+
final List<Integer> intArr,
50+
final List<String> textArr) {
4651
return new AutoValue_Coffee(
4752
name,
4853
supId,
@@ -55,7 +60,9 @@ public static Coffee create(
5560
created,
5661
updated,
5762
uid,
58-
rownum);
63+
rownum,
64+
new ArrayList<>(intArr),
65+
new ArrayList<>(textArr));
5966
}
6067

6168
public abstract String name();
@@ -82,10 +89,15 @@ public static Coffee create(
8289

8390
public abstract Long rownum();
8491

92+
public abstract List<Integer> intArr();
93+
94+
public abstract List<String> textArr();
95+
8596
public String insertStatement() {
8697
return String.format(
8798
Locale.ENGLISH,
88-
"INSERT INTO COFFEES " + "VALUES ('%s', %s, '%s', %f, %f, %b, %d, %d, '%s', %s, '%s', %d)",
99+
"INSERT INTO COFFEES " + "VALUES ('%s', %s, '%s', %f, %f, %b, %d, %d, '%s', %s, '%s', %d,"
100+
+ " ARRAY [%s], ARRAY ['%s'])",
89101
name(),
90102
supId().orElse(null),
91103
price().toString(),
@@ -97,7 +109,9 @@ public String insertStatement() {
97109
created(),
98110
updated().orElse(null),
99111
uid(),
100-
rownum());
112+
rownum(),
113+
String.join(",", intArr().stream().map(x -> (CharSequence) x.toString())::iterator),
114+
String.join("','", textArr()));
101115
}
102116

103117
public static String ddl() {
@@ -114,7 +128,9 @@ public static String ddl() {
114128
+ "\"CREATED\" TIMESTAMP NOT NULL,"
115129
+ "\"UPDATED\" TIMESTAMP,"
116130
+ "\"UID\" UUID NOT NULL,"
117-
+ "\"ROWNUM\" BIGINT NOT NULL);";
131+
+ "\"ROWNUM\" BIGINT NOT NULL,"
132+
+ "\"INT_ARR\" INTEGER ARRAY NOT NULL,"
133+
+ "\"TEXT_ARR\" VARCHAR ARRAY NOT NULL);";
118134
}
119135

120136
public static Coffee COFFEE1 =
@@ -130,7 +146,19 @@ public static String ddl() {
130146
new java.sql.Timestamp(1488300933000L),
131147
Optional.empty(),
132148
UUID.fromString("123e4567-e89b-12d3-a456-426655440000"),
133-
1L);
149+
1L,
150+
new ArrayList<Integer>() {{
151+
add(5);
152+
add(7);
153+
add(11);
154+
}},
155+
new ArrayList<String>() {{
156+
add("rock");
157+
add("scissors");
158+
add("paper");
159+
}}
160+
);
161+
134162
public static Coffee COFFEE2 =
135163
create(
136164
"colombian caffee",
@@ -144,5 +172,16 @@ public static String ddl() {
144172
new java.sql.Timestamp(1488300723000L),
145173
Optional.empty(),
146174
UUID.fromString("123e4567-e89b-a456-12d3-426655440000"),
147-
2L);
175+
2L,
176+
new ArrayList<Integer>() {{
177+
add(7);
178+
add(11);
179+
add(23);
180+
}},
181+
new ArrayList<String>() {{
182+
add("scissors");
183+
add("paper");
184+
add("rock");
185+
}}
186+
);
148187
}

dbeam-core/src/test/java/com/spotify/dbeam/avro/JdbcAvroRecordTest.java

+21-6
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,9 @@
3434
import java.sql.ResultSetMetaData;
3535
import java.sql.SQLException;
3636
import java.sql.Timestamp;
37+
import java.util.ArrayList;
38+
import java.util.Arrays;
39+
import java.util.Collections;
3740
import java.util.List;
3841
import java.util.Optional;
3942
import java.util.stream.Collectors;
@@ -42,9 +45,11 @@
4245
import org.apache.avro.file.DataFileReader;
4346
import org.apache.avro.file.DataFileWriter;
4447
import org.apache.avro.file.SeekableByteArrayInput;
48+
import org.apache.avro.generic.GenericData;
4549
import org.apache.avro.generic.GenericDatumReader;
4650
import org.apache.avro.generic.GenericDatumWriter;
4751
import org.apache.avro.generic.GenericRecord;
52+
import org.apache.avro.util.Utf8;
4853
import org.junit.Assert;
4954
import org.junit.BeforeClass;
5055
import org.junit.Test;
@@ -62,7 +67,7 @@ public static void beforeAll() throws SQLException, ClassNotFoundException {
6267

6368
@Test
6469
public void shouldCreateSchema() throws ClassNotFoundException, SQLException {
65-
final int fieldCount = 12;
70+
final int fieldCount = 14;
6671
final Schema actual =
6772
JdbcAvroSchema.createSchemaByReadingOneRow(
6873
DbTestHelper.createConnection(CONNECTION_URL),
@@ -92,7 +97,9 @@ public void shouldCreateSchema() throws ClassNotFoundException, SQLException {
9297
"CREATED",
9398
"UPDATED",
9499
"UID",
95-
"ROWNUM"),
100+
"ROWNUM",
101+
"INT_ARR",
102+
"TEXT_ARR"),
96103
actual.getFields().stream().map(Schema.Field::name).collect(Collectors.toList()));
97104
for (Schema.Field f : actual.getFields()) {
98105
Assert.assertEquals(Schema.Type.UNION, f.schema().getType());
@@ -128,7 +135,7 @@ public void shouldCreateSchema() throws ClassNotFoundException, SQLException {
128135

129136
@Test
130137
public void shouldCreateSchemaWithLogicalTypes() throws ClassNotFoundException, SQLException {
131-
final int fieldCount = 12;
138+
final int fieldCount = 14;
132139
final Schema actual =
133140
JdbcAvroSchema.createSchemaByReadingOneRow(
134141
DbTestHelper.createConnection(CONNECTION_URL),
@@ -163,8 +170,10 @@ public void shouldEncodeResultSetToValidAvro()
163170
throws ClassNotFoundException, SQLException, IOException {
164171
final ResultSet rs =
165172
DbTestHelper.createConnection(CONNECTION_URL)
166-
.createStatement()
173+
.createStatement(ResultSet.TYPE_SCROLL_SENSITIVE, ResultSet.CONCUR_READ_ONLY)
167174
.executeQuery("SELECT * FROM COFFEES");
175+
176+
rs.first();
168177
final Schema schema =
169178
JdbcAvroSchema.createAvroSchema(
170179
rs, "dbeam_generated", "connection", Optional.empty(), "doc", false);
@@ -173,6 +182,7 @@ public void shouldEncodeResultSetToValidAvro()
173182
new DataFileWriter<>(new GenericDatumWriter<>(schema));
174183
final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
175184
dataFileWriter.create(schema, outputStream);
185+
rs.previous();
176186
// convert and write
177187
while (rs.next()) {
178188
dataFileWriter.appendEncoded(converter.convertResultSetIntoAvroBytes());
@@ -194,8 +204,11 @@ public void shouldEncodeResultSetToValidAvro()
194204
.findFirst()
195205
.orElseThrow(() -> new IllegalArgumentException("not found"));
196206

197-
Assert.assertEquals(12, record.getSchema().getFields().size());
207+
Assert.assertEquals(14, record.getSchema().getFields().size());
198208
Assert.assertEquals(schema, record.getSchema());
209+
List<String> actualTxtArray =
210+
((GenericData.Array<Utf8>) record.get(13)).stream().map(x -> x.toString()).collect(
211+
Collectors.toList());
199212
final Coffee actual =
200213
Coffee.create(
201214
record.get(0).toString(),
@@ -209,7 +222,9 @@ public void shouldEncodeResultSetToValidAvro()
209222
new java.sql.Timestamp((Long) record.get(8)),
210223
Optional.ofNullable((Long) record.get(9)).map(Timestamp::new),
211224
TestHelper.byteBufferToUuid((ByteBuffer) record.get(10)),
212-
(Long) record.get(11));
225+
(Long) record.get(11),
226+
new ArrayList<>((GenericData.Array<Integer>) record.get(12)),
227+
actualTxtArray);
213228
Assert.assertEquals(Coffee.COFFEE1, actual);
214229
}
215230

0 commit comments

Comments
 (0)