Skip to content

Commit

Permalink
Improve test coverage of UTFString.write
Browse files Browse the repository at this point in the history
  • Loading branch information
Nathan Howell committed Dec 2, 2016
1 parent 5707218 commit 27c102d
Show file tree
Hide file tree
Showing 2 changed files with 122 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -148,11 +148,19 @@ public void writeTo(ByteBuffer buffer) {
}

public void writeTo(OutputStream out) throws IOException {
if (base instanceof byte[] &&
offset >= BYTE_ARRAY_OFFSET &&
offset < (long) Integer.MAX_VALUE) {
// the offset includes the object header... but it is only needed for unsafe copies
out.write((byte[]) base, (int) offset - BYTE_ARRAY_OFFSET, numBytes);
if (base instanceof byte[] && offset >= BYTE_ARRAY_OFFSET) {
final byte[] bytes = (byte[]) base;

// the offset includes an object header... this is only needed for unsafe copies
final long arrayOffset = offset - BYTE_ARRAY_OFFSET;

// verify that the offset and length points somewhere inside the byte array
// and that the offset can safely be truncated to a 32-bit integer
if ((long) bytes.length < arrayOffset + numBytes) {
throw new ArrayIndexOutOfBoundsException();
}

out.write(bytes, (int) arrayOffset, numBytes);
} else {
out.write(getBytes());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,15 +17,22 @@

package org.apache.spark.unsafe.types;

import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;

import com.google.common.collect.ImmutableMap;
import org.apache.spark.unsafe.Platform;
import org.junit.Test;

import static org.junit.Assert.*;

import static org.apache.spark.unsafe.Platform.BYTE_ARRAY_OFFSET;
import static org.apache.spark.unsafe.types.UTF8String.*;

public class UTF8StringSuite {
Expand Down Expand Up @@ -499,4 +506,106 @@ public void soundex() {
assertEquals(fromString("123").soundex(), fromString("123"));
assertEquals(fromString("世界千世").soundex(), fromString("世界千世"));
}

@Test
public void writeToOutputStreamUnderflow() throws IOException {
// offset underflow is apparently supported?
final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
final byte[] test = "01234567".getBytes(StandardCharsets.UTF_8);

for (int i = 1; i <= Platform.BYTE_ARRAY_OFFSET; ++i) {
UTF8String.fromAddress(test, Platform.BYTE_ARRAY_OFFSET - i, test.length + i)
.writeTo(outputStream);
final ByteBuffer buffer = ByteBuffer.wrap(outputStream.toByteArray(), i, test.length);
assertEquals("01234567", StandardCharsets.UTF_8.decode(buffer).toString());
outputStream.reset();
}
}

@Test
public void writeToOutputStreamSlice() throws IOException {
final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
final byte[] test = "01234567".getBytes(StandardCharsets.UTF_8);

for (int i = 0; i < test.length; ++i) {
for (int j = 0; j < test.length - i; ++j) {
UTF8String.fromAddress(test, Platform.BYTE_ARRAY_OFFSET + i, j)
.writeTo(outputStream);

assertArrayEquals(Arrays.copyOfRange(test, i, i + j), outputStream.toByteArray());
outputStream.reset();
}
}
}

@Test
public void writeToOutputStreamOverflow() throws IOException {
final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
final byte[] test = "01234567".getBytes(StandardCharsets.UTF_8);

final HashSet<Long> offsets = new HashSet<>();
for (int i = 0; i < 16; ++i) {
// touch more points around MAX_VALUE
offsets.add((long) Integer.MAX_VALUE - i);
// subtract off BYTE_ARRAY_OFFSET to avoid wrapping around to a negative value,
// which will hit the slower copy path instead of the optimized one
offsets.add(Long.MAX_VALUE - BYTE_ARRAY_OFFSET - i);
}

for (long i = 1; i > 0L; i <<= 1) {
for (long j = 0; j < 32L; ++j) {
offsets.add(i + j);
}
}

for (final long offset : offsets) {
try {
fromAddress(test, BYTE_ARRAY_OFFSET + offset, test.length)
.writeTo(outputStream);

throw new IllegalStateException(Long.toString(offset));
} catch (ArrayIndexOutOfBoundsException e) {
// ignore
} finally {
outputStream.reset();
}
}
}

@Test
public void writeToOutputStream() throws IOException {
final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
EMPTY_UTF8.writeTo(outputStream);
assertEquals("", outputStream.toString("UTF-8"));
outputStream.reset();

fromString("数据砖很重").writeTo(outputStream);
assertEquals(
"数据砖很重",
outputStream.toString("UTF-8"));
outputStream.reset();
}

@Test
public void writeToOutputStreamIntArray() throws IOException {
// verify that writes work on objects that are not byte arrays
final ByteBuffer buffer = StandardCharsets.UTF_8.encode("大千世界");
buffer.position(0);
buffer.order(ByteOrder.LITTLE_ENDIAN);

final int length = buffer.limit();
assertEquals(12, length);

final int ints = length / 4;
final int[] array = new int[ints];

for (int i = 0; i < ints; ++i) {
array[i] = buffer.getInt();
}

final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
fromAddress(array, Platform.INT_ARRAY_OFFSET, length)
.writeTo(outputStream);
assertEquals("大千世界", outputStream.toString("UTF-8"));
}
}

0 comments on commit 27c102d

Please sign in to comment.