[
https://issues.apache.org/jira/browse/BEAM-14513?focusedWorklogId=775306&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-775306
]
ASF GitHub Bot logged work on BEAM-14513:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 27/May/22 01:10
Start Date: 27/May/22 01:10
Worklog Time Spent: 10m
Work Description: youngoli commented on code in PR #17748:
URL: https://github.com/apache/beam/pull/17748#discussion_r883186349
##########
sdks/go/pkg/beam/io/fhirio/read_test.go:
##########
@@ -0,0 +1,84 @@
+// 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 fhirio
+
+import (
+ "errors"
+ "net/http"
+ "strings"
+ "testing"
+
+ "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/passert"
+ "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/ptest"
+)
+
+func TestRead(t *testing.T) {
+ testCases := []struct {
+ name string
+ client fhirStoreClient
+ containedError string
+ }{
+ {
+ name: "Read Request Failed",
+ client: &fakeFhirStoreClient{
+ fakeReadResources: func(resource string)
(*http.Response, error) {
+ return nil, errors.New("")
+ },
+ },
+ containedError: "Failed to fetch resource",
Review Comment:
The test is currently a bit brittle as it will break whenever the error
messages are modified. I recommend that, if distinguishing the different errors
is an expectation for users, moving the error messages to unexported constants
in read.go and use those constants in both the read transform and unit tests.
If it gets a little tricky with raw strings, you can also try something with
[custom error
types](https://www.digitalocean.com/community/tutorials/creating-custom-errors-in-go).
On the other hand, I think this test would be fine if you simply validate
that an error is returned without checking the error message. It depends on how
important you think it is for users that these specific error messages are
provided.
##########
sdks/go/pkg/beam/io/fhirio/read.go:
##########
@@ -0,0 +1,98 @@
+// 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 fhirio provides an API for reading and writing resources to Google
+// Cloud Healthcare Fhir stores.
+// Experimental.
+package fhirio
+
+import (
+ "context"
+ "io"
+ "reflect"
+ "time"
+
+ "github.com/apache/beam/sdks/v2/go/pkg/beam"
+ "github.com/apache/beam/sdks/v2/go/pkg/beam/core/metrics"
+ "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors"
+)
+
+func init() {
+ beam.RegisterType(reflect.TypeOf((*readResourceFn)(nil)))
+}
+
+type readResourceFn struct {
+ client fhirStoreClient
+ readResourceErrors *metrics.Counter
+ readResourceSuccess *metrics.Counter
+ readResourceLatencyMs *metrics.Distribution
Review Comment:
I'd suggest using the beam forward declarations for metrics unless you have
specific reasons not to. As shown in the [large_wordcount
example](https://github.com/apache/beam/blob/master/sdks/go/examples/large_wordcount/large_wordcount.go#L199).
```suggestion
readResourceErrors beam.Counter
readResourceSuccess beam.Counter
readResourceLatencyMs beam.Distribution
```
##########
sdks/go/pkg/beam/io/fhirio/read.go:
##########
@@ -0,0 +1,98 @@
+// 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 fhirio provides an API for reading and writing resources to Google
+// Cloud Healthcare Fhir stores.
+// Experimental.
+package fhirio
+
+import (
+ "context"
+ "io"
+ "reflect"
+ "time"
+
+ "github.com/apache/beam/sdks/v2/go/pkg/beam"
+ "github.com/apache/beam/sdks/v2/go/pkg/beam/core/metrics"
+ "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors"
+)
+
+func init() {
+ beam.RegisterType(reflect.TypeOf((*readResourceFn)(nil)))
+}
+
+type readResourceFn struct {
+ client fhirStoreClient
+ readResourceErrors *metrics.Counter
+ readResourceSuccess *metrics.Counter
+ readResourceLatencyMs *metrics.Distribution
+}
+
+func (fn readResourceFn) String() string {
+ return "readResourceFn"
+}
+
+func (fn *readResourceFn) Setup() {
+ if fn.client == nil {
+ fn.client = newFhirStoreClient()
+ }
+ fn.readResourceErrors = metrics.NewCounter(fn.String(),
baseMetricPrefix+"read_resource_error_count")
+ fn.readResourceSuccess = metrics.NewCounter(fn.String(),
baseMetricPrefix+"read_resource_success_count")
+ fn.readResourceLatencyMs = metrics.NewDistribution(fn.String(),
baseMetricPrefix+"read_resource_latency_ms")
+}
+
+func (fn *readResourceFn) ProcessElement(ctx context.Context, resourcePath
string, emitResource func(string), emitDeadLetter func(string)) {
+ timeBeforeReadRequest := time.Now()
+ response, err := fn.client.readResource(resourcePath)
+ fn.readResourceLatencyMs.Update(ctx,
time.Now().Sub(timeBeforeReadRequest).Milliseconds())
+
+ if err != nil {
+ fn.readResourceErrors.Inc(ctx, 1)
+ emitDeadLetter(errors.Wrapf(err, "Failed to fetch resource
[%s].", resourcePath).Error())
Review Comment:
Style nit: Go style is to have error messages start in lowercase and not end
in punctuation unless there's good reason otherwise. [See
here](https://github.com/golang/go/wiki/CodeReviewComments#error-strings).
```suggestion
emitDeadLetter(errors.Wrapf(err, "failed to fetch resource
[%s]", resourcePath).Error())
```
This suggestion goes for all the other error messages below as well.
Issue Time Tracking
-------------------
Worklog Id: (was: 775306)
Time Spent: 2h 50m (was: 2h 40m)
> Add Read transform and initial HealthcareClient in Go SDK
> ---------------------------------------------------------
>
> Key: BEAM-14513
> URL: https://issues.apache.org/jira/browse/BEAM-14513
> Project: Beam
> Issue Type: Improvement
> Components: io-go-gcp
> Reporter: Lucas Nogueira
> Assignee: Lucas Nogueira
> Priority: P2
> Time Spent: 2h 50m
> Remaining Estimate: 0h
>
> This is the first of many tickets to bring
> [FhirIO|https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java]
> from Java into the Go SDK.
> It consists of:
> * Adding the
> [Read|https://github.com/apache/beam/blob/3e683606d9a03e7da3d37a83eb16c3a6b96068cd/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java#L455]
> transform in Go SDK
> * Adding a basic implementation of the
> [HttpHealthcareClient|https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java]
> in Go, to be able to authenticate and make the requests needed for the
> transform above.
> Buganizer links:
> - [b/233887489|https://b.corp.google.com/issues/233887489].
> - [b/204355275|https://buganizer.corp.google.com/issues/204355275].
--
This message was sent by Atlassian Jira
(v8.20.7#820007)