Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 28647200CBE for ; Fri, 2 Jun 2017 00:56:53 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 272FE160BDF; Thu, 1 Jun 2017 22:56:53 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 471BB160BC4 for ; Fri, 2 Jun 2017 00:56:52 +0200 (CEST) Received: (qmail 78575 invoked by uid 500); 1 Jun 2017 22:56:51 -0000 Mailing-List: contact commits-help@beam.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@beam.apache.org Delivered-To: mailing list commits@beam.apache.org Received: (qmail 78562 invoked by uid 99); 1 Jun 2017 22:56:51 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 01 Jun 2017 22:56:51 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 62C37DFFB5; Thu, 1 Jun 2017 22:56:51 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: kenn@apache.org To: commits@beam.apache.org Date: Thu, 01 Jun 2017 22:56:51 -0000 Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: [1/2] beam git commit: Add GroupByKey translation archived-at: Thu, 01 Jun 2017 22:56:53 -0000 Repository: beam Updated Branches: refs/heads/master ca657c4f7 -> 217f085f2 Add GroupByKey translation Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/3dd2fb1e Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/3dd2fb1e Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/3dd2fb1e Branch: refs/heads/master Commit: 3dd2fb1e951bbfde8053bccef32aa73c51f9845d Parents: 82b81e5 Author: Kenneth Knowles Authored: Thu May 25 06:45:27 2017 -0700 Committer: Kenneth Knowles Committed: Thu Jun 1 12:39:39 2017 -0700 ---------------------------------------------------------------------- .../construction/GroupByKeyTranslation.java | 62 ++++++++++++++++++++ .../construction/GroupByKeyTranslationTest.java | 44 ++++++++++++++ 2 files changed, 106 insertions(+) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/3dd2fb1e/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupByKeyTranslation.java ---------------------------------------------------------------------- diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupByKeyTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupByKeyTranslation.java new file mode 100644 index 0000000..db73461 --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupByKeyTranslation.java @@ -0,0 +1,62 @@ +/* + * 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. + */ + +package org.apache.beam.runners.core.construction; + +import com.google.auto.service.AutoService; +import java.util.Collections; +import java.util.Map; +import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.common.runner.v1.RunnerApi; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.PTransform; + +/** + * Utility methods for translating a {@link GroupByKey} to and from {@link RunnerApi} + * representations. + */ +public class GroupByKeyTranslation { + + static class GroupByKeyTranslator implements TransformPayloadTranslator> { + @Override + public String getUrn(GroupByKey transform) { + return PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN; + } + + @Override + public FunctionSpec translate( + AppliedPTransform> transform, SdkComponents components) { + return FunctionSpec.newBuilder() + .setUrn(getUrn(transform.getTransform())) + .build(); + } + } + + + /** Registers {@link GroupByKeyTranslator}. */ + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class Registrar implements TransformPayloadTranslatorRegistrar { + @Override + public Map, ? extends TransformPayloadTranslator> + getTransformPayloadTranslators() { + return Collections.singletonMap(GroupByKey.class, new GroupByKeyTranslator()); + } + } +} http://git-wip-us.apache.org/repos/asf/beam/blob/3dd2fb1e/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/GroupByKeyTranslationTest.java ---------------------------------------------------------------------- diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/GroupByKeyTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/GroupByKeyTranslationTest.java new file mode 100644 index 0000000..22681f7 --- /dev/null +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/GroupByKeyTranslationTest.java @@ -0,0 +1,44 @@ +/* + * 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. + */ + +package org.apache.beam.runners.core.construction; + +import static org.apache.beam.runners.core.construction.PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; + +import org.apache.beam.sdk.transforms.GroupByKey; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link GroupByKeyTranslation}. */ +@RunWith(JUnit4.class) +public class GroupByKeyTranslationTest { + + /** + * Tests that the translator is registered so the URN can be retrieved (the only thing you can + * meaningfully do with a {@link GroupByKey}). + */ + @Test + public void testUrnRetrievable() throws Exception { + assertThat( + PTransformTranslation.urnForTransform(GroupByKey.create()), + equalTo(GROUP_BY_KEY_TRANSFORM_URN)); + } +}