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
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
package org.apache.kafka.common.compress;

import org.apache.kafka.common.record.internal.CompressionType;
import org.apache.kafka.common.utils.ByteUtils;
import org.apache.kafka.common.utils.internals.ByteUtils;

import net.jpountz.lz4.LZ4Compressor;
import net.jpountz.lz4.LZ4Factory;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,9 @@
import org.apache.kafka.common.errors.SslAuthenticationException;
import org.apache.kafka.common.security.auth.KafkaPrincipal;
import org.apache.kafka.common.utils.ByteBufferUnmapper;
import org.apache.kafka.common.utils.ByteUtils;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.common.utils.internals.ByteUtils;

import org.slf4j.Logger;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.kafka.common.protocol;

import org.apache.kafka.common.utils.ByteUtils;
import org.apache.kafka.common.utils.internals.ByteUtils;

import java.nio.ByteBuffer;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,8 @@

package org.apache.kafka.common.protocol;

import org.apache.kafka.common.utils.ByteUtils;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.common.utils.internals.ByteUtils;

import java.io.Closeable;
import java.io.DataOutputStream;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
import org.apache.kafka.common.record.internal.UnalignedMemoryRecords;
import org.apache.kafka.common.requests.RequestHeader;
import org.apache.kafka.common.requests.ResponseHeader;
import org.apache.kafka.common.utils.ByteUtils;
import org.apache.kafka.common.utils.internals.ByteUtils;

import java.nio.ByteBuffer;
import java.util.ArrayDeque;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
package org.apache.kafka.common.protocol.types;

import org.apache.kafka.common.protocol.types.Type.DocumentedType;
import org.apache.kafka.common.utils.ByteUtils;
import org.apache.kafka.common.utils.internals.ByteUtils;

import java.nio.ByteBuffer;
import java.util.Optional;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
package org.apache.kafka.common.protocol.types;

import org.apache.kafka.common.protocol.types.Type.DocumentedType;
import org.apache.kafka.common.utils.ByteUtils;
import org.apache.kafka.common.utils.internals.ByteUtils;

import java.nio.ByteBuffer;
import java.util.Collections;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.record.internal.BaseRecords;
import org.apache.kafka.common.record.internal.MemoryRecords;
import org.apache.kafka.common.utils.ByteUtils;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.common.utils.internals.ByteUtils;

import java.nio.ByteBuffer;
import java.util.Optional;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,9 @@
import org.apache.kafka.common.utils.AbstractIterator;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.apache.kafka.common.utils.ByteUtils;
import org.apache.kafka.common.utils.CloseableIterator;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.common.utils.internals.ByteUtils;

import java.io.DataOutputStream;
import java.io.IOException;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,8 @@
import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.header.internals.RecordHeader;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.utils.ByteUtils;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.common.utils.internals.ByteUtils;

import java.io.DataOutputStream;
import java.io.IOException;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,9 @@
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.apache.kafka.common.utils.ByteUtils;
import org.apache.kafka.common.utils.CloseableIterator;
import org.apache.kafka.common.utils.Crc32C;
import org.apache.kafka.common.utils.internals.ByteUtils;

import java.io.IOException;
import java.io.InputStream;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,9 @@
import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.apache.kafka.common.utils.ByteUtils;
import org.apache.kafka.common.utils.Checksums;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.common.utils.internals.ByteUtils;

import java.io.DataOutputStream;
import java.io.IOException;
Expand Down
14 changes: 7 additions & 7 deletions clients/src/main/java/org/apache/kafka/common/utils/Bytes.java
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
*/
package org.apache.kafka.common.utils;

import org.apache.kafka.common.utils.internals.BytesUtils;
import org.apache.kafka.common.utils.internals.ByteUtils;

import java.io.Serializable;
import java.util.Arrays;
Expand Down Expand Up @@ -113,7 +113,7 @@ public boolean equals(Object other) {

@Override
public int compareTo(Bytes that) {
return BytesUtils.BYTES_LEXICO_COMPARATOR.compare(this.bytes, that.bytes);
return ByteUtils.BYTES_LEXICO_COMPARATOR.compare(this.bytes, that.bytes);
}

@Override
Expand Down Expand Up @@ -166,17 +166,17 @@ private static String toString(final byte[] b, int off, int len) {
* @return A new copy of the incremented byte array.
* @throws IndexOutOfBoundsException if incrementing causes the underlying input byte array to overflow.
* @deprecated This method is not part of the public API and will be removed in version 5.0.
* Internal Kafka code should use {@link org.apache.kafka.common.utils.internals.BytesUtils#increment(Bytes)} instead.
* Internal Kafka code should use {@link org.apache.kafka.common.utils.internals.ByteUtils#increment(Bytes)} instead.
*/
@Deprecated(since = "4.3", forRemoval = true)
public static Bytes increment(Bytes input) throws IndexOutOfBoundsException {
return BytesUtils.increment(input);
return ByteUtils.increment(input);
}

/**
* A byte array comparator based on lexicographic ordering.
* @deprecated This field is not part of the public API and will be removed in version 5.0.
* Internal Kafka code should use {@link org.apache.kafka.common.utils.internals.BytesUtils#BYTES_LEXICO_COMPARATOR} instead.
* Internal Kafka code should use {@link org.apache.kafka.common.utils.internals.ByteUtils#BYTES_LEXICO_COMPARATOR} instead.
*/
@Deprecated(since = "4.3", forRemoval = true)
public static final ByteArrayComparator BYTES_LEXICO_COMPARATOR = new LexicographicByteArrayComparator();
Expand All @@ -185,7 +185,7 @@ public static Bytes increment(Bytes input) throws IndexOutOfBoundsException {
* A byte array comparator interface.
*
* @deprecated This interface is not part of the public API and will be removed in version 5.0.
* Internal Kafka code should use {@link org.apache.kafka.common.utils.internals.BytesUtils.ByteArrayComparator} instead.
* Internal Kafka code should use {@link org.apache.kafka.common.utils.internals.ByteUtils.ByteArrayComparator} instead.
*/
@Deprecated(since = "4.3", forRemoval = true)
public interface ByteArrayComparator extends Comparator<byte[]>, Serializable {
Expand All @@ -194,6 +194,6 @@ int compare(final byte[] buffer1, int offset1, int length1,
final byte[] buffer2, int offset2, int length2);
}

private static class LexicographicByteArrayComparator extends BytesUtils.LexicographicByteArrayComparator implements ByteArrayComparator {
private static class LexicographicByteArrayComparator extends ByteUtils.LexicographicByteArrayComparator implements ByteArrayComparator {
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.common.utils;
package org.apache.kafka.common.utils.internals;

import org.apache.kafka.common.utils.Bytes;

import java.io.DataInput;
import java.io.DataOutput;
Expand All @@ -23,6 +25,8 @@
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Comparator;

/**
* This classes exposes low-level methods for reading/writing from byte streams or buffers.
Expand All @@ -36,6 +40,66 @@ public final class ByteUtils {

private ByteUtils() {}

/**
* Increment the underlying byte array by adding 1.
*
* @param input - The byte array to increment
* @return A new copy of the incremented byte array
* @throws IndexOutOfBoundsException if incrementing causes the underlying input byte array to overflow
*/
public static Bytes increment(Bytes input) throws IndexOutOfBoundsException {
byte[] inputArr = input.get();
byte[] ret = new byte[inputArr.length];
int carry = 1;
for (int i = inputArr.length - 1; i >= 0; i--) {
if (inputArr[i] == (byte) 0xFF && carry == 1) {
ret[i] = (byte) 0x00;
} else {
ret[i] = (byte) (inputArr[i] + carry);
carry = 0;
}
}
if (carry == 0) {
return Bytes.wrap(ret);
} else {
throw new IndexOutOfBoundsException();
}
}

/**
* A byte array comparator based on lexicographic ordering.
*/
public static final ByteArrayComparator BYTES_LEXICO_COMPARATOR = new LexicographicByteArrayComparator();

public interface ByteArrayComparator extends Comparator<byte[]> {

int compare(final byte[] buffer1, int offset1, int length1,
final byte[] buffer2, int offset2, int length2);
}

public static class LexicographicByteArrayComparator implements ByteArrayComparator {

@Override
public int compare(byte[] buffer1, byte[] buffer2) {
return compare(buffer1, 0, buffer1.length, buffer2, 0, buffer2.length);
}

public int compare(final byte[] buffer1, int offset1, int length1,
final byte[] buffer2, int offset2, int length2) {

// short circuit equal case
if (buffer1 == buffer2 &&
offset1 == offset2 &&
length1 == length2) {
return 0;
}

int end1 = offset1 + length1;
int end2 = offset2 + length2;
return Arrays.compareUnsigned(buffer1, offset1, end1, buffer2, offset2, end2);
}
}

/**
* Read an unsigned integer from the current position in the buffer, incrementing the position by 4 bytes
*
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

package org.apache.kafka.common.utils;
package org.apache.kafka.common.utils.internals;

import java.util.AbstractCollection;
import java.util.AbstractSequentialList;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

package org.apache.kafka.common.utils;
package org.apache.kafka.common.utils.internals;

import java.util.ArrayList;
import java.util.Collections;
Expand All @@ -24,7 +24,7 @@

/**
* A memory-efficient hash multiset which tracks the order of insertion of elements.
* See org.apache.kafka.common.utils.ImplicitLinkedHashCollection for implementation details.
* See org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollection for implementation details.
*
* This class is a multi-set because it allows multiple elements to be inserted that
* have equivalent keys.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.MessageUtil;
import org.apache.kafka.common.protocol.ObjectSerializationCache;
import org.apache.kafka.common.utils.ByteUtils;
import org.apache.kafka.common.utils.internals.ByteUtils;

import com.fasterxml.jackson.databind.JsonNode;

Expand Down
Loading
Loading