You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by GitBox <gi...@apache.org> on 2019/01/16 02:53:41 UTC

[beam] Diff for: [GitHub] reuvenlax merged pull request #7500: [BEAM-4076] Add antlr4 to beam

diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
index dcc78a5416ab..eb8a57d639e5 100644
--- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
+++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
@@ -359,6 +359,8 @@ class BeamModulePlugin implements Plugin<Project> {
         activemq_junit                              : "org.apache.activemq.tooling:activemq-junit:5.13.1",
         activemq_kahadb_store                       : "org.apache.activemq:activemq-kahadb-store:5.13.1",
         activemq_mqtt                               : "org.apache.activemq:activemq-mqtt:5.13.1",
+        antlr                                       : "org.antlr:antlr4:4.7",
+        antlr_runtime                               : "org.antlr:antlr4-runtime:4.7",
         apex_common                                 : "org.apache.apex:apex-common:$apex_core_version",
         apex_engine                                 : "org.apache.apex:apex-engine:$apex_core_version",
         args4j                                      : "args4j:args4j:2.33",
@@ -626,7 +628,7 @@ class BeamModulePlugin implements Plugin<Project> {
           '-Xlint:all',
           '-Werror',
           '-XepDisableWarningsInGeneratedCode',
-          '-XepExcludedPaths:(.*/)?(build/generated.*avro-java|build/generated)/.*',
+          '-XepExcludedPaths:(.*/)?(build/generated-src|build/generated.*avro-java|build/generated)/.*',
           '-Xep:MutableConstantField:OFF' // Guava's immutable collections cannot appear on API surface.
         ]
         + (defaultLintSuppressions + configuration.disableLintWarnings).collect { "-Xlint:-${it}" })
@@ -1442,6 +1444,20 @@ class BeamModulePlugin implements Plugin<Project> {
     // or be left here.
     project.ext.applyAvroNature = { project.apply plugin: "com.commercehub.gradle.plugin.avro" }
 
+    project.ext.applyAntlrNature = {
+      project.apply plugin: 'antlr'
+      def generatedDir = "${project.buildDir}/generated/source-src/antlr/main/java/"
+      project.sourceSets {
+        generated { java.srcDir generatedDir }
+      }
+      project.idea {
+        module {
+          sourceDirs += project.file(generatedDir)
+          generatedSourceDirs += project.file(generatedDir)
+        }
+      }
+    }
+
     // Creates a task to run the quickstart for a runner.
     // Releases version and URL, can be overriden for a RC release with
     // ./gradlew :release:runJavaExamplesValidationTask -Pver=2.3.0 -Prepourl=https://repository.apache.org/content/repositories/orgapachebeam-1027
diff --git a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
index c8aa2f291719..545c2f8cc26c 100644
--- a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
+++ b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
@@ -410,6 +410,13 @@
     <Package name="org.apache.beam.sdk.extensions.sql.impl.parser.impl"/>
   </Match>
 
+  <Match>
+    <!--
+  Classes in this package is auto-generated, let's disable the findbugs for it.
+  -->
+    <Package name="org.apache.beam.sdk.schemas.parser.generated"/>
+  </Match>
+
   <Match>
     <!--
   Classes in this package is auto-generated, let's disable the findbugs for it.
diff --git a/sdks/java/build-tools/src/main/resources/beam/suppressions.xml b/sdks/java/build-tools/src/main/resources/beam/suppressions.xml
index e3ab90dff46a..d9cdc52b40da 100644
--- a/sdks/java/build-tools/src/main/resources/beam/suppressions.xml
+++ b/sdks/java/build-tools/src/main/resources/beam/suppressions.xml
@@ -27,6 +27,7 @@
 
   <!-- suppress all checks in the generated directories -->
   <suppress checks=".*" files=".+[\\\/]generated[\\\/].+\.java" />
+  <suppress checks=".*" files=".+[\\\/]generated-src[\\\/].+\.java" />
   <suppress checks=".*" files=".+[\\\/]generated-sources[\\\/].+\.java" />
   <suppress checks=".*" files=".+[\\\/]generated-test-sources[\\\/].+\.java" />
 </suppressions>
diff --git a/sdks/java/core/build.gradle b/sdks/java/core/build.gradle
index 689efea49ac2..c6d29fc615fd 100644
--- a/sdks/java/core/build.gradle
+++ b/sdks/java/core/build.gradle
@@ -22,13 +22,20 @@ applyJavaNature(shadowClosure: DEFAULT_SHADOW_CLOSURE << {
     include(dependency(library.java.protobuf_java))
     include(dependency(library.java.byte_buddy))
     include(dependency("org.apache.commons:.*"))
+    include(dependency(library.java.antlr_runtime))
   }
   relocate "com.google.thirdparty", getJavaRelocatedPath("com.google.thirdparty")
   relocate "com.google.protobuf", getJavaRelocatedPath("com.google.protobuf")
   relocate "net.bytebuddy", getJavaRelocatedPath("net.bytebuddy")
   relocate "org.apache.commons", getJavaRelocatedPath("org.apache.commons")
+  relocate "org.antlr.v4", getJavaRelocatedPath("org.antlr.v4")
 })
 applyAvroNature()
+applyAntlrNature()
+
+generateGrammarSource {
+  arguments += ["-visitor"]
+}
 
 description = "Apache Beam :: SDKs :: Java :: Core"
 ext.summary = """Beam SDK Java All provides a simple, Java-based
@@ -51,9 +58,11 @@ test {
 }
 
 dependencies {
+  antlr library.java.antlr
   // Required to load constants from the model, e.g. max timestamp for global window
   shadow project(path: ":beam-model-pipeline", configuration: "shadow")
   shadow library.java.vendored_guava_20_0
+  compile library.java.antlr_runtime
   compile library.java.protobuf_java
   compile library.java.byte_buddy
   compile library.java.commons_compress
diff --git a/sdks/java/core/src/main/antlr/java/org/apache/beam/sdk/schemas/parser/generated/FieldSpecifierNotation.g4 b/sdks/java/core/src/main/antlr/java/org/apache/beam/sdk/schemas/parser/generated/FieldSpecifierNotation.g4
new file mode 100644
index 000000000000..a869304bcf4f
--- /dev/null
+++ b/sdks/java/core/src/main/antlr/java/org/apache/beam/sdk/schemas/parser/generated/FieldSpecifierNotation.g4
@@ -0,0 +1,52 @@
+grammar FieldSpecifierNotation;
+
+@header {
+/*
+ * 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.sdk.schemas.parser.generated;
+}
+
+fieldSpecifier: dotExpression;
+
+dotExpression: dotExpressionComponent ('.' dotExpressionComponent)*;
+
+dotExpressionComponent: qualifiedComponent # qualifyComponent
+                      | IDENTIFIER  # simpleIdentifier
+                      | WILDCARD     # wildcard
+                      ;
+
+qualifiedComponent: IDENTIFIER qualifierList
+                  ;
+
+qualifierList: arrayQualifier (qualifierList)* # arrayQualifierList
+             | mapQualifier (qualifierList)*   # mapQualifierList
+             ;
+
+arrayQualifier: '[]'
+              | '[*]'
+              ;
+
+mapQualifier: '{}'
+            | '{*}'
+            ;
+
+IDENTIFIER: [a-zA-Z0-9]+ ;
+
+WILDCARD: '*';
+
+WS: [ \t\n\r]+ -> skip ;
\ No newline at end of file
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/parser/FieldAccessDescriptorParser.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/parser/FieldAccessDescriptorParser.java
new file mode 100644
index 000000000000..85d8dcf837c2
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/parser/FieldAccessDescriptorParser.java
@@ -0,0 +1,87 @@
+/*
+ * 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.sdk.schemas.parser;
+
+import javax.annotation.Nullable;
+import org.antlr.v4.runtime.CharStream;
+import org.antlr.v4.runtime.CharStreams;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.antlr.v4.runtime.TokenStream;
+import org.apache.beam.sdk.schemas.FieldAccessDescriptor;
+import org.apache.beam.sdk.schemas.parser.generated.FieldSpecifierNotationBaseVisitor;
+import org.apache.beam.sdk.schemas.parser.generated.FieldSpecifierNotationLexer;
+import org.apache.beam.sdk.schemas.parser.generated.FieldSpecifierNotationParser;
+import org.apache.beam.sdk.schemas.parser.generated.FieldSpecifierNotationParser.DotExpressionContext;
+import org.apache.beam.sdk.schemas.parser.generated.FieldSpecifierNotationParser.FieldSpecifierContext;
+import org.apache.beam.sdk.schemas.parser.generated.FieldSpecifierNotationParser.QualifiedComponentContext;
+import org.apache.beam.sdk.schemas.parser.generated.FieldSpecifierNotationParser.QualifyComponentContext;
+import org.apache.beam.sdk.schemas.parser.generated.FieldSpecifierNotationParser.SimpleIdentifierContext;
+import org.apache.beam.sdk.schemas.parser.generated.FieldSpecifierNotationParser.WildcardContext;
+
+/** Parse a textual representation of a field access descriptor. */
+public class FieldAccessDescriptorParser {
+  @Nullable
+  /** Parse the string. */
+  public static FieldAccessDescriptor parse(String expr) {
+    CharStream charStream = CharStreams.fromString(expr);
+    FieldSpecifierNotationLexer lexer = new FieldSpecifierNotationLexer(charStream);
+    TokenStream tokens = new CommonTokenStream(lexer);
+    FieldSpecifierNotationParser parser = new FieldSpecifierNotationParser(tokens);
+    return new BuildFieldAccessDescriptor().visit(parser.dotExpression());
+  }
+
+  private static class BuildFieldAccessDescriptor
+      extends FieldSpecifierNotationBaseVisitor<FieldAccessDescriptor> {
+
+    @Nullable
+    @Override
+    public FieldAccessDescriptor visitFieldSpecifier(FieldSpecifierContext ctx) {
+      return ctx.dotExpression().accept(this);
+    }
+
+    @Nullable
+    @Override
+    public FieldAccessDescriptor visitDotExpression(DotExpressionContext ctx) {
+      return null;
+    }
+
+    @Nullable
+    @Override
+    public FieldAccessDescriptor visitQualifyComponent(QualifyComponentContext ctx) {
+      return ctx.qualifiedComponent().accept(this);
+    }
+
+    @Nullable
+    @Override
+    public FieldAccessDescriptor visitSimpleIdentifier(SimpleIdentifierContext ctx) {
+      return null;
+    }
+
+    @Nullable
+    @Override
+    public FieldAccessDescriptor visitWildcard(WildcardContext ctx) {
+      return null;
+    }
+
+    @Nullable
+    @Override
+    public FieldAccessDescriptor visitQualifiedComponent(QualifiedComponentContext ctx) {
+      return null;
+    }
+  }
+}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/parser/generated/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/parser/generated/package-info.java
new file mode 100644
index 000000000000..f24eb8c8b0d7
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/parser/generated/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+/**
+ * Defines utilities for deailing with schemas.
+ *
+ * <p>For further details, see the documentation for each class in this package.
+ */
+@DefaultAnnotation(NonNull.class)
+package org.apache.beam.sdk.schemas.parser.generated;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/parser/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/parser/package-info.java
new file mode 100644
index 000000000000..0751713c77e9
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/parser/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+/**
+ * Defines utilities for deailing with schemas.
+ *
+ * <p>For further details, see the documentation for each class in this package.
+ */
+@DefaultAnnotation(NonNull.class)
+package org.apache.beam.sdk.schemas.parser;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
diff --git a/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java b/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java
deleted file mode 100644
index 834e41a50cf6..000000000000
--- a/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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;
-
-import static org.apache.beam.sdk.util.ApiSurface.containsOnlyPackages;
-import static org.hamcrest.MatcherAssert.assertThat;
-
-import java.io.IOException;
-import java.util.Set;
-import org.apache.beam.sdk.util.ApiSurface;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableSet;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** API surface verification for {@link org.apache.beam}. */
-@RunWith(JUnit4.class)
-public class SdkCoreApiSurfaceTest {
-
-  /**
-   * All classes transitively reachable via only public method signatures of the SDK.
-   *
-   * <p>Note that our idea of "public" does not include various internal-only APIs.
-   */
-  public static ApiSurface getSdkApiSurface(final ClassLoader classLoader) throws IOException {
-    return ApiSurface.ofPackage("org.apache.beam", classLoader)
-        .pruningPattern("org[.]apache[.]beam[.].*Test")
-        // Exposes Guava, but not intended for users
-        .pruningClassName("org.apache.beam.sdk.util.common.ReflectHelpers")
-        // test only
-        .pruningClassName("org.apache.beam.sdk.testing.InterceptingUrlClassLoader")
-        // test only
-        .pruningPrefix("org.apache.beam.model.")
-        .pruningPrefix("org.apache.beam.vendor.")
-        .pruningPrefix("java");
-  }
-
-  @Test
-  public void testSdkApiSurface() throws Exception {
-
-    @SuppressWarnings("unchecked")
-    final Set<String> allowed =
-        ImmutableSet.of(
-            "org.apache.beam",
-            "com.fasterxml.jackson.annotation",
-            "com.fasterxml.jackson.core",
-            "com.fasterxml.jackson.databind",
-            "org.apache.avro",
-            "org.hamcrest",
-            // via DataflowMatchers
-            "org.codehaus.jackson",
-            // via Avro
-            "org.joda.time",
-            "org.junit");
-
-    assertThat(getSdkApiSurface(getClass().getClassLoader()), containsOnlyPackages(allowed));
-  }
-}
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/FieldAccessDescriptorParserTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/FieldAccessDescriptorParserTest.java
new file mode 100644
index 000000000000..bcca3e88c0d6
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/FieldAccessDescriptorParserTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.sdk.schemas;
+
+import org.apache.beam.sdk.schemas.parser.FieldAccessDescriptorParser;
+import org.junit.Test;
+
+/** Tests for {@link FieldAccessDescriptorParser}. */
+public class FieldAccessDescriptorParserTest {
+  @Test
+  public void testSmoke() {
+    FieldAccessDescriptorParser.parse("a.b[][*].c.*");
+  }
+}


With regards,
Apache Git Services