Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Don't re-encode byte[] values in SortValues transform #31025

Open
wants to merge 3 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,9 @@
import java.io.IOException;
import java.util.Iterator;
import javax.annotation.Nonnull;
import org.apache.beam.sdk.coders.ByteArrayCoder;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.CoderException;
import org.apache.beam.sdk.coders.IterableCoder;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.transforms.DoFn;
Expand Down Expand Up @@ -154,11 +156,12 @@ public void processElement(ProcessContext c) {

try {
Sorter sorter = BufferedExternalSorter.create(sorterOptions);

final ThrowingFunction<KV<SecondaryKeyT, ValueT>, KV<byte[], byte[]>> toBytesKvFn =
new KvByteTranslator().toBytesKvFn;

for (KV<SecondaryKeyT, ValueT> record : records) {
sorter.add(
KV.of(
CoderUtils.encodeToByteArray(keyCoder, record.getKey()),
CoderUtils.encodeToByteArray(valueCoder, record.getValue())));
sorter.add(toBytesKvFn.apply(record));
}

c.output(KV.of(c.element().getKey(), new DecodingIterable(sorter.sort())));
Expand All @@ -167,6 +170,55 @@ public void processElement(ProcessContext c) {
}
}

@FunctionalInterface
private interface ThrowingFunction<FromT, ToT> {
ToT apply(FromT t) throws IOException, CoderException;
}

private class KvByteTranslator {
final ThrowingFunction<KV<SecondaryKeyT, ValueT>, KV<byte[], byte[]>> toBytesKvFn;
final ThrowingFunction<KV<byte[], byte[]>, KV<SecondaryKeyT, ValueT>> fromBytesKvFn;

KvByteTranslator() {
if (keyCoder instanceof ByteArrayCoder && valueCoder instanceof ByteArrayCoder) {
toBytesKvFn = (kv) -> (KV<byte[], byte[]>) kv;
fromBytesKvFn = (kv) -> (KV<SecondaryKeyT, ValueT>) kv;
} else if (keyCoder instanceof ByteArrayCoder) {
toBytesKvFn =
(kv) ->
KV.of(
(byte[]) kv.getKey(),
CoderUtils.encodeToByteArray(valueCoder, kv.getValue()));
fromBytesKvFn =
(kv) ->
KV.of(
(SecondaryKeyT) kv.getKey(),
CoderUtils.decodeFromByteArray(valueCoder, kv.getValue()));
} else if (valueCoder instanceof ByteArrayCoder) {
toBytesKvFn =
(kv) ->
KV.of(
CoderUtils.encodeToByteArray(keyCoder, kv.getKey()), (byte[]) kv.getValue());
fromBytesKvFn =
(kv) ->
KV.of(
CoderUtils.decodeFromByteArray(keyCoder, kv.getKey()),
(ValueT) kv.getValue());
} else {
toBytesKvFn =
(kv) ->
KV.of(
CoderUtils.encodeToByteArray(keyCoder, kv.getKey()),
CoderUtils.encodeToByteArray(valueCoder, kv.getValue()));
fromBytesKvFn =
(kv) ->
KV.of(
CoderUtils.decodeFromByteArray(keyCoder, kv.getKey()),
CoderUtils.decodeFromByteArray(valueCoder, kv.getValue()));
}
}
}

private class DecodingIterable implements Iterable<KV<SecondaryKeyT, ValueT>> {
final Iterable<KV<byte[], byte[]>> iterable;

Expand All @@ -183,9 +235,11 @@ public Iterator<KV<SecondaryKeyT, ValueT>> iterator() {

private class DecodingIterator implements Iterator<KV<SecondaryKeyT, ValueT>> {
final Iterator<KV<byte[], byte[]>> iterator;
final ThrowingFunction<KV<byte[], byte[]>, KV<SecondaryKeyT, ValueT>> fromBytesKvFn;

DecodingIterator(Iterator<KV<byte[], byte[]>> iterator) {
this.iterator = iterator;
this.fromBytesKvFn = new KvByteTranslator().fromBytesKvFn;
}

@Override
Expand All @@ -197,9 +251,7 @@ public boolean hasNext() {
public KV<SecondaryKeyT, ValueT> next() {
KV<byte[], byte[]> next = iterator.next();
try {
return KV.of(
CoderUtils.decodeFromByteArray(keyCoder, next.getKey()),
CoderUtils.decodeFromByteArray(valueCoder, next.getValue()));
return fromBytesKvFn.apply(next);
} catch (IOException e) {
throw new RuntimeException(e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,10 @@
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.is;

import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Create;
Expand Down Expand Up @@ -67,30 +70,141 @@ public void testSecondaryKeySorting() {
grouped.apply(SortValues.create(BufferedExternalSorter.options()));

PAssert.that(groupedAndSorted)
.satisfies(new AssertThatHasExpectedContentsForTestSecondaryKeySorting());
.satisfies(
new AssertThatHasExpectedContentsForTestSecondaryKeySorting<>(
Arrays.asList(
KV.of(
"key1",
Arrays.asList(
KV.of("secondaryKey1", 10),
KV.of("secondaryKey2", 20),
KV.of("secondaryKey3", 30))),
KV.of(
"key2",
Arrays.asList(KV.of("secondaryKey1", 100), KV.of("secondaryKey2", 200))))));

p.run();
}

@Test
public void testSecondaryKeyByteOptimization() {
PCollection<KV<String, KV<byte[], Integer>>> input =
p.apply(
Create.of(
Arrays.asList(
KV.of("key1", KV.of("secondaryKey2".getBytes(StandardCharsets.UTF_8), 20)),
KV.of("key2", KV.of("secondaryKey2".getBytes(StandardCharsets.UTF_8), 200)),
KV.of("key1", KV.of("secondaryKey3".getBytes(StandardCharsets.UTF_8), 30)),
KV.of("key1", KV.of("secondaryKey1".getBytes(StandardCharsets.UTF_8), 10)),
KV.of("key2", KV.of("secondaryKey1".getBytes(StandardCharsets.UTF_8), 100)))));

// Group by Key, bringing <SecondaryKey, Value> pairs for the same Key together.
PCollection<KV<String, Iterable<KV<byte[], Integer>>>> grouped =
input.apply(GroupByKey.create());

// For every Key, sort the iterable of <SecondaryKey, Value> pairs by SecondaryKey.
PCollection<KV<String, Iterable<KV<byte[], Integer>>>> groupedAndSorted =
grouped.apply(SortValues.create(BufferedExternalSorter.options()));

PAssert.that(groupedAndSorted)
.satisfies(
new AssertThatHasExpectedContentsForTestSecondaryKeySorting<>(
Arrays.asList(
KV.of(
"key1",
Arrays.asList(
KV.of("secondaryKey1".getBytes(StandardCharsets.UTF_8), 10),
KV.of("secondaryKey2".getBytes(StandardCharsets.UTF_8), 20),
KV.of("secondaryKey3".getBytes(StandardCharsets.UTF_8), 30))),
KV.of(
"key2",
Arrays.asList(
KV.of("secondaryKey1".getBytes(StandardCharsets.UTF_8), 100),
KV.of("secondaryKey2".getBytes(StandardCharsets.UTF_8), 200))))));

p.run();
}

@Test
public void testSecondaryKeyAndValueByteOptimization() {
PCollection<KV<String, KV<byte[], byte[]>>> input =
p.apply(
Create.of(
Arrays.asList(
KV.of(
"key1",
KV.of("secondaryKey2".getBytes(StandardCharsets.UTF_8), new byte[] {1})),
KV.of(
"key2",
KV.of("secondaryKey2".getBytes(StandardCharsets.UTF_8), new byte[] {2})),
KV.of(
"key1",
KV.of("secondaryKey3".getBytes(StandardCharsets.UTF_8), new byte[] {3})),
KV.of(
"key1",
KV.of("secondaryKey1".getBytes(StandardCharsets.UTF_8), new byte[] {4})),
KV.of(
"key2",
KV.of("secondaryKey1".getBytes(StandardCharsets.UTF_8), new byte[] {5})))));

// Group by Key, bringing <SecondaryKey, Value> pairs for the same Key together.
PCollection<KV<String, Iterable<KV<byte[], byte[]>>>> grouped =
input.apply(GroupByKey.create());

// For every Key, sort the iterable of <SecondaryKey, Value> pairs by SecondaryKey.
PCollection<KV<String, Iterable<KV<byte[], byte[]>>>> groupedAndSorted =
grouped.apply(SortValues.create(BufferedExternalSorter.options()));

PAssert.that(groupedAndSorted)
.satisfies(
new AssertThatHasExpectedContentsForTestSecondaryKeySorting<>(
Arrays.asList(
KV.of(
"key1",
Arrays.asList(
KV.of("secondaryKey1".getBytes(StandardCharsets.UTF_8), new byte[] {4}),
KV.of("secondaryKey2".getBytes(StandardCharsets.UTF_8), new byte[] {1}),
KV.of(
"secondaryKey3".getBytes(StandardCharsets.UTF_8), new byte[] {3}))),
KV.of(
"key2",
Arrays.asList(
KV.of("secondaryKey1".getBytes(StandardCharsets.UTF_8), new byte[] {5}),
KV.of(
"secondaryKey2".getBytes(StandardCharsets.UTF_8),
new byte[] {2}))))));

p.run();
}

static class AssertThatHasExpectedContentsForTestSecondaryKeySorting
implements SerializableFunction<Iterable<KV<String, Iterable<KV<String, Integer>>>>, Void> {
static class AssertThatHasExpectedContentsForTestSecondaryKeySorting<SecondaryKeyT, ValueT>
implements SerializableFunction<
Iterable<KV<String, Iterable<KV<SecondaryKeyT, ValueT>>>>, Void> {
final List<KV<String, List<KV<SecondaryKeyT, ValueT>>>> expected;

AssertThatHasExpectedContentsForTestSecondaryKeySorting(
List<KV<String, List<KV<SecondaryKeyT, ValueT>>>> expected) {
this.expected = expected;
}

@SuppressWarnings("unchecked")
@Override
public Void apply(Iterable<KV<String, Iterable<KV<String, Integer>>>> actual) {
public Void apply(Iterable<KV<String, Iterable<KV<SecondaryKeyT, ValueT>>>> actual) {
assertThat(
actual,
containsInAnyOrder(
KvMatcher.isKv(
is("key1"),
contains(
KvMatcher.isKv(is("secondaryKey1"), is(10)),
KvMatcher.isKv(is("secondaryKey2"), is(20)),
KvMatcher.isKv(is("secondaryKey3"), is(30)))),
KvMatcher.isKv(
is("key2"),
contains(
KvMatcher.isKv(is("secondaryKey1"), is(100)),
KvMatcher.isKv(is("secondaryKey2"), is(200))))));
expected.stream()
.map(
kv1 ->
KvMatcher.isKv(
is(kv1.getKey()),
contains(
kv1.getValue().stream()
.map(
kv2 ->
KvMatcher.isKv(is(kv2.getKey()), is(kv2.getValue())))
.collect(Collectors.toList()))))
.collect(Collectors.toList())));
return null;
}
}
Expand Down