You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2020/04/20 20:24:29 UTC

[GitHub] [beam] lostluck opened a new pull request #11468: [BEAM-9789] Fix lock error. Add test.

lostluck opened a new pull request #11468:
URL: https://github.com/apache/beam/pull/11468


   Fix locking error introduced to the harness when resolving https://issues.apache.org/jira/browse/BEAM-9618 
   
   The root is: Always check the error *before* locking. And if defer mu.Unlock() isn't used (or as in this case, able to be used), always unlock before returning the error.
   
   Deadlocks can prevent the error case you're returning from being transmitted.
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [ ] Format the pull request title like `[BEAM-XXX] Fixes bug in ApproximateQuantiles`, where you replace `BEAM-XXX` with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   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).
   
   Post-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   Lang | SDK | Apex | Dataflow | Flink | Gearpump | Samza | Spark
   --- | --- | --- | --- | --- | --- | --- | ---
   Go | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/)
   Java | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/)
   Python | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/) | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Python2_PVR_Flink_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Python2_PVR_Flink_Cron/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python35_VR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python35_VR_Flink/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/)
   XLang | --- | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/)
   
   Pre-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   --- |Java | Python | Go | Website
   --- | --- | --- | --- | ---
   Non-portable | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/) 
   Portable | --- | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/) | --- | ---
   
   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.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] robertwb commented on issue #11468: [BEAM-9789] Fix lock error. Add test.

Posted by GitBox <gi...@apache.org>.
robertwb commented on issue #11468:
URL: https://github.com/apache/beam/pull/11468#issuecomment-617325661


   Thanks for finding and fixing this.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lostluck commented on a change in pull request #11468: [BEAM-9789] Fix lock error. Add test.

Posted by GitBox <gi...@apache.org>.
lostluck commented on a change in pull request #11468:
URL: https://github.com/apache/beam/pull/11468#discussion_r411797808



##########
File path: sdks/go/pkg/beam/core/runtime/harness/harness_test.go
##########
@@ -0,0 +1,169 @@
+// 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 harness
+
+import (
+	"fmt"
+	"strings"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/exec"
+	fnpb "github.com/apache/beam/sdks/go/pkg/beam/model/fnexecution_v1"
+	pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	"github.com/golang/protobuf/proto"
+)
+
+// validDescriptor describes a valid pipeline with a source and a sink, but doesn't do anything else.
+func validDescriptor(t *testing.T) *fnpb.ProcessBundleDescriptor {
+	t.Helper()
+	port := &fnpb.RemoteGrpcPort{
+		CoderId: "c1",
+		ApiServiceDescriptor: &pipepb.ApiServiceDescriptor{
+			Url: "hostname:port",
+		},
+	}
+	portBytes, err := proto.Marshal(port)
+	if err != nil {
+		t.Fatalf("bad port: %v", err)
+	}
+	return &fnpb.ProcessBundleDescriptor{
+		Id: "test",
+		Transforms: map[string]*pipepb.PTransform{
+			"source": &pipepb.PTransform{
+				Spec: &pipepb.FunctionSpec{
+					Urn:     "beam:runner:source:v1",
+					Payload: portBytes,
+				},
+				Outputs: map[string]string{
+					"o1": "p1",
+				},
+			},
+			"sink": &pipepb.PTransform{
+				Spec: &pipepb.FunctionSpec{
+					Urn:     "beam:runner:sink:v1",
+					Payload: portBytes,
+				},
+				Inputs: map[string]string{
+					"i1": "p1",
+				},
+			},
+		},
+		Pcollections: map[string]*pipepb.PCollection{
+			"p1": &pipepb.PCollection{
+				CoderId: "c1",
+			},
+		},
+		Coders: map[string]*pipepb.Coder{
+			"c1": &pipepb.Coder{
+				Spec: &pipepb.FunctionSpec{
+					Urn: "beam:coder:windowed_value:v1",
+				},
+				ComponentCoderIds: []string{"c2", "c3"},
+			},
+			"c2": &pipepb.Coder{
+				Spec: &pipepb.FunctionSpec{
+					Urn: "beam:coder:varint:v1",
+				},
+			},
+			"c3": &pipepb.Coder{
+				Spec: &pipepb.FunctionSpec{
+					Urn: "beam:coder:global_window:v1",
+				},
+			},
+		},
+	}
+
+}
+
+func invalidDescriptor(t *testing.T) *fnpb.ProcessBundleDescriptor {
+	return &fnpb.ProcessBundleDescriptor{}
+}
+
+func TestControl_getOrCreatePlan(t *testing.T) {
+	testBDID := bundleDescriptorID("test")
+	testPlan, err := exec.UnmarshalPlan(validDescriptor(t))
+	if err != nil {
+		t.Fatal("bad testPlan")
+	}
+	tests := []struct {
+		name               string
+		lookupErr, planErr error
+		lookupDesc         *fnpb.ProcessBundleDescriptor
+		descriptors        map[bundleDescriptorID]*fnpb.ProcessBundleDescriptor
+		plans              map[bundleDescriptorID][]*exec.Plan
+	}{
+		{
+			name:       "OK",
+			lookupDesc: validDescriptor(t),
+		}, {
+			name: "cachedDescriptor",
+			descriptors: map[bundleDescriptorID]*fnpb.ProcessBundleDescriptor{
+				testBDID: validDescriptor(t),
+			},
+		}, {
+			name: "cachedPlan",
+			plans: map[bundleDescriptorID][]*exec.Plan{
+				testBDID: []*exec.Plan{testPlan},
+			},
+		}, {
+			name:      "badLookup",

Review comment:
       Ah good catch. It should be a plan error instead, but I'll remove the case.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lostluck commented on issue #11468: [BEAM-9789] Fix lock error. Add test.

Posted by GitBox <gi...@apache.org>.
lostluck commented on issue #11468:
URL: https://github.com/apache/beam/pull/11468#issuecomment-616789253


   R: @youngoli 
   cc: @robertwb 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] youngoli commented on a change in pull request #11468: [BEAM-9789] Fix lock error. Add test.

Posted by GitBox <gi...@apache.org>.
youngoli commented on a change in pull request #11468:
URL: https://github.com/apache/beam/pull/11468#discussion_r411793672



##########
File path: sdks/go/pkg/beam/core/runtime/harness/harness_test.go
##########
@@ -0,0 +1,169 @@
+// 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 harness
+
+import (
+	"fmt"
+	"strings"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/exec"
+	fnpb "github.com/apache/beam/sdks/go/pkg/beam/model/fnexecution_v1"
+	pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	"github.com/golang/protobuf/proto"
+)
+
+// validDescriptor describes a valid pipeline with a source and a sink, but doesn't do anything else.
+func validDescriptor(t *testing.T) *fnpb.ProcessBundleDescriptor {
+	t.Helper()
+	port := &fnpb.RemoteGrpcPort{
+		CoderId: "c1",
+		ApiServiceDescriptor: &pipepb.ApiServiceDescriptor{
+			Url: "hostname:port",
+		},
+	}
+	portBytes, err := proto.Marshal(port)
+	if err != nil {
+		t.Fatalf("bad port: %v", err)
+	}
+	return &fnpb.ProcessBundleDescriptor{
+		Id: "test",
+		Transforms: map[string]*pipepb.PTransform{
+			"source": &pipepb.PTransform{
+				Spec: &pipepb.FunctionSpec{
+					Urn:     "beam:runner:source:v1",
+					Payload: portBytes,
+				},
+				Outputs: map[string]string{
+					"o1": "p1",
+				},
+			},
+			"sink": &pipepb.PTransform{
+				Spec: &pipepb.FunctionSpec{
+					Urn:     "beam:runner:sink:v1",
+					Payload: portBytes,
+				},
+				Inputs: map[string]string{
+					"i1": "p1",
+				},
+			},
+		},
+		Pcollections: map[string]*pipepb.PCollection{
+			"p1": &pipepb.PCollection{
+				CoderId: "c1",
+			},
+		},
+		Coders: map[string]*pipepb.Coder{
+			"c1": &pipepb.Coder{
+				Spec: &pipepb.FunctionSpec{
+					Urn: "beam:coder:windowed_value:v1",
+				},
+				ComponentCoderIds: []string{"c2", "c3"},
+			},
+			"c2": &pipepb.Coder{
+				Spec: &pipepb.FunctionSpec{
+					Urn: "beam:coder:varint:v1",
+				},
+			},
+			"c3": &pipepb.Coder{
+				Spec: &pipepb.FunctionSpec{
+					Urn: "beam:coder:global_window:v1",
+				},
+			},
+		},
+	}
+
+}
+
+func invalidDescriptor(t *testing.T) *fnpb.ProcessBundleDescriptor {
+	return &fnpb.ProcessBundleDescriptor{}
+}
+
+func TestControl_getOrCreatePlan(t *testing.T) {
+	testBDID := bundleDescriptorID("test")
+	testPlan, err := exec.UnmarshalPlan(validDescriptor(t))
+	if err != nil {
+		t.Fatal("bad testPlan")
+	}
+	tests := []struct {
+		name               string
+		lookupErr, planErr error
+		lookupDesc         *fnpb.ProcessBundleDescriptor
+		descriptors        map[bundleDescriptorID]*fnpb.ProcessBundleDescriptor
+		plans              map[bundleDescriptorID][]*exec.Plan
+	}{
+		{
+			name:       "OK",
+			lookupDesc: validDescriptor(t),
+		}, {
+			name: "cachedDescriptor",
+			descriptors: map[bundleDescriptorID]*fnpb.ProcessBundleDescriptor{
+				testBDID: validDescriptor(t),
+			},
+		}, {
+			name: "cachedPlan",
+			plans: map[bundleDescriptorID][]*exec.Plan{
+				testBDID: []*exec.Plan{testPlan},
+			},
+		}, {
+			name:      "badLookup",

Review comment:
       What's the difference between the "badLookup" and "nilLookup" tests? Doesn't `lookupDesc` default to nil if it's not specified, making these two tests identical except with different `lookupErr`s?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org