Skip to content

Commit 6702d39

Browse files
treff7esautumnust
authored andcommitted
[GOBBLIN-1312][GOBBLIN-1318] Bumping parquet lib to 1.11.1 to remove hadoop-lzo dependency
Bumping parquet lib to 1.11.1 to remove hadoop-lzo dependency which caused build error as twitter's maven repo is unreliable. Removing twitter parquet completly and using apache parquet everywhere bumping gobblin-parquet module to use parquet 1.11.1 Disabling parquetOutputFormatTest test until https://issues.apache.org/jira/browse/GOBBLIN-1318 is fixed Changing UTF8 to STRING JsonIntermediateToParquetConverter test to support the latest parquet Closes apache#3150 from treff7es/remove-lzo-dependency
1 parent 7352cad commit 6702d39

File tree

15 files changed

+121
-105
lines changed

15 files changed

+121
-105
lines changed

defaultEnvironment.gradle

-3
Original file line numberDiff line numberDiff line change
@@ -28,9 +28,6 @@ subprojects {
2828
maven {
2929
url "http://conjars.org/repo"
3030
}
31-
maven {
32-
url "https://maven.twttr.com/"
33-
}
3431
}
3532

3633
project.buildDir = new File(project.rootProject.buildDir, project.name)

gobblin-example/src/main/java/org/apache/gobblin/example/wikipedia/EmbeddedWikipediaExample.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,8 @@
2222

2323
import org.apache.commons.cli.CommandLine;
2424

25+
import com.google.common.base.Joiner;
26+
2527
import org.apache.gobblin.annotation.Alias;
2628
import org.apache.gobblin.configuration.ConfigurationKeys;
2729
import org.apache.gobblin.publisher.BaseDataPublisher;
@@ -36,8 +38,6 @@
3638
import org.apache.gobblin.writer.Destination;
3739
import org.apache.gobblin.writer.WriterOutputFormat;
3840

39-
import avro.shaded.com.google.common.base.Joiner;
40-
4141

4242
/**
4343
* Embedded Gobblin to run Wikipedia example.

gobblin-modules/gobblin-parquet-apache/src/test/resources/converter/JsonIntermediateToParquetConverter.json

+7-7
Original file line numberDiff line numberDiff line change
@@ -54,7 +54,7 @@
5454
}
5555
],
5656
"expectedRecord": "a: 5 ; b: 5.0 ; c: 8.0 ; d: true ; e: somestring ; f: 2018-01-01 ; g: 1545083047 ;",
57-
"expectedSchema": "message test_table{ ; required int32 a ; ; required float b ; ; required double c ; ; required boolean d ; ; required binary e (UTF8) ; ; required binary f (UTF8) ; ; required binary g (UTF8) ; ; } ; "
57+
"expectedSchema": "message test_table{ ; required int32 a ; ; required float b ; ; required double c ; ; required boolean d ; ; required binary e (STRING) ; ; required binary f (STRING) ; ; required binary g (STRING) ; ; } ; "
5858
},
5959
"array": {
6060
"record": {
@@ -134,7 +134,7 @@
134134
}
135135
],
136136
"expectedRecord": "somearray ; item:1 ; item:2 ; item:3 ; somearray1 ; item:1 ; item:2 ; item:3 ; somearray2 ; item:1.0 ; item:2.0 ; item:3.0 ; somearray3 ; item:1.0 ; item:2.0 ; item:3.0 ; somearray4 ; item:true ; item:false ; item:true ; somearray5 ; item:hello ; item:world ; ",
137-
"expectedSchema": "message test_table { ; optional group somearray { ; repeated int32 item ; ; } ; required groupsomearray1 { ; repeated int64 item ; ; } ; required groupsomearray2 { ; repeated float item ; ; } ; required groupsomearray3 { ; repeated double item ; ; } ; required groupsomearray4 { ; repeated boolean item ; ; } ; required groupsomearray5 { ; repeated binary item(UTF8) ; ; } ; } ; "
137+
"expectedSchema": "message test_table { ; optional group somearray { ; repeated int32 item ; ; } ; required groupsomearray1 { ; repeated int64 item ; ; } ; required groupsomearray2 { ; repeated float item ; ; } ; required groupsomearray3 { ; repeated double item ; ; } ; required groupsomearray4 { ; repeated boolean item ; ; } ; required groupsomearray5 { ; repeated binary item(STRING) ; ; } ; } ; "
138138
},
139139
"enum": {
140140
"record": {
@@ -154,7 +154,7 @@
154154
}
155155
],
156156
"expectedRecord": "some_enum : HELLO ;",
157-
"expectedSchema": "message test_table { ; optional binary some_enum (UTF8) ;; } ;"
157+
"expectedSchema": "message test_table { ; optional binary some_enum (STRING) ;; } ;"
158158
},
159159
"enum1": {
160160
"record": {
@@ -174,7 +174,7 @@
174174
}
175175
],
176176
"expectedRecord": "some_enum : HELLO ;",
177-
"expectedSchema": "message test_table { ; required binary some_enum (UTF8) ;; } ;"
177+
"expectedSchema": "message test_table { ; required binary some_enum (STRING) ;; } ;"
178178
},
179179
"record": {
180180
"record": {
@@ -218,7 +218,7 @@
218218
}
219219
],
220220
"expectedRecord": "some_record ; name:me ; age:22 ; some_array ; item:3 ; item:4 ; item:5 ;",
221-
"expectedSchema": "message test_table { ; required group some_record { ; required binary name (UTF8) ; ; required int64 age ; ; required group some_array { ; repeated int32 item ; ; } ; } ; } ; "
221+
"expectedSchema": "message test_table { ; required group some_record { ; required binary name (STRING) ; ; required int64 age ; ; required group some_array { ; repeated int32 item ; ; } ; } ; } ; "
222222
},
223223
"map": {
224224
"schema": [
@@ -238,7 +238,7 @@
238238
}
239239
},
240240
"expectedRecord": "cityToCountry; map; key:ny;value:US; map; key:london;value:UK; map; key:delhi;value:India;",
241-
"expectedSchema": "message test_table { ; required groupcityToCountry { ; repeated group map { ; required binary key (UTF8) ; ; required binary value (UTF8) ; ; } ; } ; } ;"
241+
"expectedSchema": "message test_table { ; required groupcityToCountry { ; repeated group map { ; required binary key (STRING) ; ; required binary value (STRING) ; ; } ; } ; } ;"
242242
},
243243
"nullValueInOptionalField": {
244244
"record": {
@@ -256,4 +256,4 @@
256256
"expectedRecord": "",
257257
"expectedSchema": "message test_table {; optional int32 a ;; };"
258258
}
259-
}
259+
}

gobblin-modules/gobblin-parquet/build.gradle

+3-3
Original file line numberDiff line numberDiff line change
@@ -22,9 +22,9 @@ dependencies {
2222
compile project(":gobblin-modules:gobblin-parquet-common")
2323

2424
compile externalDependency.gson
25-
compile externalDependency.twitterParquet
26-
compile externalDependency.twitterParquetAvro
27-
compile externalDependency.twitterParquetProto
25+
compile externalDependency.parquetHadoop
26+
compile externalDependency.parquetAvro
27+
compile externalDependency.parquetProto
2828

2929
testCompile externalDependency.testng
3030
testCompile externalDependency.mockito

gobblin-modules/gobblin-parquet/src/main/java/org/apache/gobblin/converter/parquet/JsonElementConversionFactory.java

+26-26
Original file line numberDiff line numberDiff line change
@@ -22,37 +22,37 @@
2222
import java.util.List;
2323
import java.util.Map;
2424

25+
import org.apache.parquet.example.data.Group;
26+
import org.apache.parquet.example.data.simple.BinaryValue;
27+
import org.apache.parquet.example.data.simple.BooleanValue;
28+
import org.apache.parquet.example.data.simple.DoubleValue;
29+
import org.apache.parquet.example.data.simple.FloatValue;
30+
import org.apache.parquet.example.data.simple.IntegerValue;
31+
import org.apache.parquet.example.data.simple.LongValue;
32+
import org.apache.parquet.io.api.Binary;
33+
import org.apache.parquet.schema.GroupType;
34+
import org.apache.parquet.schema.LogicalTypeAnnotation;
35+
import org.apache.parquet.schema.MessageType;
36+
import org.apache.parquet.schema.PrimitiveType;
37+
import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
38+
import org.apache.parquet.schema.Type;
39+
import org.apache.parquet.schema.Types;
40+
2541
import com.google.gson.JsonArray;
2642
import com.google.gson.JsonElement;
2743
import com.google.gson.JsonObject;
2844

29-
import parquet.example.data.Group;
30-
import parquet.example.data.simple.BinaryValue;
31-
import parquet.example.data.simple.BooleanValue;
32-
import parquet.example.data.simple.DoubleValue;
33-
import parquet.example.data.simple.FloatValue;
34-
import parquet.example.data.simple.IntegerValue;
35-
import parquet.example.data.simple.LongValue;
36-
import parquet.io.api.Binary;
37-
import parquet.schema.GroupType;
38-
import parquet.schema.MessageType;
39-
import parquet.schema.PrimitiveType;
40-
import parquet.schema.PrimitiveType.PrimitiveTypeName;
41-
import parquet.schema.Type;
42-
import parquet.schema.Types;
43-
4445
import org.apache.gobblin.converter.parquet.JsonSchema.*;
4546

47+
import static org.apache.gobblin.converter.parquet.JsonElementConversionFactory.RecordConverter.RecordType.CHILD;
4648
import static org.apache.gobblin.converter.parquet.JsonSchema.*;
4749
import static org.apache.gobblin.converter.parquet.JsonSchema.InputType.STRING;
48-
import static org.apache.gobblin.converter.parquet.JsonElementConversionFactory.RecordConverter.RecordType.CHILD;
49-
import static parquet.schema.OriginalType.UTF8;
50-
import static parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
51-
import static parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
52-
import static parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
53-
import static parquet.schema.Type.Repetition.OPTIONAL;
54-
import static parquet.schema.Type.Repetition.REPEATED;
55-
import static parquet.schema.Type.Repetition.REQUIRED;
50+
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
51+
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
52+
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
53+
import static org.apache.parquet.schema.Type.Repetition.OPTIONAL;
54+
import static org.apache.parquet.schema.Type.Repetition.REPEATED;
55+
import static org.apache.parquet.schema.Type.Repetition.REQUIRED;
5656

5757

5858
/**
@@ -293,13 +293,13 @@ BinaryValue convertField(JsonElement value) {
293293
protected Type buildSchema() {
294294
String columnName = this.jsonSchema.getColumnName();
295295
if (this.repeated) {
296-
return Types.repeated(BINARY).as(UTF8).named(columnName);
296+
return Types.repeated(BINARY).as(LogicalTypeAnnotation.StringLogicalTypeAnnotation.stringType()).named(columnName);
297297
}
298298
switch (optionalOrRequired(this.jsonSchema)) {
299299
case OPTIONAL:
300-
return Types.optional(BINARY).as(UTF8).named(columnName);
300+
return Types.optional(BINARY).as(LogicalTypeAnnotation.StringLogicalTypeAnnotation.stringType()).named(columnName);
301301
case REQUIRED:
302-
return Types.required(BINARY).as(UTF8).named(columnName);
302+
return Types.required(BINARY).as(LogicalTypeAnnotation.StringLogicalTypeAnnotation.stringType()).named(columnName);
303303
default:
304304
throw new RuntimeException("Unsupported Repetition type");
305305
}

gobblin-modules/gobblin-parquet/src/main/java/org/apache/gobblin/converter/parquet/JsonIntermediateToParquetGroupConverter.java

+3-3
Original file line numberDiff line numberDiff line change
@@ -16,12 +16,12 @@
1616
*/
1717
package org.apache.gobblin.converter.parquet;
1818

19+
import org.apache.parquet.example.data.Group;
20+
import org.apache.parquet.schema.MessageType;
21+
1922
import com.google.gson.JsonArray;
2023
import com.google.gson.JsonObject;
2124

22-
import parquet.example.data.Group;
23-
import parquet.schema.MessageType;
24-
2525
import org.apache.gobblin.configuration.WorkUnitState;
2626
import org.apache.gobblin.converter.Converter;
2727
import org.apache.gobblin.converter.DataConversionException;

gobblin-modules/gobblin-parquet/src/main/java/org/apache/gobblin/converter/parquet/ParquetGroup.java

+38-17
Original file line numberDiff line numberDiff line change
@@ -19,23 +19,23 @@
1919
import java.util.ArrayList;
2020
import java.util.List;
2121

22-
import parquet.example.data.Group;
23-
import parquet.example.data.simple.BinaryValue;
24-
import parquet.example.data.simple.BooleanValue;
25-
import parquet.example.data.simple.DoubleValue;
26-
import parquet.example.data.simple.FloatValue;
27-
import parquet.example.data.simple.Int96Value;
28-
import parquet.example.data.simple.IntegerValue;
29-
import parquet.example.data.simple.LongValue;
30-
import parquet.example.data.simple.NanoTime;
31-
import parquet.example.data.simple.Primitive;
32-
import parquet.io.api.Binary;
33-
import parquet.io.api.RecordConsumer;
34-
import parquet.schema.GroupType;
35-
import parquet.schema.PrimitiveType;
36-
import parquet.schema.Type;
37-
38-
import static parquet.schema.Type.Repetition.REPEATED;
22+
import org.apache.parquet.example.data.Group;
23+
import org.apache.parquet.example.data.simple.BinaryValue;
24+
import org.apache.parquet.example.data.simple.BooleanValue;
25+
import org.apache.parquet.example.data.simple.DoubleValue;
26+
import org.apache.parquet.example.data.simple.FloatValue;
27+
import org.apache.parquet.example.data.simple.Int96Value;
28+
import org.apache.parquet.example.data.simple.IntegerValue;
29+
import org.apache.parquet.example.data.simple.LongValue;
30+
import org.apache.parquet.example.data.simple.NanoTime;
31+
import org.apache.parquet.example.data.simple.Primitive;
32+
import org.apache.parquet.io.api.Binary;
33+
import org.apache.parquet.io.api.RecordConsumer;
34+
import org.apache.parquet.schema.GroupType;
35+
import org.apache.parquet.schema.PrimitiveType;
36+
import org.apache.parquet.schema.Type;
37+
38+
import static org.apache.parquet.schema.Type.Repetition.REPEATED;
3939

4040

4141
/**
@@ -140,6 +140,22 @@ public int getInteger(int fieldIndex, int index) {
140140
return ((IntegerValue) this.getValue(fieldIndex, index)).getInteger();
141141
}
142142

143+
@Override
144+
public long getLong(int fieldIndex, int index) {
145+
return ((LongValue) this.getValue(fieldIndex, index)).getLong();
146+
147+
}
148+
149+
@Override
150+
public double getDouble(int fieldIndex, int index) {
151+
return ((DoubleValue) this.getValue(fieldIndex, index)).getDouble();
152+
}
153+
154+
@Override
155+
public float getFloat(int fieldIndex, int index) {
156+
return ((FloatValue) this.getValue(fieldIndex, index)).getFloat();
157+
}
158+
143159
public boolean getBoolean(int fieldIndex, int index) {
144160
return ((BooleanValue) this.getValue(fieldIndex, index)).getBoolean();
145161
}
@@ -194,6 +210,11 @@ public void add(int fieldIndex, double value) {
194210
this.add(fieldIndex, new DoubleValue(value));
195211
}
196212

213+
@Override
214+
public void add(int i, Group group) {
215+
this.data[i].add(group);
216+
}
217+
197218
public GroupType getType() {
198219
return this.schema;
199220
}

gobblin-modules/gobblin-parquet/src/main/java/org/apache/gobblin/writer/ParquetDataWriterBuilder.java

+10-10
Original file line numberDiff line numberDiff line change
@@ -20,19 +20,19 @@
2020

2121
import org.apache.avro.Schema;
2222
import org.apache.hadoop.conf.Configuration;
23+
import org.apache.parquet.avro.AvroParquetWriter;
24+
import org.apache.parquet.column.ParquetProperties;
25+
import org.apache.parquet.example.data.Group;
26+
import org.apache.parquet.hadoop.ParquetWriter;
27+
import org.apache.parquet.hadoop.api.WriteSupport;
28+
import org.apache.parquet.hadoop.example.GroupWriteSupport;
29+
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
30+
import org.apache.parquet.proto.ProtoParquetWriter;
31+
import org.apache.parquet.schema.MessageType;
2332

2433
import com.google.protobuf.Message;
2534

2635
import lombok.extern.slf4j.Slf4j;
27-
import parquet.avro.AvroParquetWriter;
28-
import parquet.column.ParquetProperties;
29-
import parquet.example.data.Group;
30-
import parquet.hadoop.ParquetWriter;
31-
import parquet.hadoop.api.WriteSupport;
32-
import parquet.hadoop.example.GroupWriteSupport;
33-
import parquet.hadoop.metadata.CompressionCodecName;
34-
import parquet.proto.ProtoParquetWriter;
35-
import parquet.schema.MessageType;
3636

3737
import org.apache.gobblin.parquet.writer.AbstractParquetDataWriterBuilder;
3838
import org.apache.gobblin.parquet.writer.ParquetWriterConfiguration;
@@ -114,4 +114,4 @@ public void close()
114114
}
115115
};
116116
}
117-
}
117+
}

gobblin-modules/gobblin-parquet/src/test/java/org/apache/gobblin/converter/parquet/JsonIntermediateToParquetGroupConverterTest.java

+2-3
Original file line numberDiff line numberDiff line change
@@ -19,16 +19,15 @@
1919
import java.io.InputStreamReader;
2020
import java.lang.reflect.Type;
2121

22+
import org.apache.parquet.example.data.Group;
23+
import org.apache.parquet.schema.MessageType;
2224
import org.testng.annotations.BeforeClass;
2325
import org.testng.annotations.Test;
2426

2527
import com.google.gson.Gson;
2628
import com.google.gson.JsonObject;
2729
import com.google.gson.reflect.TypeToken;
2830

29-
import parquet.example.data.Group;
30-
import parquet.schema.MessageType;
31-
3231
import org.apache.gobblin.configuration.SourceState;
3332
import org.apache.gobblin.configuration.WorkUnitState;
3433
import org.apache.gobblin.converter.DataConversionException;

gobblin-modules/gobblin-parquet/src/test/java/org/apache/gobblin/writer/ParquetHdfsDataWriterTest.java

+9-9
Original file line numberDiff line numberDiff line change
@@ -29,15 +29,15 @@
2929
import org.testng.annotations.BeforeMethod;
3030
import org.testng.annotations.Test;
3131

32-
import parquet.avro.AvroParquetReader;
33-
import parquet.example.data.Group;
34-
import parquet.example.data.simple.convert.GroupRecordConverter;
35-
import parquet.hadoop.ParquetReader;
36-
import parquet.hadoop.api.InitContext;
37-
import parquet.hadoop.api.ReadSupport;
38-
import parquet.io.api.RecordMaterializer;
39-
import parquet.proto.ProtoParquetReader;
40-
import parquet.schema.MessageType;
32+
import org.apache.parquet.avro.AvroParquetReader;
33+
import org.apache.parquet.example.data.Group;
34+
import org.apache.parquet.example.data.simple.convert.GroupRecordConverter;
35+
import org.apache.parquet.hadoop.ParquetReader;
36+
import org.apache.parquet.hadoop.api.InitContext;
37+
import org.apache.parquet.hadoop.api.ReadSupport;
38+
import org.apache.parquet.io.api.RecordMaterializer;
39+
import org.apache.parquet.proto.ProtoParquetReader;
40+
import org.apache.parquet.schema.MessageType;
4141

4242
import org.apache.gobblin.parquet.writer.ParquetRecordFormat;
4343
import org.apache.gobblin.parquet.writer.test.ParquetHdfsDataWriterTestBase;

gobblin-modules/gobblin-parquet/src/test/java/org/apache/gobblin/writer/TestConstants.java

+7-7
Original file line numberDiff line numberDiff line change
@@ -16,12 +16,12 @@
1616
*/
1717
package org.apache.gobblin.writer;
1818

19-
import parquet.example.data.Group;
20-
import parquet.example.data.simple.SimpleGroup;
21-
import parquet.schema.MessageType;
22-
import parquet.schema.OriginalType;
23-
import parquet.schema.PrimitiveType;
24-
import parquet.schema.Types;
19+
import org.apache.parquet.example.data.Group;
20+
import org.apache. parquet.example.data.simple.SimpleGroup;
21+
import org.apache.parquet.schema.MessageType;
22+
import org.apache.parquet.schema.OriginalType;
23+
import org.apache.parquet.schema.PrimitiveType;
24+
import org.apache.parquet.schema.Types;
2525

2626
import org.apache.gobblin.parquet.writer.test.TestConstantsBase;
2727
import org.apache.gobblin.test.TestRecord;
@@ -47,4 +47,4 @@ public Group convertToParquetGroup(TestRecord record) {
4747
return group;
4848
}
4949

50-
}
50+
}

0 commit comments

Comments
 (0)