diff --git a/lang/java/avro/src/main/java/org/apache/avro/Conversions.java b/lang/java/avro/src/main/java/org/apache/avro/Conversions.java index 99ad500647e..2fa15eb959c 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/Conversions.java +++ b/lang/java/avro/src/main/java/org/apache/avro/Conversions.java @@ -192,7 +192,7 @@ public BigDecimal fromBytes(final ByteBuffer value, final Schema schema, final L BigInteger bg = null; ByteBuffer buffer = decoder.readBytes(null); byte[] array = buffer.array(); - if (array != null && array.length > 0) { + if (array.length > 0) { bg = new BigInteger(array); } diff --git a/lang/java/avro/src/main/java/org/apache/avro/JsonProperties.java b/lang/java/avro/src/main/java/org/apache/avro/JsonProperties.java index 0c100baa98f..5384b8595fc 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/JsonProperties.java +++ b/lang/java/avro/src/main/java/org/apache/avro/JsonProperties.java @@ -150,9 +150,9 @@ private Null() { // Also, we only ever ADD to the collection, never changing a value, so // putWithAbsent is the // only modifier - private ConcurrentMap props = new ConcurrentHashMap() { + private final ConcurrentMap props = new ConcurrentHashMap<>() { private static final long serialVersionUID = 1L; - private Queue> propOrder = new ConcurrentLinkedQueue<>(); + private final Queue> propOrder = new ConcurrentLinkedQueue<>(); @Override public JsonNode putIfAbsent(String key, JsonNode value) { @@ -170,10 +170,10 @@ public JsonNode put(String key, JsonNode value) { @Override public Set> entrySet() { - return new AbstractSet>() { + return new AbstractSet<>() { @Override public Iterator> iterator() { - return new Iterator>() { + return new Iterator<>() { Iterator> it = propOrder.iterator(); @Override @@ -196,7 +196,7 @@ public int size() { } }; - private Set reserved; + private final Set reserved; JsonProperties(Set reserved) { this.reserved = reserved; @@ -206,7 +206,7 @@ public int size() { this.reserved = reserved; for (Entry a : propMap.entrySet()) { Object v = a.getValue(); - JsonNode json = null; + JsonNode json; if (v instanceof String) { json = TextNode.valueOf((String) v); } else if (v instanceof JsonNode) { diff --git a/lang/java/avro/src/main/java/org/apache/avro/Protocol.java b/lang/java/avro/src/main/java/org/apache/avro/Protocol.java index 905f2778c6b..3404b93d4a5 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/Protocol.java +++ b/lang/java/avro/src/main/java/org/apache/avro/Protocol.java @@ -25,7 +25,6 @@ import java.nio.charset.StandardCharsets; import java.security.MessageDigest; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashSet; @@ -71,11 +70,9 @@ public class Protocol extends JsonProperties { public static final long VERSION = 1; // Support properties for both Protocol and Message objects - private static final Set MESSAGE_RESERVED = Collections - .unmodifiableSet(new HashSet<>(Arrays.asList("doc", "response", "request", "errors", "one-way"))); + private static final Set MESSAGE_RESERVED = Set.of("doc", "response", "request", "errors", "one-way"); - private static final Set FIELD_RESERVED = Collections - .unmodifiableSet(new HashSet<>(Arrays.asList("name", "type", "doc", "default", "aliases"))); + private static final Set FIELD_RESERVED = Set.of("name", "type", "doc", "default", "aliases"); /** A protocol message. */ public class Message extends JsonProperties { @@ -255,8 +252,8 @@ void toJson1(Set knownNames, JsonGenerator gen) throws IOException { /** Union type for generating system errors. */ public static final Schema SYSTEM_ERRORS = Schema.createUnion(Collections.singletonList(SYSTEM_ERROR)); - private static final Set PROTOCOL_RESERVED = Collections - .unmodifiableSet(new HashSet<>(Arrays.asList("namespace", "protocol", "doc", "messages", "types", "errors"))); + private static final Set PROTOCOL_RESERVED = Set.of("namespace", "protocol", "doc", "messages", "types", + "errors"); private Protocol() { super(PROTOCOL_RESERVED); diff --git a/lang/java/avro/src/main/java/org/apache/avro/Resolver.java b/lang/java/avro/src/main/java/org/apache/avro/Resolver.java index 117c9e3911f..8b62b24d757 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/Resolver.java +++ b/lang/java/avro/src/main/java/org/apache/avro/Resolver.java @@ -435,7 +435,7 @@ public static class RecordAdjust extends Action { * fields that will be read from the writer: these n are in the order * dictated by writer's schema. The remaining m fields will be read from * default values (actions for these default values are found in - * {@link RecordAdjust#defaults}. + * {@link RecordAdjust#defaults}). */ public final Field[] readerOrder; diff --git a/lang/java/avro/src/main/java/org/apache/avro/file/BZip2Codec.java b/lang/java/avro/src/main/java/org/apache/avro/file/BZip2Codec.java index fe90557fa2e..8fd6b6a09bd 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/file/BZip2Codec.java +++ b/lang/java/avro/src/main/java/org/apache/avro/file/BZip2Codec.java @@ -64,7 +64,7 @@ public ByteBuffer decompress(ByteBuffer compressedData) throws IOException { try (BZip2CompressorInputStream inputStream = new BZip2CompressorInputStream(bais)) { - int readCount = -1; + int readCount; while ((readCount = inputStream.read(buffer, compressedData.position(), buffer.length)) > 0) { baos.write(buffer, 0, readCount); } diff --git a/lang/java/avro/src/main/java/org/apache/avro/file/DataFileConstants.java b/lang/java/avro/src/main/java/org/apache/avro/file/DataFileConstants.java index fe269ca06b2..4664f5410df 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/file/DataFileConstants.java +++ b/lang/java/avro/src/main/java/org/apache/avro/file/DataFileConstants.java @@ -27,7 +27,6 @@ private DataFileConstants() { public static final byte VERSION = 1; public static final byte[] MAGIC = new byte[] { (byte) 'O', (byte) 'b', (byte) 'j', VERSION }; - public static final long FOOTER_BLOCK = -1; public static final int SYNC_SIZE = 16; public static final int DEFAULT_SYNC_INTERVAL = 4000 * SYNC_SIZE; diff --git a/lang/java/avro/src/main/java/org/apache/avro/file/DataFileStream.java b/lang/java/avro/src/main/java/org/apache/avro/file/DataFileStream.java index aa458684635..4bf1ec5b6b1 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/file/DataFileStream.java +++ b/lang/java/avro/src/main/java/org/apache/avro/file/DataFileStream.java @@ -63,7 +63,7 @@ private Header() { } } - private DatumReader reader; + private final DatumReader reader; private long blockSize; private boolean availableBlock = false; private Header header; @@ -94,7 +94,7 @@ public DataFileStream(InputStream in, DatumReader reader) throws IOException /** * create an uninitialized DataFileStream */ - protected DataFileStream(DatumReader reader) throws IOException { + protected DataFileStream(DatumReader reader) { this.reader = reader; } @@ -147,7 +147,7 @@ void initialize(InputStream in, byte[] magic) throws IOException { } /** Initialize the stream without reading from it. */ - void initialize(Header header) throws IOException { + void initialize(Header header) { this.header = header; this.codec = resolveCodec(); reader.setSchema(header.schema); @@ -366,22 +366,6 @@ private DataBlock(long numEntries, int blockSize) { this.numEntries = numEntries; } - byte[] getData() { - return data; - } - - long getNumEntries() { - return numEntries; - } - - int getBlockSize() { - return blockSize; - } - - boolean isFlushOnWrite() { - return flushOnWrite; - } - void setFlushOnWrite(boolean flushOnWrite) { this.flushOnWrite = flushOnWrite; } diff --git a/lang/java/avro/src/main/java/org/apache/avro/file/DataFileWriter.java b/lang/java/avro/src/main/java/org/apache/avro/file/DataFileWriter.java index 58235da8848..c4e031b75d9 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/file/DataFileWriter.java +++ b/lang/java/avro/src/main/java/org/apache/avro/file/DataFileWriter.java @@ -25,7 +25,7 @@ import org.apache.avro.io.DatumWriter; import org.apache.avro.io.EncoderFactory; import org.apache.avro.util.NonCopyingByteArrayOutputStream; -import org.apache.commons.compress.utils.IOUtils; +import org.apache.commons.io.IOUtils; import java.io.BufferedOutputStream; import java.io.Closeable; @@ -56,7 +56,7 @@ */ public class DataFileWriter implements Closeable, Flushable { private Schema schema; - private DatumWriter dout; + private final DatumWriter dout; private OutputStream underlyingStream; @@ -117,11 +117,10 @@ public DataFileWriter setCodec(CodecFactory c) { * is written. In this case, the {@linkplain #flush()} must be called to flush * the stream. * - * Invalid values throw IllegalArgumentException - * * @param syncInterval the approximate number of uncompressed bytes to write in * each block * @return this DataFileWriter + * @throws IllegalArgumentException if syncInterval is invalid */ public DataFileWriter setSyncInterval(int syncInterval) { if (syncInterval < 32 || syncInterval > (1 << 30)) { @@ -193,7 +192,7 @@ public DataFileWriter create(Schema schema, OutputStream outs, byte[] sync) t * Set whether this writer should flush the block to the stream every time a * sync marker is written. By default, the writer will flush the buffer each * time a sync marker is written (if the block size limit is reached or the - * {@linkplain #sync()} is called. + * {@linkplain #sync()} is called). * * @param flushOnEveryBlock - If set to false, this writer will not flush the * block to the stream until {@linkplain #flush()} is @@ -475,11 +474,11 @@ public void close() throws IOException { } } - private class BufferedFileOutputStream extends BufferedOutputStream { + private static class BufferedFileOutputStream extends BufferedOutputStream { private long position; // start of buffer private class PositionFilter extends FilterOutputStream { - public PositionFilter(OutputStream out) throws IOException { + public PositionFilter(OutputStream out) { super(out); } @@ -490,7 +489,7 @@ public void write(byte[] b, int off, int len) throws IOException { } } - public BufferedFileOutputStream(OutputStream out) throws IOException { + public BufferedFileOutputStream(OutputStream out) { super(null); this.out = new PositionFilter(out); } diff --git a/lang/java/avro/src/main/java/org/apache/avro/file/DeflateCodec.java b/lang/java/avro/src/main/java/org/apache/avro/file/DeflateCodec.java index 87498d3ee82..e6d58e46a13 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/file/DeflateCodec.java +++ b/lang/java/avro/src/main/java/org/apache/avro/file/DeflateCodec.java @@ -40,7 +40,7 @@ public class DeflateCodec extends Codec { private static final int DEFAULT_BUFFER_SIZE = 8192; static class Option extends CodecFactory { - private int compressionLevel; + private final int compressionLevel; Option(int compressionLevel) { this.compressionLevel = compressionLevel; @@ -55,8 +55,8 @@ protected Codec createInstance() { private Deflater deflater; private Inflater inflater; // currently only do 'nowrap' -- RFC 1951, not zlib - private boolean nowrap = true; - private int compressionLevel; + private final boolean nowrap = true; + private final int compressionLevel; public DeflateCodec(int compressionLevel) { this.compressionLevel = compressionLevel; diff --git a/lang/java/avro/src/main/java/org/apache/avro/file/FileReader.java b/lang/java/avro/src/main/java/org/apache/avro/file/FileReader.java index 07229d59ee8..9a54cf055ef 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/file/FileReader.java +++ b/lang/java/avro/src/main/java/org/apache/avro/file/FileReader.java @@ -31,7 +31,7 @@ public interface FileReader extends Iterator, Iterable, Closeable { /** * Read the next datum from the file. - * + * * @param reuse an instance to reuse. * @throws NoSuchElementException if no more remain in the file. */ @@ -39,7 +39,7 @@ public interface FileReader extends Iterator, Iterable, Closeable { /** * Move to the next synchronization point after a position. To process a range - * of file entires, call this with the starting position, then check + * of file entries, call this with the starting position, then check * {@link #pastSync(long)} with the end point before each call to * {@link #next()}. */ diff --git a/lang/java/avro/src/main/java/org/apache/avro/file/SnappyCodec.java b/lang/java/avro/src/main/java/org/apache/avro/file/SnappyCodec.java index 72bf0b74822..454d2925deb 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/file/SnappyCodec.java +++ b/lang/java/avro/src/main/java/org/apache/avro/file/SnappyCodec.java @@ -26,7 +26,7 @@ /** * Implements Snappy compression and decompression. */ public class SnappyCodec extends Codec { - private CRC32 crc32 = new CRC32(); + private final CRC32 crc32 = new CRC32(); static class Option extends CodecFactory { static { @@ -72,7 +72,7 @@ public ByteBuffer decompress(ByteBuffer in) throws IOException { crc32.reset(); crc32.update(out.array(), 0, size); - if (in.getInt(((Buffer) in).limit() - 4) != (int) crc32.getValue()) + if (in.getInt(in.limit() - 4) != (int) crc32.getValue()) throw new IOException("Checksum failure"); return out; diff --git a/lang/java/avro/src/main/java/org/apache/avro/file/XZCodec.java b/lang/java/avro/src/main/java/org/apache/avro/file/XZCodec.java index 3052f2a4160..bc674b73466 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/file/XZCodec.java +++ b/lang/java/avro/src/main/java/org/apache/avro/file/XZCodec.java @@ -26,7 +26,6 @@ import org.apache.avro.util.NonCopyingByteArrayOutputStream; import org.apache.commons.compress.compressors.xz.XZCompressorInputStream; import org.apache.commons.compress.compressors.xz.XZCompressorOutputStream; -import org.apache.commons.compress.utils.IOUtils; /** * Implements xz compression and decompression. */ public class XZCodec extends Codec { @@ -34,7 +33,7 @@ public class XZCodec extends Codec { private static final int DEFAULT_BUFFER_SIZE = 8192; static class Option extends CodecFactory { - private int compressionLevel; + private final int compressionLevel; Option(int compressionLevel) { this.compressionLevel = compressionLevel; @@ -46,7 +45,7 @@ protected Codec createInstance() { } } - private int compressionLevel; + private final int compressionLevel; public XZCodec(int compressionLevel) { this.compressionLevel = compressionLevel; @@ -72,7 +71,7 @@ public ByteBuffer decompress(ByteBuffer data) throws IOException { InputStream bytesIn = new ByteArrayInputStream(data.array(), computeOffset(data), data.remaining()); try (InputStream ios = new XZCompressorInputStream(bytesIn)) { - IOUtils.copy(ios, baos); + ios.transferTo(baos); } return baos.asByteBuffer(); } diff --git a/lang/java/avro/src/main/java/org/apache/avro/file/ZstandardCodec.java b/lang/java/avro/src/main/java/org/apache/avro/file/ZstandardCodec.java index f778b2fe356..0d4e31958d9 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/file/ZstandardCodec.java +++ b/lang/java/avro/src/main/java/org/apache/avro/file/ZstandardCodec.java @@ -24,7 +24,6 @@ import java.nio.ByteBuffer; import org.apache.avro.util.NonCopyingByteArrayOutputStream; -import org.apache.commons.compress.utils.IOUtils; public class ZstandardCodec extends Codec { public final static int DEFAULT_COMPRESSION = 3; @@ -82,7 +81,7 @@ public ByteBuffer decompress(ByteBuffer compressedData) throws IOException { InputStream bytesIn = new ByteArrayInputStream(compressedData.array(), computeOffset(compressedData), compressedData.remaining()); try (InputStream ios = ZstandardLoader.input(bytesIn, useBufferPool)) { - IOUtils.copy(ios, baos); + ios.transferTo(baos); } return baos.asByteBuffer(); } diff --git a/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java b/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java index 362ebdc9cfc..42191bff055 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java +++ b/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java @@ -137,9 +137,9 @@ private void loadConversions() { } } - private Map> conversions = new HashMap<>(); + private final Map> conversions = new HashMap<>(); - private Map, Map>> conversionsByClass = new IdentityHashMap<>(); + private final Map, Map>> conversionsByClass = new IdentityHashMap<>(); public Collection> getConversions() { return conversions.values(); @@ -364,7 +364,7 @@ public int hashCode() { @Override public Iterator iterator() { - return new Iterator() { + return new Iterator<>() { private int position = 0; @Override @@ -551,8 +551,8 @@ public int compareTo(Fixed that) { /** Default implementation of {@link GenericEnumSymbol}. */ public static class EnumSymbol implements GenericEnumSymbol { - private Schema schema; - private String symbol; + private final Schema schema; + private final String symbol; public EnumSymbol(Schema schema, String symbol) { this.schema = schema; diff --git a/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumWriter.java b/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumWriter.java index deeac0b1f2b..20a856c4dc3 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumWriter.java +++ b/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumWriter.java @@ -219,9 +219,7 @@ protected ClassCastException addClassCastMsg(ClassCastException e, String s) { /** Helper method for adding a message to an Avro Type Exception . */ protected AvroTypeException addAvroTypeMsg(AvroTypeException e, String s) { - AvroTypeException result = new AvroTypeException(e.getMessage() + s); - result.initCause(e.getCause() == null ? e : e.getCause()); - return result; + return new AvroTypeException(e.getMessage() + s, e.getCause() == null ? e : e.getCause()); } /** @@ -282,7 +280,7 @@ protected void writeArray(Schema schema, Object datum, Encoder out) throws IOExc long actualSize = 0; out.writeArrayStart(); out.setItemCount(size); - for (Iterator it = getArrayElements(datum); it.hasNext();) { + for (Iterator it = getArrayElements(datum); it.hasNext();) { out.startItem(); try { write(element, it.next(), out); diff --git a/lang/java/avro/src/main/java/org/apache/avro/generic/PrimitivesArrays.java b/lang/java/avro/src/main/java/org/apache/avro/generic/PrimitivesArrays.java index d34ce0f5bcb..814dd917477 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/generic/PrimitivesArrays.java +++ b/lang/java/avro/src/main/java/org/apache/avro/generic/PrimitivesArrays.java @@ -58,7 +58,7 @@ public Integer get(int i) { /** * Direct primitive int access. - * + * * @param i : index. * @return value at index. */ @@ -162,7 +162,7 @@ public Long get(int i) { /** * Direct primitive int access. - * + * * @param i : index. * @return value at index. */ @@ -273,7 +273,7 @@ public Boolean get(int i) { /** * Direct primitive int access. - * + * * @param i : index. * @return value at index. */ @@ -321,7 +321,7 @@ public void add(int location, boolean o) { size++; for (int index = this.size / 8; index > (location / 8); index--) { elements[index] <<= 1; - if (index > 0 && (elements[index - 1] & (1 << Byte.SIZE)) > 0) { + if ((elements[index - 1] & (1 << Byte.SIZE)) > 0) { elements[index] |= 1; } } @@ -431,7 +431,7 @@ public Float get(int i) { /** * Direct primitive int access. - * + * * @param i : index. * @return value at index. */ @@ -535,7 +535,7 @@ public Double get(int i) { /** * Direct primitive int access. - * + * * @param i : index. * @return value at index. */ diff --git a/lang/java/avro/src/main/java/org/apache/avro/io/BinaryData.java b/lang/java/avro/src/main/java/org/apache/avro/io/BinaryData.java index b6126ec236c..9ce68e9f4cf 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/io/BinaryData.java +++ b/lang/java/avro/src/main/java/org/apache/avro/io/BinaryData.java @@ -255,11 +255,11 @@ private static int hashCode(HashData data, Schema schema) throws IOException { case UNION: return hashCode(data, schema.getTypes().get(decoder.readInt())); case FIXED: - return hashBytes(1, data, schema.getFixedSize(), false); + return hashBytes(data, schema.getFixedSize(), false); case STRING: - return hashBytes(1, data, decoder.readInt(), false); + return hashBytes(data, decoder.readInt(), false); case BYTES: - return hashBytes(1, data, decoder.readInt(), true); + return hashBytes(data, decoder.readInt(), true); case NULL: return 0; default: @@ -267,8 +267,8 @@ private static int hashCode(HashData data, Schema schema) throws IOException { } } - private static int hashBytes(int init, HashData data, int len, boolean rev) throws IOException { - int hashCode = init; + private static int hashBytes(HashData data, int len, boolean rev) throws IOException { + int hashCode = 1; byte[] bytes = data.decoder.getBuf(); int start = data.decoder.getPos(); int end = start + len; diff --git a/lang/java/avro/src/main/java/org/apache/avro/io/BinaryDecoder.java b/lang/java/avro/src/main/java/org/apache/avro/io/BinaryDecoder.java index 7217be3addd..69e0186c85c 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/io/BinaryDecoder.java +++ b/lang/java/avro/src/main/java/org/apache/avro/io/BinaryDecoder.java @@ -199,7 +199,7 @@ public long readLong() throws IOException { if (b > 0x7f) { // only the low 28 bits can be set, so this won't carry // the sign bit to the long - l = innerLongDecode((long) n); + l = innerLongDecode(n); } else { l = n; } @@ -779,7 +779,7 @@ public int available() throws IOException { } private static class InputStreamByteSource extends ByteSource { - private InputStream in; + private final InputStream in; protected boolean isEof = false; private InputStreamByteSource(InputStream in) { @@ -909,7 +909,7 @@ public void close() throws IOException { */ private static class ByteArrayByteSource extends ByteSource { private static final int MIN_SIZE = 16; - private byte[] data; + private final byte[] data; private int position; private int max; private boolean compacted = false; @@ -949,7 +949,7 @@ protected void skipSourceBytes(long length) throws IOException { } @Override - protected long trySkipBytes(long length) throws IOException { + protected long trySkipBytes(long length) { // the buffer is shared, so this should return 0 max = ba.getLim(); position = ba.getPos(); @@ -974,13 +974,13 @@ protected void readRaw(byte[] data, int off, int len) throws IOException { } @Override - protected int tryReadRaw(byte[] data, int off, int len) throws IOException { + protected int tryReadRaw(byte[] data, int off, int len) { // the buffer is shared, nothing to read return 0; } @Override - protected void compactAndFill(byte[] buf, int pos, int minPos, int remaining) throws IOException { + protected void compactAndFill(byte[] buf, int pos, int minPos, int remaining) { // this implementation does not want to mutate the array passed in, // so it makes a new tiny buffer unless it has been compacted once before if (!compacted) { diff --git a/lang/java/avro/src/main/java/org/apache/avro/io/BlockingBinaryEncoder.java b/lang/java/avro/src/main/java/org/apache/avro/io/BlockingBinaryEncoder.java index d0bfc8f075e..9a0d9e414b0 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/io/BlockingBinaryEncoder.java +++ b/lang/java/avro/src/main/java/org/apache/avro/io/BlockingBinaryEncoder.java @@ -89,7 +89,7 @@ public enum State { * this case, {@link BlockedValue#start} is zero. The header for such a block * has _already been written_ (we've written out a header indicating that the * block has a single item, and we put a "zero" down for the byte-count to - * indicate that we don't know the physical length of the buffer. Any blocks + * indicate that we don't know the physical length of the buffer). Any blocks * _containing_ this block must be in the {@link #OVERFLOW} state. */ OVERFLOW @@ -130,7 +130,7 @@ public BlockedValue() { * Check invariants of this and also the BlockedValue * containing this. */ - public boolean check(BlockedValue prev, int pos) { + public void check(BlockedValue prev, int pos) { assert state != State.ROOT || type == null; assert (state == State.ROOT || type == Schema.Type.ARRAY || type == Schema.Type.MAP); @@ -156,7 +156,6 @@ public boolean check(BlockedValue prev, int pos) { assert prev.state == State.ROOT || prev.state == State.OVERFLOW; break; } - return false; } } @@ -179,7 +178,7 @@ public boolean check(BlockedValue prev, int pos) { // buffer large enough for up to two ints for a block header // rounded up to a multiple of 4 bytes. - private byte[] headerBuffer = new byte[12]; + private final byte[] headerBuffer = new byte[12]; private boolean check() { assert buf != null; @@ -438,7 +437,7 @@ private void endBlockedValue() throws IOException { * Called when we've finished writing the last item in an overflow buffer. When * this is finished, the top of the stack will be an empty block in the * "regular" state. - * + * * @throws IOException */ private void finishOverflow() throws IOException { diff --git a/lang/java/avro/src/main/java/org/apache/avro/io/BlockingDirectBinaryEncoder.java b/lang/java/avro/src/main/java/org/apache/avro/io/BlockingDirectBinaryEncoder.java index 2ef2375e640..9f391a31921 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/io/BlockingDirectBinaryEncoder.java +++ b/lang/java/avro/src/main/java/org/apache/avro/io/BlockingDirectBinaryEncoder.java @@ -60,7 +60,7 @@ public class BlockingDirectBinaryEncoder extends DirectBinaryEncoder { * Create a writer that sends its output to the underlying stream * out. * - * @param out The Outputstream to write to + * @param out The OutputStream to write to */ public BlockingDirectBinaryEncoder(OutputStream out) { super(out); diff --git a/lang/java/avro/src/main/java/org/apache/avro/io/DirectBinaryDecoder.java b/lang/java/avro/src/main/java/org/apache/avro/io/DirectBinaryDecoder.java index 71f3ed593af..ac251550da2 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/io/DirectBinaryDecoder.java +++ b/lang/java/avro/src/main/java/org/apache/avro/io/DirectBinaryDecoder.java @@ -190,7 +190,7 @@ public InputStream inputStream() { } @Override - public boolean isEnd() throws IOException { + public boolean isEnd() { throw new UnsupportedOperationException(); } } diff --git a/lang/java/avro/src/main/java/org/apache/avro/io/EncoderFactory.java b/lang/java/avro/src/main/java/org/apache/avro/io/EncoderFactory.java index 2039f30097a..eaa83ba8baa 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/io/EncoderFactory.java +++ b/lang/java/avro/src/main/java/org/apache/avro/io/EncoderFactory.java @@ -166,7 +166,6 @@ public int getBlockSize() { * reuse is null, this will be a new instance. If reuse is * not null, then the returned instance may be a new instance or * reuse reconfigured to use out. - * @throws IOException * @see BufferedBinaryEncoder * @see Encoder */ @@ -287,7 +286,6 @@ public BinaryEncoder blockingDirectBinaryEncoder(OutputStream out, BinaryEncoder * reuse is null, this will be a new instance. If reuse is * not null, then the returned instance may be a new instance or * reuse reconfigured to use out. - * @throws IOException * @see BlockingBinaryEncoder * @see Encoder */ @@ -403,7 +401,7 @@ JsonEncoder jsonEncoder(Schema schema, JsonGenerator gen) throws IOException { * {@link ValidatingEncoder} is not thread-safe. * * @param schema The Schema to validate operations against. Cannot be null. - * @param encoder The Encoder to wrap. Cannot be be null. + * @param encoder The Encoder to wrap. Cannot be null. * @return A ValidatingEncoder configured to wrap encoder and validate * against schema * @throws IOException diff --git a/lang/java/avro/src/main/java/org/apache/avro/io/FastReaderBuilder.java b/lang/java/avro/src/main/java/org/apache/avro/io/FastReaderBuilder.java index f6e1ed5aae2..dfd4b4463a1 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/io/FastReaderBuilder.java +++ b/lang/java/avro/src/main/java/org/apache/avro/io/FastReaderBuilder.java @@ -277,7 +277,7 @@ private FieldReader getNonConvertedReader(Action action) throws IOException { throw new IllegalStateException("Error getting reader for action type " + action.getClass()); } case DO_NOTHING: - return getReaderForBaseType(action.reader, action.writer); + return getReaderForBaseType(action.reader); case RECORD: return createRecordReader((RecordAdjust) action); case ENUM: @@ -297,7 +297,7 @@ private FieldReader getNonConvertedReader(Action action) throws IOException { } } - private FieldReader getReaderForBaseType(Schema readerSchema, Schema writerSchema) throws IOException { + private FieldReader getReaderForBaseType(Schema readerSchema) { switch (readerSchema.getType()) { case NULL: return (old, decoder) -> { @@ -307,7 +307,7 @@ private FieldReader getReaderForBaseType(Schema readerSchema, Schema writerSchem case BOOLEAN: return (old, decoder) -> decoder.readBoolean(); case STRING: - return createStringReader(readerSchema, writerSchema); + return createStringReader(readerSchema); case INT: return (old, decoder) -> decoder.readInt(); case LONG: @@ -319,7 +319,7 @@ private FieldReader getReaderForBaseType(Schema readerSchema, Schema writerSchem case BYTES: return createBytesReader(); case FIXED: - return createFixedReader(readerSchema, writerSchema); + return createFixedReader(readerSchema); case RECORD: // covered by action type case UNION: // covered by action type case ENUM: // covered by action type @@ -330,7 +330,7 @@ private FieldReader getReaderForBaseType(Schema readerSchema, Schema writerSchem } } - private FieldReader createPromotingReader(Promote promote) throws IOException { + private FieldReader createPromotingReader(Promote promote) { switch (promote.reader.getType()) { case BYTES: return (reuse, decoder) -> ByteBuffer.wrap(decoder.readString(null).getBytes()); @@ -364,7 +364,7 @@ private FieldReader createPromotingReader(Promote promote) throws IOException { "No promotion possible for type " + promote.writer.getType() + " to " + promote.reader.getType()); } - private FieldReader createStringReader(Schema readerSchema, Schema writerSchema) { + private FieldReader createStringReader(Schema readerSchema) { FieldReader stringReader = createSimpleStringReader(readerSchema); if (isClassPropEnabled()) { return getTransformingStringReader(readerSchema.getProp(SpecificData.CLASS_PROP), stringReader); @@ -497,7 +497,7 @@ private FieldReader createEnumReader(EnumAdjust action) { }); } - private FieldReader createFixedReader(Schema readerSchema, Schema writerSchema) { + private FieldReader createFixedReader(Schema readerSchema) { return reusingReader((reuse, decoder) -> { GenericFixed fixed = (GenericFixed) data.createFixed(reuse, readerSchema); decoder.readFixed(fixed.bytes(), 0, readerSchema.getFixedSize()); @@ -516,9 +516,9 @@ public static FieldReader reusingReader(ReusingFieldReader reader) { public interface FieldReader extends DatumReader { @Override - public Object read(Object reuse, Decoder decoder) throws IOException; + Object read(Object reuse, Decoder decoder) throws IOException; - public default boolean canReuse() { + default boolean canReuse() { return false; } @@ -530,7 +530,7 @@ default void setSchema(Schema schema) { public interface ReusingFieldReader extends FieldReader { @Override - public default boolean canReuse() { + default boolean canReuse() { return true; } } @@ -608,7 +608,7 @@ public Object read(Object reuse, Decoder decoder) throws IOException { } public interface ExecutionStep { - public void execute(Object record, Decoder decoder) throws IOException; + void execute(Object record, Decoder decoder) throws IOException; } } diff --git a/lang/java/avro/src/main/java/org/apache/avro/io/JsonDecoder.java b/lang/java/avro/src/main/java/org/apache/avro/io/JsonDecoder.java index 2ad496a5b87..b4ebc050878 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/io/JsonDecoder.java +++ b/lang/java/avro/src/main/java/org/apache/avro/io/JsonDecoder.java @@ -47,7 +47,7 @@ */ public class JsonDecoder extends ParsingDecoder implements Parser.ActionHandler { private JsonParser in; - private static JsonFactory jsonFactory = new JsonFactory(); + private static final JsonFactory jsonFactory = new JsonFactory(); Stack reorderBuffers = new Stack<>(); ReorderBuffer currentReorderBuffer; @@ -268,8 +268,7 @@ public ByteBuffer readBytes(ByteBuffer old) throws IOException { } private byte[] readByteArray() throws IOException { - byte[] result = in.getText().getBytes(StandardCharsets.ISO_8859_1); - return result; + return in.getText().getBytes(StandardCharsets.ISO_8859_1); } @Override diff --git a/lang/java/avro/src/main/java/org/apache/avro/io/JsonEncoder.java b/lang/java/avro/src/main/java/org/apache/avro/io/JsonEncoder.java index 46fb6b5cb58..50f44c0afd4 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/io/JsonEncoder.java +++ b/lang/java/avro/src/main/java/org/apache/avro/io/JsonEncoder.java @@ -50,7 +50,7 @@ * JsonEncoder is not thread-safe. */ public class JsonEncoder extends ParsingEncoder implements Parser.ActionHandler { - private static final String LINE_SEPARATOR = System.getProperty("line.separator"); + private static final String LINE_SEPARATOR = System.lineSeparator(); final Parser parser; private JsonGenerator out; private boolean includeNamespace = true; @@ -88,7 +88,7 @@ public void flush() throws IOException { enum JsonOptions { Pretty, - // Prevent underlying outputstream to be flush for optimisation purpose. + // Prevent underlying OutputStream to be flush for optimisation purpose. NoFlushStream } @@ -98,7 +98,7 @@ private static JsonGenerator getJsonGenerator(OutputStream out, Set Objects.requireNonNull(out, "OutputStream cannot be null"); JsonGenerator g = new JsonFactory().createGenerator(out, JsonEncoding.UTF8); if (options.contains(JsonOptions.NoFlushStream)) { - g = g.configure(JsonGenerator.Feature.FLUSH_PASSED_TO_STREAM, false); + g.configure(JsonGenerator.Feature.FLUSH_PASSED_TO_STREAM, false); } final PrettyPrinter pp; if (options.contains(JsonOptions.Pretty)) { @@ -170,15 +170,13 @@ public JsonEncoder configure(OutputStream out, boolean autoflush) throws IOExcep * @param generator The JsonGenerator to direct output to. Cannot be null. * @throws IOException * @throws NullPointerException if {@code generator} is {@code null} - * @return this JsonEncoder */ - private JsonEncoder configure(JsonGenerator generator) throws IOException { + private void configure(JsonGenerator generator) throws IOException { Objects.requireNonNull(generator, "JsonGenerator cannot be null"); if (null != parser) { flush(); } this.out = generator; - return this; } @Override diff --git a/lang/java/avro/src/main/java/org/apache/avro/io/ValidatingEncoder.java b/lang/java/avro/src/main/java/org/apache/avro/io/ValidatingEncoder.java index d7440c7406e..d61967751a0 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/io/ValidatingEncoder.java +++ b/lang/java/avro/src/main/java/org/apache/avro/io/ValidatingEncoder.java @@ -36,7 +36,7 @@ * and configure. *

* ValidatingEncoder is not thread-safe. - * + * * @see Encoder * @see EncoderFactory */ @@ -44,12 +44,12 @@ public class ValidatingEncoder extends ParsingEncoder implements Parser.ActionHa protected Encoder out; protected final Parser parser; - ValidatingEncoder(Symbol root, Encoder out) throws IOException { + ValidatingEncoder(Symbol root, Encoder out) { this.out = out; this.parser = new Parser(root, this); } - ValidatingEncoder(Schema schema, Encoder in) throws IOException { + ValidatingEncoder(Schema schema, Encoder in) { this(new ValidatingGrammarGenerator().generate(schema), in); } @@ -60,7 +60,7 @@ public void flush() throws IOException { /** * Reconfigures this ValidatingEncoder to wrap the encoder provided. - * + * * @param encoder The Encoder to wrap for validation. * @return This ValidatingEncoder. */ diff --git a/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Symbol.java b/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Symbol.java index a18f3fdbcd5..a24240978b7 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Symbol.java +++ b/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Symbol.java @@ -94,7 +94,7 @@ static Symbol root(Symbol... symbols) { /** * A convenience method to construct a sequence. - * + * * @param production The constituent symbols of the sequence. */ static Symbol seq(Symbol... production) { @@ -103,7 +103,7 @@ static Symbol seq(Symbol... production) { /** * A convenience method to construct a repeater. - * + * * @param symsToRepeat The symbols to repeat in the repeater. */ static Symbol repeat(Symbol endSymbol, Symbol... symsToRepeat) { @@ -119,7 +119,7 @@ static Symbol alt(Symbol[] symbols, String[] labels) { /** * A convenience method to construct an ErrorAction. - * + * * @param e */ static Symbol error(String e) { @@ -128,7 +128,7 @@ static Symbol error(String e) { /** * A convenience method to construct a ResolvingAction. - * + * * @param w The writer symbol * @param r The reader symbol */ @@ -238,7 +238,7 @@ private static void copyFixups(List fixups, Symbol[] out, int outPos, Sym /** * Returns the amount of space required to flatten the given sub-array of * symbols. - * + * * @param symbols The array of input symbols. * @param start The index where the subarray starts. * @return The number of symbols that will be produced if one expands the given @@ -317,7 +317,7 @@ public int size() { @Override public Iterator iterator() { - return new Iterator() { + return new Iterator<>() { private int pos = production.length; @Override diff --git a/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ValidatingGrammarGenerator.java b/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ValidatingGrammarGenerator.java index 7798f520ae6..2f2e9cdc1c0 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ValidatingGrammarGenerator.java +++ b/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ValidatingGrammarGenerator.java @@ -41,7 +41,7 @@ public Symbol generate(Schema schema) { * given schema sc. If there is already an entry for the given schema * in the given map seen then that entry is returned. Otherwise a new * symbol is generated and an entry is inserted into the map. - * + * * @param sc The schema for which the start symbol is required * @param seen A map of schema to symbol mapping done so far. * @return The start symbol for the schema diff --git a/lang/java/avro/src/main/java/org/apache/avro/message/RawMessageDecoder.java b/lang/java/avro/src/main/java/org/apache/avro/message/RawMessageDecoder.java index ad2b1d31e49..917e5be88e3 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/message/RawMessageDecoder.java +++ b/lang/java/avro/src/main/java/org/apache/avro/message/RawMessageDecoder.java @@ -78,9 +78,7 @@ public RawMessageDecoder(GenericData model, Schema schema) { * @param writeSchema the {@link Schema} used to decode buffers */ public RawMessageDecoder(GenericData model, Schema writeSchema, Schema readSchema) { - Schema writeSchema1 = writeSchema; - Schema readSchema1 = readSchema; - this.reader = model.createDatumReader(writeSchema1, readSchema1); + this.reader = model.createDatumReader(writeSchema, readSchema); } @Override diff --git a/lang/java/avro/src/main/java/org/apache/avro/message/RawMessageEncoder.java b/lang/java/avro/src/main/java/org/apache/avro/message/RawMessageEncoder.java index 4df0d4c3683..230c6c1feab 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/message/RawMessageEncoder.java +++ b/lang/java/avro/src/main/java/org/apache/avro/message/RawMessageEncoder.java @@ -81,9 +81,8 @@ public RawMessageEncoder(GenericData model, Schema schema) { * @param shouldCopy whether to copy buffers before returning encoded results */ public RawMessageEncoder(GenericData model, Schema schema, boolean shouldCopy) { - Schema writeSchema = schema; this.copyOutputBytes = shouldCopy; - this.writer = model.createDatumWriter(writeSchema); + this.writer = model.createDatumWriter(schema); } @Override diff --git a/lang/java/avro/src/main/java/org/apache/avro/reflect/FieldAccessReflect.java b/lang/java/avro/src/main/java/org/apache/avro/reflect/FieldAccessReflect.java index 5d51be054be..df258f9d50d 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/reflect/FieldAccessReflect.java +++ b/lang/java/avro/src/main/java/org/apache/avro/reflect/FieldAccessReflect.java @@ -40,8 +40,8 @@ protected FieldAccessor getAccessor(Field field) { private static class ReflectionBasedAccessor extends FieldAccessor { protected final Field field; - private boolean isStringable; - private boolean isCustomEncoded; + private final boolean isStringable; + private final boolean isCustomEncoded; public ReflectionBasedAccessor(Field field) { this.field = field; @@ -105,7 +105,7 @@ protected boolean isCustomEncoded() { private static final class ReflectionBasesAccessorCustomEncoded extends ReflectionBasedAccessor { - private CustomEncoding encoding; + private final CustomEncoding encoding; public ReflectionBasesAccessorCustomEncoded(Field f, CustomEncoding encoding) { super(f); diff --git a/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectData.java b/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectData.java index ab702a60e87..d4a4bec30fe 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectData.java +++ b/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectData.java @@ -545,8 +545,7 @@ Schema createNonStringMapSchema(Type keyType, Type valueType, Map T load(String name, Class type) throws Exception { - return ReflectionUtil.class.getClassLoader().loadClass(name).asSubclass(type).getDeclaredConstructor() - .newInstance(); - } - public static FieldAccess getFieldAccess() { return fieldAccess; } @@ -79,8 +72,7 @@ private static boolean validate(FieldAccess access) throws Exception { } private static final class AccessorTestClass { - private boolean b = true; - protected byte by = 0xf; + private byte by = 0xf; public char c = 'c'; short s = 123; int i = 999; @@ -92,7 +84,7 @@ private static final class AccessorTestClass { private boolean validate(FieldAccess access) throws Exception { boolean valid = true; - valid &= validField(access, "b", b, false); + valid &= validField(access, "b", true, false); valid &= validField(access, "by", by, (byte) 0xaf); valid &= validField(access, "c", c, 'C'); valid &= validField(access, "s", s, (short) 321); @@ -149,30 +141,6 @@ private static Map, Type> resolveTypeVariables(Class iface, M return reuse; } - private static Supplier getConstructorAsSupplier(Class clazz) { - try { - MethodHandles.Lookup lookup = MethodHandles.lookup(); - MethodHandle constructorHandle = lookup.findConstructor(clazz, MethodType.methodType(void.class)); - - CallSite site = LambdaMetafactory.metafactory(lookup, "get", MethodType.methodType(Supplier.class), - constructorHandle.type().generic(), constructorHandle, constructorHandle.type()); - - return (Supplier) site.getTarget().invokeExact(); - } catch (Throwable t) { - // if anything goes wrong, don't provide a Supplier - return null; - } - } - - private static Supplier getOneArgConstructorAsSupplier(Class clazz, Class argumentClass, V argument) { - Function supplierFunction = getConstructorAsFunction(argumentClass, clazz); - if (supplierFunction != null) { - return () -> supplierFunction.apply(argument); - } else { - return null; - } - } - public static Function getConstructorAsFunction(Class parameterClass, Class clazz) { try { MethodHandles.Lookup lookup = MethodHandles.lookup(); diff --git a/lang/java/avro/src/main/java/org/apache/avro/specific/ExternalizableInput.java b/lang/java/avro/src/main/java/org/apache/avro/specific/ExternalizableInput.java index 503db7d16fe..8fa56ae1232 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/specific/ExternalizableInput.java +++ b/lang/java/avro/src/main/java/org/apache/avro/specific/ExternalizableInput.java @@ -42,11 +42,6 @@ public void close() throws IOException { in.close(); } - @Override - public boolean markSupported() { - return false; - } - @Override public int read() throws IOException { return in.read(); diff --git a/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java b/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java index c7b5eaed8a7..def8b02fccf 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java +++ b/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java @@ -619,7 +619,7 @@ public InstanceSupplier getNewRecordSupplier(Schema schema) { boolean useSchema = SchemaConstructable.class.isAssignableFrom(c); Constructor meth = CTOR_CACHE.apply(c); - Object[] params = useSchema ? new Object[] { schema } : (Object[]) null; + Object[] params = useSchema ? new Object[] { schema } : null; return (old, sch) -> { try { diff --git a/lang/java/avro/src/main/java/org/apache/avro/util/ByteBufferInputStream.java b/lang/java/avro/src/main/java/org/apache/avro/util/ByteBufferInputStream.java index f0ae5cc8a5e..6abb62015dc 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/util/ByteBufferInputStream.java +++ b/lang/java/avro/src/main/java/org/apache/avro/util/ByteBufferInputStream.java @@ -25,7 +25,7 @@ /** Utility to present {@link ByteBuffer} data as an {@link InputStream}. */ public class ByteBufferInputStream extends InputStream { - private List buffers; + private final List buffers; private int current; public ByteBufferInputStream(List buffers) { @@ -90,7 +90,7 @@ public ByteBuffer readBuffer(int length) throws IOException { /** * Returns the next non-empty buffer. */ - private ByteBuffer getBuffer() throws IOException { + private ByteBuffer getBuffer() { while (current < buffers.size()) { ByteBuffer buffer = buffers.get(current); if (buffer.hasRemaining())