Skip to content

Commit

Permalink
Merge pull request #13999: [BEAM-11415] Fix misc. bugs affecting Go X…
Browse files Browse the repository at this point in the history
…Lang tests.

[BEAM-11415] Fix misc. bugs affecting Go XLang tests.
  • Loading branch information
youngoli authored Feb 17, 2021
2 parents c8cc2a3 + 130e60e commit edeaf67
Show file tree
Hide file tree
Showing 4 changed files with 27 additions and 2 deletions.
8 changes: 7 additions & 1 deletion sdks/go/pkg/beam/core/runtime/xlangx/expand.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,10 +48,11 @@ func Expand(edge *graph.MultiEdge, ext *graph.ExternalTransform) error {
extTransform := transforms[extTransformID]
for extTransform.UniqueName != "External" {
delete(transforms, extTransformID)
p, err := pipelinex.Normalize(p)
p, err = pipelinex.Normalize(p) // Update root transform IDs.
if err != nil {
return err
}
transforms = p.GetComponents().GetTransforms()
extTransformID = p.GetRootTransformIds()[0]
extTransform = transforms[extTransformID]
}
Expand Down Expand Up @@ -121,5 +122,10 @@ func queryExpansionService(
err = errors.Wrapf(err, "expansion failed")
return nil, errors.WithContextf(err, "expanding transform with ExpansionRequest: %v", req)
}
if len(res.GetError()) != 0 { // ExpansionResponse includes an error.
err := errors.New(res.GetError())
err = errors.Wrapf(err, "expansion failed")
return nil, errors.WithContextf(err, "expanding transform with ExpansionRequest: %v", req)
}
return res, nil
}
2 changes: 2 additions & 0 deletions sdks/go/test/integration/primitives/cogbk_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,8 @@
// See the License for the specific language governing permissions and
// limitations under the License.

// Package primitives contains tests on basic well-known Beam transforms, such
// as ParDo, Flatten, etc.
package primitives

import (
Expand Down
17 changes: 17 additions & 0 deletions sdks/go/test/integration/xlang/xlang.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
// 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 xlang contains integration tests for cross-language transforms.
package xlang
2 changes: 1 addition & 1 deletion sdks/go/test/run_validatesrunner_tests.sh
Original file line number Diff line number Diff line change
Expand Up @@ -276,7 +276,7 @@ ARGS="$ARGS --staging_location=$GCS_LOCATION/staging-validatesrunner-test"
ARGS="$ARGS --temp_location=$GCS_LOCATION/temp-validatesrunner-test"
ARGS="$ARGS --dataflow_worker_jar=$DATAFLOW_WORKER_JAR"
ARGS="$ARGS --endpoint=$ENDPOINT"
if [[-n "$EXPANSION_ADDR"]]; then
if [[ -n "$EXPANSION_ADDR" ]]; then
ARGS="$ARGS --expansion_addr=$EXPANSION_ADDR"
fi

Expand Down

0 comments on commit edeaf67

Please sign in to comment.