Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions core/src/main/java/org/apache/iceberg/ManifestFiles.java
Original file line number Diff line number Diff line change
Expand Up @@ -226,6 +226,7 @@ static ManifestFile copyRewriteManifest(int formatVersion,
}
}

@SuppressWarnings("Finally")
private static ManifestFile copyManifestInternal(int formatVersion, ManifestReader<DataFile> reader,
OutputFile outputFile, long snapshotId,
SnapshotSummary.Builder summaryBuilder,
Expand Down
3 changes: 1 addition & 2 deletions core/src/main/java/org/apache/iceberg/MetricsModes.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import java.io.ObjectStreamException;
import java.io.Serializable;
import java.util.Locale;
import java.util.Objects;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
Expand Down Expand Up @@ -134,7 +133,7 @@ public boolean equals(Object other) {

@Override
public int hashCode() {
return Objects.hash(length);
return Integer.hashCode(length);
}
}

Expand Down
1 change: 1 addition & 0 deletions core/src/main/java/org/apache/iceberg/avro/Avro.java
Original file line number Diff line number Diff line change
Expand Up @@ -577,6 +577,7 @@ public static class ReadBuilder {
private org.apache.iceberg.Schema schema = null;
private Function<Schema, DatumReader<?>> createReaderFunc = null;
private BiFunction<org.apache.iceberg.Schema, Schema, DatumReader<?>> createReaderBiFunc = null;
@SuppressWarnings("UnnecessaryLambda")
private final Function<Schema, DatumReader<?>> defaultCreateReaderFunc = readSchema -> {
GenericAvroReader<?> reader = new GenericAvroReader<>(readSchema);
reader.setClassLoader(loader);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ public InputFilesDecryptor(CombinedScanTask combinedTask, FileIO io, EncryptionM
.map(entry -> EncryptedFiles.encryptedInput(io.newInputFile(entry.getKey()), entry.getValue()));

// decrypt with the batch call to avoid multiple RPCs to a key server, if possible
@SuppressWarnings("StreamToIterable")
Iterable<InputFile> decryptedFiles = encryption.decrypt(encrypted::iterator);

Map<String, InputFile> files = Maps.newHashMapWithExpectedSize(keyMetadata.size());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -196,7 +196,7 @@ public void renameTable(TableIdentifier from, TableIdentifier to) {
err -> {
// SQLite doesn't set SQLState or throw SQLIntegrityConstraintViolationException
if (err instanceof SQLIntegrityConstraintViolationException ||
err.getMessage() != null && err.getMessage().contains("constraint failed")) {
(err.getMessage() != null && err.getMessage().contains("constraint failed"))) {
throw new AlreadyExistsException("Table already exists: %s", to);
}
},
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
package org.apache.iceberg.rest;

import java.time.OffsetDateTime;
import java.time.ZoneOffset;
import java.util.List;
import java.util.Map;
import java.util.Set;
Expand Down Expand Up @@ -166,7 +167,7 @@ public static LoadTableResponse stageTableCreate(Catalog catalog, Namespace name
}

Map<String, String> properties = Maps.newHashMap();
properties.put("created-at", OffsetDateTime.now().toString());
properties.put("created-at", OffsetDateTime.now(ZoneOffset.UTC).toString());
properties.putAll(request.properties());

String location;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -352,6 +352,7 @@ private ScheduledExecutorService tokenRefreshExecutor() {
return refreshExecutor;
}

@SuppressWarnings("FutureReturnValueIgnored")
private void scheduleTokenRefresh(
AuthSession session, long startTimeMillis, long expiresIn, TimeUnit unit) {
// convert expiration interval to milliseconds
Expand Down
1 change: 1 addition & 0 deletions core/src/main/java/org/apache/iceberg/util/Tasks.java
Original file line number Diff line number Diff line change
Expand Up @@ -569,6 +569,7 @@ public static <I> Builder<I> foreach(I... items) {
return new Builder<>(Arrays.asList(items));
}

@SuppressWarnings("StreamToIterable")
public static <I> Builder<I> foreach(Stream<I> items) {
return new Builder<>(items::iterator);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,6 +123,7 @@ public static ByteBuffer doubleToOrderedBytes(double val, ByteBuffer reuse) {
* This implementation just uses a set size to for all output byte representations. Truncating longer strings
* and right padding 0 for shorter strings.
*/
@SuppressWarnings("ByteBufferBackingArray")
public static ByteBuffer stringToOrderedBytes(String val, int length, ByteBuffer reuse, CharsetEncoder encoder) {
Preconditions.checkArgument(encoder.charset().equals(StandardCharsets.UTF_8),
"Cannot use an encoder not using UTF_8 as it's Charset");
Expand All @@ -140,6 +141,7 @@ public static ByteBuffer stringToOrderedBytes(String val, int length, ByteBuffer
* Return a bytebuffer with the given bytes truncated to length, or filled with 0's to length depending on whether
* the given bytes are larger or smaller than the given length.
*/
@SuppressWarnings("ByteBufferBackingArray")
public static ByteBuffer byteTruncateOrFill(byte[] val, int length, ByteBuffer reuse) {
ByteBuffer bytes = ByteBuffers.reuse(reuse, length);
if (val.length < length) {
Expand All @@ -164,6 +166,7 @@ static byte[] interleaveBits(byte[][] columnsBinary, int interleavedSize) {
* @param interleavedSize the number of bytes to use in the output
* @return the columnbytes interleaved
*/
@SuppressWarnings("ByteBufferBackingArray")
public static byte[] interleaveBits(byte[][] columnsBinary, int interleavedSize, ByteBuffer reuse) {
byte[] interleavedBytes = reuse.array();
Arrays.fill(interleavedBytes, 0, interleavedSize, (byte) 0x00);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

import java.io.IOException;
import java.time.OffsetDateTime;
import java.time.ZoneOffset;
import java.util.List;
import java.util.Map;
import org.apache.iceberg.Schema;
Expand Down Expand Up @@ -200,7 +201,7 @@ public void testBasicProjection() throws Exception {
Record record = GenericRecord.create(writeSchema.asStruct());
record.setField("id", 34L);
record.setField("data", "test");
record.setField("time", OffsetDateTime.now());
record.setField("time", OffsetDateTime.now(ZoneOffset.UTC));

Schema idOnly = new Schema(
Types.NestedField.required(0, "id", Types.LongType.get())
Expand Down