Skip to content

Commit 783acdf

Browse files
committed
spotless
1 parent 3a89bdc commit 783acdf

File tree

4 files changed

+26
-27
lines changed

4 files changed

+26
-27
lines changed

parquet-avro/src/main/java/org/apache/parquet/avro/AvroWriteSupport.java

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -46,8 +46,8 @@
4646
import org.apache.parquet.schema.LogicalTypeAnnotation.UUIDLogicalTypeAnnotation;
4747
import org.apache.parquet.schema.MessageType;
4848
import org.apache.parquet.schema.Type;
49-
import org.apache.parquet.variant.VariantValueWriter;
5049
import org.apache.parquet.variant.Variant;
50+
import org.apache.parquet.variant.VariantValueWriter;
5151
import org.slf4j.Logger;
5252
import org.slf4j.LoggerFactory;
5353

@@ -235,9 +235,7 @@ private void writeVariantFields(GroupType schema, Schema avroSchema, Object reco
235235
recordConsumer.startGroup();
236236
writeRecordFields(schema, avroSchema, record);
237237
recordConsumer.endGroup();
238-
return;
239238
}
240-
241239
}
242240

243241
private void writeRecordFields(GroupType schema, Schema avroSchema, Object record) {

parquet-avro/src/test/java/org/apache/parquet/avro/TestWriteVariant.java

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -123,7 +123,8 @@ public TestWriteVariant() throws Exception {
123123
ob.appendKey("e");
124124
ob.appendNull();
125125
b.endObject();
126-
}).getMetadataBuffer();
126+
})
127+
.getMetadataBuffer();
127128

128129
TEST_OBJECT = variant(TEST_METADATA, b -> {
129130
VariantObjectBuilder ob = b.startObject();
@@ -268,8 +269,8 @@ public void testShreddedValues() throws IOException {
268269

269270
GenericRecord actual = writeAndRead(testSchema, record);
270271
assertEquals(record.get(0), actual.get(0));
271-
Variant actualV = new Variant((ByteBuffer) ((GenericRecord) actual.get(1)).get(1),
272-
(ByteBuffer) ((GenericRecord) actual.get(1)).get(0));
272+
Variant actualV = new Variant((ByteBuffer) ((GenericRecord) actual.get(1)).get(1), (ByteBuffer)
273+
((GenericRecord) actual.get(1)).get(0));
273274
AvroTestUtil.assertEquivalent(v, actualV);
274275
}
275276
}
@@ -288,8 +289,9 @@ public void testMixedShredding() throws IOException {
288289
List<GenericRecord> actual = writeAndRead(testSchema, expected);
289290
assertEquals(actual.size(), VARIANTS.length);
290291
for (int i = 0; i < VARIANTS.length; i++) {
291-
Variant actualV = new Variant((ByteBuffer) ((GenericRecord) actual.get(i).get(1)).get(1),
292-
(ByteBuffer) ((GenericRecord) actual.get(i).get(1)).get(0));
292+
Variant actualV =
293+
new Variant((ByteBuffer) ((GenericRecord) actual.get(i).get(1)).get(1), (ByteBuffer)
294+
((GenericRecord) actual.get(i).get(1)).get(0));
293295
AvroTestUtil.assertEquivalent(VARIANTS[i], actualV);
294296
}
295297
}

parquet-variant/src/main/java/org/apache/parquet/variant/VariantUtil.java

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -866,17 +866,17 @@ public static int valueSize(ByteBuffer value) {
866866
VariantUtil.ObjectInfo info = VariantUtil.getObjectInfo(slice(value, pos));
867867
return info.dataStartOffset
868868
+ readUnsigned(
869-
value,
870-
pos + info.offsetStartOffset + info.numElements * info.offsetSize,
871-
info.offsetSize);
869+
value,
870+
pos + info.offsetStartOffset + info.numElements * info.offsetSize,
871+
info.offsetSize);
872872
}
873873
case ARRAY: {
874874
VariantUtil.ArrayInfo info = VariantUtil.getArrayInfo(slice(value, pos));
875875
return info.dataStartOffset
876876
+ readUnsigned(
877-
value,
878-
pos + info.offsetStartOffset + info.numElements * info.offsetSize,
879-
info.offsetSize);
877+
value,
878+
pos + info.offsetStartOffset + info.numElements * info.offsetSize,
879+
info.offsetSize);
880880
}
881881
default:
882882
switch (typeInfo) {

parquet-variant/src/main/java/org/apache/parquet/variant/VariantValueWriter.java

Lines changed: 12 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -19,8 +19,6 @@
1919
package org.apache.parquet.variant;
2020

2121
import java.nio.ByteBuffer;
22-
import java.util.HashMap;
23-
import org.apache.parquet.format.VariantType;
2422
import org.apache.parquet.io.api.Binary;
2523
import org.apache.parquet.io.api.RecordConsumer;
2624
import org.apache.parquet.schema.GroupType;
@@ -127,9 +125,11 @@ private boolean isTypeCompatible(Variant.Type variantType, Type typedValueField)
127125
&& ((LogicalTypeAnnotation.IntLogicalTypeAnnotation) logicalType).getBitWidth() == 16;
128126
case INT:
129127
return primitiveTypeName == PrimitiveType.PrimitiveTypeName.INT32
130-
&& (logicalType == null ||
131-
(logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation &&
132-
((LogicalTypeAnnotation.IntLogicalTypeAnnotation) logicalType).getBitWidth() == 32));
128+
&& (logicalType == null
129+
|| (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation
130+
&& ((LogicalTypeAnnotation.IntLogicalTypeAnnotation) logicalType)
131+
.getBitWidth()
132+
== 32));
133133
case LONG:
134134
return primitiveTypeName == PrimitiveType.PrimitiveTypeName.INT64
135135
&& (logicalType == null
@@ -157,18 +157,17 @@ private boolean isTypeCompatible(Variant.Type variantType, Type typedValueField)
157157
case TIMESTAMP_NANOS_NTZ:
158158
case TIMESTAMP_TZ:
159159
case TIMESTAMP_NANOS_TZ:
160-
if (primitiveTypeName == PrimitiveType.PrimitiveTypeName.INT64 &&
161-
logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) {
160+
if (primitiveTypeName == PrimitiveType.PrimitiveTypeName.INT64
161+
&& logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) {
162162
LogicalTypeAnnotation.TimestampLogicalTypeAnnotation annotation =
163163
(LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) logicalType;
164164
boolean micros = annotation.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS;
165165
boolean nanos = annotation.getUnit() == LogicalTypeAnnotation.TimeUnit.NANOS;
166-
boolean adjustedToUTC = annotation.isAdjustedToUTC();
167-
return
168-
(variantType == Variant.Type.TIMESTAMP_TZ && micros && adjustedToUTC) ||
169-
(variantType == Variant.Type.TIMESTAMP_NTZ && micros && !adjustedToUTC) ||
170-
(variantType == Variant.Type.TIMESTAMP_NANOS_TZ && nanos && adjustedToUTC) ||
171-
(variantType == Variant.Type.TIMESTAMP_NANOS_NTZ && nanos && !adjustedToUTC);
166+
boolean adjustedToUTC = annotation.isAdjustedToUTC();
167+
return (variantType == Variant.Type.TIMESTAMP_TZ && micros && adjustedToUTC)
168+
|| (variantType == Variant.Type.TIMESTAMP_NTZ && micros && !adjustedToUTC)
169+
|| (variantType == Variant.Type.TIMESTAMP_NANOS_TZ && nanos && adjustedToUTC)
170+
|| (variantType == Variant.Type.TIMESTAMP_NANOS_NTZ && nanos && !adjustedToUTC);
172171
} else {
173172
return false;
174173
}

0 commit comments

Comments
 (0)