Skip to content

Commit

Permalink
API: Provide better error message for invalid FileFormat enum (apache…
Browse files Browse the repository at this point in the history
  • Loading branch information
nastra authored Oct 11, 2022
1 parent b71c37d commit 8c6adf6
Show file tree
Hide file tree
Showing 89 changed files with 115 additions and 195 deletions.
11 changes: 11 additions & 0 deletions api/src/main/java/org/apache/iceberg/FileFormat.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@
*/
package org.apache.iceberg;

import java.util.Locale;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.types.Comparators;

/** Enum of supported file formats. */
Expand Down Expand Up @@ -64,4 +66,13 @@ public static FileFormat fromFileName(CharSequence filename) {

return null;
}

public static FileFormat fromString(String fileFormat) {
Preconditions.checkArgument(null != fileFormat, "Invalid file format: null");
try {
return FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
} catch (IllegalArgumentException e) {
throw new IllegalArgumentException(String.format("Invalid file format: %s", fileFormat), e);
}
}
}
2 changes: 1 addition & 1 deletion core/src/main/java/org/apache/iceberg/BaseFile.java
Original file line number Diff line number Diff line change
Expand Up @@ -249,7 +249,7 @@ public void put(int i, Object value) {
this.filePath = value.toString();
return;
case 2:
this.format = FileFormat.valueOf(value.toString());
this.format = FileFormat.fromString(value.toString());
return;
case 3:
this.partitionSpecId = (value != null) ? (Integer) value : -1;
Expand Down
3 changes: 1 addition & 2 deletions core/src/main/java/org/apache/iceberg/DataFiles.java
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@

import java.nio.ByteBuffer;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import org.apache.hadoop.fs.FileStatus;
import org.apache.iceberg.encryption.EncryptedOutputFile;
Expand Down Expand Up @@ -210,7 +209,7 @@ public Builder withPath(String newFilePath) {
}

public Builder withFormat(String newFormat) {
this.format = FileFormat.valueOf(newFormat.toUpperCase(Locale.ENGLISH));
this.format = FileFormat.fromString(newFormat);
return this;
}

Expand Down
3 changes: 1 addition & 2 deletions core/src/main/java/org/apache/iceberg/FileMetadata.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
package org.apache.iceberg;

import java.nio.ByteBuffer;
import java.util.Locale;
import java.util.Map;
import org.apache.hadoop.fs.FileStatus;
import org.apache.iceberg.encryption.EncryptedOutputFile;
Expand Down Expand Up @@ -145,7 +144,7 @@ public Builder withPath(String newFilePath) {
}

public Builder withFormat(String newFormat) {
this.format = FileFormat.valueOf(newFormat.toUpperCase(Locale.ENGLISH));
this.format = FileFormat.fromString(newFormat);
return this;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;

import java.util.Locale;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.iceberg.FileFormat;
Expand Down Expand Up @@ -126,7 +125,7 @@ private Builder(Table table, int partitionId, long taskId) {

String formatAsString =
table.properties().getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT);
this.format = FileFormat.valueOf(formatAsString.toUpperCase(Locale.ROOT));
this.format = FileFormat.fromString(formatAsString);
}

public Builder defaultSpec(PartitionSpec newDefaultSpec) {
Expand Down
3 changes: 1 addition & 2 deletions data/src/test/java/org/apache/iceberg/TestSplitScan.java
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Locale;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.data.GenericAppenderFactory;
import org.apache.iceberg.data.IcebergGenerics;
Expand Down Expand Up @@ -66,7 +65,7 @@ public static Object[] parameters() {
private final FileFormat format;

public TestSplitScan(String format) {
this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH));
this.format = FileFormat.fromString(format);
}

@Before
Expand Down
3 changes: 1 addition & 2 deletions data/src/test/java/org/apache/iceberg/data/FileHelpers.java
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DataFiles;
Expand Down Expand Up @@ -146,6 +145,6 @@ private static EncryptedOutputFile encrypt(OutputFile out) {

private static FileFormat defaultFormat(Map<String, String> properties) {
String formatString = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT);
return FileFormat.valueOf(formatString.toUpperCase(Locale.ENGLISH));
return FileFormat.fromString(formatString);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@
import java.nio.ByteOrder;
import java.util.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
Expand Down Expand Up @@ -87,7 +86,7 @@ public static Object[] parameters() {
private final FileFormat format;

public TestLocalScan(String format) {
this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH));
this.format = FileFormat.fromString(format);
}

private String sharedTableLocation = null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,6 @@
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.List;
import java.util.Locale;
import java.util.UUID;
import org.apache.avro.generic.GenericData.Record;
import org.apache.avro.generic.GenericRecordBuilder;
Expand Down Expand Up @@ -98,7 +97,7 @@ public static Object[] parameters() {
private final FileFormat format;

public TestMetricsRowGroupFilter(String format) {
this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH));
this.format = FileFormat.fromString(format);
}

private static final Types.StructType structFieldType =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@
import java.time.OffsetDateTime;
import java.time.format.DateTimeFormatter;
import java.util.List;
import java.util.Locale;
import java.util.UUID;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
Expand Down Expand Up @@ -277,7 +276,7 @@ public static Object[][] parameters() {

public TestMetricsRowGroupFilterTypes(
String format, String column, Object readValue, Object skipValue) {
this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH));
this.format = FileFormat.fromString(format);
this.column = column;
this.readValue = readValue;
this.skipValue = skipValue;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Locale;
import java.util.Set;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.FileFormat;
Expand Down Expand Up @@ -77,7 +76,7 @@ public static Object[] parameters() {

public TestAppenderFactory(String fileFormat, boolean partitioned) {
super(FORMAT_V2);
this.format = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
this.format = FileFormat.fromString(fileFormat);
this.partitioned = partitioned;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import java.nio.file.Paths;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
import java.util.stream.Collectors;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.PartitionSpec;
Expand Down Expand Up @@ -60,7 +59,7 @@ public static Object[][] parameters() {

public TestBaseTaskWriter(String fileFormat) {
super(FORMAT_V2);
this.format = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
this.format = FileFormat.fromString(fileFormat);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Locale;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DeleteFile;
import org.apache.iceberg.FileFormat;
Expand Down Expand Up @@ -66,7 +65,7 @@ public static Object[] parameters() {

public TestGenericSortedPosDeleteWriter(String fileFormat) {
super(FORMAT_V2);
this.format = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
this.format = FileFormat.fromString(fileFormat);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
import java.util.function.Function;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DeleteFile;
Expand Down Expand Up @@ -75,7 +74,7 @@ public static Object[][] parameters() {

public TestTaskEqualityDeltaWriter(String fileFormat) {
super(FORMAT_V2);
this.format = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
this.format = FileFormat.fromString(fileFormat);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
*/
package org.apache.iceberg.flink;

import java.util.Locale;
import java.util.Map;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.iceberg.DistributionMode;
Expand Down Expand Up @@ -81,7 +80,7 @@ public FileFormat dataFileFormat() {
.tableProperty(TableProperties.DEFAULT_FILE_FORMAT)
.defaultValue(TableProperties.DEFAULT_FILE_FORMAT_DEFAULT)
.parse();
return FileFormat.valueOf(valueAsString.toUpperCase(Locale.ENGLISH));
return FileFormat.fromString(valueAsString);
}

public long targetDataFileSize() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT;

import java.io.Serializable;
import java.util.Locale;
import java.util.Map;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.StringData;
Expand Down Expand Up @@ -192,11 +191,11 @@ static class Builder {

String dataFileFormatName =
properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT);
this.dataFileFormat = FileFormat.valueOf(dataFileFormatName.toUpperCase(Locale.ENGLISH));
this.dataFileFormat = FileFormat.fromString(dataFileFormatName);

String deleteFileFormatName =
properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName);
this.deleteFileFormat = FileFormat.valueOf(deleteFileFormatName.toUpperCase(Locale.ENGLISH));
this.deleteFileFormat = FileFormat.fromString(deleteFileFormatName);
}

Builder dataFileFormat(FileFormat newDataFileFormat) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@

import java.util.Collection;
import java.util.List;
import java.util.Locale;
import java.util.stream.Collectors;
import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.configuration.Configuration;
Expand Down Expand Up @@ -74,7 +73,7 @@ public RowDataRewriter(
table.properties(),
TableProperties.DEFAULT_FILE_FORMAT,
TableProperties.DEFAULT_FILE_FORMAT_DEFAULT);
FileFormat format = FileFormat.valueOf(formatString.toUpperCase(Locale.ENGLISH));
FileFormat format = FileFormat.fromString(formatString);
RowType flinkSchema = FlinkSchemaUtil.convert(table.schema());
this.taskWriterFactory =
new RowDataTaskWriterFactory(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@
import java.nio.file.Paths;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
import java.util.stream.Collectors;
import org.apache.flink.table.data.RowData;
import org.apache.iceberg.FileContent;
Expand Down Expand Up @@ -68,7 +67,7 @@ public static Object[][] parameters() {

public TestDeltaTaskWriter(String fileFormat) {
super(FORMAT_V2);
this.format = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
this.format = FileFormat.fromString(fileFormat);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.flink.api.common.typeinfo.TypeInformation;
Expand Down Expand Up @@ -97,7 +96,7 @@ public static Object[][] parameters() {
}

public TestFlinkIcebergSink(String format, int parallelism, boolean partitioned) {
this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH));
this.format = FileFormat.fromString(format);
this.parallelism = parallelism;
this.partitioned = partitioned;
}
Expand Down Expand Up @@ -378,7 +377,7 @@ public void testOverrideWriteConfigWithUnknownFileFormat() {
AssertHelpers.assertThrows(
"Should fail with invalid file format.",
IllegalArgumentException.class,
"No enum constant org.apache.iceberg.FileFormat.UNRECOGNIZED",
"Invalid file format: UNRECOGNIZED",
() -> {
builder.append();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.functions.KeySelector;
Expand Down Expand Up @@ -112,7 +111,7 @@ public static Object[][] parameters() {
public TestFlinkIcebergSinkV2(
String format, int parallelism, boolean partitioned, String writeDistributionMode) {
super(FORMAT_V2);
this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH));
this.format = FileFormat.fromString(format);
this.parallelism = parallelism;
this.partitioned = partitioned;
this.writeDistributionMode = writeDistributionMode;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.List;
import java.util.Locale;
import java.util.stream.Collectors;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.JobID;
Expand Down Expand Up @@ -90,7 +89,7 @@ public static Object[][] parameters() {

public TestIcebergFilesCommitter(String format, int formatVersion) {
super(formatVersion);
this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH));
this.format = FileFormat.fromString(format);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import org.apache.flink.streaming.api.operators.BoundedOneInput;
Expand Down Expand Up @@ -88,7 +87,7 @@ public static Object[][] parameters() {
}

public TestIcebergStreamWriter(String format, boolean partitioned) {
this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH));
this.format = FileFormat.fromString(format);
this.partitioned = partitioned;
}

Expand Down
Loading

0 comments on commit 8c6adf6

Please sign in to comment.