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/06/04 21:13:22 UTC

[GitHub] [beam] lostluck opened a new pull request #11925: [BEAM-9615] Add string coder utility functions.

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


   This adds initial utility functions for encoding and decoding utf8 strings in the Go SDK.
   
   Doesn't make use of them yet. In practice this is already how strings are encoded in the Go SDK, but marked as "custom" coders rather than the built in URN.
   
   ------------------------
   
   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_ValidatesRunner_Flink_Java11/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/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] lostluck commented on a change in pull request #11925: [BEAM-9615] Add string coder utility functions.

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



##########
File path: sdks/go/pkg/beam/core/graph/coder/stringutf8.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 coder
+
+import (
+	"io"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/ioutilx"
+)
+
+const bufCap = 64
+
+// EncodeStringUTF8LP encodes a UTF string with a length prefix.

Review comment:
       I removed it. It's part of the beam spec for an encoded UTF8 string, so the call out is not necessary.




----------------------------------------------------------------
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 #11925: [BEAM-9615] Add string coder utility functions.

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



##########
File path: sdks/go/pkg/beam/core/graph/coder/stringutf8.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 coder
+
+import (
+	"io"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/ioutilx"
+)
+
+const bufCap = 64
+
+// EncodeStringUTF8LP encodes a UTF string with a length prefix.

Review comment:
       It's part of the StringUTF8 so I could remove the explicit call out. However, in Java an python tests for the string encodings, they use those common encodings (which is why we have a separate encoding/decoding tests with the golden values), but those encodings *do not* include the length prefix.
   
   I'd rather have the positive inclusion of a length prefix in the name, instead of having the helper method be "withoutLP" instead.
   
   Some of that is from the archaic concept of "nested" and "unnested" coders which is largely phased out. In practice, if you have a variable amount of data, a length prefix is required.




----------------------------------------------------------------
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] tysonjh commented on a change in pull request #11925: [BEAM-9615] Add string coder utility functions.

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



##########
File path: sdks/go/pkg/beam/core/graph/coder/stringutf8_test.go
##########
@@ -0,0 +1,122 @@
+// 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 coder
+
+import (
+	"bytes"
+	"encoding/base64"
+	"io"
+	"strings"
+	"testing"
+	"unicode/utf8"
+)
+
+var testValues = []string{
+	"",
+	"a",
+	"13",
+	"hello",
+	"a longer string with spaces and all that",
+	"a string with a \n newline",
+	"スタリング",
+	"I am the very model of a modern major general.\nI've information animal, vegetable, and mineral",
+}
+
+// Base64 encoded versions of the above strings, without the length prefix.
+var testEncodings = []string{
+	"",
+	"YQ",
+	"MTM",
+	"aGVsbG8",
+	"YSBsb25nZXIgc3RyaW5nIHdpdGggc3BhY2VzIGFuZCBhbGwgdGhhdA",
+	"YSBzdHJpbmcgd2l0aCBhIAogbmV3bGluZQ",
+	"44K544K_44Oq44Oz44Kw",
+	"SSBhbSB0aGUgdmVyeSBtb2RlbCBvZiBhIG1vZGVybiBtYWpvciBnZW5lcmFsLgpJJ3ZlIGluZm9ybWF0aW9uIGFuaW1hbCwgdmVnZXRhYmxlLCBhbmQgbWluZXJhbA",
+}
+
+// TestLen serves as a verification that string lengths
+// match their equivalent byte lengths, and not their rune
+// representation.
+func TestLen(t *testing.T) {
+	runeCount := []int{0, 1, 2, 5, 40, 25, 5, 94}
+	for i, s := range testValues {
+		if got, want := len(s), len([]byte(s)); got != want {
+			t.Errorf("string and []byte len do not match. got %v, want %v", got, want)
+		}
+		if got, want := utf8.RuneCountInString(s), runeCount[i]; got != want {
+			t.Errorf("Rune count of %q change len do not match. got %v, want %v", s, got, want)
+		}
+	}
+}
+
+func TestEncodeStringUTF8(t *testing.T) {
+	for i, s := range testValues {
+		s := s
+		want := testEncodings[i]
+		t.Run(s, func(t *testing.T) {
+			var b strings.Builder
+			base64enc := base64.NewEncoder(base64.RawURLEncoding, &b)
+
+			if err := encodeStringUTF8(s, base64enc); err != nil {
+				t.Fatal(err)
+			}
+			base64enc.Close()
+			got := b.String()
+			if got != want {
+				t.Errorf("encodeStringUTF8(%q) = %q, want %q", s, got, want)
+			}
+		})
+	}
+}
+
+func TestDecodeStringUTF8(t *testing.T) {
+	for i, s := range testEncodings {
+		s := s
+		want := testValues[i]
+		t.Run(want, func(t *testing.T) {
+			b := bytes.NewBufferString(s)
+			base64dec := base64.NewDecoder(base64.RawURLEncoding, b)
+
+			got, err := decodeStringUTF8(int64(len(want)), base64dec)
+			if err != nil && err != io.EOF {
+				t.Fatal(err)
+			}
+			if got != want {
+				t.Errorf("decodeStringUTF8(%q) = %q, want %q", s, got, want)
+			}
+		})
+	}
+}
+
+func TestEncodeDecodeStringUTF8LP(t *testing.T) {
+	for _, s := range testValues {
+		want := s
+		t.Run(want, func(t *testing.T) {
+			var build strings.Builder
+			if err := EncodeStringUTF8LP(want, &build); err != nil {

Review comment:
       If the LP part got removed, or unused somehow, in both encode/decode (a stretch to be sure), then this test would pass despite there being no LP. My thought was that since the exposed methods suggest LP as part of the abstraction, it should be explicitly verified to avoid any surprises.
   
   I'll leave it up to your discretion.




----------------------------------------------------------------
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 #11925: [BEAM-9615] Add string coder utility functions.

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



##########
File path: sdks/go/pkg/beam/core/graph/coder/stringutf8_test.go
##########
@@ -0,0 +1,122 @@
+// 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 coder
+
+import (
+	"bytes"
+	"encoding/base64"
+	"io"
+	"strings"
+	"testing"
+	"unicode/utf8"
+)
+
+var testValues = []string{
+	"",
+	"a",
+	"13",
+	"hello",
+	"a longer string with spaces and all that",
+	"a string with a \n newline",
+	"スタリング",
+	"I am the very model of a modern major general.\nI've information animal, vegetable, and mineral",
+}
+
+// Base64 encoded versions of the above strings, without the length prefix.
+var testEncodings = []string{
+	"",
+	"YQ",
+	"MTM",
+	"aGVsbG8",
+	"YSBsb25nZXIgc3RyaW5nIHdpdGggc3BhY2VzIGFuZCBhbGwgdGhhdA",
+	"YSBzdHJpbmcgd2l0aCBhIAogbmV3bGluZQ",
+	"44K544K_44Oq44Oz44Kw",
+	"SSBhbSB0aGUgdmVyeSBtb2RlbCBvZiBhIG1vZGVybiBtYWpvciBnZW5lcmFsLgpJJ3ZlIGluZm9ybWF0aW9uIGFuaW1hbCwgdmVnZXRhYmxlLCBhbmQgbWluZXJhbA",
+}
+
+// TestLen serves as a verification that string lengths
+// match their equivalent byte lengths, and not their rune
+// representation.
+func TestLen(t *testing.T) {
+	runeCount := []int{0, 1, 2, 5, 40, 25, 5, 94}
+	for i, s := range testValues {
+		if got, want := len(s), len([]byte(s)); got != want {
+			t.Errorf("string and []byte len do not match. got %v, want %v", got, want)
+		}
+		if got, want := utf8.RuneCountInString(s), runeCount[i]; got != want {
+			t.Errorf("Rune count of %q change len do not match. got %v, want %v", s, got, want)
+		}
+	}
+}
+
+func TestEncodeStringUTF8(t *testing.T) {
+	for i, s := range testValues {
+		s := s
+		want := testEncodings[i]
+		t.Run(s, func(t *testing.T) {
+			var b strings.Builder
+			base64enc := base64.NewEncoder(base64.RawURLEncoding, &b)
+
+			if err := encodeStringUTF8(s, base64enc); err != nil {
+				t.Fatal(err)
+			}
+			base64enc.Close()
+			got := b.String()
+			if got != want {
+				t.Errorf("encodeStringUTF8(%q) = %q, want %q", s, got, want)
+			}
+		})
+	}
+}
+
+func TestDecodeStringUTF8(t *testing.T) {
+	for i, s := range testEncodings {
+		s := s
+		want := testValues[i]
+		t.Run(want, func(t *testing.T) {
+			b := bytes.NewBufferString(s)
+			base64dec := base64.NewDecoder(base64.RawURLEncoding, b)
+
+			got, err := decodeStringUTF8(int64(len(want)), base64dec)
+			if err != nil && err != io.EOF {
+				t.Fatal(err)
+			}
+			if got != want {
+				t.Errorf("decodeStringUTF8(%q) = %q, want %q", s, got, want)
+			}
+		})
+	}
+}
+
+func TestEncodeDecodeStringUTF8LP(t *testing.T) {
+	for _, s := range testValues {
+		want := s
+		t.Run(want, func(t *testing.T) {
+			var build strings.Builder
+			if err := EncodeStringUTF8LP(want, &build); err != nil {

Review comment:
       Technically, this test has the LP verification. If the LP weren't present, it wouldn't be possible to get the result back again on decode since the decoder won't know how much data to read.




----------------------------------------------------------------
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 #11925: [BEAM-9615] Add string coder utility functions.

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



##########
File path: sdks/go/pkg/beam/core/graph/coder/stringutf8_test.go
##########
@@ -0,0 +1,122 @@
+// 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 coder
+
+import (
+	"bytes"
+	"encoding/base64"
+	"io"
+	"strings"
+	"testing"
+	"unicode/utf8"
+)
+
+var testValues = []string{
+	"",
+	"a",
+	"13",
+	"hello",
+	"a longer string with spaces and all that",
+	"a string with a \n newline",
+	"スタリング",
+	"I am the very model of a modern major general.\nI've information animal, vegetable, and mineral",
+}
+
+// Base64 encoded versions of the above strings, without the length prefix.
+var testEncodings = []string{
+	"",
+	"YQ",
+	"MTM",
+	"aGVsbG8",
+	"YSBsb25nZXIgc3RyaW5nIHdpdGggc3BhY2VzIGFuZCBhbGwgdGhhdA",
+	"YSBzdHJpbmcgd2l0aCBhIAogbmV3bGluZQ",
+	"44K544K_44Oq44Oz44Kw",
+	"SSBhbSB0aGUgdmVyeSBtb2RlbCBvZiBhIG1vZGVybiBtYWpvciBnZW5lcmFsLgpJJ3ZlIGluZm9ybWF0aW9uIGFuaW1hbCwgdmVnZXRhYmxlLCBhbmQgbWluZXJhbA",
+}
+
+// TestLen serves as a verification that string lengths
+// match their equivalent byte lengths, and not their rune
+// representation.
+func TestLen(t *testing.T) {
+	runeCount := []int{0, 1, 2, 5, 40, 25, 5, 94}
+	for i, s := range testValues {
+		if got, want := len(s), len([]byte(s)); got != want {
+			t.Errorf("string and []byte len do not match. got %v, want %v", got, want)
+		}
+		if got, want := utf8.RuneCountInString(s), runeCount[i]; got != want {
+			t.Errorf("Rune count of %q change len do not match. got %v, want %v", s, got, want)
+		}
+	}
+}
+
+func TestEncodeStringUTF8(t *testing.T) {
+	for i, s := range testValues {
+		s := s
+		want := testEncodings[i]
+		t.Run(s, func(t *testing.T) {
+			var b strings.Builder
+			base64enc := base64.NewEncoder(base64.RawURLEncoding, &b)
+
+			if err := encodeStringUTF8(s, base64enc); err != nil {
+				t.Fatal(err)
+			}
+			base64enc.Close()
+			got := b.String()
+			if got != want {
+				t.Errorf("encodeStringUTF8(%q) = %q, want %q", s, got, want)
+			}
+		})
+	}
+}
+
+func TestDecodeStringUTF8(t *testing.T) {
+	for i, s := range testEncodings {
+		s := s
+		want := testValues[i]
+		t.Run(want, func(t *testing.T) {
+			b := bytes.NewBufferString(s)
+			base64dec := base64.NewDecoder(base64.RawURLEncoding, b)
+
+			got, err := decodeStringUTF8(int64(len(want)), base64dec)
+			if err != nil && err != io.EOF {
+				t.Fatal(err)
+			}
+			if got != want {
+				t.Errorf("decodeStringUTF8(%q) = %q, want %q", s, got, want)
+			}
+		})
+	}
+}
+
+func TestEncodeDecodeStringUTF8LP(t *testing.T) {
+	for _, s := range testValues {
+		want := s
+		t.Run(want, func(t *testing.T) {
+			var build strings.Builder
+			if err := EncodeStringUTF8LP(want, &build); err != nil {

Review comment:
       By removing the callout for the LP, it's part of the implementation details. Eg if you expect the next value is a StringUTF8, then it must be length prefixed. This code would only work in that situation.
   
   




----------------------------------------------------------------
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] tysonjh commented on a change in pull request #11925: [BEAM-9615] Add string coder utility functions.

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



##########
File path: sdks/go/pkg/beam/core/graph/coder/stringutf8.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 coder
+
+import (
+	"io"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/ioutilx"
+)
+
+const bufCap = 64
+
+// EncodeStringUTF8LP encodes a UTF string with a length prefix.

Review comment:
       Sounds good.




----------------------------------------------------------------
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 pull request #11925: [BEAM-9615] Add string coder utility functions.

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


   Ah dang it. I thought I had pushed the commit with the rename, but it was waiting on a password. I'll have that as another PR.


----------------------------------------------------------------
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] tysonjh commented on a change in pull request #11925: [BEAM-9615] Add string coder utility functions.

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



##########
File path: sdks/go/pkg/beam/core/graph/coder/stringutf8.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 coder
+
+import (
+	"io"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/ioutilx"
+)
+
+const bufCap = 64
+
+// EncodeStringUTF8LP encodes a UTF string with a length prefix.

Review comment:
       Is the length prefix more than an implementation detail or should this just be named EncodeStringUTF8?

##########
File path: sdks/go/pkg/beam/core/graph/coder/stringutf8_test.go
##########
@@ -0,0 +1,122 @@
+// 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 coder
+
+import (
+	"bytes"
+	"encoding/base64"
+	"io"
+	"strings"
+	"testing"
+	"unicode/utf8"
+)
+
+var testValues = []string{
+	"",
+	"a",
+	"13",
+	"hello",
+	"a longer string with spaces and all that",
+	"a string with a \n newline",
+	"スタリング",
+	"I am the very model of a modern major general.\nI've information animal, vegetable, and mineral",
+}
+
+// Base64 encoded versions of the above strings, without the length prefix.
+var testEncodings = []string{
+	"",
+	"YQ",
+	"MTM",
+	"aGVsbG8",
+	"YSBsb25nZXIgc3RyaW5nIHdpdGggc3BhY2VzIGFuZCBhbGwgdGhhdA",
+	"YSBzdHJpbmcgd2l0aCBhIAogbmV3bGluZQ",
+	"44K544K_44Oq44Oz44Kw",
+	"SSBhbSB0aGUgdmVyeSBtb2RlbCBvZiBhIG1vZGVybiBtYWpvciBnZW5lcmFsLgpJJ3ZlIGluZm9ybWF0aW9uIGFuaW1hbCwgdmVnZXRhYmxlLCBhbmQgbWluZXJhbA",
+}
+
+// TestLen serves as a verification that string lengths
+// match their equivalent byte lengths, and not their rune
+// representation.
+func TestLen(t *testing.T) {
+	runeCount := []int{0, 1, 2, 5, 40, 25, 5, 94}
+	for i, s := range testValues {
+		if got, want := len(s), len([]byte(s)); got != want {
+			t.Errorf("string and []byte len do not match. got %v, want %v", got, want)
+		}
+		if got, want := utf8.RuneCountInString(s), runeCount[i]; got != want {
+			t.Errorf("Rune count of %q change len do not match. got %v, want %v", s, got, want)
+		}
+	}
+}
+
+func TestEncodeStringUTF8(t *testing.T) {
+	for i, s := range testValues {
+		s := s
+		want := testEncodings[i]
+		t.Run(s, func(t *testing.T) {
+			var b strings.Builder
+			base64enc := base64.NewEncoder(base64.RawURLEncoding, &b)
+
+			if err := encodeStringUTF8(s, base64enc); err != nil {
+				t.Fatal(err)
+			}
+			base64enc.Close()
+			got := b.String()
+			if got != want {
+				t.Errorf("encodeStringUTF8(%q) = %q, want %q", s, got, want)
+			}
+		})
+	}
+}
+
+func TestDecodeStringUTF8(t *testing.T) {
+	for i, s := range testEncodings {
+		s := s
+		want := testValues[i]
+		t.Run(want, func(t *testing.T) {
+			b := bytes.NewBufferString(s)
+			base64dec := base64.NewDecoder(base64.RawURLEncoding, b)
+
+			got, err := decodeStringUTF8(int64(len(want)), base64dec)
+			if err != nil && err != io.EOF {
+				t.Fatal(err)
+			}
+			if got != want {
+				t.Errorf("decodeStringUTF8(%q) = %q, want %q", s, got, want)
+			}
+		})
+	}
+}
+
+func TestEncodeDecodeStringUTF8LP(t *testing.T) {
+	for _, s := range testValues {
+		want := s
+		t.Run(want, func(t *testing.T) {
+			var build strings.Builder
+			if err := EncodeStringUTF8LP(want, &build); err != nil {

Review comment:
       Maybe a test that verifies the LP is present in the encoded result and one that verifies a failure on decode when the LP is not present?




----------------------------------------------------------------
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 pull request #11925: [BEAM-9615] Add string coder utility functions.

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


   R: @tysonjh 
   cc: @youngoli 


----------------------------------------------------------------
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 #11925: [BEAM-9615] Add string coder utility functions.

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



##########
File path: sdks/go/pkg/beam/core/graph/coder/stringutf8_test.go
##########
@@ -0,0 +1,122 @@
+// 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 coder
+
+import (
+	"bytes"
+	"encoding/base64"
+	"io"
+	"strings"
+	"testing"
+	"unicode/utf8"
+)
+
+var testValues = []string{
+	"",
+	"a",
+	"13",
+	"hello",
+	"a longer string with spaces and all that",
+	"a string with a \n newline",
+	"スタリング",
+	"I am the very model of a modern major general.\nI've information animal, vegetable, and mineral",
+}
+
+// Base64 encoded versions of the above strings, without the length prefix.
+var testEncodings = []string{
+	"",
+	"YQ",
+	"MTM",
+	"aGVsbG8",
+	"YSBsb25nZXIgc3RyaW5nIHdpdGggc3BhY2VzIGFuZCBhbGwgdGhhdA",
+	"YSBzdHJpbmcgd2l0aCBhIAogbmV3bGluZQ",
+	"44K544K_44Oq44Oz44Kw",
+	"SSBhbSB0aGUgdmVyeSBtb2RlbCBvZiBhIG1vZGVybiBtYWpvciBnZW5lcmFsLgpJJ3ZlIGluZm9ybWF0aW9uIGFuaW1hbCwgdmVnZXRhYmxlLCBhbmQgbWluZXJhbA",
+}
+
+// TestLen serves as a verification that string lengths
+// match their equivalent byte lengths, and not their rune
+// representation.
+func TestLen(t *testing.T) {
+	runeCount := []int{0, 1, 2, 5, 40, 25, 5, 94}
+	for i, s := range testValues {
+		if got, want := len(s), len([]byte(s)); got != want {
+			t.Errorf("string and []byte len do not match. got %v, want %v", got, want)
+		}
+		if got, want := utf8.RuneCountInString(s), runeCount[i]; got != want {
+			t.Errorf("Rune count of %q change len do not match. got %v, want %v", s, got, want)
+		}
+	}
+}
+
+func TestEncodeStringUTF8(t *testing.T) {
+	for i, s := range testValues {
+		s := s
+		want := testEncodings[i]
+		t.Run(s, func(t *testing.T) {
+			var b strings.Builder
+			base64enc := base64.NewEncoder(base64.RawURLEncoding, &b)
+
+			if err := encodeStringUTF8(s, base64enc); err != nil {
+				t.Fatal(err)
+			}
+			base64enc.Close()
+			got := b.String()
+			if got != want {
+				t.Errorf("encodeStringUTF8(%q) = %q, want %q", s, got, want)
+			}
+		})
+	}
+}
+
+func TestDecodeStringUTF8(t *testing.T) {
+	for i, s := range testEncodings {
+		s := s
+		want := testValues[i]
+		t.Run(want, func(t *testing.T) {
+			b := bytes.NewBufferString(s)
+			base64dec := base64.NewDecoder(base64.RawURLEncoding, b)
+
+			got, err := decodeStringUTF8(int64(len(want)), base64dec)
+			if err != nil && err != io.EOF {
+				t.Fatal(err)
+			}
+			if got != want {
+				t.Errorf("decodeStringUTF8(%q) = %q, want %q", s, got, want)
+			}
+		})
+	}
+}
+
+func TestEncodeDecodeStringUTF8LP(t *testing.T) {
+	for _, s := range testValues {
+		want := s
+		t.Run(want, func(t *testing.T) {
+			var build strings.Builder
+			if err := EncodeStringUTF8LP(want, &build); err != nil {

Review comment:
       That did occur to me as well, but it would fail the moment we run a wordcount since nothing could get decoded properly, and the runner would end up with very strange data under/over reads when it tries to use the initial character encodings as varints. In practice, this is not going to change, and certainly not to remove both length prefixes at the same time.




----------------------------------------------------------------
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 merged pull request #11925: [BEAM-9615] Add string coder utility functions.

Posted by GitBox <gi...@apache.org>.
lostluck merged pull request #11925:
URL: https://github.com/apache/beam/pull/11925


   


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