Skip to content

Commit 577f8aa

Browse files
Apply spotless
1 parent bccc14a commit 577f8aa

File tree

10 files changed

+137
-96
lines changed

10 files changed

+137
-96
lines changed

adapter/avro/src/main/java/org/apache/arrow/adapter/avro/ArrowToAvroUtils.java

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -335,8 +335,10 @@ private static <T> T buildBaseTypeSchema(
335335
// Arrow uses "$data$" as the field name for list items, that is not a valid Avro name
336336
Field itemField = field.getChildren().get(0);
337337
if (ListVector.DATA_VECTOR_NAME.equals(itemField.getName())) {
338-
Field safeItemField = new Field("item", itemField.getFieldType(), itemField.getChildren());
339-
Field safeListField = new Field(field.getName(), field.getFieldType(), List.of(safeItemField));
338+
Field safeItemField =
339+
new Field("item", itemField.getFieldType(), itemField.getChildren());
340+
Field safeListField =
341+
new Field(field.getName(), field.getFieldType(), List.of(safeItemField));
340342
return buildArraySchema(builder.array(), safeListField, namespace);
341343
} else {
342344
return buildArraySchema(builder.array(), field, namespace);

adapter/avro/src/main/java/org/apache/arrow/adapter/avro/AvroToArrow.java

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -63,14 +63,15 @@ public static AvroToArrowVectorIterator avroToArrowIterator(
6363
/**
6464
* Convert an Avro schema to its Arrow equivalent.
6565
*
66-
* <p>The resulting set of Arrow fields matches what would be set in the VSR after calling avroToArrow()
67-
* or avroToArrowIterator(), respecting the configuration in the config parameter.
66+
* <p>The resulting set of Arrow fields matches what would be set in the VSR after calling
67+
* avroToArrow() or avroToArrowIterator(), respecting the configuration in the config parameter.
6868
*
6969
* @param schema The Avro schema to convert
7070
* @param config Configuration options for conversion
7171
* @return The equivalent Arrow schema
7272
*/
73-
public static org.apache.arrow.vector.types.pojo.Schema avroToAvroSchema(Schema schema, AvroToArrowConfig config) {
73+
public static org.apache.arrow.vector.types.pojo.Schema avroToAvroSchema(
74+
Schema schema, AvroToArrowConfig config) {
7475

7576
Preconditions.checkNotNull(schema, "Avro schema object cannot be null");
7677
Preconditions.checkNotNull(config, "config cannot be null");

adapter/avro/src/main/java/org/apache/arrow/adapter/avro/AvroToArrowUtils.java

Lines changed: 81 additions & 40 deletions
Original file line numberDiff line numberDiff line change
@@ -40,8 +40,8 @@
4040
import org.apache.arrow.adapter.avro.consumers.AvroIntConsumer;
4141
import org.apache.arrow.adapter.avro.consumers.AvroLongConsumer;
4242
import org.apache.arrow.adapter.avro.consumers.AvroMapConsumer;
43-
import org.apache.arrow.adapter.avro.consumers.AvroNullableConsumer;
4443
import org.apache.arrow.adapter.avro.consumers.AvroNullConsumer;
44+
import org.apache.arrow.adapter.avro.consumers.AvroNullableConsumer;
4545
import org.apache.arrow.adapter.avro.consumers.AvroStringConsumer;
4646
import org.apache.arrow.adapter.avro.consumers.AvroStructConsumer;
4747
import org.apache.arrow.adapter.avro.consumers.AvroUnionsConsumer;
@@ -50,8 +50,8 @@
5050
import org.apache.arrow.adapter.avro.consumers.SkipConsumer;
5151
import org.apache.arrow.adapter.avro.consumers.SkipFunction;
5252
import org.apache.arrow.adapter.avro.consumers.logical.AvroDateConsumer;
53-
import org.apache.arrow.adapter.avro.consumers.logical.AvroDecimalConsumer;
5453
import org.apache.arrow.adapter.avro.consumers.logical.AvroDecimal256Consumer;
54+
import org.apache.arrow.adapter.avro.consumers.logical.AvroDecimalConsumer;
5555
import org.apache.arrow.adapter.avro.consumers.logical.AvroTimeMicroConsumer;
5656
import org.apache.arrow.adapter.avro.consumers.logical.AvroTimeMillisConsumer;
5757
import org.apache.arrow.adapter.avro.consumers.logical.AvroTimestampMicrosConsumer;
@@ -66,8 +66,8 @@
6666
import org.apache.arrow.vector.BigIntVector;
6767
import org.apache.arrow.vector.BitVector;
6868
import org.apache.arrow.vector.DateDayVector;
69-
import org.apache.arrow.vector.DecimalVector;
7069
import org.apache.arrow.vector.Decimal256Vector;
70+
import org.apache.arrow.vector.DecimalVector;
7171
import org.apache.arrow.vector.FieldVector;
7272
import org.apache.arrow.vector.FixedSizeBinaryVector;
7373
import org.apache.arrow.vector.Float4Vector;
@@ -180,17 +180,19 @@ private static Consumer createConsumer(
180180

181181
switch (type) {
182182
case UNION:
183-
boolean nullableUnion = schema.getTypes().stream().anyMatch(t -> t.getType() == Schema.Type.NULL);
183+
boolean nullableUnion =
184+
schema.getTypes().stream().anyMatch(t -> t.getType() == Schema.Type.NULL);
184185
if (schema.getTypes().size() == 2 && nullableUnion && config.isHandleNullable()) {
185186
// For a simple nullable (null | type), interpret the union as a single nullable field.
186-
// Requires setting handleNullable in the config, otherwise fall back on the literal interpretation.
187+
// Requires setting handleNullable in the config, otherwise fall back on the literal
188+
// interpretation.
187189
int nullIndex = schema.getTypes().get(0).getType() == Schema.Type.NULL ? 0 : 1;
188190
int childIndex = nullIndex == 0 ? 1 : 0;
189191
Schema childSchema = schema.getTypes().get(childIndex);
190-
Consumer<?> childConsumer = createConsumer(childSchema, name, true, config, consumerVector);
192+
Consumer<?> childConsumer =
193+
createConsumer(childSchema, name, true, config, consumerVector);
191194
consumer = new AvroNullableConsumer<>(childConsumer, nullIndex);
192-
}
193-
else {
195+
} else {
194196
// Literal interpretation of a union, which may or may not include a null element.
195197
consumer = createUnionConsumer(schema, name, nullableUnion, config, consumerVector);
196198
}
@@ -222,9 +224,13 @@ private static Consumer createConsumer(
222224
nullable, arrowType, /* dictionary= */ null, getMetaData(schema, extProps));
223225
vector = createVector(consumerVector, fieldType, name, allocator);
224226
if (schema.getFixedSize() <= 16) {
225-
consumer = new AvroDecimalConsumer.FixedDecimalConsumer((DecimalVector) vector, schema.getFixedSize());
227+
consumer =
228+
new AvroDecimalConsumer.FixedDecimalConsumer(
229+
(DecimalVector) vector, schema.getFixedSize());
226230
} else {
227-
consumer = new AvroDecimal256Consumer.FixedDecimal256Consumer((Decimal256Vector) vector, schema.getFixedSize());
231+
consumer =
232+
new AvroDecimal256Consumer.FixedDecimal256Consumer(
233+
(Decimal256Vector) vector, schema.getFixedSize());
228234
}
229235
} else {
230236
arrowType = new ArrowType.FixedSizeBinary(schema.getFixedSize());
@@ -335,7 +341,8 @@ private static Consumer createConsumer(
335341
if (decimalType.getPrecision() <= 38) {
336342
consumer = new AvroDecimalConsumer.BytesDecimalConsumer((DecimalVector) vector);
337343
} else {
338-
consumer = new AvroDecimal256Consumer.BytesDecimal256Consumer((Decimal256Vector) vector);
344+
consumer =
345+
new AvroDecimal256Consumer.BytesDecimal256Consumer((Decimal256Vector) vector);
339346
}
340347
} else {
341348
arrowType = new ArrowType.Binary();
@@ -473,7 +480,8 @@ private static Consumer createSkipConsumer(Schema schema) {
473480
return new SkipConsumer(skipFunction);
474481
}
475482

476-
static org.apache.arrow.vector.types.pojo.Schema createArrowSchema(Schema schema, AvroToArrowConfig config) {
483+
static org.apache.arrow.vector.types.pojo.Schema createArrowSchema(
484+
Schema schema, AvroToArrowConfig config) {
477485

478486
// Create an Arrow schema matching the structure of vectors built by createCompositeConsumer()
479487

@@ -541,7 +549,11 @@ private static Field avroSchemaToField(
541549
}
542550

543551
private static Field avroSchemaToField(
544-
Schema schema, String name, boolean nullable, AvroToArrowConfig config, Map<String, String> externalProps) {
552+
Schema schema,
553+
String name,
554+
boolean nullable,
555+
AvroToArrowConfig config,
556+
Map<String, String> externalProps) {
545557

546558
final Schema.Type type = schema.getType();
547559
final LogicalType logicalType = schema.getLogicalType();
@@ -550,16 +562,18 @@ private static Field avroSchemaToField(
550562

551563
switch (type) {
552564
case UNION:
553-
boolean nullableUnion = schema.getTypes().stream().anyMatch(t -> t.getType() == Schema.Type.NULL);
565+
boolean nullableUnion =
566+
schema.getTypes().stream().anyMatch(t -> t.getType() == Schema.Type.NULL);
554567
if (nullableUnion && schema.getTypes().size() == 2 && config.isHandleNullable()) {
555568
// For a simple nullable (null | type), interpret the union as a single nullable field.
556-
// Requires setting handleNullable in the config, otherwise fall back on the literal interpretation.
557-
Schema childSchema = schema.getTypes().get(0).getType() == Schema.Type.NULL
558-
? schema.getTypes().get(1)
559-
: schema.getTypes().get(0);
569+
// Requires setting handleNullable in the config, otherwise fall back on the literal
570+
// interpretation.
571+
Schema childSchema =
572+
schema.getTypes().get(0).getType() == Schema.Type.NULL
573+
? schema.getTypes().get(1)
574+
: schema.getTypes().get(0);
560575
return avroSchemaToField(childSchema, name, true, config, externalProps);
561-
}
562-
else {
576+
} else {
563577
// Literal interpretation of a union, which may or may not include a null element.
564578
for (int i = 0; i < schema.getTypes().size(); i++) {
565579
Schema childSchema = schema.getTypes().get(i);
@@ -585,10 +599,12 @@ private static Field avroSchemaToField(
585599
FieldType structFieldType =
586600
new FieldType(false, new ArrowType.Struct(), /* dictionary= */ null);
587601
Field structField =
588-
new Field(MapVector.DATA_VECTOR_NAME, structFieldType, Arrays.asList(keyField, valueField));
602+
new Field(
603+
MapVector.DATA_VECTOR_NAME, structFieldType, Arrays.asList(keyField, valueField));
589604
children.add(structField);
590605
fieldType =
591-
createFieldType(nullable, new ArrowType.Map(/* keysSorted= */ false), schema, externalProps);
606+
createFieldType(
607+
nullable, new ArrowType.Map(/* keysSorted= */ false), schema, externalProps);
592608
break;
593609
case RECORD:
594610
final Set<String> skipFieldNames = config.getSkipFieldNames();
@@ -674,10 +690,12 @@ private static Field avroSchemaToField(
674690
fieldType = createFieldType(nullable, longArrowType, schema, externalProps);
675691
break;
676692
case FLOAT:
677-
fieldType = createFieldType(nullable, new ArrowType.FloatingPoint(SINGLE), schema, externalProps);
693+
fieldType =
694+
createFieldType(nullable, new ArrowType.FloatingPoint(SINGLE), schema, externalProps);
678695
break;
679696
case DOUBLE:
680-
fieldType = createFieldType(nullable, new ArrowType.FloatingPoint(DOUBLE), schema, externalProps);
697+
fieldType =
698+
createFieldType(nullable, new ArrowType.FloatingPoint(DOUBLE), schema, externalProps);
681699
break;
682700
case BYTES:
683701
final ArrowType bytesArrowType;
@@ -707,11 +725,16 @@ private static Field avroSchemaToField(
707725
}
708726

709727
private static Consumer createArrayConsumer(
710-
Schema schema, String name, boolean nullable, AvroToArrowConfig config, FieldVector consumerVector) {
728+
Schema schema,
729+
String name,
730+
boolean nullable,
731+
AvroToArrowConfig config,
732+
FieldVector consumerVector) {
711733

712734
ListVector listVector;
713735
if (consumerVector == null) {
714-
final Field field = avroSchemaToField(schema, name, nullable, config, /* externalProps = */ null);
736+
final Field field =
737+
avroSchemaToField(schema, name, nullable, config, /* externalProps= */ null);
715738
listVector = (ListVector) field.createVector(config.getAllocator());
716739
} else {
717740
listVector = (ListVector) consumerVector;
@@ -727,13 +750,18 @@ private static Consumer createArrayConsumer(
727750
}
728751

729752
private static Consumer createStructConsumer(
730-
Schema schema, String name, boolean nullable, AvroToArrowConfig config, FieldVector consumerVector) {
753+
Schema schema,
754+
String name,
755+
boolean nullable,
756+
AvroToArrowConfig config,
757+
FieldVector consumerVector) {
731758

732759
final Set<String> skipFieldNames = config.getSkipFieldNames();
733760

734761
StructVector structVector;
735762
if (consumerVector == null) {
736-
final Field field = avroSchemaToField(schema, name, nullable, config, createExternalProps(schema));
763+
final Field field =
764+
avroSchemaToField(schema, name, nullable, config, createExternalProps(schema));
737765
structVector = (StructVector) field.createVector(config.getAllocator());
738766
} else {
739767
structVector = (StructVector) consumerVector;
@@ -764,11 +792,16 @@ private static Consumer createStructConsumer(
764792
}
765793

766794
private static Consumer createEnumConsumer(
767-
Schema schema, String name, boolean nullable, AvroToArrowConfig config, FieldVector consumerVector) {
795+
Schema schema,
796+
String name,
797+
boolean nullable,
798+
AvroToArrowConfig config,
799+
FieldVector consumerVector) {
768800

769801
BaseIntVector indexVector;
770802
if (consumerVector == null) {
771-
final Field field = avroSchemaToField(schema, name, nullable, config, createExternalProps(schema));
803+
final Field field =
804+
avroSchemaToField(schema, name, nullable, config, createExternalProps(schema));
772805
indexVector = (BaseIntVector) field.createVector(config.getAllocator());
773806
} else {
774807
indexVector = (BaseIntVector) consumerVector;
@@ -788,11 +821,16 @@ private static Consumer createEnumConsumer(
788821
}
789822

790823
private static Consumer createMapConsumer(
791-
Schema schema, String name, boolean nullable, AvroToArrowConfig config, FieldVector consumerVector) {
824+
Schema schema,
825+
String name,
826+
boolean nullable,
827+
AvroToArrowConfig config,
828+
FieldVector consumerVector) {
792829

793830
MapVector mapVector;
794831
if (consumerVector == null) {
795-
final Field field = avroSchemaToField(schema, name, nullable, config, /* externalProps = */ null);
832+
final Field field =
833+
avroSchemaToField(schema, name, nullable, config, /* externalProps= */ null);
796834
mapVector = (MapVector) field.createVector(config.getAllocator());
797835
} else {
798836
mapVector = (MapVector) consumerVector;
@@ -818,7 +856,11 @@ private static Consumer createMapConsumer(
818856
}
819857

820858
private static Consumer createUnionConsumer(
821-
Schema schema, String name, boolean nullableUnion, AvroToArrowConfig config, FieldVector consumerVector) {
859+
Schema schema,
860+
String name,
861+
boolean nullableUnion,
862+
AvroToArrowConfig config,
863+
FieldVector consumerVector) {
822864
final int size = schema.getTypes().size();
823865

824866
UnionVector unionVector;
@@ -837,7 +879,8 @@ private static Consumer createUnionConsumer(
837879
for (int i = 0; i < size; i++) {
838880
FieldVector child = childVectors.get(i);
839881
Schema subSchema = schema.getTypes().get(i);
840-
Consumer delegate = createConsumer(subSchema, subSchema.getName(), nullableUnion, config, child);
882+
Consumer delegate =
883+
createConsumer(subSchema, subSchema.getName(), nullableUnion, config, child);
841884
delegates[i] = delegate;
842885
types[i] = child.getMinorType();
843886
}
@@ -904,8 +947,8 @@ static VectorSchemaRoot avroToArrowVectors(
904947

905948
// Do not include props that are part of the Avro format itself as field metadata
906949
// These are already represented in the field / type structure and are not custom attributes
907-
private static final List<String> AVRO_FORMAT_METADATA = Arrays.asList(
908-
"logicalType", "precision", "scale");
950+
private static final List<String> AVRO_FORMAT_METADATA =
951+
Arrays.asList("logicalType", "precision", "scale");
909952

910953
private static Map<String, String> getMetaData(Schema schema) {
911954
Map<String, String> metadata = new HashMap<>();
@@ -955,8 +998,7 @@ private static FieldType createFieldType(
955998
Map<String, String> externalProps,
956999
DictionaryEncoding dictionary) {
9571000

958-
return createFieldType(
959-
/* nullable= */ false, arrowType, schema, externalProps, dictionary);
1001+
return createFieldType(/* nullable= */ false, arrowType, schema, externalProps, dictionary);
9601002
}
9611003

9621004
private static FieldType createFieldType(
@@ -966,8 +1008,7 @@ private static FieldType createFieldType(
9661008
Map<String, String> externalProps,
9671009
DictionaryEncoding dictionary) {
9681010

969-
return new FieldType(
970-
nullable, arrowType, dictionary, getMetaData(schema, externalProps));
1011+
return new FieldType(nullable, arrowType, dictionary, getMetaData(schema, externalProps));
9711012
}
9721013

9731014
private static String convertAliases(Set<String> aliases) {

adapter/avro/src/main/java/org/apache/arrow/adapter/avro/consumers/logical/AvroDecimal256Consumer.java

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -14,17 +14,15 @@
1414
* See the License for the specific language governing permissions and
1515
* limitations under the License.
1616
*/
17-
1817
package org.apache.arrow.adapter.avro.consumers.logical;
1918

19+
import java.io.IOException;
20+
import java.nio.ByteBuffer;
2021
import org.apache.arrow.adapter.avro.consumers.BaseAvroConsumer;
2122
import org.apache.arrow.util.Preconditions;
2223
import org.apache.arrow.vector.Decimal256Vector;
2324
import org.apache.avro.io.Decoder;
2425

25-
import java.io.IOException;
26-
import java.nio.ByteBuffer;
27-
2826
/**
2927
* Consumer which consume 256-bit decimal type values from avro decoder. Write the data to {@link
3028
* Decimal256Vector}.

adapter/avro/src/main/java/org/apache/arrow/adapter/avro/consumers/logical/AvroTimestampMicrosTzConsumer.java

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -14,15 +14,13 @@
1414
* See the License for the specific language governing permissions and
1515
* limitations under the License.
1616
*/
17-
1817
package org.apache.arrow.adapter.avro.consumers.logical;
1918

19+
import java.io.IOException;
2020
import org.apache.arrow.adapter.avro.consumers.BaseAvroConsumer;
2121
import org.apache.arrow.vector.TimeStampMicroTZVector;
2222
import org.apache.avro.io.Decoder;
2323

24-
import java.io.IOException;
25-
2624
/**
2725
* Consumer which consumes timestamp-micros values from avro decoder. Write the data to {@link
2826
* TimeStampMicroTZVector}.

adapter/avro/src/main/java/org/apache/arrow/adapter/avro/consumers/logical/AvroTimestampMillisTzConsumer.java

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -14,15 +14,13 @@
1414
* See the License for the specific language governing permissions and
1515
* limitations under the License.
1616
*/
17-
1817
package org.apache.arrow.adapter.avro.consumers.logical;
1918

19+
import java.io.IOException;
2020
import org.apache.arrow.adapter.avro.consumers.BaseAvroConsumer;
2121
import org.apache.arrow.vector.TimeStampMilliTZVector;
2222
import org.apache.avro.io.Decoder;
2323

24-
import java.io.IOException;
25-
2624
/**
2725
* Consumer which consume timestamp-millis values from avro decoder. Write the data to {@link
2826
* TimeStampMilliTZVector}.

adapter/avro/src/main/java/org/apache/arrow/adapter/avro/consumers/logical/AvroTimestampNanosConsumer.java

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -14,15 +14,13 @@
1414
* See the License for the specific language governing permissions and
1515
* limitations under the License.
1616
*/
17-
1817
package org.apache.arrow.adapter.avro.consumers.logical;
1918

19+
import java.io.IOException;
2020
import org.apache.arrow.adapter.avro.consumers.BaseAvroConsumer;
2121
import org.apache.arrow.vector.TimeStampNanoVector;
2222
import org.apache.avro.io.Decoder;
2323

24-
import java.io.IOException;
25-
2624
/**
2725
* Consumer which consume local-timestamp-nanos values from avro decoder. Write the data to {@link
2826
* TimeStampNanoVector}.

0 commit comments

Comments
 (0)