[ 
https://issues.apache.org/jira/browse/BEAM-14513?focusedWorklogId=774850&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-774850
 ]

ASF GitHub Bot logged work on BEAM-14513:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 25/May/22 23:50
            Start Date: 25/May/22 23:50
    Worklog Time Spent: 10m 
      Work Description: youngoli commented on code in PR #17748:
URL: https://github.com/apache/beam/pull/17748#discussion_r882199263


##########
sdks/go/pkg/beam/io/healthcare/fhirio/read.go:
##########
@@ -0,0 +1,87 @@
+// 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

Review Comment:
   Please add a package comment. If the IO is still unfinished/experimental you 
should also specify that. This package and any exported functionality within it 
will be visible to users on the next beam release, so keeping it documented is 
important, even while it's a work-in-progress.



##########
sdks/go/pkg/beam/io/healthcare/fhirio/read.go:
##########
@@ -0,0 +1,87 @@
+// 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 (
+       "context"

Review Comment:
   style nit: In the Go SDK we try to put standard library packages separate 
from third party ones. 
[Example](https://github.com/apache/beam/blob/master/sdks/go/examples/wordcount/wordcount.go#L69).



##########
sdks/go/pkg/beam/io/healthcare/fhirio/read.go:
##########
@@ -0,0 +1,87 @@
+// 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 (
+       "context"
+       "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"
+       "io"
+       "reflect"
+       "time"
+)
+
+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, resourceId 
string, emitResource func(string), emitDeadLetter func(string)) {
+       timeBeforeReadRequest := time.Now()
+       response, err := fn.client.readResource(resourceId)
+       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].", resourceId).Error())
+               return
+       }
+
+       if response.StatusCode != 200 {
+               fn.readResourceErrors.Inc(ctx, 1)
+               emitDeadLetter(errors.Errorf("Fetched resource [%s] returned 
bad status [%d].", resourceId, response.StatusCode).Error())
+               return
+       }
+
+       bytes, err := io.ReadAll(response.Body)
+       if err != nil {
+               fn.readResourceErrors.Inc(ctx, 1)
+               emitDeadLetter(errors.Wrapf(err, "Error while reading response 
body of resource [%s]", resourceId).Error())
+               return
+       }
+
+       fn.readResourceSuccess.Inc(ctx, 1)
+       emitResource(string(bytes))
+}
+
+func Read(s beam.Scope, resourceIds beam.PCollection) (beam.PCollection, 
beam.PCollection) {

Review Comment:
   Please add a comment documenting Read.



##########
sdks/go/pkg/beam/io/healthcare/fhirio/fakes.go:
##########
@@ -0,0 +1,40 @@
+// 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 (
+       "net/http"
+)
+
+type fakeFhirStoreClient struct {

Review Comment:
   Since this is only used for tests, I would put it in some file ending in 
`_test.go` so it doesn't get compiled into non-test binaries. Maybe in 
read_test.go for now, but even renaming this to fakes_test.go works.



##########
sdks/go/pkg/beam/io/healthcare/fhirio/read_test.go:
##########
@@ -0,0 +1,83 @@
+// 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"
+       "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/passert"
+       "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/ptest"
+       "net/http"
+       "strings"
+       "testing"
+)
+
+func Test_readFailures(t *testing.T) {

Review Comment:
   Style nit: Test names usually use underscores to separate variations on 
testing the same "unit". In this situation, I would just go with `TestRead` 
because that's the only function being tested and there aren't multiple tests 
of Read that need to be distinguished.



##########
sdks/go/pkg/beam/io/healthcare/gcpcommon.go:
##########
@@ -0,0 +1,34 @@
+// 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 healthcare
+
+import (
+       "context"
+       "github.com/apache/beam/sdks/v2/go/pkg/beam/core"
+       healthcarex "google.golang.org/api/healthcare/v1"
+       "google.golang.org/api/option"
+)
+
+const userAgent = "apache-beam-io-google-cloud-platform-healthcare/" + 
core.SdkVersion
+
+func NewGcpHealthcareService() *healthcarex.Service {

Review Comment:
   Question: Is this function intended to be user-facing? Or is it only 
exported for developer convenience? If it's the latter, we should hide it from 
users by putting it in an `internal/`subdirectory. [More 
info](https://dave.cheney.net/2019/10/06/use-internal-packages-to-reduce-your-public-api-surface)





Issue Time Tracking
-------------------

    Worklog Id:     (was: 774850)
    Time Spent: 40m  (was: 0.5h)

> 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: 40m
>  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)

Reply via email to