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 runners/java-fn-execution/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ dependencies {
shadow library.java.grpc_netty
shadow library.java.slf4j_api
testCompile project(":beam-sdks-parent:beam-sdks-java-parent:beam-sdks-java-fn-execution").sourceSets.test.output
testCompile project(path: ":beam-runners-parent:beam-runners-core-construction-java", configuration: "shadow")
testCompile library.java.junit
testCompile library.java.hamcrest_core
testCompile library.java.hamcrest_library
Expand Down
6 changes: 6 additions & 0 deletions runners/java-fn-execution/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -97,6 +97,12 @@
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.beam</groupId>
<artifactId>beam-runners-core-construction-java</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.beam</groupId>
<artifactId>beam-sdks-java-fn-execution</artifactId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,183 @@
/*
* Copyright (C) 2017 Google Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not
* use this file except in compliance with the License. You may obtain a copy of
* the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/

package org.apache.beam.runners.fnexecution.graph;

import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
import java.util.Set;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.model.pipeline.v1.RunnerApi.Coder;
import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
import org.apache.beam.model.pipeline.v1.RunnerApi.MessageWithComponents;
import org.apache.beam.sdk.coders.ByteArrayCoder;
import org.apache.beam.sdk.coders.LengthPrefixCoder;

/**
* Utilities for replacing or wrapping unknown coders with {@link LengthPrefixCoder}.
*
* <p>TODO: Support a dynamic list of well known coders using either registration or manual listing.
*/
public class LengthPrefixUnknownCoders {
private static final String BYTES_CODER_TYPE = "urn:beam:coders:bytes:0.1";
private static final String LENGTH_PREFIX_CODER_TYPE = "urn:beam:coders:length_prefix:0.1";
private static final Set<String> WELL_KNOWN_CODER_URNS =
ImmutableSet.of(
BYTES_CODER_TYPE,
"urn:beam:coders:kv:0.1",
"urn:beam:coders:varint:0.1",
"urn:beam:coders:interval_window:0.1",
"urn:beam:coders:stream:0.1",
LENGTH_PREFIX_CODER_TYPE,
"urn:beam:coders:global_window:0.1",
"urn:beam:coders:windowed_value:0.1");

/**
* Recursively traverse the coder tree and wrap the first unknown coder in every branch with a
* {@link LengthPrefixCoder} unless an ancestor coder is itself a {@link LengthPrefixCoder}. If
* {@code replaceWithByteArrayCoder} is set, then replace that unknown coder with a
* {@link ByteArrayCoder}. Note that no ids that are generated will collide with the ids supplied
* within the {@link Components#getCodersMap() coder map} key space.
*
* @param coderId The root coder contained within {@code coders} to start the recursive descent
* from.
* @param components Contains the root coder and all component coders.
* @param replaceWithByteArrayCoder whether to replace an unknown coder with a
* {@link ByteArrayCoder}.
* @return A {@link MessageWithComponents} with the
* {@link MessageWithComponents#getCoder() root coder} and its component coders. Note that no ids
* that are generated will collide with the ids supplied within the
* {@link Components#getCodersMap() coder map} key space.
*/
public static RunnerApi.MessageWithComponents forCoder(
String coderId,
RunnerApi.Components components,
boolean replaceWithByteArrayCoder) {

RunnerApi.Coder currentCoder = components.getCodersOrThrow(coderId);

// We handle three cases:
// 1) the requested coder is already a length prefix coder. In this case we just honor the
// request to replace the coder with a byte array coder.
// 2) the requested coder is a known coder but not a length prefix coder. In this case we
// rebuild the coder by recursively length prefixing any unknown component coders.
// 3) the requested coder is an unknown coder. In this case we either wrap the requested coder
// with a length prefix coder or replace it with a length prefix byte array coder.
if (LENGTH_PREFIX_CODER_TYPE.equals(currentCoder.getSpec().getSpec().getUrn())) {
if (replaceWithByteArrayCoder) {
return createLengthPrefixByteArrayCoder(coderId, components);
}

MessageWithComponents.Builder rvalBuilder = MessageWithComponents.newBuilder();
rvalBuilder.setCoder(currentCoder);
rvalBuilder.setComponents(components);
return rvalBuilder.build();
} else if (WELL_KNOWN_CODER_URNS.contains(currentCoder.getSpec().getSpec().getUrn())) {
return lengthPrefixUnknownComponentCoders(coderId, components, replaceWithByteArrayCoder);
} else {
return lengthPrefixUnknownCoder(coderId, components, replaceWithByteArrayCoder);
}
}

private static MessageWithComponents lengthPrefixUnknownComponentCoders(
String coderId,
RunnerApi.Components components,
boolean replaceWithByteArrayCoder) {

MessageWithComponents.Builder rvalBuilder = MessageWithComponents.newBuilder();
RunnerApi.Coder currentCoder = components.getCodersOrThrow(coderId);
RunnerApi.Coder.Builder updatedCoder = currentCoder.toBuilder();
// Rebuild the component coder ids to handle if any of the component coders changed.
updatedCoder.clearComponentCoderIds();
for (final String componentCoderId : currentCoder.getComponentCoderIdsList()) {
MessageWithComponents componentCoder =
forCoder(componentCoderId, components, replaceWithByteArrayCoder);
String newComponentCoderId = componentCoderId;
if (!components.getCodersOrThrow(componentCoderId).equals(componentCoder.getCoder())) {
// Generate a new id if the component coder changed.
newComponentCoderId = generateUniqueId(
coderId + "-length_prefix",
Sets.union(components.getCodersMap().keySet(),
rvalBuilder.getComponents().getCodersMap().keySet()));
}
updatedCoder.addComponentCoderIds(newComponentCoderId);
rvalBuilder.getComponentsBuilder().putCoders(newComponentCoderId, componentCoder.getCoder());
// Insert all component coders of the component coder.
rvalBuilder.getComponentsBuilder().putAllCoders(
componentCoder.getComponents().getCodersMap());
}
rvalBuilder.setCoder(updatedCoder);

return rvalBuilder.build();
}

// If we are handling an unknown URN then we need to wrap it with a length prefix coder.
// If requested we also replace the unknown coder with a byte array coder.
private static MessageWithComponents lengthPrefixUnknownCoder(
String coderId,
RunnerApi.Components components,
boolean replaceWithByteArrayCoder) {
MessageWithComponents.Builder rvalBuilder = MessageWithComponents.newBuilder();
RunnerApi.Coder currentCoder = components.getCodersOrThrow(coderId);

String lengthPrefixComponentCoderId = coderId;
if (replaceWithByteArrayCoder) {
return createLengthPrefixByteArrayCoder(coderId, components);
} else {
rvalBuilder.getComponentsBuilder().putCoders(coderId, currentCoder);
}

rvalBuilder.getCoderBuilder()
.addComponentCoderIds(lengthPrefixComponentCoderId)
.getSpecBuilder()
.getSpecBuilder()
.setUrn(LENGTH_PREFIX_CODER_TYPE);
return rvalBuilder.build();
}

private static MessageWithComponents createLengthPrefixByteArrayCoder(
String coderId,
RunnerApi.Components components) {
MessageWithComponents.Builder rvalBuilder = MessageWithComponents.newBuilder();

String byteArrayCoderId = generateUniqueId(
coderId + "-byte_array",
Sets.union(components.getCodersMap().keySet(),
rvalBuilder.getComponents().getCodersMap().keySet()));
Coder.Builder byteArrayCoder = Coder.newBuilder();
byteArrayCoder.getSpecBuilder().getSpecBuilder().setUrn(BYTES_CODER_TYPE);
rvalBuilder.getComponentsBuilder().putCoders(byteArrayCoderId,
byteArrayCoder.build());
rvalBuilder.getCoderBuilder()
.addComponentCoderIds(byteArrayCoderId)
.getSpecBuilder()
.getSpecBuilder()
.setUrn(LENGTH_PREFIX_CODER_TYPE);

return rvalBuilder.build();
}

/**
* Generates a unique id given a prefix and the set of existing ids.
*/
static String generateUniqueId(String prefix, Set<String> existingIds) {
int i = 0;
while (existingIds.contains(prefix + i)) {
i += 1;
}
return prefix + i;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

/**
* Utilities for a Beam runner to interact with the pipeline proto representation.
*/
package org.apache.beam.runners.fnexecution.graph;
Original file line number Diff line number Diff line change
@@ -0,0 +1,147 @@
/*
* Copyright (C) 2017 Google Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not
* use this file except in compliance with the License. You may obtain a copy of
* the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.beam.runners.fnexecution.graph;

import static org.junit.Assert.assertEquals;

import com.google.common.collect.ImmutableList;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.Collection;
import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
import org.apache.beam.model.pipeline.v1.RunnerApi.MessageWithComponents;
import org.apache.beam.runners.core.construction.CoderTranslation;
import org.apache.beam.runners.core.construction.RehydratedComponents;
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.CustomCoder;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.coders.LengthPrefixCoder;
import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
import org.apache.beam.sdk.util.WindowedValue;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameter;
import org.junit.runners.Parameterized.Parameters;

/** Tests for {@link LengthPrefixUnknownCoders}. */
@RunWith(Parameterized.class)
public class LengthPrefixUnknownCodersTest {

private static class UnknownCoder extends CustomCoder<String> {
private static final Coder<?> INSTANCE = new UnknownCoder();
@Override
public void encode(String value, OutputStream outStream) throws CoderException, IOException {
}

@Override
public String decode(InputStream inStream) throws CoderException, IOException {
return "";
}

@Override
public int hashCode() {
return 1278890232;
}

@Override
public boolean equals(Object obj) {
return obj instanceof UnknownCoder;
}
}

@Parameters
public static Collection<Object[]> data() {
return ImmutableList.of(
/** Test wrapping unknown coders with {@code LengthPrefixCoder}. */
new Object[] {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I prefer the use of an AutoValue inner class here instead of an Object[]

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm going to stick with the Object[] for now since it will make the migration to JUnit 5 parameterized tests simpler.

WindowedValue.getFullCoder(
KvCoder.of(UnknownCoder.INSTANCE, UnknownCoder.INSTANCE),
GlobalWindow.Coder.INSTANCE),
WindowedValue.getFullCoder(
KvCoder.of(LengthPrefixCoder.of(UnknownCoder.INSTANCE),
LengthPrefixCoder.of(UnknownCoder.INSTANCE)),
GlobalWindow.Coder.INSTANCE),
false
},
/**
* Test bypassing unknown coders that are already wrapped with
* {@code LengthPrefixCoder}.
*/
new Object[] {
WindowedValue.getFullCoder(
KvCoder.of(UnknownCoder.INSTANCE,
LengthPrefixCoder.of(UnknownCoder.INSTANCE)),
GlobalWindow.Coder.INSTANCE),
WindowedValue.getFullCoder(
KvCoder.of(LengthPrefixCoder.of(UnknownCoder.INSTANCE),
LengthPrefixCoder.of(UnknownCoder.INSTANCE)),
GlobalWindow.Coder.INSTANCE),
false
},
/** Test replacing unknown coders with {@code LengthPrefixCoder<ByteArray>}. */
new Object[] {
WindowedValue.getFullCoder(
KvCoder.of(LengthPrefixCoder.of(UnknownCoder.INSTANCE),
UnknownCoder.INSTANCE),
GlobalWindow.Coder.INSTANCE),
WindowedValue.getFullCoder(
KvCoder.of(LengthPrefixCoder.of(ByteArrayCoder.of()),
LengthPrefixCoder.of(ByteArrayCoder.of())),
GlobalWindow.Coder.INSTANCE),
true
},
/** Test skipping a top level length prefix coder. */
new Object[] {
LengthPrefixCoder.of(UnknownCoder.INSTANCE),
LengthPrefixCoder.of(UnknownCoder.INSTANCE),
false
},
/** Test replacing a top level length prefix coder with byte array coder. */
new Object[] {
LengthPrefixCoder.of(UnknownCoder.INSTANCE),
LengthPrefixCoder.of(ByteArrayCoder.of()),
true
}
);
}

@Parameter
public Coder<?> original;

@Parameter(1)
public Coder<?> expected;

@Parameter(2)
public boolean replaceWithByteArray;

@Test
public void test() throws IOException {
MessageWithComponents originalCoderProto = CoderTranslation.toProto(original);
Components.Builder builder = originalCoderProto.getComponents().toBuilder();
String coderId = LengthPrefixUnknownCoders.generateUniqueId("rootTestId",
originalCoderProto.getComponents().getCodersMap().keySet());
builder.putCoders(coderId, originalCoderProto.getCoder());
MessageWithComponents updatedCoderProto = LengthPrefixUnknownCoders.forCoder(
coderId, builder.build(), replaceWithByteArray);
assertEquals(expected,
CoderTranslation.fromProto(updatedCoderProto.getCoder(),
RehydratedComponents.forComponents(updatedCoderProto.getComponents())));
}
}