diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 90f8a4c382bcb..be1a00e2f56ed 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -11,368 +11,7 @@ Thank you for your contribution! Follow this checklist to help us incorporate yo See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier). -`ValidatesRunner` compliance status (on master branch) --------------------------------------------------------- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
LangULRDataflowFlinkSamzaSparkTwister2
Go--- - - Build Status - - - - Build Status - - - - Build Status - - - - Build Status - - ---
Java - - Build Status - - - - Build Status -
- - Build Status -
- - Build Status -
- - Build Status -
- - Build Status -
-
- - Build Status -
- - Build Status -
- - Build Status -
- - Build Status - -
- - Build Status -
- - Build Status - -
- - Build Status -
- - Build Status -
- - Build Status - -
- - Build Status - -
Python--- - - Build Status -
- - Build Status -
- - Build Status - -
- - Build Status -
- - Build Status - -
- - Build Status - - - - Build Status - - ---
XLang - - Build Status - - - - Build Status -
- - Build Status -
- - Build Status -
-
- - Build Status - - - - Build Status - - - - Build Status - - ---
- -Examples testing status on various runners --------------------------------------------------------- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
LangULRDataflowFlinkSamzaSparkTwister2
Go---------------------
Java--- - - Build Status -
- - Build Status -
- - Build Status -
-
---------------
Python---------------------
XLang---------------------
- -Post-Commit SDK/Transform Integration Tests Status (on master branch) ------------------------------------------------------------------------------------------------- - - - - - - - - - - - - - - - - -
GoJavaPython
- - Build Status - - - - Build Status - - - - Build Status -
- - Build Status -
- - Build Status - -
- -Pre-Commit Tests Status (on master branch) ------------------------------------------------------------------------------------------------- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
---JavaPythonGoWebsiteWhitespaceTypescript
Non-portable - - Build Status -
-
- - Build Status -
- - Build Status -
- - Build Status -
- - Build Status - -
- - Build Status - - - - Build Status - - - - Build Status - - - - Build Status - -
Portable--- - - Build Status - - - - Build Status - - ---------
- -See [.test-infra/jenkins/README](https://github.com/apache/beam/blob/master/.test-infra/jenkins/README.md) for trigger phrase, status and link of all Jenkins jobs. - +To check the build health, please visit [https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md](https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md) GitHub Actions Tests Status (on master branch) ------------------------------------------------------------------------------------------------ diff --git a/.github/REVIEWERS.yml b/.github/REVIEWERS.yml new file mode 100644 index 0000000000000..009489798a457 --- /dev/null +++ b/.github/REVIEWERS.yml @@ -0,0 +1,21 @@ +# 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. + +labels: +- name: "Go" + reviewers: ["damccorm", "lostluck", "jrmccluskey", "youngoli", "riteshghorse"] + exclusionList: [] # These users will never be suggested as reviewers +# I don't know the other areas well enough to assess who the normal committers/contributors who might want to be reviewers are +fallbackReviewers: [] # List of committers to use when no label matches diff --git a/.github/codecov.yml b/.github/codecov.yml index 295234e93cde8..b01fdc714d1d6 100644 --- a/.github/codecov.yml +++ b/.github/codecov.yml @@ -29,12 +29,16 @@ coverage: status: project: python: + flags: + - python target: auto threshold: 5% base: auto paths: - "sdks/python" go: + flags: + - go target: auto threshold: 5% base: auto @@ -59,3 +63,8 @@ ignore: - "**/*_test.py" - "**/*_test_py3*.py" - "**/*_microbenchmark.py" + +# See https://docs.codecov.com/docs/flags for options. +flag_management: + default_rules: # the rules that will be followed for any flag added, generally + carryforward: true # recommended for multi-lang mono-repos. \ No newline at end of file diff --git a/.github/workflows/go_tests.yml b/.github/workflows/go_tests.yml index d909a27001c3a..f58bfcc8a8a31 100644 --- a/.github/workflows/go_tests.yml +++ b/.github/workflows/go_tests.yml @@ -44,7 +44,10 @@ jobs: run: "cd sdks/go/pkg && rm -rf .coverage || :" - name: Run coverage run: cd sdks/go/pkg && go test -coverprofile=coverage.txt -covermode=atomic ./... - - name: Upload to codecov - run: bash <(curl -s https://codecov.io/bash) + - uses: codecov/codecov-action@v2 + with: + flags: go + files: ./sdks/go/pkg/coverage.txt + name: go-unittests - name: Run vet run: cd sdks/go/pkg/beam && go vet --copylocks=false --unsafeptr=false ./... diff --git a/.gitignore b/.gitignore index a749d90d585b4..84720d319b163 100644 --- a/.gitignore +++ b/.gitignore @@ -126,3 +126,4 @@ website/www/yarn-error.log **/*.tfstate **/*.tfstate.* **/*.hcl +**/*.tfvars diff --git a/.test-infra/BUILD_STATUS.md b/.test-infra/BUILD_STATUS.md new file mode 100644 index 0000000000000..1972076df1e2b --- /dev/null +++ b/.test-infra/BUILD_STATUS.md @@ -0,0 +1,390 @@ + + + +`ValidatesRunner` compliance status (on master branch) +-------------------------------------------------------- + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
LangULRDataflowFlinkSamzaSparkTwister2
Go--- + + Build Status + + + + Build Status + + + + Build Status + + + + Build Status + + ---
Java + + Build Status + + + + Build Status +
+ + Build Status +
+ + Build Status +
+ + Build Status +
+ + Build Status +
+
+ + Build Status +
+ + Build Status +
+ + Build Status +
+ + Build Status + +
+ + Build Status +
+ + Build Status + +
+ + Build Status +
+ + Build Status +
+ + Build Status + +
+ + Build Status + +
Python--- + + Build Status +
+ + Build Status +
+ + Build Status + +
+ + Build Status +
+ + Build Status + +
+ + Build Status + + + + Build Status + + ---
XLang + + Build Status + + + + Build Status +
+ + Build Status +
+ + Build Status +
+
+ + Build Status + + + + Build Status + + + + Build Status + + ---
+ +Examples testing status on various runners +-------------------------------------------------------- + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
LangULRDataflowFlinkSamzaSparkTwister2
Go---------------------
Java--- + + Build Status +
+ + Build Status +
+ + Build Status +
+
---------------
Python---------------------
XLang---------------------
+ +Post-Commit SDK/Transform Integration Tests Status (on master branch) +------------------------------------------------------------------------------------------------ + + + + + + + + + + + + + + + + +
GoJavaPython
+ + Build Status + + + + Build Status + + + + Build Status +
+ + Build Status +
+ + Build Status + +
+ +Pre-Commit Tests Status (on master branch) +------------------------------------------------------------------------------------------------ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
---JavaPythonGoWebsiteWhitespaceTypescript
Non-portable + + Build Status +
+
+ + Build Status +
+ + Build Status +
+ + Build Status +
+ + Build Status + +
+ + Build Status + + + + Build Status + + + + Build Status + + + + Build Status + +
Portable--- + + Build Status + + + + Build Status + + ---------
+ +See [.test-infra/jenkins/README](https://github.com/apache/beam/blob/master/.test-infra/jenkins/README.md) for trigger phrase, status and link of all Jenkins jobs. + + +GitHub Actions Tests Status (on master branch) +------------------------------------------------------------------------------------------------ +[![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule) +[![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule) +[![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule) + +See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI. diff --git a/.test-infra/jenkins/job_PreCommit_Java.groovy b/.test-infra/jenkins/job_PreCommit_Java.groovy index aced96ab9bce8..444d2400abac5 100644 --- a/.test-infra/jenkins/job_PreCommit_Java.groovy +++ b/.test-infra/jenkins/job_PreCommit_Java.groovy @@ -23,7 +23,6 @@ PrecommitJobBuilder builder = new PrecommitJobBuilder( nameBase: 'Java', gradleTask: ':javaPreCommit', gradleSwitches: [ - '-PretryFlakyTest=true', '-PdisableSpotlessCheck=true' ], // spotless checked in separate pre-commit triggerPathPatterns: [ @@ -40,11 +39,7 @@ PrecommitJobBuilder builder = new PrecommitJobBuilder( ) builder.build { publishers { - archiveJunit('**/build/test-results/**/*.xml') { - testDataPublishers { - publishFlakyTestsReport() - } - } + archiveJunit('**/build/test-results/**/*.xml') recordIssues { tools { errorProne() diff --git a/.test-infra/jenkins/job_PreCommit_Python.groovy b/.test-infra/jenkins/job_PreCommit_Python.groovy index 9a2249572b2ad..0b5270f96eef0 100644 --- a/.test-infra/jenkins/job_PreCommit_Python.groovy +++ b/.test-infra/jenkins/job_PreCommit_Python.groovy @@ -21,9 +21,6 @@ import PrecommitJobBuilder PrecommitJobBuilder builder = new PrecommitJobBuilder( scope: this, nameBase: 'Python', - gradleSwitches: [ - '-PretryFlakyTest=true', - ], gradleTask: ':pythonPreCommit', timeoutMins: 180, triggerPathPatterns: [ @@ -35,10 +32,6 @@ PrecommitJobBuilder builder = new PrecommitJobBuilder( builder.build { // Publish all test results to Jenkins. publishers { - archiveJunit('**/pytest*.xml'){ - testDataPublishers { - publishFlakyTestsReport() - } - } + archiveJunit('**/pytest*.xml') } } diff --git a/README.md b/README.md index 672893546a690..b67429bb04aff 100644 --- a/README.md +++ b/README.md @@ -25,6 +25,7 @@ [![Maven Version](https://maven-badges.herokuapp.com/maven-central/org.apache.beam/beam-sdks-java-core/badge.svg)](http://search.maven.org/#search|gav|1|g:"org.apache.beam") [![PyPI version](https://badge.fury.io/py/apache-beam.svg)](https://badge.fury.io/py/apache-beam) +[![Go version](https://pkg.go.dev/badge/github.com/apache/beam/sdks/v2/go.svg)](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go) [![Python coverage](https://codecov.io/gh/apache/beam/branch/master/graph/badge.svg)](https://codecov.io/gh/apache/beam) [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule) [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule) diff --git a/build.gradle.kts b/build.gradle.kts index c53f8e9fe6c26..a49109d31cf4d 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -193,6 +193,7 @@ tasks.register("javaPostCommit") { dependsOn(":sdks:java:io:amazon-web-services2:integrationTest") dependsOn(":sdks:java:extensions:ml:postCommit") dependsOn(":sdks:java:io:kafka:kafkaVersionsCompatibilityTest") + dependsOn(":sdks:java:io:neo4j:integrationTest") } tasks.register("javaHadoopVersionsTest") { diff --git a/buildSrc/build.gradle.kts b/buildSrc/build.gradle.kts index 212074ed93d2a..f7992ed6086de 100644 --- a/buildSrc/build.gradle.kts +++ b/buildSrc/build.gradle.kts @@ -54,7 +54,6 @@ dependencies { runtimeOnly("ca.cutterslade.gradle:gradle-dependency-analyze:1.8.3") // Enable dep analysis runtimeOnly("gradle.plugin.net.ossindex:ossindex-gradle-plugin:0.4.11") // Enable dep vulnerability analysis runtimeOnly("org.checkerframework:checkerframework-gradle-plugin:0.5.16") // Enable enhanced static checking plugin - runtimeOnly("org.gradle:test-retry-gradle-plugin:1.3.1") // Enable flaky tests mitigation } // Because buildSrc is built and tested automatically _before_ gradle 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 6c6b140c6cf56..b11b590ede869 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -409,23 +409,6 @@ class BeamModulePlugin implements Plugin { project.tasks.withType(Test) { jacoco.enabled = enabled } } - // Add Retry Gradle Plugin to mitigate flaky tests - if (project.hasProperty("retryFlakyTest")) { - project.apply plugin: "org.gradle.test-retry" - project.tasks.withType(Test) { - reports{ - junitXml{ - mergeReruns = true - } - } - retry { - failOnPassedAfterRetry = false - maxFailures = 10 - maxRetries = 3 - } - } - } - // Apply a plugin which provides tasks for dependency / property / task reports. // See https://docs.gradle.org/current/userguide/project_reports_plugin.html // for further details. This is typically very useful to look at the "htmlDependencyReport" @@ -471,8 +454,12 @@ class BeamModulePlugin implements Plugin { def checkerframework_version = "3.10.0" def classgraph_version = "4.8.104" def errorprone_version = "2.10.0" + // Try to keep gax_version consistent with gax-grpc version in google_cloud_platform_libraries_bom + def gax_version = "2.8.1" def google_clients_version = "1.32.1" def google_cloud_bigdataoss_version = "2.2.4" + // Try to keep google_cloud_spanner_version consistent with google_cloud_spanner_bom in google_cloud_platform_libraries_bom + def google_cloud_spanner_version = "6.17.4" def google_code_gson_version = "2.8.9" def google_oauth_clients_version = "1.32.1" // Try to keep grpc_version consistent with gRPC version in google_cloud_platform_libraries_bom @@ -560,6 +547,7 @@ class BeamModulePlugin implements Plugin { flogger_system_backend : "com.google.flogger:flogger-system-backend:0.7.3", gax : "com.google.api:gax", // google_cloud_platform_libraries_bom sets version gax_grpc : "com.google.api:gax-grpc", // google_cloud_platform_libraries_bom sets version + gax_grpc_test : "com.google.api:gax-grpc:$gax_version:testlib", // google_cloud_platform_libraries_bom sets version gax_httpjson : "com.google.api:gax-httpjson", // google_cloud_platform_libraries_bom sets version google_api_client : "com.google.api-client:google-api-client:$google_clients_version", // for the libraries using $google_clients_version below. google_api_client_jackson2 : "com.google.api-client:google-api-client-jackson2:$google_clients_version", @@ -591,6 +579,7 @@ class BeamModulePlugin implements Plugin { // Update libraries-bom version on sdks/java/container/license_scripts/dep_urls_java.yaml google_cloud_platform_libraries_bom : "com.google.cloud:libraries-bom:24.2.0", google_cloud_spanner : "com.google.cloud:google-cloud-spanner", // google_cloud_platform_libraries_bom sets version + google_cloud_spanner_test : "com.google.cloud:google-cloud-spanner:$google_cloud_spanner_version:tests", google_code_gson : "com.google.code.gson:gson:$google_code_gson_version", // google-http-client's version is explicitly declared for sdks/java/maven-archetypes/examples // This version should be in line with the one in com.google.cloud:libraries-bom. @@ -1333,7 +1322,8 @@ class BeamModulePlugin implements Plugin { } if (configuration.validateShadowJar) { - project.task('validateShadedJarDoesntLeakNonProjectClasses', dependsOn: 'shadowJar') { + def validateShadedJarDoesntLeakNonProjectClasses = project.tasks.register('validateShadedJarDoesntLeakNonProjectClasses') { + dependsOn 'shadowJar' ext.outFile = project.file("${project.reportsDir}/${name}.out") inputs.files(project.configurations.shadow.artifacts.files) .withPropertyName("shadowArtifactsFiles") @@ -1357,10 +1347,10 @@ class BeamModulePlugin implements Plugin { } } } - project.tasks.check.dependsOn project.tasks.validateShadedJarDoesntLeakNonProjectClasses + project.tasks.check.dependsOn validateShadedJarDoesntLeakNonProjectClasses } } else { - project.task("testJar", type: Jar, { + project.tasks.register("testJar", Jar) { group = "Jar" description = "Create a JAR of test classes" classifier = "tests" @@ -1370,7 +1360,7 @@ class BeamModulePlugin implements Plugin { exclude "META-INF/*.SF" exclude "META-INF/*.DSA" exclude "META-INF/*.RSA" - }) + } project.artifacts.testRuntimeMigration project.testJar } @@ -1392,7 +1382,8 @@ class BeamModulePlugin implements Plugin { } } - project.task("jmh", type: JavaExec, dependsOn: project.classes, { + project.tasks.register("jmh", JavaExec) { + dependsOn project.classes mainClass = "org.openjdk.jmh.Main" classpath = project.sourceSets.main.runtimeClasspath // For a list of arguments, see @@ -1424,12 +1415,13 @@ class BeamModulePlugin implements Plugin { args 'org.apache.beam' } args '-foe=true' - }) + } // Single shot of JMH benchmarks ensures that they can execute. // // Note that these tests will fail on JVMs that JMH doesn't support. - project.task("jmhTest", type: JavaExec, dependsOn: project.classes, { + def jmhTest = project.tasks.register("jmhTest", JavaExec) { + dependsOn project.classes mainClass = "org.openjdk.jmh.Main" classpath = project.sourceSets.main.runtimeClasspath @@ -1442,8 +1434,8 @@ class BeamModulePlugin implements Plugin { args '-f=0' args '-wf=0' args '-foe=true' - }) - project.check.dependsOn("jmhTest") + } + project.check.dependsOn jmhTest } project.ext.includeInJavaBom = configuration.publish diff --git a/playground/README.md b/playground/README.md index a119a6eb08b32..875755df9a14e 100644 --- a/playground/README.md +++ b/playground/README.md @@ -22,7 +22,7 @@ The Beam Playground is a web application to run Beam code snippets in a modern browser. This directory holds code to build, test, and deploy the frontend and backend services. -# Requirements +# Development Requirements The following requirements are needed for development, testing, and deploying. @@ -55,12 +55,7 @@ cd beam ./gradlew playground:generateProto ``` -# Backend application +# Deployment -All backend's files are placed into the [backend](./backend) folder. For more details about the backend there are -also [Contribution](./backend/CONTRIBUTE.md) guide and [README](./backend/README.md) file. - -# Frontend application - -All frontend's files are placed into the [frontend](./frontend) folder. For more details about the frontend there are -also [Contribution](./frontend/CONTRIBUTE.md) guide and [README](./frontend/README.md) file. \ No newline at end of file +See [terraform](./terraform/README.md) for details on how to build and deploy +the application and its dependent infrastructure. diff --git a/playground/backend/internal/cache/local/local_cache_test.go b/playground/backend/internal/cache/local/local_cache_test.go index c4d5a580d07a1..34b2654ad065c 100644 --- a/playground/backend/internal/cache/local/local_cache_test.go +++ b/playground/backend/internal/cache/local/local_cache_test.go @@ -58,6 +58,8 @@ func TestLocalCache_GetValue(t *testing.T) { preparedItemsMap[preparedId][preparedSubKey] = value preparedExpMap := make(map[uuid.UUID]time.Time) preparedExpMap[preparedId] = time.Now().Add(time.Millisecond) + endedExpMap := make(map[uuid.UUID]time.Time) + endedExpMap[preparedId] = time.Now().Add(-time.Millisecond) type fields struct { cleanupInterval time.Duration items map[uuid.UUID]map[cache.SubKey]interface{} @@ -76,7 +78,7 @@ func TestLocalCache_GetValue(t *testing.T) { wantErr bool }{ { - name: "Get exist value", + name: "Get existing value", fields: fields{ cleanupInterval: cleanupInterval, items: preparedItemsMap, @@ -91,7 +93,7 @@ func TestLocalCache_GetValue(t *testing.T) { wantErr: false, }, { - name: "Get not exist value", + name: "Get not existing value", fields: fields{ cleanupInterval: cleanupInterval, items: make(map[uuid.UUID]map[cache.SubKey]interface{}), @@ -103,6 +105,21 @@ func TestLocalCache_GetValue(t *testing.T) { want: nil, wantErr: true, }, + { + name: "Get an existing value that is expiring", + fields: fields{ + cleanupInterval: cleanupInterval, + items: preparedItemsMap, + pipelinesExpiration: endedExpMap, + }, + args: args{ + ctx: context.Background(), + pipelineId: preparedId, + subKey: preparedSubKey, + }, + want: nil, + wantErr: true, + }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { @@ -159,6 +176,21 @@ func TestLocalCache_SetValue(t *testing.T) { }, wantErr: false, }, + { + name: "Set value for RunOutputIndex subKey", + fields: fields{ + cleanupInterval: cleanupInterval, + items: make(map[uuid.UUID]map[cache.SubKey]interface{}), + pipelinesExpiration: preparedExpMap, + }, + args: args{ + ctx: context.Background(), + pipelineId: preparedId, + subKey: cache.RunOutputIndex, + value: 5, + }, + wantErr: false, + }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { @@ -180,6 +212,9 @@ func TestLocalCache_SetValue(t *testing.T) { func TestLocalCache_SetExpTime(t *testing.T) { preparedId, _ := uuid.NewUUID() + preparedItems := make(map[uuid.UUID]map[cache.SubKey]interface{}) + preparedItems[preparedId] = make(map[cache.SubKey]interface{}) + preparedItems[preparedId][cache.Status] = 1 type fields struct { cleanupInterval time.Duration items map[uuid.UUID]map[cache.SubKey]interface{} @@ -191,12 +226,27 @@ func TestLocalCache_SetExpTime(t *testing.T) { expTime time.Duration } tests := []struct { - name string - fields fields - args args + name string + fields fields + args args + wantErr bool }{ { name: "Set expiration time", + fields: fields{ + cleanupInterval: cleanupInterval, + items: preparedItems, + pipelinesExpiration: make(map[uuid.UUID]time.Time), + }, + args: args{ + ctx: context.Background(), + pipelineId: preparedId, + expTime: time.Minute, + }, + wantErr: false, + }, + { + name: "Set expiration time for not existing value in cache items", fields: fields{ cleanupInterval: cleanupInterval, items: make(map[uuid.UUID]map[cache.SubKey]interface{}), @@ -207,6 +257,7 @@ func TestLocalCache_SetExpTime(t *testing.T) { pipelineId: preparedId, expTime: time.Minute, }, + wantErr: true, }, } for _, tt := range tests { @@ -216,14 +267,13 @@ func TestLocalCache_SetExpTime(t *testing.T) { items: tt.fields.items, pipelinesExpiration: tt.fields.pipelinesExpiration, } - _ = lc.SetValue(tt.args.ctx, preparedId, cache.Status, 1) err := lc.SetExpTime(tt.args.ctx, tt.args.pipelineId, tt.args.expTime) - if err != nil { - t.Error(err) - } - expTime, found := lc.pipelinesExpiration[tt.args.pipelineId] - if expTime.Round(time.Second) != time.Now().Add(tt.args.expTime).Round(time.Second) || !found { - t.Errorf("Expiration time of the pipeline: %s not set in cache.", tt.args.pipelineId) + if (err != nil) != tt.wantErr { + t.Errorf("SetExpTime() error = %v, wantErr %v", err, tt.wantErr) + expTime, found := lc.pipelinesExpiration[tt.args.pipelineId] + if expTime.Round(time.Second) != time.Now().Add(tt.args.expTime).Round(time.Second) || !found { + t.Errorf("Expiration time of the pipeline: %s not set in cache.", tt.args.pipelineId) + } } }) } @@ -429,6 +479,16 @@ func TestLocalCache_startGC(t *testing.T) { pipelinesExpiration: preparedExpMap, }, }, + { + // Test case with calling startGC method with nil cache items. + // As a result, items stay the same. + name: "Checking for deleting expired pipelines with nil cache items", + fields: fields{ + cleanupInterval: time.Microsecond, + items: nil, + pipelinesExpiration: preparedExpMap, + }, + }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { @@ -540,3 +600,34 @@ func TestLocalCache_clearItems(t *testing.T) { }) } } + +func TestNew(t *testing.T) { + items := make(map[uuid.UUID]map[cache.SubKey]interface{}) + pipelinesExpiration := make(map[uuid.UUID]time.Time) + type args struct { + ctx context.Context + } + tests := []struct { + name string + args args + want *Cache + }{ + { + name: "Initialize local cache", + args: args{ctx: context.Background()}, + want: &Cache{ + cleanupInterval: cleanupInterval, + items: items, + pipelinesExpiration: pipelinesExpiration, + catalog: nil, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + if got := New(tt.args.ctx); !reflect.DeepEqual(got, tt.want) { + t.Errorf("New() = %v, want %v", got, tt.want) + } + }) + } +} diff --git a/playground/backend/internal/cache/redis/redis_cache_test.go b/playground/backend/internal/cache/redis/redis_cache_test.go index 02dec43b5d0ed..60a0ecfa2c252 100644 --- a/playground/backend/internal/cache/redis/redis_cache_test.go +++ b/playground/backend/internal/cache/redis/redis_cache_test.go @@ -54,7 +54,7 @@ func TestRedisCache_GetValue(t *testing.T) { wantErr bool }{ { - name: "error during HGet operation", + name: "Error during HGet operation", mocks: func() { mock.ExpectHGet(pipelineId.String(), string(marshSubKey)).SetErr(fmt.Errorf("MOCK_ERROR")) }, @@ -68,7 +68,7 @@ func TestRedisCache_GetValue(t *testing.T) { wantErr: true, }, { - name: "all success", + name: "Get existing value", mocks: func() { mock.ExpectHGet(pipelineId.String(), string(marshSubKey)).SetVal(string(marshValue)) }, @@ -122,7 +122,7 @@ func TestRedisCache_SetExpTime(t *testing.T) { wantErr bool }{ { - name: "error during Exists operation", + name: "Error during Exists operation", mocks: func() { mock.ExpectExists(pipelineId.String()).SetErr(fmt.Errorf("MOCK_ERROR")) }, @@ -148,7 +148,7 @@ func TestRedisCache_SetExpTime(t *testing.T) { wantErr: true, }, { - name: "error during Expire operation", + name: "Set expiration time with error during Expire operation", mocks: func() { mock.ExpectExists(pipelineId.String()).SetVal(1) mock.ExpectExpire(pipelineId.String(), expTime).SetErr(fmt.Errorf("MOCK_ERROR")) @@ -162,7 +162,7 @@ func TestRedisCache_SetExpTime(t *testing.T) { wantErr: true, }, { - name: "all success", + name: "Set expiration time", mocks: func() { mock.ExpectExists(pipelineId.String()).SetVal(1) mock.ExpectExpire(pipelineId.String(), expTime).SetVal(true) @@ -215,7 +215,7 @@ func TestRedisCache_SetValue(t *testing.T) { wantErr bool }{ { - name: "error during HSet operation", + name: "Error during HSet operation", mocks: func() { mock.ExpectHSet(pipelineId.String(), marshSubKey, marshValue).SetErr(fmt.Errorf("MOCK_ERROR")) }, @@ -229,7 +229,7 @@ func TestRedisCache_SetValue(t *testing.T) { wantErr: true, }, { - name: "all success", + name: "Set correct value", mocks: func() { mock.ExpectHSet(pipelineId.String(), marshSubKey, marshValue).SetVal(1) mock.ExpectExpire(pipelineId.String(), time.Minute*15).SetVal(true) @@ -243,6 +243,21 @@ func TestRedisCache_SetValue(t *testing.T) { }, wantErr: false, }, + { + name: "Set incorrect value", + mocks: func() { + mock.ExpectHSet(pipelineId.String(), marshSubKey, marshValue).SetVal(1) + mock.ExpectExpire(pipelineId.String(), time.Minute*15).SetVal(true) + }, + fields: fields{client}, + args: args{ + ctx: context.Background(), + pipelineId: pipelineId, + subKey: subKey, + value: make(chan int), + }, + wantErr: true, + }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { @@ -318,7 +333,7 @@ func TestCache_GetCatalog(t *testing.T) { wantErr bool }{ { - name: "Success", + name: "Get existing catalog", mocks: func() { mock.ExpectGet(cache.ExamplesCatalog).SetVal(string(catalogMarsh)) }, @@ -416,7 +431,7 @@ func TestCache_SetCatalog(t *testing.T) { wantErr bool }{ { - name: "Success", + name: "Set catalog", mocks: func() { mock.ExpectSet(cache.ExamplesCatalog, catalogMarsh, 0).SetVal("") }, @@ -465,7 +480,7 @@ func Test_newRedisCache(t *testing.T) { wantErr bool }{ { - name: "error during Ping operation", + name: "Error during Ping operation", args: args{ ctx: context.Background(), addr: address, @@ -487,6 +502,9 @@ func Test_unmarshalBySubKey(t *testing.T) { statusValue, _ := json.Marshal(status) output := "MOCK_OUTPUT" outputValue, _ := json.Marshal(output) + canceledValue, _ := json.Marshal(false) + runOutputIndex := 0 + runOutputIndexValue, _ := json.Marshal(runOutputIndex) type args struct { ctx context.Context subKey cache.SubKey @@ -499,7 +517,7 @@ func Test_unmarshalBySubKey(t *testing.T) { wantErr bool }{ { - name: "status subKey", + name: "Status subKey", args: args{ ctx: context.Background(), subKey: cache.Status, @@ -509,7 +527,7 @@ func Test_unmarshalBySubKey(t *testing.T) { wantErr: false, }, { - name: "runOutput subKey", + name: "RunOutput subKey", args: args{ subKey: cache.RunOutput, value: string(outputValue), @@ -518,7 +536,7 @@ func Test_unmarshalBySubKey(t *testing.T) { wantErr: false, }, { - name: "compileOutput subKey", + name: "CompileOutput subKey", args: args{ subKey: cache.CompileOutput, value: string(outputValue), @@ -527,7 +545,7 @@ func Test_unmarshalBySubKey(t *testing.T) { wantErr: false, }, { - name: "graph subKey", + name: "Graph subKey", args: args{ subKey: cache.Graph, value: string(outputValue), @@ -535,6 +553,28 @@ func Test_unmarshalBySubKey(t *testing.T) { want: output, wantErr: false, }, + { + // Test case with calling unmarshalBySubKey method with Canceled subKey. + // As a result, want to receive false. + name: "Canceled subKey", + args: args{ + subKey: cache.Canceled, + value: string(canceledValue), + }, + want: false, + wantErr: false, + }, + { + // Test case with calling unmarshalBySubKey method with RunOutputIndex subKey. + // As a result, want to receive expected runOutputIndex. + name: "RunOutputIndex subKey", + args: args{ + subKey: cache.RunOutputIndex, + value: string(runOutputIndexValue), + }, + want: float64(runOutputIndex), + wantErr: false, + }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { diff --git a/playground/backend/internal/code_processing/code_processing.go b/playground/backend/internal/code_processing/code_processing.go index 88baac8a59294..e39e1a39b553c 100644 --- a/playground/backend/internal/code_processing/code_processing.go +++ b/playground/backend/internal/code_processing/code_processing.go @@ -23,7 +23,6 @@ import ( "beam.apache.org/playground/backend/internal/executors" "beam.apache.org/playground/backend/internal/fs_tool" "beam.apache.org/playground/backend/internal/logger" - "beam.apache.org/playground/backend/internal/preparers" "beam.apache.org/playground/backend/internal/setup_tools/builder" "beam.apache.org/playground/backend/internal/streaming" "beam.apache.org/playground/backend/internal/utils" @@ -35,7 +34,6 @@ import ( "io" "os" "os/exec" - "path/filepath" "reflect" "sync" "time" @@ -114,7 +112,7 @@ func runStep(ctx context.Context, cacheService cache.Cache, paths *fs_tool.LifeC var runError bytes.Buffer runOutput := streaming.RunOutputWriter{Ctx: pipelineLifeCycleCtx, CacheService: cacheService, PipelineId: pipelineId} go readLogFile(pipelineLifeCycleCtx, ctx, cacheService, paths.AbsoluteLogFilePath, pipelineId, stopReadLogsChannel, finishReadLogsChannel) - go readGraphFile(pipelineLifeCycleCtx, ctx, cacheService, filepath.Join(paths.AbsoluteBaseFolderPath, preparers.GraphFileName), pipelineId) + go readGraphFile(pipelineLifeCycleCtx, ctx, cacheService, paths.AbsoluteGraphFilePath, pipelineId) if sdkEnv.ApacheBeamSdk == pb.Sdk_SDK_GO { // For go SDK all logs are placed to stdErr. diff --git a/playground/backend/internal/fs_tool/fs.go b/playground/backend/internal/fs_tool/fs.go index 0cea7c57eada3..b0984279d7f87 100644 --- a/playground/backend/internal/fs_tool/fs.go +++ b/playground/backend/internal/fs_tool/fs.go @@ -40,6 +40,7 @@ type LifeCyclePaths struct { AbsoluteExecutableFilePath string // /path/to/workingDir/pipelinesFolder/{pipelineId}/bin/{pipelineId}.{executableFileExtension} AbsoluteBaseFolderPath string // /path/to/workingDir/pipelinesFolder/{pipelineId} AbsoluteLogFilePath string // /path/to/workingDir/pipelinesFolder/{pipelineId}/logs.log + AbsoluteGraphFilePath string // /path/to/workingDir/pipelinesFolder/{pipelineId}/graph.dot ProjectDir string // /path/to/workingDir/ ExecutableName func(string) (string, error) } diff --git a/playground/backend/internal/fs_tool/fs_test.go b/playground/backend/internal/fs_tool/fs_test.go index 0379466e222e8..8ff4b0d456e2b 100644 --- a/playground/backend/internal/fs_tool/fs_test.go +++ b/playground/backend/internal/fs_tool/fs_test.go @@ -17,6 +17,7 @@ package fs_tool import ( pb "beam.apache.org/playground/backend/internal/api/v1" + "beam.apache.org/playground/backend/internal/utils" "fmt" "github.com/google/uuid" "os" @@ -294,6 +295,7 @@ func TestNewLifeCycle(t *testing.T) { AbsoluteExecutableFilePath: filepath.Join(execFileFolder, fmt.Sprintf("%s%s", pipelineId.String(), javaCompiledFileExtension)), AbsoluteBaseFolderPath: baseFileFolder, AbsoluteLogFilePath: filepath.Join(baseFileFolder, logFileName), + AbsoluteGraphFilePath: filepath.Join(baseFileFolder, utils.GraphFileName), }, }, }, @@ -315,6 +317,7 @@ func TestNewLifeCycle(t *testing.T) { AbsoluteExecutableFilePath: filepath.Join(execFileFolder, fmt.Sprintf("%s%s", pipelineId.String(), goExecutableFileExtension)), AbsoluteBaseFolderPath: baseFileFolder, AbsoluteLogFilePath: filepath.Join(baseFileFolder, logFileName), + AbsoluteGraphFilePath: filepath.Join(baseFileFolder, utils.GraphFileName), }, }, }, @@ -336,6 +339,7 @@ func TestNewLifeCycle(t *testing.T) { AbsoluteExecutableFilePath: filepath.Join(baseFileFolder, fmt.Sprintf("%s%s", pipelineId.String(), pythonExecutableFileExtension)), AbsoluteBaseFolderPath: baseFileFolder, AbsoluteLogFilePath: filepath.Join(baseFileFolder, logFileName), + AbsoluteGraphFilePath: filepath.Join(baseFileFolder, utils.GraphFileName), }, }, }, diff --git a/playground/backend/internal/fs_tool/go_fs_test.go b/playground/backend/internal/fs_tool/go_fs_test.go index 55dbf968b3e9f..a0211751d16ca 100644 --- a/playground/backend/internal/fs_tool/go_fs_test.go +++ b/playground/backend/internal/fs_tool/go_fs_test.go @@ -16,6 +16,7 @@ package fs_tool import ( + "beam.apache.org/playground/backend/internal/utils" "github.com/google/uuid" "path/filepath" "reflect" @@ -57,6 +58,7 @@ func Test_newGoLifeCycle(t *testing.T) { AbsoluteExecutableFilePath: filepath.Join(binFileFolder, pipelineId.String()+goExecutableFileExtension), AbsoluteBaseFolderPath: baseFileFolder, AbsoluteLogFilePath: filepath.Join(baseFileFolder, logFileName), + AbsoluteGraphFilePath: filepath.Join(baseFileFolder, utils.GraphFileName), }, }, }, diff --git a/playground/backend/internal/fs_tool/java_fs_test.go b/playground/backend/internal/fs_tool/java_fs_test.go index d0d9aa72b369f..d51e7c0318465 100644 --- a/playground/backend/internal/fs_tool/java_fs_test.go +++ b/playground/backend/internal/fs_tool/java_fs_test.go @@ -16,6 +16,7 @@ package fs_tool import ( + "beam.apache.org/playground/backend/internal/utils" "github.com/google/uuid" "os" "path/filepath" @@ -58,6 +59,7 @@ func Test_newJavaLifeCycle(t *testing.T) { AbsoluteExecutableFilePath: filepath.Join(binFileFolder, pipelineId.String()+javaCompiledFileExtension), AbsoluteBaseFolderPath: baseFileFolder, AbsoluteLogFilePath: filepath.Join(baseFileFolder, logFileName), + AbsoluteGraphFilePath: filepath.Join(baseFileFolder, utils.GraphFileName), ExecutableName: executableName, }, }, diff --git a/playground/backend/internal/fs_tool/lc_constructor.go b/playground/backend/internal/fs_tool/lc_constructor.go index 097c554147c4d..a33f07b5633f5 100644 --- a/playground/backend/internal/fs_tool/lc_constructor.go +++ b/playground/backend/internal/fs_tool/lc_constructor.go @@ -16,6 +16,7 @@ package fs_tool import ( + "beam.apache.org/playground/backend/internal/utils" "github.com/google/uuid" "path/filepath" ) @@ -39,6 +40,7 @@ func newCompilingLifeCycle(pipelineId uuid.UUID, pipelinesFolder, sourceFileExte absExecFilePath, _ := filepath.Abs(filepath.Join(absExecFileFolderPath, execFileName)) absBaseFolderPath, _ := filepath.Abs(baseFileFolder) absLogFilePath, _ := filepath.Abs(filepath.Join(absBaseFolderPath, logFileName)) + absGraphFilePath, _ := filepath.Abs(filepath.Join(absBaseFolderPath, utils.GraphFileName)) return &LifeCycle{ folderGlobs: []string{baseFileFolder, srcFileFolder, binFileFolder}, @@ -51,6 +53,7 @@ func newCompilingLifeCycle(pipelineId uuid.UUID, pipelinesFolder, sourceFileExte AbsoluteExecutableFilePath: absExecFilePath, AbsoluteBaseFolderPath: absBaseFolderPath, AbsoluteLogFilePath: absLogFilePath, + AbsoluteGraphFilePath: absGraphFilePath, }, } } @@ -63,6 +66,7 @@ func newInterpretedLifeCycle(pipelineId uuid.UUID, pipelinesFolder, sourceFileEx absFileFolderPath, _ := filepath.Abs(sourceFileFolder) absFilePath, _ := filepath.Abs(filepath.Join(absFileFolderPath, fileName)) absLogFilePath, _ := filepath.Abs(filepath.Join(absFileFolderPath, logFileName)) + absGraphFilePath, _ := filepath.Abs(filepath.Join(absFileFolderPath, utils.GraphFileName)) return &LifeCycle{ folderGlobs: []string{sourceFileFolder}, @@ -75,6 +79,7 @@ func newInterpretedLifeCycle(pipelineId uuid.UUID, pipelinesFolder, sourceFileEx AbsoluteExecutableFilePath: absFilePath, AbsoluteBaseFolderPath: absFileFolderPath, AbsoluteLogFilePath: absLogFilePath, + AbsoluteGraphFilePath: absGraphFilePath, }, } } diff --git a/playground/backend/internal/fs_tool/python_fs_test.go b/playground/backend/internal/fs_tool/python_fs_test.go index 628722cb8470a..98ee39e9805d6 100644 --- a/playground/backend/internal/fs_tool/python_fs_test.go +++ b/playground/backend/internal/fs_tool/python_fs_test.go @@ -16,6 +16,7 @@ package fs_tool import ( + "beam.apache.org/playground/backend/internal/utils" "github.com/google/uuid" "path/filepath" "reflect" @@ -55,6 +56,7 @@ func Test_newPythonLifeCycle(t *testing.T) { AbsoluteExecutableFilePath: filepath.Join(baseFileFolder, pipelineId.String()+pythonExecutableFileExtension), AbsoluteBaseFolderPath: baseFileFolder, AbsoluteLogFilePath: filepath.Join(baseFileFolder, logFileName), + AbsoluteGraphFilePath: filepath.Join(baseFileFolder, utils.GraphFileName), }, }, }, diff --git a/playground/backend/internal/preparers/java_preparers.go b/playground/backend/internal/preparers/java_preparers.go index 881afe039f573..e42d53f8b43b0 100644 --- a/playground/backend/internal/preparers/java_preparers.go +++ b/playground/backend/internal/preparers/java_preparers.go @@ -21,6 +21,7 @@ import ( "bufio" "fmt" "io" + "io/ioutil" "os" "regexp" "strings" @@ -35,6 +36,14 @@ const ( pathSeparatorPattern = os.PathSeparator tmpFileSuffix = "tmp" javaPublicClassNamePattern = "public class (.*?) [{|implements(.*)]" + pipelineNamePattern = `Pipeline\s([A-z|0-9_]*)\s=\sPipeline\.create` + graphSavePattern = "String dotString = org.apache.beam.runners.core.construction.renderer.PipelineDotRenderer.toDotString(%s);\n" + + " try (java.io.PrintWriter out = new java.io.PrintWriter(\"%s\")) {\n " + + " out.println(dotString);\n " + + " } catch (java.io.FileNotFoundException e) {\n" + + " e.printStackTrace();\n " + + "\n}\n" + graphRunPattern = "(.*%s.run.*;)" ) //JavaPreparersBuilder facet of PreparersBuilder @@ -87,12 +96,48 @@ func (builder *JavaPreparersBuilder) WithFileNameChanger() *JavaPreparersBuilder return builder } +//WithGraphHandler adds code to save the graph +func (builder *JavaPreparersBuilder) WithGraphHandler() *JavaPreparersBuilder { + graphCodeAdder := Preparer{ + Prepare: addCodeToSaveGraph, + Args: []interface{}{builder.filePath}, + } + builder.AddPreparer(graphCodeAdder) + return builder +} + +func addCodeToSaveGraph(args ...interface{}) error { + filePath := args[0].(string) + pipelineObjectName, _ := findPipelineObjectName(filePath) + graphSaveCode := fmt.Sprintf(graphSavePattern, pipelineObjectName, utils.GraphFileName) + + if pipelineObjectName != utils.EmptyLine { + reg := regexp.MustCompile(fmt.Sprintf(graphRunPattern, pipelineObjectName)) + code, err := ioutil.ReadFile(filePath) + if err != nil { + logger.Error("Can't read file") + return err + } + result := reg.ReplaceAllString(string(code), fmt.Sprintf(`%s$1`, graphSaveCode)) + if err != nil { + logger.Error("Can't add graph extraction code") + return err + } + if err = ioutil.WriteFile(filePath, []byte(result), 0666); err != nil { + logger.Error("Can't rewrite file %s", filePath) + return err + } + } + return nil +} + // GetJavaPreparers returns preparation methods that should be applied to Java code func GetJavaPreparers(builder *PreparersBuilder, isUnitTest bool, isKata bool) { if !isUnitTest && !isKata { builder.JavaPreparers(). WithPublicClassRemover(). - WithPackageChanger() + WithPackageChanger(). + WithGraphHandler() } if isUnitTest { builder.JavaPreparers(). @@ -102,10 +147,27 @@ func GetJavaPreparers(builder *PreparersBuilder, isUnitTest bool, isKata bool) { if isKata { builder.JavaPreparers(). WithPublicClassRemover(). - WithPackageRemover() + WithPackageRemover(). + WithGraphHandler() } } +// findPipelineObjectName finds name of pipeline in JAVA code when pipeline is created +func findPipelineObjectName(filepath string) (string, error) { + reg := regexp.MustCompile(pipelineNamePattern) + b, err := ioutil.ReadFile(filepath) + if err != nil { + return "", err + } + matches := reg.FindStringSubmatch(string(b)) + if len(matches) > 0 { + return matches[1], nil + } else { + return "", nil + } + +} + // replace processes file by filePath and replaces all patterns to newPattern func replace(args ...interface{}) error { filePath := args[0].(string) diff --git a/playground/backend/internal/preparers/java_preparers_test.go b/playground/backend/internal/preparers/java_preparers_test.go index 23720b4444957..913cd1ce6dcba 100644 --- a/playground/backend/internal/preparers/java_preparers_test.go +++ b/playground/backend/internal/preparers/java_preparers_test.go @@ -102,7 +102,7 @@ func TestGetJavaPreparers(t *testing.T) { { name: "Test number of preparers for code", args: args{"MOCK_FILEPATH", false, false}, - want: 2, + want: 3, }, { name: "Test number of preparers for unit test", @@ -112,7 +112,7 @@ func TestGetJavaPreparers(t *testing.T) { { name: "Test number of preparers for kata", args: args{"MOCK_FILEPATH", false, true}, - want: 2, + want: 3, }, } for _, tt := range tests { @@ -171,3 +171,50 @@ func Test_changeJavaTestFileName(t *testing.T) { }) } } + +func Test_findPipelineObjectName(t *testing.T) { + code := "package org.apache.beam.examples;\n\n/*\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements. See the NOTICE file\n * distributed with this work for additional information\n * regarding copyright ownership. The ASF licenses this file\n * to you under the Apache License, Version 2.0 (the\n * \"License\"); you may not use this file except in compliance\n * with the License. You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n// beam-playground:\n// name: Branching\n// description: Task from katas to branch out the numbers to two different transforms, one transform\n// is multiplying each number by 5 and the other transform is multiplying each number by 10.\n// multifile: false\n// categories:\n// - Branching\n// - Core Transforms\n\nimport static org.apache.beam.sdk.values.TypeDescriptors.integers;\n\nimport org.apache.beam.sdk.Pipeline;\nimport org.apache.beam.sdk.options.PipelineOptions;\nimport org.apache.beam.sdk.options.PipelineOptionsFactory;\nimport org.apache.beam.sdk.transforms.Create;\nimport org.apache.beam.sdk.transforms.MapElements;\nimport org.apache.beam.sdk.values.PCollection;\nimport org.apache.beam.runners.core.construction.renderer.PipelineDotRenderer;\n\npublic class Task {\n\n\n\n public static void main(String[] args) {\n PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();\n Pipeline pipeline = Pipeline.create(options);\n\n PCollection numbers =\n pipeline.apply(Create.of(1, 2, 3, 4, 5));\n\n PCollection mult5Results = applyMultiply5Transform(numbers);\n PCollection mult10Results = applyMultiply10Transform(numbers);\n\n mult5Results.apply(\"Log multiply 5\", Log.ofElements(\"Multiplied by 5: \"));\n mult10Results.apply(\"Log multiply 10\", Log.ofElements(\"Multiplied by 10: \"));\n\n String dotString = PipelineDotRenderer.toDotString(pipeline);\n System.out.println(dotString);\n pipeline.run();\n\n }\n\n static PCollection applyMultiply5Transform(PCollection input) {\n return input.apply(\"Multiply by 5\", MapElements.into(integers()).via(num -> num * 5));\n }\n\n static PCollection applyMultiply10Transform(PCollection input) {\n return input.apply(\"Multiply by 10\", MapElements.into(integers()).via(num -> num * 10));\n }\n\n}\n" + lc := createTempFileWithCode(code) + codeWithoutPipeline := "package org.apache.beam.examples;\n\n/*\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements. See the NOTICE file\n * distributed with this work for additional information\n * regarding copyright ownership. The ASF licenses this file\n * to you under the Apache License, Version 2.0 (the\n * \"License\"); you may not use this file except in compliance\n * with the License. You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n// beam-playground:\n// name: Branching\n// description: Task from katas to branch out the numbers to two different transforms, one transform\n// is multiplying each number by 5 and the other transform is multiplying each number by 10.\n// multifile: false\n// categories:\n// - Branching\n// - Core Transforms\n\nimport static org.apache.beam.sdk.values.TypeDescriptors.integers;\n\nimport org.apache.beam.sdk.Pipeline;\nimport org.apache.beam.sdk.options.PipelineOptions;\nimport org.apache.beam.sdk.options.PipelineOptionsFactory;\nimport org.apache.beam.sdk.transforms.Create;\nimport org.apache.beam.sdk.transforms.MapElements;\nimport org.apache.beam.sdk.values.PCollection;\nimport org.apache.beam.runners.core.construction.renderer.PipelineDotRenderer;\n\npublic class Task {\n\n\n\n public static void main(String[] args) {\n PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();\n PCollection numbers =\n pipeline.apply(Create.of(1, 2, 3, 4, 5));\n\n PCollection mult5Results = applyMultiply5Transform(numbers);\n PCollection mult10Results = applyMultiply10Transform(numbers);\n\n mult5Results.apply(\"Log multiply 5\", Log.ofElements(\"Multiplied by 5: \"));\n mult10Results.apply(\"Log multiply 10\", Log.ofElements(\"Multiplied by 10: \"));\n\n String dotString = PipelineDotRenderer.toDotString(pipeline);\n System.out.println(dotString);\n pipeline.run();\n\n }\n\n static PCollection applyMultiply5Transform(PCollection input) {\n return input.apply(\"Multiply by 5\", MapElements.into(integers()).via(num -> num * 5));\n }\n\n static PCollection applyMultiply10Transform(PCollection input) {\n return input.apply(\"Multiply by 10\", MapElements.into(integers()).via(num -> num * 10));\n }\n\n}\n" + lcWithoutPipeline := createTempFileWithCode(codeWithoutPipeline) + path, _ := os.Getwd() + defer os.RemoveAll(filepath.Join(path, "temp")) + + type args struct { + filepath string + } + tests := []struct { + name string + args args + want string + wantErr bool + }{ + {name: "pipeline name found", args: args{filepath: lc.AbsoluteSourceFilePath}, wantErr: false, want: "pipeline"}, + {name: "pipeline name not found", args: args{filepath: lcWithoutPipeline.AbsoluteSourceFilePath}, wantErr: false, want: ""}, + {name: "file not found", args: args{filepath: "someFile"}, wantErr: true, want: ""}, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got, err := findPipelineObjectName(tt.args.filepath) + if (err != nil) != tt.wantErr { + t.Errorf("findPipelineObjectName() error = %v, wantErr %v", err, tt.wantErr) + return + } + if got != tt.want { + t.Errorf("findPipelineObjectName() got = %v, want %v", got, tt.want) + } + }) + } +} + +func createTempFileWithCode(code string) fs_tool.LifeCyclePaths { + path, err := os.Getwd() + if err != nil { + panic(err) + } + lc, _ := fs_tool.NewLifeCycle(pb.Sdk_SDK_JAVA, uuid.New(), filepath.Join(path, "temp")) + _ = lc.CreateFolders() + + _ = lc.CreateSourceCodeFile(code) + return lc.Paths +} diff --git a/playground/backend/internal/preparers/python_preparers.go b/playground/backend/internal/preparers/python_preparers.go index 91ca4a72d9bb5..38aa4216cabf7 100644 --- a/playground/backend/internal/preparers/python_preparers.go +++ b/playground/backend/internal/preparers/python_preparers.go @@ -32,7 +32,6 @@ const ( indentationPattern = `^(%s){0,1}\w+` findPipelinePattern = `^(\s*)(.+) = beam.Pipeline` runPipelinePattern = `^(\s*).*%s.run\(\)` - GraphFileName = "graph.dot" ) // GetPythonPreparers returns preparation methods that should be applied to Python code diff --git a/playground/backend/internal/setup_tools/builder/setup_builder.go b/playground/backend/internal/setup_tools/builder/setup_builder.go index c4d05b1bc410d..51031fd983057 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder.go @@ -22,6 +22,7 @@ import ( "beam.apache.org/playground/backend/internal/fs_tool" "beam.apache.org/playground/backend/internal/preparers" "beam.apache.org/playground/backend/internal/utils" + "beam.apache.org/playground/backend/internal/validators" "fmt" "path/filepath" "strings" @@ -36,7 +37,7 @@ const ( // Validator return executor with set args for validator func Validator(paths *fs_tool.LifeCyclePaths, sdkEnv *environment.BeamEnvs) (*executors.ExecutorBuilder, error) { sdk := sdkEnv.ApacheBeamSdk - val, err := utils.GetValidators(sdk, paths.AbsoluteSourceFilePath) + val, err := validators.GetValidators(sdk, paths.AbsoluteSourceFilePath) if err != nil { return nil, err } diff --git a/playground/backend/internal/setup_tools/builder/setup_builder_test.go b/playground/backend/internal/setup_tools/builder/setup_builder_test.go index 5613b71ef49e7..6df72caa46791 100644 --- a/playground/backend/internal/setup_tools/builder/setup_builder_test.go +++ b/playground/backend/internal/setup_tools/builder/setup_builder_test.go @@ -21,7 +21,6 @@ import ( "beam.apache.org/playground/backend/internal/executors" "beam.apache.org/playground/backend/internal/fs_tool" "beam.apache.org/playground/backend/internal/preparers" - "beam.apache.org/playground/backend/internal/utils" "beam.apache.org/playground/backend/internal/validators" "fmt" "github.com/google/uuid" @@ -52,7 +51,7 @@ func setup() { } func TestValidator(t *testing.T) { - vals, err := utils.GetValidators(sdkEnv.ApacheBeamSdk, paths.AbsoluteSourceFilePath) + vals, err := validators.GetValidators(sdkEnv.ApacheBeamSdk, paths.AbsoluteSourceFilePath) if err != nil { panic(err) } diff --git a/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper.go b/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper.go index 025b688e32012..e17a31f8f4154 100644 --- a/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper.go +++ b/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper.go @@ -19,6 +19,7 @@ import ( pb "beam.apache.org/playground/backend/internal/api/v1" "beam.apache.org/playground/backend/internal/fs_tool" "beam.apache.org/playground/backend/internal/logger" + "beam.apache.org/playground/backend/internal/utils" "bufio" "errors" "github.com/google/uuid" @@ -172,6 +173,7 @@ func prepareSbtFiles(lc *fs_tool.LifeCycle, pipelineFolder string, workingDir st absFileFolderPath, _ := filepath.Abs(sourceFileFolder) absFilePath, _ := filepath.Abs(filepath.Join(absFileFolderPath, fileName)) absLogFilePath, _ := filepath.Abs(filepath.Join(absFileFolderPath, logFileName)) + absGraphFilePath, _ := filepath.Abs(filepath.Join(absFileFolderPath, utils.GraphFileName)) projectFolder, _ := filepath.Abs(filepath.Join(pipelineFolder, scioProjectName)) executableName := lc.Paths.ExecutableName @@ -195,6 +197,7 @@ func prepareSbtFiles(lc *fs_tool.LifeCycle, pipelineFolder string, workingDir st AbsoluteExecutableFilePath: absFilePath, AbsoluteBaseFolderPath: absFileFolderPath, AbsoluteLogFilePath: absLogFilePath, + AbsoluteGraphFilePath: absGraphFilePath, ProjectDir: projectFolder, }, } diff --git a/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper_test.go b/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper_test.go index 213816840a90f..9b5aa5ea984fd 100644 --- a/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper_test.go +++ b/playground/backend/internal/setup_tools/life_cycle/life_cycle_setuper_test.go @@ -18,6 +18,7 @@ package life_cycle import ( playground "beam.apache.org/playground/backend/internal/api/v1" "beam.apache.org/playground/backend/internal/fs_tool" + "beam.apache.org/playground/backend/internal/utils" "fmt" "github.com/google/uuid" "io/fs" @@ -123,6 +124,7 @@ func TestSetup(t *testing.T) { AbsoluteExecutableFilePath: filepath.Join(execFileFolder, fmt.Sprintf("%s%s", successPipelineId.String(), javaCompiledFileExtension)), AbsoluteBaseFolderPath: baseFileFolder, AbsoluteLogFilePath: filepath.Join(baseFileFolder, logFileName), + AbsoluteGraphFilePath: filepath.Join(baseFileFolder, utils.GraphFileName), }, }, wantErr: false, diff --git a/playground/backend/internal/utils/preparares_utils.go b/playground/backend/internal/utils/preparares_utils.go index de90eb7f2c6bd..c66d3b7306a01 100644 --- a/playground/backend/internal/utils/preparares_utils.go +++ b/playground/backend/internal/utils/preparares_utils.go @@ -29,7 +29,7 @@ import ( const ( indentationReplacement = "$0" - emptyLine = "" + EmptyLine = "" GraphFileName = "graph.dot" pythonGraphCodePattern = "$0# Write graph to file\n$0from apache_beam.runners.interactive.display import pipeline_graph\n$0dot = pipeline_graph.PipelineGraph(%s).get_dot()\n$0with open('%s', 'w') as file:\n$0 file.write(dot)\n" newLinePattern = "\n" @@ -50,12 +50,12 @@ func ReplaceSpacesWithEquals(pipelineOptions string) string { // InitVars creates empty variables func InitVars() (string, string, error, bool, PipelineDefinitionType) { - return emptyLine, emptyLine, errors.New(emptyLine), false, RegularDefinition + return EmptyLine, EmptyLine, errors.New(EmptyLine), false, RegularDefinition } // AddGraphToEndOfFile if no place for graph was found adds graph code to the end of the file func AddGraphToEndOfFile(spaces string, err error, tempFile *os.File, pipelineName string) { - line := emptyLine + line := EmptyLine regs := []*regexp.Regexp{regexp.MustCompile("^")} _, err = wrap(addGraphCode)(tempFile, &line, &spaces, &pipelineName, ®s) } diff --git a/playground/backend/internal/utils/validators_utils_test.go b/playground/backend/internal/utils/validators_utils_test.go deleted file mode 100644 index 8ec7da9bc7bd4..0000000000000 --- a/playground/backend/internal/utils/validators_utils_test.go +++ /dev/null @@ -1,79 +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 utils - -import ( - playground "beam.apache.org/playground/backend/internal/api/v1" - "beam.apache.org/playground/backend/internal/validators" - "reflect" - "testing" -) - -func TestGetValidators(t *testing.T) { - type args struct { - sdk playground.Sdk - filepath string - } - tests := []struct { - name string - args args - want *[]validators.Validator - wantErr bool - }{ - { - // Test case with calling GetValidators method with incorrect SDK. - // As a result, want to receive an error. - name: "incorrect sdk", - args: args{ - sdk: playground.Sdk_SDK_UNSPECIFIED, - filepath: "", - }, - want: nil, - wantErr: true, - }, - { - // Test case with calling GetValidators method with correct SDK. - // As a result, want to receive an expected slice of validators. - name: "correct sdk", - args: args{ - sdk: playground.Sdk_SDK_JAVA, - filepath: "", - }, - want: validators.GetJavaValidators(""), - wantErr: false, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - got, err := GetValidators(tt.args.sdk, tt.args.filepath) - if (err != nil) != tt.wantErr { - t.Errorf("GetValidators() err = %v, wantErr %v", err, tt.wantErr) - } - if got != nil { - if !reflect.DeepEqual(len(*got), len(*tt.want)) { - t.Errorf("GetValidators() len = %v, want %v", len(*got), len(*tt.want)) - } - for i := range *got { - gotVal := (*got)[i] - wantVal := (*tt.want)[i] - if !reflect.DeepEqual(gotVal.Args, wantVal.Args) { - t.Errorf("GetValidators() %d = %v, want %v", i, gotVal.Args, wantVal.Args) - } - } - } - }) - } -} diff --git a/playground/backend/internal/validators/validator.go b/playground/backend/internal/validators/validator.go index cf504f7ade62e..c25455a4b7a2c 100644 --- a/playground/backend/internal/validators/validator.go +++ b/playground/backend/internal/validators/validator.go @@ -15,6 +15,11 @@ package validators +import ( + pb "beam.apache.org/playground/backend/internal/api/v1" + "fmt" +) + const ( UnitTestValidatorName = "UnitTest" KatasValidatorName = "Katas" @@ -26,3 +31,21 @@ type Validator struct { Args []interface{} Name string } + +// GetValidators returns slice of validators.Validator according to sdk +func GetValidators(sdk pb.Sdk, filepath string) (*[]Validator, error) { + var val *[]Validator + switch sdk { + case pb.Sdk_SDK_JAVA: + val = GetJavaValidators(filepath) + case pb.Sdk_SDK_GO: + val = GetGoValidators(filepath) + case pb.Sdk_SDK_PYTHON: + val = GetPyValidators(filepath) + case pb.Sdk_SDK_SCIO: + val = GetScioValidators(filepath) + default: + return nil, fmt.Errorf("incorrect sdk: %s", sdk) + } + return val, nil +} diff --git a/playground/backend/internal/validators/validator_test.go b/playground/backend/internal/validators/validator_test.go index 59b0f519a751e..52a8ef3b80811 100644 --- a/playground/backend/internal/validators/validator_test.go +++ b/playground/backend/internal/validators/validator_test.go @@ -16,8 +16,10 @@ package validators import ( + playground "beam.apache.org/playground/backend/internal/api/v1" "fmt" "os" + "reflect" "testing" ) @@ -59,3 +61,59 @@ func writeFile(path string, code string) { panic(fmt.Errorf("error during test setup: %s", err.Error())) } } + +func TestGetValidators(t *testing.T) { + type args struct { + sdk playground.Sdk + filepath string + } + tests := []struct { + name string + args args + want *[]Validator + wantErr bool + }{ + { + // Test case with calling GetValidators method with incorrect SDK. + // As a result, want to receive an error. + name: "incorrect sdk", + args: args{ + sdk: playground.Sdk_SDK_UNSPECIFIED, + filepath: "", + }, + want: nil, + wantErr: true, + }, + { + // Test case with calling GetValidators method with correct SDK. + // As a result, want to receive an expected slice of validators. + name: "correct sdk", + args: args{ + sdk: playground.Sdk_SDK_JAVA, + filepath: "", + }, + want: GetJavaValidators(""), + wantErr: false, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got, err := GetValidators(tt.args.sdk, tt.args.filepath) + if (err != nil) != tt.wantErr { + t.Errorf("GetValidators() err = %v, wantErr %v", err, tt.wantErr) + } + if got != nil { + if !reflect.DeepEqual(len(*got), len(*tt.want)) { + t.Errorf("GetValidators() len = %v, want %v", len(*got), len(*tt.want)) + } + for i := range *got { + gotVal := (*got)[i] + wantVal := (*tt.want)[i] + if !reflect.DeepEqual(gotVal.Args, wantVal.Args) { + t.Errorf("GetValidators() %d = %v, want %v", i, gotVal.Args, wantVal.Args) + } + } + } + }) + } +} diff --git a/playground/frontend/assets/multifile.svg b/playground/frontend/assets/multifile.svg new file mode 100644 index 0000000000000..2a27ab26cea62 --- /dev/null +++ b/playground/frontend/assets/multifile.svg @@ -0,0 +1,22 @@ + + + + + diff --git a/playground/frontend/lib/config.g.dart b/playground/frontend/lib/config.g.dart index e189b32dd4cc2..c81baa5554c35 100644 --- a/playground/frontend/lib/config.g.dart +++ b/playground/frontend/lib/config.g.dart @@ -16,14 +16,14 @@ * limitations under the License. */ -const String kApiClientURL = - 'https://backend-dot-datatokenization.uc.r.appspot.com'; const String kAnalyticsUA = 'UA-73650088-1'; +const String kApiClientURL = + 'https://backend-router-beta-dot-apache-beam-testing.appspot.com'; const String kApiJavaClientURL = - 'https://backend-dot-datatokenization.uc.r.appspot.com/java/'; + 'https://backend-java-beta-dot-apache-beam-testing.appspot.com'; const String kApiGoClientURL = - 'https://backend-dot-datatokenization.uc.r.appspot.com/go/'; + 'https://backend-go-beta-dot-apache-beam-testing.appspot.com'; const String kApiPythonClientURL = - 'https://backend-dot-datatokenization.uc.r.appspot.com/python/'; + 'https://backend-python-beta-dot-apache-beam-testing.appspot.com'; const String kApiScioClientURL = - 'https://backend-dot-datatokenization.uc.r.appspot.com/scio/'; + 'https://backend-scio-beta-dot-apache-beam-testing.appspot.com'; diff --git a/playground/frontend/lib/constants/assets.dart b/playground/frontend/lib/constants/assets.dart index deaad152d9430..796a97daa87fb 100644 --- a/playground/frontend/lib/constants/assets.dart +++ b/playground/frontend/lib/constants/assets.dart @@ -33,6 +33,7 @@ const kCopyIconAsset = 'copy.svg'; const kLinkIconAsset = 'link.svg'; const kDragHorizontalIconAsset = 'drag_horizontal.svg'; const kDragVerticalIconAsset = 'drag_vertical.svg'; +const kMultifileIconAsset = 'multifile.svg'; // notifications icons const kErrorNotificationIconAsset = 'error_notification.svg'; diff --git a/playground/frontend/lib/constants/links.dart b/playground/frontend/lib/constants/links.dart index e3b34ebc81652..688a7ad9be7ef 100644 --- a/playground/frontend/lib/constants/links.dart +++ b/playground/frontend/lib/constants/links.dart @@ -24,3 +24,4 @@ const kApacheBeamGithubLink = 'https://github.com/apache/beam'; const kBeamWebsiteLink = 'https://beam.apache.org/'; const kScioGithubLink = 'https://github.com/spotify/scio'; const kAboutBeamLink = 'https://beam.apache.org/get-started/beam-overview'; +const kAddExampleLink = 'https://beam.apache.org/get-started/try-beam-playground/#how-to-add-new-examples'; diff --git a/playground/frontend/lib/constants/sizes.dart b/playground/frontend/lib/constants/sizes.dart index 3962a192bf717..74f61ae307a96 100644 --- a/playground/frontend/lib/constants/sizes.dart +++ b/playground/frontend/lib/constants/sizes.dart @@ -52,6 +52,7 @@ const double kCursorSize = 1.0; // container size const double kContainerHeight = 40.0; +const double kCaptionFontSize = 10.0; const double kCodeFontSize = 14.0; const double kLabelFontSize = 16.0; const double kHintFontSize = 16.0; diff --git a/playground/frontend/lib/l10n/app_en.arb b/playground/frontend/lib/l10n/app_en.arb index df18234b71dd1..2b2b4707811ba 100644 --- a/playground/frontend/lib/l10n/app_en.arb +++ b/playground/frontend/lib/l10n/app_en.arb @@ -27,10 +27,6 @@ "@cancelExecution": { "description": "Title for the cancel execution notification" }, - "unknownExample": "Unknown Example", - "@unknownExample": { - "description": "Unknown example text part" - }, "log": "Log", "@log": { "description": "Title for the log section" @@ -188,7 +184,23 @@ "description": "Text value label" }, "pipelineOptionsError": "Please check the format (example: --key1 value1 --key2 value2), only alphanumeric and \",*,/,-,:,;,',. symbols are allowed", - "@value": { + "@pipelineOptionsError": { "description": "Pipeline options parse error" + }, + "viewOnGithub": "View on GitHub", + "@viewOnGithub": { + "description": "View on Github button" + }, + "addExample": "Add your own example", + "@addExample": { + "description": "Add example link text" + }, + "multifile": "Multifile", + "@multifile": { + "description": "Multifile example" + }, + "multifileWarning": "Multifile not supported yet for running in playground. Open it on github.", + "@multifileWarning": { + "description": "Multifile not supported text" } } \ No newline at end of file diff --git a/playground/frontend/lib/modules/analytics/analytics_service.dart b/playground/frontend/lib/modules/analytics/analytics_service.dart index 4d6d0c7f6ed11..7c4911e52a466 100644 --- a/playground/frontend/lib/modules/analytics/analytics_service.dart +++ b/playground/frontend/lib/modules/analytics/analytics_service.dart @@ -91,11 +91,11 @@ class AnalyticsService { safeSendEvent(kFeedbackCategory, kClickReportIssueEvent); } - void trackClickRunEvent(ExampleModel? example) { + void trackClickRunEvent(String exampleName) { safeSendEvent( kRunCodeCategory, kClickRunEvent, - label: example?.path ?? '', + label: exampleName, ); } diff --git a/playground/frontend/lib/modules/editor/components/run_button.dart b/playground/frontend/lib/modules/editor/components/run_button.dart index 6aecc1f74d80d..d31e36d213515 100644 --- a/playground/frontend/lib/modules/editor/components/run_button.dart +++ b/playground/frontend/lib/modules/editor/components/run_button.dart @@ -32,12 +32,14 @@ class RunButton extends StatelessWidget { final bool isRunning; final VoidCallback runCode; final VoidCallback cancelRun; + final bool disabled; const RunButton({ Key? key, required this.isRunning, required this.runCode, required this.cancelRun, + this.disabled = false, }) : super(key: key); @override @@ -71,9 +73,16 @@ class RunButton extends StatelessWidget { } return Text(buttonText); }), - onPressed: !isRunning ? runCode : cancelRun, + onPressed: onPressHandler(), ), ), ); } + + onPressHandler() { + if (disabled) { + return null; + } + return !isRunning ? runCode : cancelRun; + } } diff --git a/playground/frontend/lib/modules/examples/components/description_popover/description_popover.dart b/playground/frontend/lib/modules/examples/components/description_popover/description_popover.dart index 6a37c8ff4bb30..5de8144a5cf24 100644 --- a/playground/frontend/lib/modules/examples/components/description_popover/description_popover.dart +++ b/playground/frontend/lib/modules/examples/components/description_popover/description_popover.dart @@ -17,9 +17,13 @@ */ import 'package:flutter/material.dart'; +import 'package:flutter_svg/flutter_svg.dart'; +import 'package:flutter_gen/gen_l10n/app_localizations.dart'; +import 'package:playground/constants/assets.dart'; import 'package:playground/constants/font_weight.dart'; import 'package:playground/constants/sizes.dart'; import 'package:playground/modules/examples/models/example_model.dart'; +import 'package:url_launcher/url_launcher.dart'; const kDescriptionWidth = 300.0; @@ -30,26 +34,43 @@ class DescriptionPopover extends StatelessWidget { @override Widget build(BuildContext context) { + final hasLink = example.link?.isNotEmpty ?? false; return SizedBox( width: kDescriptionWidth, child: Card( child: Padding( padding: const EdgeInsets.all(kLgSpacing), child: Wrap( - runSpacing: kSmSpacing, + runSpacing: kMdSpacing, children: [ - Text( - example.name, - style: const TextStyle( - fontSize: kTitleFontSize, - fontWeight: kBoldWeight, - ), - ), - Text(example.description), + title, + description, + if (hasLink) getViewOnGithub(context), ], ), ), ), ); } + + Widget get title => Text( + example.name, + style: const TextStyle( + fontSize: kTitleFontSize, + fontWeight: kBoldWeight, + ), + ); + + Widget get description => Text(example.description); + + Widget getViewOnGithub(BuildContext context) { + AppLocalizations appLocale = AppLocalizations.of(context)!; + return TextButton.icon( + icon: SvgPicture.asset(kGithubIconAsset), + onPressed: () { + launch(example.link ?? ''); + }, + label: Text(appLocale.viewOnGithub), + ); + } } diff --git a/playground/frontend/lib/modules/examples/components/description_popover/description_popover_button.dart b/playground/frontend/lib/modules/examples/components/description_popover/description_popover_button.dart index d0eb6f7223e41..d2e60a9fd3b7a 100644 --- a/playground/frontend/lib/modules/examples/components/description_popover/description_popover_button.dart +++ b/playground/frontend/lib/modules/examples/components/description_popover/description_popover_button.dart @@ -28,6 +28,8 @@ class DescriptionPopoverButton extends StatelessWidget { final ExampleModel example; final Alignment followerAnchor; final Alignment targetAnchor; + final void Function()? onOpen; + final void Function()? onClose; const DescriptionPopoverButton({ Key? key, @@ -35,6 +37,8 @@ class DescriptionPopoverButton extends StatelessWidget { required this.example, required this.followerAnchor, required this.targetAnchor, + this.onOpen, + this.onClose, }) : super(key: key); @override @@ -62,12 +66,15 @@ class DescriptionPopoverButton extends StatelessWidget { ExampleModel example, Alignment followerAnchor, Alignment targetAnchor, - ) { + ) async { // close previous description dialog Navigator.of(context, rootNavigator: true).popUntil((route) { return route.isFirst; }); - showAlignedDialog( + if (onOpen != null) { + onOpen!(); + } + await showAlignedDialog( context: context, builder: (dialogContext) => DescriptionPopover( example: example, @@ -76,5 +83,8 @@ class DescriptionPopoverButton extends StatelessWidget { targetAnchor: targetAnchor, barrierColor: Colors.transparent, ); + if (onClose != null) { + onClose!(); + } } } diff --git a/playground/frontend/lib/modules/examples/components/example_list/example_item_actions.dart b/playground/frontend/lib/modules/examples/components/example_list/example_item_actions.dart new file mode 100644 index 0000000000000..9cae77f6073b4 --- /dev/null +++ b/playground/frontend/lib/modules/examples/components/example_list/example_item_actions.dart @@ -0,0 +1,66 @@ +/* + * 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. + */ + +import 'package:flutter/material.dart'; +import 'package:playground/modules/examples/components/description_popover/description_popover_button.dart'; +import 'package:playground/modules/examples/models/example_model.dart'; +import 'package:playground/modules/examples/models/popover_state.dart'; +import 'package:provider/provider.dart'; + +import '../multifile_popover/multifile_popover_button.dart'; + +class ExampleItemActions extends StatelessWidget { + final ExampleModel example; + final BuildContext parentContext; + + const ExampleItemActions( + {Key? key, required this.parentContext, required this.example}) + : super(key: key); + + @override + Widget build(BuildContext context) { + return Row( + children: [ + if (example.isMultiFile) multifilePopover, + descriptionPopover, + ], + ); + } + + Widget get multifilePopover => MultifilePopoverButton( + parentContext: parentContext, + example: example, + followerAnchor: Alignment.topLeft, + targetAnchor: Alignment.topRight, + onOpen: () => _setPopoverOpen(parentContext, true), + onClose: () => _setPopoverOpen(parentContext, false), + ); + + Widget get descriptionPopover => DescriptionPopoverButton( + parentContext: parentContext, + example: example, + followerAnchor: Alignment.topLeft, + targetAnchor: Alignment.topRight, + onOpen: () => _setPopoverOpen(parentContext, true), + onClose: () => _setPopoverOpen(parentContext, false), + ); + + void _setPopoverOpen(BuildContext context, bool isOpen) { + Provider.of(context, listen: false).setOpen(isOpen); + } +} diff --git a/playground/frontend/lib/modules/examples/components/example_list/expansion_panel_item.dart b/playground/frontend/lib/modules/examples/components/example_list/expansion_panel_item.dart index 74a6e23fb53df..7f29c441cd140 100644 --- a/playground/frontend/lib/modules/examples/components/example_list/expansion_panel_item.dart +++ b/playground/frontend/lib/modules/examples/components/example_list/expansion_panel_item.dart @@ -19,7 +19,7 @@ import 'package:flutter/material.dart'; import 'package:playground/constants/sizes.dart'; import 'package:playground/modules/analytics/analytics_service.dart'; -import 'package:playground/modules/examples/components/description_popover/description_popover_button.dart'; +import 'package:playground/modules/examples/components/example_list/example_item_actions.dart'; import 'package:playground/modules/examples/models/example_model.dart'; import 'package:playground/pages/playground/states/examples_state.dart'; import 'package:playground/pages/playground/states/playground_state.dart'; @@ -72,12 +72,7 @@ class ExpansionPanelItem extends StatelessWidget { ? const TextStyle(fontWeight: FontWeight.bold) : const TextStyle(), ), - DescriptionPopoverButton( - parentContext: context, - example: example, - followerAnchor: Alignment.topLeft, - targetAnchor: Alignment.topRight, - ), + ExampleItemActions(parentContext: context, example: example), ], ), ), diff --git a/playground/frontend/lib/modules/examples/components/multifile_popover/multifile_popover.dart b/playground/frontend/lib/modules/examples/components/multifile_popover/multifile_popover.dart new file mode 100644 index 0000000000000..00db2d71ecf47 --- /dev/null +++ b/playground/frontend/lib/modules/examples/components/multifile_popover/multifile_popover.dart @@ -0,0 +1,67 @@ +/* + * 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. + */ + +import 'package:flutter/material.dart'; +import 'package:flutter_svg/flutter_svg.dart'; +import 'package:flutter_gen/gen_l10n/app_localizations.dart'; +import 'package:playground/constants/assets.dart'; +import 'package:playground/constants/font_weight.dart'; +import 'package:playground/constants/sizes.dart'; +import 'package:playground/modules/examples/models/example_model.dart'; +import 'package:url_launcher/url_launcher.dart'; + +const kMultifileWidth = 300.0; + +class MultifilePopover extends StatelessWidget { + final ExampleModel example; + + const MultifilePopover({Key? key, required this.example}) : super(key: key); + + @override + Widget build(BuildContext context) { + AppLocalizations appLocale = AppLocalizations.of(context)!; + return SizedBox( + width: kMultifileWidth, + child: Card( + child: Padding( + padding: const EdgeInsets.all(kLgSpacing), + child: Wrap( + runSpacing: kMdSpacing, + children: [ + Text( + appLocale.multifile, + style: const TextStyle( + fontSize: kTitleFontSize, + fontWeight: kBoldWeight, + ), + ), + Text(appLocale.multifileWarning), + TextButton.icon( + icon: SvgPicture.asset(kGithubIconAsset), + onPressed: () { + launch(example.link ?? ''); + }, + label: Text(appLocale.viewOnGithub), + ), + ], + ), + ), + ), + ); + } +} diff --git a/playground/frontend/lib/modules/examples/components/multifile_popover/multifile_popover_button.dart b/playground/frontend/lib/modules/examples/components/multifile_popover/multifile_popover_button.dart new file mode 100644 index 0000000000000..149194dfafd4f --- /dev/null +++ b/playground/frontend/lib/modules/examples/components/multifile_popover/multifile_popover_button.dart @@ -0,0 +1,88 @@ +/* + * 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. + */ + +import 'package:aligned_dialog/aligned_dialog.dart'; +import 'package:flutter/material.dart'; +import 'package:flutter_svg/flutter_svg.dart'; +import 'package:playground/constants/assets.dart'; +import 'package:playground/constants/sizes.dart'; +import 'package:playground/modules/examples/components/multifile_popover/multifile_popover.dart'; +import 'package:playground/modules/examples/models/example_model.dart'; + +class MultifilePopoverButton extends StatelessWidget { + final BuildContext? parentContext; + final ExampleModel example; + final Alignment followerAnchor; + final Alignment targetAnchor; + final void Function()? onOpen; + final void Function()? onClose; + + const MultifilePopoverButton({ + Key? key, + this.parentContext, + required this.example, + required this.followerAnchor, + required this.targetAnchor, + this.onOpen, + this.onClose, + }) : super(key: key); + + @override + Widget build(BuildContext context) { + return IconButton( + iconSize: kIconSizeMd, + splashRadius: kIconButtonSplashRadius, + icon: SvgPicture.asset(kMultifileIconAsset), + onPressed: () { + _showMultifilePopover( + parentContext ?? context, + example, + followerAnchor, + targetAnchor, + ); + }, + ); + } + + void _showMultifilePopover( + BuildContext context, + ExampleModel example, + Alignment followerAnchor, + Alignment targetAnchor, + ) async { + // close previous dialogs + Navigator.of(context, rootNavigator: true).popUntil((route) { + return route.isFirst; + }); + if (onOpen != null) { + onOpen!(); + } + await showAlignedDialog( + context: context, + builder: (dialogContext) => MultifilePopover( + example: example, + ), + followerAnchor: followerAnchor, + targetAnchor: targetAnchor, + barrierColor: Colors.transparent, + ); + if (onClose != null) { + onClose!(); + } + } +} diff --git a/playground/frontend/lib/modules/examples/components/outside_click_handler.dart b/playground/frontend/lib/modules/examples/components/outside_click_handler.dart new file mode 100644 index 0000000000000..16db300254fa4 --- /dev/null +++ b/playground/frontend/lib/modules/examples/components/outside_click_handler.dart @@ -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. + */ + +import 'package:flutter/material.dart'; +import 'package:playground/modules/examples/models/popover_state.dart'; +import 'package:provider/provider.dart'; + +class OutsideClickHandler extends StatelessWidget { + final void Function() onTap; + + const OutsideClickHandler({Key? key, required this.onTap}) : super(key: key); + + @override + Widget build(BuildContext context) { + return Consumer(builder: (context, state, child) { + if (state.isOpen) { + return Container(); + } + return GestureDetector( + onTap: onTap, + child: Container( + color: Colors.transparent, + height: double.infinity, + width: double.infinity, + ), + ); + }); + } +} diff --git a/playground/frontend/lib/modules/examples/example_selector.dart b/playground/frontend/lib/modules/examples/example_selector.dart index 1994a65d612ca..8824d6e3af384 100644 --- a/playground/frontend/lib/modules/examples/example_selector.dart +++ b/playground/frontend/lib/modules/examples/example_selector.dart @@ -17,21 +17,26 @@ */ import 'package:flutter/material.dart'; +import 'package:flutter_gen/gen_l10n/app_localizations.dart'; import 'package:playground/components/loading_indicator/loading_indicator.dart'; import 'package:playground/config/theme.dart'; +import 'package:playground/constants/links.dart'; import 'package:playground/constants/sizes.dart'; import 'package:playground/modules/examples/components/examples_components.dart'; +import 'package:playground/modules/examples/components/outside_click_handler.dart'; +import 'package:playground/modules/examples/models/popover_state.dart'; import 'package:playground/modules/examples/models/selector_size_model.dart'; import 'package:playground/pages/playground/states/example_selector_state.dart'; import 'package:playground/pages/playground/states/examples_state.dart'; import 'package:playground/pages/playground/states/playground_state.dart'; import 'package:provider/provider.dart'; +import 'package:url_launcher/url_launcher.dart'; const int kAnimationDurationInMilliseconds = 80; const Offset kAnimationBeginOffset = Offset(0.0, -0.02); const Offset kAnimationEndOffset = Offset(0.0, 0.0); const double kAdditionalDyAlignment = 50.0; -const double kLgContainerHeight = 444.0; +const double kLgContainerHeight = 490.0; const double kLgContainerWidth = 400.0; class ExampleSelector extends StatefulWidget { @@ -121,60 +126,63 @@ class _ExampleSelectorState extends State return OverlayEntry( builder: (context) { - return Consumer2( - builder: (context, exampleState, playgroundState, child) => Stack( - children: [ - GestureDetector( - onTap: () { - closeDropdown(exampleState); - // handle description dialogs - Navigator.of(context, rootNavigator: true).popUntil((route) { - return route.isFirst; - }); - }, - child: Container( - color: Colors.transparent, - height: double.infinity, - width: double.infinity, - ), - ), - ChangeNotifierProvider( - create: (context) => ExampleSelectorState( - exampleState, - playgroundState, - exampleState.getCategories(playgroundState.sdk)!, - ), - builder: (context, _) => Positioned( - left: posModel.xAlignment, - top: posModel.yAlignment + kAdditionalDyAlignment, - child: SlideTransition( - position: offsetAnimation, - child: Material( - elevation: kElevation.toDouble(), - child: Container( - height: kLgContainerHeight, - width: kLgContainerWidth, - decoration: BoxDecoration( - color: Theme.of(context).backgroundColor, - borderRadius: BorderRadius.circular(kMdBorderRadius), + return ChangeNotifierProvider( + create: (context) => PopoverState(false), + builder: (context, state) { + return Consumer2( + builder: (context, exampleState, playgroundState, child) => Stack( + children: [ + OutsideClickHandler( + onTap: () { + closeDropdown(exampleState); + // handle description dialogs + Navigator.of(context, rootNavigator: true).popUntil((route) { + return route.isFirst; + }); + }, + ), + ChangeNotifierProvider( + create: (context) => ExampleSelectorState( + exampleState, + playgroundState, + exampleState.getCategories(playgroundState.sdk)!, + ), + builder: (context, _) => Positioned( + left: posModel.xAlignment, + top: posModel.yAlignment + kAdditionalDyAlignment, + child: SlideTransition( + position: offsetAnimation, + child: Material( + elevation: kElevation.toDouble(), + child: Container( + height: kLgContainerHeight, + width: kLgContainerWidth, + decoration: BoxDecoration( + color: Theme.of(context).backgroundColor, + borderRadius: BorderRadius.circular(kMdBorderRadius), + ), + child: exampleState.sdkCategories == null || + playgroundState.selectedExample == null + ? const LoadingIndicator(size: kContainerHeight) + : _buildDropdownContent(context, playgroundState), + ), ), - child: exampleState.sdkCategories == null || - playgroundState.selectedExample == null - ? const LoadingIndicator(size: kContainerHeight) - : _buildDropdownContent(playgroundState), ), ), ), - ), + ], ), - ], - ), + ); + } ); }, ); } - Widget _buildDropdownContent(PlaygroundState playgroundState) { + Widget _buildDropdownContent( + BuildContext context, + PlaygroundState playgroundState, + ) { return Column( children: [ SearchField(controller: textController), @@ -185,6 +193,28 @@ class _ExampleSelectorState extends State animationController: animationController, dropdown: examplesDropdown, ), + Divider( + height: kDividerHeight, + color: ThemeColors.of(context).greyColor, + indent: kLgSpacing, + endIndent: kLgSpacing, + ), + SizedBox( + width: double.infinity, + child: TextButton( + child: Padding( + padding: const EdgeInsets.all(kXlSpacing), + child: Align( + alignment: Alignment.centerLeft, + child: Text( + AppLocalizations.of(context)!.addExample, + style: TextStyle(color: ThemeColors.of(context).primary), + ), + ), + ), + onPressed: () => launch(kAddExampleLink), + ), + ) ], ); } diff --git a/playground/frontend/lib/modules/examples/models/example_model.dart b/playground/frontend/lib/modules/examples/models/example_model.dart index ff95440427d36..e6f850ae4815e 100644 --- a/playground/frontend/lib/modules/examples/models/example_model.dart +++ b/playground/frontend/lib/modules/examples/models/example_model.dart @@ -43,6 +43,8 @@ class ExampleModel with Comparable { final String name; final String path; final String description; + bool isMultiFile; + String? link; String? source; String? outputs; String? logs; @@ -54,6 +56,8 @@ class ExampleModel with Comparable { required this.path, required this.description, required this.type, + this.isMultiFile = false, + this.link, this.source, this.outputs, this.logs, diff --git a/playground/frontend/lib/modules/examples/models/popover_state.dart b/playground/frontend/lib/modules/examples/models/popover_state.dart new file mode 100644 index 0000000000000..af51052cb79f0 --- /dev/null +++ b/playground/frontend/lib/modules/examples/models/popover_state.dart @@ -0,0 +1,32 @@ +/* + * 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. + */ + +import 'package:flutter/cupertino.dart'; + +class PopoverState extends ChangeNotifier { + bool _isOpen; + + bool get isOpen => _isOpen; + + PopoverState(this._isOpen); + + setOpen(bool open) { + _isOpen = open; + notifyListeners(); + } +} diff --git a/playground/frontend/lib/modules/examples/repositories/example_client/grpc_example_client.dart b/playground/frontend/lib/modules/examples/repositories/example_client/grpc_example_client.dart index 2b91d915fab1c..672b2e52cd6cb 100644 --- a/playground/frontend/lib/modules/examples/repositories/example_client/grpc_example_client.dart +++ b/playground/frontend/lib/modules/examples/repositories/example_client/grpc_example_client.dart @@ -224,6 +224,8 @@ class GrpcExampleClient implements ExampleClient { type: _exampleTypeFromString(example.type), path: example.cloudPath, pipelineOptions: example.pipelineOptions, + isMultiFile: example.multifile, + link: example.link, ); } } diff --git a/playground/frontend/lib/modules/output/components/output.dart b/playground/frontend/lib/modules/output/components/output.dart index e3d6a5c98993c..0cbf7e311d524 100644 --- a/playground/frontend/lib/modules/output/components/output.dart +++ b/playground/frontend/lib/modules/output/components/output.dart @@ -64,7 +64,7 @@ class _OutputState extends State with SingleTickerProviderStateMixin { children: [ OutputHeader( tabController: tabController, - showOutputPlacements: widget.isEmbedded, + showOutputPlacements: !widget.isEmbedded, showGraph: widget.showGraph, ), Expanded( diff --git a/playground/frontend/lib/pages/embedded_playground/components/embedded_appbar_title.dart b/playground/frontend/lib/pages/embedded_playground/components/embedded_appbar_title.dart index 07ad337f1574b..703f765c62091 100644 --- a/playground/frontend/lib/pages/embedded_playground/components/embedded_appbar_title.dart +++ b/playground/frontend/lib/pages/embedded_playground/components/embedded_appbar_title.dart @@ -26,8 +26,8 @@ import 'package:playground/constants/sizes.dart'; import 'package:playground/modules/analytics/analytics_service.dart'; import 'package:playground/modules/editor/components/run_button.dart'; import 'package:playground/modules/notifications/components/notification.dart'; -import 'package:playground/modules/sdk/models/sdk.dart'; import 'package:playground/pages/playground/states/playground_state.dart'; +import 'package:playground/utils/analytics_utils.dart'; import 'package:provider/provider.dart'; class EmbeddedAppBarTitle extends StatelessWidget { @@ -53,18 +53,20 @@ class EmbeddedAppBarTitle extends StatelessWidget { }, runCode: () { final stopwatch = Stopwatch()..start(); + final exampleName = getAnalyticsExampleName( + state.selectedExample, + state.isExampleChanged, + state.sdk, + ); state.runCode( onFinish: () { AnalyticsService.get(context).trackRunTimeEvent( - state.selectedExample?.path ?? - '${AppLocalizations.of(context)!.unknownExample}, sdk ${state.sdk.displayName}', + exampleName, stopwatch.elapsedMilliseconds, ); }, ); - AnalyticsService.get(context).trackClickRunEvent( - state.selectedExample, - ); + AnalyticsService.get(context).trackClickRunEvent(exampleName); }, ), const ToggleThemeIconButton(), diff --git a/playground/frontend/lib/pages/playground/components/editor_textarea_wrapper.dart b/playground/frontend/lib/pages/playground/components/editor_textarea_wrapper.dart index c7182ddcefc43..4f88f3d4204ce 100644 --- a/playground/frontend/lib/pages/playground/components/editor_textarea_wrapper.dart +++ b/playground/frontend/lib/pages/playground/components/editor_textarea_wrapper.dart @@ -23,10 +23,12 @@ import 'package:playground/modules/analytics/analytics_service.dart'; import 'package:playground/modules/editor/components/editor_textarea.dart'; import 'package:playground/modules/editor/components/run_button.dart'; import 'package:playground/modules/examples/components/description_popover/description_popover_button.dart'; +import 'package:playground/modules/examples/components/multifile_popover/multifile_popover_button.dart'; import 'package:playground/modules/examples/models/example_model.dart'; import 'package:playground/modules/notifications/components/notification.dart'; import 'package:playground/modules/sdk/models/sdk.dart'; import 'package:playground/pages/playground/states/playground_state.dart'; +import 'package:playground/utils/analytics_utils.dart'; import 'package:provider/provider.dart'; class CodeTextAreaWrapper extends StatelessWidget { @@ -52,7 +54,7 @@ class CodeTextAreaWrapper extends StatelessWidget { children: [ Positioned.fill( child: EditorTextArea( - enabled: true, + enabled: !(state.selectedExample?.isMultiFile ?? false), example: state.selectedExample, sdk: state.sdk, onSourceChange: state.setSource, @@ -65,13 +67,21 @@ class CodeTextAreaWrapper extends StatelessWidget { height: kButtonHeight, child: Row( children: [ - if (state.selectedExample != null) + if (state.selectedExample != null) ...[ + if (state.selectedExample?.isMultiFile ?? false) + MultifilePopoverButton( + example: state.selectedExample!, + followerAnchor: Alignment.topRight, + targetAnchor: Alignment.bottomRight, + ), DescriptionPopoverButton( example: state.selectedExample!, followerAnchor: Alignment.topRight, targetAnchor: Alignment.bottomRight, ), + ], RunButton( + disabled: state.selectedExample?.isMultiFile ?? false, isRunning: state.isCodeRunning, cancelRun: () { state.cancelRun().catchError( @@ -86,17 +96,21 @@ class CodeTextAreaWrapper extends StatelessWidget { AnalyticsService analyticsService = AnalyticsService.get(context); final stopwatch = Stopwatch()..start(); + final exampleName = getAnalyticsExampleName( + state.selectedExample, + state.isExampleChanged, + state.sdk, + ); state.runCode( onFinish: () { analyticsService.trackRunTimeEvent( - state.selectedExample?.path ?? - '${AppLocalizations.of(context)!.unknownExample}, sdk ${state.sdk.displayName}', + exampleName, stopwatch.elapsedMilliseconds, ); }, ); AnalyticsService.get(context) - .trackClickRunEvent(state.selectedExample); + .trackClickRunEvent(exampleName); }, ), ], diff --git a/playground/frontend/lib/pages/playground/states/playground_state.dart b/playground/frontend/lib/pages/playground/states/playground_state.dart index 8d08e2fc91122..22f94849afe2a 100644 --- a/playground/frontend/lib/pages/playground/states/playground_state.dart +++ b/playground/frontend/lib/pages/playground/states/playground_state.dart @@ -79,6 +79,14 @@ class PlaygroundState with ChangeNotifier { Stream? get executionTime => _executionTime?.stream; + bool get isExampleChanged { + return selectedExample?.source != source || _arePipelineOptionsChanges; + } + + bool get _arePipelineOptionsChanges { + return pipelineOptions != (_selectedExample?.pipelineOptions ?? ''); + } + bool get graphAvailable => selectedExample?.type != ExampleType.test && [SDK.java, SDK.python].contains(sdk); @@ -139,9 +147,7 @@ class PlaygroundState with ChangeNotifier { } _executionTime?.close(); _executionTime = _createExecutionTimeStream(); - if (_selectedExample?.source == source && - _selectedExample?.outputs != null && - !_arePipelineOptionsChanges) { + if (!isExampleChanged && _selectedExample?.outputs != null) { _showPrecompiledResult(); } else { final request = RunCodeRequestWrapper( @@ -177,10 +183,6 @@ class PlaygroundState with ChangeNotifier { notifyListeners(); } - bool get _arePipelineOptionsChanges { - return pipelineOptions != (_selectedExample?.pipelineOptions ?? ''); - } - _showPrecompiledResult() async { _result = RunCodeResult( status: RunCodeStatus.preparation, diff --git a/playground/frontend/lib/utils/analytics_utils.dart b/playground/frontend/lib/utils/analytics_utils.dart new file mode 100644 index 0000000000000..a9c2081cbea9c --- /dev/null +++ b/playground/frontend/lib/utils/analytics_utils.dart @@ -0,0 +1,28 @@ +/* + * 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. + */ + +import 'package:playground/modules/examples/models/example_model.dart'; +import 'package:playground/modules/sdk/models/sdk.dart'; + +String getAnalyticsExampleName(ExampleModel? example, bool isExampleChanged, SDK sdk) { + final customCodeName = 'Custom code, sdk ${sdk.displayName}'; + if (isExampleChanged) { + return customCodeName; + } + return example?.path ?? customCodeName; +} diff --git a/playground/terraform/README.md b/playground/terraform/README.md index b9afe92fcd2a1..fc24b151b0f54 100644 --- a/playground/terraform/README.md +++ b/playground/terraform/README.md @@ -17,91 +17,25 @@ under the License. --> -## Requirements and setup +# Requirements -In order install a Playground cluster on GCP using Terraform, you'll need: +The following items need to be setup for the Playground cluster deployment on GCP: -* An [GCP account](https://cloud.google.com/) and the [`gcloud`](https://cloud.google.com/sdk/gcloud) command-line tool +* [GCP account](https://cloud.google.com/) +* [`gcloud` command-line tool](https://cloud.google.com/sdk/gcloud) and required setup i.e. login * [Terraform](https://www.terraform.io/downloads.html) tool +* [Docker](https://www.docker.com/get-started) -You'll also need to make sure that you're currently logged into your GCP account via the `gcloud` tool: +# Deployment steps -```bash -$ gcloud auth login -``` -In other case you'll need an environment variable `GOOGLE_APPLICATION_CREDENTIALS` set to JSON key for service account that will be used to deploy resources +## 1. Provision infrastructure -## Installation +To deploy Playground infrastructure follow [README.md](./infrastructure/README.md) for infrastructure module. -You can install Terraform using the instructions [here](https://www.terraform.io/intro/getting-started/install.html). +## 2. Build containers +TBD +## 3. Deploy application -## Creating GCP resources using Terraform - -To get started building GCP resources with Terraform, you'll need to install all Terraform dependencies: - -```bash -$ terraform init -# This will create a .terraform folder -``` - -Once you've done that, you can apply the default Terraform configuration: - -```bash -$ terraform apply -``` - -You should then see this prompt: - -```bash -Do you want to perform these actions? - Terraform will perform the actions described above. - Only 'yes' will be accepted to approve. - - Enter a value: -``` - -Type `yes` and hit **Enter**. Applying the configuration could take several minutes. When it's finished, you should see `Apply complete!` along with some other information, including the number of resources created. - -### Applying a non-default configuration - -You can apply a non-default Terraform configuration by changing the values in the `terraform.tfvars` file. The following variables are available: - -Variable name | Description | Default -:-------------|:------------|:------- -`project_id` | GCP Project ID that will be used to create resources | None -`docker_registry_address` | Address of docker registry | None -`docker_image_name` | Docker Image Name To Be Deployed | `beam_playground-backend`/`beam_playground-frontend` -`docker_image_tag` | Docker Image Tag To Be Deployed | `latest` -`repository_location` | Location of Artifact Registry | `us-central1` -`repository_id` | ID of Artifact Registry | `playground_repository` -`examples_bucket_name` | Name of Bucket to Store Playground Examples | `playground-examples` -`examples_bucket_location` | Location of Playground Examples Bucket | `US` -`examples_storage_class` | Examples Bucket Storage Class | `STANDARD` -`terraform_bucket_name` | Name of Bucket to Store Terraform States | `playground-terraform` -`terraform_bucket_location` | Location of Playground Terraform Bucket | `US` -`terraform_storage_class` | Terraform Bucket Storage Class | `STANDARD` -`vpc_name` | Name of VPC to be created | `playground-vpc` -`create_subnets` | Auto Create Subnets Inside VPC | `true` -`mtu` | MTU Inside VPC | `1460` - -### What is installed - -After applying terraform following resources will be created: - -* GCP [Artifact Registry](https://cloud.google.com/artifact-registry) used to store application docker files: -* VPC to run GCP App Engine VM's ([VPC](https://cloud.google.com/vpc)) -* 2 buckets to store Examples for Playground Application and Terraform states [buckets](https://cloud.google.com/storage/docs/key-terms#buckets) -* 2 GCP [App Engine](https://cloud.google.com/appengine) services for backend and frontend applications - - -### Destroying your cluster - -At any point, you can destroy all GCP resources associated with your cluster using Terraform's `destroy` command: - -```bash -$ terraform destroy -``` - - +TBD diff --git a/playground/terraform/applications/README.md b/playground/terraform/applications/README.md new file mode 100644 index 0000000000000..84c1f3fa43c8b --- /dev/null +++ b/playground/terraform/applications/README.md @@ -0,0 +1,59 @@ + + +## Deployment to AppEngine + +*Note: All requirements are listed in the [README.md](../README.md) of the Terraform module.* + +Installation of all Terraform dependencies is required to get started building GCP resources with Terraform: + +```bash +$ terraform init +# This will create a .terraform folder +``` + +Once it has been done, default Terraform configuration can be applied: + +```bash +$ terraform apply +``` + +Then the following dialog will be displayed in the console: + +```bash +Do you want to perform these actions? + Terraform will perform the actions described above. + Only 'yes' will be accepted to approve. + + Enter a value: +``` + +Type `yes` and hit **Enter**. Applying of the configuration could take several minutes. `Apply complete!` will be displayed +when it is finished, along with the number of created resources. + +### Applying a non-default configuration + +To apply non-default Terraform configuration, pass the corresponding values as a variables with `terraform apply` command. +All variables are listed in the [variables.tf (TBD)](variables.tf) file. + +### What is installed + +TBD + + diff --git a/playground/terraform/applications/backend-go/main.tf b/playground/terraform/applications/backend-go/main.tf index 47d68476954a5..48b2759fe1c00 100644 --- a/playground/terraform/applications/backend-go/main.tf +++ b/playground/terraform/applications/backend-go/main.tf @@ -18,10 +18,10 @@ # resource "google_app_engine_flexible_app_version" "backend_app_go" { - version_id = "v1" - project = "${var.project_id}" - service = "${var.service_name}" - runtime = "custom" + version_id = "v1" + project = var.project_id + service = var.service_name + runtime = "custom" delete_service_on_destroy = true liveness_check { @@ -35,7 +35,7 @@ resource "google_app_engine_flexible_app_version" "backend_app_go" { automatic_scaling { max_total_instances = 7 min_total_instances = 2 - cool_down_period = "120s" + cool_down_period = "120s" cpu_utilization { target_utilization = 0.7 } @@ -43,14 +43,14 @@ resource "google_app_engine_flexible_app_version" "backend_app_go" { resources { memory_gb = 16 - cpu = 8 + cpu = 8 } env_variables = { - CACHE_TYPE="${var.cache_type}" - CACHE_ADDRESS="${var.cache_address}:6379" - NUM_PARALLEL_JOBS=30 - LAUNCH_SITE = "app_engine" + CACHE_TYPE = var.cache_type + CACHE_ADDRESS = "${var.cache_address}:6379" + NUM_PARALLEL_JOBS = 30 + LAUNCH_SITE = "app_engine" } deployment { diff --git a/playground/terraform/applications/backend-go/variables.tf b/playground/terraform/applications/backend-go/variables.tf index 52df044a879a4..6a69ef630bed3 100644 --- a/playground/terraform/applications/backend-go/variables.tf +++ b/playground/terraform/applications/backend-go/variables.tf @@ -32,7 +32,7 @@ variable "docker_image_name" { variable "docker_image_tag" { description = "Docker Image Tag To Be Deployed" - default = "latest" + default = "latest" } variable "service_name" { diff --git a/playground/terraform/applications/backend-java/main.tf b/playground/terraform/applications/backend-java/main.tf index f2c47011cb255..e7dc97fec4d77 100644 --- a/playground/terraform/applications/backend-java/main.tf +++ b/playground/terraform/applications/backend-java/main.tf @@ -18,16 +18,16 @@ # resource "google_app_engine_flexible_app_version" "backend_app" { - version_id = "v1" - project = "${var.project_id}" - service = "${var.service_name}" - runtime = "custom" + version_id = "v1" + project = var.project_id + service = var.service_name + runtime = "custom" delete_service_on_destroy = true - - + + liveness_check { - path = "/liveness" - initial_delay = "40s" + path = "/liveness" + initial_delay = "40s" } readiness_check { @@ -44,15 +44,15 @@ resource "google_app_engine_flexible_app_version" "backend_app" { } env_variables = { - CACHE_TYPE="${var.cache_type}" - CACHE_ADDRESS="${var.cache_address}:6379" - NUM_PARALLEL_JOBS=10 - LAUNCH_SITE = "app_engine" + CACHE_TYPE = var.cache_type + CACHE_ADDRESS = "${var.cache_address}:6379" + NUM_PARALLEL_JOBS = 10 + LAUNCH_SITE = "app_engine" } resources { memory_gb = 16 - cpu = 8 + cpu = 8 volumes { name = "inmemory" size_gb = var.volume_size diff --git a/playground/terraform/applications/backend-java/variables.tf b/playground/terraform/applications/backend-java/variables.tf index 62bc438b65b8b..0330aa0af6a0e 100644 --- a/playground/terraform/applications/backend-java/variables.tf +++ b/playground/terraform/applications/backend-java/variables.tf @@ -32,19 +32,19 @@ variable "docker_image_name" { variable "docker_image_tag" { description = "Docker Image Tag To Be Deployed" - default = "latest" + default = "latest" } variable "memory_size" { description = "RAM in GB. The requested memory for the application" - type = number - default = 2 + type = number + default = 2 } variable "volume_size" { description = "Size of the in memory file system to be used by the application, in GB" - type = number - default = 1 + type = number + default = 1 } variable "service_name" { diff --git a/playground/terraform/applications/backend-python/main.tf b/playground/terraform/applications/backend-python/main.tf index 73ed1701a1c3c..e4d71db4047ac 100644 --- a/playground/terraform/applications/backend-python/main.tf +++ b/playground/terraform/applications/backend-python/main.tf @@ -18,24 +18,26 @@ # resource "google_app_engine_flexible_app_version" "backend_app_python" { - version_id = "v1" - project = "${var.project_id}" - service = "${var.service_name}" - runtime = "custom" + version_id = "v1" + project = var.project_id + service = var.service_name + runtime = "custom" delete_service_on_destroy = true liveness_check { - path = "" + path = "/liveness" + initial_delay = "40s" } readiness_check { path = "/readiness" } + automatic_scaling { max_total_instances = 7 min_total_instances = 2 - cool_down_period = "120s" + cool_down_period = "120s" cpu_utilization { target_utilization = 0.7 } @@ -43,14 +45,14 @@ resource "google_app_engine_flexible_app_version" "backend_app_python" { resources { memory_gb = 16 - cpu = 8 + cpu = 8 } env_variables = { - CACHE_TYPE="${var.cache_type}" - CACHE_ADDRESS="${var.cache_address}:6379" - NUM_PARALLEL_JOBS=70 - LAUNCH_SITE = "app_engine" + CACHE_TYPE = var.cache_type + CACHE_ADDRESS = "${var.cache_address}:6379" + NUM_PARALLEL_JOBS = 70 + LAUNCH_SITE = "app_engine" } deployment { diff --git a/playground/terraform/applications/backend-python/variables.tf b/playground/terraform/applications/backend-python/variables.tf index 4a309e47c9a58..518f20b14b785 100644 --- a/playground/terraform/applications/backend-python/variables.tf +++ b/playground/terraform/applications/backend-python/variables.tf @@ -27,12 +27,12 @@ variable "docker_registry_address" { variable "docker_image_name" { description = "Docker Image Name To Be Deployed" - default = "beam_playground-backend-python" + default = "beam_playground-backend-python" } variable "docker_image_tag" { description = "Docker Image Tag To Be Deployed" - default = "latest" + default = "latest" } variable "service_name" { diff --git a/playground/terraform/applications/backend-router/main.tf b/playground/terraform/applications/backend-router/main.tf index 92cc652bfd9d9..7227b4b4367b2 100644 --- a/playground/terraform/applications/backend-router/main.tf +++ b/playground/terraform/applications/backend-router/main.tf @@ -18,14 +18,15 @@ # resource "google_app_engine_flexible_app_version" "backend_app_router" { - version_id = "v1" - project = "${var.project_id}" - service = "${var.service_name}" - runtime = "custom" + version_id = "v1" + project = var.project_id + service = var.service_name + runtime = "custom" delete_service_on_destroy = true liveness_check { - path = "" + path = "/liveness" + initial_delay = "40s" } readiness_check { @@ -34,7 +35,7 @@ resource "google_app_engine_flexible_app_version" "backend_app_router" { automatic_scaling { max_total_instances = 3 - min_total_instances = 2 + min_total_instances = 1 cool_down_period = "120s" cpu_utilization { target_utilization = 0.7 @@ -42,17 +43,14 @@ resource "google_app_engine_flexible_app_version" "backend_app_router" { } resources { - memory_gb = 16 - cpu = 8 + memory_gb = 4 + cpu = 2 } env_variables = { - CACHE_TYPE="${var.cache_type}" - CACHE_ADDRESS="${var.cache_address}:6379" - NUM_PARALLEL_JOBS=30 - LAUNCH_SITE = "app_engine" - PIPELINE_EXPIRATION_TIMEOUT = "5m" - KEY_EXPIRATION_TIME = "7m" + CACHE_TYPE = var.cache_type + CACHE_ADDRESS = "${var.cache_address}:6379" + LAUNCH_SITE = "app_engine" } deployment { diff --git a/playground/terraform/applications/backend-router/variables.tf b/playground/terraform/applications/backend-router/variables.tf index 8211843dd7c82..37f693569cd89 100644 --- a/playground/terraform/applications/backend-router/variables.tf +++ b/playground/terraform/applications/backend-router/variables.tf @@ -32,7 +32,7 @@ variable "docker_image_name" { variable "docker_image_tag" { description = "Docker Image Tag To Be Deployed" - default = "latest" + default = "latest" } variable "service_name" { diff --git a/playground/terraform/applications/frontend/main.tf b/playground/terraform/applications/frontend/main.tf index d8549d62f69b4..91f75b6dac44d 100644 --- a/playground/terraform/applications/frontend/main.tf +++ b/playground/terraform/applications/frontend/main.tf @@ -18,10 +18,10 @@ # resource "google_app_engine_flexible_app_version" "frontend_app" { - version_id = "v1" - project = "${var.project_id}" - service = "${var.service_name}" - runtime = "custom" + version_id = "v1" + project = var.project_id + service = var.service_name + runtime = "custom" delete_service_on_destroy = true liveness_check { diff --git a/playground/terraform/applications/frontend/variables.tf b/playground/terraform/applications/frontend/variables.tf index 1e5b2a27b175c..20317903651ef 100644 --- a/playground/terraform/applications/frontend/variables.tf +++ b/playground/terraform/applications/frontend/variables.tf @@ -32,7 +32,7 @@ variable "docker_image_name" { variable "docker_image_tag" { description = "Docker Image Tag To Be Deployed" - default = "latest" + default = "latest" } variable "service_name" { diff --git a/playground/terraform/infrastructure/README.md b/playground/terraform/infrastructure/README.md new file mode 100644 index 0000000000000..90a522316b445 --- /dev/null +++ b/playground/terraform/infrastructure/README.md @@ -0,0 +1,70 @@ + + +# Overview + +This directory provisions required infrastructure for the application. + +# Requirements + +See [playground/README.md](../README.md) for a list of the requirements +prior to following these instructions. + +# Usage + +## Terraform init + +Follow conventional terraform workflow to build this solution. +You will be prompted for required variables. +Alternatively, you may create a `vars.tfvars` file and +apply the `-var-file=vars.tfvars` flag. + +Initialize the terraform environment. + +``` +terraform init +``` + +## Terraform plan + +Plan the terraform solution. + +``` +terraform plan +``` + +or + +``` +terraform plan -var-file=vars.tfvars +``` + +## Terraform apply + +Apply the terraform solution. + +``` +terraform apply +``` + +or + +``` +terraform apply -var-file=vars.tfvars +``` \ No newline at end of file diff --git a/playground/terraform/modules/artifact_registry/main.tf b/playground/terraform/infrastructure/artifact_registry/main.tf similarity index 79% rename from playground/terraform/modules/artifact_registry/main.tf rename to playground/terraform/infrastructure/artifact_registry/main.tf index 6bbdf53462d41..d790f9d5a8282 100644 --- a/playground/terraform/modules/artifact_registry/main.tf +++ b/playground/terraform/infrastructure/artifact_registry/main.tf @@ -1,4 +1,3 @@ - # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -19,11 +18,12 @@ # resource "google_artifact_registry_repository" "playground_repo" { + // TODO: remove when generally available provider = google-beta - project = "${var.project_id}" - location = "${var.repository_location}" - repository_id = "${var.repository_id}" - description = "Playground docker repository" - format = "DOCKER" + project = var.project_id + location = var.repository_location + repository_id = var.repository_id + description = "Playground docker repository" + format = "DOCKER" } diff --git a/playground/terraform/modules/artifact_registry/output.tf b/playground/terraform/infrastructure/artifact_registry/output.tf similarity index 77% rename from playground/terraform/modules/artifact_registry/output.tf rename to playground/terraform/infrastructure/artifact_registry/output.tf index 89a8af415de07..822f589b9a0e1 100644 --- a/playground/terraform/modules/artifact_registry/output.tf +++ b/playground/terraform/infrastructure/artifact_registry/output.tf @@ -1,4 +1,3 @@ - # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -19,13 +18,13 @@ # output "registry_name" { - value = "${google_artifact_registry_repository.playground_repo.name}" + value = google_artifact_registry_repository.playground_repo.name } -output "regirsty_id" { - value = "${google_artifact_registry_repository.playground_repo.id}" +output "registry_id" { + value = google_artifact_registry_repository.playground_repo.id } output "location" { - value = "${google_artifact_registry_repository.playground_repo.location}" + value = google_artifact_registry_repository.playground_repo.location } diff --git a/playground/terraform/modules/artifact_registry/variables.tf b/playground/terraform/infrastructure/artifact_registry/variables.tf similarity index 99% rename from playground/terraform/modules/artifact_registry/variables.tf rename to playground/terraform/infrastructure/artifact_registry/variables.tf index 7dcacd864d00c..f3c450fecc45a 100644 --- a/playground/terraform/modules/artifact_registry/variables.tf +++ b/playground/terraform/infrastructure/artifact_registry/variables.tf @@ -1,4 +1,3 @@ - # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file diff --git a/playground/terraform/modules/buckets/main.tf b/playground/terraform/infrastructure/buckets/main.tf similarity index 70% rename from playground/terraform/modules/buckets/main.tf rename to playground/terraform/infrastructure/buckets/main.tf index 4d85cf983e89b..b0f51d9f6681f 100644 --- a/playground/terraform/modules/buckets/main.tf +++ b/playground/terraform/infrastructure/buckets/main.tf @@ -1,4 +1,3 @@ - # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -19,22 +18,21 @@ # resource "google_storage_bucket" "examples_bucket" { - name = "${var.examples_bucket_name}" - location = "${var.examples_bucket_location}" - project = "${var.project_id}" - storage_class = "${var.examples_storage_class}" - uniform_bucket_level_access = true + name = var.examples_bucket_name + location = var.examples_bucket_location + project = var.project_id + storage_class = var.examples_storage_class } resource "google_storage_bucket_access_control" "public_rule" { bucket = google_storage_bucket.examples_bucket.name - role = "VIEWER" + role = "READER" entity = "allUsers" } resource "google_storage_bucket" "terraform_bucket" { - name = "${var.terraform_bucket_name}" - location = "${var.terraform_bucket_location}" - project = "${var.project_id}" - storage_class = "${var.terraform_storage_class}" + name = var.terraform_bucket_name + location = var.terraform_bucket_location + project = var.project_id + storage_class = var.terraform_storage_class } diff --git a/playground/terraform/modules/buckets/output.tf b/playground/terraform/infrastructure/buckets/output.tf similarity index 69% rename from playground/terraform/modules/buckets/output.tf rename to playground/terraform/infrastructure/buckets/output.tf index 15a0ca88a09dc..9118f527fe667 100644 --- a/playground/terraform/modules/buckets/output.tf +++ b/playground/terraform/infrastructure/buckets/output.tf @@ -18,33 +18,33 @@ # output "examples-bucket-id" { - value = "${google_storage_bucket.examples_bucket.id}" + value = google_storage_bucket.examples_bucket.id } output "examples-bucket-name" { - value = "${google_storage_bucket.examples_bucket.name}" + value = google_storage_bucket.examples_bucket.name } output "examples-bucket-project" { - value = "${google_storage_bucket.examples_bucket.project}" + value = google_storage_bucket.examples_bucket.project } output "examples-bucket-location" { - value = "${google_storage_bucket.examples_bucket.location}" + value = google_storage_bucket.examples_bucket.location } output "terraform-bucket-id" { - value = "${google_storage_bucket.terraform_bucket.id}" + value = google_storage_bucket.terraform_bucket.id } output "terraform-bucket-name" { - value = "${google_storage_bucket.terraform_bucket.name}" + value = google_storage_bucket.terraform_bucket.name } output "terraform-bucket-project" { - value = "${google_storage_bucket.terraform_bucket.project}" + value = google_storage_bucket.terraform_bucket.project } output "terraform-bucket-location" { - value = "${google_storage_bucket.terraform_bucket.location}" + value = google_storage_bucket.terraform_bucket.location } diff --git a/playground/terraform/modules/buckets/variables.tf b/playground/terraform/infrastructure/buckets/variables.tf similarity index 99% rename from playground/terraform/modules/buckets/variables.tf rename to playground/terraform/infrastructure/buckets/variables.tf index a0aba8b76d2f8..4e256fd0dbd3b 100644 --- a/playground/terraform/modules/buckets/variables.tf +++ b/playground/terraform/infrastructure/buckets/variables.tf @@ -1,4 +1,3 @@ - # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file diff --git a/playground/terraform/modules/gke/main.tf b/playground/terraform/infrastructure/gke/main.tf similarity index 68% rename from playground/terraform/modules/gke/main.tf rename to playground/terraform/infrastructure/gke/main.tf index bbe1862fbfac3..bdc11d1b6b29d 100644 --- a/playground/terraform/modules/gke/main.tf +++ b/playground/terraform/infrastructure/gke/main.tf @@ -1,4 +1,3 @@ - # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -18,28 +17,24 @@ # under the License. # -terraform { - backend "gcs" { - bucket = "playground_terraform" - } -} - resource "google_container_cluster" "playground-gke" { - name = "playground-examples" - project = "${var.project_id}" - location = "us-central1-a" - initial_node_count = "${var.node_count}" + name = var.name + project = var.project_id + location = var.location + initial_node_count = var.node_count + network = var.network + subnetwork = var.subnetwork node_config { - machine_type = "${var.machine_type}" - service_account = "${var.service_account}" + machine_type = var.machine_type + service_account = var.service_account_email - oauth_scopes = [ + oauth_scopes = [ "https://www.googleapis.com/auth/cloud-platform" ] labels = { - component = "beam-playground" + component = "beam-playground" } - tags = ["beam-playground"] + tags = ["beam-playground"] } } diff --git a/playground/terraform/modules/gke/output.tf b/playground/terraform/infrastructure/gke/output.tf similarity index 93% rename from playground/terraform/modules/gke/output.tf rename to playground/terraform/infrastructure/gke/output.tf index b380083b7609e..96c5796cc1eef 100644 --- a/playground/terraform/modules/gke/output.tf +++ b/playground/terraform/infrastructure/gke/output.tf @@ -1,4 +1,3 @@ - # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -19,5 +18,5 @@ # output "gke_name" { - value = "${google_container_cluster.playground-gke.name}" + value = google_container_cluster.playground-gke.name } diff --git a/playground/terraform/infrastructure/gke/variables.tf b/playground/terraform/infrastructure/gke/variables.tf new file mode 100644 index 0000000000000..3d7cb491e11e1 --- /dev/null +++ b/playground/terraform/infrastructure/gke/variables.tf @@ -0,0 +1,54 @@ +# +# 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. +# + +variable "project_id" { + description = "The GCP Project ID where Playground Applications will be created" +} + +variable "machine_type" { + description = "Node pool machine types" + default = "e2-standard-4" +} + +variable "node_count" { + description = "Node pool size" + default = 1 +} + +variable "service_account_email" { + description = "Service account email" +} + +variable "name" { + description = "Name of GKE cluster" + default = "playground-examples" +} + +variable "location" { + description = "Location of GKE cluster" + default = "us-central1-a" +} + +variable "network" { + description = "GCP network within which resources are provisioned" +} + +variable "subnetwork" { + description = "GCP subnetwork within which resources are provisioned" +} diff --git a/playground/terraform/infrastructure/main.tf b/playground/terraform/infrastructure/main.tf new file mode 100644 index 0000000000000..702bae61c1bdf --- /dev/null +++ b/playground/terraform/infrastructure/main.tf @@ -0,0 +1,81 @@ +# +# 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. +# + +module "setup" { + source = "./setup" + project_id = var.project_id + region = var.region + service_account_id = var.service_account_id +} + +module "network" { + depends_on = [module.setup] + source = "./network" + project_id = var.project_id + region = var.region +} + +module "buckets" { + depends_on = [module.setup] + source = "./buckets" + project_id = var.project_id + terraform_bucket_name = var.terraform_bucket_name + terraform_storage_class = var.examples_storage_class + terraform_bucket_location = var.terraform_bucket_location + examples_bucket_name = var.examples_bucket_name + examples_storage_class = var.examples_storage_class + examples_bucket_location = var.examples_bucket_location +} + +module "artifact_registry" { + depends_on = [module.setup, module.buckets] + source = "./artifact_registry" + project_id = var.project_id + repository_id = var.repository_id + repository_location = var.repository_location +} + +module "memorystore" { + depends_on = [module.setup, module.artifact_registry] + source = "./memorystore" + project_id = var.project_id + terraform_state_bucket_name = var.terraform_bucket_name + redis_version = var.redis_version + redis_region = var.redis_region + redis_name = var.redis_name + redis_tier = var.redis_tier + redis_replica_count = var.redis_replica_count + redis_memory_size_gb = var.redis_memory_size_gb + read_replicas_mode = var.read_replicas_mode + network = module.network.network + subnetwork = module.network.subnetwork +} + +module "gke" { + depends_on = [module.setup, module.artifact_registry, module.memorystore] + source = "./gke" + project_id = var.project_id + machine_type = var.gke_machine_type + node_count = var.gke_node_count + name = var.gke_name + location = var.gke_location + service_account_email = module.setup.service_account_email + network = module.network.network + subnetwork = module.network.subnetwork +} diff --git a/playground/terraform/infrastructure/memorystore/main.tf b/playground/terraform/infrastructure/memorystore/main.tf new file mode 100644 index 0000000000000..9248fa7e84c25 --- /dev/null +++ b/playground/terraform/infrastructure/memorystore/main.tf @@ -0,0 +1,48 @@ +# +# 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. +# + + +data "terraform_remote_state" "remote_state_vpc" { + backend = "gcs" + config = { + bucket = var.terraform_state_bucket_name + } +} + +data "google_compute_network" "default" { + project = var.project_id + name = var.network +} + +# Redis for storing state of Playground application. +# In this cache Playground instances stores pipeline's statuses, outputs and pipeline's graph +resource "google_redis_instance" "cache" { + // TODO: remove when replica_count, etc is generally available + provider = google-beta + project = var.project_id + region = var.redis_region + name = var.redis_name + tier = var.redis_tier + memory_size_gb = var.redis_memory_size_gb + replica_count = var.redis_replica_count + redis_version = var.redis_version + display_name = var.display_name + read_replicas_mode = var.read_replicas_mode + authorized_network = data.google_compute_network.default.id +} diff --git a/playground/terraform/infrastructure/memorystore/variables.tf b/playground/terraform/infrastructure/memorystore/variables.tf new file mode 100644 index 0000000000000..31c79bde11fce --- /dev/null +++ b/playground/terraform/infrastructure/memorystore/variables.tf @@ -0,0 +1,77 @@ +# +# 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. +# + +variable "project_id" { + description = "The GCP Project ID where Playground Applications will be created" +} + +variable "redis_version" { + description = "The GCP Project ID where Playground Applications will be created" + default = "REDIS_6_X" +} + +variable "terraform_state_bucket_name" { + description = "Bucket name for terraform state" + default = "beam_playground_terraform" +} + +variable "redis_region" { + description = "Region of Redis" + default = "us-central1" +} + +variable "redis_name" { + description = "Name of Redis" + default = "playground-backend-cache" +} + +variable "redis_tier" { + description = "Tier of Redis" + default = "STANDARD_HA" +} + +variable "redis_replica_count" { + type = number + description = "Redis's replica count" + default = 1 +} + +variable "redis_memory_size_gb" { + type = number + description = "Size of Redis memory" + default = 5 +} + +variable "display_name" { + default = "Playground Cache" + description = "Display name for Redis service" +} + +variable "read_replicas_mode" { + description = "Read replica mode. Can only be specified when trying to create the instance." + default = "READ_REPLICAS_ENABLED" +} + +variable "network" { + description = "GCP network within which resources are provisioned" +} + +variable "subnetwork" { + description = "GCP subnetwork within which resources are provisioned" +} diff --git a/playground/terraform/infrastructure/network/main.tf b/playground/terraform/infrastructure/network/main.tf new file mode 100644 index 0000000000000..f9c346b2cd077 --- /dev/null +++ b/playground/terraform/infrastructure/network/main.tf @@ -0,0 +1,33 @@ +# +# 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. +# + +resource "google_compute_network" "playground" { + project = var.project_id + name = var.network_name + auto_create_subnetworks = false +} + +resource "google_compute_subnetwork" "playground" { + ip_cidr_range = var.subnetwork_cidr_range + name = var.subnetwork_name + network = google_compute_network.playground.id + region = var.region + project = var.project_id + private_ip_google_access = true +} diff --git a/playground/terraform/modules/vpc/output.tf b/playground/terraform/infrastructure/network/output.tf similarity index 83% rename from playground/terraform/modules/vpc/output.tf rename to playground/terraform/infrastructure/network/output.tf index f96d6ba499617..abaa9f9cbb6c6 100644 --- a/playground/terraform/modules/vpc/output.tf +++ b/playground/terraform/infrastructure/network/output.tf @@ -1,4 +1,3 @@ - # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -18,10 +17,10 @@ # under the License. # -output "vpc_name" { - value = "${google_compute_network.playground_vpc.name}" +output "network" { + value = google_compute_network.playground.name } -output "vpc_id" { - value = "${google_compute_network.playground_vpc.id}" +output "subnetwork" { + value = google_compute_subnetwork.playground.name } diff --git a/playground/terraform/modules/vpc/variables.tf b/playground/terraform/infrastructure/network/variables.tf similarity index 68% rename from playground/terraform/modules/vpc/variables.tf rename to playground/terraform/infrastructure/network/variables.tf index 61fcbbd43b861..1d8e1b8063668 100644 --- a/playground/terraform/modules/vpc/variables.tf +++ b/playground/terraform/infrastructure/network/variables.tf @@ -1,4 +1,3 @@ - # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -22,20 +21,21 @@ variable "project_id" { description = "The GCP Project ID where Playground Applications will be created" } -variable "vpc_name" { +variable "region" { + description = "The Google Cloud Platform (GCP) region in which to provision resources" +} + +variable "network_name" { description = "Name of VPC to be created" default = "playground-vpc" } -variable "create_subnets" { - description = "Auto Create Subnets Inside VPC" - default = true +variable "subnetwork_name" { + description = "Name of VPC to be created" + default = "playground-vpc" } -variable "mtu" { - description = "MTU Inside VPC" - default = 1460 +variable "subnetwork_cidr_range" { + description = "The address range for this subnet, in CIDR notation. Use a standard private VPC network address range: for example, 10.0.0.0/9." + default = "10.128.0.0/20" } - - - diff --git a/playground/terraform/infrastructure/provider.tf b/playground/terraform/infrastructure/provider.tf new file mode 100644 index 0000000000000..ae8c6f0af5799 --- /dev/null +++ b/playground/terraform/infrastructure/provider.tf @@ -0,0 +1,33 @@ +# +# 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. +# + +provider "google" { + region = var.region + // TODO may need to run module.setup first independent of this solution and add the terraform service account as a variable + // This allows us to use a service account to provision resources without downloading or storing service account keys + # impersonate_service_account = module.setup.terraform_service_account_email +} + +// TODO: required by artifact registry and memorystore; remove when generally available +provider "google-beta" { + region = var.region + // TODO may need to run module.setup first independent of this solution and add the terraform service account as a variable + // This allows us to use a service account to provision resources without downloading or storing service account keys + # impersonate_service_account = module.setup.terraform_service_account_email +} diff --git a/playground/terraform/infrastructure/setup/iam.tf b/playground/terraform/infrastructure/setup/iam.tf new file mode 100644 index 0000000000000..789940d0a199a --- /dev/null +++ b/playground/terraform/infrastructure/setup/iam.tf @@ -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. +# + +#resource "google_service_account" "terraform_service_account" { +# account_id = "terraform" +# display_name = "terraform" +#} +# +#resource "google_project_iam_member" "terraform_service_account_roles" { +# for_each = toset([ +# // TODO: add the required roles to provision resources (not OWNER :-)!) +# ]) +# role = each.key +# member = "serviceAccount:${google_service_account.terraform_service_account.email}" +# project = var.project_id +#} +# +#resource "google_service_account_iam_binding" "terraform_service_account_token_permissions" { +# service_account_id = google_service_account.terraform_service_account.id +# members = [ +# "user:${var.developer_account_email}" // TODO: add variable +# ] +# role = "roles/iam.serviceAccountTokenCreator" +#} + +#resource "google_service_account_iam_binding" "application_service_account_binding" { +# members = [ +# "serviceAccount:${google_service_account.terraform_service_account.email}" +# ] +# role = "roles/iam.serviceAccountUser" +# service_account_id = google_service_account.playground_service_account.id +#} + +resource "google_service_account" "playground_service_account" { + account_id = var.service_account_id + display_name = var.service_account_id +} + +resource "google_project_iam_member" "terraform_service_account_roles" { + for_each = toset([ + "roles/container.serviceAgent", + ]) + role = each.key + member = "serviceAccount:${google_service_account.playground_service_account.email}" + project = var.project_id +} \ No newline at end of file diff --git a/playground/terraform/infrastructure/setup/output.tf b/playground/terraform/infrastructure/setup/output.tf new file mode 100644 index 0000000000000..547bc4fe97706 --- /dev/null +++ b/playground/terraform/infrastructure/setup/output.tf @@ -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. +# + +output "service_account_email" { + value = google_service_account.playground_service_account.email +} \ No newline at end of file diff --git a/playground/terraform/infrastructure/setup/provider.tf b/playground/terraform/infrastructure/setup/provider.tf new file mode 100644 index 0000000000000..8576aa7e94f73 --- /dev/null +++ b/playground/terraform/infrastructure/setup/provider.tf @@ -0,0 +1,23 @@ +# +# 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. +# + +provider "google" { + region = var.region + project = var.project_id +} diff --git a/playground/terraform/modules/vpc/main.tf b/playground/terraform/infrastructure/setup/services.tf similarity index 76% rename from playground/terraform/modules/vpc/main.tf rename to playground/terraform/infrastructure/setup/services.tf index 8865276bba17a..52a3306d81db8 100644 --- a/playground/terraform/modules/vpc/main.tf +++ b/playground/terraform/infrastructure/setup/services.tf @@ -1,4 +1,3 @@ - # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -18,9 +17,14 @@ # under the License. # -resource "google_compute_network" "playground_vpc" { - project = "${var.project_id}" - name = "${var.vpc_name}" - auto_create_subnetworks = "${var.create_subnets}" - mtu = "${var.mtu}" +resource "google_project_service" "required_services" { + for_each = toset([ + "artifactregistry", + "compute", + "container", + "redis", + ]) + service = "${each.key}.googleapis.com" + disable_on_destroy = false } + diff --git a/playground/terraform/modules/gke/variables.tf b/playground/terraform/infrastructure/setup/variables.tf similarity index 79% rename from playground/terraform/modules/gke/variables.tf rename to playground/terraform/infrastructure/setup/variables.tf index 379f703e8fea8..8271fde897f9f 100644 --- a/playground/terraform/modules/gke/variables.tf +++ b/playground/terraform/infrastructure/setup/variables.tf @@ -1,4 +1,3 @@ - # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -22,16 +21,10 @@ variable "project_id" { description = "The GCP Project ID where Playground Applications will be created" } -variable "machine_type" { - description = "Node pool machine types" - default = "e2-standard-4" -} - -variable "node_count" { - description = "Node pool size" - default = 1 +variable "region" { + description = "The GCP region within which we provision resources" } -variable "service_account" { - description = "Service account email" +variable "service_account_id" { + description = "Service account ID" } diff --git a/playground/terraform/infrastructure/variables.tf b/playground/terraform/infrastructure/variables.tf new file mode 100644 index 0000000000000..217e778c06e2a --- /dev/null +++ b/playground/terraform/infrastructure/variables.tf @@ -0,0 +1,165 @@ +# +# 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. +# + +variable "project_id" { + description = "The GCP Project ID where Playground Applications will be created" +} + +variable "region" { + description = "The GCP region within which we provision resources" + default = "us-central1" +} + +#IAM + +variable "service_account_id" { + description = "Service account ID" + default = "beam-playground" +} + +#GCS + +variable "examples_bucket_name" { + description = "Name of Bucket to Store Playground Examples" + default = "playground-examples" +} + +variable "examples_bucket_location" { + description = "Location of Playground Examples Bucket" + default = "US" +} + +variable "examples_storage_class" { + description = "Examples Bucket Storage Class" + default = "STANDARD" +} + +variable "terraform_bucket_name" { + description = "Name of Bucket to Store Terraform States" + default = "playground_terraform" +} + +variable "terraform_bucket_location" { + description = "Location of Playground Examples Bucket" + default = "US" +} + +variable "terraform_storage_class" { + description = "Terraform Bucket Storage Class" + default = "STANDARD" +} + +# Artifact Registry + +variable "repository_id" { + description = "ID of Artifact Registry" + default = "playground-repository" +} + +variable "repository_location" { + description = "Location of Artifact Registry" + default = "us-central1" +} + +variable "service_account" { + description = "Service account id" + default = "service-account-playground" +} + +#Redis + +variable "redis_version" { + description = "The GCP Project ID where Playground Applications will be created" + default = "REDIS_6_X" +} + +variable "terraform_state_bucket_name" { + description = "Bucket name for terraform state" + default = "beam_playground_terraform" +} + +variable "redis_region" { + description = "Region of Redis" + default = "us-central1" +} + +variable "redis_name" { + description = "Name of Redis" + default = "playground-backend-cache" +} + +variable "redis_tier" { + description = "Tier of Redis" + default = "STANDARD_HA" +} + +variable "redis_replica_count" { + description = "Redis's replica count" + default = 1 +} + +variable "redis_memory_size_gb" { + description = "Size of Redis memory" + default = 5 +} + +variable "read_replicas_mode" { + description = "Read replica mode. Can only be specified when trying to create the instance." + default = "READ_REPLICAS_ENABLED" +} + +#VPC + +variable "vpc_name" { + description = "Name of VPC to be created" + default = "playground-vpc" +} + +variable "create_subnets" { + description = "Auto Create Subnets Inside VPC" + default = true +} + +variable "mtu" { + description = "MTU Inside VPC" + default = 1460 +} + +# GKE + + +variable "gke_machine_type" { + description = "Node pool machine types" + default = "e2-standard-4" +} + +variable "gke_node_count" { + description = "Node pool size" + default = 1 +} + +variable "gke_name" { + description = "Name of GKE cluster" + default = "playground-examples" +} + +variable "gke_location" { + description = "Location of GKE cluster" + default = "us-central1-a" +} diff --git a/playground/terraform/provider.tf b/playground/terraform/provider.tf index 0d212f5f00791..3227ce52fed54 100644 --- a/playground/terraform/provider.tf +++ b/playground/terraform/provider.tf @@ -1,4 +1,3 @@ - # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -19,8 +18,9 @@ # provider "google" { - region = "us-central" + region = "us-central" } +# TODO Please remove it when all resources are available in the stable version provider "google-beta" { - region = "us-central" + region = "us-central" } diff --git a/release/src/main/scripts/cut_release_branch.sh b/release/src/main/scripts/cut_release_branch.sh index 1e633934bdce8..a7b10530cf4c8 100755 --- a/release/src/main/scripts/cut_release_branch.sh +++ b/release/src/main/scripts/cut_release_branch.sh @@ -125,7 +125,7 @@ echo "==================Current working branch=======================" echo ${RELEASE_BRANCH} echo "===============================================================" -sed -i -e "s/'beam-master-.*'/'beam-${RELEASE}'/g" runners/google-cloud-dataflow-java/build.gradle +sed -i -e "s/'beam-master-.*'/'${RELEASE}'/g" runners/google-cloud-dataflow-java/build.gradle echo "===============Update release branch as following==============" git diff diff --git a/release/src/main/scripts/set_version.sh b/release/src/main/scripts/set_version.sh index 67420d6dda96e..387dd03d69acd 100755 --- a/release/src/main/scripts/set_version.sh +++ b/release/src/main/scripts/set_version.sh @@ -76,9 +76,6 @@ if [[ -z "$IS_SNAPSHOT_VERSION" ]] ; then sed -i -e "s/^__version__ = .*/__version__ = '${TARGET_VERSION}'/" sdks/python/apache_beam/version.py sed -i -e "s/sdk_version=.*/sdk_version=$TARGET_VERSION/" gradle.properties sed -i -e "s/SdkVersion = .*/SdkVersion = \"$TARGET_VERSION\"/" sdks/go/pkg/beam/core/core.go - # TODO: [BEAM-4767] - sed -i -e "s/'dataflow.fnapi_container_version' : .*/'dataflow.fnapi_container_version' : '${TARGET_VERSION}',/" runners/google-cloud-dataflow-java/build.gradle - sed -i -e "s/'dataflow.legacy_container_version' : .*/'dataflow.legacy_container_version' : 'beam-${TARGET_VERSION}',/" runners/google-cloud-dataflow-java/build.gradle else # For snapshot version: # Java/gradle appends -SNAPSHOT diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleStateRegistry.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleStateRegistry.java index a9cb6441c3ad0..94744d0cf74a8 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleStateRegistry.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleStateRegistry.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import javax.annotation.Nullable; import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; import org.apache.beam.vendor.grpc.v1p43p2.com.google.protobuf.ByteString; @@ -64,12 +65,16 @@ public Map getExecutionTimeMonitoringData(ShortIdMap shortId for (SimpleExecutionState state : executionStates) { if (state.getTotalMillis() != 0) { String shortId = state.getTotalMillisShortId(shortIds); - if (result.containsKey(shortId)) { - // This can happen due to flatten unzipping. - result.put(shortId, state.mergeTotalMillisPayload(result.get(shortId))); - } else { - result.put(shortId, state.getTotalMillisPayload()); - } + result.compute( + shortId, + (String k, @Nullable ByteString existing) -> { + if (existing != null) { + // This can happen due to flatten unzipping. + return state.mergeTotalMillisPayload(existing); + } else { + return state.getTotalMillisPayload(); + } + }); } } return result; diff --git a/runners/flink/flink_runner.gradle b/runners/flink/flink_runner.gradle index 2b6392f46b9bc..bc0cc188f0632 100644 --- a/runners/flink/flink_runner.gradle +++ b/runners/flink/flink_runner.gradle @@ -290,7 +290,7 @@ tasks.register('validatesRunner') { // Generates :runners:flink:1.13:runQuickstartJavaFlinkLocal createJavaExamplesArchetypeValidationTask(type: 'Quickstart', runner: 'FlinkLocal') -task examplesIntegrationTest(type: Test) { +tasks.register("examplesIntegrationTest", Test) { group = "Verification" // Disable gradle cache outputs.upToDateWhen { false } diff --git a/runners/flink/job-server/flink_job_server.gradle b/runners/flink/job-server/flink_job_server.gradle index 981a2cd870b16..2b3bbdfceb337 100644 --- a/runners/flink/job-server/flink_job_server.gradle +++ b/runners/flink/job-server/flink_job_server.gradle @@ -212,7 +212,7 @@ project.ext.validatesPortableRunnerBatch = portableValidatesRunnerTask("Batch", project.ext.validatesPortableRunnerStreaming = portableValidatesRunnerTask("Streaming", true, false, false) project.ext.validatesPortableRunnerStreamingCheckpoint = portableValidatesRunnerTask("StreamingCheckpointing", true, true, false) -task validatesPortableRunner() { +tasks.register("validatesPortableRunner") { dependsOn validatesPortableRunnerDocker dependsOn validatesPortableRunnerBatch dependsOn validatesPortableRunnerStreaming @@ -265,7 +265,8 @@ createCrossLanguageValidatesRunnerTask( ) // miniCluster jar starts an embedded Flink cluster intended for use in testing. -task miniCluster(type: Jar, dependsOn: shadowJar) { +tasks.register("miniCluster", Jar) { + dependsOn shadowJar archiveBaseName = "${project.archivesBaseName}-mini-cluster" dependencies { runtimeOnly project(path: flinkRunnerProject, configuration: "miniCluster") diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 89d185bf53512..355fc4547dfdc 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -367,7 +367,13 @@ task printRunnerV2PipelineOptions { task validatesRunner { group = "Verification" description "Validates Dataflow runner" - dependsOn(createLegacyWorkerValidatesRunnerTest(name: 'validatesRunnerLegacyWorkerTest')) + dependsOn(createLegacyWorkerValidatesRunnerTest( + name: 'validatesRunnerLegacyWorkerTest', + excludedTests: [ + // TODO(BEAM-13952) + 'org.apache.beam.sdk.transforms.GroupByKeyTest$BasicTests.testAfterProcessingTimeContinuationTriggerUsingState', + ] + )) } task validatesRunnerStreaming { @@ -382,6 +388,10 @@ task validatesRunnerStreaming { 'org.apache.beam.sdk.testing.UsesRequiresTimeSortedInput', 'org.apache.beam.sdk.testing.UsesSetState', ], + excludedTests: [ + // TODO(BEAM-13952) + 'org.apache.beam.sdk.transforms.GroupByKeyTest$BasicTests.testAfterProcessingTimeContinuationTriggerUsingState' + ] )) } @@ -473,6 +483,9 @@ task validatesRunnerV2 { 'org.apache.beam.sdk.transforms.GroupByKeyTest$WindowTests.testRewindowWithTimestampCombiner', 'org.apache.beam.sdk.transforms.FlattenTest.testFlattenWithDifferentInputAndOutputCoders2', + + // TODO(BEAM-13952) + 'org.apache.beam.sdk.transforms.GroupByKeyTest$BasicTests.testAfterProcessingTimeContinuationTriggerUsingState', ] )) } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java index 264827f33aa69..2995a485527cd 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java @@ -177,6 +177,30 @@ public Dataflow create(PipelineOptions options) { void setNumberOfWorkerHarnessThreads(int value); + /** + * Maximum number of bundles outstanding from windmill before the worker stops requesting. + * + *

If <= 0, use the default value of 100 + getNumberOfWorkerHarnessThreads() + */ + @Description( + "Maximum number of bundles outstanding from windmill before the worker stops requesting.") + @Default.Integer(0) + int getMaxBundlesFromWindmillOutstanding(); + + void setMaxBundlesFromWindmillOutstanding(int value); + + /** + * Maximum number of bytes outstanding from windmill before the worker stops requesting. + * + *

If <= 0, use the default value of 50% of jvm memory. + */ + @Description( + "Maximum number of bytes outstanding from windmill before the worker stops requesting. If <= 0, use the default value of 50% of jvm memory.") + @Default.Long(0) + long getMaxBytesFromWindmillOutstanding(); + + void setMaxBytesFromWindmillOutstanding(long value); + /** * If {@literal true}, save a heap dump before killing a thread or process which is GC thrashing * or out of memory. The location of the heap file will either be echoed back to the user, or the diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 236e47715a367..c49865aa7511c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -191,10 +191,6 @@ public class StreamingDataflowWorker { // Maximum number of threads for processing. Currently each thread processes one key at a time. static final int MAX_PROCESSING_THREADS = 300; static final long THREAD_EXPIRATION_TIME_SEC = 60; - // Maximum work units retrieved from Windmill and queued before processing. Limiting this delays - // retrieving extra work from Windmill without working on it, leading to better - // prioritization / utilization. - static final int MAX_WORK_UNITS_QUEUED = 100; static final long TARGET_COMMIT_BUNDLE_BYTES = 32 << 20; static final int MAX_COMMIT_QUEUE_BYTES = 500 << 20; // 500MB static final int NUM_COMMIT_STREAMS = 1; @@ -210,9 +206,6 @@ public class StreamingDataflowWorker { // Matches kWindmillCounterUpdate in workflow_worker_service_multi_hubs.cc. private static final String WINDMILL_COUNTER_UPDATE_WORK_ID = "3"; - /** Maximum number of items to return in a GetWork request. */ - private static final long MAX_GET_WORK_ITEMS = MAX_WORK_UNITS_QUEUED + MAX_PROCESSING_THREADS; - /** Maximum number of failure stacktraces to report in each update sent to backend. */ private static final int MAX_FAILURES_TO_REPORT_IN_UPDATE = 1000; @@ -666,7 +659,8 @@ public static StreamingDataflowWorker fromDataflowWorkerHarnessOptions( chooseMaximumNumberOfThreads(), THREAD_EXPIRATION_TIME_SEC, TimeUnit.SECONDS, - MAX_WORK_UNITS_QUEUED, + chooseMaximumBundlesOutstanding(), + chooseMaximumBytesOutstanding(), threadFactory); maxSinkBytes = @@ -785,6 +779,22 @@ private int chooseMaximumNumberOfThreads() { return MAX_PROCESSING_THREADS; } + private int chooseMaximumBundlesOutstanding() { + int maxBundles = options.getMaxBundlesFromWindmillOutstanding(); + if (maxBundles > 0) { + return maxBundles; + } + return chooseMaximumNumberOfThreads() + 100; + } + + private long chooseMaximumBytesOutstanding() { + long maxMem = options.getMaxBytesFromWindmillOutstanding(); + if (maxMem > 0) { + return maxMem; + } + return Runtime.getRuntime().maxMemory() / 2; + } + void addStateNameMappings(Map nameMap) { stateNameMap.putAll(nameMap); } @@ -804,7 +814,7 @@ public void setMaxWorkItemCommitBytes(int maxWorkItemCommitBytes) { @VisibleForTesting public boolean workExecutorIsEmpty() { - return workUnitExecutor.getQueue().isEmpty(); + return workUnitExecutor.executorQueueIsEmpty(); } public void start() { @@ -949,9 +959,6 @@ public void stop() { memoryMonitor.stop(); memoryMonitorThread.join(); workUnitExecutor.shutdown(); - if (!workUnitExecutor.awaitTermination(5, TimeUnit.MINUTES)) { - throw new RuntimeException("Work executor did not terminate within 5 minutes"); - } for (ComputationState state : computationMap.values()) { state.close(); } @@ -1065,7 +1072,7 @@ void streamingDispatchLoop() { windmillServer.getWorkStream( Windmill.GetWorkRequest.newBuilder() .setClientId(clientId) - .setMaxItems(MAX_GET_WORK_ITEMS) + .setMaxItems(chooseMaximumBundlesOutstanding()) .setMaxBytes(MAX_GET_WORK_FETCH_BYTES) .build(), (String computation, @@ -1236,7 +1243,8 @@ private void callFinalizeCallbacks(Windmill.WorkItem work) { } catch (Throwable t) { LOG.error("Source checkpoint finalization failed:", t); } - }); + }, + 0); } } } @@ -1548,7 +1556,7 @@ private void process( if (retryLocally) { // Try again after some delay and at the end of the queue to avoid a tight loop. sleep(retryLocallyDelayMs); - workUnitExecutor.forceExecute(work); + workUnitExecutor.forceExecute(work, work.getWorkItem().getSerializedSize()); } else { // Consider the item invalid. It will eventually be retried by Windmill if it still needs to // be processed. @@ -1726,7 +1734,7 @@ private Windmill.GetWorkResponse getWork() { return windmillServer.getWork( Windmill.GetWorkRequest.newBuilder() .setClientId(clientId) - .setMaxItems(MAX_GET_WORK_ITEMS) + .setMaxItems(chooseMaximumBundlesOutstanding()) .setMaxBytes(MAX_GET_WORK_FETCH_BYTES) .build()); } @@ -2285,7 +2293,7 @@ public boolean activateWork(ShardedKey shardedKey, Work work) { // Fall through to execute without the lock held. } } - executor.execute(work); + executor.execute(work, work.getWorkItem().getSerializedSize()); return true; } @@ -2327,7 +2335,7 @@ public void completeWork(ShardedKey shardedKey, long workToken) { } } if (nextWork != null) { - executor.forceExecute(nextWork); + executor.forceExecute(nextWork, nextWork.getWorkItem().getSerializedSize()); } } @@ -2511,27 +2519,9 @@ public void doGet(HttpServletRequest request, HttpServletResponse response) thro } private class MetricsDataProvider implements StatusDataProvider { - @Override public void appendSummaryHtml(PrintWriter writer) { - writer.println( - "Worker Threads: " - + workUnitExecutor.getPoolSize() - + "/" - + workUnitExecutor.getMaximumPoolSize() - + "
"); - writer.println("Active Threads: " + workUnitExecutor.getActiveCount() + "
"); - writer.println( - "Work Queue Size: " - + workUnitExecutor.getQueue().size() - + "/" - + MAX_WORK_UNITS_QUEUED - + "
"); - writer.print("Commit Queue: "); - appendHumanizedBytes(commitQueue.weight(), writer); - writer.print(", "); - writer.print(commitQueue.size()); - writer.println(" elements
"); + writer.println(workUnitExecutor.summaryHtml()); writer.print("Active commit: "); appendHumanizedBytes(activeCommitBytes.get(), writer); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java index 9a51e8c18febe..29a4ea7c5355c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java @@ -18,62 +18,138 @@ package org.apache.beam.runners.dataflow.worker.util; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.Semaphore; import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Monitor; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Monitor.Guard; -/** Executor that blocks on execute() if its queue is full. */ +/** An executor for executing work on windmill items. */ @SuppressWarnings({ "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) }) -public class BoundedQueueExecutor extends ThreadPoolExecutor { - private static class ReducableSemaphore extends Semaphore { - ReducableSemaphore(int permits) { - super(permits); - } - - @Override - public void reducePermits(int permits) { - super.reducePermits(permits); - } - } +public class BoundedQueueExecutor { + private final ThreadPoolExecutor executor; + private final int maximumElementsOutstanding; + private final long maximumBytesOutstanding; - private ReducableSemaphore semaphore; + private final Monitor monitor = new Monitor(); + private int elementsOutstanding = 0; + private long bytesOutstanding = 0; public BoundedQueueExecutor( int maximumPoolSize, long keepAliveTime, TimeUnit unit, - int maximumQueueSize, + int maximumElementsOutstanding, + long maximumBytesOutstanding, ThreadFactory threadFactory) { - super( - maximumPoolSize, - maximumPoolSize, - keepAliveTime, - unit, - new LinkedBlockingQueue(), - threadFactory); - this.semaphore = new ReducableSemaphore(maximumQueueSize); - allowCoreThreadTimeOut(true); + executor = + new ThreadPoolExecutor( + maximumPoolSize, + maximumPoolSize, + keepAliveTime, + unit, + new LinkedBlockingQueue<>(), + threadFactory); + executor.allowCoreThreadTimeOut(true); + this.maximumElementsOutstanding = maximumElementsOutstanding; + this.maximumBytesOutstanding = maximumBytesOutstanding; } - // Before adding a Runnable to the queue, acquire the semaphore. - @Override - public void execute(Runnable r) { - semaphore.acquireUninterruptibly(); - super.execute(r); + // Before adding a Work to the queue, check that there are enough bytes of space or no other + // outstanding elements of work. + public void execute(Runnable work, long workBytes) { + monitor.enterWhenUninterruptibly( + new Guard(monitor) { + @Override + public boolean isSatisfied() { + return elementsOutstanding == 0 + || (bytesAvailable() >= workBytes + && elementsOutstanding < maximumElementsOutstanding); + } + }); + executeLockHeld(work, workBytes); } // Forcibly add something to the queue, ignoring the length limit. - public void forceExecute(Runnable r) { - semaphore.reducePermits(1); - super.execute(r); + public void forceExecute(Runnable work, long workBytes) { + monitor.enter(); + executeLockHeld(work, workBytes); + } + + public void shutdown() throws InterruptedException { + executor.shutdown(); + if (!executor.awaitTermination(5, TimeUnit.MINUTES)) { + throw new RuntimeException("Work executor did not terminate within 5 minutes"); + } + } + + public boolean executorQueueIsEmpty() { + return executor.getQueue().isEmpty(); + } + + public String summaryHtml() { + monitor.enter(); + try { + StringBuilder builder = new StringBuilder(); + builder.append("Worker Threads: "); + builder.append(executor.getPoolSize()); + builder.append("/"); + builder.append(executor.getMaximumPoolSize()); + builder.append("
/n"); + + builder.append("Active Threads: "); + builder.append(executor.getActiveCount()); + builder.append("
/n"); + + builder.append("Work Queue Size: "); + builder.append(elementsOutstanding); + builder.append("/"); + builder.append(maximumElementsOutstanding); + builder.append("
/n"); + + builder.append("Work Queue Bytes: "); + builder.append(bytesOutstanding); + builder.append("/"); + builder.append(maximumBytesOutstanding); + builder.append("
/n"); + + return builder.toString(); + } finally { + monitor.leave(); + } + } + + private void executeLockHeld(Runnable work, long workBytes) { + bytesOutstanding += workBytes; + ++elementsOutstanding; + monitor.leave(); + + try { + executor.execute( + () -> { + try { + work.run(); + } finally { + decrementCounters(workBytes); + } + }); + } catch (RuntimeException e) { + // If the execute() call threw an exception, decrement counters here. + decrementCounters(workBytes); + throw e; + } + } + + private void decrementCounters(long workBytes) { + monitor.enter(); + --elementsOutstanding; + bytesOutstanding -= workBytes; + monitor.leave(); } - // Release the semaphore after taking a Runnable off the queue. - @Override - public void beforeExecute(Thread t, Runnable r) { - semaphore.release(); + private long bytesAvailable() { + return maximumBytesOutstanding - bytesOutstanding; } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index c175d634508e6..c383e5d16e599 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -2693,14 +2693,14 @@ public void testActiveWork() throws Exception { MockWork m1 = new MockWork(1); assertTrue(computationState.activateWork(key1, m1)); - Mockito.verify(mockExecutor).execute(m1); + Mockito.verify(mockExecutor).execute(m1, m1.getWorkItem().getSerializedSize()); computationState.completeWork(key1, 1); Mockito.verifyNoMoreInteractions(mockExecutor); // Verify work queues. MockWork m2 = new MockWork(2); assertTrue(computationState.activateWork(key1, m2)); - Mockito.verify(mockExecutor).execute(m2); + Mockito.verify(mockExecutor).execute(m2, m2.getWorkItem().getSerializedSize()); MockWork m3 = new MockWork(3); assertTrue(computationState.activateWork(key1, m3)); Mockito.verifyNoMoreInteractions(mockExecutor); @@ -2708,19 +2708,19 @@ public void testActiveWork() throws Exception { // Verify another key is a separate queue. MockWork m4 = new MockWork(4); assertTrue(computationState.activateWork(key2, m4)); - Mockito.verify(mockExecutor).execute(m4); + Mockito.verify(mockExecutor).execute(m4, m4.getWorkItem().getSerializedSize()); computationState.completeWork(key2, 4); Mockito.verifyNoMoreInteractions(mockExecutor); computationState.completeWork(key1, 2); - Mockito.verify(mockExecutor).forceExecute(m3); + Mockito.verify(mockExecutor).forceExecute(m3, m3.getWorkItem().getSerializedSize()); computationState.completeWork(key1, 3); Mockito.verifyNoMoreInteractions(mockExecutor); // Verify duplicate work dropped. MockWork m5 = new MockWork(5); computationState.activateWork(key1, m5); - Mockito.verify(mockExecutor).execute(m5); + Mockito.verify(mockExecutor).execute(m5, m5.getWorkItem().getSerializedSize()); assertFalse(computationState.activateWork(key1, m5)); Mockito.verifyNoMoreInteractions(mockExecutor); computationState.completeWork(key1, 5); @@ -2743,14 +2743,14 @@ public void testActiveWorkForShardedKeys() throws Exception { MockWork m1 = new MockWork(1); assertTrue(computationState.activateWork(key1Shard1, m1)); - Mockito.verify(mockExecutor).execute(m1); + Mockito.verify(mockExecutor).execute(m1, m1.getWorkItem().getSerializedSize()); computationState.completeWork(key1Shard1, 1); Mockito.verifyNoMoreInteractions(mockExecutor); // Verify work queues. MockWork m2 = new MockWork(2); assertTrue(computationState.activateWork(key1Shard1, m2)); - Mockito.verify(mockExecutor).execute(m2); + Mockito.verify(mockExecutor).execute(m2, m2.getWorkItem().getSerializedSize()); MockWork m3 = new MockWork(3); assertTrue(computationState.activateWork(key1Shard1, m3)); Mockito.verifyNoMoreInteractions(mockExecutor); @@ -2760,7 +2760,7 @@ public void testActiveWorkForShardedKeys() throws Exception { assertFalse(computationState.activateWork(key1Shard1, m4)); Mockito.verifyNoMoreInteractions(mockExecutor); assertTrue(computationState.activateWork(key1Shard2, m4)); - Mockito.verify(mockExecutor).execute(m4); + Mockito.verify(mockExecutor).execute(m4, m4.getWorkItem().getSerializedSize()); // Verify duplicate work dropped assertFalse(computationState.activateWork(key1Shard2, m4)); diff --git a/runners/samza/build.gradle b/runners/samza/build.gradle index acacf9f8cd607..68ea1780bd465 100644 --- a/runners/samza/build.gradle +++ b/runners/samza/build.gradle @@ -86,7 +86,7 @@ configurations.all { exclude group: "org.slf4j", module: "slf4j-jdk14" } -task validatesRunner(type: Test) { +tasks.register("validatesRunner", Test) { group = "Verification" description "Validates Samza runner" systemProperty "beamTestPipelineOptions", JsonOutput.toJson([ diff --git a/runners/samza/job-server/build.gradle b/runners/samza/job-server/build.gradle index 8302f2d0b7173..a8adca036a674 100644 --- a/runners/samza/job-server/build.gradle +++ b/runners/samza/job-server/build.gradle @@ -185,7 +185,7 @@ def portableValidatesRunnerTask(String name, boolean docker) { project.ext.validatesPortableRunnerDocker = portableValidatesRunnerTask("Docker", true) project.ext.validatesPortableRunnerEmbedded = portableValidatesRunnerTask("Embedded", false) -task validatesPortableRunner() { +tasks.register("validatesPortableRunner") { dependsOn validatesPortableRunnerDocker dependsOn validatesPortableRunnerEmbedded } diff --git a/runners/spark/job-server/spark_job_server.gradle b/runners/spark/job-server/spark_job_server.gradle index a483284f5074b..18ab88f7b0d07 100644 --- a/runners/spark/job-server/spark_job_server.gradle +++ b/runners/spark/job-server/spark_job_server.gradle @@ -213,7 +213,7 @@ project.ext.validatesPortableRunnerDocker= portableValidatesRunnerTask("Docker", project.ext.validatesPortableRunnerBatch = portableValidatesRunnerTask("Batch", false, false) project.ext.validatesPortableRunnerStreaming = portableValidatesRunnerTask("Streaming", true, false) -task validatesPortableRunner() { +tasks.register("validatesPortableRunner") { dependsOn validatesPortableRunnerDocker dependsOn validatesPortableRunnerBatch dependsOn validatesPortableRunnerStreaming diff --git a/runners/spark/spark_runner.gradle b/runners/spark/spark_runner.gradle index 08124142560ad..7d08b582b0780 100644 --- a/runners/spark/spark_runner.gradle +++ b/runners/spark/spark_runner.gradle @@ -237,7 +237,7 @@ hadoopVersions.each {kv -> } } -task validatesRunnerBatch(type: Test) { +def validatesRunnerBatch = tasks.register("validatesRunnerBatch", Test) { group = "Verification" // Disable gradle cache outputs.upToDateWhen { false } @@ -290,7 +290,7 @@ task validatesRunnerBatch(type: Test) { jvmArgs '-Xmx3g' } -task validatesRunnerStreaming(type: Test) { +def validatesRunnerStreaming = tasks.register("validatesRunnerStreaming", Test) { group = "Verification" // Disable gradle cache outputs.upToDateWhen { false } @@ -316,7 +316,7 @@ task validatesRunnerStreaming(type: Test) { } } -task validatesStructuredStreamingRunnerBatch(type: Test) { +tasks.register("validatesStructuredStreamingRunnerBatch", Test) { group = "Verification" // Disable gradle cache outputs.upToDateWhen { false } @@ -385,7 +385,7 @@ task validatesStructuredStreamingRunnerBatch(type: Test) { } } -task validatesRunner { +tasks.register("validatesRunner") { group = "Verification" description "Validates Spark runner" dependsOn validatesRunnerBatch @@ -398,14 +398,15 @@ task validatesRunner { // Generates :runners:spark:*:runQuickstartJavaSpark task createJavaExamplesArchetypeValidationTask(type: 'Quickstart', runner: 'Spark') -task hadoopVersionsTest(group: "Verification") { +tasks.register("hadoopVersionsTest") { + group = "Verification" def taskNames = hadoopVersions.keySet().stream() .map{num -> "hadoopVersion${num}Test"} .collect(Collectors.toList()) dependsOn taskNames } -task examplesIntegrationTest(type: Test) { +tasks.register("examplesIntegrationTest", Test) { group = "Verification" // Disable gradle cache outputs.upToDateWhen { false } @@ -434,7 +435,8 @@ task examplesIntegrationTest(type: Test) { } hadoopVersions.each {kv -> - task "hadoopVersion${kv.key}Test"(type: Test, group: "Verification") { + tasks.register("hadoopVersion${kv.key}Test", Test) { + group = "Verification" description = "Runs Spark tests with Hadoop version $kv.value" classpath = configurations."hadoopVersion$kv.key" + sourceSets.test.runtimeClasspath systemProperty "beam.spark.test.reuseSparkContext", "true" diff --git a/runners/twister2/build.gradle b/runners/twister2/build.gradle index 36671b318d0c7..a363bacb4a718 100644 --- a/runners/twister2/build.gradle +++ b/runners/twister2/build.gradle @@ -68,7 +68,7 @@ dependencies { } } -task validatesRunnerBatch(type: Test) { +def validatesRunnerBatch = tasks.register("validatesRunnerBatch", Test) { group = "Verification" def pipelineOptions = JsonOutput.toJson([ "--runner=Twister2TestRunner", @@ -101,7 +101,7 @@ task validatesRunnerBatch(type: Test) { maxHeapSize = '6g' } -task validatesRunner { +tasks.register("validatesRunner") { group = "Verification" description "Validates Twister2 Runner" dependsOn validatesRunnerBatch diff --git a/scripts/ci/pr-bot/README.md b/scripts/ci/pr-bot/README.md new file mode 100644 index 0000000000000..d6a55d45e9f13 --- /dev/null +++ b/scripts/ci/pr-bot/README.md @@ -0,0 +1,45 @@ + + +# PR Bot + +This directory holds all the code (except for Actions Workflows) for our PR bot designed to improve the PR experience. +For a list of commands to use when interacting with the bot, see [Commands.md](./Commands.md). +For a design doc explaining the design and implementation, see [Automate Reviewer Assignment](https://docs.google.com/document/d/1FhRPRD6VXkYlLAPhNfZB7y2Yese2FCWBzjx67d3TjBo/edit#) + +## Build/Test + +To build, run: + +``` +npm install +npm run build +``` + +To run the tests: + +``` +npm test +``` + +Before checking in code, run prettier on it: + +``` +npm run format +``` \ No newline at end of file diff --git a/scripts/ci/pr-bot/package-lock.json b/scripts/ci/pr-bot/package-lock.json new file mode 100644 index 0000000000000..cf881686c1405 --- /dev/null +++ b/scripts/ci/pr-bot/package-lock.json @@ -0,0 +1,2062 @@ +{ + "name": "pr-bot", + "version": "1.0.0", + "lockfileVersion": 2, + "requires": true, + "packages": { + "": { + "name": "pr-bot", + "version": "1.0.0", + "dependencies": { + "@actions/exec": "^1.1.0", + "@actions/github": "^5.0.0", + "@octokit/rest": "^18.12.0", + "js-yaml": "^4.1.0", + "prettier": "^2.5.1" + }, + "devDependencies": { + "@types/mocha": "^9.1.0", + "@types/node": "^16.11.7", + "mocha": "^9.1.3", + "typescript": "4.2.4" + } + }, + "node_modules/@actions/exec": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@actions/exec/-/exec-1.1.0.tgz", + "integrity": "sha512-LImpN9AY0J1R1mEYJjVJfSZWU4zYOlEcwSTgPve1rFQqK5AwrEs6uWW5Rv70gbDIQIAUwI86z6B+9mPK4w9Sbg==", + "dependencies": { + "@actions/io": "^1.0.1" + } + }, + "node_modules/@actions/github": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/@actions/github/-/github-5.0.0.tgz", + "integrity": "sha512-QvE9eAAfEsS+yOOk0cylLBIO/d6WyWIOvsxxzdrPFaud39G6BOkUwScXZn1iBzQzHyu9SBkkLSWlohDWdsasAQ==", + "dependencies": { + "@actions/http-client": "^1.0.11", + "@octokit/core": "^3.4.0", + "@octokit/plugin-paginate-rest": "^2.13.3", + "@octokit/plugin-rest-endpoint-methods": "^5.1.1" + } + }, + "node_modules/@actions/http-client": { + "version": "1.0.11", + "resolved": "https://registry.npmjs.org/@actions/http-client/-/http-client-1.0.11.tgz", + "integrity": "sha512-VRYHGQV1rqnROJqdMvGUbY/Kn8vriQe/F9HR2AlYHzmKuM/p3kjNuXhmdBfcVgsvRWTz5C5XW5xvndZrVBuAYg==", + "dependencies": { + "tunnel": "0.0.6" + } + }, + "node_modules/@actions/io": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/@actions/io/-/io-1.1.1.tgz", + "integrity": "sha512-Qi4JoKXjmE0O67wAOH6y0n26QXhMKMFo7GD/4IXNVcrtLjUlGjGuVys6pQgwF3ArfGTQu0XpqaNr0YhED2RaRA==" + }, + "node_modules/@octokit/auth-token": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@octokit/auth-token/-/auth-token-2.5.0.tgz", + "integrity": "sha512-r5FVUJCOLl19AxiuZD2VRZ/ORjp/4IN98Of6YJoJOkY75CIBuYfmiNHGrDwXr+aLGG55igl9QrxX3hbiXlLb+g==", + "dependencies": { + "@octokit/types": "^6.0.3" + } + }, + "node_modules/@octokit/core": { + "version": "3.5.1", + "resolved": "https://registry.npmjs.org/@octokit/core/-/core-3.5.1.tgz", + "integrity": "sha512-omncwpLVxMP+GLpLPgeGJBF6IWJFjXDS5flY5VbppePYX9XehevbDykRH9PdCdvqt9TS5AOTiDide7h0qrkHjw==", + "dependencies": { + "@octokit/auth-token": "^2.4.4", + "@octokit/graphql": "^4.5.8", + "@octokit/request": "^5.6.0", + "@octokit/request-error": "^2.0.5", + "@octokit/types": "^6.0.3", + "before-after-hook": "^2.2.0", + "universal-user-agent": "^6.0.0" + } + }, + "node_modules/@octokit/endpoint": { + "version": "6.0.12", + "resolved": "https://registry.npmjs.org/@octokit/endpoint/-/endpoint-6.0.12.tgz", + "integrity": "sha512-lF3puPwkQWGfkMClXb4k/eUT/nZKQfxinRWJrdZaJO85Dqwo/G0yOC434Jr2ojwafWJMYqFGFa5ms4jJUgujdA==", + "dependencies": { + "@octokit/types": "^6.0.3", + "is-plain-object": "^5.0.0", + "universal-user-agent": "^6.0.0" + } + }, + "node_modules/@octokit/graphql": { + "version": "4.8.0", + "resolved": "https://registry.npmjs.org/@octokit/graphql/-/graphql-4.8.0.tgz", + "integrity": "sha512-0gv+qLSBLKF0z8TKaSKTsS39scVKF9dbMxJpj3U0vC7wjNWFuIpL/z76Qe2fiuCbDRcJSavkXsVtMS6/dtQQsg==", + "dependencies": { + "@octokit/request": "^5.6.0", + "@octokit/types": "^6.0.3", + "universal-user-agent": "^6.0.0" + } + }, + "node_modules/@octokit/openapi-types": { + "version": "11.2.0", + "resolved": "https://registry.npmjs.org/@octokit/openapi-types/-/openapi-types-11.2.0.tgz", + "integrity": "sha512-PBsVO+15KSlGmiI8QAzaqvsNlZlrDlyAJYcrXBCvVUxCp7VnXjkwPoFHgjEJXx3WF9BAwkA6nfCUA7i9sODzKA==" + }, + "node_modules/@octokit/plugin-paginate-rest": { + "version": "2.17.0", + "resolved": "https://registry.npmjs.org/@octokit/plugin-paginate-rest/-/plugin-paginate-rest-2.17.0.tgz", + "integrity": "sha512-tzMbrbnam2Mt4AhuyCHvpRkS0oZ5MvwwcQPYGtMv4tUa5kkzG58SVB0fcsLulOZQeRnOgdkZWkRUiyBlh0Bkyw==", + "dependencies": { + "@octokit/types": "^6.34.0" + }, + "peerDependencies": { + "@octokit/core": ">=2" + } + }, + "node_modules/@octokit/plugin-request-log": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/@octokit/plugin-request-log/-/plugin-request-log-1.0.4.tgz", + "integrity": "sha512-mLUsMkgP7K/cnFEw07kWqXGF5LKrOkD+lhCrKvPHXWDywAwuDUeDwWBpc69XK3pNX0uKiVt8g5z96PJ6z9xCFA==", + "peerDependencies": { + "@octokit/core": ">=3" + } + }, + "node_modules/@octokit/plugin-rest-endpoint-methods": { + "version": "5.13.0", + "resolved": "https://registry.npmjs.org/@octokit/plugin-rest-endpoint-methods/-/plugin-rest-endpoint-methods-5.13.0.tgz", + "integrity": "sha512-uJjMTkN1KaOIgNtUPMtIXDOjx6dGYysdIFhgA52x4xSadQCz3b/zJexvITDVpANnfKPW/+E0xkOvLntqMYpviA==", + "dependencies": { + "@octokit/types": "^6.34.0", + "deprecation": "^2.3.1" + }, + "peerDependencies": { + "@octokit/core": ">=3" + } + }, + "node_modules/@octokit/request": { + "version": "5.6.3", + "resolved": "https://registry.npmjs.org/@octokit/request/-/request-5.6.3.tgz", + "integrity": "sha512-bFJl0I1KVc9jYTe9tdGGpAMPy32dLBXXo1dS/YwSCTL/2nd9XeHsY616RE3HPXDVk+a+dBuzyz5YdlXwcDTr2A==", + "dependencies": { + "@octokit/endpoint": "^6.0.1", + "@octokit/request-error": "^2.1.0", + "@octokit/types": "^6.16.1", + "is-plain-object": "^5.0.0", + "node-fetch": "^2.6.7", + "universal-user-agent": "^6.0.0" + } + }, + "node_modules/@octokit/request-error": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/@octokit/request-error/-/request-error-2.1.0.tgz", + "integrity": "sha512-1VIvgXxs9WHSjicsRwq8PlR2LR2x6DwsJAaFgzdi0JfJoGSO8mYI/cHJQ+9FbN21aa+DrgNLnwObmyeSC8Rmpg==", + "dependencies": { + "@octokit/types": "^6.0.3", + "deprecation": "^2.0.0", + "once": "^1.4.0" + } + }, + "node_modules/@octokit/rest": { + "version": "18.12.0", + "resolved": "https://registry.npmjs.org/@octokit/rest/-/rest-18.12.0.tgz", + "integrity": "sha512-gDPiOHlyGavxr72y0guQEhLsemgVjwRePayJ+FcKc2SJqKUbxbkvf5kAZEWA/MKvsfYlQAMVzNJE3ezQcxMJ2Q==", + "dependencies": { + "@octokit/core": "^3.5.1", + "@octokit/plugin-paginate-rest": "^2.16.8", + "@octokit/plugin-request-log": "^1.0.4", + "@octokit/plugin-rest-endpoint-methods": "^5.12.0" + } + }, + "node_modules/@octokit/types": { + "version": "6.34.0", + "resolved": "https://registry.npmjs.org/@octokit/types/-/types-6.34.0.tgz", + "integrity": "sha512-s1zLBjWhdEI2zwaoSgyOFoKSl109CUcVBCc7biPJ3aAf6LGLU6szDvi31JPU7bxfla2lqfhjbbg/5DdFNxOwHw==", + "dependencies": { + "@octokit/openapi-types": "^11.2.0" + } + }, + "node_modules/@types/mocha": { + "version": "9.1.0", + "resolved": "https://registry.npmjs.org/@types/mocha/-/mocha-9.1.0.tgz", + "integrity": "sha512-QCWHkbMv4Y5U9oW10Uxbr45qMMSzl4OzijsozynUAgx3kEHUdXB00udx2dWDQ7f2TU2a2uuiFaRZjCe3unPpeg==", + "dev": true + }, + "node_modules/@types/node": { + "version": "16.11.25", + "resolved": "https://registry.npmjs.org/@types/node/-/node-16.11.25.tgz", + "integrity": "sha512-NrTwfD7L1RTc2qrHQD4RTTy4p0CO2LatKBEKEds3CaVuhoM/+DJzmWZl5f+ikR8cm8F5mfJxK+9rQq07gRiSjQ==", + "dev": true + }, + "node_modules/@ungap/promise-all-settled": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@ungap/promise-all-settled/-/promise-all-settled-1.1.2.tgz", + "integrity": "sha512-sL/cEvJWAnClXw0wHk85/2L0G6Sj8UB0Ctc1TEMbKSsmpRosqhwj9gWgFRZSrBr2f9tiXISwNhCPmlfqUqyb9Q==", + "dev": true + }, + "node_modules/ansi-colors": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/ansi-colors/-/ansi-colors-4.1.1.tgz", + "integrity": "sha512-JoX0apGbHaUJBNl6yF+p6JAFYZ666/hhCGKN5t9QFjbJQKUU/g8MNbFDbvfrgKXvI1QpZplPOnwIo99lX/AAmA==", + "dev": true, + "engines": { + "node": ">=6" + } + }, + "node_modules/ansi-regex": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-5.0.1.tgz", + "integrity": "sha512-quJQXlTSUGL2LH9SUXo8VwsY4soanhgo6LNSm84E1LBcE8s3O0wpdiRzyR9z/ZZJMlMWv37qOOb9pdJlMUEKFQ==", + "dev": true, + "engines": { + "node": ">=8" + } + }, + "node_modules/ansi-styles": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-4.3.0.tgz", + "integrity": "sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg==", + "dev": true, + "dependencies": { + "color-convert": "^2.0.1" + }, + "engines": { + "node": ">=8" + }, + "funding": { + "url": "https://github.com/chalk/ansi-styles?sponsor=1" + } + }, + "node_modules/anymatch": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/anymatch/-/anymatch-3.1.2.tgz", + "integrity": "sha512-P43ePfOAIupkguHUycrc4qJ9kz8ZiuOUijaETwX7THt0Y/GNK7v0aa8rY816xWjZ7rJdA5XdMcpVFTKMq+RvWg==", + "dev": true, + "dependencies": { + "normalize-path": "^3.0.0", + "picomatch": "^2.0.4" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/argparse": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/argparse/-/argparse-2.0.1.tgz", + "integrity": "sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q==" + }, + "node_modules/balanced-match": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-1.0.2.tgz", + "integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==", + "dev": true + }, + "node_modules/before-after-hook": { + "version": "2.2.2", + "resolved": "https://registry.npmjs.org/before-after-hook/-/before-after-hook-2.2.2.tgz", + "integrity": "sha512-3pZEU3NT5BFUo/AD5ERPWOgQOCZITni6iavr5AUw5AUwQjMlI0kzu5btnyD39AF0gUEsDPwJT+oY1ORBJijPjQ==" + }, + "node_modules/binary-extensions": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/binary-extensions/-/binary-extensions-2.2.0.tgz", + "integrity": "sha512-jDctJ/IVQbZoJykoeHbhXpOlNBqGNcwXJKJog42E5HDPUwQTSdjCHdihjj0DlnheQ7blbT6dHOafNAiS8ooQKA==", + "dev": true, + "engines": { + "node": ">=8" + } + }, + "node_modules/brace-expansion": { + "version": "1.1.11", + "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-1.1.11.tgz", + "integrity": "sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA==", + "dev": true, + "dependencies": { + "balanced-match": "^1.0.0", + "concat-map": "0.0.1" + } + }, + "node_modules/braces": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.2.tgz", + "integrity": "sha512-b8um+L1RzM3WDSzvhm6gIz1yfTbBt6YTlcEKAvsmqCZZFw46z626lVj9j1yEPW33H5H+lBQpZMP1k8l+78Ha0A==", + "dev": true, + "dependencies": { + "fill-range": "^7.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/browser-stdout": { + "version": "1.3.1", + "resolved": "https://registry.npmjs.org/browser-stdout/-/browser-stdout-1.3.1.tgz", + "integrity": "sha512-qhAVI1+Av2X7qelOfAIYwXONood6XlZE/fXaBSmW/T5SzLAmCgzi+eiWE7fUvbHaeNBQH13UftjpXxsfLkMpgw==", + "dev": true + }, + "node_modules/camelcase": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-6.3.0.tgz", + "integrity": "sha512-Gmy6FhYlCY7uOElZUSbxo2UCDH8owEk996gkbrpsgGtrJLM3J7jGxl9Ic7Qwwj4ivOE5AWZWRMecDdF7hqGjFA==", + "dev": true, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/chalk": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-4.1.2.tgz", + "integrity": "sha512-oKnbhFyRIXpUuez8iBMmyEa4nbj4IOQyuhc/wy9kY7/WVPcwIO9VA668Pu8RkO7+0G76SLROeyw9CpQ061i4mA==", + "dev": true, + "dependencies": { + "ansi-styles": "^4.1.0", + "supports-color": "^7.1.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/chalk/chalk?sponsor=1" + } + }, + "node_modules/chalk/node_modules/supports-color": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-7.2.0.tgz", + "integrity": "sha512-qpCAvRl9stuOHveKsn7HncJRvv501qIacKzQlO/+Lwxc9+0q2wLyv4Dfvt80/DPn2pqOBsJdDiogXGR9+OvwRw==", + "dev": true, + "dependencies": { + "has-flag": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/chokidar": { + "version": "3.5.3", + "resolved": "https://registry.npmjs.org/chokidar/-/chokidar-3.5.3.tgz", + "integrity": "sha512-Dr3sfKRP6oTcjf2JmUmFJfeVMvXBdegxB0iVQ5eb2V10uFJUCAS8OByZdVAyVb8xXNz3GjjTgj9kLWsZTqE6kw==", + "dev": true, + "funding": [ + { + "type": "individual", + "url": "https://paulmillr.com/funding/" + } + ], + "dependencies": { + "anymatch": "~3.1.2", + "braces": "~3.0.2", + "glob-parent": "~5.1.2", + "is-binary-path": "~2.1.0", + "is-glob": "~4.0.1", + "normalize-path": "~3.0.0", + "readdirp": "~3.6.0" + }, + "engines": { + "node": ">= 8.10.0" + }, + "optionalDependencies": { + "fsevents": "~2.3.2" + } + }, + "node_modules/cliui": { + "version": "7.0.4", + "resolved": "https://registry.npmjs.org/cliui/-/cliui-7.0.4.tgz", + "integrity": "sha512-OcRE68cOsVMXp1Yvonl/fzkQOyjLSu/8bhPDfQt0e0/Eb283TKP20Fs2MqoPsr9SwA595rRCA+QMzYc9nBP+JQ==", + "dev": true, + "dependencies": { + "string-width": "^4.2.0", + "strip-ansi": "^6.0.0", + "wrap-ansi": "^7.0.0" + } + }, + "node_modules/color-convert": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz", + "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==", + "dev": true, + "dependencies": { + "color-name": "~1.1.4" + }, + "engines": { + "node": ">=7.0.0" + } + }, + "node_modules/color-name": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz", + "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==", + "dev": true + }, + "node_modules/concat-map": { + "version": "0.0.1", + "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz", + "integrity": "sha1-2Klr13/Wjfd5OnMDajug1UBdR3s=", + "dev": true + }, + "node_modules/debug": { + "version": "4.3.3", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.3.tgz", + "integrity": "sha512-/zxw5+vh1Tfv+4Qn7a5nsbcJKPaSvCDhojn6FEl9vupwK2VCSDtEiEtqr8DFtzYFOdz63LBkxec7DYuc2jon6Q==", + "dev": true, + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/debug/node_modules/ms": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", + "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==", + "dev": true + }, + "node_modules/decamelize": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/decamelize/-/decamelize-4.0.0.tgz", + "integrity": "sha512-9iE1PgSik9HeIIw2JO94IidnE3eBoQrFJ3w7sFuzSX4DpmZ3v5sZpUiV5Swcf6mQEF+Y0ru8Neo+p+nyh2J+hQ==", + "dev": true, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/deprecation": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/deprecation/-/deprecation-2.3.1.tgz", + "integrity": "sha512-xmHIy4F3scKVwMsQ4WnVaS8bHOx0DmVwRywosKhaILI0ywMDWPtBSku2HNxRvF7jtwDRsoEwYQSfbxj8b7RlJQ==" + }, + "node_modules/diff": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/diff/-/diff-5.0.0.tgz", + "integrity": "sha512-/VTCrvm5Z0JGty/BWHljh+BAiw3IK+2j87NGMu8Nwc/f48WoDAC395uomO9ZD117ZOBaHmkX1oyLvkVM/aIT3w==", + "dev": true, + "engines": { + "node": ">=0.3.1" + } + }, + "node_modules/emoji-regex": { + "version": "8.0.0", + "resolved": "https://registry.npmjs.org/emoji-regex/-/emoji-regex-8.0.0.tgz", + "integrity": "sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A==", + "dev": true + }, + "node_modules/escalade": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/escalade/-/escalade-3.1.1.tgz", + "integrity": "sha512-k0er2gUkLf8O0zKJiAhmkTnJlTvINGv7ygDNPbeIsX/TJjGJZHuh9B2UxbsaEkmlEo9MfhrSzmhIlhRlI2GXnw==", + "dev": true, + "engines": { + "node": ">=6" + } + }, + "node_modules/escape-string-regexp": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz", + "integrity": "sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA==", + "dev": true, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/fill-range": { + "version": "7.0.1", + "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-7.0.1.tgz", + "integrity": "sha512-qOo9F+dMUmC2Lcb4BbVvnKJxTPjCm+RRpe4gDuGrzkL7mEVl/djYSu2OdQ2Pa302N4oqkSg9ir6jaLWJ2USVpQ==", + "dev": true, + "dependencies": { + "to-regex-range": "^5.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/find-up": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-5.0.0.tgz", + "integrity": "sha512-78/PXT1wlLLDgTzDs7sjq9hzz0vXD+zn+7wypEe4fXQxCmdmqfGsEPQxmiCSQI3ajFV91bVSsvNtrJRiW6nGng==", + "dev": true, + "dependencies": { + "locate-path": "^6.0.0", + "path-exists": "^4.0.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/flat": { + "version": "5.0.2", + "resolved": "https://registry.npmjs.org/flat/-/flat-5.0.2.tgz", + "integrity": "sha512-b6suED+5/3rTpUBdG1gupIl8MPFCAMA0QXwmljLhvCUKcUvdE4gWky9zpuGCcXHOsz4J9wPGNWq6OKpmIzz3hQ==", + "dev": true, + "bin": { + "flat": "cli.js" + } + }, + "node_modules/fs.realpath": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/fs.realpath/-/fs.realpath-1.0.0.tgz", + "integrity": "sha1-FQStJSMVjKpA20onh8sBQRmU6k8=", + "dev": true + }, + "node_modules/fsevents": { + "version": "2.3.2", + "resolved": "https://registry.npmjs.org/fsevents/-/fsevents-2.3.2.tgz", + "integrity": "sha512-xiqMQR4xAeHTuB9uWm+fFRcIOgKBMiOBP+eXiyT7jsgVCq1bkVygt00oASowB7EdtpOHaaPgKt812P9ab+DDKA==", + "dev": true, + "hasInstallScript": true, + "optional": true, + "os": [ + "darwin" + ], + "engines": { + "node": "^8.16.0 || ^10.6.0 || >=11.0.0" + } + }, + "node_modules/get-caller-file": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/get-caller-file/-/get-caller-file-2.0.5.tgz", + "integrity": "sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg==", + "dev": true, + "engines": { + "node": "6.* || 8.* || >= 10.*" + } + }, + "node_modules/glob": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/glob/-/glob-7.2.0.tgz", + "integrity": "sha512-lmLf6gtyrPq8tTjSmrO94wBeQbFR3HbLHbuyD69wuyQkImp2hWqMGB47OX65FBkPffO641IP9jWa1z4ivqG26Q==", + "dev": true, + "dependencies": { + "fs.realpath": "^1.0.0", + "inflight": "^1.0.4", + "inherits": "2", + "minimatch": "^3.0.4", + "once": "^1.3.0", + "path-is-absolute": "^1.0.0" + }, + "engines": { + "node": "*" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" + } + }, + "node_modules/glob-parent": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-5.1.2.tgz", + "integrity": "sha512-AOIgSQCepiJYwP3ARnGx+5VnTu2HBYdzbGP45eLw1vr3zB3vZLeyed1sC9hnbcOc9/SrMyM5RPQrkGz4aS9Zow==", + "dev": true, + "dependencies": { + "is-glob": "^4.0.1" + }, + "engines": { + "node": ">= 6" + } + }, + "node_modules/growl": { + "version": "1.10.5", + "resolved": "https://registry.npmjs.org/growl/-/growl-1.10.5.tgz", + "integrity": "sha512-qBr4OuELkhPenW6goKVXiv47US3clb3/IbuWF9KNKEijAy9oeHxU9IgzjvJhHkUzhaj7rOUD7+YGWqUjLp5oSA==", + "dev": true, + "engines": { + "node": ">=4.x" + } + }, + "node_modules/has-flag": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz", + "integrity": "sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==", + "dev": true, + "engines": { + "node": ">=8" + } + }, + "node_modules/he": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/he/-/he-1.2.0.tgz", + "integrity": "sha512-F/1DnUGPopORZi0ni+CvrCgHQ5FyEAHRLSApuYWMmrbSwoN2Mn/7k+Gl38gJnR7yyDZk6WLXwiGod1JOWNDKGw==", + "dev": true, + "bin": { + "he": "bin/he" + } + }, + "node_modules/inflight": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/inflight/-/inflight-1.0.6.tgz", + "integrity": "sha1-Sb1jMdfQLQwJvJEKEHW6gWW1bfk=", + "dev": true, + "dependencies": { + "once": "^1.3.0", + "wrappy": "1" + } + }, + "node_modules/inherits": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/inherits/-/inherits-2.0.4.tgz", + "integrity": "sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ==", + "dev": true + }, + "node_modules/is-binary-path": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-binary-path/-/is-binary-path-2.1.0.tgz", + "integrity": "sha512-ZMERYes6pDydyuGidse7OsHxtbI7WVeUEozgR/g7rd0xUimYNlvZRE/K2MgZTjWy725IfelLeVcEM97mmtRGXw==", + "dev": true, + "dependencies": { + "binary-extensions": "^2.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/is-extglob": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/is-extglob/-/is-extglob-2.1.1.tgz", + "integrity": "sha1-qIwCU1eR8C7TfHahueqXc8gz+MI=", + "dev": true, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-fullwidth-code-point": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/is-fullwidth-code-point/-/is-fullwidth-code-point-3.0.0.tgz", + "integrity": "sha512-zymm5+u+sCsSWyD9qNaejV3DFvhCKclKdizYaJUuHA83RLjb7nSuGnddCHGv0hk+KY7BMAlsWeK4Ueg6EV6XQg==", + "dev": true, + "engines": { + "node": ">=8" + } + }, + "node_modules/is-glob": { + "version": "4.0.3", + "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-4.0.3.tgz", + "integrity": "sha512-xelSayHH36ZgE7ZWhli7pW34hNbNl8Ojv5KVmkJD4hBdD3th8Tfk9vYasLM+mXWOZhFkgZfxhLSnrwRr4elSSg==", + "dev": true, + "dependencies": { + "is-extglob": "^2.1.1" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-number": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/is-number/-/is-number-7.0.0.tgz", + "integrity": "sha512-41Cifkg6e8TylSpdtTpeLVMqvSBEVzTttHvERD741+pnZ8ANv0004MRL43QKPDlK9cGvNp6NZWZUBlbGXYxxng==", + "dev": true, + "engines": { + "node": ">=0.12.0" + } + }, + "node_modules/is-plain-obj": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-plain-obj/-/is-plain-obj-2.1.0.tgz", + "integrity": "sha512-YWnfyRwxL/+SsrWYfOpUtz5b3YD+nyfkHvjbcanzk8zgyO4ASD67uVMRt8k5bM4lLMDnXfriRhOpemw+NfT1eA==", + "dev": true, + "engines": { + "node": ">=8" + } + }, + "node_modules/is-plain-object": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/is-plain-object/-/is-plain-object-5.0.0.tgz", + "integrity": "sha512-VRSzKkbMm5jMDoKLbltAkFQ5Qr7VDiTFGXxYFXXowVj387GeGNOCsOH6Msy00SGZ3Fp84b1Naa1psqgcCIEP5Q==", + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-unicode-supported": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/is-unicode-supported/-/is-unicode-supported-0.1.0.tgz", + "integrity": "sha512-knxG2q4UC3u8stRGyAVJCOdxFmv5DZiRcdlIaAQXAbSfJya+OhopNotLQrstBhququ4ZpuKbDc/8S6mgXgPFPw==", + "dev": true, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/isexe": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/isexe/-/isexe-2.0.0.tgz", + "integrity": "sha1-6PvzdNxVb/iUehDcsFctYz8s+hA=", + "dev": true + }, + "node_modules/js-yaml": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-4.1.0.tgz", + "integrity": "sha512-wpxZs9NoxZaJESJGIZTyDEaYpl0FKSA+FB9aJiyemKhMwkxQg63h4T1KJgUGHpTqPDNRcmmYLugrRjJlBtWvRA==", + "dependencies": { + "argparse": "^2.0.1" + }, + "bin": { + "js-yaml": "bin/js-yaml.js" + } + }, + "node_modules/locate-path": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-6.0.0.tgz", + "integrity": "sha512-iPZK6eYjbxRu3uB4/WZ3EsEIMJFMqAoopl3R+zuq0UjcAm/MO6KCweDgPfP3elTztoKP3KtnVHxTn2NHBSDVUw==", + "dev": true, + "dependencies": { + "p-locate": "^5.0.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/log-symbols": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/log-symbols/-/log-symbols-4.1.0.tgz", + "integrity": "sha512-8XPvpAA8uyhfteu8pIvQxpJZ7SYYdpUivZpGy6sFsBuKRY/7rQGavedeB8aK+Zkyq6upMFVL/9AW6vOYzfRyLg==", + "dev": true, + "dependencies": { + "chalk": "^4.1.0", + "is-unicode-supported": "^0.1.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/minimatch": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.0.4.tgz", + "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==", + "dev": true, + "dependencies": { + "brace-expansion": "^1.1.7" + }, + "engines": { + "node": "*" + } + }, + "node_modules/mocha": { + "version": "9.2.0", + "resolved": "https://registry.npmjs.org/mocha/-/mocha-9.2.0.tgz", + "integrity": "sha512-kNn7E8g2SzVcq0a77dkphPsDSN7P+iYkqE0ZsGCYWRsoiKjOt+NvXfaagik8vuDa6W5Zw3qxe8Jfpt5qKf+6/Q==", + "dev": true, + "dependencies": { + "@ungap/promise-all-settled": "1.1.2", + "ansi-colors": "4.1.1", + "browser-stdout": "1.3.1", + "chokidar": "3.5.3", + "debug": "4.3.3", + "diff": "5.0.0", + "escape-string-regexp": "4.0.0", + "find-up": "5.0.0", + "glob": "7.2.0", + "growl": "1.10.5", + "he": "1.2.0", + "js-yaml": "4.1.0", + "log-symbols": "4.1.0", + "minimatch": "3.0.4", + "ms": "2.1.3", + "nanoid": "3.2.0", + "serialize-javascript": "6.0.0", + "strip-json-comments": "3.1.1", + "supports-color": "8.1.1", + "which": "2.0.2", + "workerpool": "6.2.0", + "yargs": "16.2.0", + "yargs-parser": "20.2.4", + "yargs-unparser": "2.0.0" + }, + "bin": { + "_mocha": "bin/_mocha", + "mocha": "bin/mocha" + }, + "engines": { + "node": ">= 12.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/mochajs" + } + }, + "node_modules/ms": { + "version": "2.1.3", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.3.tgz", + "integrity": "sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA==", + "dev": true + }, + "node_modules/nanoid": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/nanoid/-/nanoid-3.2.0.tgz", + "integrity": "sha512-fmsZYa9lpn69Ad5eDn7FMcnnSR+8R34W9qJEijxYhTbfOWzr22n1QxCMzXLK+ODyW2973V3Fux959iQoUxzUIA==", + "dev": true, + "bin": { + "nanoid": "bin/nanoid.cjs" + }, + "engines": { + "node": "^10 || ^12 || ^13.7 || ^14 || >=15.0.1" + } + }, + "node_modules/node-fetch": { + "version": "2.6.7", + "resolved": "https://registry.npmjs.org/node-fetch/-/node-fetch-2.6.7.tgz", + "integrity": "sha512-ZjMPFEfVx5j+y2yF35Kzx5sF7kDzxuDj6ziH4FFbOp87zKDZNx8yExJIb05OGF4Nlt9IHFIMBkRl41VdvcNdbQ==", + "dependencies": { + "whatwg-url": "^5.0.0" + }, + "engines": { + "node": "4.x || >=6.0.0" + }, + "peerDependencies": { + "encoding": "^0.1.0" + }, + "peerDependenciesMeta": { + "encoding": { + "optional": true + } + } + }, + "node_modules/normalize-path": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/normalize-path/-/normalize-path-3.0.0.tgz", + "integrity": "sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA==", + "dev": true, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/once": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/once/-/once-1.4.0.tgz", + "integrity": "sha1-WDsap3WWHUsROsF9nFC6753Xa9E=", + "dependencies": { + "wrappy": "1" + } + }, + "node_modules/p-limit": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-3.1.0.tgz", + "integrity": "sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ==", + "dev": true, + "dependencies": { + "yocto-queue": "^0.1.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/p-locate": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-5.0.0.tgz", + "integrity": "sha512-LaNjtRWUBY++zB5nE/NwcaoMylSPk+S+ZHNB1TzdbMJMny6dynpAGt7X/tl/QYq3TIeE6nxHppbo2LGymrG5Pw==", + "dev": true, + "dependencies": { + "p-limit": "^3.0.2" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/path-exists": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/path-exists/-/path-exists-4.0.0.tgz", + "integrity": "sha512-ak9Qy5Q7jYb2Wwcey5Fpvg2KoAc/ZIhLSLOSBmRmygPsGwkVVt0fZa0qrtMz+m6tJTAHfZQ8FnmB4MG4LWy7/w==", + "dev": true, + "engines": { + "node": ">=8" + } + }, + "node_modules/path-is-absolute": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/path-is-absolute/-/path-is-absolute-1.0.1.tgz", + "integrity": "sha1-F0uSaHNVNP+8es5r9TpanhtcX18=", + "dev": true, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/picomatch": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/picomatch/-/picomatch-2.3.1.tgz", + "integrity": "sha512-JU3teHTNjmE2VCGFzuY8EXzCDVwEqB2a8fsIvwaStHhAWJEeVd1o1QD80CU6+ZdEXXSLbSsuLwJjkCBWqRQUVA==", + "dev": true, + "engines": { + "node": ">=8.6" + }, + "funding": { + "url": "https://github.com/sponsors/jonschlinkert" + } + }, + "node_modules/prettier": { + "version": "2.5.1", + "resolved": "https://registry.npmjs.org/prettier/-/prettier-2.5.1.tgz", + "integrity": "sha512-vBZcPRUR5MZJwoyi3ZoyQlc1rXeEck8KgeC9AwwOn+exuxLxq5toTRDTSaVrXHxelDMHy9zlicw8u66yxoSUFg==", + "bin": { + "prettier": "bin-prettier.js" + }, + "engines": { + "node": ">=10.13.0" + } + }, + "node_modules/randombytes": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/randombytes/-/randombytes-2.1.0.tgz", + "integrity": "sha512-vYl3iOX+4CKUWuxGi9Ukhie6fsqXqS9FE2Zaic4tNFD2N2QQaXOMFbuKK4QmDHC0JO6B1Zp41J0LpT0oR68amQ==", + "dev": true, + "dependencies": { + "safe-buffer": "^5.1.0" + } + }, + "node_modules/readdirp": { + "version": "3.6.0", + "resolved": "https://registry.npmjs.org/readdirp/-/readdirp-3.6.0.tgz", + "integrity": "sha512-hOS089on8RduqdbhvQ5Z37A0ESjsqz6qnRcffsMU3495FuTdqSm+7bhJ29JvIOsBDEEnan5DPu9t3To9VRlMzA==", + "dev": true, + "dependencies": { + "picomatch": "^2.2.1" + }, + "engines": { + "node": ">=8.10.0" + } + }, + "node_modules/require-directory": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/require-directory/-/require-directory-2.1.1.tgz", + "integrity": "sha1-jGStX9MNqxyXbiNE/+f3kqam30I=", + "dev": true, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/safe-buffer": { + "version": "5.2.1", + "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.2.1.tgz", + "integrity": "sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ==", + "dev": true, + "funding": [ + { + "type": "github", + "url": "https://github.com/sponsors/feross" + }, + { + "type": "patreon", + "url": "https://www.patreon.com/feross" + }, + { + "type": "consulting", + "url": "https://feross.org/support" + } + ] + }, + "node_modules/serialize-javascript": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.0.tgz", + "integrity": "sha512-Qr3TosvguFt8ePWqsvRfrKyQXIiW+nGbYpy8XK24NQHE83caxWt+mIymTT19DGFbNWNLfEwsrkSmN64lVWB9ag==", + "dev": true, + "dependencies": { + "randombytes": "^2.1.0" + } + }, + "node_modules/string-width": { + "version": "4.2.3", + "resolved": "https://registry.npmjs.org/string-width/-/string-width-4.2.3.tgz", + "integrity": "sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g==", + "dev": true, + "dependencies": { + "emoji-regex": "^8.0.0", + "is-fullwidth-code-point": "^3.0.0", + "strip-ansi": "^6.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/strip-ansi": { + "version": "6.0.1", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-6.0.1.tgz", + "integrity": "sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A==", + "dev": true, + "dependencies": { + "ansi-regex": "^5.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/strip-json-comments": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/strip-json-comments/-/strip-json-comments-3.1.1.tgz", + "integrity": "sha512-6fPc+R4ihwqP6N/aIv2f1gMH8lOVtWQHoqC4yK6oSDVVocumAsfCqjkXnqiYMhmMwS/mEHLp7Vehlt3ql6lEig==", + "dev": true, + "engines": { + "node": ">=8" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/supports-color": { + "version": "8.1.1", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-8.1.1.tgz", + "integrity": "sha512-MpUEN2OodtUzxvKQl72cUF7RQ5EiHsGvSsVG0ia9c5RbWGL2CI4C7EpPS8UTBIplnlzZiNuV56w+FuNxy3ty2Q==", + "dev": true, + "dependencies": { + "has-flag": "^4.0.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/chalk/supports-color?sponsor=1" + } + }, + "node_modules/to-regex-range": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/to-regex-range/-/to-regex-range-5.0.1.tgz", + "integrity": "sha512-65P7iz6X5yEr1cwcgvQxbbIw7Uk3gOy5dIdtZ4rDveLqhrdJP+Li/Hx6tyK0NEb+2GCyneCMJiGqrADCSNk8sQ==", + "dev": true, + "dependencies": { + "is-number": "^7.0.0" + }, + "engines": { + "node": ">=8.0" + } + }, + "node_modules/tr46": { + "version": "0.0.3", + "resolved": "https://registry.npmjs.org/tr46/-/tr46-0.0.3.tgz", + "integrity": "sha1-gYT9NH2snNwYWZLzpmIuFLnZq2o=" + }, + "node_modules/tunnel": { + "version": "0.0.6", + "resolved": "https://registry.npmjs.org/tunnel/-/tunnel-0.0.6.tgz", + "integrity": "sha512-1h/Lnq9yajKY2PEbBadPXj3VxsDDu844OnaAo52UVmIzIvwwtBPIuNvkjuzBlTWpfJyUbG3ez0KSBibQkj4ojg==", + "engines": { + "node": ">=0.6.11 <=0.7.0 || >=0.7.3" + } + }, + "node_modules/typescript": { + "version": "4.2.4", + "resolved": "https://registry.npmjs.org/typescript/-/typescript-4.2.4.tgz", + "integrity": "sha512-V+evlYHZnQkaz8TRBuxTA92yZBPotr5H+WhQ7bD3hZUndx5tGOa1fuCgeSjxAzM1RiN5IzvadIXTVefuuwZCRg==", + "dev": true, + "bin": { + "tsc": "bin/tsc", + "tsserver": "bin/tsserver" + }, + "engines": { + "node": ">=4.2.0" + } + }, + "node_modules/universal-user-agent": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/universal-user-agent/-/universal-user-agent-6.0.0.tgz", + "integrity": "sha512-isyNax3wXoKaulPDZWHQqbmIx1k2tb9fb3GGDBRxCscfYV2Ch7WxPArBsFEG8s/safwXTT7H4QGhaIkTp9447w==" + }, + "node_modules/webidl-conversions": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/webidl-conversions/-/webidl-conversions-3.0.1.tgz", + "integrity": "sha1-JFNCdeKnvGvnvIZhHMFq4KVlSHE=" + }, + "node_modules/whatwg-url": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/whatwg-url/-/whatwg-url-5.0.0.tgz", + "integrity": "sha1-lmRU6HZUYuN2RNNib2dCzotwll0=", + "dependencies": { + "tr46": "~0.0.3", + "webidl-conversions": "^3.0.0" + } + }, + "node_modules/which": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/which/-/which-2.0.2.tgz", + "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==", + "dev": true, + "dependencies": { + "isexe": "^2.0.0" + }, + "bin": { + "node-which": "bin/node-which" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/workerpool": { + "version": "6.2.0", + "resolved": "https://registry.npmjs.org/workerpool/-/workerpool-6.2.0.tgz", + "integrity": "sha512-Rsk5qQHJ9eowMH28Jwhe8HEbmdYDX4lwoMWshiCXugjtHqMD9ZbiqSDLxcsfdqsETPzVUtX5s1Z5kStiIM6l4A==", + "dev": true + }, + "node_modules/wrap-ansi": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-7.0.0.tgz", + "integrity": "sha512-YVGIj2kamLSTxw6NsZjoBxfSwsn0ycdesmc4p+Q21c5zPuZ1pl+NfxVdxPtdHvmNVOQ6XSYG4AUtyt/Fi7D16Q==", + "dev": true, + "dependencies": { + "ansi-styles": "^4.0.0", + "string-width": "^4.1.0", + "strip-ansi": "^6.0.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/chalk/wrap-ansi?sponsor=1" + } + }, + "node_modules/wrappy": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/wrappy/-/wrappy-1.0.2.tgz", + "integrity": "sha1-tSQ9jz7BqjXxNkYFvA0QNuMKtp8=" + }, + "node_modules/y18n": { + "version": "5.0.8", + "resolved": "https://registry.npmjs.org/y18n/-/y18n-5.0.8.tgz", + "integrity": "sha512-0pfFzegeDWJHJIAmTLRP2DwHjdF5s7jo9tuztdQxAhINCdvS+3nGINqPd00AphqJR/0LhANUS6/+7SCb98YOfA==", + "dev": true, + "engines": { + "node": ">=10" + } + }, + "node_modules/yargs": { + "version": "16.2.0", + "resolved": "https://registry.npmjs.org/yargs/-/yargs-16.2.0.tgz", + "integrity": "sha512-D1mvvtDG0L5ft/jGWkLpG1+m0eQxOfaBvTNELraWj22wSVUMWxZUvYgJYcKh6jGGIkJFhH4IZPQhR4TKpc8mBw==", + "dev": true, + "dependencies": { + "cliui": "^7.0.2", + "escalade": "^3.1.1", + "get-caller-file": "^2.0.5", + "require-directory": "^2.1.1", + "string-width": "^4.2.0", + "y18n": "^5.0.5", + "yargs-parser": "^20.2.2" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/yargs-parser": { + "version": "20.2.4", + "resolved": "https://registry.npmjs.org/yargs-parser/-/yargs-parser-20.2.4.tgz", + "integrity": "sha512-WOkpgNhPTlE73h4VFAFsOnomJVaovO8VqLDzy5saChRBFQFBoMYirowyW+Q9HB4HFF4Z7VZTiG3iSzJJA29yRA==", + "dev": true, + "engines": { + "node": ">=10" + } + }, + "node_modules/yargs-unparser": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/yargs-unparser/-/yargs-unparser-2.0.0.tgz", + "integrity": "sha512-7pRTIA9Qc1caZ0bZ6RYRGbHJthJWuakf+WmHK0rVeLkNrrGhfoabBNdue6kdINI6r4if7ocq9aD/n7xwKOdzOA==", + "dev": true, + "dependencies": { + "camelcase": "^6.0.0", + "decamelize": "^4.0.0", + "flat": "^5.0.2", + "is-plain-obj": "^2.1.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/yocto-queue": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/yocto-queue/-/yocto-queue-0.1.0.tgz", + "integrity": "sha512-rVksvsnNCdJ/ohGc6xgPwyN8eheCxsiLM8mxuE/t/mOVqJewPuO1miLpTHQiRgTKCLexL4MeAFVagts7HmNZ2Q==", + "dev": true, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + } + }, + "dependencies": { + "@actions/exec": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@actions/exec/-/exec-1.1.0.tgz", + "integrity": "sha512-LImpN9AY0J1R1mEYJjVJfSZWU4zYOlEcwSTgPve1rFQqK5AwrEs6uWW5Rv70gbDIQIAUwI86z6B+9mPK4w9Sbg==", + "requires": { + "@actions/io": "^1.0.1" + } + }, + "@actions/github": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/@actions/github/-/github-5.0.0.tgz", + "integrity": "sha512-QvE9eAAfEsS+yOOk0cylLBIO/d6WyWIOvsxxzdrPFaud39G6BOkUwScXZn1iBzQzHyu9SBkkLSWlohDWdsasAQ==", + "requires": { + "@actions/http-client": "^1.0.11", + "@octokit/core": "^3.4.0", + "@octokit/plugin-paginate-rest": "^2.13.3", + "@octokit/plugin-rest-endpoint-methods": "^5.1.1" + } + }, + "@actions/http-client": { + "version": "1.0.11", + "resolved": "https://registry.npmjs.org/@actions/http-client/-/http-client-1.0.11.tgz", + "integrity": "sha512-VRYHGQV1rqnROJqdMvGUbY/Kn8vriQe/F9HR2AlYHzmKuM/p3kjNuXhmdBfcVgsvRWTz5C5XW5xvndZrVBuAYg==", + "requires": { + "tunnel": "0.0.6" + } + }, + "@actions/io": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/@actions/io/-/io-1.1.1.tgz", + "integrity": "sha512-Qi4JoKXjmE0O67wAOH6y0n26QXhMKMFo7GD/4IXNVcrtLjUlGjGuVys6pQgwF3ArfGTQu0XpqaNr0YhED2RaRA==" + }, + "@octokit/auth-token": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/@octokit/auth-token/-/auth-token-2.5.0.tgz", + "integrity": "sha512-r5FVUJCOLl19AxiuZD2VRZ/ORjp/4IN98Of6YJoJOkY75CIBuYfmiNHGrDwXr+aLGG55igl9QrxX3hbiXlLb+g==", + "requires": { + "@octokit/types": "^6.0.3" + } + }, + "@octokit/core": { + "version": "3.5.1", + "resolved": "https://registry.npmjs.org/@octokit/core/-/core-3.5.1.tgz", + "integrity": "sha512-omncwpLVxMP+GLpLPgeGJBF6IWJFjXDS5flY5VbppePYX9XehevbDykRH9PdCdvqt9TS5AOTiDide7h0qrkHjw==", + "requires": { + "@octokit/auth-token": "^2.4.4", + "@octokit/graphql": "^4.5.8", + "@octokit/request": "^5.6.0", + "@octokit/request-error": "^2.0.5", + "@octokit/types": "^6.0.3", + "before-after-hook": "^2.2.0", + "universal-user-agent": "^6.0.0" + } + }, + "@octokit/endpoint": { + "version": "6.0.12", + "resolved": "https://registry.npmjs.org/@octokit/endpoint/-/endpoint-6.0.12.tgz", + "integrity": "sha512-lF3puPwkQWGfkMClXb4k/eUT/nZKQfxinRWJrdZaJO85Dqwo/G0yOC434Jr2ojwafWJMYqFGFa5ms4jJUgujdA==", + "requires": { + "@octokit/types": "^6.0.3", + "is-plain-object": "^5.0.0", + "universal-user-agent": "^6.0.0" + } + }, + "@octokit/graphql": { + "version": "4.8.0", + "resolved": "https://registry.npmjs.org/@octokit/graphql/-/graphql-4.8.0.tgz", + "integrity": "sha512-0gv+qLSBLKF0z8TKaSKTsS39scVKF9dbMxJpj3U0vC7wjNWFuIpL/z76Qe2fiuCbDRcJSavkXsVtMS6/dtQQsg==", + "requires": { + "@octokit/request": "^5.6.0", + "@octokit/types": "^6.0.3", + "universal-user-agent": "^6.0.0" + } + }, + "@octokit/openapi-types": { + "version": "11.2.0", + "resolved": "https://registry.npmjs.org/@octokit/openapi-types/-/openapi-types-11.2.0.tgz", + "integrity": "sha512-PBsVO+15KSlGmiI8QAzaqvsNlZlrDlyAJYcrXBCvVUxCp7VnXjkwPoFHgjEJXx3WF9BAwkA6nfCUA7i9sODzKA==" + }, + "@octokit/plugin-paginate-rest": { + "version": "2.17.0", + "resolved": "https://registry.npmjs.org/@octokit/plugin-paginate-rest/-/plugin-paginate-rest-2.17.0.tgz", + "integrity": "sha512-tzMbrbnam2Mt4AhuyCHvpRkS0oZ5MvwwcQPYGtMv4tUa5kkzG58SVB0fcsLulOZQeRnOgdkZWkRUiyBlh0Bkyw==", + "requires": { + "@octokit/types": "^6.34.0" + } + }, + "@octokit/plugin-request-log": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/@octokit/plugin-request-log/-/plugin-request-log-1.0.4.tgz", + "integrity": "sha512-mLUsMkgP7K/cnFEw07kWqXGF5LKrOkD+lhCrKvPHXWDywAwuDUeDwWBpc69XK3pNX0uKiVt8g5z96PJ6z9xCFA==", + "requires": {} + }, + "@octokit/plugin-rest-endpoint-methods": { + "version": "5.13.0", + "resolved": "https://registry.npmjs.org/@octokit/plugin-rest-endpoint-methods/-/plugin-rest-endpoint-methods-5.13.0.tgz", + "integrity": "sha512-uJjMTkN1KaOIgNtUPMtIXDOjx6dGYysdIFhgA52x4xSadQCz3b/zJexvITDVpANnfKPW/+E0xkOvLntqMYpviA==", + "requires": { + "@octokit/types": "^6.34.0", + "deprecation": "^2.3.1" + } + }, + "@octokit/request": { + "version": "5.6.3", + "resolved": "https://registry.npmjs.org/@octokit/request/-/request-5.6.3.tgz", + "integrity": "sha512-bFJl0I1KVc9jYTe9tdGGpAMPy32dLBXXo1dS/YwSCTL/2nd9XeHsY616RE3HPXDVk+a+dBuzyz5YdlXwcDTr2A==", + "requires": { + "@octokit/endpoint": "^6.0.1", + "@octokit/request-error": "^2.1.0", + "@octokit/types": "^6.16.1", + "is-plain-object": "^5.0.0", + "node-fetch": "^2.6.7", + "universal-user-agent": "^6.0.0" + } + }, + "@octokit/request-error": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/@octokit/request-error/-/request-error-2.1.0.tgz", + "integrity": "sha512-1VIvgXxs9WHSjicsRwq8PlR2LR2x6DwsJAaFgzdi0JfJoGSO8mYI/cHJQ+9FbN21aa+DrgNLnwObmyeSC8Rmpg==", + "requires": { + "@octokit/types": "^6.0.3", + "deprecation": "^2.0.0", + "once": "^1.4.0" + } + }, + "@octokit/rest": { + "version": "18.12.0", + "resolved": "https://registry.npmjs.org/@octokit/rest/-/rest-18.12.0.tgz", + "integrity": "sha512-gDPiOHlyGavxr72y0guQEhLsemgVjwRePayJ+FcKc2SJqKUbxbkvf5kAZEWA/MKvsfYlQAMVzNJE3ezQcxMJ2Q==", + "requires": { + "@octokit/core": "^3.5.1", + "@octokit/plugin-paginate-rest": "^2.16.8", + "@octokit/plugin-request-log": "^1.0.4", + "@octokit/plugin-rest-endpoint-methods": "^5.12.0" + } + }, + "@octokit/types": { + "version": "6.34.0", + "resolved": "https://registry.npmjs.org/@octokit/types/-/types-6.34.0.tgz", + "integrity": "sha512-s1zLBjWhdEI2zwaoSgyOFoKSl109CUcVBCc7biPJ3aAf6LGLU6szDvi31JPU7bxfla2lqfhjbbg/5DdFNxOwHw==", + "requires": { + "@octokit/openapi-types": "^11.2.0" + } + }, + "@types/mocha": { + "version": "9.1.0", + "resolved": "https://registry.npmjs.org/@types/mocha/-/mocha-9.1.0.tgz", + "integrity": "sha512-QCWHkbMv4Y5U9oW10Uxbr45qMMSzl4OzijsozynUAgx3kEHUdXB00udx2dWDQ7f2TU2a2uuiFaRZjCe3unPpeg==", + "dev": true + }, + "@types/node": { + "version": "16.11.25", + "resolved": "https://registry.npmjs.org/@types/node/-/node-16.11.25.tgz", + "integrity": "sha512-NrTwfD7L1RTc2qrHQD4RTTy4p0CO2LatKBEKEds3CaVuhoM/+DJzmWZl5f+ikR8cm8F5mfJxK+9rQq07gRiSjQ==", + "dev": true + }, + "@ungap/promise-all-settled": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@ungap/promise-all-settled/-/promise-all-settled-1.1.2.tgz", + "integrity": "sha512-sL/cEvJWAnClXw0wHk85/2L0G6Sj8UB0Ctc1TEMbKSsmpRosqhwj9gWgFRZSrBr2f9tiXISwNhCPmlfqUqyb9Q==", + "dev": true + }, + "ansi-colors": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/ansi-colors/-/ansi-colors-4.1.1.tgz", + "integrity": "sha512-JoX0apGbHaUJBNl6yF+p6JAFYZ666/hhCGKN5t9QFjbJQKUU/g8MNbFDbvfrgKXvI1QpZplPOnwIo99lX/AAmA==", + "dev": true + }, + "ansi-regex": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-5.0.1.tgz", + "integrity": "sha512-quJQXlTSUGL2LH9SUXo8VwsY4soanhgo6LNSm84E1LBcE8s3O0wpdiRzyR9z/ZZJMlMWv37qOOb9pdJlMUEKFQ==", + "dev": true + }, + "ansi-styles": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-4.3.0.tgz", + "integrity": "sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg==", + "dev": true, + "requires": { + "color-convert": "^2.0.1" + } + }, + "anymatch": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/anymatch/-/anymatch-3.1.2.tgz", + "integrity": "sha512-P43ePfOAIupkguHUycrc4qJ9kz8ZiuOUijaETwX7THt0Y/GNK7v0aa8rY816xWjZ7rJdA5XdMcpVFTKMq+RvWg==", + "dev": true, + "requires": { + "normalize-path": "^3.0.0", + "picomatch": "^2.0.4" + } + }, + "argparse": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/argparse/-/argparse-2.0.1.tgz", + "integrity": "sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q==" + }, + "balanced-match": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-1.0.2.tgz", + "integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==", + "dev": true + }, + "before-after-hook": { + "version": "2.2.2", + "resolved": "https://registry.npmjs.org/before-after-hook/-/before-after-hook-2.2.2.tgz", + "integrity": "sha512-3pZEU3NT5BFUo/AD5ERPWOgQOCZITni6iavr5AUw5AUwQjMlI0kzu5btnyD39AF0gUEsDPwJT+oY1ORBJijPjQ==" + }, + "binary-extensions": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/binary-extensions/-/binary-extensions-2.2.0.tgz", + "integrity": "sha512-jDctJ/IVQbZoJykoeHbhXpOlNBqGNcwXJKJog42E5HDPUwQTSdjCHdihjj0DlnheQ7blbT6dHOafNAiS8ooQKA==", + "dev": true + }, + "brace-expansion": { + "version": "1.1.11", + "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-1.1.11.tgz", + "integrity": "sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA==", + "dev": true, + "requires": { + "balanced-match": "^1.0.0", + "concat-map": "0.0.1" + } + }, + "braces": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.2.tgz", + "integrity": "sha512-b8um+L1RzM3WDSzvhm6gIz1yfTbBt6YTlcEKAvsmqCZZFw46z626lVj9j1yEPW33H5H+lBQpZMP1k8l+78Ha0A==", + "dev": true, + "requires": { + "fill-range": "^7.0.1" + } + }, + "browser-stdout": { + "version": "1.3.1", + "resolved": "https://registry.npmjs.org/browser-stdout/-/browser-stdout-1.3.1.tgz", + "integrity": "sha512-qhAVI1+Av2X7qelOfAIYwXONood6XlZE/fXaBSmW/T5SzLAmCgzi+eiWE7fUvbHaeNBQH13UftjpXxsfLkMpgw==", + "dev": true + }, + "camelcase": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-6.3.0.tgz", + "integrity": "sha512-Gmy6FhYlCY7uOElZUSbxo2UCDH8owEk996gkbrpsgGtrJLM3J7jGxl9Ic7Qwwj4ivOE5AWZWRMecDdF7hqGjFA==", + "dev": true + }, + "chalk": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-4.1.2.tgz", + "integrity": "sha512-oKnbhFyRIXpUuez8iBMmyEa4nbj4IOQyuhc/wy9kY7/WVPcwIO9VA668Pu8RkO7+0G76SLROeyw9CpQ061i4mA==", + "dev": true, + "requires": { + "ansi-styles": "^4.1.0", + "supports-color": "^7.1.0" + }, + "dependencies": { + "supports-color": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-7.2.0.tgz", + "integrity": "sha512-qpCAvRl9stuOHveKsn7HncJRvv501qIacKzQlO/+Lwxc9+0q2wLyv4Dfvt80/DPn2pqOBsJdDiogXGR9+OvwRw==", + "dev": true, + "requires": { + "has-flag": "^4.0.0" + } + } + } + }, + "chokidar": { + "version": "3.5.3", + "resolved": "https://registry.npmjs.org/chokidar/-/chokidar-3.5.3.tgz", + "integrity": "sha512-Dr3sfKRP6oTcjf2JmUmFJfeVMvXBdegxB0iVQ5eb2V10uFJUCAS8OByZdVAyVb8xXNz3GjjTgj9kLWsZTqE6kw==", + "dev": true, + "requires": { + "anymatch": "~3.1.2", + "braces": "~3.0.2", + "fsevents": "~2.3.2", + "glob-parent": "~5.1.2", + "is-binary-path": "~2.1.0", + "is-glob": "~4.0.1", + "normalize-path": "~3.0.0", + "readdirp": "~3.6.0" + } + }, + "cliui": { + "version": "7.0.4", + "resolved": "https://registry.npmjs.org/cliui/-/cliui-7.0.4.tgz", + "integrity": "sha512-OcRE68cOsVMXp1Yvonl/fzkQOyjLSu/8bhPDfQt0e0/Eb283TKP20Fs2MqoPsr9SwA595rRCA+QMzYc9nBP+JQ==", + "dev": true, + "requires": { + "string-width": "^4.2.0", + "strip-ansi": "^6.0.0", + "wrap-ansi": "^7.0.0" + } + }, + "color-convert": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz", + "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==", + "dev": true, + "requires": { + "color-name": "~1.1.4" + } + }, + "color-name": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz", + "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==", + "dev": true + }, + "concat-map": { + "version": "0.0.1", + "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz", + "integrity": "sha1-2Klr13/Wjfd5OnMDajug1UBdR3s=", + "dev": true + }, + "debug": { + "version": "4.3.3", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.3.tgz", + "integrity": "sha512-/zxw5+vh1Tfv+4Qn7a5nsbcJKPaSvCDhojn6FEl9vupwK2VCSDtEiEtqr8DFtzYFOdz63LBkxec7DYuc2jon6Q==", + "dev": true, + "requires": { + "ms": "2.1.2" + }, + "dependencies": { + "ms": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", + "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==", + "dev": true + } + } + }, + "decamelize": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/decamelize/-/decamelize-4.0.0.tgz", + "integrity": "sha512-9iE1PgSik9HeIIw2JO94IidnE3eBoQrFJ3w7sFuzSX4DpmZ3v5sZpUiV5Swcf6mQEF+Y0ru8Neo+p+nyh2J+hQ==", + "dev": true + }, + "deprecation": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/deprecation/-/deprecation-2.3.1.tgz", + "integrity": "sha512-xmHIy4F3scKVwMsQ4WnVaS8bHOx0DmVwRywosKhaILI0ywMDWPtBSku2HNxRvF7jtwDRsoEwYQSfbxj8b7RlJQ==" + }, + "diff": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/diff/-/diff-5.0.0.tgz", + "integrity": "sha512-/VTCrvm5Z0JGty/BWHljh+BAiw3IK+2j87NGMu8Nwc/f48WoDAC395uomO9ZD117ZOBaHmkX1oyLvkVM/aIT3w==", + "dev": true + }, + "emoji-regex": { + "version": "8.0.0", + "resolved": "https://registry.npmjs.org/emoji-regex/-/emoji-regex-8.0.0.tgz", + "integrity": "sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A==", + "dev": true + }, + "escalade": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/escalade/-/escalade-3.1.1.tgz", + "integrity": "sha512-k0er2gUkLf8O0zKJiAhmkTnJlTvINGv7ygDNPbeIsX/TJjGJZHuh9B2UxbsaEkmlEo9MfhrSzmhIlhRlI2GXnw==", + "dev": true + }, + "escape-string-regexp": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz", + "integrity": "sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA==", + "dev": true + }, + "fill-range": { + "version": "7.0.1", + "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-7.0.1.tgz", + "integrity": "sha512-qOo9F+dMUmC2Lcb4BbVvnKJxTPjCm+RRpe4gDuGrzkL7mEVl/djYSu2OdQ2Pa302N4oqkSg9ir6jaLWJ2USVpQ==", + "dev": true, + "requires": { + "to-regex-range": "^5.0.1" + } + }, + "find-up": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-5.0.0.tgz", + "integrity": "sha512-78/PXT1wlLLDgTzDs7sjq9hzz0vXD+zn+7wypEe4fXQxCmdmqfGsEPQxmiCSQI3ajFV91bVSsvNtrJRiW6nGng==", + "dev": true, + "requires": { + "locate-path": "^6.0.0", + "path-exists": "^4.0.0" + } + }, + "flat": { + "version": "5.0.2", + "resolved": "https://registry.npmjs.org/flat/-/flat-5.0.2.tgz", + "integrity": "sha512-b6suED+5/3rTpUBdG1gupIl8MPFCAMA0QXwmljLhvCUKcUvdE4gWky9zpuGCcXHOsz4J9wPGNWq6OKpmIzz3hQ==", + "dev": true + }, + "fs.realpath": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/fs.realpath/-/fs.realpath-1.0.0.tgz", + "integrity": "sha1-FQStJSMVjKpA20onh8sBQRmU6k8=", + "dev": true + }, + "fsevents": { + "version": "2.3.2", + "resolved": "https://registry.npmjs.org/fsevents/-/fsevents-2.3.2.tgz", + "integrity": "sha512-xiqMQR4xAeHTuB9uWm+fFRcIOgKBMiOBP+eXiyT7jsgVCq1bkVygt00oASowB7EdtpOHaaPgKt812P9ab+DDKA==", + "dev": true, + "optional": true + }, + "get-caller-file": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/get-caller-file/-/get-caller-file-2.0.5.tgz", + "integrity": "sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg==", + "dev": true + }, + "glob": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/glob/-/glob-7.2.0.tgz", + "integrity": "sha512-lmLf6gtyrPq8tTjSmrO94wBeQbFR3HbLHbuyD69wuyQkImp2hWqMGB47OX65FBkPffO641IP9jWa1z4ivqG26Q==", + "dev": true, + "requires": { + "fs.realpath": "^1.0.0", + "inflight": "^1.0.4", + "inherits": "2", + "minimatch": "^3.0.4", + "once": "^1.3.0", + "path-is-absolute": "^1.0.0" + } + }, + "glob-parent": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-5.1.2.tgz", + "integrity": "sha512-AOIgSQCepiJYwP3ARnGx+5VnTu2HBYdzbGP45eLw1vr3zB3vZLeyed1sC9hnbcOc9/SrMyM5RPQrkGz4aS9Zow==", + "dev": true, + "requires": { + "is-glob": "^4.0.1" + } + }, + "growl": { + "version": "1.10.5", + "resolved": "https://registry.npmjs.org/growl/-/growl-1.10.5.tgz", + "integrity": "sha512-qBr4OuELkhPenW6goKVXiv47US3clb3/IbuWF9KNKEijAy9oeHxU9IgzjvJhHkUzhaj7rOUD7+YGWqUjLp5oSA==", + "dev": true + }, + "has-flag": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz", + "integrity": "sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==", + "dev": true + }, + "he": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/he/-/he-1.2.0.tgz", + "integrity": "sha512-F/1DnUGPopORZi0ni+CvrCgHQ5FyEAHRLSApuYWMmrbSwoN2Mn/7k+Gl38gJnR7yyDZk6WLXwiGod1JOWNDKGw==", + "dev": true + }, + "inflight": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/inflight/-/inflight-1.0.6.tgz", + "integrity": "sha1-Sb1jMdfQLQwJvJEKEHW6gWW1bfk=", + "dev": true, + "requires": { + "once": "^1.3.0", + "wrappy": "1" + } + }, + "inherits": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/inherits/-/inherits-2.0.4.tgz", + "integrity": "sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ==", + "dev": true + }, + "is-binary-path": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-binary-path/-/is-binary-path-2.1.0.tgz", + "integrity": "sha512-ZMERYes6pDydyuGidse7OsHxtbI7WVeUEozgR/g7rd0xUimYNlvZRE/K2MgZTjWy725IfelLeVcEM97mmtRGXw==", + "dev": true, + "requires": { + "binary-extensions": "^2.0.0" + } + }, + "is-extglob": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/is-extglob/-/is-extglob-2.1.1.tgz", + "integrity": "sha1-qIwCU1eR8C7TfHahueqXc8gz+MI=", + "dev": true + }, + "is-fullwidth-code-point": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/is-fullwidth-code-point/-/is-fullwidth-code-point-3.0.0.tgz", + "integrity": "sha512-zymm5+u+sCsSWyD9qNaejV3DFvhCKclKdizYaJUuHA83RLjb7nSuGnddCHGv0hk+KY7BMAlsWeK4Ueg6EV6XQg==", + "dev": true + }, + "is-glob": { + "version": "4.0.3", + "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-4.0.3.tgz", + "integrity": "sha512-xelSayHH36ZgE7ZWhli7pW34hNbNl8Ojv5KVmkJD4hBdD3th8Tfk9vYasLM+mXWOZhFkgZfxhLSnrwRr4elSSg==", + "dev": true, + "requires": { + "is-extglob": "^2.1.1" + } + }, + "is-number": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/is-number/-/is-number-7.0.0.tgz", + "integrity": "sha512-41Cifkg6e8TylSpdtTpeLVMqvSBEVzTttHvERD741+pnZ8ANv0004MRL43QKPDlK9cGvNp6NZWZUBlbGXYxxng==", + "dev": true + }, + "is-plain-obj": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-plain-obj/-/is-plain-obj-2.1.0.tgz", + "integrity": "sha512-YWnfyRwxL/+SsrWYfOpUtz5b3YD+nyfkHvjbcanzk8zgyO4ASD67uVMRt8k5bM4lLMDnXfriRhOpemw+NfT1eA==", + "dev": true + }, + "is-plain-object": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/is-plain-object/-/is-plain-object-5.0.0.tgz", + "integrity": "sha512-VRSzKkbMm5jMDoKLbltAkFQ5Qr7VDiTFGXxYFXXowVj387GeGNOCsOH6Msy00SGZ3Fp84b1Naa1psqgcCIEP5Q==" + }, + "is-unicode-supported": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/is-unicode-supported/-/is-unicode-supported-0.1.0.tgz", + "integrity": "sha512-knxG2q4UC3u8stRGyAVJCOdxFmv5DZiRcdlIaAQXAbSfJya+OhopNotLQrstBhququ4ZpuKbDc/8S6mgXgPFPw==", + "dev": true + }, + "isexe": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/isexe/-/isexe-2.0.0.tgz", + "integrity": "sha1-6PvzdNxVb/iUehDcsFctYz8s+hA=", + "dev": true + }, + "js-yaml": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-4.1.0.tgz", + "integrity": "sha512-wpxZs9NoxZaJESJGIZTyDEaYpl0FKSA+FB9aJiyemKhMwkxQg63h4T1KJgUGHpTqPDNRcmmYLugrRjJlBtWvRA==", + "requires": { + "argparse": "^2.0.1" + } + }, + "locate-path": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-6.0.0.tgz", + "integrity": "sha512-iPZK6eYjbxRu3uB4/WZ3EsEIMJFMqAoopl3R+zuq0UjcAm/MO6KCweDgPfP3elTztoKP3KtnVHxTn2NHBSDVUw==", + "dev": true, + "requires": { + "p-locate": "^5.0.0" + } + }, + "log-symbols": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/log-symbols/-/log-symbols-4.1.0.tgz", + "integrity": "sha512-8XPvpAA8uyhfteu8pIvQxpJZ7SYYdpUivZpGy6sFsBuKRY/7rQGavedeB8aK+Zkyq6upMFVL/9AW6vOYzfRyLg==", + "dev": true, + "requires": { + "chalk": "^4.1.0", + "is-unicode-supported": "^0.1.0" + } + }, + "minimatch": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.0.4.tgz", + "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==", + "dev": true, + "requires": { + "brace-expansion": "^1.1.7" + } + }, + "mocha": { + "version": "9.2.0", + "resolved": "https://registry.npmjs.org/mocha/-/mocha-9.2.0.tgz", + "integrity": "sha512-kNn7E8g2SzVcq0a77dkphPsDSN7P+iYkqE0ZsGCYWRsoiKjOt+NvXfaagik8vuDa6W5Zw3qxe8Jfpt5qKf+6/Q==", + "dev": true, + "requires": { + "@ungap/promise-all-settled": "1.1.2", + "ansi-colors": "4.1.1", + "browser-stdout": "1.3.1", + "chokidar": "3.5.3", + "debug": "4.3.3", + "diff": "5.0.0", + "escape-string-regexp": "4.0.0", + "find-up": "5.0.0", + "glob": "7.2.0", + "growl": "1.10.5", + "he": "1.2.0", + "js-yaml": "4.1.0", + "log-symbols": "4.1.0", + "minimatch": "3.0.4", + "ms": "2.1.3", + "nanoid": "3.2.0", + "serialize-javascript": "6.0.0", + "strip-json-comments": "3.1.1", + "supports-color": "8.1.1", + "which": "2.0.2", + "workerpool": "6.2.0", + "yargs": "16.2.0", + "yargs-parser": "20.2.4", + "yargs-unparser": "2.0.0" + } + }, + "ms": { + "version": "2.1.3", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.3.tgz", + "integrity": "sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA==", + "dev": true + }, + "nanoid": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/nanoid/-/nanoid-3.2.0.tgz", + "integrity": "sha512-fmsZYa9lpn69Ad5eDn7FMcnnSR+8R34W9qJEijxYhTbfOWzr22n1QxCMzXLK+ODyW2973V3Fux959iQoUxzUIA==", + "dev": true + }, + "node-fetch": { + "version": "2.6.7", + "resolved": "https://registry.npmjs.org/node-fetch/-/node-fetch-2.6.7.tgz", + "integrity": "sha512-ZjMPFEfVx5j+y2yF35Kzx5sF7kDzxuDj6ziH4FFbOp87zKDZNx8yExJIb05OGF4Nlt9IHFIMBkRl41VdvcNdbQ==", + "requires": { + "whatwg-url": "^5.0.0" + } + }, + "normalize-path": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/normalize-path/-/normalize-path-3.0.0.tgz", + "integrity": "sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA==", + "dev": true + }, + "once": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/once/-/once-1.4.0.tgz", + "integrity": "sha1-WDsap3WWHUsROsF9nFC6753Xa9E=", + "requires": { + "wrappy": "1" + } + }, + "p-limit": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-3.1.0.tgz", + "integrity": "sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ==", + "dev": true, + "requires": { + "yocto-queue": "^0.1.0" + } + }, + "p-locate": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-5.0.0.tgz", + "integrity": "sha512-LaNjtRWUBY++zB5nE/NwcaoMylSPk+S+ZHNB1TzdbMJMny6dynpAGt7X/tl/QYq3TIeE6nxHppbo2LGymrG5Pw==", + "dev": true, + "requires": { + "p-limit": "^3.0.2" + } + }, + "path-exists": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/path-exists/-/path-exists-4.0.0.tgz", + "integrity": "sha512-ak9Qy5Q7jYb2Wwcey5Fpvg2KoAc/ZIhLSLOSBmRmygPsGwkVVt0fZa0qrtMz+m6tJTAHfZQ8FnmB4MG4LWy7/w==", + "dev": true + }, + "path-is-absolute": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/path-is-absolute/-/path-is-absolute-1.0.1.tgz", + "integrity": "sha1-F0uSaHNVNP+8es5r9TpanhtcX18=", + "dev": true + }, + "picomatch": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/picomatch/-/picomatch-2.3.1.tgz", + "integrity": "sha512-JU3teHTNjmE2VCGFzuY8EXzCDVwEqB2a8fsIvwaStHhAWJEeVd1o1QD80CU6+ZdEXXSLbSsuLwJjkCBWqRQUVA==", + "dev": true + }, + "prettier": { + "version": "2.5.1", + "resolved": "https://registry.npmjs.org/prettier/-/prettier-2.5.1.tgz", + "integrity": "sha512-vBZcPRUR5MZJwoyi3ZoyQlc1rXeEck8KgeC9AwwOn+exuxLxq5toTRDTSaVrXHxelDMHy9zlicw8u66yxoSUFg==" + }, + "randombytes": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/randombytes/-/randombytes-2.1.0.tgz", + "integrity": "sha512-vYl3iOX+4CKUWuxGi9Ukhie6fsqXqS9FE2Zaic4tNFD2N2QQaXOMFbuKK4QmDHC0JO6B1Zp41J0LpT0oR68amQ==", + "dev": true, + "requires": { + "safe-buffer": "^5.1.0" + } + }, + "readdirp": { + "version": "3.6.0", + "resolved": "https://registry.npmjs.org/readdirp/-/readdirp-3.6.0.tgz", + "integrity": "sha512-hOS089on8RduqdbhvQ5Z37A0ESjsqz6qnRcffsMU3495FuTdqSm+7bhJ29JvIOsBDEEnan5DPu9t3To9VRlMzA==", + "dev": true, + "requires": { + "picomatch": "^2.2.1" + } + }, + "require-directory": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/require-directory/-/require-directory-2.1.1.tgz", + "integrity": "sha1-jGStX9MNqxyXbiNE/+f3kqam30I=", + "dev": true + }, + "safe-buffer": { + "version": "5.2.1", + "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.2.1.tgz", + "integrity": "sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ==", + "dev": true + }, + "serialize-javascript": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.0.tgz", + "integrity": "sha512-Qr3TosvguFt8ePWqsvRfrKyQXIiW+nGbYpy8XK24NQHE83caxWt+mIymTT19DGFbNWNLfEwsrkSmN64lVWB9ag==", + "dev": true, + "requires": { + "randombytes": "^2.1.0" + } + }, + "string-width": { + "version": "4.2.3", + "resolved": "https://registry.npmjs.org/string-width/-/string-width-4.2.3.tgz", + "integrity": "sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g==", + "dev": true, + "requires": { + "emoji-regex": "^8.0.0", + "is-fullwidth-code-point": "^3.0.0", + "strip-ansi": "^6.0.1" + } + }, + "strip-ansi": { + "version": "6.0.1", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-6.0.1.tgz", + "integrity": "sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A==", + "dev": true, + "requires": { + "ansi-regex": "^5.0.1" + } + }, + "strip-json-comments": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/strip-json-comments/-/strip-json-comments-3.1.1.tgz", + "integrity": "sha512-6fPc+R4ihwqP6N/aIv2f1gMH8lOVtWQHoqC4yK6oSDVVocumAsfCqjkXnqiYMhmMwS/mEHLp7Vehlt3ql6lEig==", + "dev": true + }, + "supports-color": { + "version": "8.1.1", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-8.1.1.tgz", + "integrity": "sha512-MpUEN2OodtUzxvKQl72cUF7RQ5EiHsGvSsVG0ia9c5RbWGL2CI4C7EpPS8UTBIplnlzZiNuV56w+FuNxy3ty2Q==", + "dev": true, + "requires": { + "has-flag": "^4.0.0" + } + }, + "to-regex-range": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/to-regex-range/-/to-regex-range-5.0.1.tgz", + "integrity": "sha512-65P7iz6X5yEr1cwcgvQxbbIw7Uk3gOy5dIdtZ4rDveLqhrdJP+Li/Hx6tyK0NEb+2GCyneCMJiGqrADCSNk8sQ==", + "dev": true, + "requires": { + "is-number": "^7.0.0" + } + }, + "tr46": { + "version": "0.0.3", + "resolved": "https://registry.npmjs.org/tr46/-/tr46-0.0.3.tgz", + "integrity": "sha1-gYT9NH2snNwYWZLzpmIuFLnZq2o=" + }, + "tunnel": { + "version": "0.0.6", + "resolved": "https://registry.npmjs.org/tunnel/-/tunnel-0.0.6.tgz", + "integrity": "sha512-1h/Lnq9yajKY2PEbBadPXj3VxsDDu844OnaAo52UVmIzIvwwtBPIuNvkjuzBlTWpfJyUbG3ez0KSBibQkj4ojg==" + }, + "typescript": { + "version": "4.2.4", + "resolved": "https://registry.npmjs.org/typescript/-/typescript-4.2.4.tgz", + "integrity": "sha512-V+evlYHZnQkaz8TRBuxTA92yZBPotr5H+WhQ7bD3hZUndx5tGOa1fuCgeSjxAzM1RiN5IzvadIXTVefuuwZCRg==", + "dev": true + }, + "universal-user-agent": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/universal-user-agent/-/universal-user-agent-6.0.0.tgz", + "integrity": "sha512-isyNax3wXoKaulPDZWHQqbmIx1k2tb9fb3GGDBRxCscfYV2Ch7WxPArBsFEG8s/safwXTT7H4QGhaIkTp9447w==" + }, + "webidl-conversions": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/webidl-conversions/-/webidl-conversions-3.0.1.tgz", + "integrity": "sha1-JFNCdeKnvGvnvIZhHMFq4KVlSHE=" + }, + "whatwg-url": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/whatwg-url/-/whatwg-url-5.0.0.tgz", + "integrity": "sha1-lmRU6HZUYuN2RNNib2dCzotwll0=", + "requires": { + "tr46": "~0.0.3", + "webidl-conversions": "^3.0.0" + } + }, + "which": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/which/-/which-2.0.2.tgz", + "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==", + "dev": true, + "requires": { + "isexe": "^2.0.0" + } + }, + "workerpool": { + "version": "6.2.0", + "resolved": "https://registry.npmjs.org/workerpool/-/workerpool-6.2.0.tgz", + "integrity": "sha512-Rsk5qQHJ9eowMH28Jwhe8HEbmdYDX4lwoMWshiCXugjtHqMD9ZbiqSDLxcsfdqsETPzVUtX5s1Z5kStiIM6l4A==", + "dev": true + }, + "wrap-ansi": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-7.0.0.tgz", + "integrity": "sha512-YVGIj2kamLSTxw6NsZjoBxfSwsn0ycdesmc4p+Q21c5zPuZ1pl+NfxVdxPtdHvmNVOQ6XSYG4AUtyt/Fi7D16Q==", + "dev": true, + "requires": { + "ansi-styles": "^4.0.0", + "string-width": "^4.1.0", + "strip-ansi": "^6.0.0" + } + }, + "wrappy": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/wrappy/-/wrappy-1.0.2.tgz", + "integrity": "sha1-tSQ9jz7BqjXxNkYFvA0QNuMKtp8=" + }, + "y18n": { + "version": "5.0.8", + "resolved": "https://registry.npmjs.org/y18n/-/y18n-5.0.8.tgz", + "integrity": "sha512-0pfFzegeDWJHJIAmTLRP2DwHjdF5s7jo9tuztdQxAhINCdvS+3nGINqPd00AphqJR/0LhANUS6/+7SCb98YOfA==", + "dev": true + }, + "yargs": { + "version": "16.2.0", + "resolved": "https://registry.npmjs.org/yargs/-/yargs-16.2.0.tgz", + "integrity": "sha512-D1mvvtDG0L5ft/jGWkLpG1+m0eQxOfaBvTNELraWj22wSVUMWxZUvYgJYcKh6jGGIkJFhH4IZPQhR4TKpc8mBw==", + "dev": true, + "requires": { + "cliui": "^7.0.2", + "escalade": "^3.1.1", + "get-caller-file": "^2.0.5", + "require-directory": "^2.1.1", + "string-width": "^4.2.0", + "y18n": "^5.0.5", + "yargs-parser": "^20.2.2" + } + }, + "yargs-parser": { + "version": "20.2.4", + "resolved": "https://registry.npmjs.org/yargs-parser/-/yargs-parser-20.2.4.tgz", + "integrity": "sha512-WOkpgNhPTlE73h4VFAFsOnomJVaovO8VqLDzy5saChRBFQFBoMYirowyW+Q9HB4HFF4Z7VZTiG3iSzJJA29yRA==", + "dev": true + }, + "yargs-unparser": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/yargs-unparser/-/yargs-unparser-2.0.0.tgz", + "integrity": "sha512-7pRTIA9Qc1caZ0bZ6RYRGbHJthJWuakf+WmHK0rVeLkNrrGhfoabBNdue6kdINI6r4if7ocq9aD/n7xwKOdzOA==", + "dev": true, + "requires": { + "camelcase": "^6.0.0", + "decamelize": "^4.0.0", + "flat": "^5.0.2", + "is-plain-obj": "^2.1.0" + } + }, + "yocto-queue": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/yocto-queue/-/yocto-queue-0.1.0.tgz", + "integrity": "sha512-rVksvsnNCdJ/ohGc6xgPwyN8eheCxsiLM8mxuE/t/mOVqJewPuO1miLpTHQiRgTKCLexL4MeAFVagts7HmNZ2Q==", + "dev": true + } + } +} diff --git a/scripts/ci/pr-bot/package.json b/scripts/ci/pr-bot/package.json new file mode 100644 index 0000000000000..5f6ae37a9f024 --- /dev/null +++ b/scripts/ci/pr-bot/package.json @@ -0,0 +1,27 @@ +{ + "name": "pr-bot", + "version": "1.0.0", + "description": "Scripts for the Beam PR bot", + "main": "processNewPrs.js", + "scripts": { + "build": "tsc", + "format": "prettier --write *.ts shared/**/*.ts test/**/*.ts", + "test": "mocha lib/test", + "processNewPrs": "npm run build && node lib/processNewPrs.js", + "processPrUpdate": "npm run build && node lib/processPrUpdate.js", + "gatherMetrics": "npm run build && node lib/gatherMetrics.js" + }, + "dependencies": { + "@actions/exec": "^1.1.0", + "@actions/github": "^5.0.0", + "@octokit/rest": "^18.12.0", + "js-yaml": "^4.1.0", + "prettier": "^2.5.1" + }, + "devDependencies": { + "@types/mocha": "^9.1.0", + "@types/node": "^16.11.7", + "mocha": "^9.1.3", + "typescript": "4.2.4" + } +} diff --git a/scripts/ci/pr-bot/shared/checks.ts b/scripts/ci/pr-bot/shared/checks.ts new file mode 100644 index 0000000000000..51bcf18025928 --- /dev/null +++ b/scripts/ci/pr-bot/shared/checks.ts @@ -0,0 +1,109 @@ +/* + * 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. + */ + +const { getGitHubClient } = require("./githubUtils"); + +export interface CheckStatus { + completed: boolean; + succeeded: boolean; +} + +// Returns the status of the most recent checks runs - +export async function getChecksStatus( + owner: string, + repo: string, + checkSha: string +): Promise { + let checkStatus: CheckStatus = { + completed: true, + succeeded: true, + }; + const mostRecentChecks = await getMostRecentChecks(owner, repo, checkSha); + for (let i = 0; i < mostRecentChecks.length; i++) { + if (mostRecentChecks[i].status != "completed") { + checkStatus.completed = false; + } + if (mostRecentChecks[i].conclusion != "success") { + checkStatus.succeeded = false; + } + } + + return checkStatus; +} + +async function getMostRecentChecks( + owner: string, + repo: string, + checkSha: string +): Promise { + let mostRecentChecks: any[] = []; + const checksByName = await getChecksByName(owner, repo, checkSha); + + const checkNames = Object.keys(checksByName); + for (let i = 0; i < checkNames.length; i++) { + let checks = checksByName[checkNames[i]]; + let mostRecent = checks.sort((a, b) => + a.completionTime > b.completionTime ? 1 : -1 + )[0]; + mostRecentChecks.push(mostRecent); + } + + return mostRecentChecks; +} + +async function getChecksByName( + owner: string, + repo: string, + checkSha: string +): Promise { + const githubClient = getGitHubClient(); + const allChecks = ( + await githubClient.rest.checks.listForRef({ + owner: owner, + repo: repo, + ref: checkSha, + }) + ).data.check_runs; + let checksByName = {}; + allChecks.forEach((checkRun) => { + if (!shouldExcludeCheck(checkRun)) { + let name = checkRun.name; + let check = { + status: checkRun.status, + conclusion: checkRun.conclusion, + completionTime: checkRun.completed_at, + }; + if (!checksByName[name]) { + checksByName[name] = [check]; + } else { + checksByName[name].push(check); + } + } + }); + + return checksByName; +} + +// Returns checks we should exclude because they are flaky or not always predictive of pr mergability. +// Currently just excludes codecov. +function shouldExcludeCheck(check): boolean { + if (check.name.toLowerCase().indexOf("codecov") != -1) { + return true; + } + return false; +} diff --git a/scripts/ci/pr-bot/shared/commentStrings.ts b/scripts/ci/pr-bot/shared/commentStrings.ts new file mode 100644 index 0000000000000..e70a721c95fa7 --- /dev/null +++ b/scripts/ci/pr-bot/shared/commentStrings.ts @@ -0,0 +1,72 @@ +/* + * 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. + */ + +export function allChecksPassed(reviewersToNotify: string[]): string { + return `All checks have passed: @${reviewersToNotify.join(" ")}`; +} + +export function assignCommitter(committer: string): string { + return `R: @${committer} for final approval`; +} + +export function assignReviewer(labelToReviewerMapping: any): string { + let commentString = + "Assigning reviewers. If you would like to opt out of this review, comment `assign to next reviewer`:\n\n"; + + for (let label in labelToReviewerMapping) { + let reviewer = labelToReviewerMapping[label]; + if (label === "no-matching-label") { + commentString += `R: @${reviewer} added as fallback since no labels match configuration\n`; + } else { + commentString += `R: @${reviewer} for label ${label}.\n`; + } + } + + commentString += ` +Available commands: +- \`stop reviewer notifications\` - opt out of the automated review tooling +- \`remind me after tests pass\` - tag the comment author after tests pass +- \`waiting on author\` - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)`; + return commentString; +} + +export function failingChecksCantAssign(): string { + return "Checks are failing. Will not request review until checks are succeeding. If you'd like to override that behavior, comment `assign set of reviewers`"; +} + +export function someChecksFailing(reviewersToNotify: string[]): string { + return `Some checks have failed: @${reviewersToNotify.join(" ")}`; +} + +export function stopNotifications(reason: string): string { + return `Stopping reviewer notifications for this pull request: ${reason}`; +} + +export function remindReviewerAfterTestsPass(requester: string): string { + return `Ok - I'll remind @${requester} after tests pass`; +} + +export function reviewersAlreadyAssigned(reviewers: string[]): string { + return `Reviewers are already assigned to this PR: ${reviewers + .map((reviewer) => "@" + reviewer) + .join(" ")}`; +} + +export function noLegalReviewers(): string { + return "No reviewers could be found from any of the labels on the PR or in the fallback reviewers list. Check the config file to make sure reviewers are configured"; +} diff --git a/scripts/ci/pr-bot/shared/constants.ts b/scripts/ci/pr-bot/shared/constants.ts new file mode 100644 index 0000000000000..859bf8589e623 --- /dev/null +++ b/scripts/ci/pr-bot/shared/constants.ts @@ -0,0 +1,30 @@ +/* + * 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. + */ + +const path = require("path"); + +export const REPO_OWNER = "apache"; +export const REPO = "beam"; +export const PATH_TO_CONFIG_FILE = path.join( + __dirname, + "../../../../../.github/REVIEWERS.yml" +); +export const PATH_TO_METRICS_CSV = path.resolve( + path.join(__dirname, "../../metrics.csv") +); +export const BOT_NAME = "github-actions"; diff --git a/scripts/ci/pr-bot/shared/githubUtils.ts b/scripts/ci/pr-bot/shared/githubUtils.ts new file mode 100644 index 0000000000000..e650ca52b0d09 --- /dev/null +++ b/scripts/ci/pr-bot/shared/githubUtils.ts @@ -0,0 +1,96 @@ +/* + * 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. + */ + +const { Octokit } = require("@octokit/rest"); +const { REPO_OWNER, REPO } = require("./constants"); + +export interface Label { + name: string; +} + +export function getGitHubClient() { + let auth = process.env["GITHUB_TOKEN"]; + if (!auth) { + throw new Error( + "No github token provided - process.env['GITHUB_TOKEN'] must be set." + ); + } + return new Octokit({ auth }); +} + +export async function addPrComment(pullNumber: number, body: string) { + await getGitHubClient().rest.issues.createComment({ + owner: REPO_OWNER, + repo: REPO, + issue_number: pullNumber, + body, + }); +} + +export async function nextActionReviewers( + pullNumber: number, + existingLabels: Label[] +) { + let newLabels = removeNextActionLabel(existingLabels); + newLabels.push("Next Action: Reviewers"); + await getGitHubClient().rest.issues.setLabels({ + owner: REPO_OWNER, + repo: REPO, + issue_number: pullNumber, + labels: newLabels, + }); +} + +export async function nextActionAuthor( + pullNumber: number, + existingLabels: Label[] +) { + let newLabels = removeNextActionLabel(existingLabels); + newLabels.push("Next Action: Author"); + await getGitHubClient().rest.issues.setLabels({ + owner: REPO_OWNER, + repo: REPO, + issue_number: pullNumber, + labels: newLabels, + }); +} + +export async function checkIfCommitter(username: string): Promise { + const permissionLevel = ( + await getGitHubClient().rest.repos.getCollaboratorPermissionLevel({ + owner: REPO_OWNER, + repo: REPO, + username, + }) + ).data; + + return ( + permissionLevel.permission === "write" || + permissionLevel.permission === "admin" + ); +} + +function removeNextActionLabel(existingLabels: Label[]): string[] { + return existingLabels + .filter( + (label) => + label.name != "Next Action: Reviewers" && + label.name != "Next Action: Author" + ) + .map((label) => label.name); +} diff --git a/scripts/ci/pr-bot/shared/persistentState.ts b/scripts/ci/pr-bot/shared/persistentState.ts new file mode 100644 index 0000000000000..f7fade16f9b93 --- /dev/null +++ b/scripts/ci/pr-bot/shared/persistentState.ts @@ -0,0 +1,121 @@ +/* + * 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. + */ + +const exec = require("@actions/exec"); +const fs = require("fs"); +const path = require("path"); +const { Pr } = require("./pr"); +const { ReviewersForLabel } = require("./reviewersForLabel"); +const { BOT_NAME } = require("./constants"); + +function getPrFileName(prNumber) { + return `pr-${prNumber}.json`.toLowerCase(); +} + +function getReviewersForLabelFileName(label) { + return `reviewers-for-label-${label}.json`.toLowerCase(); +} + +async function commitStateToRepo() { + await exec.exec("git pull origin pr-bot-state"); + await exec.exec("git add state/*"); + await exec.exec(`git commit -m "Updating config from bot" --allow-empty`); + await exec.exec("git push origin pr-bot-state"); +} + +export class PersistentState { + private switchedBranch = false; + + // Returns a Pr object representing the current saved state of the pr. + async getPrState(prNumber: number): Promise { + var fileName = getPrFileName(prNumber); + return new Pr(await this.getState(fileName, "state/pr-state")); + } + + // Writes a Pr object representing the current saved state of the pr to persistent storage. + async writePrState(prNumber: number, newState: any) { + var fileName = getPrFileName(prNumber); + await this.writeState(fileName, "state/pr-state", new Pr(newState)); + } + + // Returns a ReviewersForLabel object representing the current saved state of which reviewers have reviewed recently. + async getReviewersForLabelState( + label: string + ): Promise { + var fileName = getReviewersForLabelFileName(label); + return new ReviewersForLabel(label, await this.getState(fileName, "state")); + } + + // Writes a ReviewersForLabel object representing the current saved state of which reviewers have reviewed recently. + async writeReviewersForLabelState(label: string, newState: any) { + var fileName = getReviewersForLabelFileName(label); + await this.writeState( + fileName, + "state", + new ReviewersForLabel(label, newState) + ); + } + + private async getState(fileName, baseDirectory) { + await this.ensureCorrectBranch(); + fileName = path.join(baseDirectory, fileName); + if (!fs.existsSync(fileName)) { + return null; + } + return JSON.parse(fs.readFileSync(fileName, { encoding: "utf-8" })); + } + + private async writeState(fileName, baseDirectory, state) { + await this.ensureCorrectBranch(); + fileName = path.join(baseDirectory, fileName); + if (!fs.existsSync(baseDirectory)) { + fs.mkdirSync(baseDirectory, { recursive: true }); + } + fs.writeFileSync(fileName, JSON.stringify(state, null, 2), { + encoding: "utf-8", + }); + await commitStateToRepo(); + } + + private async ensureCorrectBranch() { + if (this.switchedBranch) { + return; + } + console.log( + "Switching to branch pr-bot-state for reading/storing persistent state between runs" + ); + try { + await exec.exec(`git config user.name ${BOT_NAME}`); + await exec.exec(`git config user.email ${BOT_NAME}@github.com`); + await exec.exec("git config pull.rebase false"); + await exec.exec("git fetch origin pr-bot-state"); + await exec.exec("git checkout pr-bot-state"); + } catch { + console.log( + "Couldnt find branch pr-bot-state in origin, trying to create it" + ); + try { + await exec.exec("git checkout -b pr-bot-state"); + } catch { + console.log("Creating branch failed, trying a simple checkout."); + await exec.exec("git checkout pr-bot-state"); + } + } + this.switchedBranch = true; + } +} diff --git a/scripts/ci/pr-bot/shared/pr.ts b/scripts/ci/pr-bot/shared/pr.ts new file mode 100644 index 0000000000000..c1419b8419cee --- /dev/null +++ b/scripts/ci/pr-bot/shared/pr.ts @@ -0,0 +1,89 @@ +/* + * 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. + */ + +const github = require("./githubUtils"); + +export class Pr { + public commentedAboutFailingChecks: boolean; + public reviewersAssignedForLabels: { [key: string]: string }; + public nextAction: string; + public stopReviewerNotifications: boolean; + public remindAfterTestsPass: string[]; + + constructor(propertyDictionary) { + this.commentedAboutFailingChecks = false; + this.reviewersAssignedForLabels = {}; // map of label to reviewer + this.nextAction = "Author"; + this.stopReviewerNotifications = false; + this.remindAfterTestsPass = []; // List of handles + + if (!propertyDictionary) { + return; + } + if (propertyDictionary) { + if ("commentedAboutFailingChecks" in propertyDictionary) { + this.commentedAboutFailingChecks = + propertyDictionary["commentedAboutFailingChecks"]; + } + if ("reviewersAssignedForLabels" in propertyDictionary) { + this.reviewersAssignedForLabels = + propertyDictionary["reviewersAssignedForLabels"]; + } + if ("nextAction" in propertyDictionary) { + this.nextAction = propertyDictionary["nextAction"]; + } + if ("stopReviewerNotifications" in propertyDictionary) { + this.stopReviewerNotifications = + propertyDictionary["stopReviewerNotifications"]; + } + if ("remindAfterTestsPass" in propertyDictionary) { + this.remindAfterTestsPass = propertyDictionary["remindAfterTestsPass"]; + } + } + } + + // Returns a label that the reviewer is assigned for. + // If none, returns an empty string + getLabelForReviewer(reviewer: string): string { + const labels = Object.keys(this.reviewersAssignedForLabels); + for (let i = 0; i < labels.length; i++) { + let label = labels[i]; + if (this.reviewersAssignedForLabels[label] === reviewer) { + return label; + } + } + + return ""; + } + + // Returns whether any of the assigned reviewers are committers + async isAnyAssignedReviewerCommitter(): Promise { + const labels = Object.keys(this.reviewersAssignedForLabels); + for (let i = 0; i < labels.length; i++) { + if ( + await github.checkIfCommitter( + this.reviewersAssignedForLabels[labels[i]] + ) + ) { + return true; + } + } + + return false; + } +} diff --git a/scripts/ci/pr-bot/shared/reviewerConfig.ts b/scripts/ci/pr-bot/shared/reviewerConfig.ts new file mode 100644 index 0000000000000..f8300adbdfaf3 --- /dev/null +++ b/scripts/ci/pr-bot/shared/reviewerConfig.ts @@ -0,0 +1,97 @@ +/* + * 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. + */ + +const yaml = require("js-yaml"); +const fs = require("fs"); +import { Label } from "./githubUtils"; + +export class ReviewerConfig { + private config: any; + constructor(pathToConfigFile) { + this.config = yaml.load( + fs.readFileSync(pathToConfigFile, { encoding: "utf-8" }) + ); + } + + // Given a list of labels and an exclusion list of reviewers not to include (e.g. the author) + // returns all possible reviewers for each label + getReviewersForLabels( + labels: Label[], + exclusionList: string[] + ): { [key: string]: string[] } { + let reviewersFound = false; + let labelToReviewerMapping = {}; + labels.forEach((label) => { + let reviewers = this.getReviewersForLabel(label.name, exclusionList); + if (reviewers.length > 0) { + labelToReviewerMapping[label.name] = reviewers; + reviewersFound = true; + } + }); + if (!reviewersFound) { + const fallbackReviewers = this.getFallbackReviewers(exclusionList); + if (fallbackReviewers.length > 0) { + labelToReviewerMapping["no-matching-label"] = + this.getFallbackReviewers(exclusionList); + } + } + return labelToReviewerMapping; + } + + // Get possible reviewers excluding the author. + getReviewersForLabel(label: string, exclusionList: string[]): string[] { + var labelObjects = this.config.labels; + const labelObject = labelObjects.find( + (labelObject) => labelObject.name.toLowerCase() === label.toLowerCase() + ); + if (!labelObject) { + return []; + } + + return this.excludeFromReviewers(labelObject.reviewers, exclusionList); + } + + getExclusionListForLabel(label: string): string[] { + var labelObjects = this.config.labels; + const labelObject = labelObjects.find( + (labelObject) => labelObject.name.toLowerCase() === label.toLowerCase() + ); + return labelObject?.exclusionList ?? []; + } + + // Get fallback reviewers excluding the author. + getFallbackReviewers(exclusionList: string[]): string[] { + return this.excludeFromReviewers( + this.config.fallbackReviewers, + exclusionList + ); + } + + private excludeFromReviewers( + reviewers: string[], + exclusionList: string[] + ): string[] { + if (!exclusionList) { + return reviewers; + } + + return reviewers.filter( + (reviewer) => exclusionList.indexOf(reviewer) == -1 + ); + } +} diff --git a/scripts/ci/pr-bot/shared/reviewersForLabel.ts b/scripts/ci/pr-bot/shared/reviewersForLabel.ts new file mode 100644 index 0000000000000..971f3f1cd7a53 --- /dev/null +++ b/scripts/ci/pr-bot/shared/reviewersForLabel.ts @@ -0,0 +1,102 @@ +/* + * 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. + */ + +const github = require("./githubUtils"); + +export class ReviewersForLabel { + public label: string; + public dateOfLastReviewAssignment: { [key: string]: number }; + + constructor( + label: string, + propertyDictionary: { + dateOfLastReviewAssignment: { [key: string]: number }; + } + ) { + this.label = label; + this.dateOfLastReviewAssignment = {}; // map of reviewer to date + + if (!propertyDictionary) { + return; + } + if ("dateOfLastReviewAssignment" in propertyDictionary) { + this.dateOfLastReviewAssignment = + propertyDictionary["dateOfLastReviewAssignment"]; + } + } + + // Given a list of available reviewers, + // returns the next reviewer up based on who has reviewed least recently. + // Updates this object to reflect their assignment. + assignNextReviewer(availableReviewers: string[]): string { + if (availableReviewers.length === 0) { + throw new Error(`No reviewers available for label ${this.label}`); + } + + if (!this.dateOfLastReviewAssignment[availableReviewers[0]]) { + this.dateOfLastReviewAssignment[availableReviewers[0]] = Date.now(); + return availableReviewers[0]; + } + + let earliestDate = this.dateOfLastReviewAssignment[availableReviewers[0]]; + let earliestReviewer = availableReviewers[0]; + + for (let i = 0; i < availableReviewers.length; i++) { + let availableReviewer = availableReviewers[i]; + if (!this.dateOfLastReviewAssignment[availableReviewer]) { + this.dateOfLastReviewAssignment[availableReviewer] = Date.now(); + return availableReviewer; + } + if (earliestDate > this.dateOfLastReviewAssignment[availableReviewer]) { + earliestDate = this.dateOfLastReviewAssignment[availableReviewer]; + earliestReviewer = availableReviewer; + } + } + + this.dateOfLastReviewAssignment[earliestReviewer] = Date.now(); + return earliestReviewer; + } + + // Given the up to date list of available reviewers (excluding the author), + // returns the next reviewer up based on who has reviewed least recently. + // Updates this object to reflect their assignment. + async assignNextCommitter(availableReviewers: string[]): Promise { + let earliestDate = Date.now(); + let earliestCommitter: string = ""; + + for (let i = 0; i < availableReviewers.length; i++) { + let availableReviewer = availableReviewers[i]; + if (await github.checkIfCommitter(availableReviewer)) { + if (!this.dateOfLastReviewAssignment[availableReviewer]) { + this.dateOfLastReviewAssignment[availableReviewer] = Date.now(); + return availableReviewer; + } + if (earliestDate > this.dateOfLastReviewAssignment[availableReviewer]) { + earliestDate = this.dateOfLastReviewAssignment[availableReviewer]; + earliestCommitter = availableReviewer; + } + } + } + + if (!earliestCommitter) { + throw new Error(`No committers available for label ${this.label}`); + } + this.dateOfLastReviewAssignment[earliestCommitter] = Date.now(); + return earliestCommitter; + } +} diff --git a/scripts/ci/pr-bot/shared/userCommand.ts b/scripts/ci/pr-bot/shared/userCommand.ts new file mode 100644 index 0000000000000..34e2741a3f6d4 --- /dev/null +++ b/scripts/ci/pr-bot/shared/userCommand.ts @@ -0,0 +1,231 @@ +/* + * 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. + */ + +const github = require("./githubUtils"); +const commentStrings = require("./commentStrings"); +const { BOT_NAME } = require("./constants"); +const { StateClient } = require("./persistentState"); +const { ReviewerConfig } = require("./reviewerConfig"); + +// Reads the comment and processes the command if one is contained in it. +// Returns true if it runs a command, false otherwise. +export async function processCommand( + payload: any, + commentAuthor: string, + commentText: string, + stateClient: typeof StateClient, + reviewerConfig: typeof ReviewerConfig +) { + // Don't process any commands from our bot. + if (commentAuthor === BOT_NAME) { + return false; + } + console.log(commentAuthor); + + const pullNumber = payload.issue?.number || payload.pull_request?.number; + commentText = commentText.toLowerCase(); + if (commentText.indexOf("r: @") > -1) { + await manuallyAssignedToReviewer(pullNumber, stateClient); + } else if (commentText.indexOf("assign to next reviewer") > -1) { + await assignToNextReviewer( + payload, + commentAuthor, + pullNumber, + stateClient, + reviewerConfig + ); + } else if (commentText.indexOf("stop reviewer notifications") > -1) { + await stopReviewerNotifications( + pullNumber, + stateClient, + "requested by reviewer" + ); + } else if (commentText.indexOf("remind me after tests pass") > -1) { + await remindAfterTestsPass(pullNumber, commentAuthor, stateClient); + } else if (commentText.indexOf("waiting on author") > -1) { + await waitOnAuthor(payload, pullNumber, stateClient); + } else if (commentText.indexOf("assign set of reviewers") > -1) { + await assignReviewerSet(payload, pullNumber, stateClient, reviewerConfig); + } else { + return false; + } + + return true; +} + +async function assignToNextReviewer( + payload: any, + commentAuthor: string, + pullNumber: number, + stateClient: typeof StateClient, + reviewerConfig: typeof ReviewerConfig +) { + let prState = await stateClient.getPrState(pullNumber); + let labelOfReviewer = prState.getLabelForReviewer(payload.sender.login); + if (labelOfReviewer) { + let reviewersState = await stateClient.getReviewersForLabelState( + labelOfReviewer + ); + const pullAuthor = + payload.issue?.user?.login || payload.pull_request?.user?.login; + let availableReviewers = reviewerConfig.getReviewersForLabel( + labelOfReviewer, + [commentAuthor, pullAuthor] + ); + let chosenReviewer = reviewersState.assignNextReviewer(availableReviewers); + prState.reviewersAssignedForLabels[labelOfReviewer] = chosenReviewer; + + // Comment assigning reviewer + console.log(`Assigning ${chosenReviewer}`); + await github.addPrComment( + pullNumber, + commentStrings.assignReviewer(prState.reviewersAssignedForLabels) + ); + + // Set next action to reviewer + const existingLabels = + payload.issue?.labels || payload.pull_request?.labels; + await github.nextActionReviewers(pullNumber, existingLabels); + prState.nextAction = "Reviewers"; + + // Persist state + await stateClient.writePrState(pullNumber, prState); + await stateClient.writeReviewersForLabelState( + labelOfReviewer, + reviewersState + ); + } +} + +// If they've manually assigned a reviewer, just silence notifications and ignore this pr going forward. +// TODO(damccorm) - we could try to do something more intelligent here like figuring out which label that reviewer belongs to. +async function manuallyAssignedToReviewer( + pullNumber: number, + stateClient: typeof StateClient +) { + await stopReviewerNotifications( + pullNumber, + stateClient, + "review requested by someone other than the bot, ceding control" + ); +} + +async function stopReviewerNotifications( + pullNumber: number, + stateClient: typeof StateClient, + reason: string +) { + let prState = await stateClient.getPrState(pullNumber); + prState.stopReviewerNotifications = true; + await stateClient.writePrState(pullNumber, prState); + + // Comment acknowledging command + await github.addPrComment( + pullNumber, + commentStrings.stopNotifications(reason) + ); +} + +async function remindAfterTestsPass( + pullNumber: number, + username: string, + stateClient: typeof StateClient +) { + let prState = await stateClient.getPrState(pullNumber); + prState.remindAfterTestsPass.push(username); + await stateClient.writePrState(pullNumber, prState); + + // Comment acknowledging command + await github.addPrComment( + pullNumber, + commentStrings.remindReviewerAfterTestsPass(username) + ); +} + +async function waitOnAuthor( + payload: any, + pullNumber: number, + stateClient: typeof StateClient +) { + const existingLabels = payload.issue?.labels || payload.pull_request?.labels; + await github.nextActionAuthor(pullNumber, existingLabels); + let prState = await stateClient.getPrState(pullNumber); + prState.nextAction = "Author"; + await stateClient.writePrState(pullNumber, prState); +} + +async function assignReviewerSet( + payload: any, + pullNumber: number, + stateClient: typeof StateClient, + reviewerConfig: typeof ReviewerConfig +) { + let prState = await stateClient.getPrState(pullNumber); + if (Object.values(prState.reviewersAssignedForLabels).length > 0) { + await github.addPrComment( + pullNumber, + commentStrings.reviewersAlreadyAssigned( + Object.values(prState.reviewersAssignedForLabels) + ) + ); + return; + } + + const existingLabels = payload.issue?.labels || payload.pull_request?.labels; + const pullAuthor = + payload.issue?.user?.login || payload.pull_request?.user?.login; + const reviewersForLabels = reviewerConfig.getReviewersForLabels( + existingLabels, + [pullAuthor] + ); + let reviewerStateToUpdate = {}; + var labels = Object.keys(reviewersForLabels); + if (!labels || labels.length == 0) { + await github.addPrComment( + pullNumber, + commentStrings.noLegalReviewers(existingLabels) + ); + return; + } + for (let i = 0; i < labels.length; i++) { + let label = labels[i]; + let availableReviewers = reviewersForLabels[label]; + let reviewersState = await stateClient.getReviewersForLabelState(label); + let chosenReviewer = reviewersState.assignNextReviewer(availableReviewers); + reviewerStateToUpdate[label] = reviewersState; + prState.reviewersAssignedForLabels[label] = chosenReviewer; + } + console.log(`Assigning reviewers for pr ${pullNumber}`); + await github.addPrComment( + pullNumber, + commentStrings.assignReviewer(prState.reviewersAssignedForLabels) + ); + + github.nextActionReviewers(pullNumber, existingLabels); + prState.nextAction = "Reviewers"; + + await stateClient.writePrState(pullNumber, prState); + let labelsToUpdate = Object.keys(reviewerStateToUpdate); + for (let i = 0; i < labelsToUpdate.length; i++) { + let label = labelsToUpdate[i]; + await stateClient.writeReviewersForLabelState( + label, + reviewerStateToUpdate[label] + ); + } +} diff --git a/scripts/ci/pr-bot/test/prTest.ts b/scripts/ci/pr-bot/test/prTest.ts new file mode 100644 index 0000000000000..b771852584a30 --- /dev/null +++ b/scripts/ci/pr-bot/test/prTest.ts @@ -0,0 +1,45 @@ +/* + * 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. + */ + +var assert = require("assert"); +const { Pr } = require("../shared/pr"); +describe("Pr", function () { + describe("getLabelForReviewer()", function () { + it("should return the label a reviewer is assigned to", function () { + let testPr = new Pr({}); + testPr.reviewersAssignedForLabels = { + Go: "testReviewer1", + Java: "testReviewer2", + Python: "testReviewer3", + }; + assert.equal("Go", testPr.getLabelForReviewer("testReviewer1")); + assert.equal("Java", testPr.getLabelForReviewer("testReviewer2")); + assert.equal("Python", testPr.getLabelForReviewer("testReviewer3")); + }); + + it("should return an empty string when a reviewer is not assigned", function () { + let testPr = new Pr({}); + testPr.reviewersAssignedForLabels = { + Go: "testReviewer1", + Java: "testReviewer2", + Python: "testReviewer3", + }; + assert.equal("", testPr.getLabelForReviewer("testReviewer4")); + }); + }); +}); diff --git a/scripts/ci/pr-bot/test/reviewerConfigTest.ts b/scripts/ci/pr-bot/test/reviewerConfigTest.ts new file mode 100644 index 0000000000000..589aab635bc95 --- /dev/null +++ b/scripts/ci/pr-bot/test/reviewerConfigTest.ts @@ -0,0 +1,238 @@ +/* + * 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. + */ + +var assert = require("assert"); +var fs = require("fs"); +const { ReviewerConfig } = require("../shared/reviewerConfig"); +const configPath = "test-config.yml"; +const configContents = `labels: +- name: "Go" + reviewers: ["testReviewer1", "testReviewer2"] + exclusionList: ["testReviewer3"] # These users will never be suggested as reviewers +# I don't know the other areas well enough to assess who the normal committers/contributors who might want to be reviewers are +- name: "Java" + reviewers: ["testReviewer3", "testReviewer2"] + exclusionList: [] # These users will never be suggested as reviewers +- name: "Python" + reviewers: ["testReviewer4"] + exclusionList: [] # These users will never be suggested as reviewers +fallbackReviewers: ["testReviewer5", "testReviewer1", "testReviewer3"] # List of committers to use when no label matches +`; +describe("ReviewerConfig", function () { + before(function () { + if (fs.existsSync(configPath)) { + fs.rmSync(configPath); + } + fs.writeFileSync(configPath, configContents); + }); + + after(function () { + fs.rmSync(configPath); + }); + + describe("getReviewersForLabels()", function () { + it("should return all reviewers configured for all labels", function () { + const config = new ReviewerConfig(configPath); + const reviewersForLabels = config.getReviewersForLabels( + [{ name: "Go" }, { name: "Java" }], + [] + ); + assert( + reviewersForLabels["Go"].find( + (reviewer) => reviewer === "testReviewer1" + ), + "Return value for Go label should include testReviewer1" + ); + assert( + reviewersForLabels["Go"].find( + (reviewer) => reviewer === "testReviewer2" + ), + "Return value for Go label should include testReviewer2" + ); + assert( + !reviewersForLabels["Go"].find( + (reviewer) => reviewer === "testReviewer3" + ), + "Return value for Go label should not include testReviewer3" + ); + assert( + !reviewersForLabels["Go"].find( + (reviewer) => reviewer === "testReviewer4" + ), + "Return value for Go label should not include testReviewer4" + ); + + assert( + reviewersForLabels["Java"].find( + (reviewer) => reviewer === "testReviewer3" + ), + "Return value for Java label should include testReviewer3" + ); + assert( + reviewersForLabels["Java"].find( + (reviewer) => reviewer === "testReviewer2" + ), + "Return value for Java label should include testReviewer2" + ); + assert( + !reviewersForLabels["Java"].find( + (reviewer) => reviewer === "testReviewer4" + ), + "Return value for Java label should not include testReviewer4" + ); + assert( + !reviewersForLabels["Java"].find( + (reviewer) => reviewer === "testReviewer1" + ), + "Return value for Java label should not include testReviewer1" + ); + assert( + !reviewersForLabels["Java"].find( + (reviewer) => reviewer === "testReviewer5" + ), + "Return value for Java label should not include testReviewer5" + ); + + assert( + Object.keys(reviewersForLabels).indexOf("Python") == -1, + "No reviewers should be included for python" + ); + }); + + it("should return no entry if a label is not configured", function () { + const config = new ReviewerConfig(configPath); + const reviewersForLabels = config.getReviewersForLabels( + [{ name: "FakeLabel" }], + [] + ); + + assert( + !("FakeLabel" in reviewersForLabels), + "FakeLabel should not be included in the returned label map" + ); + }); + + it("should exclude any reviewers who are passed into the exlusionList", function () { + const config = new ReviewerConfig(configPath); + const reviewersForLabels = config.getReviewersForLabels( + [{ name: "Go" }, { name: "Java" }], + ["testReviewer1"] + ); + + assert( + !reviewersForLabels["Go"].find( + (reviewer) => reviewer === "testReviewer1" + ), + "testReviewer1 should have been excluded from the result set" + ); + }); + }); + + describe("getReviewersForLabel()", function () { + it("should return all reviewers configured for a label", function () { + const config = new ReviewerConfig(configPath); + const reviewersForGo = config.getReviewersForLabel("Go", []); + + assert( + reviewersForGo.find((reviewer) => reviewer === "testReviewer1"), + "Return value for Go label should include testReviewer1" + ); + assert( + reviewersForGo.find((reviewer) => reviewer === "testReviewer2"), + "Return value for Go label should include testReviewer2" + ); + assert( + !reviewersForGo.find((reviewer) => reviewer === "testReviewer3"), + "Return value for Go label should not include testReviewer3" + ); + assert( + !reviewersForGo.find((reviewer) => reviewer === "testReviewer4"), + "Return value for Go label should not include testReviewer4" + ); + }); + + it("should return an empty list if a label is not configured", function () { + const config = new ReviewerConfig(configPath); + const reviewersForFakeLabel = config.getReviewersForLabel( + "FakeLabel", + [] + ); + assert.equal(0, reviewersForFakeLabel.length); + }); + + it("should exclude any reviewers who are passed into the exlusionList", function () { + const config = new ReviewerConfig(configPath); + const reviewersForGo = config.getReviewersForLabel("Go", [ + "testReviewer1", + ]); + + assert( + !reviewersForGo.find((reviewer) => reviewer === "testReviewer1"), + "Return value for Go label should not include testReviewer1" + ); + }); + }); + + describe("getExclusionListForLabel()", function () { + it("should get the exclusion list configured for a label", function () { + const config = new ReviewerConfig(configPath); + const goExclusionList = config.getExclusionListForLabel("Go"); + + assert( + goExclusionList.find((reviewer) => reviewer === "testReviewer3"), + "Return value for Go label should include testReviewer3" + ); + assert.equal(1, goExclusionList.length); + }); + }); + + describe("getFallbackReviewers()", function () { + it("should get the configured falback list", function () { + const config = new ReviewerConfig(configPath); + const fallbackReviewers = config.getFallbackReviewers([]); + + assert.equal(3, fallbackReviewers.length); + assert( + fallbackReviewers.find((reviewer) => reviewer === "testReviewer5"), + "Fallback reviewers should include testReviewer5" + ); + assert( + fallbackReviewers.find((reviewer) => reviewer === "testReviewer1"), + "Fallback reviewers should include testReviewer1" + ); + assert( + fallbackReviewers.find((reviewer) => reviewer === "testReviewer3"), + "Fallback reviewers should include testReviewer3" + ); + }); + + it("should not include excluded reviewers", function () { + const config = new ReviewerConfig(configPath); + const fallbackReviewers = config.getFallbackReviewers([ + "testReviewer1", + "testReviewer3", + ]); + + assert.equal(1, fallbackReviewers.length); + assert( + fallbackReviewers.find((reviewer) => reviewer === "testReviewer5"), + "Fallback reviewers should only include testReviewer5" + ); + }); + }); +}); diff --git a/scripts/ci/pr-bot/test/reviewersForLabelTest.ts b/scripts/ci/pr-bot/test/reviewersForLabelTest.ts new file mode 100644 index 0000000000000..f425750505927 --- /dev/null +++ b/scripts/ci/pr-bot/test/reviewersForLabelTest.ts @@ -0,0 +1,135 @@ +/* + * 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. + */ + +var assert = require("assert"); +const { ReviewersForLabel } = require("../shared/reviewersForLabel"); +describe("ReviewersForLabel", function () { + describe("assignNextReviewer()", function () { + it("should repeatedly assign the reviewer who reviewed least recently", function () { + const dateOfLastReviewAssignment = { + testReviewer1: 1, + testReviewer2: 3, + testReviewer3: 4, + testReviewer4: 2, + }; + let reviewersForGo = new ReviewersForLabel("Go", { + dateOfLastReviewAssignment: dateOfLastReviewAssignment, + }); + + assert.equal( + "testReviewer1", + reviewersForGo.assignNextReviewer([ + "testReviewer1", + "testReviewer2", + "testReviewer3", + "testReviewer4", + ]) + ); + assert.equal( + "testReviewer4", + reviewersForGo.assignNextReviewer([ + "testReviewer1", + "testReviewer2", + "testReviewer3", + "testReviewer4", + ]) + ); + assert.equal( + "testReviewer2", + reviewersForGo.assignNextReviewer([ + "testReviewer1", + "testReviewer2", + "testReviewer3", + "testReviewer4", + ]) + ); + assert.equal( + "testReviewer3", + reviewersForGo.assignNextReviewer([ + "testReviewer1", + "testReviewer2", + "testReviewer3", + "testReviewer4", + ]) + ); + assert.equal( + "testReviewer1", + reviewersForGo.assignNextReviewer([ + "testReviewer1", + "testReviewer2", + "testReviewer3", + "testReviewer4", + ]) + ); + }); + + it("should assign a reviewer who hasnt reviewed before", function () { + const dateOfLastReviewAssignment = { + testReviewer1: 1, + testReviewer2: 2, + testReviewer3: 3, + testReviewer4: 4, + }; + let reviewersForGo = new ReviewersForLabel("Go", { + dateOfLastReviewAssignment: dateOfLastReviewAssignment, + }); + + assert.equal( + "testReviewer5", + reviewersForGo.assignNextReviewer([ + "testReviewer1", + "testReviewer2", + "testReviewer3", + "testReviewer4", + "testReviewer5", + ]) + ); + }); + + it("should only assign reviewers in the availableReviewers list", function () { + const dateOfLastReviewAssignment = { + testReviewer1: 1, + testReviewer2: 2, + testReviewer3: 3, + testReviewer4: 4, + }; + let reviewersForGo = new ReviewersForLabel("Go", { + dateOfLastReviewAssignment: dateOfLastReviewAssignment, + }); + + assert.equal( + "testReviewer2", + reviewersForGo.assignNextReviewer(["testReviewer4", "testReviewer2"]) + ); + }); + + it("should throw if no reviewer available", function () { + const dateOfLastReviewAssignment = { + testReviewer1: 1, + testReviewer2: 2, + testReviewer3: 3, + testReviewer4: 4, + }; + let reviewersForGo = new ReviewersForLabel("Go", { + dateOfLastReviewAssignment: dateOfLastReviewAssignment, + }); + + assert.throws(() => reviewersForGo.assignNextReviewer([])); + }); + }); +}); diff --git a/scripts/ci/pr-bot/tsconfig.json b/scripts/ci/pr-bot/tsconfig.json new file mode 100644 index 0000000000000..719f5dff8b9f5 --- /dev/null +++ b/scripts/ci/pr-bot/tsconfig.json @@ -0,0 +1,12 @@ +{ + "compilerOptions": { + "target": "es6" /* Specify ECMAScript target version: 'ES3' (default), 'ES5', 'ES2015', 'ES2016', 'ES2017', 'ES2018', 'ES2019' or 'ESNEXT'. */, + "module": "commonjs" /* Specify module code generation: 'none', 'commonjs', 'amd', 'system', 'umd', 'es2015', or 'ESNext'. */, + "outDir": "./lib" /* Redirect output structure to the directory. */, + "rootDir": "./" /* Specify the root directory of input files. Use to control the output directory structure with --outDir. */, + "strict": true /* Enable all strict type-checking options. */, + "noImplicitAny": false /* Raise error on expressions and declarations with an implied 'any' type. */, + "esModuleInterop": true /* Enables emit interoperability between CommonJS and ES Modules via creation of namespace objects for all imports. Implies 'allowSyntheticDefaultImports'. */ + }, + "exclude": ["node_modules"] +} diff --git a/sdks/go/pkg/beam/core/graph/mtime/time.go b/sdks/go/pkg/beam/core/graph/mtime/time.go index 25ea9ce567f72..bdbb0c441af80 100644 --- a/sdks/go/pkg/beam/core/graph/mtime/time.go +++ b/sdks/go/pkg/beam/core/graph/mtime/time.go @@ -37,7 +37,7 @@ const ( // EndOfGlobalWindowTime is the timestamp at the end of the global window. It // is a day before the max timestamp. - // TODO Use GLOBAL_WINDOW_MAX_TIMESTAMP_MILLIS from the Runner API constants + // TODO(BEAM-4179) Use GLOBAL_WINDOW_MAX_TIMESTAMP_MILLIS from the Runner API constants EndOfGlobalWindowTime = MaxTimestamp - 24*60*60*1000 // ZeroTimestamp is the default zero value time. It corresponds to the unix epoch. @@ -65,7 +65,8 @@ func FromDuration(d time.Duration) Time { // FromTime returns a milli-second precision timestamp from a time.Time. func FromTime(t time.Time) Time { - return Normalize(Time(n2m(t.UnixNano()))) + // TODO(BEAM-13988): Replace t.UnixNano() with t.UnixMilli() for Go 1.17 or higher. + return Normalize(Time(t.UnixNano() / 1e6)) } // Milliseconds returns the number of milli-seconds since the Unix epoch. @@ -73,14 +74,18 @@ func (t Time) Milliseconds() int64 { return int64(t) } -// Add returns the time plus the duration. +// Add returns the time plus the duration. Input Durations of less than one +// millisecond will not increment the time due to a loss of precision when +// converting to milliseconds. func (t Time) Add(d time.Duration) Time { - return Normalize(Time(int64(t) + n2m(d.Nanoseconds()))) + return Normalize(Time(int64(t) + d.Milliseconds())) } -// Subtract returns the time minus the duration. +// Subtract returns the time minus the duration. Input Durations of less than one +// millisecond will not increment the time due to a loss of precision when +// converting to milliseconds. func (t Time) Subtract(d time.Duration) Time { - return Normalize(Time(int64(t) - n2m(d.Nanoseconds()))) + return Normalize(Time(int64(t) - d.Milliseconds())) } func (t Time) String() string { @@ -116,8 +121,3 @@ func Max(a, b Time) Time { func Normalize(t Time) Time { return Min(Max(t, MinTimestamp), MaxTimestamp) } - -// n2m converts nanoseconds to milliseconds. -func n2m(v int64) int64 { - return v / 1e6 -} diff --git a/sdks/go/pkg/beam/core/graph/mtime/time_test.go b/sdks/go/pkg/beam/core/graph/mtime/time_test.go new file mode 100644 index 0000000000000..7012184c18ce7 --- /dev/null +++ b/sdks/go/pkg/beam/core/graph/mtime/time_test.go @@ -0,0 +1,215 @@ +// 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 mtime + +import ( + "math" + "testing" + "time" +) + +func TestAdd(t *testing.T) { + tests := []struct { + name string + baseTime Time + addition time.Duration + expOut Time + }{ + { + "insignificant addition small", + Time(1000), + 1 * time.Nanosecond, + Time(1000), + }, + { + "insignificant addition large", + Time(1000), + 999999 * time.Nanosecond, + Time(1000), + }, + { + "significant addition small", + Time(1000), + 1 * time.Millisecond, + Time(1001), + }, + { + "significant addition large", + Time(1000), + 10 * time.Second, + Time(11000), + }, + { + "add past max timestamp", + MaxTimestamp, + 1 * time.Minute, + MaxTimestamp, + }, + { + "add across max boundary", + Time(int64(MaxTimestamp) - 10000), + 10 * time.Minute, + MaxTimestamp, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + if got, want := test.baseTime.Add(test.addition), test.expOut; got != want { + t.Errorf("(%v).Add(%v), got time %v, want %v", test.baseTime, test.addition, got, want) + } + }) + } +} + +func TestSubtract(t *testing.T) { + tests := []struct { + name string + baseTime Time + subtraction time.Duration + expOut Time + }{ + { + "insignificant subtraction small", + Time(1000), + 1 * time.Nanosecond, + Time(1000), + }, + { + "insignificant subtraction large", + Time(1000), + 999999 * time.Nanosecond, + Time(1000), + }, + { + "significant subtraction small", + Time(1000), + 1 * time.Millisecond, + Time(999), + }, + { + "significant subtraction large", + Time(1000), + 10 * time.Second, + Time(-9000), + }, + { + "subtract past min timestamp", + MinTimestamp, + 1 * time.Minute, + MinTimestamp, + }, + { + "subtract across min boundary", + Time(int64(MinTimestamp) + 10000), + 10 * time.Minute, + MinTimestamp, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + if got, want := test.baseTime.Subtract(test.subtraction), test.expOut; got != want { + t.Errorf("(%v).Subtract(%v), got time %v, want %v", test.baseTime, test.subtraction, got, want) + } + }) + } +} + +func TestNormalize(t *testing.T) { + tests := []struct { + name string + in Time + expOut Time + }{ + { + "min timestamp", + MinTimestamp, + MinTimestamp, + }, + { + "max timestamp", + MaxTimestamp, + MaxTimestamp, + }, + { + "end of global window", + EndOfGlobalWindowTime, + EndOfGlobalWindowTime, + }, + { + "beyond max timestamp", + Time(math.MaxInt64), + MaxTimestamp, + }, + { + "below min timestamp", + Time(math.MinInt64), + MinTimestamp, + }, + { + "normal value", + Time(int64(20000)), + Time(int64(20000)), + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + if got, want := Normalize(test.in), test.expOut; got != want { + t.Errorf("Normalize(%v), got Time %v, want %v", test.in, got, want) + } + }) + } +} + +func TestFromTime(t *testing.T) { + tests := []struct { + name string + input time.Time + expOut Time + }{ + { + "zero unix", + time.Unix(0, 0).UTC(), + Time(0), + }, + { + "behind unix", + time.Unix(-1, 0).UTC(), + Time(-1000), + }, + { + "ahead of unix", + time.Unix(1, 0).UTC(), + Time(1000), + }, + { + "insignificant time small", + time.Unix(0, 1), + Time(0), + }, + { + "insignificant time large", + time.Unix(0, 999999), + Time(0), + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + if got, want := FromTime(test.input), test.expOut; got != want { + t.Errorf("FromTime(%v), got %v, want %v", test.input, got, want) + } + }) + } +} diff --git a/sdks/go/pkg/beam/core/graph/window/trigger/trigger.go b/sdks/go/pkg/beam/core/graph/window/trigger/trigger.go index c12c03aa5e893..148a672fc695b 100644 --- a/sdks/go/pkg/beam/core/graph/window/trigger/trigger.go +++ b/sdks/go/pkg/beam/core/graph/window/trigger/trigger.go @@ -66,6 +66,9 @@ func (t *AfterCountTrigger) ElementCount() int32 { // AfterCount constructs a trigger that fires after // at least `count` number of elements are processed. func AfterCount(count int32) *AfterCountTrigger { + if count < 1 { + panic(fmt.Errorf("trigger.AfterCount(%v) must be a positive integer", count)) + } return &AfterCountTrigger{elementCount: count} } diff --git a/sdks/go/pkg/beam/core/graph/window/trigger/trigger_test.go b/sdks/go/pkg/beam/core/graph/window/trigger/trigger_test.go new file mode 100644 index 0000000000000..a23de354bfd61 --- /dev/null +++ b/sdks/go/pkg/beam/core/graph/window/trigger/trigger_test.go @@ -0,0 +1,73 @@ +// 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 trigger + +import ( + "testing" + "time" + + "github.com/google/go-cmp/cmp" +) + +func TestAfterCountTrigger(t *testing.T) { + tr := AfterCount(1) + want := int32(1) + if got := tr.ElementCount(); got != want { + t.Errorf("element count not configured correctly. got %v, want %v", got, want) + } +} + +func TestAfterProcessingTimeTrigger(t *testing.T) { + tests := []struct { + tr *AfterProcessingTimeTrigger + tt []TimestampTransform + }{ + { + tr: AfterProcessingTime().PlusDelay(time.Millisecond), + tt: []TimestampTransform{DelayTransform{Delay: 1}}, + }, + { + tr: AfterProcessingTime().PlusDelay(time.Millisecond).AlignedTo(time.Millisecond, time.Time{}), + tt: []TimestampTransform{DelayTransform{Delay: 1}, AlignToTransform{Period: 1, Offset: 0}}, + }, + } + for _, test := range tests { + if diff := cmp.Diff(test.tr.TimestampTransforms(), test.tt); diff != "" { + t.Errorf("timestamp transforms are not equal: %v", diff) + } + } +} + +func TestRepeatTrigger(t *testing.T) { + subTr := AfterCount(2) + tr := Repeat(subTr) + + if got, ok := tr.SubTrigger().(*AfterCountTrigger); ok && got != subTr { + t.Errorf("subtrigger not configured correctly. got %v, want %v", got, subTr) + } +} + +func TestAfterEndOfWindowTrigger(t *testing.T) { + earlyTr := AfterCount(50) + lateTr := Always() + tr := AfterEndOfWindow().EarlyFiring(earlyTr).LateFiring(lateTr) + + if got, ok := tr.Early().(*AfterCountTrigger); ok && got != earlyTr { + t.Errorf("early firing trigger not configured correctly. got %v, want %v", got, earlyTr) + } + if got, ok := tr.Late().(*AlwaysTrigger); ok && got != lateTr { + t.Errorf("late firing trigger not configured correctly. got %v, want %v", got, lateTr) + } +} diff --git a/sdks/go/pkg/beam/core/runtime/exec/coder_test.go b/sdks/go/pkg/beam/core/runtime/exec/coder_test.go index 25812aca4e560..02d1f81da7e05 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/coder_test.go +++ b/sdks/go/pkg/beam/core/runtime/exec/coder_test.go @@ -25,6 +25,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/reflectx" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/coderx" ) @@ -54,6 +55,9 @@ func TestCoders(t *testing.T) { return &coder.Coder{Kind: coder.Custom, Custom: c, T: typex.New(reflectx.String)} }(), val: &FullValue{Elm: "myString"}, + }, { + coder: &coder.Coder{Kind: coder.LP, Components: []*coder.Coder{coder.NewString()}}, + val: &FullValue{Elm: "myString"}, }, { coder: coder.NewKV([]*coder.Coder{coder.NewVarInt(), coder.NewBool()}), val: &FullValue{Elm: int64(72), Elm2: false}, @@ -64,6 +68,18 @@ func TestCoders(t *testing.T) { coder.NewDouble(), coder.NewBool()})}), val: &FullValue{Elm: int64(42), Elm2: &FullValue{Elm: float64(3.14), Elm2: true}}, + }, { + coder: &coder.Coder{Kind: coder.Window, Window: coder.NewGlobalWindow()}, + val: &FullValue{Windows: []typex.Window{window.GlobalWindow{}}}, + }, { + coder: &coder.Coder{Kind: coder.Window, Window: coder.NewIntervalWindow()}, + val: &FullValue{Windows: []typex.Window{window.IntervalWindow{Start: 0, End: 100}}}, + }, { + coder: coder.NewW(coder.NewVarInt(), coder.NewGlobalWindow()), + val: &FullValue{Elm: int64(13), Windows: []typex.Window{window.GlobalWindow{}}}, + }, { + coder: coder.NewPW(coder.NewString(), coder.NewGlobalWindow()), + val: &FullValue{Elm: "myString" /*Windowing info isn't encoded for PW so we can omit it here*/}, }, } { t.Run(fmt.Sprintf("%v", test.coder), func(t *testing.T) { @@ -116,4 +132,140 @@ func compareFV(t *testing.T, got *FullValue, want *FullValue) { t.Errorf("got %v [type: %s], want %v [type %s]", got, reflect.TypeOf(got), wantFv, reflect.TypeOf(wantFv)) } + + // Check if the desired FV has windowing information + if want.Windows != nil { + if gotLen, wantLen := len(got.Windows), len(want.Windows); gotLen != wantLen { + t.Fatalf("got %d windows in FV, want %v", gotLen, wantLen) + } + for i := range want.Windows { + if gotWin, wantWin := got.Windows[i], want.Windows[i]; !wantWin.Equals(gotWin) { + t.Errorf("got window %v at position %d, want %v", gotWin, i, wantWin) + } + } + } +} + +func TestIterableCoder(t *testing.T) { + cod := coder.NewI(coder.NewVarInt()) + wantVals := []int64{8, 24, 72} + val := &FullValue{Elm: wantVals} + + var buf bytes.Buffer + enc := MakeElementEncoder(cod) + if err := enc.Encode(val, &buf); err != nil { + t.Fatalf("Couldn't encode value: %v", err) + } + + dec := MakeElementDecoder(cod) + result, err := dec.Decode(&buf) + if err != nil { + t.Fatalf("Couldn't decode value: %v", err) + } + + gotVals, ok := result.Elm.([]int64) + if !ok { + t.Fatalf("got output element %v, want []int64", result.Elm) + } + + if got, want := len(gotVals), len(wantVals); got != want { + t.Errorf("got %d elements in iterable, want %d", got, want) + } + + for i := range gotVals { + if got, want := gotVals[i], wantVals[i]; got != want { + t.Errorf("got %d at position %d, want %d", got, i, want) + } + } +} + +// TODO(BEAM-10660): Update once proper timer support is added +func TestTimerCoder(t *testing.T) { + var buf bytes.Buffer + tCoder := coder.NewT(coder.NewVarInt(), coder.NewGlobalWindow()) + wantVal := &FullValue{Elm: int64(13)} + + enc := MakeElementEncoder(tCoder) + if err := enc.Encode(wantVal, &buf); err != nil { + t.Fatalf("Couldn't encode value: %v", err) + } + + dec := MakeElementDecoder(tCoder) + result, err := dec.Decode(&buf) + if err != nil { + t.Fatalf("Couldn't decode value: %v", err) + } + + compareFV(t, result, wantVal) +} + +type namedTypeForTest struct { + A, B int64 + C string +} + +func TestRowCoder(t *testing.T) { + var buf bytes.Buffer + rCoder := coder.NewR(typex.New(reflect.TypeOf((*namedTypeForTest)(nil)))) + wantStruct := &namedTypeForTest{A: int64(8), B: int64(24), C: "myString"} + wantVal := &FullValue{Elm: wantStruct} + + enc := MakeElementEncoder(rCoder) + if err := enc.Encode(wantVal, &buf); err != nil { + t.Fatalf("Couldn't encode value: %v", err) + } + + dec := MakeElementDecoder(rCoder) + result, err := dec.Decode(&buf) + if err != nil { + t.Fatalf("Couldn't decode value: %v", err) + } + gotPtr, ok := result.Elm.(*namedTypeForTest) + gotStruct := *gotPtr + if !ok { + t.Fatalf("got %v, want namedTypeForTest struct", result.Elm) + } + if got, want := gotStruct.A, wantStruct.A; got != want { + t.Errorf("got A field value %d, want %d", got, want) + } + if got, want := gotStruct.B, wantStruct.B; got != want { + t.Errorf("got B field value %d, want %d", got, want) + } + if got, want := gotStruct.C, wantStruct.C; got != want { + t.Errorf("got C field value %v, want %v", got, want) + } +} + +func TestPaneCoder(t *testing.T) { + pn := coder.NewPane(0x04) + val := &FullValue{Pane: pn} + cod := &coder.Coder{Kind: coder.PaneInfo} + + var buf bytes.Buffer + enc := MakeElementEncoder(cod) + if err := enc.Encode(val, &buf); err != nil { + t.Fatalf("Couldn't encode value: %v", err) + } + + dec := MakeElementDecoder(cod) + result, err := dec.Decode(&buf) + if err != nil { + t.Fatalf("Couldn't decode value: %v", err) + } + + if got, want := result.Pane.Timing, pn.Timing; got != want { + t.Errorf("got pane timing %v, want %v", got, want) + } + if got, want := result.Pane.IsFirst, pn.IsFirst; got != want { + t.Errorf("got IsFirst %v, want %v", got, want) + } + if got, want := result.Pane.IsLast, pn.IsLast; got != want { + t.Errorf("got IsLast %v, want %v", got, want) + } + if got, want := result.Pane.Index, pn.Index; got != want { + t.Errorf("got pane index %v, want %v", got, want) + } + if got, want := result.Pane.NonSpeculativeIndex, pn.NonSpeculativeIndex; got != want { + t.Errorf("got pane non-speculative index %v, want %v", got, want) + } } diff --git a/sdks/go/pkg/beam/core/util/stringx/bytes.go b/sdks/go/pkg/beam/core/util/stringx/bytes.go index b2110fa1072ce..bf4cd111dddec 100644 --- a/sdks/go/pkg/beam/core/util/stringx/bytes.go +++ b/sdks/go/pkg/beam/core/util/stringx/bytes.go @@ -15,6 +15,9 @@ // Package stringx contains utilities for working with strings. It // complements the standard "strings" package. +// +// Deprecated: the utilities in this package are unused within the code base +// and will be removed in a future Beam release. package stringx // ToBytes converts a string to a byte slice. diff --git a/sdks/go/pkg/beam/internal/errors/errors_test.go b/sdks/go/pkg/beam/internal/errors/errors_test.go index 614609ff80b89..26e0057a74679 100644 --- a/sdks/go/pkg/beam/internal/errors/errors_test.go +++ b/sdks/go/pkg/beam/internal/errors/errors_test.go @@ -57,6 +57,9 @@ func TestWrap(t *testing.T) { }, { err: Wrap(Wrap(New(base), msg1), msg2), want: errorStructure{{ERROR, msg2}, {ERROR, msg1}, {ERROR, base}}, + }, { + err: Wrap(nil, msg1), + want: nil, }, } for _, test := range tests { @@ -76,6 +79,13 @@ func TestWrapf(t *testing.T) { } } +func TestWrapf_NilErr(t *testing.T) { + err := Wrapf(nil, "%s %d", "ten", 10) + if err != nil { + t.Errorf(`Wrapf(nil, "%%s %%d", "ten", 10). Want: nil, Got: %q`, err) + } +} + func TestContext(t *testing.T) { tests := []struct { err error @@ -90,6 +100,9 @@ func TestContext(t *testing.T) { }, { err: Wrap(WithContext(WithContext(Wrap(New(base), msg1), ctx1), ctx2), msg2), want: errorStructure{{ERROR, msg2}, {CONTEXT, ctx2}, {CONTEXT, ctx1}, {ERROR, msg1}, {ERROR, base}}, + }, { + err: WithContext(nil, ctx1), + want: nil, }, } for _, test := range tests { @@ -100,6 +113,13 @@ func TestContext(t *testing.T) { } } +func TestWithContextf_NilErr(t *testing.T) { + err := WithContextf(nil, "%s %d", "ten", 10) + if err != nil { + t.Errorf(`WithContextf(nil, "%%s %%d", "ten", 10). Want: nil, Got: %q`, err) + } +} + func TestWithContextf(t *testing.T) { want := fmt.Sprintf("%s %d", "ten", 10) err := WithContextf(New(base), "%s %d", "ten", 10) @@ -129,6 +149,9 @@ func TestTopLevelMsg(t *testing.T) { }, { err: Wrap(SetTopLevelMsg(WithContext(SetTopLevelMsg(New(base), top1), ctx1), top2), msg1), want: top2, + }, { + err: SetTopLevelMsg(nil, top1), + want: "", }, } for _, test := range tests { @@ -147,10 +170,51 @@ func TestSetTopLevelMsgf(t *testing.T) { } } +func TestSetTopLevelMsgf_NilErr(t *testing.T) { + want := "" + err := SetTopLevelMsgf(nil, "%s %d", "ten", 10) + if getTop(err) != want { + t.Errorf("Incorrect formatting. Want: %q, Got: %q", want, getTop(err)) + } +} + +func TestError(t *testing.T) { + tests := []struct { + err error + want string + }{ + { + err: Wrap(New(base), msg1), + want: "message 1\n\tcaused by:\nbase", + }, + { + err: SetTopLevelMsg(New(base), top1), + want: "top level message 1\nFull error:\nbase", + }, + { + err: SetTopLevelMsg(Wrap(Wrap(New(base), msg1), msg2), top1), + want: "top level message 1\nFull error:\nmessage 2\n\tcaused by:\nmessage 1\n\tcaused by:\nbase", + }, + } + + for _, test := range tests { + if be, ok := test.err.(*beamError); ok { + if got, want := be.Error(), test.want; got != want { + t.Errorf("Incorrect formatting. Want: %q, Got: %q", want, got) + } + } else { + t.Errorf("Error should be type *beamError, got: %q", test.err) + } + } +} + // getStructure extracts the structure of an error, outputting a slice that // represents the nested messages in that error in the order they are output // and with the type of message (context or error) described. func getStructure(e error) errorStructure { + if e == nil { + return nil + } var structure errorStructure for { @@ -175,6 +239,10 @@ func getStructure(e error) errorStructure { } func equalStructure(left errorStructure, right errorStructure) bool { + if left == nil || right == nil { + return left == nil && right == nil + } + if len(left) != len(right) { return false } diff --git a/sdks/go/pkg/beam/runners/dataflow/dataflow.go b/sdks/go/pkg/beam/runners/dataflow/dataflow.go index 4cd877f5afde0..1a7da076755a0 100644 --- a/sdks/go/pkg/beam/runners/dataflow/dataflow.go +++ b/sdks/go/pkg/beam/runners/dataflow/dataflow.go @@ -159,8 +159,59 @@ func Execute(ctx context.Context, p *beam.Pipeline) (beam.PipelineResult, error) panic("Beam has not been initialized. Call beam.Init() before pipeline construction.") } - // (1) Gather job options + beam.PipelineOptions.LoadOptionsFromFlags(flagFilter) + opts, err := getJobOptions(ctx) + if err != nil { + return nil, err + } + + // (1) Build and submit + // NOTE(herohde) 10/8/2018: the last segment of the names must be "worker" and "dataflow-worker.jar". + id := fmt.Sprintf("go-%v-%v", atomic.AddInt32(&unique, 1), time.Now().UnixNano()) + + modelURL := gcsx.Join(*stagingLocation, id, "model") + workerURL := gcsx.Join(*stagingLocation, id, "worker") + jarURL := gcsx.Join(*stagingLocation, id, "dataflow-worker.jar") + xlangURL := gcsx.Join(*stagingLocation, id, "xlang") + + edges, _, err := p.Build() + if err != nil { + return nil, err + } + artifactURLs, err := dataflowlib.ResolveXLangArtifacts(ctx, edges, opts.Project, xlangURL) + if err != nil { + return nil, errors.WithContext(err, "resolving cross-language artifacts") + } + opts.ArtifactURLs = artifactURLs + environment, err := graphx.CreateEnvironment(ctx, jobopts.GetEnvironmentUrn(ctx), getContainerImage) + if err != nil { + return nil, errors.WithContext(err, "creating environment for model pipeline") + } + model, err := graphx.Marshal(edges, &graphx.Options{Environment: environment}) + if err != nil { + return nil, errors.WithContext(err, "generating model pipeline") + } + err = pipelinex.ApplySdkImageOverrides(model, jobopts.GetSdkImageOverrides()) + if err != nil { + return nil, errors.WithContext(err, "applying container image overrides") + } + + if *dryRun { + log.Info(ctx, "Dry-run: not submitting job!") + + log.Info(ctx, proto.MarshalTextString(model)) + job, err := dataflowlib.Translate(ctx, model, opts, workerURL, jarURL, modelURL) + if err != nil { + return nil, err + } + dataflowlib.PrintJob(ctx, job) + return nil, nil + } + return dataflowlib.Execute(ctx, model, opts, workerURL, jarURL, modelURL, *endpoint, *executeAsync) +} + +func getJobOptions(ctx context.Context) (*dataflowlib.JobOptions, error) { project := gcpopts.GetProjectFromFlagOrEnvironment(ctx) if project == "" { return nil, errors.New("no Google Cloud project specified. Use --project=") @@ -254,51 +305,9 @@ func Execute(ctx context.Context, p *beam.Pipeline) (beam.PipelineResult, error) opts.TempLocation = gcsx.Join(*stagingLocation, "tmp") } - // (1) Build and submit - // NOTE(herohde) 10/8/2018: the last segment of the names must be "worker" and "dataflow-worker.jar". - id := fmt.Sprintf("go-%v-%v", atomic.AddInt32(&unique, 1), time.Now().UnixNano()) - - modelURL := gcsx.Join(*stagingLocation, id, "model") - workerURL := gcsx.Join(*stagingLocation, id, "worker") - jarURL := gcsx.Join(*stagingLocation, id, "dataflow-worker.jar") - xlangURL := gcsx.Join(*stagingLocation, id, "xlang") - - edges, _, err := p.Build() - if err != nil { - return nil, err - } - artifactURLs, err := dataflowlib.ResolveXLangArtifacts(ctx, edges, opts.Project, xlangURL) - if err != nil { - return nil, errors.WithContext(err, "resolving cross-language artifacts") - } - opts.ArtifactURLs = artifactURLs - environment, err := graphx.CreateEnvironment(ctx, jobopts.GetEnvironmentUrn(ctx), getContainerImage) - if err != nil { - return nil, errors.WithContext(err, "creating environment for model pipeline") - } - model, err := graphx.Marshal(edges, &graphx.Options{Environment: environment}) - if err != nil { - return nil, errors.WithContext(err, "generating model pipeline") - } - err = pipelinex.ApplySdkImageOverrides(model, jobopts.GetSdkImageOverrides()) - if err != nil { - return nil, errors.WithContext(err, "applying container image overrides") - } - - if *dryRun { - log.Info(ctx, "Dry-run: not submitting job!") - - log.Info(ctx, proto.MarshalTextString(model)) - job, err := dataflowlib.Translate(ctx, model, opts, workerURL, jarURL, modelURL) - if err != nil { - return nil, err - } - dataflowlib.PrintJob(ctx, job) - return nil, nil - } - - return dataflowlib.Execute(ctx, model, opts, workerURL, jarURL, modelURL, *endpoint, *executeAsync) + return opts, nil } + func gcsRecorderHook(opts []string) perf.CaptureHook { bucket, prefix, err := gcsx.ParseObject(opts[0]) if err != nil { diff --git a/sdks/go/pkg/beam/runners/dataflow/dataflow_test.go b/sdks/go/pkg/beam/runners/dataflow/dataflow_test.go index 1e2844630c4b3..568860194ec18 100644 --- a/sdks/go/pkg/beam/runners/dataflow/dataflow_test.go +++ b/sdks/go/pkg/beam/runners/dataflow/dataflow_test.go @@ -15,7 +15,12 @@ package dataflow -import "testing" +import ( + "github.com/apache/beam/sdks/v2/go/pkg/beam/options/gcpopts" + "github.com/apache/beam/sdks/v2/go/pkg/beam/options/jobopts" + "sort" + "testing" +) func TestDontUseFlagAsPipelineOption(t *testing.T) { f := "dummy_flag" @@ -27,3 +32,131 @@ func TestDontUseFlagAsPipelineOption(t *testing.T) { t.Fatalf("%q should be in the filter, but isn't set", f) } } + +func TestGetJobOptions(t *testing.T) { + *labels = `{"label1": "val1", "label2": "val2"}` + *stagingLocation = "gs://testStagingLocation" + *autoscalingAlgorithm = "NONE" + *minCPUPlatform = "testPlatform" + + *gcpopts.Project = "testProject" + *gcpopts.Region = "testRegion" + + *jobopts.Experiments = "use_runner_v2,use_portable_job_submission" + *jobopts.JobName = "testJob" + + opts, err := getJobOptions(nil) + if err != nil { + t.Fatalf("getJobOptions() returned error %q, want %q", err, "nil") + } + if got, want := opts.Name, "testJob"; got != want { + t.Errorf("getJobOptions().Name = %q, want %q", got, want) + } + if got, want := len(opts.Experiments), 3; got != want { + t.Errorf("len(getJobOptions().Experiments) = %q, want %q", got, want) + } else { + sort.Strings(opts.Experiments) + expectedExperiments := []string{"min_cpu_platform=testPlatform", "use_portable_job_submission", "use_runner_v2"} + for i := 0; i < 3; i++ { + if got, want := opts.Experiments[i], expectedExperiments[i]; got != want { + t.Errorf("getJobOptions().Experiments = %q, want %q", got, want) + } + } + } + if got, want := opts.Project, "testProject"; got != want { + t.Errorf("getJobOptions().Project = %q, want %q", got, want) + } + if got, want := opts.Region, "testRegion"; got != want { + t.Errorf("getJobOptions().Region = %q, want %q", got, want) + } + if got, want := len(opts.Labels), 2; got != want { + t.Errorf("len(getJobOptions().Labels) = %q, want %q", got, want) + } else { + if got, want := opts.Labels["label1"], "val1"; got != want { + t.Errorf("getJobOptions().Labels[\"label1\"] = %q, want %q", got, want) + } + if got, want := opts.Labels["label2"], "val2"; got != want { + t.Errorf("getJobOptions().Labels[\"label2\"] = %q, want %q", got, want) + } + } + if got, want := opts.TempLocation, "gs://testStagingLocation/tmp"; got != want { + t.Errorf("getJobOptions().TempLocation = %q, want %q", got, want) + } +} + +func TestGetJobOptions_NoExperimentsSet(t *testing.T) { + *labels = `{"label1": "val1", "label2": "val2"}` + *stagingLocation = "gs://testStagingLocation" + *autoscalingAlgorithm = "NONE" + *minCPUPlatform = "" + + *gcpopts.Project = "testProject" + *gcpopts.Region = "testRegion" + + *jobopts.Experiments = "" + *jobopts.JobName = "testJob" + + opts, err := getJobOptions(nil) + + if err != nil { + t.Fatalf("getJobOptions() returned error %q, want %q", err, "nil") + } + if got, want := len(opts.Experiments), 2; got != want { + t.Fatalf("len(getJobOptions().Experiments) = %q, want %q", got, want) + } + sort.Strings(opts.Experiments) + expectedExperiments := []string{"use_portable_job_submission", "use_unified_worker"} + for i := 0; i < 2; i++ { + if got, want := opts.Experiments[i], expectedExperiments[i]; got != want { + t.Errorf("getJobOptions().Experiments = %q, want %q", got, want) + } + } +} + +func TestGetJobOptions_NoStagingLocation(t *testing.T) { + *stagingLocation = "" + *gcpopts.Project = "testProject" + *gcpopts.Region = "testRegion" + + _, err := getJobOptions(nil) + if err == nil { + t.Fatalf("getJobOptions() returned error nil, want an error") + } +} + +func TestGetJobOptions_InvalidAutoscaling(t *testing.T) { + *labels = `{"label1": "val1", "label2": "val2"}` + *stagingLocation = "gs://testStagingLocation" + *autoscalingAlgorithm = "INVALID" + *minCPUPlatform = "testPlatform" + + *gcpopts.Project = "testProject" + *gcpopts.Region = "testRegion" + + *jobopts.Experiments = "use_runner_v2,use_portable_job_submission" + *jobopts.JobName = "testJob" + + _, err := getJobOptions(nil) + if err == nil { + t.Fatalf("getJobOptions() returned error nil, want an error") + } +} + +func TestGetJobOptions_DockerNoImage(t *testing.T) { + *jobopts.EnvironmentType = "docker" + *jobopts.EnvironmentConfig = "testContainerImage" + + if got, want := getContainerImage(nil), "testContainerImage"; got != want { + t.Fatalf("getContainerImage() = %q, want %q", got, want) + } +} + +func TestGetJobOptions_DockerWithImage(t *testing.T) { + *jobopts.EnvironmentType = "docker" + *jobopts.EnvironmentConfig = "testContainerImage" + *image = "testContainerImageOverride" + + if got, want := getContainerImage(nil), "testContainerImageOverride"; got != want { + t.Fatalf("getContainerImage() = %q, want %q", got, want) + } +} diff --git a/sdks/go/pkg/beam/util/errorx/guarded.go b/sdks/go/pkg/beam/util/errorx/guarded.go index cc0b07b4eee1f..186885b717c62 100644 --- a/sdks/go/pkg/beam/util/errorx/guarded.go +++ b/sdks/go/pkg/beam/util/errorx/guarded.go @@ -39,7 +39,7 @@ func (g *GuardedError) TrySetError(err error) bool { g.mu.Lock() defer g.mu.Unlock() - upd := g.err == nil + upd := (g.err == nil) if upd { g.err = err } diff --git a/sdks/go/pkg/beam/util/errorx/guarded_test.go b/sdks/go/pkg/beam/util/errorx/guarded_test.go new file mode 100644 index 0000000000000..1e9c9b2247060 --- /dev/null +++ b/sdks/go/pkg/beam/util/errorx/guarded_test.go @@ -0,0 +1,45 @@ +// 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 errorx + +import ( + "errors" + "testing" +) + +func TestTrySetError(t *testing.T) { + var gu GuardedError + setErr := errors.New("attempted error") + success := gu.TrySetError(setErr) + if !success { + t.Fatal("got false when trying to set error, want true") + } + if got, want := gu.Error(), setErr; got != want { + t.Errorf("got error %v when checking message, want %v", got, want) + } +} + +func TestTrySetError_bad(t *testing.T) { + setErr := errors.New("old error") + gu := &GuardedError{err: setErr} + success := gu.TrySetError(setErr) + if success { + t.Fatal("got true when trying to set error, want false") + } + if got, want := gu.Error(), setErr; got != want { + t.Errorf("got error %v when checking message, want %v", got, want) + } +} diff --git a/playground/backend/internal/utils/validators_utils.go b/sdks/go/pkg/beam/util/gcsx/example_test.go similarity index 54% rename from playground/backend/internal/utils/validators_utils.go rename to sdks/go/pkg/beam/util/gcsx/example_test.go index 5d9406e912df7..52664a054a878 100644 --- a/playground/backend/internal/utils/validators_utils.go +++ b/sdks/go/pkg/beam/util/gcsx/example_test.go @@ -13,28 +13,35 @@ // See the License for the specific language governing permissions and // limitations under the License. -package utils +package gcsx_test import ( - pb "beam.apache.org/playground/backend/internal/api/v1" - "beam.apache.org/playground/backend/internal/validators" - "fmt" + "context" + "time" + + "cloud.google.com/go/storage" + "github.com/apache/beam/sdks/v2/go/pkg/beam/util/gcsx" ) -// GetValidators returns slice of validators.Validator according to sdk -func GetValidators(sdk pb.Sdk, filepath string) (*[]validators.Validator, error) { - var val *[]validators.Validator - switch sdk { - case pb.Sdk_SDK_JAVA: - val = validators.GetJavaValidators(filepath) - case pb.Sdk_SDK_GO: - val = validators.GetGoValidators(filepath) - case pb.Sdk_SDK_PYTHON: - val = validators.GetPyValidators(filepath) - case pb.Sdk_SDK_SCIO: - val = validators.GetScioValidators(filepath) - default: - return nil, fmt.Errorf("incorrect sdk: %s", sdk) +func Example() { + ctx := context.Background() + c, err := gcsx.NewClient(ctx, storage.ScopeReadOnly) + if err != nil { + // do something + } + + buckets, object, err := gcsx.ParseObject("gs://some-bucket/some-object") + if err != nil { + // do something } - return val, nil + + ctx, cancel := context.WithTimeout(ctx, 30*time.Second) + defer cancel() + + bytes, err := gcsx.ReadObject(ctx, c, buckets, object) + if err != nil { + // do something + } + + _ = bytes } diff --git a/sdks/go/pkg/beam/util/gcsx/gcs_test.go b/sdks/go/pkg/beam/util/gcsx/gcs_test.go index 52664a054a878..90fb4b59f2fe8 100644 --- a/sdks/go/pkg/beam/util/gcsx/gcs_test.go +++ b/sdks/go/pkg/beam/util/gcsx/gcs_test.go @@ -13,35 +13,86 @@ // See the License for the specific language governing permissions and // limitations under the License. -package gcsx_test +package gcsx import ( - "context" - "time" + "strings" + "testing" - "cloud.google.com/go/storage" - "github.com/apache/beam/sdks/v2/go/pkg/beam/util/gcsx" + "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" ) -func Example() { - ctx := context.Background() - c, err := gcsx.NewClient(ctx, storage.ScopeReadOnly) - if err != nil { - // do something +func TestMakeObject(t *testing.T) { + if got, want := MakeObject("some-bucket", "some/path"), "gs://some-bucket/some/path"; got != want { + t.Fatalf("MakeObject() Got: %v Want: %v", got, want) } +} - buckets, object, err := gcsx.ParseObject("gs://some-bucket/some-object") - if err != nil { - // do something +func TestParseObject(t *testing.T) { + tests := []struct { + object string + bucket string + path string + err error + }{ + { + object: "gs://some-bucket/some-object", + bucket: "some-bucket", + path: "some-object", + err: nil, + }, + { + object: "gs://some-bucket", + bucket: "some-bucket", + path: "", + err: nil, + }, + { + object: "gs://", + bucket: "", + path: "", + err: errors.Errorf("object gs:// must have bucket"), + }, + { + object: "other://some-bucket/some-object", + bucket: "", + path: "", + err: errors.Errorf("object other://some-bucket/some-object must have 'gs' scheme"), + }, } - ctx, cancel := context.WithTimeout(ctx, 30*time.Second) - defer cancel() - - bytes, err := gcsx.ReadObject(ctx, c, buckets, object) - if err != nil { - // do something + for _, test := range tests { + if bucket, path, err := ParseObject(test.object); bucket != test.bucket || path != test.path || (err != nil && test.err == nil) || (err == nil && test.err != nil) { + t.Errorf("ParseObject(%v) Got: %v, %v, %v Want: %v, %v, %v", test.object, bucket, path, err, test.bucket, test.path, test.err) + } } +} - _ = bytes +func TestJoin(t *testing.T) { + tests := []struct { + object string + elms []string + result string + }{ + { + object: "gs://some-bucket/some-object", + elms: []string{"some/path", "more/pathing"}, + result: "gs://some-bucket/some-object/some/path/more/pathing", + }, + { + object: "gs://some-bucket/some-object", + elms: []string{"some/path"}, + result: "gs://some-bucket/some-object/some/path", + }, + { + object: "gs://some-bucket/some-object", + elms: []string{}, + result: "gs://some-bucket/some-object", + }, + } + for _, test := range tests { + if got, want := Join(test.object, test.elms...), test.result; got != want { + t.Errorf("Join(%v, %v) Got: %v Want: %v", test.object, strings.Join(test.elms, ", "), got, want) + } + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java index 4cb7eb4eac45f..d014f08ac2b8c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java @@ -36,11 +36,13 @@ import com.fasterxml.jackson.databind.annotation.JsonSerialize; import com.fasterxml.jackson.databind.deser.DefaultDeserializationContext; import com.fasterxml.jackson.databind.deser.impl.MethodProperty; +import com.fasterxml.jackson.databind.deser.impl.TypeWrappedDeserializer; import com.fasterxml.jackson.databind.introspect.AnnotatedMember; import com.fasterxml.jackson.databind.introspect.AnnotatedMethod; import com.fasterxml.jackson.databind.introspect.AnnotationCollector; import com.fasterxml.jackson.databind.introspect.BeanPropertyDefinition; import com.fasterxml.jackson.databind.introspect.TypeResolutionContext; +import com.fasterxml.jackson.databind.jsontype.TypeDeserializer; import com.fasterxml.jackson.databind.node.TreeTraversingParser; import com.fasterxml.jackson.databind.ser.DefaultSerializerProvider; import com.fasterxml.jackson.databind.type.TypeBindings; @@ -1730,21 +1732,23 @@ private static JsonDeserializer computeDeserializerForMethod(Method meth BeanProperty prop = createBeanProperty(method); AnnotatedMember annotatedMethod = prop.getMember(); + DefaultDeserializationContext context = DESERIALIZATION_CONTEXT.get(); Object maybeDeserializerClass = - DESERIALIZATION_CONTEXT - .get() - .getAnnotationIntrospector() - .findDeserializer(annotatedMethod); + context.getAnnotationIntrospector().findDeserializer(annotatedMethod); JsonDeserializer jsonDeserializer = - DESERIALIZATION_CONTEXT - .get() - .deserializerInstance(annotatedMethod, maybeDeserializerClass); + context.deserializerInstance(annotatedMethod, maybeDeserializerClass); if (jsonDeserializer == null) { - jsonDeserializer = - DESERIALIZATION_CONTEXT.get().findContextualValueDeserializer(prop.getType(), prop); + jsonDeserializer = context.findContextualValueDeserializer(prop.getType(), prop); } + + TypeDeserializer typeDeserializer = + context.getFactory().findTypeDeserializer(context.getConfig(), prop.getType()); + if (typeDeserializer != null) { + jsonDeserializer = new TypeWrappedDeserializer(typeDeserializer, jsonDeserializer); + } + return jsonDeserializer; } catch (JsonMappingException e) { throw new RuntimeException(e); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java index 94fd3f41faacd..ffdfbc8681a13 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java @@ -39,6 +39,8 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonProcessingException; @@ -60,7 +62,6 @@ import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; import org.apache.beam.model.jobmanagement.v1.JobApi.PipelineOptionDescriptor; import org.apache.beam.model.jobmanagement.v1.JobApi.PipelineOptionType; @@ -1070,6 +1071,53 @@ public void testComplexTypes() { assertEquals("value2", options.getObjectValue().get().value2); } + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME) + @JsonSubTypes({ + @JsonSubTypes.Type(value = PolymorphicTypeOne.class, name = "one"), + @JsonSubTypes.Type(value = PolymorphicTypeTwo.class, name = "two") + }) + public abstract static class PolymorphicType { + String key; + + @JsonProperty("key") + public String getKey() { + return key; + } + + public void setKey(String key) { + this.key = key; + } + } + + public static class PolymorphicTypeOne extends PolymorphicType {} + + public static class PolymorphicTypeTwo extends PolymorphicType {} + + public interface PolymorphicTypes extends PipelineOptions { + PolymorphicType getObject(); + + void setObject(PolymorphicType value); + + ValueProvider getObjectValue(); + + void setObjectValue(ValueProvider value); + } + + @Test + public void testPolymorphicType() { + String[] args = + new String[] { + "--object={\"key\":\"value\",\"@type\":\"one\"}", + "--objectValue={\"key\":\"value\",\"@type\":\"two\"}" + }; + PolymorphicTypes options = PipelineOptionsFactory.fromArgs(args).as(PolymorphicTypes.class); + assertEquals("value", options.getObject().key); + assertEquals(PolymorphicTypeOne.class, options.getObject().getClass()); + + assertEquals("value", options.getObjectValue().get().key); + assertEquals(PolymorphicTypeTwo.class, options.getObjectValue().get().getClass()); + } + @Test public void testMissingArgument() { String[] args = new String[] {}; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index 7a20afbd151e8..89c05b8f9f238 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -180,6 +180,7 @@ static class Factory @@ -242,6 +243,7 @@ static class Factory onTimerContext; private final OnWindowExpirationContext onWindowExpirationContext; private final FinishBundleArgumentProvider finishBundleArgumentProvider; + private final Duration allowedLateness; /** * Used to guarantee a consistent view of this {@link FnApiDoFnRunner} while setting up for {@link @@ -344,6 +346,7 @@ private interface TriFunction { Map windowingStrategies, Consumer addStartFunction, Consumer addFinishFunction, + Consumer addResetFunction, Consumer addTearDownFunction, Function>> getPCollectionConsumer, TriFunction>, Coder> addPCollectionConsumer, @@ -457,6 +460,13 @@ private interface TriFunction { } timerFamilyInfos = timerFamilyInfosBuilder.build(); + this.mainInputId = ParDoTranslation.getMainInputName(pTransform); + this.allowedLateness = + rehydratedComponents + .getPCollection(pTransform.getInputsOrThrow(mainInputId)) + .getWindowingStrategy() + .getAllowedLateness(); + } catch (IOException exn) { throw new IllegalArgumentException("Malformed ParDoPayload", exn); } @@ -473,12 +483,11 @@ private interface TriFunction { this.bundleFinalizer = bundleFinalizer; this.onTimerContext = new OnTimerContext(); this.onWindowExpirationContext = new OnWindowExpirationContext<>(); + this.timerBundleTracker = + new FnApiTimerBundleTracker( + keyCoder, windowCoder, this::getCurrentKey, () -> currentWindow); + addResetFunction.accept(timerBundleTracker::reset); - try { - this.mainInputId = ParDoTranslation.getMainInputName(pTransform); - } catch (IOException e) { - throw new RuntimeException(e); - } this.mainOutputConsumers = (Collection>>) (Collection) localNameToConsumer.get(mainOutputTag.getId()); @@ -756,9 +765,6 @@ private Object getCurrentKey() { } private void startBundle() { - timerBundleTracker = - new FnApiTimerBundleTracker( - keyCoder, windowCoder, this::getCurrentKey, () -> currentWindow); doFnInvoker.invokeStartBundle(startBundleArgumentProvider); } @@ -1694,14 +1700,9 @@ private void processTimer( // The timerIdOrTimerFamilyId contains either a timerId from timer declaration or // timerFamilyId // from timer family declaration. - String timerId = - timerIdOrTimerFamilyId.startsWith(TimerFamilyDeclaration.PREFIX) - ? "" - : timerIdOrTimerFamilyId; - String timerFamilyId = - timerIdOrTimerFamilyId.startsWith(TimerFamilyDeclaration.PREFIX) - ? timerIdOrTimerFamilyId - : ""; + boolean isFamily = timerIdOrTimerFamilyId.startsWith(TimerFamilyDeclaration.PREFIX); + String timerId = isFamily ? "" : timerIdOrTimerFamilyId; + String timerFamilyId = isFamily ? timerIdOrTimerFamilyId : ""; processTimerDirect(timerFamilyId, timerId, timeDomain, timer); } } @@ -1778,7 +1779,6 @@ private class FnApiTimer implements org.apache.beam.sdk.state.Timer { private final K userKey; private final String dynamicTimerTag; private final TimeDomain timeDomain; - private final Duration allowedLateness; private final Instant fireTimestamp; private final Instant elementTimestampOrTimerHoldTimestamp; private final BoundedWindow boundedWindow; @@ -1817,18 +1817,6 @@ private class FnApiTimer implements org.apache.beam.sdk.state.Timer { throw new IllegalArgumentException( String.format("Unknown or unsupported time domain %s", timeDomain)); } - - try { - this.allowedLateness = - rehydratedComponents - .getPCollection( - pTransform.getInputsOrThrow(ParDoTranslation.getMainInputName(pTransform))) - .getWindowingStrategy() - .getAllowedLateness(); - } catch (IOException e) { - throw new IllegalArgumentException( - String.format("Unable to get allowed lateness for timer %s", timerIdOrFamily)); - } } @Override diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiTimerBundleTracker.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiTimerBundleTracker.java index f8a1dc3f9a258..39f735d98efe4 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiTimerBundleTracker.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiTimerBundleTracker.java @@ -153,6 +153,10 @@ public FnApiTimerBundleTracker( }); } + public void reset() { + timerModifications.clear(); + } + public void timerModified(String timerFamilyOrId, TimeDomain timeDomain, Timer timer) { ByteString keyString = encodedCurrentKeySupplier.get(); ByteString windowString = encodedCurrentWindowSupplier.get(); diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java index 13b8c24ed74af..9970e802ee340 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java @@ -509,6 +509,7 @@ public void testOrderOfSetupTeardownCalls() throws Exception { PCollection.newBuilder() .setWindowingStrategyId("window-strategy") .setCoderId("2L-output-coder") + .setIsBounded(IsBounded.Enum.BOUNDED) .build()) .putWindowingStrategies( "window-strategy", diff --git a/sdks/java/io/amazon-web-services2/build.gradle b/sdks/java/io/amazon-web-services2/build.gradle index 1af0532fd13c8..dceb4c41bed91 100644 --- a/sdks/java/io/amazon-web-services2/build.gradle +++ b/sdks/java/io/amazon-web-services2/build.gradle @@ -59,10 +59,10 @@ dependencies { testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:io:kinesis", configuration: "testRuntimeMigration") testImplementation "io.findify:s3mock_2.12:0.2.6" + testImplementation 'org.elasticmq:elasticmq-rest-sqs_2.12:1.3.5' testImplementation library.java.mockito_core testImplementation library.java.guava_testlib testImplementation library.java.junit - testImplementation 'org.elasticmq:elasticmq-rest-sqs_2.12:0.15.6' // later versions conflict with s3mock testImplementation library.java.hamcrest testImplementation "org.assertj:assertj-core:3.11.1" testRuntimeOnly library.java.slf4j_jdk14 diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sqs/SqsUnboundedReaderTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sqs/SqsUnboundedReaderTest.java index 2c2329974df2b..5b1ac0cbb28b2 100644 --- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sqs/SqsUnboundedReaderTest.java +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sqs/SqsUnboundedReaderTest.java @@ -37,7 +37,6 @@ import org.apache.beam.sdk.io.aws2.sqs.EmbeddedSqsServer.TestCaseEnv; import org.apache.beam.sdk.util.CoderUtils; import org.junit.ClassRule; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -110,7 +109,6 @@ public void testAckDeletedMessage() throws IOException { } @Test - @Ignore("Behavior of SQSRestServer is broken: https://issues.apache.org/jira/browse/BEAM-13738") public void testExtendDeletedMessage() throws IOException { setupMessages(DATA); Clock clock = mock(Clock.class); diff --git a/sdks/java/io/amazon-web-services2/src/test/resources/application.conf b/sdks/java/io/amazon-web-services2/src/test/resources/application.conf new file mode 100644 index 0000000000000..f1fcb1e9f3da4 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/resources/application.conf @@ -0,0 +1,21 @@ +############################################################################### +# 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. +############################################################################### + +# Enable implicit HEAD requests using GET routes for S3Mock +# https://doc.akka.io/docs/akka-http/current/migration-guide/migration-guide-10.2.x.html#transparent-head-requests-now-disabled-by-default +akka.http.server.transparent-head-requests = on diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index be4e551cd6b9a..364321e0b27ed 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -52,6 +52,7 @@ dependencies { // BEAM-13781: gax-grpc's gRPC version was older than Beam declared exclude group: 'io.grpc', module: 'grpc-netty-shaded' } + implementation library.java.gax_grpc_test implementation library.java.gax_httpjson permitUnusedDeclared library.java.gax_httpjson // BEAM-8755 implementation library.java.google_api_client @@ -146,6 +147,7 @@ dependencies { testImplementation library.java.powermock testImplementation library.java.powermock_mockito testImplementation library.java.joda_time + testImplementation library.java.google_cloud_spanner_test testRuntimeOnly library.java.slf4j_jdk14 } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index f6d433498fde1..97a9bfa1595c7 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -74,7 +74,6 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableSchemaToJsonSchema; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableSpecToTableRef; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TimePartitioningToJson; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.Method; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryResourceNaming.JobType; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.JobService; @@ -91,6 +90,8 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; +import org.apache.beam.sdk.schemas.FieldAccessDescriptor; +import org.apache.beam.sdk.schemas.ProjectionProducer; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; @@ -117,6 +118,7 @@ import org.apache.beam.sdk.values.ValueInSingleWindow; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Predicates; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; @@ -730,7 +732,8 @@ public Read withTemplateCompatibility() { /** Implementation of {@link BigQueryIO#read(SerializableFunction)}. */ @AutoValue - public abstract static class TypedRead extends PTransform> { + public abstract static class TypedRead extends PTransform> + implements ProjectionProducer>> { /** Determines the method used to read data from BigQuery. */ public enum Method { /** The default behavior if no method is explicitly set. Currently {@link #EXPORT}. */ @@ -1619,6 +1622,29 @@ public TypedRead withTestServices(BigQueryServices testServices) { public TypedRead useAvroLogicalTypes() { return toBuilder().setUseAvroLogicalTypes(true).build(); } + + @Override + public boolean supportsProjectionPushdown() { + // We can't do projection pushdown when a query is set. The query may project certain fields + // itself, and we can't know without parsing the query. + return Method.DIRECT_READ.equals(getMethod()) && getQuery() == null; + } + + @Override + public PTransform> actuateProjectionPushdown( + Map, FieldAccessDescriptor> outputFields) { + Preconditions.checkArgument(supportsProjectionPushdown()); + FieldAccessDescriptor fieldAccessDescriptor = outputFields.get(new TupleTag<>("output")); + org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull( + fieldAccessDescriptor, "Expected pushdown on the main output (tagged 'output')"); + Preconditions.checkArgument( + outputFields.size() == 1, + "Expected only to pushdown on the main output (tagged 'output'). Requested tags: %s", + outputFields.keySet()); + ImmutableList fields = + ImmutableList.copyOf(fieldAccessDescriptor.fieldNamesAccessed()); + return withSelectedFields(fields); + } } static String getExtractDestinationUri(String extractDestinationDir) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java index e34863ea97de0..a4223dc804eaf 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.gcp.spanner; +import com.google.api.gax.grpc.testing.LocalChannelProvider; import com.google.api.gax.retrying.RetrySettings; import com.google.api.gax.rpc.FixedHeaderProvider; import com.google.api.gax.rpc.ServerStreamingCallSettings; @@ -107,34 +108,59 @@ public static SpannerAccessor getOrCreate(SpannerConfig spannerConfig) { private static SpannerAccessor createAndConnect(SpannerConfig spannerConfig) { SpannerOptions.Builder builder = SpannerOptions.newBuilder(); - ValueProvider commitDeadline = spannerConfig.getCommitDeadline(); - if (commitDeadline != null && commitDeadline.get().getMillis() > 0) { + // Set retryable codes for all API methods + if (spannerConfig.getRetryableCodes() != null) { + builder + .getSpannerStubSettingsBuilder() + .applyToAllUnaryMethods( + input -> { + input.setRetryableCodes(spannerConfig.getRetryableCodes()); + return null; + }); + builder + .getSpannerStubSettingsBuilder() + .executeStreamingSqlSettings() + .setRetryableCodes(spannerConfig.getRetryableCodes()); + } + // Set commit retry settings + UnaryCallSettings.Builder commitSettings = + builder.getSpannerStubSettingsBuilder().commitSettings(); + ValueProvider commitDeadline = spannerConfig.getCommitDeadline(); + if (spannerConfig.getCommitRetrySettings() != null) { + commitSettings.setRetrySettings(spannerConfig.getCommitRetrySettings()); + } else if (commitDeadline != null && commitDeadline.get().getMillis() > 0) { // Set the GRPC deadline on the Commit API call. - UnaryCallSettings.Builder commitSettings = - builder.getSpannerStubSettingsBuilder().commitSettings(); - RetrySettings.Builder commitRetrySettings = commitSettings.getRetrySettings().toBuilder(); + RetrySettings.Builder commitRetrySettingsBuilder = + commitSettings.getRetrySettings().toBuilder(); commitSettings.setRetrySettings( - commitRetrySettings + commitRetrySettingsBuilder .setTotalTimeout(org.threeten.bp.Duration.ofMillis(commitDeadline.get().getMillis())) .setMaxRpcTimeout(org.threeten.bp.Duration.ofMillis(commitDeadline.get().getMillis())) .setInitialRpcTimeout( org.threeten.bp.Duration.ofMillis(commitDeadline.get().getMillis())) .build()); } - // Setting the timeout for streaming read to 2 hours. This is 1 hour by default - // after BEAM 2.20. + + // Set execute streaming sql retry settings ServerStreamingCallSettings.Builder executeStreamingSqlSettings = builder.getSpannerStubSettingsBuilder().executeStreamingSqlSettings(); - RetrySettings.Builder executeSqlStreamingRetrySettings = - executeStreamingSqlSettings.getRetrySettings().toBuilder(); - executeStreamingSqlSettings.setRetrySettings( - executeSqlStreamingRetrySettings - .setInitialRpcTimeout(org.threeten.bp.Duration.ofMinutes(120)) - .setMaxRpcTimeout(org.threeten.bp.Duration.ofMinutes(120)) - .setTotalTimeout(org.threeten.bp.Duration.ofMinutes(120)) - .build()); + if (spannerConfig.getExecuteStreamingSqlRetrySettings() != null) { + executeStreamingSqlSettings.setRetrySettings( + spannerConfig.getExecuteStreamingSqlRetrySettings()); + } else { + // Setting the timeout for streaming read to 2 hours. This is 1 hour by default + // after BEAM 2.20. + RetrySettings.Builder executeSqlStreamingRetrySettings = + executeStreamingSqlSettings.getRetrySettings().toBuilder(); + executeStreamingSqlSettings.setRetrySettings( + executeSqlStreamingRetrySettings + .setInitialRpcTimeout(org.threeten.bp.Duration.ofMinutes(120)) + .setMaxRpcTimeout(org.threeten.bp.Duration.ofMinutes(120)) + .setTotalTimeout(org.threeten.bp.Duration.ofMinutes(120)) + .build()); + } ValueProvider projectId = spannerConfig.getProjectId(); if (projectId != null) { @@ -151,6 +177,10 @@ private static SpannerAccessor createAndConnect(SpannerConfig spannerConfig) { ValueProvider emulatorHost = spannerConfig.getEmulatorHost(); if (emulatorHost != null) { builder.setEmulatorHost(emulatorHost.get()); + if (spannerConfig.getIsLocalChannelProvider() != null + && spannerConfig.getIsLocalChannelProvider().get()) { + builder.setChannelProvider(LocalChannelProvider.create(emulatorHost.get())); + } builder.setCredentials(NoCredentials.getInstance()); } String userAgentString = USER_AGENT_PREFIX + "/" + ReleaseInfo.getReleaseInfo().getVersion(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java index 7fcc0b5676b92..a00b7896c35af 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java @@ -19,6 +19,8 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; +import com.google.api.gax.retrying.RetrySettings; +import com.google.api.gax.rpc.StatusCode.Code; import com.google.auto.value.AutoValue; import com.google.cloud.ServiceFactory; import com.google.cloud.spanner.Options.RpcPriority; @@ -28,6 +30,7 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; @@ -56,10 +59,18 @@ public abstract class SpannerConfig implements Serializable { public abstract @Nullable ValueProvider getEmulatorHost(); + public abstract @Nullable ValueProvider getIsLocalChannelProvider(); + public abstract @Nullable ValueProvider getCommitDeadline(); public abstract @Nullable ValueProvider getMaxCumulativeBackoff(); + public abstract @Nullable RetrySettings getExecuteStreamingSqlRetrySettings(); + + public abstract @Nullable RetrySettings getCommitRetrySettings(); + + public abstract @Nullable ImmutableSet getRetryableCodes(); + public abstract @Nullable ValueProvider getRpcPriority(); @VisibleForTesting @@ -117,10 +128,19 @@ public abstract static class Builder { abstract Builder setEmulatorHost(ValueProvider emulatorHost); + abstract Builder setIsLocalChannelProvider(ValueProvider isLocalChannelProvider); + abstract Builder setCommitDeadline(ValueProvider commitDeadline); abstract Builder setMaxCumulativeBackoff(ValueProvider maxCumulativeBackoff); + abstract Builder setExecuteStreamingSqlRetrySettings( + RetrySettings executeStreamingSqlRetrySettings); + + abstract Builder setCommitRetrySettings(RetrySettings commitRetrySettings); + + abstract Builder setRetryableCodes(ImmutableSet retryableCodes); + abstract Builder setServiceFactory(ServiceFactory serviceFactory); abstract Builder setRpcPriority(ValueProvider rpcPriority); @@ -160,6 +180,10 @@ public SpannerConfig withEmulatorHost(ValueProvider emulatorHost) { return toBuilder().setEmulatorHost(emulatorHost).build(); } + public SpannerConfig withIsLocalChannelProvider(ValueProvider isLocalChannelProvider) { + return toBuilder().setIsLocalChannelProvider(isLocalChannelProvider).build(); + } + public SpannerConfig withCommitDeadline(Duration commitDeadline) { return withCommitDeadline(ValueProvider.StaticValueProvider.of(commitDeadline)); } @@ -176,6 +200,21 @@ public SpannerConfig withMaxCumulativeBackoff(ValueProvider maxCumulat return toBuilder().setMaxCumulativeBackoff(maxCumulativeBackoff).build(); } + public SpannerConfig withExecuteStreamingSqlRetrySettings( + RetrySettings executeStreamingSqlRetrySettings) { + return toBuilder() + .setExecuteStreamingSqlRetrySettings(executeStreamingSqlRetrySettings) + .build(); + } + + public SpannerConfig withCommitRetrySettings(RetrySettings commitRetrySettings) { + return toBuilder().setCommitRetrySettings(commitRetrySettings).build(); + } + + public SpannerConfig withRetryableCodes(ImmutableSet retryableCodes) { + return toBuilder().setRetryableCodes(retryableCodes).build(); + } + @VisibleForTesting SpannerConfig withServiceFactory(ServiceFactory serviceFactory) { return toBuilder().setServiceFactory(serviceFactory).build(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index 9fd929d097132..863d88ab54e68 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -23,6 +23,8 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; +import com.google.api.gax.retrying.RetrySettings; +import com.google.api.gax.rpc.StatusCode.Code; import com.google.auto.value.AutoValue; import com.google.cloud.ServiceFactory; import com.google.cloud.Timestamp; @@ -79,6 +81,7 @@ import org.apache.beam.sdk.metrics.Distribution; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; @@ -115,6 +118,7 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Stopwatch; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.primitives.UnsignedBytes; import org.checkerframework.checker.nullness.qual.Nullable; @@ -1349,6 +1353,8 @@ public abstract static class ReadChangeStream abstract @Nullable String getMetadataDatabase(); + abstract @Nullable String getMetadataTable(); + abstract Timestamp getInclusiveStartAt(); abstract @Nullable Timestamp getInclusiveEndAt(); @@ -1370,6 +1376,8 @@ abstract static class Builder { abstract Builder setMetadataDatabase(String metadataDatabase); + abstract Builder setMetadataTable(String metadataTable); + abstract Builder setInclusiveStartAt(Timestamp inclusiveStartAt); abstract Builder setInclusiveEndAt(Timestamp inclusiveEndAt); @@ -1434,6 +1442,10 @@ public ReadChangeStream withMetadataDatabase(String metadataDatabase) { return toBuilder().setMetadataDatabase(metadataDatabase).build(); } + public ReadChangeStream withMetadataTable(String metadataTable) { + return toBuilder().setMetadataTable(metadataTable).build(); + } + /** Specifies the time that the change stream should be read from. */ public ReadChangeStream withInclusiveStartAt(Timestamp timestamp) { return toBuilder().setInclusiveStartAt(timestamp).build(); @@ -1497,7 +1509,8 @@ && getInclusiveStartAt().toSqlTimestamp().after(getInclusiveEndAt().toSqlTimesta final String partitionMetadataDatabaseId = MoreObjects.firstNonNull(getMetadataDatabase(), changeStreamDatabaseId.getDatabase()); final String partitionMetadataTableName = - generatePartitionMetadataTableName(partitionMetadataDatabaseId); + MoreObjects.firstNonNull( + getMetadataTable(), generatePartitionMetadataTableName(partitionMetadataDatabaseId)); if (getTraceSampleProbability() != null) { TraceConfig globalTraceConfig = Tracing.getTraceConfig(); @@ -1511,16 +1524,36 @@ && getInclusiveStartAt().toSqlTimestamp().after(getInclusiveEndAt().toSqlTimesta .spanBuilder("SpannerIO.ReadChangeStream.expand") .setRecordEvents(true) .startScopedSpan()) { - final SpannerConfig changeStreamSpannerConfig = getSpannerConfig(); + SpannerConfig changeStreamSpannerConfig = getSpannerConfig(); + // Set default retryable errors for ReadChangeStream + if (changeStreamSpannerConfig.getRetryableCodes() == null) { + ImmutableSet defaultRetryableCodes = + ImmutableSet.of(Code.UNAVAILABLE, Code.ABORTED); + changeStreamSpannerConfig = + changeStreamSpannerConfig + .toBuilder() + .setRetryableCodes(defaultRetryableCodes) + .build(); + } + // Set default retry timeouts for ReadChangeStream + if (changeStreamSpannerConfig.getExecuteStreamingSqlRetrySettings() == null) { + changeStreamSpannerConfig = + changeStreamSpannerConfig + .toBuilder() + .setExecuteStreamingSqlRetrySettings( + RetrySettings.newBuilder() + .setTotalTimeout(org.threeten.bp.Duration.ofMinutes(5)) + .setInitialRpcTimeout(org.threeten.bp.Duration.ofMinutes(1)) + .setMaxRpcTimeout(org.threeten.bp.Duration.ofMinutes(1)) + .build()) + .build(); + } final SpannerConfig partitionMetadataSpannerConfig = - SpannerConfig.create() - .withProjectId(changeStreamSpannerConfig.getProjectId()) - .withHost(changeStreamSpannerConfig.getHost()) - .withInstanceId(partitionMetadataInstanceId) - .withDatabaseId(partitionMetadataDatabaseId) - .withCommitDeadline(changeStreamSpannerConfig.getCommitDeadline()) - .withEmulatorHost(changeStreamSpannerConfig.getEmulatorHost()) - .withMaxCumulativeBackoff(changeStreamSpannerConfig.getMaxCumulativeBackoff()); + changeStreamSpannerConfig + .toBuilder() + .setInstanceId(StaticValueProvider.of(partitionMetadataInstanceId)) + .setDatabaseId(StaticValueProvider.of(partitionMetadataDatabaseId)) + .build(); final String changeStreamName = getChangeStreamName(); // FIXME: The backend only supports microsecond granularity. Remove when fixed. final Timestamp startTimestamp = TimestampConverter.truncateNanos(getInclusiveStartAt()); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryTest.java index 86f538a4d0f3b..b23de6b3fc334 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryTest.java @@ -93,7 +93,10 @@ import org.junit.runners.JUnit4; import org.junit.runners.model.Statement; -/** Tests for {@link BigQueryIO#readTableRows()} using {@link Method#DIRECT_READ}. */ +/** + * Tests for {@link BigQueryIO#read(SerializableFunction)} using {@link Method#DIRECT_READ} and + * {@link BigQueryIO.TypedRead#fromQuery(String)}. + */ @RunWith(JUnit4.class) public class BigQueryIOStorageQueryTest { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java index fcf335c01b309..8f8ab867a9101 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java @@ -25,6 +25,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -92,20 +93,29 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.Method; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StorageClient; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.ConversionOptions; import org.apache.beam.sdk.io.gcp.testing.FakeBigQueryServices; import org.apache.beam.sdk.io.gcp.testing.FakeBigQueryServices.FakeBigQueryServerStream; import org.apache.beam.sdk.io.gcp.testing.FakeDatasetService; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; +import org.apache.beam.sdk.schemas.FieldAccessDescriptor; +import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; +import org.hamcrest.Matchers; import org.junit.After; import org.junit.Before; import org.junit.Rule; @@ -2064,6 +2074,56 @@ public void testStreamSourceSplitAtFractionFailsWhenParentIsPastSplitPointArrow( assertFalse(parent.advance()); } + @Test + public void testActuateProjectionPushdown() { + org.apache.beam.sdk.schemas.Schema schema = + org.apache.beam.sdk.schemas.Schema.builder() + .addStringField("foo") + .addStringField("bar") + .build(); + TypedRead read = + BigQueryIO.read( + record -> + BigQueryUtils.toBeamRow( + record.getRecord(), schema, ConversionOptions.builder().build())) + .withMethod(Method.DIRECT_READ) + .withCoder(SchemaCoder.of(schema)); + + assertTrue(read.supportsProjectionPushdown()); + PTransform> pushdownT = + read.actuateProjectionPushdown( + ImmutableMap.of(new TupleTag<>("output"), FieldAccessDescriptor.withFieldNames("foo"))); + + TypedRead pushdownRead = (TypedRead) pushdownT; + assertEquals(Method.DIRECT_READ, pushdownRead.getMethod()); + assertThat(pushdownRead.getSelectedFields().get(), Matchers.containsInAnyOrder("foo")); + } + + @Test + public void testReadFromQueryDoesNotSupportProjectionPushdown() { + org.apache.beam.sdk.schemas.Schema schema = + org.apache.beam.sdk.schemas.Schema.builder() + .addStringField("foo") + .addStringField("bar") + .build(); + TypedRead read = + BigQueryIO.read( + record -> + BigQueryUtils.toBeamRow( + record.getRecord(), schema, ConversionOptions.builder().build())) + .fromQuery("SELECT bar FROM `dataset.table`") + .withMethod(Method.DIRECT_READ) + .withCoder(SchemaCoder.of(schema)); + + assertFalse(read.supportsProjectionPushdown()); + assertThrows( + IllegalArgumentException.class, + () -> + read.actuateProjectionPushdown( + ImmutableMap.of( + new TupleTag<>("output"), FieldAccessDescriptor.withFieldNames("foo")))); + } + private static org.apache.arrow.vector.types.pojo.Field field( String name, boolean nullable, diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java new file mode 100644 index 0000000000000..34d7cd4a91a7d --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java @@ -0,0 +1,497 @@ +/* + * 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.io.gcp.spanner.changestreams; + +import static org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_CREATED_AT; +import static org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_END_TIMESTAMP; +import static org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_FINISHED_AT; +import static org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_HEARTBEAT_MILLIS; +import static org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_PARENT_TOKENS; +import static org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_PARTITION_TOKEN; +import static org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_RUNNING_AT; +import static org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_SCHEDULED_AT; +import static org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_START_TIMESTAMP; +import static org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_STATE; +import static org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_WATERMARK; +import static org.hamcrest.MatcherAssert.assertThat; + +import com.google.api.gax.grpc.testing.MockServiceHelper; +import com.google.api.gax.retrying.RetrySettings; +import com.google.cloud.Timestamp; +import com.google.cloud.spanner.ErrorCode; +import com.google.cloud.spanner.MockSpannerServiceImpl; +import com.google.cloud.spanner.MockSpannerServiceImpl.SimulatedExecutionTime; +import com.google.cloud.spanner.MockSpannerServiceImpl.StatementResult; +import com.google.cloud.spanner.Statement; +import com.google.protobuf.ListValue; +import com.google.protobuf.NullValue; +import com.google.protobuf.Value; +import com.google.spanner.v1.ExecuteSqlRequest; +import com.google.spanner.v1.ResultSet; +import com.google.spanner.v1.ResultSetMetadata; +import com.google.spanner.v1.StructType; +import com.google.spanner.v1.StructType.Field; +import com.google.spanner.v1.Type; +import com.google.spanner.v1.TypeCode; +import io.grpc.Status; +import java.io.Serializable; +import java.util.Collections; +import org.apache.beam.sdk.Pipeline.PipelineExecutionException; +import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; +import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.DaoFactory; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionMetadata.State; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; +import org.hamcrest.Matchers; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class SpannerChangeStreamErrorTest implements Serializable { + + public static final String SPANNER_HOST = "my-host"; + private static final String TEST_PROJECT = "my-project"; + private static final String TEST_INSTANCE = "my-instance"; + private static final String TEST_DATABASE = "my-database"; + private static final String TEST_TABLE = "my-metadata-table"; + private static final String TEST_CHANGE_STREAM = "my-change-stream"; + + @Rule + public final transient TestPipeline pipeline = + TestPipeline.create().enableAbandonedNodeEnforcement(false); + + @Rule public final transient ExpectedException thrown = ExpectedException.none(); + + private MockSpannerServiceImpl mockSpannerService; + private MockServiceHelper serviceHelper; + + @Before + public void setUp() throws Exception { + mockSpannerService = new MockSpannerServiceImpl(); + serviceHelper = + new MockServiceHelper(SPANNER_HOST, Collections.singletonList(mockSpannerService)); + serviceHelper.start(); + serviceHelper.reset(); + } + + @After + public void tearDown() throws NoSuchFieldException, IllegalAccessException { + serviceHelper.reset(); + serviceHelper.stop(); + mockSpannerService.reset(); + resetDaoFactoryFields(); + } + + @Test + public void testResourceExhaustedDoesNotRetry() { + mockSpannerService.setExecuteStreamingSqlExecutionTime( + SimulatedExecutionTime.ofStickyException(Status.RESOURCE_EXHAUSTED.asRuntimeException())); + + final Timestamp now = Timestamp.now(); + final Timestamp after3Seconds = + Timestamp.ofTimeSecondsAndNanos(now.getSeconds() + 3, now.getNanos()); + try { + pipeline.apply( + SpannerIO.readChangeStream() + .withSpannerConfig(getSpannerConfig()) + .withChangeStreamName(TEST_CHANGE_STREAM) + .withMetadataDatabase(TEST_DATABASE) + .withInclusiveStartAt(now) + .withInclusiveEndAt(after3Seconds)); + pipeline.run().waitUntilFinish(); + } finally { + thrown.expect(PipelineExecutionException.class); + thrown.expectMessage(ErrorCode.RESOURCE_EXHAUSTED.name()); + } + } + + @Test + public void testUnavailableExceptionRetries() { + mockSpannerService.setExecuteStreamingSqlExecutionTime( + SimulatedExecutionTime.ofExceptions( + ImmutableSet.of( + Status.UNAVAILABLE.asRuntimeException(), + Status.RESOURCE_EXHAUSTED.asRuntimeException()))); + + final Timestamp now = Timestamp.now(); + final Timestamp after3Seconds = + Timestamp.ofTimeSecondsAndNanos(now.getSeconds() + 3, now.getNanos()); + try { + pipeline.apply( + SpannerIO.readChangeStream() + .withSpannerConfig(getSpannerConfig()) + .withChangeStreamName(TEST_CHANGE_STREAM) + .withMetadataDatabase(TEST_DATABASE) + .withInclusiveStartAt(now) + .withInclusiveEndAt(after3Seconds)); + pipeline.run().waitUntilFinish(); + } finally { + assertThat( + mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.greaterThan(1)); + thrown.expect(PipelineExecutionException.class); + thrown.expectMessage(ErrorCode.RESOURCE_EXHAUSTED.name()); + } + } + + @Test + public void testAbortedExceptionRetries() { + mockSpannerService.setExecuteStreamingSqlExecutionTime( + SimulatedExecutionTime.ofStickyException(Status.ABORTED.asRuntimeException())); + + final Timestamp now = Timestamp.now(); + final Timestamp after3Seconds = + Timestamp.ofTimeSecondsAndNanos(now.getSeconds() + 3, now.getNanos()); + try { + pipeline.apply( + SpannerIO.readChangeStream() + .withSpannerConfig(getSpannerConfig()) + .withChangeStreamName(TEST_CHANGE_STREAM) + .withMetadataDatabase(TEST_DATABASE) + .withInclusiveStartAt(now) + .withInclusiveEndAt(after3Seconds)); + pipeline.run().waitUntilFinish(); + } finally { + assertThat( + mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.greaterThan(1)); + thrown.expect(PipelineExecutionException.class); + thrown.expectMessage(ErrorCode.ABORTED.name()); + } + } + + @Test + public void testUnknownExceptionDoesNotRetry() { + mockSpannerService.setExecuteStreamingSqlExecutionTime( + SimulatedExecutionTime.ofStickyException(Status.UNKNOWN.asRuntimeException())); + + final Timestamp now = Timestamp.now(); + final Timestamp after3Seconds = + Timestamp.ofTimeSecondsAndNanos(now.getSeconds() + 3, now.getNanos()); + try { + pipeline.apply( + SpannerIO.readChangeStream() + .withSpannerConfig(getSpannerConfig()) + .withChangeStreamName(TEST_CHANGE_STREAM) + .withMetadataDatabase(TEST_DATABASE) + .withInclusiveStartAt(now) + .withInclusiveEndAt(after3Seconds)); + pipeline.run().waitUntilFinish(); + } finally { + assertThat( + mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.equalTo(1)); + thrown.expect(PipelineExecutionException.class); + thrown.expectMessage(ErrorCode.UNKNOWN.name()); + } + } + + @Test + public void testInvalidRecordReceived() { + final Timestamp now = Timestamp.now(); + final Timestamp after3Seconds = + Timestamp.ofTimeSecondsAndNanos(now.getSeconds() + 3, now.getNanos()); + + mockTableExists(); + ResultSet getPartitionResultSet = mockGetParentPartition(now, after3Seconds); + mockGetWatermark(now); + mockGetPartitionsAfter( + Timestamp.ofTimeSecondsAndNanos(now.getSeconds(), now.getNanos() + 1000), + getPartitionResultSet); + mockGetPartitionsAfter( + Timestamp.ofTimeSecondsAndNanos(now.getSeconds(), now.getNanos() - 1000), + getPartitionResultSet); + mockInvalidChangeStreamRecordReceived(now, after3Seconds); + + try { + pipeline.apply( + SpannerIO.readChangeStream() + .withSpannerConfig(getSpannerConfig()) + .withChangeStreamName(TEST_CHANGE_STREAM) + .withMetadataDatabase(TEST_DATABASE) + .withMetadataTable(TEST_TABLE) + .withInclusiveStartAt(now) + .withInclusiveEndAt(after3Seconds)); + pipeline.run().waitUntilFinish(); + } finally { + thrown.expect(PipelineExecutionException.class); + thrown.expectMessage("Field not found"); + } + } + + private void mockInvalidChangeStreamRecordReceived(Timestamp now, Timestamp after3Seconds) { + Statement changeStreamQueryStatement = + Statement.newBuilder( + "SELECT * FROM READ_my-change-stream( start_timestamp => @startTimestamp, end_timestamp => @endTimestamp, partition_token => @partitionToken, read_options => null, heartbeat_milliseconds => @heartbeatMillis)") + .bind("startTimestamp") + .to(now) + .bind("endTimestamp") + .to(after3Seconds) + .bind("partitionToken") + .to((String) null) + .bind("heartbeatMillis") + .to(500) + .build(); + ResultSetMetadata readChangeStreamResultSetMetadata = + ResultSetMetadata.newBuilder() + .setRowType( + StructType.newBuilder() + .addFields( + Field.newBuilder() + .setName("COL1") + .setType( + Type.newBuilder() + .setCode(TypeCode.ARRAY) + .setArrayElementType( + Type.newBuilder() + .setCode(TypeCode.STRUCT) + .setStructType( + StructType.newBuilder() + .addFields( + Field.newBuilder() + .setName("field_name") + .setType( + Type.newBuilder() + .setCode(TypeCode.STRUCT) + .setStructType( + StructType.newBuilder() + .addFields( + Field.newBuilder() + .setType( + Type + .newBuilder() + .setCode( + TypeCode + .STRING))))))))))) + .build(); + ResultSet readChangeStreamResultSet = + ResultSet.newBuilder() + .addRows( + ListValue.newBuilder() + .addValues( + Value.newBuilder() + .setListValue( + ListValue.newBuilder() + .addValues( + Value.newBuilder() + .setListValue( + ListValue.newBuilder() + .addValues( + Value.newBuilder() + .setListValue( + ListValue.newBuilder() + .addValues( + Value.newBuilder() + .setStringValue( + "bad_value"))))))))) + .setMetadata(readChangeStreamResultSetMetadata) + .build(); + mockSpannerService.putStatementResult( + StatementResult.query(changeStreamQueryStatement, readChangeStreamResultSet)); + } + + private void mockGetPartitionsAfter(Timestamp timestamp, ResultSet getPartitionResultSet) { + Statement getPartitionsAfterStatement = + Statement.newBuilder( + "SELECT * FROM my-metadata-table WHERE CreatedAt > @timestamp ORDER BY CreatedAt ASC, StartTimestamp ASC") + .bind("timestamp") + .to(Timestamp.ofTimeSecondsAndNanos(timestamp.getSeconds(), timestamp.getNanos())) + .build(); + mockSpannerService.putStatementResult( + StatementResult.query(getPartitionsAfterStatement, getPartitionResultSet)); + } + + private void mockGetWatermark(Timestamp now) { + Statement watermarkStatement = + Statement.newBuilder( + "SELECT Watermark FROM my-metadata-table WHERE State != @state ORDER BY Watermark ASC LIMIT 1") + .bind("state") + .to(State.FINISHED.name()) + .build(); + ResultSetMetadata watermarkResultSetMetadata = + ResultSetMetadata.newBuilder() + .setRowType( + StructType.newBuilder() + .addFields( + Field.newBuilder() + .setName("Watermark") + .setType(Type.newBuilder().setCode(TypeCode.TIMESTAMP).build()) + .build()) + .build()) + .build(); + ResultSet watermarkResultSet = + ResultSet.newBuilder() + .addRows( + ListValue.newBuilder() + .addValues(Value.newBuilder().setStringValue(now.toString()).build()) + .build()) + .setMetadata(watermarkResultSetMetadata) + .build(); + mockSpannerService.putStatementResult( + StatementResult.query(watermarkStatement, watermarkResultSet)); + } + + private ResultSet mockGetParentPartition(Timestamp now, Timestamp after3Seconds) { + Statement getPartitionStatement = + Statement.newBuilder("SELECT * FROM my-metadata-table WHERE PartitionToken = @partition") + .bind("partition") + .to("Parent0") + .build(); + ResultSet getPartitionResultSet = + ResultSet.newBuilder() + .addRows( + ListValue.newBuilder() + .addValues(Value.newBuilder().setStringValue("Parent0")) + .addValues(Value.newBuilder().setListValue(ListValue.newBuilder().build())) + .addValues(Value.newBuilder().setStringValue(now.toString())) + .addValues(Value.newBuilder().setStringValue(after3Seconds.toString())) + .addValues(Value.newBuilder().setStringValue("500")) + .addValues(Value.newBuilder().setStringValue(State.CREATED.name())) + .addValues(Value.newBuilder().setStringValue(now.toString())) + .addValues(Value.newBuilder().setStringValue(now.toString())) + .addValues(Value.newBuilder().setNullValue(NullValue.NULL_VALUE).build()) + .addValues(Value.newBuilder().setNullValue(NullValue.NULL_VALUE).build()) + .addValues(Value.newBuilder().setNullValue(NullValue.NULL_VALUE).build()) + .build()) + .setMetadata(PARTITION_METADATA_RESULT_SET_METADATA) + .build(); + mockSpannerService.putStatementResult( + StatementResult.query(getPartitionStatement, getPartitionResultSet)); + return getPartitionResultSet; + } + + private void mockTableExists() { + Statement tableExistsStatement = + Statement.of( + "SELECT t.table_name FROM information_schema.tables AS t WHERE t.table_catalog = '' AND t.table_schema = '' AND t.table_name = 'my-metadata-table'"); + ResultSetMetadata tableExistsResultSetMetadata = + ResultSetMetadata.newBuilder() + .setRowType( + StructType.newBuilder() + .addFields( + Field.newBuilder() + .setName("table_name") + .setType(Type.newBuilder().setCode(TypeCode.STRING).build()) + .build()) + .build()) + .build(); + ResultSet tableExistsResultSet = + ResultSet.newBuilder() + .addRows( + ListValue.newBuilder() + .addValues(Value.newBuilder().setStringValue(TEST_TABLE).build()) + .build()) + .setMetadata(tableExistsResultSetMetadata) + .build(); + mockSpannerService.putStatementResult( + StatementResult.query(tableExistsStatement, tableExistsResultSet)); + } + + private SpannerConfig getSpannerConfig() { + RetrySettings quickRetrySettings = + RetrySettings.newBuilder() + .setInitialRetryDelay(org.threeten.bp.Duration.ofMillis(250)) + .setMaxRetryDelay(org.threeten.bp.Duration.ofSeconds(1)) + .setRetryDelayMultiplier(5) + .setTotalTimeout(org.threeten.bp.Duration.ofSeconds(1)) + .build(); + return SpannerConfig.create() + .withEmulatorHost(StaticValueProvider.of(SPANNER_HOST)) + .withIsLocalChannelProvider(StaticValueProvider.of(true)) + .withCommitRetrySettings(quickRetrySettings) + .withExecuteStreamingSqlRetrySettings(quickRetrySettings) + .withProjectId(TEST_PROJECT) + .withInstanceId(TEST_INSTANCE) + .withDatabaseId(TEST_DATABASE); + } + + private static void resetDaoFactoryFields() throws NoSuchFieldException, IllegalAccessException { + java.lang.reflect.Field partitionMetadataAdminDaoField = + DaoFactory.class.getDeclaredField("partitionMetadataAdminDao"); + partitionMetadataAdminDaoField.setAccessible(true); + partitionMetadataAdminDaoField.set(null, null); + java.lang.reflect.Field partitionMetadataDaoInstanceField = + DaoFactory.class.getDeclaredField("partitionMetadataDaoInstance"); + partitionMetadataDaoInstanceField.setAccessible(true); + partitionMetadataDaoInstanceField.set(null, null); + java.lang.reflect.Field changeStreamDaoInstanceField = + DaoFactory.class.getDeclaredField("changeStreamDaoInstance"); + changeStreamDaoInstanceField.setAccessible(true); + changeStreamDaoInstanceField.set(null, null); + } + + private static final ResultSetMetadata PARTITION_METADATA_RESULT_SET_METADATA = + ResultSetMetadata.newBuilder() + .setRowType( + StructType.newBuilder() + .addFields( + Field.newBuilder() + .setName(COLUMN_PARTITION_TOKEN) + .setType(Type.newBuilder().setCode(TypeCode.STRING)) + .build()) + .addFields( + Field.newBuilder() + .setName(COLUMN_PARENT_TOKENS) + .setType( + Type.newBuilder() + .setCode(TypeCode.ARRAY) + .setArrayElementType(Type.newBuilder().setCode(TypeCode.STRING))) + .build()) + .addFields( + Field.newBuilder() + .setName(COLUMN_START_TIMESTAMP) + .setType(Type.newBuilder().setCode(TypeCode.TIMESTAMP))) + .addFields( + Field.newBuilder() + .setName(COLUMN_END_TIMESTAMP) + .setType(Type.newBuilder().setCode(TypeCode.TIMESTAMP))) + .addFields( + Field.newBuilder() + .setName(COLUMN_HEARTBEAT_MILLIS) + .setType(Type.newBuilder().setCode(TypeCode.INT64))) + .addFields( + Field.newBuilder() + .setName(COLUMN_STATE) + .setType(Type.newBuilder().setCode(TypeCode.STRING))) + .addFields( + Field.newBuilder() + .setName(COLUMN_WATERMARK) + .setType(Type.newBuilder().setCode(TypeCode.TIMESTAMP))) + .addFields( + Field.newBuilder() + .setName(COLUMN_CREATED_AT) + .setType(Type.newBuilder().setCode(TypeCode.TIMESTAMP))) + .addFields( + Field.newBuilder() + .setName(COLUMN_SCHEDULED_AT) + .setType(Type.newBuilder().setCode(TypeCode.TIMESTAMP))) + .addFields( + Field.newBuilder() + .setName(COLUMN_RUNNING_AT) + .setType(Type.newBuilder().setCode(TypeCode.TIMESTAMP))) + .addFields( + Field.newBuilder() + .setName(COLUMN_FINISHED_AT) + .setType(Type.newBuilder().setCode(TypeCode.TIMESTAMP))) + .build()) + .build(); +} diff --git a/sdks/java/io/neo4j/OWNERS b/sdks/java/io/neo4j/OWNERS new file mode 100644 index 0000000000000..0ff6e82359fb6 --- /dev/null +++ b/sdks/java/io/neo4j/OWNERS @@ -0,0 +1,5 @@ +# See the OWNERS docs at https://s.apache.org/beam-owners + +reviewers: + - mcasters + diff --git a/sdks/java/io/neo4j/build.gradle b/sdks/java/io/neo4j/build.gradle new file mode 100644 index 0000000000000..9d5adfc32b1d0 --- /dev/null +++ b/sdks/java/io/neo4j/build.gradle @@ -0,0 +1,39 @@ +/* + * 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. + */ + +plugins { id 'org.apache.beam.module' } +applyJavaNature(automaticModuleName: 'org.apache.beam.sdk.io.neo4j') +provideIntegrationTestingDependencies() +enableJavaPerformanceTesting() + +description = "Apache Beam :: SDKs :: Java :: IO :: Neo4j" +ext.summary = "IO to read from and write to Neo4j graphs" + +dependencies { + implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation "org.neo4j.driver:neo4j-java-driver:4.4.3" + implementation library.java.slf4j_api + implementation library.java.vendored_guava_26_0_jre + testImplementation library.java.junit + testImplementation library.java.hamcrest + testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") + testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration") + testImplementation "org.testcontainers:neo4j:1.16.2" + testRuntimeOnly library.java.slf4j_jdk14 + testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") +} diff --git a/sdks/java/io/neo4j/src/main/java/org/apache/beam/sdk/io/neo4j/Neo4jIO.java b/sdks/java/io/neo4j/src/main/java/org/apache/beam/sdk/io/neo4j/Neo4jIO.java new file mode 100644 index 0000000000000..9b011aff410d2 --- /dev/null +++ b/sdks/java/io/neo4j/src/main/java/org/apache/beam/sdk/io/neo4j/Neo4jIO.java @@ -0,0 +1,1221 @@ +/* + * 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.io.neo4j; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.value.AutoValue; +import java.io.Serializable; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.beam.repackaged.core.org.apache.commons.lang3.StringUtils; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.harness.JvmInitializer; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaRegistry; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.display.HasDisplayData; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.neo4j.driver.AuthToken; +import org.neo4j.driver.AuthTokens; +import org.neo4j.driver.Config; +import org.neo4j.driver.Driver; +import org.neo4j.driver.GraphDatabase; +import org.neo4j.driver.Record; +import org.neo4j.driver.Result; +import org.neo4j.driver.Session; +import org.neo4j.driver.SessionConfig; +import org.neo4j.driver.TransactionConfig; +import org.neo4j.driver.TransactionWork; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is a Beam IO to read from, and write data to, Neo4j. + * + *

+ * + *

+ * + *

Driver configuration

+ * + *

To read from or write to Neo4j you have to provide a {@link DriverConfiguration} using
+ * {@link DriverConfiguration#create()} or {@link DriverConfiguration#create(String, String, + * String)} (URL, username and password). Note that subclasses of DriverConfiguration must also be + * {@link Serializable}).
+ * At the level of the Neo4j driver configuration you can specify a Neo4j {@link Config} object with + * {@link DriverConfiguration#withConfig(Config)}. This way you can configure the Neo4j driver + * characteristics. Likewise, you can control the characteristics of Neo4j sessions by optionally + * passing a {@link SessionConfig} object to {@link ReadAll} or {@link WriteUnwind}. For example, + * the session configuration will allow you to target a specific database or set a fetch size. + * Finally, in even rarer cases you might need to configure the various aspects of Neo4j + * transactions, for example their timeout. You can do this with a Neo4j {@link TransactionConfig} + * object. + * + *

+ * + *

+ * + *

Neo4j Aura

+ * + *

If you have trouble connecting to a Neo4j Aura database please try to disable a few security + * algorithms in your JVM. This makes sure that the right one is picked to connect: + * + *

+ * + *

{@code
+ * Security.setProperty(
+ *       "jdk.tls.disabledAlgorithms",
+ *       "SSLv3, RC4, DES, MD5withRSA, DH keySize < 1024, EC keySize < 224, 3DES_EDE_CBC, anon, NULL");
+ * }
+ * + *

+ * + *

+ * + *

To execute this code on GCP Dataflow you can create a class which extends {@link + * JvmInitializer} and implement the {@link JvmInitializer#onStartup()} method. You need to annotate + * this new class with {@link com.google.auto.service.AutoService} + * + *

{@code
+ * @AutoService(value = JvmInitializer.class)
+ * }
+ * + *

+ * + *

Reading from Neo4j

+ * + *

{@link Neo4jIO#readAll()} source returns a bounded collection of {@code OuptutT} as a {@code + * PCollection}. OutputT is the type returned by the provided {@link RowMapper}. It accepts + * parameters as input in the form of {@code ParameterT} as a {@code PCollection} + * + *

The following example reads ages to return the IDs of Person nodes. It runs a Cypher query for + * each provided age. + * + *

The mapping {@link SerializableFunction} maps input values to each execution of the Cypher + * statement. In the function simply return a map containing the parameters you want to set. + * + *

The {@link RowMapper} converts output Neo4j {@link Record} values to the output of the source. + * + *

{@code
+ * pipeline
+ *   .apply(Create.of(40, 50, 60))
+ *   .apply(Neo4jIO.readAll()
+ *     .withDriverConfiguration(Neo4jIO.DriverConfiguration.create("neo4j://localhost:7687", "neo4j", "password"))
+ *     .withCypher("MATCH(n:Person) WHERE n.age = $age RETURN n.id")
+ *     .withReadTransaction()
+ *     .withCoder(StringUtf8Coder.of())
+ *     .withParametersFunction( age -> Collections.singletonMap( "age", age ))
+ *     .withRowMapper( record -> return record.get(0).asString() )
+ *   );
+ * }
+ * + *

Writing to Neo4j

+ * + *

The Neo4j {@link WriteUnwind} transform supports writing data to a graph. It writes a {@link + * PCollection} to the graph by collecting a batch of elements after which all elements in the batch + * are written together to Neo4j. + * + *

Like the source, to configure this sink, you have to provide a {@link DriverConfiguration}. + * + *

In the following example we'll merge a collection of {@link org.apache.beam.sdk.values.Row} + * into Person nodes. Since this is a Sink it has no output and as such no RowMapper is needed. The + * rows are being used as a container for the parameters of the Cypher statement. The used Cypher in + * question needs to be an UNWIND statement. Like in the read case, the parameters {@link + * SerializableFunction} converts parameter values to a {@link Map}. The difference here is that the + * resulting Map is stored in a {@link List} (containing maps) which in turn is stored in another + * Map under the name provided by the {@link WriteUnwind#withUnwindMapName(String)} method. All of + * this is handled automatically. You do need to provide the unwind map name so that you can + * reference that in the UNWIND statement. + * + *

+ * + *

For example: + * + *

{@code
+ * pipeline
+ *   .apply(...)
+ *   .apply(Neo4jIO.writeUnwind()
+ *      .withDriverConfiguration(Neo4jIO.DriverConfiguration.create("neo4j://localhost:7687", "neo4j", "password"))
+ *      .withUnwindMapName("rows")
+ *      .withCypher("UNWIND $rows AS row MERGE(n:Person { id : row.id } ) SET n.firstName = row.first, n.lastName = row.last")
+ *      .withParametersFunction( row -> ImmutableMap.of(
+ *        "id", row.getString("id),
+ *        "first", row.getString("firstName")
+ *        "last", row.getString("lastName")))
+ *    );
+ * }
+ */ +@Experimental(Experimental.Kind.SOURCE_SINK) +public class Neo4jIO { + + private static final Logger LOG = LoggerFactory.getLogger(Neo4jIO.class); + + /** + * Read all rows using a Neo4j Cypher query. + * + * @param Type of the data representing query parameters. + * @param Type of the data to be read. + */ + public static ReadAll readAll() { + return new AutoValue_Neo4jIO_ReadAll.Builder().build(); + } + + /** + * Write all rows using a Neo4j Cypher UNWIND cypher statement. This sets a default batch size of + * 5000. + * + * @param Type of the data representing query parameters. + */ + public static WriteUnwind writeUnwind() { + return new AutoValue_Neo4jIO_WriteUnwind.Builder() + .setBatchSize(ValueProvider.StaticValueProvider.of(5000L)) + .build(); + } + + private static PCollection getOutputPCollection( + PCollection input, + DoFn writeFn, + @Nullable Coder coder) { + PCollection output = input.apply(ParDo.of(writeFn)); + if (coder != null) { + output.setCoder(coder); + try { + TypeDescriptor typeDesc = coder.getEncodedTypeDescriptor(); + SchemaRegistry registry = input.getPipeline().getSchemaRegistry(); + Schema schema = registry.getSchema(typeDesc); + output.setSchema( + schema, + typeDesc, + registry.getToRowFunction(typeDesc), + registry.getFromRowFunction(typeDesc)); + } catch (NoSuchSchemaException e) { + // ignore + } + } + return output; + } + + /** + * An interface used by {@link ReadAll} for converting each row of a Neo4j {@link Result} record + * {@link Record} into an element of the resulting {@link PCollection}. + */ + @FunctionalInterface + public interface RowMapper extends Serializable { + T mapRow(Record record) throws Exception; + } + + /** + * A convenience method to clarify the way {@link ValueProvider} works to the static code checker + * framework for {@link Nullable} values. + * + * @param valueProvider + * @param + * @return The provided value or null if none was specified. + */ + private static T getProvidedValue(@Nullable ValueProvider valueProvider) { + if (valueProvider == null) { + return (T) null; + } + return valueProvider.get(); + } + + /** This describes all the information needed to create a Neo4j {@link Session}. */ + @AutoValue + public abstract static class DriverConfiguration implements Serializable { + public static DriverConfiguration create() { + return new AutoValue_Neo4jIO_DriverConfiguration.Builder() + .build() + .withDefaultConfig(true) + .withConfig(Config.defaultConfig()); + } + + public static DriverConfiguration create(String url, String username, String password) { + checkArgument(url != null, "url can not be null"); + checkArgument(username != null, "username can not be null"); + checkArgument(password != null, "password can not be null"); + return new AutoValue_Neo4jIO_DriverConfiguration.Builder() + .build() + .withDefaultConfig(true) + .withConfig(Config.defaultConfig()) + .withUrl(url) + .withUsername(username) + .withPassword(password); + } + + abstract @Nullable ValueProvider getUrl(); + + abstract @Nullable ValueProvider> getUrls(); + + abstract @Nullable ValueProvider getUsername(); + + abstract @Nullable ValueProvider getPassword(); + + abstract @Nullable Config getConfig(); + + abstract @Nullable ValueProvider getHasDefaultConfig(); + + abstract Builder builder(); + + public DriverConfiguration withUrl(String url) { + return withUrl(ValueProvider.StaticValueProvider.of(url)); + } + + public DriverConfiguration withUrl(ValueProvider url) { + Preconditions.checkArgument( + url != null, "a neo4j connection URL can not be empty or null", url); + Preconditions.checkArgument( + StringUtils.isNotEmpty(url.get()), + "a neo4j connection URL can not be empty or null", + url); + return builder().setUrl(url).build(); + } + + public DriverConfiguration withUrls(List urls) { + return withUrls(ValueProvider.StaticValueProvider.of(urls)); + } + + public DriverConfiguration withUrls(ValueProvider> urls) { + Preconditions.checkArgument( + urls != null, "a list of neo4j connection URLs can not be empty or null", urls); + Preconditions.checkArgument( + urls.get() != null && !urls.get().isEmpty(), + "a neo4j connection URL can not be empty or null", + urls); + return builder().setUrls(urls).build(); + } + + public DriverConfiguration withConfig(Config config) { + return builder().setConfig(config).build(); + } + + public DriverConfiguration withUsername(String username) { + return withUsername(ValueProvider.StaticValueProvider.of(username)); + } + + public DriverConfiguration withUsername(ValueProvider username) { + Preconditions.checkArgument(username != null, "neo4j username can not be null", username); + Preconditions.checkArgument( + username.get() != null, "neo4j username can not be null", username); + return builder().setUsername(username).build(); + } + + public DriverConfiguration withPassword(String password) { + return withPassword(ValueProvider.StaticValueProvider.of(password)); + } + + public DriverConfiguration withPassword(ValueProvider password) { + Preconditions.checkArgument(password != null, "neo4j password can not be null", password); + Preconditions.checkArgument( + password.get() != null, "neo4j password can not be null", password); + return builder().setPassword(password).build(); + } + + public DriverConfiguration withDefaultConfig(boolean useDefault) { + return withDefaultConfig(ValueProvider.StaticValueProvider.of(useDefault)); + } + + public DriverConfiguration withDefaultConfig(ValueProvider useDefault) { + Preconditions.checkArgument( + useDefault != null, "withDefaultConfig parameter useDefault can not be null", useDefault); + Preconditions.checkArgument( + useDefault.get() != null, + "withDefaultConfig parameter useDefault can not be null", + useDefault); + return builder().setHasDefaultConfig(useDefault).build(); + } + + void populateDisplayData(DisplayData.Builder builder) { + builder.addIfNotNull(DisplayData.item("neo4j-url", getUrl())); + builder.addIfNotNull(DisplayData.item("neo4j-username", getUsername())); + builder.addIfNotNull( + DisplayData.item( + "neo4j-password", getPassword() != null ? "" : "")); + } + + Driver buildDriver() { + // Create the Neo4j Driver + // This uses the provided Neo4j configuration along with URLs, username and password + // + Config config = getConfig(); + if (config == null) { + throw new RuntimeException("please provide a neo4j config"); + } + // We're trying to work around a subtle serialisation bug in the Neo4j Java driver. + // The fix is work in progress. For now, we harden our code to avoid + // wild goose chases. + // + Boolean hasDefaultConfig = getProvidedValue(getHasDefaultConfig()); + if (hasDefaultConfig != null && hasDefaultConfig) { + config = Config.defaultConfig(); + } + + // Get the list of the URI to connect with + // + List uris = new ArrayList<>(); + String url = getProvidedValue(getUrl()); + if (url != null) { + try { + uris.add(new URI(url)); + } catch (URISyntaxException e) { + throw new RuntimeException("Error creating URI from URL '" + url + "'", e); + } + } + List providedUrls = getProvidedValue(getUrls()); + if (providedUrls != null) { + for (String providedUrl : providedUrls) { + try { + uris.add(new URI(providedUrl)); + } catch (URISyntaxException e) { + throw new RuntimeException( + "Error creating URI '" + + providedUrl + + "' from a list of " + + providedUrls.size() + + " URLs", + e); + } + } + } + + // A specific routing driver can be used to connect to specific clustered configurations. + // Often we don't need it because the Java driver automatically can figure this out + // automatically. To keep things simple we use the routing driver in case we have more + // than one URL specified. This is an exceptional case. + // + Driver driver; + AuthToken authTokens = + getAuthToken(getProvidedValue(getUsername()), getProvidedValue(getPassword())); + if (uris.size() > 1) { + driver = GraphDatabase.routingDriver(uris, authTokens, config); + } else { + // Just take the first URI that was provided + driver = GraphDatabase.driver(uris.get(0), authTokens, config); + } + + return driver; + } + + /** + * Certain embedded scenarios and so on actually allow for having no authentication at all. + * + * @param username The username if one is needed + * @param password The password if one is needed + * @return The AuthToken + */ + protected AuthToken getAuthToken(String username, String password) { + if (username != null && password != null) { + return AuthTokens.basic(username, password); + } else { + return AuthTokens.none(); + } + } + + /** + * The Builder class below is not visible. We use it to service the "with" methods below the + * Builder class. + */ + @AutoValue.Builder + abstract static class Builder { + abstract Builder setUrl(ValueProvider url); + + abstract Builder setUrls(ValueProvider> url); + + abstract Builder setUsername(ValueProvider username); + + abstract Builder setPassword(ValueProvider password); + + abstract Builder setConfig(Config config); + + abstract Builder setHasDefaultConfig(ValueProvider useDefault); + + abstract DriverConfiguration build(); + } + } + + /** This is the class which handles the work behind the {@link #readAll} method. */ + @AutoValue + public abstract static class ReadAll + extends PTransform, PCollection> { + + abstract @Nullable SerializableFunction getDriverProviderFn(); + + abstract @Nullable SessionConfig getSessionConfig(); + + abstract @Nullable TransactionConfig getTransactionConfig(); + + abstract @Nullable ValueProvider getCypher(); + + abstract @Nullable ValueProvider getWriteTransaction(); + + abstract @Nullable RowMapper getRowMapper(); + + abstract @Nullable SerializableFunction> + getParametersFunction(); + + abstract @Nullable Coder getCoder(); + + abstract @Nullable ValueProvider getLogCypher(); + + abstract Builder toBuilder(); + + public ReadAll withDriverConfiguration(DriverConfiguration config) { + return toBuilder() + .setDriverProviderFn(new DriverProviderFromDriverConfiguration(config)) + .build(); + } + + public ReadAll withCypher(String cypher) { + checkArgument( + cypher != null, "Neo4jIO.readAll().withCypher(query) called with null cypher query"); + return withCypher(ValueProvider.StaticValueProvider.of(cypher)); + } + + public ReadAll withCypher(ValueProvider cypher) { + checkArgument(cypher != null, "Neo4jIO.readAll().withCypher(cypher) called with null cypher"); + return toBuilder().setCypher(cypher).build(); + } + + public ReadAll withSessionConfig(SessionConfig sessionConfig) { + checkArgument( + sessionConfig != null, + "Neo4jIO.readAll().withSessionConfig(sessionConfig) called with null sessionConfig"); + return toBuilder().setSessionConfig(sessionConfig).build(); + } + + public ReadAll withTransactionConfig(TransactionConfig transactionConfig) { + checkArgument( + transactionConfig != null, + "Neo4jIO.readAll().withTransactionConfig(transactionConfig) called with null transactionConfig"); + return toBuilder().setTransactionConfig(transactionConfig).build(); + } + + public ReadAll withRowMapper(RowMapper rowMapper) { + checkArgument( + rowMapper != null, + "Neo4jIO.readAll().withRowMapper(rowMapper) called with null rowMapper"); + return toBuilder().setRowMapper(rowMapper).build(); + } + + public ReadAll withParametersFunction( + SerializableFunction> parametersFunction) { + checkArgument( + parametersFunction != null, + "Neo4jIO.readAll().withParametersFunction(parametersFunction) called with null parametersFunction"); + return toBuilder().setParametersFunction(parametersFunction).build(); + } + + public ReadAll withCoder(Coder coder) { + checkArgument(coder != null, "Neo4jIO.readAll().withCoder(coder) called with null coder"); + return toBuilder().setCoder(coder).build(); + } + + public ReadAll withReadTransaction() { + return toBuilder() + .setWriteTransaction(ValueProvider.StaticValueProvider.of(Boolean.FALSE)) + .build(); + } + + public ReadAll withWriteTransaction() { + return toBuilder() + .setWriteTransaction(ValueProvider.StaticValueProvider.of(Boolean.TRUE)) + .build(); + } + + public ReadAll withCypherLogging() { + return toBuilder().setLogCypher(ValueProvider.StaticValueProvider.of(Boolean.TRUE)).build(); + } + + @Override + public PCollection expand(PCollection input) { + + final SerializableFunction driverProviderFn = getDriverProviderFn(); + final RowMapper rowMapper = getRowMapper(); + SerializableFunction> parametersFunction = + getParametersFunction(); + + final String cypher = getProvidedValue(getCypher()); + checkArgument(cypher != null, "please provide a cypher statement to execute"); + + SessionConfig sessionConfig = getSessionConfig(); + if (sessionConfig == null) { + // Create a default session configuration as recommended by Neo4j + // + sessionConfig = SessionConfig.defaultConfig(); + } + + TransactionConfig transactionConfig = getTransactionConfig(); + if (transactionConfig == null) { + transactionConfig = TransactionConfig.empty(); + } + + Boolean writeTransaction = getProvidedValue(getWriteTransaction()); + if (writeTransaction == null) { + writeTransaction = Boolean.FALSE; + } + + Boolean logCypher = getProvidedValue(getLogCypher()); + if (logCypher == null) { + logCypher = Boolean.FALSE; + } + + if (driverProviderFn == null) { + throw new RuntimeException("please provide a driver provider"); + } + if (rowMapper == null) { + throw new RuntimeException("please provide a row mapper"); + } + if (parametersFunction == null) { + parametersFunction = t -> Collections.emptyMap(); + } + + ReadFn readFn = + new ReadFn<>( + driverProviderFn, + sessionConfig, + transactionConfig, + cypher, + rowMapper, + parametersFunction, + writeTransaction, + logCypher); + + return getOutputPCollection(input, readFn, getCoder()); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + String cypher = getProvidedValue(getCypher()); + if (cypher == null) { + cypher = ""; + } + builder.add(DisplayData.item("cypher", cypher)); + SerializableFunction driverProviderFn = getDriverProviderFn(); + if (driverProviderFn != null) { + if (driverProviderFn instanceof HasDisplayData) { + ((HasDisplayData) driverProviderFn).populateDisplayData(builder); + } + } + } + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setDriverProviderFn( + SerializableFunction driverProviderFn); + + abstract Builder setCypher(ValueProvider cypher); + + abstract Builder setSessionConfig(SessionConfig sessionConfig); + + abstract Builder setTransactionConfig( + TransactionConfig transactionConfig); + + abstract Builder setWriteTransaction( + ValueProvider writeTransaction); + + abstract Builder setRowMapper(RowMapper rowMapper); + + abstract Builder setParametersFunction( + SerializableFunction> parametersFunction); + + abstract Builder setCoder(Coder coder); + + abstract Builder setLogCypher(ValueProvider logCypher); + + abstract ReadAll build(); + } + } + + /** A {@link DoFn} to execute a Cypher query to read from Neo4j. */ + private static class ReadWriteFn extends DoFn { + protected static class DriverSession { + public @Nullable Driver driver; + public @Nullable Session session; + public @NonNull AtomicBoolean closed; + + protected DriverSession(Driver driver, Session session) { + this.driver = driver; + this.session = session; + this.closed = new AtomicBoolean(false); + } + + private DriverSession() { + this.driver = null; + this.session = null; + this.closed = new AtomicBoolean(true); + } + + protected static @NonNull DriverSession emptyClosed() { + return new DriverSession(); + } + } + + protected final @NonNull SerializableFunction driverProviderFn; + protected final @NonNull SessionConfig sessionConfig; + protected final @NonNull TransactionConfig transactionConfig; + + protected transient @NonNull DriverSession driverSession; + + protected ReadWriteFn( + @NonNull SerializableFunction driverProviderFn, + @NonNull SessionConfig sessionConfig, + @NonNull TransactionConfig transactionConfig) { + this.driverProviderFn = driverProviderFn; + this.sessionConfig = sessionConfig; + this.transactionConfig = transactionConfig; + this.driverSession = DriverSession.emptyClosed(); + } + + /** + * Delay the creation of driver and session until we actually have data to do something with. + */ + @Setup + public void setup() {} + + protected @NonNull Driver createDriver() { + Driver driver = driverProviderFn.apply(null); + if (driver == null) { + throw new RuntimeException("null driver given by driver provider"); + } + return driver; + } + + protected @Initialized @NonNull DriverSession buildDriverSession() { + @NonNull Driver driver = createDriver(); + @NonNull Session session = driver.session(sessionConfig); + return new DriverSession(driver, session); + } + + @StartBundle + public void startBundle() { + if (driverSession == null || driverSession.closed.get()) { + driverSession = buildDriverSession(); + } + } + + @FinishBundle + public void finishBundle() { + cleanUpDriverSession(); + } + + @Override + protected void finalize() { + cleanUpDriverSession(); + } + + protected void cleanUpDriverSession() { + if (!driverSession.closed.get()) { + try { + if (driverSession.session != null) { + driverSession.session.close(); + } + if (driverSession.driver != null) { + driverSession.driver.close(); + } + } finally { + driverSession.closed.set(true); + } + } + } + + protected String getParametersString(Map parametersMap) { + StringBuilder parametersString = new StringBuilder(); + parametersMap + .keySet() + .forEach( + key -> { + if (parametersString.length() > 0) { + parametersString.append(','); + } + parametersString.append(key).append('='); + Object value = parametersMap.get(key); + if (value == null) { + parametersString.append(""); + } else { + parametersString.append(value); + } + }); + return parametersString.toString(); + } + } + + /** A {@link DoFn} to execute a Cypher query to read from Neo4j. */ + private static class ReadFn extends ReadWriteFn { + protected final @NonNull String cypher; + protected final @NonNull RowMapper rowMapper; + protected final @Nullable SerializableFunction> + parametersFunction; + + private final boolean writeTransaction; + private final boolean logCypher; + + private ReadFn( + @NonNull SerializableFunction driverProviderFn, + @NonNull SessionConfig sessionConfig, + @NonNull TransactionConfig transactionConfig, + @NonNull String cypher, + @NonNull RowMapper rowMapper, + @Nullable SerializableFunction> parametersFunction, + boolean writeTransaction, + boolean logCypher) { + super(driverProviderFn, sessionConfig, transactionConfig); + this.cypher = cypher; + this.rowMapper = rowMapper; + this.parametersFunction = parametersFunction; + this.writeTransaction = writeTransaction; + this.logCypher = logCypher; + } + + @ProcessElement + public void processElement(ProcessContext context) { + // Map the input data to the parameters map... + // + ParameterT parameters = context.element(); + final Map parametersMap; + if (parametersFunction != null) { + parametersMap = parametersFunction.apply(parameters); + } else { + parametersMap = Collections.emptyMap(); + } + executeReadCypherStatement(context, parametersMap); + } + + private void executeReadCypherStatement( + final ProcessContext processContext, Map parametersMap) { + // The actual "reading" work needs to happen in a transaction. + // We could actually read and write here depending on the type of transaction + // we picked. As long as the Cypher statement returns values it's fine. + // + TransactionWork transactionWork = + transaction -> { + Result result = transaction.run(cypher, parametersMap); + while (result.hasNext()) { + Record record = result.next(); + try { + OutputT outputT = rowMapper.mapRow(record); + processContext.output(outputT); + } catch (Exception e) { + throw new RuntimeException("error mapping Neo4j record to row", e); + } + } + + // We deliver no specific Neo4j transaction output beyond what goes to the context + // output + // + return null; + }; + + if (logCypher) { + String parametersString = getParametersString(parametersMap); + + String readWrite = writeTransaction ? "write" : "read"; + LOG.info( + "Starting a " + + readWrite + + " transaction for cypher: " + + cypher + + ", parameters: " + + parametersString); + } + + // There are 2 ways to do a transaction on Neo4j: read or write + // It's important that the right type is selected, especially in clustered configurations. + // + if (driverSession.session == null) { + throw new RuntimeException("neo4j session was not initialized correctly"); + } else { + if (writeTransaction) { + driverSession.session.writeTransaction(transactionWork, transactionConfig); + } else { + driverSession.session.readTransaction(transactionWork, transactionConfig); + } + } + } + } + + /** + * Wraps a {@link DriverConfiguration} to provide a {@link Driver}. + * + *

At most a single {@link Driver} instance will be constructed during pipeline execution for + * each unique {@link DriverConfiguration} within the pipeline. + */ + public static class DriverProviderFromDriverConfiguration + implements SerializableFunction, HasDisplayData { + private final DriverConfiguration config; + + private DriverProviderFromDriverConfiguration(DriverConfiguration config) { + this.config = config; + } + + public static SerializableFunction of(DriverConfiguration config) { + return new DriverProviderFromDriverConfiguration(config); + } + + @Override + public Driver apply(Void input) { + return config.buildDriver(); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + config.populateDisplayData(builder); + } + } + + /** This is the class which handles the work behind the {@link #writeUnwind()} method. */ + @AutoValue + public abstract static class WriteUnwind + extends PTransform, PDone> { + + abstract @Nullable SerializableFunction getDriverProviderFn(); + + abstract @Nullable ValueProvider getSessionConfig(); + + abstract @Nullable ValueProvider getCypher(); + + abstract @Nullable ValueProvider getUnwindMapName(); + + abstract @Nullable ValueProvider getTransactionConfig(); + + abstract @Nullable SerializableFunction> + getParametersFunction(); + + abstract @Nullable ValueProvider getBatchSize(); + + abstract @Nullable ValueProvider getLogCypher(); + + abstract Builder toBuilder(); + + public WriteUnwind withDriverConfiguration(DriverConfiguration config) { + return toBuilder() + .setDriverProviderFn(new DriverProviderFromDriverConfiguration(config)) + .build(); + } + + public WriteUnwind withCypher(String cypher) { + checkArgument( + cypher != null, "Neo4jIO.writeUnwind().withCypher(query) called with null cypher query"); + return withCypher(ValueProvider.StaticValueProvider.of(cypher)); + } + + public WriteUnwind withCypher(ValueProvider cypher) { + checkArgument( + cypher != null, "Neo4jIO.writeUnwind().withCypher(cypher) called with null cypher"); + return toBuilder().setCypher(cypher).build(); + } + + public WriteUnwind withUnwindMapName(String mapName) { + checkArgument( + mapName != null, + "Neo4jIO.writeUnwind().withUnwindMapName(query) called with null mapName"); + return withUnwindMapName(ValueProvider.StaticValueProvider.of(mapName)); + } + + public WriteUnwind withUnwindMapName(ValueProvider mapName) { + checkArgument( + mapName != null, + "Neo4jIO.writeUnwind().withUnwindMapName(cypher) called with null mapName"); + return toBuilder().setUnwindMapName(mapName).build(); + } + + public WriteUnwind withTransactionConfig(TransactionConfig transactionConfig) { + checkArgument( + transactionConfig != null, + "Neo4jIO.writeUnwind().withTransactionConfig(config) called with null transactionConfig"); + return withTransactionConfig(ValueProvider.StaticValueProvider.of(transactionConfig)); + } + + public WriteUnwind withTransactionConfig( + ValueProvider transactionConfig) { + checkArgument( + transactionConfig != null, + "Neo4jIO.writeUnwind().withTransactionConfig(config) called with null transactionConfig"); + return toBuilder().setTransactionConfig(transactionConfig).build(); + } + + public WriteUnwind withSessionConfig(SessionConfig sessionConfig) { + checkArgument( + sessionConfig != null, + "Neo4jIO.writeUnwind().withSessionConfig(sessionConfig) called with null sessionConfig"); + return withSessionConfig(ValueProvider.StaticValueProvider.of(sessionConfig)); + } + + public WriteUnwind withSessionConfig(ValueProvider sessionConfig) { + checkArgument( + sessionConfig != null, + "Neo4jIO.writeUnwind().withSessionConfig(sessionConfig) called with null sessionConfig"); + return toBuilder().setSessionConfig(sessionConfig).build(); + } + + // Batch size + public WriteUnwind withBatchSize(long batchSize) { + checkArgument( + batchSize > 0, "Neo4jIO.writeUnwind().withFetchSize(query) called with batchSize<=0"); + return withBatchSize(ValueProvider.StaticValueProvider.of(batchSize)); + } + + public WriteUnwind withBatchSize(ValueProvider batchSize) { + checkArgument( + batchSize != null && batchSize.get() >= 0, + "Neo4jIO.readAll().withBatchSize(query) called with batchSize<=0"); + return toBuilder().setBatchSize(batchSize).build(); + } + + public WriteUnwind withParametersFunction( + SerializableFunction> parametersFunction) { + checkArgument( + parametersFunction != null, + "Neo4jIO.readAll().withParametersFunction(parametersFunction) called with null parametersFunction"); + return toBuilder().setParametersFunction(parametersFunction).build(); + } + + public WriteUnwind withCypherLogging() { + return toBuilder().setLogCypher(ValueProvider.StaticValueProvider.of(Boolean.TRUE)).build(); + } + + @Override + public PDone expand(PCollection input) { + + final SerializableFunction driverProviderFn = getDriverProviderFn(); + final SerializableFunction> parametersFunction = + getParametersFunction(); + SessionConfig sessionConfig = getProvidedValue(getSessionConfig()); + if (sessionConfig == null) { + sessionConfig = SessionConfig.defaultConfig(); + } + TransactionConfig transactionConfig = getProvidedValue(getTransactionConfig()); + if (transactionConfig == null) { + transactionConfig = TransactionConfig.empty(); + } + final String cypher = getProvidedValue(getCypher()); + checkArgument(cypher != null, "please provide an unwind cypher statement to execute"); + final String unwindMapName = getProvidedValue(getUnwindMapName()); + checkArgument(unwindMapName != null, "please provide an unwind map name"); + + Long batchSize = getProvidedValue(getBatchSize()); + if (batchSize == null || batchSize <= 0) { + batchSize = 5000L; + } + + Boolean logCypher = getProvidedValue(getLogCypher()); + if (logCypher == null) { + logCypher = Boolean.FALSE; + } + + if (driverProviderFn == null) { + throw new RuntimeException("please provide a driver provider"); + } + if (parametersFunction == null) { + throw new RuntimeException("please provide a parameters function"); + } + WriteUnwindFn writeFn = + new WriteUnwindFn<>( + driverProviderFn, + sessionConfig, + transactionConfig, + cypher, + parametersFunction, + batchSize, + logCypher, + unwindMapName); + + input.apply(ParDo.of(writeFn)); + + return PDone.in(input.getPipeline()); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.add(DisplayData.item("cypher", getCypher())); + + final SerializableFunction driverProviderFn = getDriverProviderFn(); + if (driverProviderFn != null) { + if (driverProviderFn instanceof HasDisplayData) { + ((HasDisplayData) driverProviderFn).populateDisplayData(builder); + } + } + } + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setDriverProviderFn( + SerializableFunction driverProviderFn); + + abstract Builder setSessionConfig(ValueProvider sessionConfig); + + abstract Builder setTransactionConfig( + ValueProvider transactionConfig); + + abstract Builder setCypher(ValueProvider cypher); + + abstract Builder setUnwindMapName(ValueProvider unwindMapName); + + abstract Builder setParametersFunction( + SerializableFunction> parametersFunction); + + abstract Builder setBatchSize(ValueProvider batchSize); + + abstract Builder setLogCypher(ValueProvider logCypher); + + abstract WriteUnwind build(); + } + } + + /** A {@link DoFn} to execute a Cypher query to read from Neo4j. */ + private static class WriteUnwindFn extends ReadWriteFn { + + private final @NonNull String cypher; + private final @Nullable SerializableFunction> + parametersFunction; + private final boolean logCypher; + private final long batchSize; + private final @NonNull String unwindMapName; + + private long elementsInput; + private boolean loggingDone; + private List> unwindList; + + private WriteUnwindFn( + @NonNull SerializableFunction driverProviderFn, + @NonNull SessionConfig sessionConfig, + @NonNull TransactionConfig transactionConfig, + @NonNull String cypher, + @Nullable SerializableFunction> parametersFunction, + long batchSize, + boolean logCypher, + String unwindMapName) { + super(driverProviderFn, sessionConfig, transactionConfig); + this.cypher = cypher; + this.parametersFunction = parametersFunction; + this.logCypher = logCypher; + this.batchSize = batchSize; + this.unwindMapName = unwindMapName; + + unwindList = new ArrayList<>(); + + elementsInput = 0; + loggingDone = false; + } + + @ProcessElement + public void processElement(ProcessContext context) { + // Map the input data to the parameters map... + // + ParameterT parameters = context.element(); + if (parametersFunction != null) { + // Every input element creates a new Map entry in unwindList + // + unwindList.add(parametersFunction.apply(parameters)); + } else { + // Someone is writing a bunch of static or procedurally generated values to Neo4j + unwindList.add(Collections.emptyMap()); + } + elementsInput++; + + if (elementsInput >= batchSize) { + // Execute the cypher query with the collected parameters map + // + executeCypherUnwindStatement(); + } + } + + private void executeCypherUnwindStatement() { + // In case of errors and no actual input read (error in mapper) we don't have input + // So we don't want to execute any cypher in this case. There's no need to generate even more + // errors + // + if (elementsInput == 0) { + return; + } + + // Add the accumulated list to the overall parameters map + // It contains a single parameter to unwind + // + final Map parametersMap = new HashMap<>(); + parametersMap.put(unwindMapName, unwindList); + + // Every "write" transaction writes a batch of elements to Neo4j. + // The changes to the database are automatically committed. + // + TransactionWork transactionWork = + transaction -> { + Result result = transaction.run(cypher, parametersMap); + while (result.hasNext()) { + // This just consumes any output but the function basically has no output + // To be revisited based on requirements. + // + result.next(); + } + return null; + }; + + if (logCypher && !loggingDone) { + String parametersString = getParametersString(parametersMap); + LOG.info( + "Starting a write transaction for unwind statement cypher: " + + cypher + + ", parameters: " + + parametersString); + loggingDone = true; + } + + if (driverSession.session == null) { + throw new RuntimeException("neo4j session was not initialized correctly"); + } else { + try { + driverSession.session.writeTransaction(transactionWork, transactionConfig); + } catch (Exception e) { + throw new RuntimeException( + "Error writing " + unwindList.size() + " rows to Neo4j with Cypher: " + cypher, e); + } + } + + // Now we need to reset the number of elements read and the parameters map + // + unwindList.clear(); + elementsInput = 0; + } + + @FinishBundle + @Override + public void finishBundle() { + executeCypherUnwindStatement(); + } + } +} diff --git a/sdks/java/io/neo4j/src/main/java/org/apache/beam/sdk/io/neo4j/package-info.java b/sdks/java/io/neo4j/src/main/java/org/apache/beam/sdk/io/neo4j/package-info.java new file mode 100644 index 0000000000000..5c8656dea28ed --- /dev/null +++ b/sdks/java/io/neo4j/src/main/java/org/apache/beam/sdk/io/neo4j/package-info.java @@ -0,0 +1,24 @@ +/* + * 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. + */ + +/** Transforms for reading from and writing to from Neo4j. */ +@Experimental(Kind.SOURCE_SINK) +package org.apache.beam.sdk.io.neo4j; + +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; diff --git a/sdks/java/io/neo4j/src/test/java/org/apache/beam/sdk/io/neo4j/Neo4jIOIT.java b/sdks/java/io/neo4j/src/test/java/org/apache/beam/sdk/io/neo4j/Neo4jIOIT.java new file mode 100644 index 0000000000000..fc8d712b6ccae --- /dev/null +++ b/sdks/java/io/neo4j/src/test/java/org/apache/beam/sdk/io/neo4j/Neo4jIOIT.java @@ -0,0 +1,275 @@ +/* + * 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.io.neo4j; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; +import static org.hamcrest.collection.IsIterableContainingInOrder.contains; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.neo4j.driver.Driver; +import org.neo4j.driver.Record; +import org.neo4j.driver.Result; +import org.neo4j.driver.Session; +import org.neo4j.driver.SessionConfig; +import org.testcontainers.containers.Neo4jContainer; +import org.testcontainers.utility.DockerImageName; + +@RunWith(JUnit4.class) +public class Neo4jIOIT { + + private static Neo4jContainer neo4jContainer; + private static String containerHostname; + private static int containerPort; + + @Rule public transient TestPipeline parameterizedReadPipeline = TestPipeline.create(); + @Rule public transient TestPipeline writeUnwindPipeline = TestPipeline.create(); + @Rule public transient TestPipeline largeWriteUnwindPipeline = TestPipeline.create(); + + @BeforeClass + public static void setup() throws Exception { + neo4jContainer = + new Neo4jContainer<>(DockerImageName.parse("neo4j").withTag(Neo4jTestUtil.NEO4J_VERSION)) + .withStartupAttempts(1) + .withAdminPassword(Neo4jTestUtil.NEO4J_PASSWORD) + .withEnv("NEO4J_dbms_default_listen_address", "0.0.0.0") + .withNetworkAliases(Neo4jTestUtil.NEO4J_NETWORK_ALIAS) + .withSharedMemorySize(256 * 1024 * 1024L); // 256MB + + // Start Neo4j + neo4jContainer.start(); + + // Start with an empty database to use for testing. + // This prevents any possibility of some old data messing up the test results. + // We add a unique constraint to see we're not trying to create nodes twice in the larger test + // below + // + containerHostname = neo4jContainer.getContainerIpAddress(); + containerPort = neo4jContainer.getMappedPort(7687); + + Neo4jTestUtil.executeOnNeo4j( + containerHostname, + containerPort, + "CREATE CONSTRAINT something_id_unique ON (n:Something) ASSERT n.id IS UNIQUE", + true); + } + + @AfterClass + public static void tearDown() { + neo4jContainer.stop(); + neo4jContainer.close(); + } + + private static class ParameterizedReadRowToLineFn extends DoFn { + @DoFn.ProcessElement + public void processElement(ProcessContext context) { + Row row = context.element(); + assert row != null; + int one = row.getInt32(0); + String string = row.getString(1); + context.output(one + "," + string); + } + } + + @Test + public void testParameterizedRead() throws Exception { + PCollection stringsCollections = + parameterizedReadPipeline.apply(Create.of(Arrays.asList("one", "two", "three"))); + + final Schema outputSchema = + Schema.of( + Schema.Field.of("One", Schema.FieldType.INT32), + Schema.Field.of("Str", Schema.FieldType.STRING)); + + SerializableFunction> parametersFunction = + string -> Collections.singletonMap("par1", string); + + Neo4jIO.RowMapper rowMapper = + record -> { + int one = record.get(0).asInt(); + String string = record.get(1).asString(); + return Row.withSchema(outputSchema).attachValues(one, string); + }; + + Neo4jIO.ReadAll read = + Neo4jIO.readAll() + .withCypher("RETURN 1, $par1") + .withDriverConfiguration( + Neo4jTestUtil.getDriverConfiguration(containerHostname, containerPort)) + .withSessionConfig(SessionConfig.forDatabase(Neo4jTestUtil.NEO4J_DATABASE)) + .withRowMapper(rowMapper) + .withParametersFunction(parametersFunction) + .withCoder(SerializableCoder.of(Row.class)) + .withCypherLogging(); + + PCollection outputRows = stringsCollections.apply(read); + + PCollection outputLines = + outputRows.apply(ParDo.of(new ParameterizedReadRowToLineFn())); + + PAssert.that(outputLines).containsInAnyOrder("1,one", "1,two", "1,three"); + + // Now run this pipeline + // + PipelineResult pipelineResult = parameterizedReadPipeline.run(); + + Assert.assertEquals(PipelineResult.State.DONE, pipelineResult.getState()); + } + + @Test + public void testWriteUnwind() throws Exception { + PCollection stringsCollections = + writeUnwindPipeline.apply(Create.of(Arrays.asList("one", "two", "three"))); + + // Every row is represented by a Map in the parameters map. + // We accumulate the rows and 'unwind' those to Neo4j for performance reasons. + // + SerializableFunction> parametersMapper = + name -> Collections.singletonMap("name", name); + + Neo4jIO.WriteUnwind read = + Neo4jIO.writeUnwind() + .withDriverConfiguration( + Neo4jTestUtil.getDriverConfiguration(containerHostname, containerPort)) + .withSessionConfig(SessionConfig.forDatabase(Neo4jTestUtil.NEO4J_DATABASE)) + .withBatchSize(5000) + .withUnwindMapName("rows") + .withCypher("UNWIND $rows AS row MERGE(n:Num { name : row.name })") + .withParametersFunction(parametersMapper) + .withCypherLogging(); + + stringsCollections.apply(read); + + // Now run this pipeline + // + PipelineResult pipelineResult = writeUnwindPipeline.run(); + + Assert.assertEquals(PipelineResult.State.DONE, pipelineResult.getState()); + + // Connect back to the Instance and verify that we have 3 nodes + // + try (Driver driver = Neo4jTestUtil.getDriver(containerHostname, containerPort)) { + try (Session session = Neo4jTestUtil.getSession(driver, true)) { + List names = + session.readTransaction( + tx -> { + List list = new ArrayList<>(); + Result result = tx.run("MATCH(n:Num) RETURN n.name"); + while (result.hasNext()) { + Record record = result.next(); + list.add(record.get(0).asString()); + } + return list; + }); + + assertThat(names, containsInAnyOrder("one", "two", "three")); + } + } + } + + @Test + public void testLargeWriteUnwind() throws Exception { + final int startId = 5000; + final int endId = 6000; + // Create 1000 IDs + List idList = new ArrayList<>(); + for (int id = startId; id < endId; id++) { + idList.add(id); + } + PCollection idCollection = largeWriteUnwindPipeline.apply(Create.of(idList)); + + // Every row is represented by a Map in the parameters map. + // We accumulate the rows and 'unwind' those to Neo4j for performance reasons. + // + SerializableFunction> parametersFunction = + id -> ImmutableMap.of("id", id, "name", "Casters", "firstName", "Matt"); + + // 1000 rows with a batch size of 123 should trigger most scenarios we can think of + // We've put a unique constraint on Something.id + // + Neo4jIO.WriteUnwind read = + Neo4jIO.writeUnwind() + .withDriverConfiguration( + Neo4jTestUtil.getDriverConfiguration(containerHostname, containerPort)) + .withSessionConfig(SessionConfig.forDatabase(Neo4jTestUtil.NEO4J_DATABASE)) + .withBatchSize(123) + .withUnwindMapName("rows") + .withCypher("UNWIND $rows AS row CREATE(n:Something { id : row.id })") + .withParametersFunction(parametersFunction) + .withCypherLogging(); + + idCollection.apply(read); + + // Now run this pipeline + // + PipelineResult pipelineResult = largeWriteUnwindPipeline.run(); + + Assert.assertEquals(PipelineResult.State.DONE, pipelineResult.getState()); + + // Connect back to the Instance and verify that we have 1000 Something nodes + // + try (Driver driver = Neo4jTestUtil.getDriver(containerHostname, containerPort)) { + try (Session session = Neo4jTestUtil.getSession(driver, true)) { + List values = + session.readTransaction( + tx -> { + List v = null; + int nrRows = 0; + Result result = + tx.run("MATCH(n:Something) RETURN count(n), min(n.id), max(n.id)"); + while (result.hasNext()) { + Record record = result.next(); + v = + Arrays.asList( + record.get(0).asInt(), + record.get(1).asInt(), + record.get(2).asInt(), + ++nrRows); + } + return v; + }); + + Assert.assertNotNull(values); + assertThat(values, contains(endId - startId, startId, endId - 1, 1)); + } + } + } +} diff --git a/sdks/java/io/neo4j/src/test/java/org/apache/beam/sdk/io/neo4j/Neo4jIOTest.java b/sdks/java/io/neo4j/src/test/java/org/apache/beam/sdk/io/neo4j/Neo4jIOTest.java new file mode 100644 index 0000000000000..3ed76a347c4df --- /dev/null +++ b/sdks/java/io/neo4j/src/test/java/org/apache/beam/sdk/io/neo4j/Neo4jIOTest.java @@ -0,0 +1,117 @@ +/* + * 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.io.neo4j; + +import java.util.Arrays; +import java.util.concurrent.TimeUnit; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.neo4j.driver.Config; + +@RunWith(JUnit4.class) +public class Neo4jIOTest { + + @Test + public void testDriverConfigurationCreate() throws Exception { + Neo4jIO.DriverConfiguration driverConfiguration = + Neo4jIO.DriverConfiguration.create("someUrl", "username", "password"); + Assert.assertEquals("someUrl", driverConfiguration.getUrl().get()); + Assert.assertEquals("username", driverConfiguration.getUsername().get()); + Assert.assertEquals("password", driverConfiguration.getPassword().get()); + } + + @Test + public void testDriverConfigurationWith() throws Exception { + Neo4jIO.DriverConfiguration driverConfiguration = Neo4jIO.DriverConfiguration.create(); + + Config config = + Config.builder() + .withEncryption() + .withConnectionAcquisitionTimeout(54321L, TimeUnit.MILLISECONDS) + .withConnectionTimeout(43210L, TimeUnit.MILLISECONDS) + .withConnectionLivenessCheckTimeout(32109L, TimeUnit.MILLISECONDS) + .withMaxConnectionLifetime(21098L, TimeUnit.MILLISECONDS) + .withMaxConnectionPoolSize(101) + .build(); + + driverConfiguration = driverConfiguration.withConfig(config); + + Config configVerify = driverConfiguration.getConfig(); + Assert.assertNotNull(configVerify); + Assert.assertEquals(true, configVerify.encrypted()); + + Assert.assertEquals(54321L, configVerify.connectionAcquisitionTimeoutMillis()); + Assert.assertEquals(43210L, configVerify.connectionTimeoutMillis()); + Assert.assertEquals(32109L, configVerify.idleTimeBeforeConnectionTest()); + Assert.assertEquals(21098L, configVerify.maxConnectionLifetimeMillis()); + Assert.assertEquals(101, configVerify.maxConnectionPoolSize()); + + driverConfiguration = driverConfiguration.withUrl("url1"); + Assert.assertEquals("url1", driverConfiguration.getUrl().get()); + + // URL and URLs can be set independently but are both used + driverConfiguration = driverConfiguration.withUrls(Arrays.asList("url2", "url3", "url4")); + Assert.assertEquals(3, driverConfiguration.getUrls().get().size()); + + driverConfiguration = driverConfiguration.withUsername("username"); + Assert.assertEquals("username", driverConfiguration.getUsername().get()); + + driverConfiguration = driverConfiguration.withPassword("password"); + Assert.assertEquals("password", driverConfiguration.getPassword().get()); + } + + @Test + public void testDriverConfigurationErrors() throws Exception { + Neo4jIO.DriverConfiguration driverConfiguration = Neo4jIO.DriverConfiguration.create(); + + try { + driverConfiguration.withUrl((String) null); + Assert.fail("Null URL is not reported"); + } catch (Exception e) { + // OK, error was reported + } + + try { + driverConfiguration.withUsername((String) null); + Assert.fail("Null user is not reported"); + } catch (Exception e) { + // OK, error was reported + } + + try { + driverConfiguration.withUsername(""); + } catch (Exception e) { + throw new AssertionError("Empty user string should not throw an error", e); + } + + try { + driverConfiguration.withPassword((String) null); + Assert.fail("Null password is not reported"); + } catch (Exception e) { + // OK, error was reported + } + + try { + driverConfiguration.withPassword(""); + } catch (Exception e) { + throw new AssertionError("Empty password string should not throw an error", e); + } + } +} diff --git a/sdks/java/io/neo4j/src/test/java/org/apache/beam/sdk/io/neo4j/Neo4jTestUtil.java b/sdks/java/io/neo4j/src/test/java/org/apache/beam/sdk/io/neo4j/Neo4jTestUtil.java new file mode 100644 index 0000000000000..adfd0664a2e9c --- /dev/null +++ b/sdks/java/io/neo4j/src/test/java/org/apache/beam/sdk/io/neo4j/Neo4jTestUtil.java @@ -0,0 +1,70 @@ +/* + * 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.io.neo4j; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Arrays; +import org.neo4j.driver.AuthTokens; +import org.neo4j.driver.Config; +import org.neo4j.driver.Driver; +import org.neo4j.driver.GraphDatabase; +import org.neo4j.driver.Session; +import org.neo4j.driver.SessionConfig; + +public class Neo4jTestUtil { + + public static final String NEO4J_VERSION = "latest"; + public static final String NEO4J_NETWORK_ALIAS = "neo4jcontainer"; + public static final String NEO4J_USERNAME = "neo4j"; + public static final String NEO4J_PASSWORD = "abcd"; + public static final String NEO4J_DATABASE = "neo4j"; + + public static final String getUrl(String hostname, int port) { + return "neo4j://" + hostname + ":" + port; + } + + public static Driver getDriver(String hostname, int port) throws URISyntaxException { + return GraphDatabase.routingDriver( + Arrays.asList(new URI(getUrl(hostname, port))), + AuthTokens.basic(NEO4J_USERNAME, NEO4J_PASSWORD), + Config.builder().build()); + } + + public static Session getSession(Driver driver, boolean withDatabase) { + SessionConfig.Builder builder = SessionConfig.builder(); + if (withDatabase) { + builder = builder.withDatabase(NEO4J_DATABASE); + } + return driver.session(builder.build()); + } + + public static Neo4jIO.DriverConfiguration getDriverConfiguration(String hostname, int port) { + return Neo4jIO.DriverConfiguration.create( + getUrl(hostname, port), NEO4J_USERNAME, NEO4J_PASSWORD); + } + + public static void executeOnNeo4j(String hostname, int port, String cypher, boolean useDatabase) + throws Exception { + try (Driver driver = Neo4jTestUtil.getDriver(hostname, port)) { + try (Session session = Neo4jTestUtil.getSession(driver, useDatabase)) { + session.run(cypher); + } + } + } +} diff --git a/sdks/python/apache_beam/dataframe/frames.py b/sdks/python/apache_beam/dataframe/frames.py index 3cbf613ddec79..00961a6f1b2f3 100644 --- a/sdks/python/apache_beam/dataframe/frames.py +++ b/sdks/python/apache_beam/dataframe/frames.py @@ -229,6 +229,16 @@ def droplevel(self, level, axis): preserves_partition_by=partitionings.Arbitrary() if axis in (1, 'column') else partitionings.Singleton())) + @frame_base.with_docs_from(pd.DataFrame) + @frame_base.args_to_kwargs(pd.DataFrame) + def swaplevel(self, **kwargs): + return frame_base.DeferredFrame.wrap( + expressions.ComputedExpression( + 'swaplevel', + lambda df: df.swaplevel(**kwargs), [self._expr], + requires_partition_by=partitionings.Arbitrary(), + preserves_partition_by=partitionings.Arbitrary())) + @frame_base.with_docs_from(pd.DataFrame) @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) diff --git a/sdks/python/apache_beam/dataframe/frames_test.py b/sdks/python/apache_beam/dataframe/frames_test.py index c493db5e1b006..ca4f561c9ab77 100644 --- a/sdks/python/apache_beam/dataframe/frames_test.py +++ b/sdks/python/apache_beam/dataframe/frames_test.py @@ -624,6 +624,16 @@ def test_merge_same_key_suffix_collision(self): nonparallel=True, check_proxy=False) + def test_swaplevel(self): + df = pd.DataFrame( + {"Grade": ["A", "B", "A", "C"]}, + index=[ + ["Final exam", "Final exam", "Coursework", "Coursework"], + ["History", "Geography", "History", "Geography"], + ["January", "February", "March", "April"], + ]) + self._run_test(lambda df: df.swaplevel(), df) + def test_value_counts_with_nans(self): # similar to doctests that verify value_counts, but include nan values to # make sure we handle them correctly. diff --git a/sdks/python/apache_beam/dataframe/pandas_doctests_test.py b/sdks/python/apache_beam/dataframe/pandas_doctests_test.py index 1a30aa85db337..19113af53e4fc 100644 --- a/sdks/python/apache_beam/dataframe/pandas_doctests_test.py +++ b/sdks/python/apache_beam/dataframe/pandas_doctests_test.py @@ -269,7 +269,6 @@ def test_dataframe_tests(self): # frames_test.py::DeferredFrameTest::test_groupby_transform_sum "df.groupby('Date')['Data'].transform('sum')", ], - 'pandas.core.frame.DataFrame.swaplevel': ['*'], 'pandas.core.frame.DataFrame.melt': ['*'], 'pandas.core.frame.DataFrame.reindex_axis': ['*'], 'pandas.core.frame.DataFrame.round': [ @@ -511,7 +510,6 @@ def test_series_tests(self): 'ser.groupby(["a", "b", "a", np.nan]).mean()', 'ser.groupby(["a", "b", "a", np.nan], dropna=False).mean()', ], - 'pandas.core.series.Series.swaplevel' :['*'] }, skip={ # Relies on setting values with iloc diff --git a/sdks/python/apache_beam/examples/complete/autocomplete_it_test.py b/sdks/python/apache_beam/examples/complete/autocomplete_it_test.py new file mode 100644 index 0000000000000..28312b7303b2a --- /dev/null +++ b/sdks/python/apache_beam/examples/complete/autocomplete_it_test.py @@ -0,0 +1,116 @@ +# +# 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. +# + +"""End-to-end test for Autocomplete example.""" +# pytype: skip-file + +import logging +import re +import unittest +import uuid + +import pytest + +from apache_beam.examples.complete import autocomplete +from apache_beam.testing.test_pipeline import TestPipeline + +# Protect against environments where gcsio library is not available. +try: + from apache_beam.io.gcp import gcsio +except ImportError: + gcsio = None + + +def read_gcs_output_file(file_pattern): + gcs = gcsio.GcsIO() + file_names = gcs.list_prefix(file_pattern).keys() + output = [] + for file_name in file_names: + output.append(gcs.open(file_name).read().decode('utf-8').strip()) + return '\n'.join(output) + + +def create_content_input_file(path, contents): + logging.info('Creating file: %s', path) + gcs = gcsio.GcsIO() + with gcs.open(path, 'w') as f: + f.write(str.encode(contents, 'utf-8')) + + +def format_output_file(output_string): + def extract_prefix_topk_words_tuples(line): + match = re.match(r'(.*): \[(.*)\]', line) + prefix = match.group(1) + topK_words_string = extract_top_k_words_tuples(match.group(2)) + return prefix, topK_words_string + + def extract_top_k_words_tuples(top_k_words_string): + top_k_list = top_k_words_string.split("), (") + return tuple( + map( + lambda top_k_string: tuple(format_top_k_tuples(top_k_string)), + top_k_list)) + + def format_top_k_tuples(top_k_string): + (frequency, words) = top_k_string.replace('(', '').replace(')', '').replace( + '\"', '').replace('\'', '').replace(' ', '').split(',') + return int(frequency), words + + return list( + map( + lambda line: extract_prefix_topk_words_tuples(line), + output_string.split('\n'))) + + +class AutocompleteIT(unittest.TestCase): + WORDS = ['this', 'this', 'that', 'to', 'to', 'to'] + EXPECTED_PREFIXES = [ + ('t', ((3, 'to'), (2, 'this'), (1, 'that'))), + ('to', ((3, 'to'), )), + ('th', ((2, 'this'), (1, 'that'))), + ('thi', ((2, 'this'), )), + ('this', ((2, 'this'), )), + ('tha', ((1, 'that'), )), + ('that', ((1, 'that'), )), + ] + + @pytest.mark.no_xdist + @pytest.mark.examples_postcommit + def test_autocomplete_output_files_on_small_input(self): + test_pipeline = TestPipeline(is_integration_test=True) + # Setup the files with expected content. + OUTPUT_FILE_DIR = \ + 'gs://temp-storage-for-end-to-end-tests/py-it-cloud/output' + output = '/'.join([OUTPUT_FILE_DIR, str(uuid.uuid4()), 'result']) + INPUT_FILE_DIR = \ + 'gs://temp-storage-for-end-to-end-tests/py-it-cloud/input' + input = '/'.join([INPUT_FILE_DIR, str(uuid.uuid4()), 'input.txt']) + create_content_input_file(input, ' '.join(self.WORDS)) + extra_opts = {'input': input, 'output': output} + + autocomplete.run(test_pipeline.get_full_options_as_args(**extra_opts)) + + # Load result file and compare. + result = read_gcs_output_file(output).strip() + + self.assertEqual( + sorted(self.EXPECTED_PREFIXES), sorted(format_output_file(result))) + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() diff --git a/sdks/python/apache_beam/examples/complete/autocomplete_test.py b/sdks/python/apache_beam/examples/complete/autocomplete_test.py index 4a0a2fc180239..8b941f38a6490 100644 --- a/sdks/python/apache_beam/examples/complete/autocomplete_test.py +++ b/sdks/python/apache_beam/examples/complete/autocomplete_test.py @@ -19,10 +19,6 @@ # pytype: skip-file -import logging -import os -import re -import tempfile import unittest import pytest @@ -33,38 +29,6 @@ from apache_beam.testing.test_utils import compute_hash from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to -from apache_beam.testing.util import open_shards - - -def format_output_file(output_string): - def extract_prefix_topk_words_tuples(line): - match = re.match(r'(.*): \[(.*)\]', line) - prefix = match.group(1) - topK_words_string = extract_top_k_words_tuples(match.group(2)) - return prefix, topK_words_string - - def extract_top_k_words_tuples(top_k_words_string): - top_k_list = top_k_words_string.split("), (") - return tuple( - map( - lambda top_k_string: tuple(format_top_k_tuples(top_k_string)), - top_k_list)) - - def format_top_k_tuples(top_k_string): - (frequency, words) = top_k_string.replace('(', '').replace(')', '').replace( - '\"', '').replace('\'', '').replace(' ', '').split(',') - return int(frequency), words - - return list( - map( - lambda line: extract_prefix_topk_words_tuples(line), - output_string.split('\n'))) - - -def create_content_input_file(path, contents): - logging.info('Creating temp file: %s', path) - with open(path, 'w') as f: - f.write(contents) class AutocompleteTest(unittest.TestCase): @@ -105,28 +69,6 @@ def test_autocomplete_it(self): assert_that(checksum, equal_to([self.KINGLEAR_HASH_SUM])) - @pytest.mark.no_xdist - @pytest.mark.examples_postcommit - def test_autocomplete_output_files_on_small_input(self): - test_pipeline = TestPipeline(is_integration_test=True) - # Setup the files with expected content. - temp_folder = tempfile.mkdtemp() - create_content_input_file( - os.path.join(temp_folder, 'input.txt'), ' '.join(self.WORDS)) - extra_opts = { - 'input': '%s/input.txt' % temp_folder, - 'output': os.path.join(temp_folder, 'result') - } - - autocomplete.run(test_pipeline.get_full_options_as_args(**extra_opts)) - - # Load result file and compare. - with open_shards(os.path.join(temp_folder, 'result-*-of-*')) as result_file: - result = result_file.read().strip() - - self.assertEqual( - sorted(self.EXPECTED_PREFIXES), sorted(format_output_file(result))) - if __name__ == '__main__': unittest.main() diff --git a/sdks/python/apache_beam/examples/complete/estimate_pi_it_test.py b/sdks/python/apache_beam/examples/complete/estimate_pi_it_test.py new file mode 100644 index 0000000000000..cda92e5da4cb2 --- /dev/null +++ b/sdks/python/apache_beam/examples/complete/estimate_pi_it_test.py @@ -0,0 +1,68 @@ +# +# 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. +# + +"""End-to-end test for Estimate Pi example.""" +# pytype: skip-file + +import json +import logging +import unittest +import uuid + +import pytest + +from apache_beam.examples.complete import estimate_pi +from apache_beam.testing.test_pipeline import TestPipeline + +# Protect against environments where gcsio library is not available. +try: + from apache_beam.io.gcp import gcsio +except ImportError: + gcsio = None + + +def read_gcs_output_file(file_pattern): + gcs = gcsio.GcsIO() + file_names = gcs.list_prefix(file_pattern).keys() + output = [] + for file_name in file_names: + output.append(gcs.open(file_name).read().decode('utf-8')) + return '\n'.join(output) + + +class EstimatePiIT(unittest.TestCase): + @pytest.mark.no_xdist + @pytest.mark.examples_postcommit + def test_estimate_pi_output_file(self): + test_pipeline = TestPipeline(is_integration_test=True) + OUTPUT_FILE = \ + 'gs://temp-storage-for-end-to-end-tests/py-it-cloud/output' + output = '/'.join([OUTPUT_FILE, str(uuid.uuid4()), 'result']) + extra_opts = {'output': output} + estimate_pi.run(test_pipeline.get_full_options_as_args(**extra_opts)) + # Load result file and compare. + result = read_gcs_output_file(output) + [_, _, estimated_pi] = json.loads(result.strip()) + # Note: Probabilistically speaking this test can fail with a probability + # that is very small (VERY) given that we run at least 100 thousand + # trials. + self.assertTrue(3.125 <= estimated_pi <= 3.155) + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() diff --git a/sdks/python/apache_beam/examples/complete/estimate_pi_test.py b/sdks/python/apache_beam/examples/complete/estimate_pi_test.py index 08ac93f47bf56..ff224f4a9cab0 100644 --- a/sdks/python/apache_beam/examples/complete/estimate_pi_test.py +++ b/sdks/python/apache_beam/examples/complete/estimate_pi_test.py @@ -19,19 +19,13 @@ # pytype: skip-file -import json import logging -import os -import tempfile import unittest -import pytest - from apache_beam.examples.complete import estimate_pi from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import BeamAssertException from apache_beam.testing.util import assert_that -from apache_beam.testing.util import open_shards def in_between(lower, upper): @@ -54,21 +48,6 @@ def test_basics(self): # trials. assert_that(result, in_between(3.125, 3.155)) - @pytest.mark.no_xdist - @pytest.mark.examples_postcommit - def test_estimate_pi_output_file(self): - test_pipeline = TestPipeline(is_integration_test=True) - temp_folder = tempfile.mkdtemp() - extra_opts = {'output': os.path.join(temp_folder, 'result')} - estimate_pi.run(test_pipeline.get_full_options_as_args(**extra_opts)) - # Load result file and compare. - with open_shards(os.path.join(temp_folder, 'result-*-of-*')) as result_file: - [_, _, estimated_pi] = json.loads(result_file.read().strip()) - # Note: Probabilistically speaking this test can fail with a probability - # that is very small (VERY) given that we run at least 100 thousand - # trials. - self.assertTrue(3.125 <= estimated_pi <= 3.155) - if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_it_test.py b/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_it_test.py new file mode 100644 index 0000000000000..64dcd06c8ecb7 --- /dev/null +++ b/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_it_test.py @@ -0,0 +1,118 @@ +# +# 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. +# + +"""End-to-end test for Top Wikipedia Sessions example.""" +# pytype: skip-file + +import json +import logging +import unittest +import uuid + +import pytest + +from apache_beam.examples.complete import top_wikipedia_sessions +from apache_beam.testing.test_pipeline import TestPipeline + +# Protect against environments where gcsio library is not available. +try: + from apache_beam.io.gcp import gcsio +except ImportError: + gcsio = None + + +def read_gcs_output_file(file_pattern): + gcs = gcsio.GcsIO() + file_names = gcs.list_prefix(file_pattern).keys() + output = [] + for file_name in file_names: + output.append(gcs.open(file_name).read().decode('utf-8')) + return '\n'.join(output) + + +def create_content_input_file(path, contents): + logging.info('Creating file: %s', path) + gcs = gcsio.GcsIO() + with gcs.open(path, 'w') as f: + f.write(str.encode(contents, 'utf-8')) + + +class ComputeTopSessionsIT(unittest.TestCase): + EDITS = [ + json.dumps({ + 'timestamp': 0.0, 'contributor_username': 'user1' + }), + json.dumps({ + 'timestamp': 0.001, 'contributor_username': 'user1' + }), + json.dumps({ + 'timestamp': 0.002, 'contributor_username': 'user1' + }), + json.dumps({ + 'timestamp': 0.0, 'contributor_username': 'user2' + }), + json.dumps({ + 'timestamp': 0.001, 'contributor_username': 'user2' + }), + json.dumps({ + 'timestamp': 3.601, 'contributor_username': 'user2' + }), + json.dumps({ + 'timestamp': 3.602, 'contributor_username': 'user2' + }), + json.dumps({ + 'timestamp': 2 * 3600.0, 'contributor_username': 'user2' + }), + json.dumps({ + 'timestamp': 35 * 24 * 3.600, 'contributor_username': 'user3' + }) + ] + + EXPECTED = [ + 'user1 : [0.0, 3600.002) : 3 : [0.0, 2592000.0)', + 'user2 : [0.0, 3603.602) : 4 : [0.0, 2592000.0)', + 'user2 : [7200.0, 10800.0) : 1 : [0.0, 2592000.0)', + 'user3 : [3024.0, 6624.0) : 1 : [0.0, 2592000.0)', + ] + + # TODO Enable when fixed this tests for Dataflow runner + @pytest.mark.sickbay_dataflow + @pytest.mark.no_xdist + @pytest.mark.examples_postcommit + def test_top_wikipedia_sessions_output_files_on_small_input(self): + test_pipeline = TestPipeline(is_integration_test=True) + # Setup the files with expected content. + OUTPUT_FILE_DIR = \ + 'gs://temp-storage-for-end-to-end-tests/py-it-cloud/output' + output = '/'.join([OUTPUT_FILE_DIR, str(uuid.uuid4()), 'result']) + INPUT_FILE_DIR = \ + 'gs://temp-storage-for-end-to-end-tests/py-it-cloud/input' + input = '/'.join([INPUT_FILE_DIR, str(uuid.uuid4()), 'input.txt']) + create_content_input_file(input, '\n'.join(self.EDITS)) + extra_opts = {'input': input, 'output': output, 'sampling_threshold': '1.0'} + top_wikipedia_sessions.run( + test_pipeline.get_full_options_as_args(**extra_opts)) + + # Load result file and compare. + result = read_gcs_output_file(output).strip().splitlines() + + self.assertEqual(self.EXPECTED, sorted(result, key=lambda x: x.split()[0])) + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() diff --git a/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_test.py b/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_test.py index a384ed513055d..3c171664e45d2 100644 --- a/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_test.py +++ b/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_test.py @@ -20,19 +20,13 @@ # pytype: skip-file import json -import logging -import os -import tempfile import unittest -import pytest - import apache_beam as beam from apache_beam.examples.complete import top_wikipedia_sessions from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to -from apache_beam.testing.util import open_shards class ComputeTopSessionsTest(unittest.TestCase): @@ -74,11 +68,6 @@ class ComputeTopSessionsTest(unittest.TestCase): 'user3 : [3024.0, 6624.0) : 1 : [0.0, 2592000.0)', ] - def create_content_input_file(self, path, contents): - logging.info('Creating temp file: %s', path) - with open(path, 'w') as f: - f.write(contents) - def test_compute_top_sessions(self): with TestPipeline() as p: edits = p | beam.Create(self.EDITS) @@ -86,28 +75,6 @@ def test_compute_top_sessions(self): assert_that(result, equal_to(self.EXPECTED)) - @pytest.mark.no_xdist - @pytest.mark.examples_postcommit - def test_top_wikipedia_sessions_output_files_on_small_input(self): - test_pipeline = TestPipeline(is_integration_test=True) - # Setup the files with expected content. - temp_folder = tempfile.mkdtemp() - self.create_content_input_file( - os.path.join(temp_folder, 'input.txt'), '\n'.join(self.EDITS)) - extra_opts = { - 'input': '%s/input.txt' % temp_folder, - 'output': os.path.join(temp_folder, 'result'), - 'sampling_threshold': '1.0' - } - top_wikipedia_sessions.run( - test_pipeline.get_full_options_as_args(**extra_opts)) - - # Load result file and compare. - with open_shards(os.path.join(temp_folder, 'result-*-of-*')) as result_file: - result = result_file.read().strip().splitlines() - - self.assertEqual(self.EXPECTED, sorted(result, key=lambda x: x.split()[0])) - if __name__ == '__main__': unittest.main() diff --git a/sdks/python/apache_beam/examples/cookbook/coders_it_test.py b/sdks/python/apache_beam/examples/cookbook/coders_it_test.py new file mode 100644 index 0000000000000..c402003485077 --- /dev/null +++ b/sdks/python/apache_beam/examples/cookbook/coders_it_test.py @@ -0,0 +1,104 @@ +# +# 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. +# + +"""End-to-end test for Coders example.""" +# pytype: skip-file + +import json +import logging +import unittest +import uuid + +import pytest + +from apache_beam.examples.cookbook import coders +from apache_beam.testing.test_pipeline import TestPipeline + +# Protect against environments where gcsio library is not available. +try: + from apache_beam.io.gcp import gcsio +except ImportError: + gcsio = None + + +def read_gcs_output_file(file_pattern): + gcs = gcsio.GcsIO() + file_names = gcs.list_prefix(file_pattern).keys() + output = [] + for file_name in file_names: + output.append(gcs.open(file_name).read().decode('utf-8').strip()) + return '\n'.join(output) + + +def create_content_input_file(path, contents): + logging.info('Creating file: %s', path) + gcs = gcsio.GcsIO() + with gcs.open(path, 'w') as f: + f.write(str.encode(contents, 'utf-8')) + + +def format_result(result_string): + def format_tuple(result_elem_list): + [country, counter] = result_elem_list + return country, int(counter.strip()) + + result_list = list( + map( + lambda result_elem: format_tuple(result_elem.split(',')), + result_string.replace('\'', '').replace('[', + '').replace(']', '').replace( + '\"', '').split('\n'))) + return result_list + + +class CodersIT(unittest.TestCase): + SAMPLE_RECORDS = [{ + 'host': ['Germany', 1], 'guest': ['Italy', 0] + }, { + 'host': ['Germany', 1], 'guest': ['Brasil', 3] + }, { + 'host': ['Brasil', 1], 'guest': ['Italy', 0] + }] + + EXPECTED_RESULT = [('Italy', 0), ('Brasil', 6), ('Germany', 3)] + + @pytest.mark.no_xdist + @pytest.mark.examples_postcommit + def test_coders_output_files_on_small_input(self): + test_pipeline = TestPipeline(is_integration_test=True) + # Setup the files with expected content. + OUTPUT_FILE_DIR = \ + 'gs://temp-storage-for-end-to-end-tests/py-it-cloud/output' + output = '/'.join([OUTPUT_FILE_DIR, str(uuid.uuid4()), 'result']) + INPUT_FILE_DIR = \ + 'gs://temp-storage-for-end-to-end-tests/py-it-cloud/input' + input = '/'.join([INPUT_FILE_DIR, str(uuid.uuid4()), 'input.txt']) + create_content_input_file( + input, '\n'.join(map(json.dumps, self.SAMPLE_RECORDS))) + extra_opts = {'input': input, 'output': output} + coders.run(test_pipeline.get_full_options_as_args(**extra_opts)) + + # Load result file and compare. + result = read_gcs_output_file(output).strip() + + self.assertEqual( + sorted(self.EXPECTED_RESULT), sorted(format_result(result))) + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() diff --git a/sdks/python/apache_beam/examples/cookbook/coders_test.py b/sdks/python/apache_beam/examples/cookbook/coders_test.py index 540c7653476e1..ade5a84a19824 100644 --- a/sdks/python/apache_beam/examples/cookbook/coders_test.py +++ b/sdks/python/apache_beam/examples/cookbook/coders_test.py @@ -19,20 +19,14 @@ # pytype: skip-file -import json import logging -import os -import tempfile import unittest -import pytest - import apache_beam as beam from apache_beam.examples.cookbook import coders from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to -from apache_beam.testing.util import open_shards class CodersTest(unittest.TestCase): @@ -47,24 +41,6 @@ class CodersTest(unittest.TestCase): EXPECTED_RESULT = [('Italy', 0), ('Brasil', 6), ('Germany', 3)] - def create_content_input_file(self, path, contents): - logging.info('Creating temp file: %s', path) - with open(path, 'w') as f: - f.write(contents) - - def format_result(self, result_string): - def format_tuple(result_elem_list): - [country, counter] = result_elem_list - return country, int(counter.strip()) - - result_list = list( - map( - lambda result_elem: format_tuple(result_elem.split(',')), - result_string.replace('\'', - '').replace('[', '').replace(']', '').replace( - '\"', '').split('\n'))) - return result_list - def test_compute_points(self): with TestPipeline() as p: records = p | 'create' >> beam.Create(self.SAMPLE_RECORDS) @@ -74,29 +50,6 @@ def test_compute_points(self): | beam.CombinePerKey(sum)) assert_that(result, equal_to(self.EXPECTED_RESULT)) - @pytest.mark.no_xdist - @pytest.mark.examples_postcommit - def test_coders_output_files_on_small_input(self): - test_pipeline = TestPipeline(is_integration_test=True) - - # Setup the files with expected content. - temp_folder = tempfile.mkdtemp() - self.create_content_input_file( - os.path.join(temp_folder, 'input.txt'), - '\n'.join(map(json.dumps, self.SAMPLE_RECORDS))) - extra_opts = { - 'input': '%s/input.txt' % temp_folder, - 'output': os.path.join(temp_folder, 'result') - } - coders.run(test_pipeline.get_full_options_as_args(**extra_opts)) - - # Load result file and compare. - with open_shards(os.path.join(temp_folder, 'result-*-of-*')) as result_file: - result = result_file.read().strip() - - self.assertEqual( - sorted(self.EXPECTED_RESULT), sorted(self.format_result(result))) - if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/sdks/python/apache_beam/io/aws/clients/s3/boto3_client.py b/sdks/python/apache_beam/io/aws/clients/s3/boto3_client.py index 878ad070555b7..aee24ac2c052c 100644 --- a/sdks/python/apache_beam/io/aws/clients/s3/boto3_client.py +++ b/sdks/python/apache_beam/io/aws/clients/s3/boto3_client.py @@ -78,6 +78,31 @@ def __init__(self, options): self._download_stream = None self._download_pos = 0 + def get_object_metadata(self, request): + """Retrieves an object's metadata. + + Args: + request: (GetRequest) input message + + Returns: + (Object) The response message. + """ + kwargs = {'Bucket': request.bucket, 'Key': request.object} + + try: + boto_response = self.client.head_object(**kwargs) + except Exception as e: + raise messages.S3ClientError(str(e), get_http_error_code(e)) + + item = messages.Item( + boto_response['ETag'], + request.object, + boto_response['LastModified'], + boto_response['ContentLength'], + boto_response['ContentType']) + + return item + def get_stream(self, request, start): """Opens a stream object starting at the given position. diff --git a/sdks/python/apache_beam/io/gcp/pubsub_integration_test.py b/sdks/python/apache_beam/io/gcp/pubsub_integration_test.py index e52d22032ed7c..6754c70068aeb 100644 --- a/sdks/python/apache_beam/io/gcp/pubsub_integration_test.py +++ b/sdks/python/apache_beam/io/gcp/pubsub_integration_test.py @@ -217,8 +217,6 @@ def test_streaming_data_only(self): @pytest.mark.it_postcommit def test_streaming_with_attributes(self): - if self.runner_name == 'TestDataflowRunner': - pytest.skip("BEAM-13218") self._test_streaming(with_attributes=True) diff --git a/sdks/python/pytest.ini b/sdks/python/pytest.ini index 36e09d51cf19d..4a52bce20900c 100644 --- a/sdks/python/pytest.ini +++ b/sdks/python/pytest.ini @@ -39,6 +39,7 @@ markers = sickbay_direct: run without sickbay-direct sickbay_spark: run without sickbay-spark sickbay_flink: run without sickbay-flink + sickbay_dataflow: run without sickbay-dataflow # Tests using this marker conflict with the xdist plugin in some way, such # as enabling save_main_session. no_xdist: run without pytest-xdist plugin diff --git a/sdks/python/test-suites/dataflow/common.gradle b/sdks/python/test-suites/dataflow/common.gradle index a91165277e802..f263bc6297e3e 100644 --- a/sdks/python/test-suites/dataflow/common.gradle +++ b/sdks/python/test-suites/dataflow/common.gradle @@ -160,7 +160,7 @@ task examples { "sdk_location": files(configurations.distTarBall.files).singleFile, "runner_v2": "true", "suite": "postCommitIT-df${pythonVersionSuffix}-xdist", - "collect": "examples_postcommit and not no_xdist" + "collect": "examples_postcommit and not no_xdist and not sickbay_dataflow" ] def cmdArgs = mapToArgString(argMap) exec { @@ -176,7 +176,7 @@ task examples { "sdk_location": files(configurations.distTarBall.files).singleFile, "runner_v2": "true", "suite": "postCommitIT-df${pythonVersionSuffix}-no-xdist", - "collect": "examples_postcommit and no_xdist" + "collect": "examples_postcommit and no_xdist and not sickbay_dataflow" ] def cmdArgs = mapToArgString(argMap) exec { diff --git a/sdks/python/test-suites/portable/common.gradle b/sdks/python/test-suites/portable/common.gradle index 037022a12dd78..ae296b157cea1 100644 --- a/sdks/python/test-suites/portable/common.gradle +++ b/sdks/python/test-suites/portable/common.gradle @@ -384,7 +384,7 @@ addTestJavaJarCreator("FlinkRunner", tasks.getByPath(":runners:flink:${latestFli addTestJavaJarCreator("SparkRunner", tasks.getByPath(":runners:spark:2:job-server:shadowJar")) def addTestFlinkUberJar(boolean saveMainSession) { - project.tasks.create(name: "testUberJarFlinkRunner${saveMainSession ? 'SaveMainSession' : ''}") { + project.tasks.register("testUberJarFlinkRunner${saveMainSession ? 'SaveMainSession' : ''}") { dependsOn ":runners:flink:${latestFlinkVersion}:job-server:shadowJar" dependsOn ":runners:flink:${latestFlinkVersion}:job-server:miniCluster" dependsOn pythonContainerTask diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 2c34f5b0482ca..e7eb3abf62ca3 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -102,7 +102,7 @@ extras = test,gcp,interactive,dataframe,aws commands = -rm .coverage {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" "--cov-report=xml --cov=. --cov-append" - codecov + codecov -F python [testenv:py37-lint] # Don't set TMPDIR to avoid "AF_UNIX path too long" errors in pylint. diff --git a/settings.gradle.kts b/settings.gradle.kts index 39ebb59bd685c..6796ad0b09d70 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -180,6 +180,7 @@ include(":sdks:java:io:kinesis:expansion-service") include(":sdks:java:io:kudu") include(":sdks:java:io:mongodb") include(":sdks:java:io:mqtt") +include(":sdks:java:io:neo4j") include(":sdks:java:io:parquet") include(":sdks:java:io:rabbitmq") include(":sdks:java:io:redis") diff --git a/website/ADD_LOGO.md b/website/ADD_LOGO.md index 337890429a153..a478ab8676412 100644 --- a/website/ADD_LOGO.md +++ b/website/ADD_LOGO.md @@ -22,6 +22,7 @@ 1. Fork [Apache Beam](https://github.com/apache/beam) repository 2. Add file with company or project name to the [case-studies](https://github.com/apache/beam/tree/master/website/www/site/content/en/case-studies) folder + e.g., `company.md` 3. Add project/company logo to the [images/logos/powered-by](https://github.com/apache/beam/tree/master/website/www/site/static/images/logos/powered-by) folder. Please use your company/project name e.g. `ricardo.png` diff --git a/website/www/site/assets/scss/_case_study.scss b/website/www/site/assets/scss/_case_study.scss index 4de3561edff26..3be4880a24447 100644 --- a/website/www/site/assets/scss/_case_study.scss +++ b/website/www/site/assets/scss/_case_study.scss @@ -95,7 +95,7 @@ } .case-study-card-img img { - height: 35px; + height: 50px; } .case-study-card-title { @@ -296,11 +296,13 @@ h2.case-study-h2 { img { width: 100%; + max-height: 100px; } } } .case-study-post { + max-width: 1200px; flex: 1; } @@ -310,6 +312,13 @@ h2.case-study-h2 { img { width: 100%; } + + &.vertical-scheme { + text-align: center; + img { + max-width: 480px; + } + } } @media screen and (max-width: $mobile) { diff --git a/website/www/site/content/en/blog/beam-2.36.0.md b/website/www/site/content/en/blog/beam-2.36.0.md index 606c35e6c9801..57edefee044e1 100644 --- a/website/www/site/content/en/blog/beam-2.36.0.md +++ b/website/www/site/content/en/blog/beam-2.36.0.md @@ -52,6 +52,7 @@ notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12319527 * Users may encounter an unexpected java.lang.ArithmeticException when outputting a timestamp for an element further than allowedSkew from an allowed DoFN skew set to a value more than Integer.MAX_VALUE. +* S3 object metadata retrieval broken in Python SDK ([BEAM-13980](https://issues.apache.org/jira/browse/BEAM-13980)) * See a full list of open [issues that affect](https://issues.apache.org/jira/issues/?jql=project%20%3D%20BEAM%20AND%20affectedVersion%20%3D%202.36.0%20ORDER%20BY%20priority%20DESC%2C%20updated%20DESC) this version. diff --git a/website/www/site/content/en/case-studies/hop.md b/website/www/site/content/en/case-studies/hop.md index c38f334ec726b..ae69c8f05f9a9 100644 --- a/website/www/site/content/en/case-studies/hop.md +++ b/website/www/site/content/en/case-studies/hop.md @@ -1,7 +1,7 @@ --- title: "Beam visual pipeline development with Hop" name: "Neo4j" -icon: /images/logos/powered-by/hop.png +icon: /images/logos/powered-by/hop.svg category: study cardTitle: "Visual Apache Beam Pipeline Design and Orchestration with Apache Hop" cardDescription: "Apache Hop is an open source data orchestration and engineering platform that extends Apache Beam with visual pipeline lifecycle management. Neo4j’s Chief Solution Architect and Apache Hop’s co-founder, Matt Casters, sees Apache Beam as a driving force behind Hop." @@ -25,7 +25,7 @@ limitations under the License.

- +

diff --git a/website/www/site/content/en/case-studies/paloalto.md b/website/www/site/content/en/case-studies/paloalto.md new file mode 100644 index 0000000000000..0fdba878fbcd5 --- /dev/null +++ b/website/www/site/content/en/case-studies/paloalto.md @@ -0,0 +1,311 @@ +--- +title: "Real-time Event Stream Processing at Scale for Palo Alto Networks" +name: "Palo Alto" +icon: "/images/logos/powered-by/paloalto.png" +category: "study" +cardTitle: "Real-time Event Stream Processing at Scale for Palo Alto Networks" +cardDescription: "Palo Alto Networks is a global cybersecurity leader that deals with processing hundreds of billions of +security events per day in real-time, which is on the high end of the industry. Apache Beam provides a high-performing, +reliable, and resilient data processing framework to support this scale. With Apache Beam, Palo Alto Networks ultimately +achieved high performance and low latency, and reduced processing costs by 60%." +authorName: "Talat Uyarer" +authorPosition: "Sr Principal Software Engineer" +authorImg: /images/case-study/paloalto/talat_uyarer.png +--- + +

+
+ +
+
+

+ “I know one thing: Beam is very powerful and the abstraction is its most significant feature. With the right abstraction we have the flexibility to run workloads where needed. Thanks to Beam, we are not locked to any vendor, and we don’t need to change anything else if we make the switch.” +

+
+
+ +
+
+
+ Talat Uyarer +
+
+ Sr Principal Software Engineer +
+
+
+
+
+
+ +# Real-time Event Stream Processing at Scale for Palo Alto Networks + +## Background + +[Palo Alto Networks, Inc.](https://www.paloaltonetworks.com/) is a global cybersecurity leader with a comprehensive +portfolio of enterprise products. Palo Alto Networks protects and provides visibility, trusted intelligence, automation, +and flexibility to [over 85K customers](https://www.paloaltonetworks.com/about-us) across clouds, networks, and devices. + +Palo Alto Networks’ integrated security operations platform - [Cortex™](https://www.paloaltonetworks.com/cortex) - +applies AI and machine learning to enable security automation, advanced threat intelligence, and effective rapid +security responses for Palo Alto Networks’ customers. (Cortex™ Data +Lake)[https://www.paloaltonetworks.com/cortex/cortex-data-lake] infrastructure collects, integrates, and normalizes +enterprises’ security data combined with trillions of multi-source artifacts. + +Cortex™ data infrastructure processes ~10 millions of security log events per second currently, at ~3 PB per day, which +are on the high end of real-time streaming processing scale in the industry. Palo Alto Networks’ Sr Principal Software +Engineer, Talat Uyarer, shared insights on how Apache Beam provides a high-performing, reliable, and resilient data +processing framework to support this scale. + +## Large-scale Streaming Infrastructure + +When building the data infrastructure from the ground up, Palo Alto Networks’ Cortex Data Lake team faced a challenging +task. We needed to ensure that the Cortex platform could stream and process petabyte-sized data coming from customers’ +firewalls, networks, and all kinds of devices to customers and internal apps with low latency and perfect quality. + +
+ Cortex™ Data Lake +
+ +To meet the SLAs, the Cortex Data Lake team had to design a large-scale data infrastructure for real-time processing and +reduce time-to-value. One of their initial architectural decisions was to leverage Apache Beam, the industry standard +for unified distributed processing, due to its portability and abstraction. + +
+

+ Beam is very flexible, its abstraction from implementation details of distributed data processing is wonderful for delivering proofs of concept really fast. +

+
+
+ +
+
+
+ Talat Uyarer +
+
+ Sr Principal Software Engineer +
+
+
+
+ +Apache Beam provides a variety of runners, offering freedom of choice between different data processing engines. Palo +Alto Networks’ data infrastructure is hosted entirely on [Google Cloud Platform](https://cloud.google.com/gcp/), +and [with Apache Beam Dataflow runner](https://beam.apache.org/documentation/runners/capability-matrix/), we could +easily benefit from [Google Cloud Dataflow](https://cloud.google.com/dataflow)’s managed service and +[autotuning](https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#horizontal-autoscaling) capabilities. +Apache Kafka was selected as the message broker for the backend, and all events were stored as binary data with a common +schema on multiple Kafka clusters. + +The Cortex Data Lake team considered the option of having separate data processing infrastructures for each customer, +with multiple upstream applications creating their own streaming jobs, consuming and processing events from Kafka +directly. Therefore we are building a multi-tenants system. However, the team anticipated possible issues related to +Kafka migrations and partition creation, as well as a lack of visibility into the tenant use cases, which might arise +when having multiple infrastructures. + +Hence, the Cortex Data Lake team took a common streaming infrastructure approach. At the core of the common data +infrastructure, Apache Beam served as a unified programming model to implement business logic just once for all internal +and customer tenant applications. + +The first data workflows that the Cortex Data Lake team implemented were simple: reading from Kafka, creating a batch +job, and writing the results to sink. The release of +the [Apache Beam version with SQL support](https://beam.apache.org/get-started/downloads/#releases) opened up new +possibilities. [Beam Calcite SQL](https://beam.apache.org/documentation/dsls/sql/calcite/overview/) provides full +support for [complex Apache Calcite data types](https://beam.apache.org/documentation/dsls/sql/calcite/data-types/), +including nested rows, in SQL statements, so developers can use SQL queries in an Apache Beam pipeline for composite +transforms. The Cortex Data Lake team decided to take advantage of the +[Beam SQL](https://beam.apache.org/documentation/dsls/sql/overview/) to write Beam pipelines with standard SQL +statements. + +The main challenge of the common infrastructure was to support a variety of business logic customizations and +user-defined functions and transform them to a variety of sink formats. Tenant applications needed to consume data from +dynamically-changing Kafka clusters, and streaming pipeline [DAGs](https://en.wikipedia.org/wiki/Directed_acyclic_graph) +had to be regenerated if the jobs’ source had been updated. + +The Cortex Data Lake team developed their own “subscription” model that allows tenant applications to “subscribe” to the +streaming job when sending job deployment requests to the REST API service. The Subscription service abstracts tenant +applications from the changes in DAG by storing infrastructure-specific information in metadata service. This way, the +streaming jobs stay in sync with the dynamic Kafka infrastructure. + +
+ Cortex™ Data Lake Subscription Service +
+ +Apache Beam is flexible, it allows creating streaming jobs dynamically, on the fly. The Apache Beam constructs allow for +generic pipeline coding, enabling pipelines that process data even if schemas are not fully defined in advance. Cortex’s +Subscription Service generates Apache Beam pipeline DAG based on the tenant application’s REST payload and submits the +job to the runner. When the job is +running, [Apache Beam SDK’s Kafka I/O](https://beam.apache.org/releases/javadoc/2.4.0/org/apache/beam/sdk/io/kafka/KafkaIO.html) +returns an unbounded collection of Kafka records as +a [PCollection](https://beam.apache.org/releases/javadoc/2.1.0/org/apache/beam/sdk/values/PCollection.html) +. [Apache Avro](https://avro.apache.org/) turns the binary Kafka representation into generic records, which are further +converted to the [Apache Beam Row](https://beam.apache.org/releases/javadoc/2.4.0/org/apache/beam/sdk/values/Row.html) +format. The Row structure supports primitives, byte arrays, and containers, and allows organizing values in the same +order as the schema definition. + +Apache Beam’s cross-language transforms allow the Cortex Data Lake team to execute SQL with Java. The output of +an [SQL Transform](https://beam.apache.org/releases/javadoc/2.7.0/org/apache/beam/sdk/extensions/sql/SqlTransform.html) +performed inside the Apache Beam pipeline is sequentially converted from Beam Row format to a generic record, then to +the output format required by a subscriber application, such as Avro, JSON, CSV, etc. + +Once the base use cases had been implemented, the Cortex Data Lake team turned to more complex transformations, such as +filtering a subset of events directly inside Apache Beam pipelines, and kept looking into customization and +optimization. + +
+

+ We have more than 10 use cases running across customers and apps. More are coming, like the machine learning use cases .... for these use cases, Beam provides a really good programming model. +

+
+
+ +
+
+
+ Talat Uyarer +
+
+ Sr Principal Software Engineer +
+
+
+
+ +Apache Beam provides a pluggable data processing model that seamlessly integrates with various tools and technologies, +which allowed the Cortex Data Lake team to customize their data processing to performance requirements and specific use +cases. + +## Customizing Serialization for Use Cases + +Palo Alto Networks’ streaming data infrastructure deals with hundreds of billions of real-time security events every +day, and even a sub-second difference in processing times is crucial. + +To enhance performance, the Cortex Data Lake team developed their own library for direct serialization and +deserialization. The library reads Avro binary records from Kafka and turns them into the Beam Row format, then converts +the Beam Row format pipeline output to the required sink format. + +This custom library replaced serializing data into generic records with steps optimized for Palo Alto Networks’ specific +use cases. Direct serialization eliminated shuffling and creating additional memory copies from processing steps. + +This customization increased serialization performance 10x times, allowing to process up to 3K events per second per +vCPU with reduced latency and infrastructure costs. + +
+ Direct Serialization from Avro to Beam Row +
+ +## In-flight Streaming Job Updates + +At a scale of thousands of jobs running concurrently, the Cortex Data Lake team faced cases when needed to improve the +pipeline code or fix bugs for an ongoing job. Google Cloud Dataflow provides a way +to [replace an “in-flight” streaming job](https://cloud.google.com/dataflow/docs/guides/updating-a-pipeline) with a new +job that runs an updated Apache Beam pipeline code. However, Palo Alto Networks needed to expand the supported +scenarios. + +To address updating jobs in the dynamically-changing Kafka infrastructure, the Cortex Data Lake team created an +additional workflow in their deployment service +which [drains the jobs](https://cloud.google.com/dataflow/docs/guides/stopping-a-pipeline#drain) if the change +is [not permitted](https://cloud.google.com/dataflow/docs/guides/updating-a-pipeline#UpdateSchemas) by the Dataflow +update and starts a new job with the exact same naming. This internal job replacement workflow allows the Cortex Data +Lake to update the jobs and payloads automatically for all use cases. + +## Handling Schema Changes In Beam SQL + +Another use case that Palo Alto Networks tackled is handling changes in data schemas for ongoing jobs. Apache Beam +allows PCollections to have [schemas](https://beam.apache.org/documentation/programming-guide/#schemas) with named +fields, that are validated at pipeline construction step. When a job is submitted, an execution plan in the form of a +Beam pipeline fragment is generated based on the latest schema. Beam SQL does not yet have built-in support for relaxed +schema compatibility for running jobs. For optimized performance, Beam SQL’s +Schema [RowCoder](https://beam.apache.org/releases/javadoc/2.4.0/org/apache/beam/sdk/coders/RowCoder.html) has a fixed +data format and doesn't handle schema evolution, so it is necessary to restart the jobs to regenerate their execution +plan. At a scale of 10K+ streaming jobs, Cortex Data Lake team wanted to avoid resubmitting the jobs as much as +possible. + +We created an internal workflow to identify the jobs with SQL queries relevant to the schema change. The schema update +workflow stores Reader schema of each job (Avro schema) and Writer schema of each Kafka message (metadata on Kafka +header) in the internal Schema Registry, compares them to the SQL queries of the running jobs, and restarts the affected +jobs only. This optimization allowed them to utilize resources more efficiently. + +## Fine-tuning Performance for Kafka Changes + +With multiple clusters and topics, and over 100K partitions in Kafka, Palo Alto Networks needed to make sure that +actively-running jobs are not being affected by the frequent Kafka infrastructure changes such as cluster migrations or +changes in partition count. + +The Cortex Data Lake team developed several internal Kafka lifecycle support tools, including a “Self Healing” service. +Depending on the amount of traffic per topic coming from a specific tenant, the internal service increases the number of +partitions or creates new topics with fewer partitions. The “Self Healing” service compares the Kafka states in the data +store and then finds and updates all related streaming Apache Beam jobs on Cloud Dataflow automatically. + +With the [release of Apache Beam 2.28.0](https://beam.apache.org/blog/beam-2.28.0/) in early +2021, [the pre-built Kafka I/O dynamic read feature](https://beam.apache.org/releases/javadoc/2.29.0/org/apache/beam/sdk/io/kafka/KafkaIO.html) +provides an out-of-the-box solution for detecting Kafka partition changes to enable cost savings and increased +performance. Kafka I/O uses WatchKafkaTopicPartitionDoFn to emit +new [TopicPartitions](https://kafka.apache.org/24/javadoc/index.html?org/apache/kafka/common/TopicPartition.html), and +allows reading from Kafka topics dynamically when certain partitions are added or stop reading from them once they are +deleted. This feature eliminated the need to create in-house Kafka monitoring tools. + +In addition to performance optimization, the Cortex Data Lake team has been exploring ways to optimize the Cloud +Dataflow costs. We looked into resource usage optimization in cases when streaming jobs consume very few incoming +events. For cost efficiency, Google Cloud Dataflow provides +the [streaming autoscaling](https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#streaming-autoscaling) +feature that adaptively changes the number of workers in response to changes in the load and resource utilization. For +some of Cortex Data Lake team’s use cases, where input data streams may quiesce for prolonged periods of time, we +implemented an internal “Cold Starter” service that analyzes Kafka topics traffic and hibernates pipelines whose input +dries up and reactivates them once their input resumes. + +Talat Uyarer presented the Cortex Data Lake’s experience of building and customizing the large-scale streaming +infrastructure during [Beam Summit 2021](https://2021.beamsummit.org/sessions/large-scale-streaming-infrastructure/). + +
+

+ I really enjoy working with Beam. If you understand its internals, the understanding empowers you to fine-tune the open source, customize it, so that it provides the best performance for your specific use case. +

+
+
+ +
+
+
+ Talat Uyarer +
+
+ Sr Principal Software Engineer +
+
+
+
+ +## Results + +The level of abstraction of Apache Beam empowered the Cortex Data Lake team to create a common infrastructure across +their internal apps and tens of thousands of customers. With Apache Beam, we implement business logic just once and +dynamically generate 10K+ streaming pipelines running in parallel for over 10 use cases. + +The Cortex Data Lake team took advantage of Apache Beam’s portability and pluggability to fine-tune and enhance their +data processing infrastructure with custom libraries and services. Palo Alto Networks ultimately achieved high +performance and low latency, processing 3K+ streaming events per second per vCPU. Combining the benefits of open source +Apache Beam and Cloud Dataflow managed service, we were able to implement use-case specific customizations and reduced +their costs by more than 60%. + +The Apache Beam open source community welcomes and encourages the contributions of its numerous members, such as Palo +Alto Networks, that leverage the powerful capabilities of Apache Beam, bring new optimizations, and empower future +innovation by sharing their expertise and actively participating in the community. + +{{< case_study_feedback "Palo Alto" >}} + +
+
diff --git a/website/www/site/content/en/community/case-study.md b/website/www/site/content/en/community/case-study.md index 4406338c5720d..73556a70ac89e 100644 --- a/website/www/site/content/en/community/case-study.md +++ b/website/www/site/content/en/community/case-study.md @@ -34,3 +34,6 @@ started! Want to tell the world you are using Apache Beam? Just walk through [this instruction](https://github.com/apache/beam/tree/master/website/ADD_LOGO.md) and make it happen! + +The Apache Beam PMC reserves the right to remove logos of companies that are not demeed to be in good standing in the +community. diff --git a/website/www/site/data/io_matrix.yaml b/website/www/site/data/io_matrix.yaml index f06c95f36a8dd..6c3e01aee25ec 100644 --- a/website/www/site/data/io_matrix.yaml +++ b/website/www/site/data/io_matrix.yaml @@ -34,7 +34,7 @@ categories: url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.avroio.html - language: go name: github.com/apache/beam/sdks/go/pkg/beam/io/avroio - url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/avroio + url: https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/avroio - transform: TextIO description: PTransforms for reading and writing text files. implementations: @@ -46,7 +46,7 @@ categories: url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.textio.html - language: go name: github.com/apache/beam/sdks/go/pkg/beam/io/textio - url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/textio + url: https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/textio - transform: TFRecordIO description: PTransforms for reading and writing [TensorFlow TFRecord](https://www.tensorflow.org/tutorials/load_data/tfrecord) files. implementations: @@ -119,7 +119,7 @@ categories: url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.gcsfilesystem.html - language: go name: github.com/apache/beam/sdks/go/pkg/beam/io/filesystem/gcs - url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/filesystem/gcs + url: https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/filesystem/gcs - transform: LocalFileSystem description: "`FileSystem` implementation for accessing files on disk." implementations: @@ -131,7 +131,7 @@ categories: url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.localfilesystem.html - language: go name: github.com/apache/beam/sdks/go/pkg/beam/io/filesystem/local - url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/filesystem/local + url: https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/filesystem/local - transform: S3FileSystem description: "`FileSystem` implementation for [Amazon S3](https://aws.amazon.com/s3/)." implementations: @@ -143,7 +143,7 @@ categories: implementations: - language: go name: github.com/apache/beam/sdks/go/pkg/beam/io/filesystem/memfs - url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/filesystem/memfs + url: https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/filesystem/memfs - name: Messaging description: These I/O connectors typically involve working with unbounded sources that come from messaging sources. rows: @@ -182,7 +182,7 @@ categories: url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.external.gcp.pubsub.html - language: go name: github.com/apache/beam/sdks/go/pkg/beam/io/pubsubio - url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/pubsubio + url: https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/pubsubio - transform: JmsIO description: An unbounded source for [JMS](https://www.oracle.com/java/technologies/java-message-service.html) destinations (queues or topics). implementations: @@ -278,7 +278,7 @@ categories: url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.bigquery.html - language: go name: github.com/apache/beam/sdks/go/pkg/beam/io/bigqueryio - url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/bigqueryio + url: https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/bigqueryio - transform: BigTableIO description: Read from (only for Java SDK) and write to [Google Cloud Bigtable](https://cloud.google.com/bigtable/). implementations: @@ -360,7 +360,7 @@ categories: implementations: - language: go name: github.com/apache/beam/sdks/go/pkg/beam/io/databaseio - url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/databaseio + url: https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/databaseio - name: Miscellaneous description: Miscellaneous I/O sources. rows: diff --git a/website/www/site/layouts/case-studies/list.html b/website/www/site/layouts/case-studies/list.html index de262d3d9a1c6..ee1f19bb04d0d 100644 --- a/website/www/site/layouts/case-studies/list.html +++ b/website/www/site/layouts/case-studies/list.html @@ -14,7 +14,7 @@

Case Studies

Apache Beam powers many of today’s leading projects, industry-specific use cases, and - startups

+ startups.

{{ $pages := .Pages.ByPublishDate.Reverse }}
{{ range where $pages "Params.category" "study" }} diff --git a/website/www/site/layouts/partials/section-menu/en/sdks.html b/website/www/site/layouts/partials/section-menu/en/sdks.html index ef0b2bbdaaf48..4b9f3f2c6055c 100644 --- a/website/www/site/layouts/partials/section-menu/en/sdks.html +++ b/website/www/site/layouts/partials/section-menu/en/sdks.html @@ -48,7 +48,7 @@ Go
  • Go SDK overview
  • -
  • Go SDK API reference Go SDK API reference External link.
  • diff --git a/website/www/site/static/images/case-study/paloalto/data_lake_scheme.png b/website/www/site/static/images/case-study/paloalto/data_lake_scheme.png new file mode 100644 index 0000000000000..857e0836bfcd1 Binary files /dev/null and b/website/www/site/static/images/case-study/paloalto/data_lake_scheme.png differ diff --git a/website/www/site/static/images/case-study/paloalto/direct_serialization.png b/website/www/site/static/images/case-study/paloalto/direct_serialization.png new file mode 100644 index 0000000000000..78d4fa36fbeb6 Binary files /dev/null and b/website/www/site/static/images/case-study/paloalto/direct_serialization.png differ diff --git a/website/www/site/static/images/case-study/paloalto/subscription_service_scheme.png b/website/www/site/static/images/case-study/paloalto/subscription_service_scheme.png new file mode 100644 index 0000000000000..c6d35e7cb63bd Binary files /dev/null and b/website/www/site/static/images/case-study/paloalto/subscription_service_scheme.png differ diff --git a/website/www/site/static/images/case-study/paloalto/talat_uyarer.png b/website/www/site/static/images/case-study/paloalto/talat_uyarer.png new file mode 100644 index 0000000000000..d1e5e1897eddc Binary files /dev/null and b/website/www/site/static/images/case-study/paloalto/talat_uyarer.png differ diff --git a/website/www/site/static/images/logos/powered-by/hop.png b/website/www/site/static/images/logos/powered-by/hop.png deleted file mode 100644 index 58c25b633fa53..0000000000000 Binary files a/website/www/site/static/images/logos/powered-by/hop.png and /dev/null differ diff --git a/website/www/site/static/images/logos/powered-by/hop.svg b/website/www/site/static/images/logos/powered-by/hop.svg new file mode 100644 index 0000000000000..728eae3f4911b --- /dev/null +++ b/website/www/site/static/images/logos/powered-by/hop.svg @@ -0,0 +1,71 @@ + + +image/svg+xml + + diff --git a/website/www/site/static/images/logos/powered-by/paloalto.png b/website/www/site/static/images/logos/powered-by/paloalto.png new file mode 100644 index 0000000000000..2e7472d51a349 Binary files /dev/null and b/website/www/site/static/images/logos/powered-by/paloalto.png differ