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 2023/01/05 05:47:56 UTC

[GitHub] [beam] riteshghorse commented on a diff in pull request #24772: [#21391]Increase unit testing coverage in the exec package

riteshghorse commented on code in PR #24772:
URL: https://github.com/apache/beam/pull/24772#discussion_r1062123696


##########
sdks/go/pkg/beam/core/runtime/exec/translate_test.go:
##########
@@ -318,3 +320,197 @@ func makeWindowMappingFn(w *window.Fn) (*pipepb.FunctionSpec, error) {
 	}
 	return wFn, nil
 }
+
+func TestInputIdToIndex(t *testing.T) {
+	tests := []struct {
+		in   string
+		want int
+	}{
+		{ // does not start with i
+			"90",
+			0,
+		},
+		{ // start with i
+			"i0",
+			0,
+		},
+		{
+			"i1",
+			1,
+		},
+		{
+			"i10",
+			10,
+		},
+	}
+
+	for _, test := range tests {
+		got, err := inputIdToIndex(test.in)
+		if !strings.HasPrefix(test.in, "i") {
+			if err == nil {
+				t.Errorf("Should return err when string does not has a prefix of i, but didn't. inputIdToIndex(%v) = (%v, %v)", test.in, got, err)
+			}
+		} else {
+			if got != test.want {
+				t.Errorf("Can not correctly convert inputId to index. inputIdToIndex(%v) = (%v, %v), want %v", test.in, got, err, test.want)
+			}
+		}
+	}
+}
+
+func TestIndexToInputId(t *testing.T) {
+	tests := []struct {
+		in   int
+		want string
+	}{
+		{
+			1,
+			"i1",
+		},
+		{
+			1000,
+			"i1000",
+		},
+	}
+
+	for _, test := range tests {
+		got := indexToInputId(test.in)
+		if got != test.want {
+			t.Errorf("Can not correctly convert index to inputId. indexToInputId(%v) = (%v), want %v", test.in, got, test.want)
+		}
+	}
+}
+
+func TestUnmarshalPort(t *testing.T) {
+	var port fnpb.RemoteGrpcPort
+
+	tests := []struct {
+		inputData   []byte
+		outputPort  Port
+		outputStr   string
+		outputError error
+	}{
+		{
+			inputData:   []byte{},
+			outputPort:  Port{URL: port.GetApiServiceDescriptor().GetUrl()},
+			outputStr:   fnpb.RemoteGrpcPort{}.CoderId,
+			outputError: nil,
+		},
+	}
+
+	for _, test := range tests {
+		port, str, err := unmarshalPort(test.inputData)
+		if err != nil && test.outputError == nil {
+			t.Errorf("There is an error where should not be. unmarshalPort(%v) = (%v, %v, %v), want (%v, %v, %v)", test.inputData, port, str, err, test.outputPort, test.outputStr, test.outputError)
+		} else if err != nil && err != test.outputError {
+			t.Errorf("There is an error that does not meet expectation. unmarshalPort(%v) = (%v, %v, %v), want (%v, %v, %v)", test.inputData, port, str, err, test.outputPort, test.outputStr, test.outputError)
+		} else if port != test.outputPort {
+			t.Errorf("The output port is not right. unmarshalPort(%v) = (%v, %v, %v), want (%v, %v, %v)", test.inputData, port, str, err, test.outputPort, test.outputStr, test.outputError)
+		} else if str != test.outputStr {
+			t.Errorf("The output string is not right. unmarshalPort(%v) = (%v, %v, %v), want (%v, %v, %v)", test.inputData, port, str, err, test.outputPort, test.outputStr, test.outputError)
+		}
+	}
+}
+
+func TestUnmarshalPlan(t *testing.T) {
+	transform := pipepb.PTransform{
+		Spec: &pipepb.FunctionSpec{
+			Urn: urnDataSource,
+		},
+		Outputs: map[string]string{},
+	}
+	tests := []struct {
+		name        string
+		inputDesc   *fnpb.ProcessBundleDescriptor
+		outputPlan  *Plan
+		outputError error
+	}{
+		{
+			name: "test_no_root_units",
+			inputDesc: &fnpb.ProcessBundleDescriptor{
+				Id:         "",
+				Transforms: map[string]*pipepb.PTransform{},
+			},
+			outputPlan:  nil,
+			outputError: errors.Errorf("no root units"),
+		},
+		{
+			name: "test_zero_transform",
+			inputDesc: &fnpb.ProcessBundleDescriptor{
+				Id: "",
+				Transforms: map[string]*pipepb.PTransform{
+					"": {},
+				},
+			},
+			outputPlan:  nil,
+			outputError: errors.Errorf("no root units"),
+		},
+		{
+			name: "test_transform_outputs_length_not_one",
+			inputDesc: &fnpb.ProcessBundleDescriptor{
+				Id: "",
+				Transforms: map[string]*pipepb.PTransform{
+					"": &transform,
+				},
+			},
+			outputPlan:  nil,
+			outputError: errors.Errorf("expected one output from DataSource, got %v", transform.GetOutputs()),
+		},
+	}
+
+	for _, test := range tests {
+		t.Run(test.name, func(t *testing.T) {
+			plan, err := UnmarshalPlan(test.inputDesc)
+			if err != nil && test.outputError == nil {
+				t.Errorf("There is an error where should not be. UnmarshalPlan(%v) = (%v, %v), want (%v, %v)", test.inputDesc, plan, err, test.outputPlan, test.outputError)
+			} else if err != nil && !reflect.DeepEqual(err, test.outputError) {
+				t.Errorf("There is an error that does not meet expectation. UnmarshalPlan(%v) = (%v, %v), want (%v, %v)", test.inputDesc, plan, err, test.outputPlan, test.outputError)
+			} else if !reflect.DeepEqual(plan, test.outputPlan) {
+				t.Errorf("The output builder is not right. UnmarshalPlan(%v) = (%v, %v), want (%v, %v)", test.inputDesc, plan, err, test.outputPlan, test.outputError)
+			}
+		})
+	}
+}
+
+func TestNewBuilder(t *testing.T) {
+	descriptor := fnpb.ProcessBundleDescriptor{
+		Id:         "",
+		Transforms: map[string]*pipepb.PTransform{},
+	}
+	tests := []struct {
+		name          string
+		inputDesc     *fnpb.ProcessBundleDescriptor
+		outputBuilder *builder
+		outputError   error
+	}{
+		{
+			name:      "test_1",
+			inputDesc: &descriptor,
+			outputBuilder: &builder{
+				desc:      &descriptor,
+				coders:    graphx.NewCoderUnmarshaller(descriptor.GetCoders()),
+				prev:      make(map[string]int),
+				succ:      make(map[string][]linkID),
+				windowing: make(map[string]*window.WindowingStrategy),
+				nodes:     make(map[string]*PCollection),
+				links:     make(map[linkID]Node),
+				units:     nil,
+				idgen:     &GenID{},
+			},
+			outputError: nil,
+		},
+	}
+
+	for _, test := range tests {
+		t.Run(test.name, func(t *testing.T) {
+			b, err := newBuilder(test.inputDesc)
+			if err != nil && test.outputError == nil {
+				t.Errorf("There is an error where should not be. newBuilder(%v) = (%v, %v), want (%v, %v)", test.inputDesc, b, err, test.outputBuilder, test.outputError)
+			} else if err != nil && err != test.outputError {
+				t.Errorf("There is an error that does not meet expectation. newBuilder(%v) = (%v, %v), want (%v, %v)", test.inputDesc, b, err, test.outputBuilder, test.outputError)

Review Comment:
   ```suggestion
   				t.Errorf("got an unexpected error: %v, want: %v", err, test.outputError)
   ```



##########
sdks/go/pkg/beam/core/runtime/exec/translate_test.go:
##########
@@ -318,3 +320,197 @@ func makeWindowMappingFn(w *window.Fn) (*pipepb.FunctionSpec, error) {
 	}
 	return wFn, nil
 }
+
+func TestInputIdToIndex(t *testing.T) {
+	tests := []struct {
+		in   string
+		want int
+	}{
+		{ // does not start with i
+			"90",
+			0,
+		},
+		{ // start with i
+			"i0",
+			0,
+		},
+		{
+			"i1",
+			1,
+		},
+		{
+			"i10",
+			10,
+		},
+	}
+
+	for _, test := range tests {
+		got, err := inputIdToIndex(test.in)
+		if !strings.HasPrefix(test.in, "i") {
+			if err == nil {
+				t.Errorf("Should return err when string does not has a prefix of i, but didn't. inputIdToIndex(%v) = (%v, %v)", test.in, got, err)
+			}
+		} else {
+			if got != test.want {
+				t.Errorf("Can not correctly convert inputId to index. inputIdToIndex(%v) = (%v, %v), want %v", test.in, got, err, test.want)
+			}
+		}
+	}
+}
+
+func TestIndexToInputId(t *testing.T) {
+	tests := []struct {
+		in   int
+		want string
+	}{
+		{
+			1,
+			"i1",
+		},
+		{
+			1000,
+			"i1000",
+		},
+	}
+
+	for _, test := range tests {
+		got := indexToInputId(test.in)
+		if got != test.want {
+			t.Errorf("Can not correctly convert index to inputId. indexToInputId(%v) = (%v), want %v", test.in, got, test.want)
+		}
+	}
+}
+
+func TestUnmarshalPort(t *testing.T) {
+	var port fnpb.RemoteGrpcPort
+
+	tests := []struct {
+		inputData   []byte
+		outputPort  Port
+		outputStr   string
+		outputError error
+	}{
+		{
+			inputData:   []byte{},
+			outputPort:  Port{URL: port.GetApiServiceDescriptor().GetUrl()},
+			outputStr:   fnpb.RemoteGrpcPort{}.CoderId,
+			outputError: nil,
+		},
+	}
+
+	for _, test := range tests {
+		port, str, err := unmarshalPort(test.inputData)
+		if err != nil && test.outputError == nil {
+			t.Errorf("There is an error where should not be. unmarshalPort(%v) = (%v, %v, %v), want (%v, %v, %v)", test.inputData, port, str, err, test.outputPort, test.outputStr, test.outputError)
+		} else if err != nil && err != test.outputError {
+			t.Errorf("There is an error that does not meet expectation. unmarshalPort(%v) = (%v, %v, %v), want (%v, %v, %v)", test.inputData, port, str, err, test.outputPort, test.outputStr, test.outputError)
+		} else if port != test.outputPort {
+			t.Errorf("The output port is not right. unmarshalPort(%v) = (%v, %v, %v), want (%v, %v, %v)", test.inputData, port, str, err, test.outputPort, test.outputStr, test.outputError)
+		} else if str != test.outputStr {
+			t.Errorf("The output string is not right. unmarshalPort(%v) = (%v, %v, %v), want (%v, %v, %v)", test.inputData, port, str, err, test.outputPort, test.outputStr, test.outputError)
+		}
+	}
+}
+
+func TestUnmarshalPlan(t *testing.T) {
+	transform := pipepb.PTransform{
+		Spec: &pipepb.FunctionSpec{
+			Urn: urnDataSource,
+		},
+		Outputs: map[string]string{},
+	}
+	tests := []struct {
+		name        string
+		inputDesc   *fnpb.ProcessBundleDescriptor
+		outputPlan  *Plan
+		outputError error
+	}{
+		{
+			name: "test_no_root_units",
+			inputDesc: &fnpb.ProcessBundleDescriptor{
+				Id:         "",
+				Transforms: map[string]*pipepb.PTransform{},
+			},
+			outputPlan:  nil,
+			outputError: errors.Errorf("no root units"),

Review Comment:
   use `errors.New()` when format string is not used



##########
sdks/go/pkg/beam/core/runtime/exec/translate_test.go:
##########
@@ -318,3 +320,197 @@ func makeWindowMappingFn(w *window.Fn) (*pipepb.FunctionSpec, error) {
 	}
 	return wFn, nil
 }
+
+func TestInputIdToIndex(t *testing.T) {
+	tests := []struct {
+		in   string
+		want int
+	}{
+		{ // does not start with i
+			"90",
+			0,
+		},
+		{ // start with i
+			"i0",
+			0,
+		},
+		{
+			"i1",
+			1,
+		},
+		{
+			"i10",
+			10,
+		},
+	}
+
+	for _, test := range tests {
+		got, err := inputIdToIndex(test.in)
+		if !strings.HasPrefix(test.in, "i") {
+			if err == nil {
+				t.Errorf("Should return err when string does not has a prefix of i, but didn't. inputIdToIndex(%v) = (%v, %v)", test.in, got, err)
+			}
+		} else {
+			if got != test.want {
+				t.Errorf("Can not correctly convert inputId to index. inputIdToIndex(%v) = (%v, %v), want %v", test.in, got, err, test.want)
+			}
+		}
+	}
+}
+
+func TestIndexToInputId(t *testing.T) {
+	tests := []struct {
+		in   int
+		want string
+	}{
+		{
+			1,
+			"i1",
+		},
+		{
+			1000,
+			"i1000",
+		},
+	}
+
+	for _, test := range tests {
+		got := indexToInputId(test.in)
+		if got != test.want {
+			t.Errorf("Can not correctly convert index to inputId. indexToInputId(%v) = (%v), want %v", test.in, got, test.want)
+		}
+	}
+}
+
+func TestUnmarshalPort(t *testing.T) {
+	var port fnpb.RemoteGrpcPort
+
+	tests := []struct {
+		inputData   []byte
+		outputPort  Port
+		outputStr   string
+		outputError error
+	}{
+		{
+			inputData:   []byte{},
+			outputPort:  Port{URL: port.GetApiServiceDescriptor().GetUrl()},
+			outputStr:   fnpb.RemoteGrpcPort{}.CoderId,
+			outputError: nil,
+		},
+	}
+
+	for _, test := range tests {
+		port, str, err := unmarshalPort(test.inputData)
+		if err != nil && test.outputError == nil {
+			t.Errorf("There is an error where should not be. unmarshalPort(%v) = (%v, %v, %v), want (%v, %v, %v)", test.inputData, port, str, err, test.outputPort, test.outputStr, test.outputError)
+		} else if err != nil && err != test.outputError {
+			t.Errorf("There is an error that does not meet expectation. unmarshalPort(%v) = (%v, %v, %v), want (%v, %v, %v)", test.inputData, port, str, err, test.outputPort, test.outputStr, test.outputError)
+		} else if port != test.outputPort {
+			t.Errorf("The output port is not right. unmarshalPort(%v) = (%v, %v, %v), want (%v, %v, %v)", test.inputData, port, str, err, test.outputPort, test.outputStr, test.outputError)
+		} else if str != test.outputStr {
+			t.Errorf("The output string is not right. unmarshalPort(%v) = (%v, %v, %v), want (%v, %v, %v)", test.inputData, port, str, err, test.outputPort, test.outputStr, test.outputError)
+		}
+	}
+}
+
+func TestUnmarshalPlan(t *testing.T) {
+	transform := pipepb.PTransform{
+		Spec: &pipepb.FunctionSpec{
+			Urn: urnDataSource,
+		},
+		Outputs: map[string]string{},
+	}
+	tests := []struct {
+		name        string
+		inputDesc   *fnpb.ProcessBundleDescriptor
+		outputPlan  *Plan
+		outputError error
+	}{
+		{
+			name: "test_no_root_units",
+			inputDesc: &fnpb.ProcessBundleDescriptor{
+				Id:         "",
+				Transforms: map[string]*pipepb.PTransform{},
+			},
+			outputPlan:  nil,
+			outputError: errors.Errorf("no root units"),
+		},
+		{
+			name: "test_zero_transform",
+			inputDesc: &fnpb.ProcessBundleDescriptor{
+				Id: "",
+				Transforms: map[string]*pipepb.PTransform{
+					"": {},
+				},
+			},
+			outputPlan:  nil,
+			outputError: errors.Errorf("no root units"),

Review Comment:
   use `errors.New()` when format string is not used



##########
sdks/go/pkg/beam/core/runtime/exec/translate_test.go:
##########
@@ -318,3 +320,197 @@ func makeWindowMappingFn(w *window.Fn) (*pipepb.FunctionSpec, error) {
 	}
 	return wFn, nil
 }
+
+func TestInputIdToIndex(t *testing.T) {
+	tests := []struct {
+		in   string
+		want int
+	}{
+		{ // does not start with i
+			"90",
+			0,
+		},
+		{ // start with i
+			"i0",
+			0,
+		},
+		{
+			"i1",
+			1,
+		},
+		{
+			"i10",
+			10,
+		},
+	}
+
+	for _, test := range tests {
+		got, err := inputIdToIndex(test.in)
+		if !strings.HasPrefix(test.in, "i") {
+			if err == nil {
+				t.Errorf("Should return err when string does not has a prefix of i, but didn't. inputIdToIndex(%v) = (%v, %v)", test.in, got, err)
+			}
+		} else {
+			if got != test.want {
+				t.Errorf("Can not correctly convert inputId to index. inputIdToIndex(%v) = (%v, %v), want %v", test.in, got, err, test.want)
+			}
+		}
+	}
+}
+
+func TestIndexToInputId(t *testing.T) {
+	tests := []struct {
+		in   int
+		want string
+	}{
+		{
+			1,
+			"i1",
+		},
+		{
+			1000,
+			"i1000",
+		},
+	}
+
+	for _, test := range tests {
+		got := indexToInputId(test.in)
+		if got != test.want {
+			t.Errorf("Can not correctly convert index to inputId. indexToInputId(%v) = (%v), want %v", test.in, got, test.want)
+		}
+	}
+}
+
+func TestUnmarshalPort(t *testing.T) {
+	var port fnpb.RemoteGrpcPort
+
+	tests := []struct {
+		inputData   []byte
+		outputPort  Port
+		outputStr   string
+		outputError error
+	}{
+		{
+			inputData:   []byte{},
+			outputPort:  Port{URL: port.GetApiServiceDescriptor().GetUrl()},
+			outputStr:   fnpb.RemoteGrpcPort{}.CoderId,
+			outputError: nil,
+		},
+	}
+
+	for _, test := range tests {
+		port, str, err := unmarshalPort(test.inputData)
+		if err != nil && test.outputError == nil {
+			t.Errorf("There is an error where should not be. unmarshalPort(%v) = (%v, %v, %v), want (%v, %v, %v)", test.inputData, port, str, err, test.outputPort, test.outputStr, test.outputError)
+		} else if err != nil && err != test.outputError {
+			t.Errorf("There is an error that does not meet expectation. unmarshalPort(%v) = (%v, %v, %v), want (%v, %v, %v)", test.inputData, port, str, err, test.outputPort, test.outputStr, test.outputError)

Review Comment:
   consider changing this to "got an unexpected error: %v, want: %v".
   similarly in other places where there are similar statements.



-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

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