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 2021/02/03 14:56:00 UTC

[GitHub] [beam] johnedmonds opened a new pull request #13889: [BEAM-11611] Add transformation for computing approximate quantiles.

johnedmonds opened a new pull request #13889:
URL: https://github.com/apache/beam/pull/13889


   Adds a new transform to the stats package: Approximate Quantiles. This allows approximately computing quantiles for given comparable data.
   
   ------------------------
   
   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 | Dataflow | Flink | Samza | Spark | Twister2
   --- | --- | --- | --- | --- | --- | ---
   Go | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/) | ---
   Java | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_VR_Dataflow_V2/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_VR_Dataflow_V2/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/badge/icon)](https://ci-beam
 .apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/) | [![Build Status](https://ci-beam.a
 pache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/lastCompletedBuild/)
   Python | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python38/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python38/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/)<br>[![Build Status](https://ci-beam
 .apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Python_PVR_Flink_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Python_PVR_Flink_Cron/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/) | ---
   XLang | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Dataflow/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Dataflow/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/) | ---
   
   Pre-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   --- |Java | Python | Go | Website | Whitespace | Typescript
   --- | --- | --- | --- | --- | --- | ---
   Non-portable | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_PythonDocker_Cron/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_PythonDocker_Cron/lastCompletedBuild/) <br>[![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_PythonDocs_Cron/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_PythonDocs_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/be
 am_PreCommit_Go_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Whitespace_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Whitespace_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Typescript_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Typescript_Cron/lastCompletedBuild/)
   Portable | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.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.
   
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


----------------------------------------------------------------
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 #13889: [BEAM-11611] Add transformation for computing approximate quantiles.

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



##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles_test.go
##########
@@ -0,0 +1,274 @@
+// 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 stats
+
+import (
+	"reflect"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/passert"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/ptest"
+	"github.com/google/go-cmp/cmp"
+)
+
+func init() {
+	beam.RegisterFunction(weightedElementToKv)
+
+	// In practice, this runs faster than plain reflection.
+	reflectx.RegisterFunc(reflect.ValueOf(less).Type(), func(_ interface{}) reflectx.Func {
+		return newIntLess()
+	})
+}
+
+type intLess struct {
+	name string
+	t    reflect.Type
+}
+
+func newIntLess() *intLess {
+	return &intLess{
+		name: reflectx.FunctionName(reflect.ValueOf(less).Interface()),
+		t:    reflect.ValueOf(less).Type(),
+	}
+}
+
+func (i *intLess) Name() string {
+	return i.name
+}
+func (i *intLess) Type() reflect.Type {
+	return i.t
+}
+func (i *intLess) Call(args []interface{}) []interface{} {
+	return []interface{}{args[0].(int) < args[1].(int)}
+}
+
+func less(a, b int) bool {
+	return a < b
+}
+
+func TestLargeQuantiles(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := 0; i < numElements; i++ {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{10006, 19973}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestLargeQuantilesReversed(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := numElements - 1; i >= 0; i-- {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{9985, 19959}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestBasicQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	tests := []struct {
+		Input    []int
+		Expected [][]int
+	}{
+		{[]int{}, [][]int{}},
+		{[]int{1}, [][]int{[]int{1}}},
+		{[]int{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20}, [][]int{[]int{6, 13}}},
+	}
+
+	for _, test := range tests {
+		p, s, in, exp := ptest.CreateList2(test.Input, test.Expected)
+		quantiles := ApproximateQuantiles(s, in, less, Opts{
+			K:            3,
+			NumQuantiles: 3,
+		})
+		passert.Equals(s, quantiles, exp)
+
+		if err := ptest.Run(p); err != nil {
+			t.Errorf("ApproximateQuantiles(%v) != %v: %v", test.Input, test.Expected, err)
+		}
+	}
+}
+
+func weightedElementToKv(e testWeightedElement) (int, int) {
+	return e.Weight, e.Element
+}
+
+type testWeightedElement struct {
+	Weight  int
+	Element int
+}
+
+func TestWeightedQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	input := []testWeightedElement{
+		{Weight: 1, Element: 1},
+		{Weight: 10, Element: 2},
+		{Weight: 1, Element: 3},
+		{Weight: 10, Element: 4}}
+	expected := []int{2, 4}
+	p, s, in, exp := ptest.CreateList2(input, [][]int{expected})
+	elementWeightKvs := beam.ParDo(s, weightedElementToKv, in, beam.TypeDefinition{Var: beam.TType, T: reflect.TypeOf(1)})
+	opts := Opts{
+		K:            3,
+		NumQuantiles: 3,
+	}
+	quantiles := ApproximateWeightedQuantiles(s, elementWeightKvs, less, opts)
+	passert.Equals(s, quantiles, exp)
+
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles(%v) != %v: %v", input, expected, err)
+	}
+}
+
+func TestWeightedQuantilesWithInternalSharding(t *testing.T) {
+	// Test shard reduction.
+	input := []testWeightedElement{
+		{Weight: 1, Element: 1},
+		{Weight: 10, Element: 2},
+		{Weight: 1, Element: 3},
+		{Weight: 10, Element: 4}}
+	expected := []int{2, 4}
+	p, s, in, exp := ptest.CreateList2(input, [][]int{expected})
+	elementWeightKvs := beam.ParDo(s, weightedElementToKv, in, beam.TypeDefinition{Var: beam.TType, T: reflect.TypeOf(1)})
+	opts := Opts{

Review comment:
       Commentary, no action required:
   While the PTransform calls shouldn't be nested to avoid PCollection variables, do feel free to nest option structs like this. It doesn't have the same, unclear ordering problem when reading it.  No need to change here, just being precise.




----------------------------------------------------------------
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 #13889: [BEAM-11611] Add transformation for computing approximate quantiles.

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



##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles_test.go
##########
@@ -0,0 +1,291 @@
+// 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 stats
+
+import (
+	"reflect"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/passert"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/ptest"
+)
+
+func init() {
+	beam.RegisterFunction(weightedElementToKv)
+
+	// In practice, this runs faster than plain reflection.

Review comment:
       In this case, I'd add a comment `// TODO(BEAM-9616): Remove once collisions don't occur for starcgen over test code and an equivalent is generated for us.` 
   
   To be clear, the code you've got is fine. The hook is being correctly used, and my repo knows I wrote many of these things manually before getting the code generator functional. Ideally manual uses of these don't need to exist, and they're all generated. Go Generics, once they exists (in hopefully 1.18 next year) will allow the code generator to be largely replaced with generic implementations instead, and then everything can happen at compile time instead.
   
   [BEAM-9616 ](https://issues.apache.org/jira/browse/BEAM-9616) is the grab bag umbrella for various bugs and improvements that need to be done to starcgen to make it more robust. 
   
   And it's not an "already runs" thing. Go generate statements are never automatically called by the go build system, and never by package users, always by package authors. The starcgen tool is so that Beam users can improve the performance of their own code with their own go generate statements, which they'd need to call as befits their development set up. See https://blog.golang.org/generate for more information.




----------------------------------------------------------------
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 #13889: [BEAM-11611] Add transformation for computing approximate quantiles.

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


   


----------------------------------------------------------------
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 #13889: [BEAM-11611] Add transformation for computing approximate quantiles.

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



##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles_test.go
##########
@@ -0,0 +1,291 @@
+// 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 stats
+
+import (
+	"reflect"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/passert"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/ptest"
+)
+
+func init() {
+	beam.RegisterFunction(weightedElementToKv)
+
+	// In practice, this runs faster than plain reflection.

Review comment:
       FYI: Consider using the code generator (starcgen) along with `go generate` to generate all the various registrations, and type assertion shims for speedier execution. And it'll generate intLess for you.
   
   At the moment, the list of things to generate type assertion shims for for the stats package is in [util_gen.go](https://github.com/apache/beam/blob/ac03e4ddc92710204ff07a1fec384e93e063f3ea/sdks/go/pkg/beam/transforms/stats/util_gen.go#L25) 
   eg. [The generated file](
   https://github.com/apache/beam/blob/ac03e4ddc92710204ff07a1fec384e93e063f3ea/sdks/go/pkg/beam/transforms/stats/stats.shims.go) for the stats package. Do not edit by hand.
   
   The code generator needs some work though, as it doesn't do well with generating for both the main package, and test files (collisions can happen), so it may be moot for this specific case. However, well worth it for the prod code.

##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles.go
##########
@@ -0,0 +1,702 @@
+// 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 stats
+
+// Approximate quantiles is implemented based on https://arxiv.org/pdf/1907.00236.pdf.
+
+import (
+	"bytes"
+	"container/heap"
+	"context"
+	"encoding/gob"
+	"encoding/json"
+	"hash/crc32"
+	"io"
+	"math"
+	"reflect"
+	"sort"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+)
+
+func init() {
+	compactorsType := reflect.TypeOf((**compactors)(nil)).Elem()
+	weightedElementType := reflect.TypeOf((*weightedElement)(nil)).Elem()
+	beam.RegisterType(compactorsType)
+	beam.RegisterType(weightedElementType)
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesInputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesMergeOnlyFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesOutputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*shardElementsFn)(nil)).Elem())
+	beam.RegisterCoder(compactorsType, encodeCompactors, decodeCompactors)
+	beam.RegisterCoder(weightedElementType, encodeWeightedElement, decodeWeightedElement)
+}
+
+// Opts contains settings used to configure how approximate quantiles are computed.
+type Opts struct {
+	// Controls the memory used and approximation error (difference between the quantile returned and the true quantile.)
+	K int
+	// Number of quantiles to return. The algorithm will return NumQuantiles - 1 numbers
+	NumQuantiles int
+	// For extremely large datasets, the Go SDK has some problems holding on to data and might OOM.

Review comment:
       Was this overall, or per key?
   
   At present, a CoGBK can read an arbitrary amount of data per key using State Backed Iterables (if the runner supports it). Side inputs presently don't have such a unbounded protocol. However I know that work on that is forthcoming.
   
   All single messages must be < ~2GB at present due to limits on protocol buffer serialization (which everything is wrapped in for SDK <-> Runner communication) . Until we have a protocol to shard large values for re-assembly on the other side, not much can be done about that by the system.  These limits are the same for all Portable implementations of beam relying on the FnAPI.  Other limits include GRPC receive limits. The Runner and the SDK both need to have larger than the 4MB default set. The SDK already has this set to 2GB IIRC but I'm not 100% about all runners.
   
   -------
   
   I suspect it's the 2GB limit being run into due to the compactors. At a certain point, it's better to break things up and emit for merging on the other side of a GBK, rather than trying to compact everything in a single bundle.  A lifted Combine is just doing some work before and after a GroupByKey, so not everything needs to be a combiner function if inconvenient. It's a touch more manual, but in some circumstances it can be better to just have two DoFns bracing a GBK, to have more readable code, and better performance. In this case, a DoFn can take advantage of multiple emits (dividing compactors as necessary), and still combine things "per key" within a bundle, emitting in a FinishBundle or whenever the cached data becomes too large in ProcessElement.
   
    The big downside is re-implementing some of the keying or hashing that is part of the SDK's Combine inside user code though, so it's probably worth waiting for another PR, rather than updating this one.

##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles.go
##########
@@ -0,0 +1,702 @@
+// 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 stats
+
+// Approximate quantiles is implemented based on https://arxiv.org/pdf/1907.00236.pdf.
+
+import (
+	"bytes"
+	"container/heap"
+	"context"
+	"encoding/gob"
+	"encoding/json"
+	"hash/crc32"
+	"io"
+	"math"
+	"reflect"
+	"sort"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+)
+
+func init() {
+	compactorsType := reflect.TypeOf((**compactors)(nil)).Elem()
+	weightedElementType := reflect.TypeOf((*weightedElement)(nil)).Elem()
+	beam.RegisterType(compactorsType)
+	beam.RegisterType(weightedElementType)
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesInputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesMergeOnlyFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesOutputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*shardElementsFn)(nil)).Elem())
+	beam.RegisterCoder(compactorsType, encodeCompactors, decodeCompactors)
+	beam.RegisterCoder(weightedElementType, encodeWeightedElement, decodeWeightedElement)
+}
+
+// Opts contains settings used to configure how approximate quantiles are computed.
+type Opts struct {
+	// Controls the memory used and approximation error (difference between the quantile returned and the true quantile.)
+	K int
+	// Number of quantiles to return. The algorithm will return NumQuantiles - 1 numbers
+	NumQuantiles int
+	// For extremely large datasets, the Go SDK has some problems holding on to data and might OOM.
+	// If ApproximateQuantiles is running out of memory, you can use this option to tune how the data is sharded internally.
+	// This parameter is optional. If unspecified, Beam will compact all elements into a single compactor at once.
+	InternalSharding []int
+}
+
+// The paper suggests reducing the size of the lower-level compactors as we grow.
+// We reduce the capacity at this rate.
+// The paper suggests 1/sqrt(2) is ideal. That's approximately 0.7.
+const capacityCoefficient float64 = 0.7
+
+type sortListHeap struct {
+	data [][]beam.T
+	less reflectx.Func2x1
+}
+
+func (s sortListHeap) Len() int            { return len(s.data) }
+func (s sortListHeap) Less(i, j int) bool  { return s.less.Call2x1(s.data[i][0], s.data[j][0]).(bool) }
+func (s sortListHeap) Swap(i, j int)       { s.data[i], s.data[j] = s.data[j], s.data[i] }
+func (s *sortListHeap) Push(x interface{}) { s.data = append(s.data, x.([]beam.T)) }
+func (s *sortListHeap) Pop() interface{} {
+	var x beam.T
+	x, s.data = s.data[len(s.data)-1], s.data[:len(s.data)-1]
+	return x
+}
+
+// compactor contains elements to be compacted.
+type compactor struct {
+	// Compaction needs to sort elements before compacting. Thus in practice, we should often have some pre-sorted data.
+	// We want to keep it separate so we can sort only the unsorted data and merge the two sorted lists together.
+	// If we're only receiving elements of weight 1, only level 0 will ever contain unsorted data and the rest of the levels will always remain sorted.
+	// To prevent repeated allocation/copying, we keep multiple sorted lists and then merge them together
+	sorted   [][]beam.T
+	unsorted []beam.T
+	// How many items should be stored in this compactor before it should get compacted.
+	// Note that this is not a hard limit.
+	// The paper suggests implementing lazy compaction which would allow
+	// compactors to temporarily exceed their capacity as long as the total
+	// elements in all compactors doesn't exceed the total capacity in all
+	// compactors. In other words, compactors can temporarily borrow capacity
+	// from each other.
+	// In the paper, this is referred to as the variable k_h.
+	capacity int
+}
+
+type compactorAsGob struct {
+	Sorted            [][]byte
+	Unsorted          []byte
+	EncodedTypeAsJSON []byte
+}
+
+func encodeElements(enc beam.ElementEncoder, elements []beam.T) ([]byte, error) {
+	var buf bytes.Buffer
+	for _, e := range elements {
+		if err := enc.Encode(e, &buf); err != nil {
+			return nil, err
+		}
+	}
+	return buf.Bytes(), nil
+}
+
+func (c *compactor) getElementType() reflect.Type {
+	for _, e := range c.sorted {
+		for _, e2 := range e {
+			return reflect.TypeOf(e2)
+		}
+	}
+	for _, e := range c.unsorted {
+		return reflect.TypeOf(e)
+	}
+	return nil
+}
+
+func (c *compactor) MarshalBinary() ([]byte, error) {
+	t := c.getElementType()
+	var buf bytes.Buffer
+	if t == nil {
+		enc := gob.NewEncoder(&buf)
+		if err := enc.Encode(compactorAsGob{}); err != nil {
+			return nil, err
+		}
+		return buf.Bytes(), nil
+	}
+	enc := beam.NewElementEncoder(t)
+	encodedSorted := make([][]byte, 0, len(c.sorted))
+	for _, sorted := range c.sorted {
+		encoded, err := encodeElements(enc, sorted)
+		if err != nil {
+			return nil, err
+		}
+		encodedSorted = append(encodedSorted, encoded)
+	}
+	encodedUnsorted, err := encodeElements(enc, c.unsorted)
+	if err != nil {
+		return nil, err
+	}
+	tAsJSON, err := beam.EncodedType{T: t}.MarshalJSON()
+	if err != nil {
+		return nil, err
+	}
+	gobEnc := gob.NewEncoder(&buf)
+	if err = gobEnc.Encode(compactorAsGob{
+		Sorted:            encodedSorted,
+		Unsorted:          encodedUnsorted,
+		EncodedTypeAsJSON: tAsJSON,
+	}); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func decodeElements(dec beam.ElementDecoder, data []byte) ([]beam.T, error) {
+	buf := bytes.NewBuffer(data)
+	ret := []beam.T{}
+	for {
+		element, err := dec.Decode(buf)
+		if err == io.EOF {
+			return ret, nil
+		} else if err != nil {
+			return nil, err
+		}
+		ret = append(ret, element)
+	}
+}
+
+func (c *compactor) UnmarshalBinary(data []byte) error {
+	var g compactorAsGob
+	var err error
+	gobDec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err = gobDec.Decode(&g); err != nil {
+		return err
+	}
+	if len(g.EncodedTypeAsJSON) == 0 {
+		return nil
+	}
+	var t beam.EncodedType
+	if err = json.Unmarshal(g.EncodedTypeAsJSON, &t); err != nil {
+		return err
+	}
+	dec := beam.NewElementDecoder(t.T)
+	decodedSorted := make([][]beam.T, 0, len(g.Sorted))
+	for _, sorted := range g.Sorted {
+		decoded, err := decodeElements(dec, sorted)
+		if err != nil {
+			return err
+		}
+		decodedSorted = append(decodedSorted, decoded)
+	}
+	c.sorted = decodedSorted
+	if c.unsorted, err = decodeElements(dec, g.Unsorted); err != nil {
+		return err
+	}
+	return nil
+}
+
+// update inserts an element into the compactor.
+func (c *compactor) update(element beam.T) {
+	c.unsorted = append(c.unsorted, element)
+}
+
+// size returns the number of elements stored in this compactor.
+func (c *compactor) size() int {
+	size := 0
+	for _, s := range c.sorted {
+		size += len(s)
+	}
+	return len(c.unsorted) + size
+}
+
+type sorter struct {
+	less reflectx.Func2x1
+	data []beam.T
+}
+
+func (s sorter) Len() int           { return len(s.data) }
+func (s sorter) Less(i, j int) bool { return s.less.Call2x1(s.data[i], s.data[j]).(bool) }
+func (s sorter) Swap(i, j int)      { s.data[i], s.data[j] = s.data[j], s.data[i] }
+
+// sort sorts the compactor and returns all the elements in sorted order.
+func (c *compactor) sort(less reflectx.Func2x1) []beam.T {
+	sort.Sort(sorter{data: c.unsorted, less: less})
+	h := sortListHeap{data: c.sorted, less: less}
+	heap.Init(&h)
+	sorted := make([]beam.T, 0, c.size()-len(c.unsorted))
+	for h.Len() > 0 {
+		s := heap.Pop(&h).([]beam.T)
+		sorted = append(sorted, s[0])
+		if len(s) > 1 {
+			heap.Push(&h, s[1:])
+		}
+	}
+	c.sorted = [][]beam.T{mergeSorted(sorted, c.unsorted, func(a, b interface{}) bool { return less.Call2x1(a, b).(bool) })}
+	c.unsorted = nil
+	if len(c.sorted[0]) == 0 {
+		c.sorted = nil
+		return nil
+	}
+	return c.sorted[0]
+}
+
+// Compactors holds the state of the quantile approximation compactors.
+type compactors struct {
+	// References "K" from the paper which influences the amount of memory used.
+	K int
+	// When compacting, we want to alternate between taking elements at even vs odd indices.
+	// The paper suggests using a random variable but we'd prefer to stay deterministic.
+	// Especially when merging two compactors we want to keep track of how often we've selected odds vs evens.
+	NumberOfCompactions int
+
+	// Each compactor takes a sample of elements.
+	// The "height" (also known as the index in this slice) of the compactor determines the weight of its elements.
+	// The weight of a compactor of height h is 2^h.
+	// For example, for h = 3 (which would be compactors[3]), the weight is 2^3 = 8. That means each element in that compactor represents 8 instances of itself.
+	Compactors []compactor
+}
+
+func (c *compactors) totalCapacity() int {
+	totalCapacity := 0
+	for _, compactor := range c.Compactors {
+		totalCapacity += compactor.capacity
+	}
+	return totalCapacity
+}
+
+func (c *compactors) size() int {
+	size := 0
+	for _, compactor := range c.Compactors {
+		size += compactor.size()
+	}
+	return size
+}
+
+// capacity computes the capacity of a compactor at a certain level.
+// The paper suggests decreasing the capacity of lower-leveled compactors as we add more elements.
+func (c *compactors) capacity(compactorLevel int) int {
+	return int(math.Ceil(math.Pow(capacityCoefficient, float64(len(c.Compactors)-compactorLevel-1))*float64(c.K))) + 1
+}
+
+// compact compacts all compactors until the total size is less than the maximum capacity of all compactors.
+func (c *compactors) compact(less reflectx.Func2x1) {
+	for c.size() > c.totalCapacity() {
+		for level, compactor := range c.Compactors {
+			if compactor.size() > compactor.capacity {
+				c.compactLevel(level, less)
+				// Merging compactors can cause us to exceed max capacity in multiple compactors.
+				if c.size() < c.totalCapacity() {
+					// Do lazy compaction as described in the paper.
+					break
+				}
+			}
+		}
+	}
+}
+
+// update inserts the given element into the compactors. If this element causes the compactors to grow too large, we perform the compaction here.
+func (c *compactors) update(element beam.T, weight int, less reflectx.Func2x1) {
+	level := int(math.Log2(float64(weight)))
+	c.growToIncludeLevel(level)
+	c.Compactors[level].update(element)
+	// From the paper, we're using the "Splitting the Input" approach.
+	remainingWeight := weight - (1 << uint(level))
+	// Only attempt compaction if we're doing the last update. Otherwise we'd be compacting too often.
+	if remainingWeight <= 0 {
+		c.compact(less)
+	} else {
+		c.update(element, remainingWeight, less)
+	}
+}
+
+// growToIncludeLevel ensures we have compactors available at the given level.
+func (c *compactors) growToIncludeLevel(level int) {
+	if len(c.Compactors)-1 >= level {
+		return
+	}
+	for i := len(c.Compactors) - 1; i < level; i++ {
+		c.Compactors = append(c.Compactors, compactor{})
+	}
+	for level := range c.Compactors {
+		c.Compactors[level].capacity = c.capacity(level)
+	}
+}
+
+// compact compacts elements in compactors.
+func (c *compactors) compactLevel(level int, less reflectx.Func2x1) {
+	c.growToIncludeLevel(level + 1)
+	jitterIndex := 0
+	// Create a temporary buffer to hold the compacted elements.
+	// Buffering the elements like this makes it easier to call mergeSorted.
+	compacted := make([]beam.T, 0, c.Compactors[level].size()/2)
+	selectEvens := c.NumberOfCompactions%2 == 0
+	c.NumberOfCompactions++
+	for _, element := range c.Compactors[level].sort(less) {
+		if (jitterIndex%2 == 0) == selectEvens {
+			compacted = append(compacted, element)
+		}
+		jitterIndex++
+	}
+	if len(compacted) > 0 {
+		c.Compactors[level+1].sorted = append(c.Compactors[level+1].sorted, compacted)
+	}
+	// Clear out the compactor at this level since we've finished compacting it. The compacted elements have already been moved to the next compactor.
+	c.Compactors[level].sorted = nil
+	c.Compactors[level].unsorted = nil
+}
+
+func encodeCompactors(c *compactors) ([]byte, error) {
+	var buf bytes.Buffer
+	enc := gob.NewEncoder(&buf)
+	if err := enc.Encode(c); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func decodeCompactors(data []byte) (*compactors, error) {
+	var compactors compactors
+	dec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err := dec.Decode(&compactors); err != nil {
+		return nil, err
+	}
+	for level := range compactors.Compactors {
+		compactors.Compactors[level].capacity = compactors.capacity(level)
+	}
+	return &compactors, nil
+}
+
+// mergeSorted takes two slices which are already sorted and returns a new slice containing all elements sorted together.
+func mergeSorted(a, b []beam.T, less func(interface{}, interface{}) bool) []beam.T {
+	output := make([]beam.T, 0, len(a)+len(b))
+	for len(a) > 0 && len(b) > 0 {
+		if less(a[0], b[0]) {
+			output = append(output, a[0])
+			a = a[1:]
+		} else {
+			output = append(output, b[0])
+			b = b[1:]
+		}
+	}
+	if len(a) > 0 {
+		output = append(output, a...)
+	} else {
+		output = append(output, b...)
+	}
+	return output
+}
+
+// mergeSortedWeighted takes two slices which are already sorted and returns a new slice containing all elements sorted together.
+func mergeSortedWeighted(a, b []weightedElement, less func(interface{}, interface{}) bool) []weightedElement {
+	output := make([]weightedElement, 0, len(a)+len(b))
+	for len(a) > 0 && len(b) > 0 {
+		if less(a[0], b[0]) {
+			output = append(output, a[0])
+			a = a[1:]
+		} else {
+			output = append(output, b[0])
+			b = b[1:]
+		}
+	}
+	if len(a) > 0 {
+		output = append(output, a...)
+	} else {
+		output = append(output, b...)
+	}
+	return output
+}
+
+// merge joins two compactors together.
+func (c *compactors) merge(other *compactors, less reflectx.Func2x1) {
+	for level := range c.Compactors {
+		if len(other.Compactors)-1 < level {
+			break
+		}
+		c.Compactors[level].unsorted = append(c.Compactors[level].unsorted, other.Compactors[level].unsorted...)
+		c.Compactors[level].sorted = append(c.Compactors[level].sorted, other.Compactors[level].sorted...)
+	}
+	if len(other.Compactors) > len(c.Compactors) {
+		c.Compactors = append(c.Compactors, other.Compactors[len(c.Compactors):]...)
+	}
+	c.NumberOfCompactions += other.NumberOfCompactions
+	c.compact(less)
+}
+
+// approximateQuantilesCombineFnState contains the payload for the combiners.
+// Ideally this would be a single combine function but in certain cases, runners attempts to do all the merges on a single machine.
+// Unfortunately the merges can be slow for extremely large datasets and large values of K. If the merge takes too long, it will get canceled and the job will never complete.
+// Thus we split up the combiners into multiple functions to force the runner to do the work in parallel.
+// This state can be shared across all of the split-up functions.
+type approximateQuantilesCombineFnState struct {
+	// The size of the compactors.
+	// The memory consumed, and the error are controlled by this parameter.
+	K int `json:"k"`
+	// Used to compare elements.
+	LessFunc beam.EncodedFunc
+	// Internally cached instance.
+	less         reflectx.Func2x1
+	NumQuantiles int `json:"numQuantiles"`
+}
+
+func (f *approximateQuantilesCombineFnState) setup() error {
+	f.less = reflectx.ToFunc2x1(f.LessFunc.Fn)
+	return nil
+}
+
+func (f *approximateQuantilesCombineFnState) createAccumulator() *compactors {
+	return &compactors{
+		K:          f.K,
+		Compactors: []compactor{{capacity: f.K}},
+	}
+}
+
+// approximateQuantilesOutputFn extracts the final output containing the quantiles.
+type approximateQuantilesOutputFn struct {
+	State approximateQuantilesCombineFnState `json:"state"`
+}
+
+func (f *approximateQuantilesOutputFn) Setup() error {
+	return f.State.setup()
+}
+
+func (f *approximateQuantilesOutputFn) CreateAccumulator() *compactors {
+	return f.State.createAccumulator()
+}
+
+func (f *approximateQuantilesOutputFn) AddInput(compactors *compactors, element *compactors) *compactors {
+	compactors.merge(element, f.State.less)
+	return compactors
+}
+
+func (f *approximateQuantilesOutputFn) MergeAccumulators(ctx context.Context, a, b *compactors) *compactors {
+	a.merge(b, f.State.less)
+	return a
+}
+
+type weightedElementAsGob struct {
+	EncodedTypeAsJSON []byte
+	Weight            int
+	Element           []byte
+}
+
+func encodeWeightedElement(element weightedElement) ([]byte, error) {
+	t := reflect.TypeOf(element.element)
+	enc := beam.NewElementEncoder(t)
+	var buf bytes.Buffer
+	if err := enc.Encode(element.element, &buf); err != nil {
+		return nil, err
+	}
+	tAsJSON, err := beam.EncodedType{T: t}.MarshalJSON()
+	if err != nil {
+		return nil, err
+	}
+	var gobBuf bytes.Buffer
+	if err := gob.NewEncoder(&gobBuf).Encode(weightedElementAsGob{
+		EncodedTypeAsJSON: tAsJSON,
+		Weight:            element.weight,
+		Element:           buf.Bytes(),
+	}); err != nil {
+		return nil, err
+	}
+	return gobBuf.Bytes(), nil
+}
+
+func decodeWeightedElement(data []byte) (weightedElement, error) {
+	var g weightedElementAsGob
+	dec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err := dec.Decode(&g); err != nil {
+		return weightedElement{}, err
+	}
+	var t beam.EncodedType
+	if err := t.UnmarshalJSON(g.EncodedTypeAsJSON); err != nil {
+		return weightedElement{}, err
+	}
+	element, err := beam.NewElementDecoder(t.T).Decode(bytes.NewBuffer(g.Element))
+	if err != nil {
+		return weightedElement{}, err
+	}
+	return weightedElement{
+		weight:  g.Weight,
+		element: element,
+	}, nil
+}
+
+type weightedElement struct {
+	weight  int
+	element beam.T
+}
+
+func toWeightedSlice(compactor compactor, less reflectx.Func2x1, weight int) []weightedElement {
+	sorted := compactor.sort(less)
+	weightedElements := make([]weightedElement, 0, len(sorted))
+	for _, element := range sorted {
+		weightedElements = append(weightedElements, weightedElement{weight: weight, element: element})
+	}
+	return weightedElements
+}
+func (f *approximateQuantilesOutputFn) ExtractOutput(ctx context.Context, compactors *compactors) []beam.T {
+	sorted := toWeightedSlice(compactors.Compactors[0], f.State.less, 1)
+	for level, compactor := range compactors.Compactors[1:] {
+		sorted = mergeSortedWeighted(sorted, toWeightedSlice(compactor, f.State.less, 1<<uint(level)), func(a, b interface{}) bool {
+			return f.State.less.Call2x1(a.(weightedElement).element, b.(weightedElement).element).(bool)
+		})
+	}
+	totalElements := 0
+	for _, element := range sorted {
+		totalElements += element.weight
+	}
+	ret := make([]beam.T, 0, f.State.NumQuantiles)
+	// Rank represents the estimate of how many elements we've seen as we iterate through the sorted list of elements stored in the compactors.
+	// Recall that each element stored in a compactor is also assigned a weight indicating how many elements from the input it represents.
+	rank := float64(0)
+	// Represents the quantile we're currently searching for.
+	currentQuantile := float64(1)
+	for _, element := range sorted {
+		rank += float64(element.weight)
+		if rank/float64(totalElements) >= currentQuantile/float64(f.State.NumQuantiles) {
+			ret = append(ret, element.element)
+			currentQuantile++
+		}
+		if currentQuantile >= float64(f.State.NumQuantiles) {
+			break
+		}
+	}
+	return ret
+}
+
+// approximateQuantilesInputFn combines elements into compactors, but not necessarily the final compactor.
+type approximateQuantilesInputFn approximateQuantilesOutputFn
+
+func (f *approximateQuantilesInputFn) Setup() error {
+	return f.State.setup()
+}
+
+func (f *approximateQuantilesInputFn) CreateAccumulator() *compactors {
+	return f.State.createAccumulator()
+}
+
+func (f *approximateQuantilesInputFn) AddInput(compactors *compactors, element weightedElement) *compactors {
+	compactors.update(element.element, element.weight, f.State.less)
+	return compactors
+}
+
+func (f *approximateQuantilesInputFn) MergeAccumulators(ctx context.Context, a, b *compactors) *compactors {
+	a.merge(b, f.State.less)
+	return a
+}
+
+func (f *approximateQuantilesInputFn) ExtractOutput(ctx context.Context, compactors *compactors) *compactors {
+	for i := range compactors.Compactors {
+		// Sort the compactors here so when we're merging them for the final output, they're already sorted and we can merge elements in order.
+		compactors.Compactors[i].sort(f.State.less)
+	}
+	return compactors
+}
+
+// approximateQuantilesMergeOnlyFn combines compactors into smaller compactors, but not necessarily the final compactor.
+type approximateQuantilesMergeOnlyFn approximateQuantilesOutputFn
+
+func (f *approximateQuantilesMergeOnlyFn) Setup() error {
+	return f.State.setup()
+}
+
+func (f *approximateQuantilesMergeOnlyFn) CreateAccumulator() *compactors {
+	return f.State.createAccumulator()
+}
+
+func (f *approximateQuantilesMergeOnlyFn) AddInput(compactors *compactors, element *compactors) *compactors {
+	compactors.merge(element, f.State.less)
+	return compactors
+}
+
+func (f *approximateQuantilesMergeOnlyFn) MergeAccumulators(ctx context.Context, a, b *compactors) *compactors {
+	a.merge(b, f.State.less)
+	return a
+}
+
+func (f *approximateQuantilesMergeOnlyFn) ExtractOutput(ctx context.Context, compactors *compactors) *compactors {
+	for i := range compactors.Compactors {
+		// Sort the compactors here so when we're merging them for the final output, they're already sorted and we can merge elements in order.
+		compactors.Compactors[i].sort(f.State.less)
+	}
+	return compactors
+}
+
+type shardElementsFn struct {
+	Shards         int              `json:"shards"`
+	T              beam.EncodedType `json:"t"`
+	elementEncoder beam.ElementEncoder
+}
+
+func (s *shardElementsFn) Setup() {
+	s.elementEncoder = beam.NewElementEncoder(s.T.T)
+}
+
+func (s *shardElementsFn) ProcessElement(element beam.T) (int, beam.T) {
+	h := crc32.NewIEEE()
+	s.elementEncoder.Encode(element, h)
+	return int(h.Sum32()) % s.Shards, element
+}
+
+func makeWeightedElement(weight int, element beam.T) weightedElement {
+	return weightedElement{weight: weight, element: element}
+}
+
+// ApproximateQuantiles computes approximate quantiles for the input PCollection<T>.
+//
+// The output PCollection contains a single element: a list of numQuantiles - 1 elements approximately splitting up the input collection into numQuantiles separate quantiles.
+// For example, if numQuantiles = 2, the returned list would contain a single element such that approximately half of the input would be less than that element and half would be greater.
+func ApproximateQuantiles(s beam.Scope, pc beam.PCollection, less interface{}, opts Opts) beam.PCollection {
+	return ApproximateWeightedQuantiles(s, beam.ParDo(s, func(e beam.T) (int, beam.T) { return 1, e }, pc), less, opts)
+}
+
+// reduce takes a PCollection<weightedElementWrapper> and returns a PCollection<*compactors>. The output PCollection may have at most shardSizes[len(shardSizes) - 1] compactors.
+func reduce(s beam.Scope, weightedElements beam.PCollection, state approximateQuantilesCombineFnState, shardSizes []int) beam.PCollection {
+	if len(shardSizes) == 0 {
+		shardSizes = []int{1}
+	}
+	shardedCompactors := beam.DropKey(s,
+		beam.CombinePerKey(s, &approximateQuantilesInputFn{State: state},
+			beam.ParDo(s, &shardElementsFn{Shards: shardSizes[0], T: beam.EncodedType{T: reflect.TypeOf((*weightedElement)(nil)).Elem()}}, weightedElements)))
+	shardSizes = shardSizes[1:]
+	compactorsType := reflect.TypeOf((**compactors)(nil)).Elem()
+	for _, shardSize := range shardSizes {
+		shardedCompactors = beam.DropKey(s,

Review comment:
       Same here please.

##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles_test.go
##########
@@ -0,0 +1,291 @@
+// 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 stats
+
+import (
+	"reflect"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/passert"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/ptest"
+)
+
+func init() {
+	beam.RegisterFunction(weightedElementToKv)
+
+	// In practice, this runs faster than plain reflection.
+	reflectx.RegisterFunc(reflect.ValueOf(less).Type(), func(_ interface{}) reflectx.Func {
+		return newIntLess()
+	})
+}
+
+type intLess struct {
+	name string
+	t    reflect.Type
+}
+
+func newIntLess() *intLess {
+	return &intLess{
+		name: reflectx.FunctionName(reflect.ValueOf(less).Interface()),
+		t:    reflect.ValueOf(less).Type(),
+	}
+}
+
+func (i *intLess) Name() string {
+	return i.name
+}
+func (i *intLess) Type() reflect.Type {
+	return i.t
+}
+func (i *intLess) Call(args []interface{}) []interface{} {
+	return []interface{}{args[0].(int) < args[1].(int)}
+}
+
+func less(a, b int) bool {
+	return a < b
+}
+
+func TestLargeQuantiles(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := 0; i < numElements; i++ {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{10006, 19973}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestLargeQuantilesReversed(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := numElements - 1; i >= 0; i-- {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{9985, 19959}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestBasicQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	tests := []struct {
+		Input    []int
+		Expected [][]int
+	}{
+		{[]int{}, [][]int{}},
+		{[]int{1}, [][]int{[]int{1}}},
+		{[]int{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20}, [][]int{[]int{6, 13}}},
+	}
+
+	for _, test := range tests {
+		p, s, in, exp := ptest.CreateList2(test.Input, test.Expected)
+		quantiles := ApproximateQuantiles(s, in, less, Opts{
+			K:            3,
+			NumQuantiles: 3,
+		})
+		passert.Equals(s, quantiles, exp)
+
+		if err := ptest.Run(p); err != nil {
+			t.Errorf("ApproximateQuantiles(%v) != %v: %v", test.Input, test.Expected, err)
+		}
+	}
+}
+
+func weightedElementToKv(e testWeightedElement) (int, int) {
+	return e.Weight, e.Element
+}
+
+type testWeightedElement struct {
+	Weight  int
+	Element int
+}
+
+func TestWeightedQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	input := []testWeightedElement{
+		{Weight: 1, Element: 1},
+		{Weight: 10, Element: 2},
+		{Weight: 1, Element: 3},
+		{Weight: 10, Element: 4}}
+	expected := []int{2, 4}
+	p, s, in, exp := ptest.CreateList2(input, [][]int{expected})
+	quantiles := ApproximateWeightedQuantiles(
+		s,
+		beam.ParDo(
+			s,
+			weightedElementToKv,
+			in,
+			beam.TypeDefinition{Var: beam.TType, T: reflect.TypeOf(1)}),
+		less,
+		Opts{
+			K:            3,
+			NumQuantiles: 3,
+		})
+	passert.Equals(
+		s,
+		quantiles,
+		exp)

Review comment:
       This can fit back onto a single line.

##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles_test.go
##########
@@ -0,0 +1,291 @@
+// 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 stats
+
+import (
+	"reflect"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/passert"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/ptest"
+)
+
+func init() {
+	beam.RegisterFunction(weightedElementToKv)
+
+	// In practice, this runs faster than plain reflection.
+	reflectx.RegisterFunc(reflect.ValueOf(less).Type(), func(_ interface{}) reflectx.Func {
+		return newIntLess()
+	})
+}
+
+type intLess struct {
+	name string
+	t    reflect.Type
+}
+
+func newIntLess() *intLess {
+	return &intLess{
+		name: reflectx.FunctionName(reflect.ValueOf(less).Interface()),
+		t:    reflect.ValueOf(less).Type(),
+	}
+}
+
+func (i *intLess) Name() string {
+	return i.name
+}
+func (i *intLess) Type() reflect.Type {
+	return i.t
+}
+func (i *intLess) Call(args []interface{}) []interface{} {
+	return []interface{}{args[0].(int) < args[1].(int)}
+}
+
+func less(a, b int) bool {
+	return a < b
+}
+
+func TestLargeQuantiles(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := 0; i < numElements; i++ {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{10006, 19973}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestLargeQuantilesReversed(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := numElements - 1; i >= 0; i-- {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{9985, 19959}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestBasicQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	tests := []struct {
+		Input    []int
+		Expected [][]int
+	}{
+		{[]int{}, [][]int{}},
+		{[]int{1}, [][]int{[]int{1}}},
+		{[]int{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20}, [][]int{[]int{6, 13}}},
+	}
+
+	for _, test := range tests {
+		p, s, in, exp := ptest.CreateList2(test.Input, test.Expected)
+		quantiles := ApproximateQuantiles(s, in, less, Opts{
+			K:            3,
+			NumQuantiles: 3,
+		})
+		passert.Equals(s, quantiles, exp)
+
+		if err := ptest.Run(p); err != nil {
+			t.Errorf("ApproximateQuantiles(%v) != %v: %v", test.Input, test.Expected, err)
+		}
+	}
+}
+
+func weightedElementToKv(e testWeightedElement) (int, int) {
+	return e.Weight, e.Element
+}
+
+type testWeightedElement struct {
+	Weight  int
+	Element int
+}
+
+func TestWeightedQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	input := []testWeightedElement{
+		{Weight: 1, Element: 1},
+		{Weight: 10, Element: 2},
+		{Weight: 1, Element: 3},
+		{Weight: 10, Element: 4}}
+	expected := []int{2, 4}
+	p, s, in, exp := ptest.CreateList2(input, [][]int{expected})
+	quantiles := ApproximateWeightedQuantiles(

Review comment:
       Unnest for readability please.

##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles_test.go
##########
@@ -0,0 +1,291 @@
+// 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 stats
+
+import (
+	"reflect"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/passert"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/ptest"
+)
+
+func init() {
+	beam.RegisterFunction(weightedElementToKv)
+
+	// In practice, this runs faster than plain reflection.
+	reflectx.RegisterFunc(reflect.ValueOf(less).Type(), func(_ interface{}) reflectx.Func {
+		return newIntLess()
+	})
+}
+
+type intLess struct {
+	name string
+	t    reflect.Type
+}
+
+func newIntLess() *intLess {
+	return &intLess{
+		name: reflectx.FunctionName(reflect.ValueOf(less).Interface()),
+		t:    reflect.ValueOf(less).Type(),
+	}
+}
+
+func (i *intLess) Name() string {
+	return i.name
+}
+func (i *intLess) Type() reflect.Type {
+	return i.t
+}
+func (i *intLess) Call(args []interface{}) []interface{} {
+	return []interface{}{args[0].(int) < args[1].(int)}
+}
+
+func less(a, b int) bool {
+	return a < b
+}
+
+func TestLargeQuantiles(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := 0; i < numElements; i++ {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{10006, 19973}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestLargeQuantilesReversed(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := numElements - 1; i >= 0; i-- {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{9985, 19959}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestBasicQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	tests := []struct {
+		Input    []int
+		Expected [][]int
+	}{
+		{[]int{}, [][]int{}},
+		{[]int{1}, [][]int{[]int{1}}},
+		{[]int{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20}, [][]int{[]int{6, 13}}},
+	}
+
+	for _, test := range tests {
+		p, s, in, exp := ptest.CreateList2(test.Input, test.Expected)
+		quantiles := ApproximateQuantiles(s, in, less, Opts{
+			K:            3,
+			NumQuantiles: 3,
+		})
+		passert.Equals(s, quantiles, exp)
+
+		if err := ptest.Run(p); err != nil {
+			t.Errorf("ApproximateQuantiles(%v) != %v: %v", test.Input, test.Expected, err)
+		}
+	}
+}
+
+func weightedElementToKv(e testWeightedElement) (int, int) {
+	return e.Weight, e.Element
+}
+
+type testWeightedElement struct {
+	Weight  int
+	Element int
+}
+
+func TestWeightedQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	input := []testWeightedElement{
+		{Weight: 1, Element: 1},
+		{Weight: 10, Element: 2},
+		{Weight: 1, Element: 3},
+		{Weight: 10, Element: 4}}
+	expected := []int{2, 4}
+	p, s, in, exp := ptest.CreateList2(input, [][]int{expected})
+	quantiles := ApproximateWeightedQuantiles(
+		s,
+		beam.ParDo(
+			s,
+			weightedElementToKv,
+			in,
+			beam.TypeDefinition{Var: beam.TType, T: reflect.TypeOf(1)}),
+		less,
+		Opts{
+			K:            3,
+			NumQuantiles: 3,
+		})
+	passert.Equals(
+		s,
+		quantiles,
+		exp)
+
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles(%v) != %v: %v", input, expected, err)
+	}
+}
+
+func TestWeightedQuantilesWithInternalSharding(t *testing.T) {
+	// Test shard reduction.
+	input := []testWeightedElement{
+		{Weight: 1, Element: 1},
+		{Weight: 10, Element: 2},
+		{Weight: 1, Element: 3},
+		{Weight: 10, Element: 4}}
+	expected := []int{2, 4}
+	p, s, in, exp := ptest.CreateList2(input, [][]int{expected})
+	quantiles := ApproximateWeightedQuantiles(
+		s,
+		beam.ParDo(
+			s,
+			weightedElementToKv,
+			in,
+			beam.TypeDefinition{Var: beam.TType, T: reflect.TypeOf(1)}),
+		less,
+		Opts{
+			K:                3,
+			NumQuantiles:     3,
+			InternalSharding: []int{4, 3, 2},
+		})
+	passert.Equals(
+		s,
+		quantiles,
+		exp)

Review comment:
       Same here, single line.

##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles.go
##########
@@ -0,0 +1,702 @@
+// 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 stats
+
+// Approximate quantiles is implemented based on https://arxiv.org/pdf/1907.00236.pdf.
+
+import (
+	"bytes"
+	"container/heap"
+	"context"
+	"encoding/gob"
+	"encoding/json"
+	"hash/crc32"
+	"io"
+	"math"
+	"reflect"
+	"sort"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+)
+
+func init() {
+	compactorsType := reflect.TypeOf((**compactors)(nil)).Elem()
+	weightedElementType := reflect.TypeOf((*weightedElement)(nil)).Elem()
+	beam.RegisterType(compactorsType)
+	beam.RegisterType(weightedElementType)
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesInputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesMergeOnlyFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesOutputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*shardElementsFn)(nil)).Elem())
+	beam.RegisterCoder(compactorsType, encodeCompactors, decodeCompactors)
+	beam.RegisterCoder(weightedElementType, encodeWeightedElement, decodeWeightedElement)
+}
+
+// Opts contains settings used to configure how approximate quantiles are computed.
+type Opts struct {
+	// Controls the memory used and approximation error (difference between the quantile returned and the true quantile.)
+	K int
+	// Number of quantiles to return. The algorithm will return NumQuantiles - 1 numbers
+	NumQuantiles int
+	// For extremely large datasets, the Go SDK has some problems holding on to data and might OOM.
+	// If ApproximateQuantiles is running out of memory, you can use this option to tune how the data is sharded internally.
+	// This parameter is optional. If unspecified, Beam will compact all elements into a single compactor at once.
+	InternalSharding []int
+}
+
+// The paper suggests reducing the size of the lower-level compactors as we grow.
+// We reduce the capacity at this rate.
+// The paper suggests 1/sqrt(2) is ideal. That's approximately 0.7.
+const capacityCoefficient float64 = 0.7
+
+type sortListHeap struct {
+	data [][]beam.T
+	less reflectx.Func2x1
+}
+
+func (s sortListHeap) Len() int            { return len(s.data) }
+func (s sortListHeap) Less(i, j int) bool  { return s.less.Call2x1(s.data[i][0], s.data[j][0]).(bool) }
+func (s sortListHeap) Swap(i, j int)       { s.data[i], s.data[j] = s.data[j], s.data[i] }
+func (s *sortListHeap) Push(x interface{}) { s.data = append(s.data, x.([]beam.T)) }
+func (s *sortListHeap) Pop() interface{} {
+	var x beam.T
+	x, s.data = s.data[len(s.data)-1], s.data[:len(s.data)-1]
+	return x
+}
+
+// compactor contains elements to be compacted.
+type compactor struct {
+	// Compaction needs to sort elements before compacting. Thus in practice, we should often have some pre-sorted data.
+	// We want to keep it separate so we can sort only the unsorted data and merge the two sorted lists together.
+	// If we're only receiving elements of weight 1, only level 0 will ever contain unsorted data and the rest of the levels will always remain sorted.
+	// To prevent repeated allocation/copying, we keep multiple sorted lists and then merge them together
+	sorted   [][]beam.T
+	unsorted []beam.T
+	// How many items should be stored in this compactor before it should get compacted.
+	// Note that this is not a hard limit.
+	// The paper suggests implementing lazy compaction which would allow
+	// compactors to temporarily exceed their capacity as long as the total
+	// elements in all compactors doesn't exceed the total capacity in all
+	// compactors. In other words, compactors can temporarily borrow capacity
+	// from each other.
+	// In the paper, this is referred to as the variable k_h.
+	capacity int
+}
+
+type compactorAsGob struct {
+	Sorted            [][]byte
+	Unsorted          []byte
+	EncodedTypeAsJSON []byte
+}
+
+func encodeElements(enc beam.ElementEncoder, elements []beam.T) ([]byte, error) {
+	var buf bytes.Buffer
+	for _, e := range elements {
+		if err := enc.Encode(e, &buf); err != nil {
+			return nil, err
+		}
+	}
+	return buf.Bytes(), nil
+}
+
+func (c *compactor) getElementType() reflect.Type {
+	for _, e := range c.sorted {
+		for _, e2 := range e {
+			return reflect.TypeOf(e2)
+		}
+	}
+	for _, e := range c.unsorted {
+		return reflect.TypeOf(e)
+	}
+	return nil
+}
+
+func (c *compactor) MarshalBinary() ([]byte, error) {
+	t := c.getElementType()
+	var buf bytes.Buffer
+	if t == nil {
+		enc := gob.NewEncoder(&buf)
+		if err := enc.Encode(compactorAsGob{}); err != nil {
+			return nil, err
+		}
+		return buf.Bytes(), nil
+	}
+	enc := beam.NewElementEncoder(t)
+	encodedSorted := make([][]byte, 0, len(c.sorted))
+	for _, sorted := range c.sorted {
+		encoded, err := encodeElements(enc, sorted)
+		if err != nil {
+			return nil, err
+		}
+		encodedSorted = append(encodedSorted, encoded)
+	}
+	encodedUnsorted, err := encodeElements(enc, c.unsorted)
+	if err != nil {
+		return nil, err
+	}
+	tAsJSON, err := beam.EncodedType{T: t}.MarshalJSON()
+	if err != nil {
+		return nil, err
+	}
+	gobEnc := gob.NewEncoder(&buf)
+	if err = gobEnc.Encode(compactorAsGob{
+		Sorted:            encodedSorted,
+		Unsorted:          encodedUnsorted,
+		EncodedTypeAsJSON: tAsJSON,
+	}); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func decodeElements(dec beam.ElementDecoder, data []byte) ([]beam.T, error) {
+	buf := bytes.NewBuffer(data)
+	ret := []beam.T{}
+	for {
+		element, err := dec.Decode(buf)
+		if err == io.EOF {
+			return ret, nil
+		} else if err != nil {
+			return nil, err
+		}
+		ret = append(ret, element)
+	}
+}
+
+func (c *compactor) UnmarshalBinary(data []byte) error {
+	var g compactorAsGob
+	var err error
+	gobDec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err = gobDec.Decode(&g); err != nil {
+		return err
+	}
+	if len(g.EncodedTypeAsJSON) == 0 {
+		return nil
+	}
+	var t beam.EncodedType
+	if err = json.Unmarshal(g.EncodedTypeAsJSON, &t); err != nil {
+		return err
+	}
+	dec := beam.NewElementDecoder(t.T)
+	decodedSorted := make([][]beam.T, 0, len(g.Sorted))
+	for _, sorted := range g.Sorted {
+		decoded, err := decodeElements(dec, sorted)
+		if err != nil {
+			return err
+		}
+		decodedSorted = append(decodedSorted, decoded)
+	}
+	c.sorted = decodedSorted
+	if c.unsorted, err = decodeElements(dec, g.Unsorted); err != nil {
+		return err
+	}
+	return nil
+}
+
+// update inserts an element into the compactor.
+func (c *compactor) update(element beam.T) {
+	c.unsorted = append(c.unsorted, element)
+}
+
+// size returns the number of elements stored in this compactor.
+func (c *compactor) size() int {
+	size := 0
+	for _, s := range c.sorted {
+		size += len(s)
+	}
+	return len(c.unsorted) + size
+}
+
+type sorter struct {
+	less reflectx.Func2x1
+	data []beam.T
+}
+
+func (s sorter) Len() int           { return len(s.data) }
+func (s sorter) Less(i, j int) bool { return s.less.Call2x1(s.data[i], s.data[j]).(bool) }
+func (s sorter) Swap(i, j int)      { s.data[i], s.data[j] = s.data[j], s.data[i] }
+
+// sort sorts the compactor and returns all the elements in sorted order.
+func (c *compactor) sort(less reflectx.Func2x1) []beam.T {
+	sort.Sort(sorter{data: c.unsorted, less: less})
+	h := sortListHeap{data: c.sorted, less: less}
+	heap.Init(&h)
+	sorted := make([]beam.T, 0, c.size()-len(c.unsorted))
+	for h.Len() > 0 {
+		s := heap.Pop(&h).([]beam.T)
+		sorted = append(sorted, s[0])
+		if len(s) > 1 {
+			heap.Push(&h, s[1:])
+		}
+	}
+	c.sorted = [][]beam.T{mergeSorted(sorted, c.unsorted, func(a, b interface{}) bool { return less.Call2x1(a, b).(bool) })}
+	c.unsorted = nil
+	if len(c.sorted[0]) == 0 {
+		c.sorted = nil
+		return nil
+	}
+	return c.sorted[0]
+}
+
+// Compactors holds the state of the quantile approximation compactors.
+type compactors struct {
+	// References "K" from the paper which influences the amount of memory used.
+	K int
+	// When compacting, we want to alternate between taking elements at even vs odd indices.
+	// The paper suggests using a random variable but we'd prefer to stay deterministic.
+	// Especially when merging two compactors we want to keep track of how often we've selected odds vs evens.
+	NumberOfCompactions int
+
+	// Each compactor takes a sample of elements.
+	// The "height" (also known as the index in this slice) of the compactor determines the weight of its elements.
+	// The weight of a compactor of height h is 2^h.
+	// For example, for h = 3 (which would be compactors[3]), the weight is 2^3 = 8. That means each element in that compactor represents 8 instances of itself.
+	Compactors []compactor
+}
+
+func (c *compactors) totalCapacity() int {
+	totalCapacity := 0
+	for _, compactor := range c.Compactors {
+		totalCapacity += compactor.capacity
+	}
+	return totalCapacity
+}
+
+func (c *compactors) size() int {
+	size := 0
+	for _, compactor := range c.Compactors {
+		size += compactor.size()
+	}
+	return size
+}
+
+// capacity computes the capacity of a compactor at a certain level.
+// The paper suggests decreasing the capacity of lower-leveled compactors as we add more elements.
+func (c *compactors) capacity(compactorLevel int) int {
+	return int(math.Ceil(math.Pow(capacityCoefficient, float64(len(c.Compactors)-compactorLevel-1))*float64(c.K))) + 1
+}
+
+// compact compacts all compactors until the total size is less than the maximum capacity of all compactors.
+func (c *compactors) compact(less reflectx.Func2x1) {
+	for c.size() > c.totalCapacity() {
+		for level, compactor := range c.Compactors {
+			if compactor.size() > compactor.capacity {
+				c.compactLevel(level, less)
+				// Merging compactors can cause us to exceed max capacity in multiple compactors.
+				if c.size() < c.totalCapacity() {
+					// Do lazy compaction as described in the paper.
+					break
+				}
+			}
+		}
+	}
+}
+
+// update inserts the given element into the compactors. If this element causes the compactors to grow too large, we perform the compaction here.
+func (c *compactors) update(element beam.T, weight int, less reflectx.Func2x1) {
+	level := int(math.Log2(float64(weight)))
+	c.growToIncludeLevel(level)
+	c.Compactors[level].update(element)
+	// From the paper, we're using the "Splitting the Input" approach.
+	remainingWeight := weight - (1 << uint(level))
+	// Only attempt compaction if we're doing the last update. Otherwise we'd be compacting too often.
+	if remainingWeight <= 0 {
+		c.compact(less)
+	} else {
+		c.update(element, remainingWeight, less)
+	}
+}
+
+// growToIncludeLevel ensures we have compactors available at the given level.
+func (c *compactors) growToIncludeLevel(level int) {
+	if len(c.Compactors)-1 >= level {
+		return
+	}
+	for i := len(c.Compactors) - 1; i < level; i++ {
+		c.Compactors = append(c.Compactors, compactor{})
+	}
+	for level := range c.Compactors {
+		c.Compactors[level].capacity = c.capacity(level)
+	}
+}
+
+// compact compacts elements in compactors.

Review comment:
       This is just to say, I love this comment.

##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles.go
##########
@@ -0,0 +1,702 @@
+// 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 stats
+
+// Approximate quantiles is implemented based on https://arxiv.org/pdf/1907.00236.pdf.
+
+import (
+	"bytes"
+	"container/heap"
+	"context"
+	"encoding/gob"
+	"encoding/json"
+	"hash/crc32"
+	"io"
+	"math"
+	"reflect"
+	"sort"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+)
+
+func init() {
+	compactorsType := reflect.TypeOf((**compactors)(nil)).Elem()
+	weightedElementType := reflect.TypeOf((*weightedElement)(nil)).Elem()
+	beam.RegisterType(compactorsType)
+	beam.RegisterType(weightedElementType)
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesInputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesMergeOnlyFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesOutputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*shardElementsFn)(nil)).Elem())
+	beam.RegisterCoder(compactorsType, encodeCompactors, decodeCompactors)
+	beam.RegisterCoder(weightedElementType, encodeWeightedElement, decodeWeightedElement)
+}
+
+// Opts contains settings used to configure how approximate quantiles are computed.
+type Opts struct {
+	// Controls the memory used and approximation error (difference between the quantile returned and the true quantile.)
+	K int
+	// Number of quantiles to return. The algorithm will return NumQuantiles - 1 numbers
+	NumQuantiles int
+	// For extremely large datasets, the Go SDK has some problems holding on to data and might OOM.
+	// If ApproximateQuantiles is running out of memory, you can use this option to tune how the data is sharded internally.
+	// This parameter is optional. If unspecified, Beam will compact all elements into a single compactor at once.
+	InternalSharding []int
+}
+
+// The paper suggests reducing the size of the lower-level compactors as we grow.
+// We reduce the capacity at this rate.
+// The paper suggests 1/sqrt(2) is ideal. That's approximately 0.7.
+const capacityCoefficient float64 = 0.7
+
+type sortListHeap struct {
+	data [][]beam.T
+	less reflectx.Func2x1
+}
+
+func (s sortListHeap) Len() int            { return len(s.data) }
+func (s sortListHeap) Less(i, j int) bool  { return s.less.Call2x1(s.data[i][0], s.data[j][0]).(bool) }
+func (s sortListHeap) Swap(i, j int)       { s.data[i], s.data[j] = s.data[j], s.data[i] }
+func (s *sortListHeap) Push(x interface{}) { s.data = append(s.data, x.([]beam.T)) }
+func (s *sortListHeap) Pop() interface{} {
+	var x beam.T
+	x, s.data = s.data[len(s.data)-1], s.data[:len(s.data)-1]
+	return x
+}
+
+// compactor contains elements to be compacted.
+type compactor struct {
+	// Compaction needs to sort elements before compacting. Thus in practice, we should often have some pre-sorted data.
+	// We want to keep it separate so we can sort only the unsorted data and merge the two sorted lists together.
+	// If we're only receiving elements of weight 1, only level 0 will ever contain unsorted data and the rest of the levels will always remain sorted.
+	// To prevent repeated allocation/copying, we keep multiple sorted lists and then merge them together
+	sorted   [][]beam.T
+	unsorted []beam.T
+	// How many items should be stored in this compactor before it should get compacted.
+	// Note that this is not a hard limit.
+	// The paper suggests implementing lazy compaction which would allow
+	// compactors to temporarily exceed their capacity as long as the total
+	// elements in all compactors doesn't exceed the total capacity in all
+	// compactors. In other words, compactors can temporarily borrow capacity
+	// from each other.
+	// In the paper, this is referred to as the variable k_h.
+	capacity int
+}
+
+type compactorAsGob struct {
+	Sorted            [][]byte
+	Unsorted          []byte
+	EncodedTypeAsJSON []byte
+}
+
+func encodeElements(enc beam.ElementEncoder, elements []beam.T) ([]byte, error) {
+	var buf bytes.Buffer
+	for _, e := range elements {
+		if err := enc.Encode(e, &buf); err != nil {
+			return nil, err
+		}
+	}
+	return buf.Bytes(), nil
+}
+
+func (c *compactor) getElementType() reflect.Type {
+	for _, e := range c.sorted {
+		for _, e2 := range e {
+			return reflect.TypeOf(e2)
+		}
+	}
+	for _, e := range c.unsorted {
+		return reflect.TypeOf(e)
+	}
+	return nil
+}
+
+func (c *compactor) MarshalBinary() ([]byte, error) {
+	t := c.getElementType()
+	var buf bytes.Buffer
+	if t == nil {
+		enc := gob.NewEncoder(&buf)
+		if err := enc.Encode(compactorAsGob{}); err != nil {
+			return nil, err
+		}
+		return buf.Bytes(), nil
+	}
+	enc := beam.NewElementEncoder(t)
+	encodedSorted := make([][]byte, 0, len(c.sorted))
+	for _, sorted := range c.sorted {
+		encoded, err := encodeElements(enc, sorted)
+		if err != nil {
+			return nil, err
+		}
+		encodedSorted = append(encodedSorted, encoded)
+	}
+	encodedUnsorted, err := encodeElements(enc, c.unsorted)
+	if err != nil {
+		return nil, err
+	}
+	tAsJSON, err := beam.EncodedType{T: t}.MarshalJSON()
+	if err != nil {
+		return nil, err
+	}
+	gobEnc := gob.NewEncoder(&buf)
+	if err = gobEnc.Encode(compactorAsGob{
+		Sorted:            encodedSorted,
+		Unsorted:          encodedUnsorted,
+		EncodedTypeAsJSON: tAsJSON,
+	}); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func decodeElements(dec beam.ElementDecoder, data []byte) ([]beam.T, error) {
+	buf := bytes.NewBuffer(data)
+	ret := []beam.T{}
+	for {
+		element, err := dec.Decode(buf)
+		if err == io.EOF {
+			return ret, nil
+		} else if err != nil {
+			return nil, err
+		}
+		ret = append(ret, element)
+	}
+}
+
+func (c *compactor) UnmarshalBinary(data []byte) error {
+	var g compactorAsGob
+	var err error
+	gobDec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err = gobDec.Decode(&g); err != nil {
+		return err
+	}
+	if len(g.EncodedTypeAsJSON) == 0 {
+		return nil
+	}
+	var t beam.EncodedType
+	if err = json.Unmarshal(g.EncodedTypeAsJSON, &t); err != nil {
+		return err
+	}
+	dec := beam.NewElementDecoder(t.T)
+	decodedSorted := make([][]beam.T, 0, len(g.Sorted))
+	for _, sorted := range g.Sorted {
+		decoded, err := decodeElements(dec, sorted)
+		if err != nil {
+			return err
+		}
+		decodedSorted = append(decodedSorted, decoded)
+	}
+	c.sorted = decodedSorted
+	if c.unsorted, err = decodeElements(dec, g.Unsorted); err != nil {
+		return err
+	}
+	return nil
+}
+
+// update inserts an element into the compactor.
+func (c *compactor) update(element beam.T) {
+	c.unsorted = append(c.unsorted, element)
+}
+
+// size returns the number of elements stored in this compactor.
+func (c *compactor) size() int {
+	size := 0
+	for _, s := range c.sorted {
+		size += len(s)
+	}
+	return len(c.unsorted) + size
+}
+
+type sorter struct {
+	less reflectx.Func2x1
+	data []beam.T
+}
+
+func (s sorter) Len() int           { return len(s.data) }
+func (s sorter) Less(i, j int) bool { return s.less.Call2x1(s.data[i], s.data[j]).(bool) }
+func (s sorter) Swap(i, j int)      { s.data[i], s.data[j] = s.data[j], s.data[i] }
+
+// sort sorts the compactor and returns all the elements in sorted order.
+func (c *compactor) sort(less reflectx.Func2x1) []beam.T {
+	sort.Sort(sorter{data: c.unsorted, less: less})
+	h := sortListHeap{data: c.sorted, less: less}
+	heap.Init(&h)
+	sorted := make([]beam.T, 0, c.size()-len(c.unsorted))
+	for h.Len() > 0 {
+		s := heap.Pop(&h).([]beam.T)
+		sorted = append(sorted, s[0])
+		if len(s) > 1 {
+			heap.Push(&h, s[1:])
+		}
+	}
+	c.sorted = [][]beam.T{mergeSorted(sorted, c.unsorted, func(a, b interface{}) bool { return less.Call2x1(a, b).(bool) })}
+	c.unsorted = nil
+	if len(c.sorted[0]) == 0 {
+		c.sorted = nil
+		return nil
+	}
+	return c.sorted[0]
+}
+
+// Compactors holds the state of the quantile approximation compactors.
+type compactors struct {
+	// References "K" from the paper which influences the amount of memory used.
+	K int
+	// When compacting, we want to alternate between taking elements at even vs odd indices.
+	// The paper suggests using a random variable but we'd prefer to stay deterministic.
+	// Especially when merging two compactors we want to keep track of how often we've selected odds vs evens.
+	NumberOfCompactions int
+
+	// Each compactor takes a sample of elements.
+	// The "height" (also known as the index in this slice) of the compactor determines the weight of its elements.
+	// The weight of a compactor of height h is 2^h.
+	// For example, for h = 3 (which would be compactors[3]), the weight is 2^3 = 8. That means each element in that compactor represents 8 instances of itself.
+	Compactors []compactor
+}
+
+func (c *compactors) totalCapacity() int {
+	totalCapacity := 0
+	for _, compactor := range c.Compactors {
+		totalCapacity += compactor.capacity
+	}
+	return totalCapacity
+}
+
+func (c *compactors) size() int {
+	size := 0
+	for _, compactor := range c.Compactors {
+		size += compactor.size()
+	}
+	return size
+}
+
+// capacity computes the capacity of a compactor at a certain level.
+// The paper suggests decreasing the capacity of lower-leveled compactors as we add more elements.
+func (c *compactors) capacity(compactorLevel int) int {
+	return int(math.Ceil(math.Pow(capacityCoefficient, float64(len(c.Compactors)-compactorLevel-1))*float64(c.K))) + 1
+}
+
+// compact compacts all compactors until the total size is less than the maximum capacity of all compactors.
+func (c *compactors) compact(less reflectx.Func2x1) {
+	for c.size() > c.totalCapacity() {
+		for level, compactor := range c.Compactors {
+			if compactor.size() > compactor.capacity {
+				c.compactLevel(level, less)
+				// Merging compactors can cause us to exceed max capacity in multiple compactors.
+				if c.size() < c.totalCapacity() {
+					// Do lazy compaction as described in the paper.
+					break
+				}
+			}
+		}
+	}
+}
+
+// update inserts the given element into the compactors. If this element causes the compactors to grow too large, we perform the compaction here.
+func (c *compactors) update(element beam.T, weight int, less reflectx.Func2x1) {
+	level := int(math.Log2(float64(weight)))
+	c.growToIncludeLevel(level)
+	c.Compactors[level].update(element)
+	// From the paper, we're using the "Splitting the Input" approach.
+	remainingWeight := weight - (1 << uint(level))
+	// Only attempt compaction if we're doing the last update. Otherwise we'd be compacting too often.
+	if remainingWeight <= 0 {
+		c.compact(less)
+	} else {
+		c.update(element, remainingWeight, less)
+	}
+}
+
+// growToIncludeLevel ensures we have compactors available at the given level.
+func (c *compactors) growToIncludeLevel(level int) {
+	if len(c.Compactors)-1 >= level {
+		return
+	}
+	for i := len(c.Compactors) - 1; i < level; i++ {
+		c.Compactors = append(c.Compactors, compactor{})
+	}
+	for level := range c.Compactors {
+		c.Compactors[level].capacity = c.capacity(level)
+	}
+}
+
+// compact compacts elements in compactors.
+func (c *compactors) compactLevel(level int, less reflectx.Func2x1) {
+	c.growToIncludeLevel(level + 1)
+	jitterIndex := 0
+	// Create a temporary buffer to hold the compacted elements.
+	// Buffering the elements like this makes it easier to call mergeSorted.
+	compacted := make([]beam.T, 0, c.Compactors[level].size()/2)
+	selectEvens := c.NumberOfCompactions%2 == 0
+	c.NumberOfCompactions++
+	for _, element := range c.Compactors[level].sort(less) {
+		if (jitterIndex%2 == 0) == selectEvens {
+			compacted = append(compacted, element)
+		}
+		jitterIndex++
+	}
+	if len(compacted) > 0 {
+		c.Compactors[level+1].sorted = append(c.Compactors[level+1].sorted, compacted)
+	}
+	// Clear out the compactor at this level since we've finished compacting it. The compacted elements have already been moved to the next compactor.
+	c.Compactors[level].sorted = nil
+	c.Compactors[level].unsorted = nil
+}
+
+func encodeCompactors(c *compactors) ([]byte, error) {
+	var buf bytes.Buffer
+	enc := gob.NewEncoder(&buf)
+	if err := enc.Encode(c); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func decodeCompactors(data []byte) (*compactors, error) {
+	var compactors compactors
+	dec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err := dec.Decode(&compactors); err != nil {
+		return nil, err
+	}
+	for level := range compactors.Compactors {
+		compactors.Compactors[level].capacity = compactors.capacity(level)
+	}
+	return &compactors, nil
+}
+
+// mergeSorted takes two slices which are already sorted and returns a new slice containing all elements sorted together.
+func mergeSorted(a, b []beam.T, less func(interface{}, interface{}) bool) []beam.T {
+	output := make([]beam.T, 0, len(a)+len(b))
+	for len(a) > 0 && len(b) > 0 {
+		if less(a[0], b[0]) {
+			output = append(output, a[0])
+			a = a[1:]
+		} else {
+			output = append(output, b[0])
+			b = b[1:]
+		}
+	}
+	if len(a) > 0 {
+		output = append(output, a...)
+	} else {
+		output = append(output, b...)
+	}
+	return output
+}
+
+// mergeSortedWeighted takes two slices which are already sorted and returns a new slice containing all elements sorted together.
+func mergeSortedWeighted(a, b []weightedElement, less func(interface{}, interface{}) bool) []weightedElement {
+	output := make([]weightedElement, 0, len(a)+len(b))
+	for len(a) > 0 && len(b) > 0 {
+		if less(a[0], b[0]) {
+			output = append(output, a[0])
+			a = a[1:]
+		} else {
+			output = append(output, b[0])
+			b = b[1:]
+		}
+	}
+	if len(a) > 0 {
+		output = append(output, a...)
+	} else {
+		output = append(output, b...)
+	}
+	return output
+}
+
+// merge joins two compactors together.
+func (c *compactors) merge(other *compactors, less reflectx.Func2x1) {
+	for level := range c.Compactors {
+		if len(other.Compactors)-1 < level {
+			break
+		}
+		c.Compactors[level].unsorted = append(c.Compactors[level].unsorted, other.Compactors[level].unsorted...)
+		c.Compactors[level].sorted = append(c.Compactors[level].sorted, other.Compactors[level].sorted...)
+	}
+	if len(other.Compactors) > len(c.Compactors) {
+		c.Compactors = append(c.Compactors, other.Compactors[len(c.Compactors):]...)
+	}
+	c.NumberOfCompactions += other.NumberOfCompactions
+	c.compact(less)
+}
+
+// approximateQuantilesCombineFnState contains the payload for the combiners.
+// Ideally this would be a single combine function but in certain cases, runners attempts to do all the merges on a single machine.
+// Unfortunately the merges can be slow for extremely large datasets and large values of K. If the merge takes too long, it will get canceled and the job will never complete.
+// Thus we split up the combiners into multiple functions to force the runner to do the work in parallel.
+// This state can be shared across all of the split-up functions.
+type approximateQuantilesCombineFnState struct {
+	// The size of the compactors.
+	// The memory consumed, and the error are controlled by this parameter.
+	K int `json:"k"`
+	// Used to compare elements.
+	LessFunc beam.EncodedFunc
+	// Internally cached instance.
+	less         reflectx.Func2x1
+	NumQuantiles int `json:"numQuantiles"`
+}
+
+func (f *approximateQuantilesCombineFnState) setup() error {
+	f.less = reflectx.ToFunc2x1(f.LessFunc.Fn)
+	return nil
+}
+
+func (f *approximateQuantilesCombineFnState) createAccumulator() *compactors {
+	return &compactors{
+		K:          f.K,
+		Compactors: []compactor{{capacity: f.K}},
+	}
+}
+
+// approximateQuantilesOutputFn extracts the final output containing the quantiles.
+type approximateQuantilesOutputFn struct {
+	State approximateQuantilesCombineFnState `json:"state"`
+}
+
+func (f *approximateQuantilesOutputFn) Setup() error {
+	return f.State.setup()
+}
+
+func (f *approximateQuantilesOutputFn) CreateAccumulator() *compactors {
+	return f.State.createAccumulator()
+}
+
+func (f *approximateQuantilesOutputFn) AddInput(compactors *compactors, element *compactors) *compactors {
+	compactors.merge(element, f.State.less)
+	return compactors
+}
+
+func (f *approximateQuantilesOutputFn) MergeAccumulators(ctx context.Context, a, b *compactors) *compactors {
+	a.merge(b, f.State.less)
+	return a
+}
+
+type weightedElementAsGob struct {
+	EncodedTypeAsJSON []byte
+	Weight            int
+	Element           []byte
+}
+
+func encodeWeightedElement(element weightedElement) ([]byte, error) {
+	t := reflect.TypeOf(element.element)
+	enc := beam.NewElementEncoder(t)
+	var buf bytes.Buffer
+	if err := enc.Encode(element.element, &buf); err != nil {
+		return nil, err
+	}
+	tAsJSON, err := beam.EncodedType{T: t}.MarshalJSON()
+	if err != nil {
+		return nil, err
+	}
+	var gobBuf bytes.Buffer
+	if err := gob.NewEncoder(&gobBuf).Encode(weightedElementAsGob{
+		EncodedTypeAsJSON: tAsJSON,
+		Weight:            element.weight,
+		Element:           buf.Bytes(),
+	}); err != nil {
+		return nil, err
+	}
+	return gobBuf.Bytes(), nil
+}
+
+func decodeWeightedElement(data []byte) (weightedElement, error) {
+	var g weightedElementAsGob
+	dec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err := dec.Decode(&g); err != nil {
+		return weightedElement{}, err
+	}
+	var t beam.EncodedType
+	if err := t.UnmarshalJSON(g.EncodedTypeAsJSON); err != nil {
+		return weightedElement{}, err
+	}
+	element, err := beam.NewElementDecoder(t.T).Decode(bytes.NewBuffer(g.Element))
+	if err != nil {
+		return weightedElement{}, err
+	}
+	return weightedElement{
+		weight:  g.Weight,
+		element: element,
+	}, nil
+}
+
+type weightedElement struct {
+	weight  int
+	element beam.T
+}
+
+func toWeightedSlice(compactor compactor, less reflectx.Func2x1, weight int) []weightedElement {
+	sorted := compactor.sort(less)
+	weightedElements := make([]weightedElement, 0, len(sorted))
+	for _, element := range sorted {
+		weightedElements = append(weightedElements, weightedElement{weight: weight, element: element})
+	}
+	return weightedElements
+}
+func (f *approximateQuantilesOutputFn) ExtractOutput(ctx context.Context, compactors *compactors) []beam.T {
+	sorted := toWeightedSlice(compactors.Compactors[0], f.State.less, 1)
+	for level, compactor := range compactors.Compactors[1:] {
+		sorted = mergeSortedWeighted(sorted, toWeightedSlice(compactor, f.State.less, 1<<uint(level)), func(a, b interface{}) bool {
+			return f.State.less.Call2x1(a.(weightedElement).element, b.(weightedElement).element).(bool)
+		})
+	}
+	totalElements := 0
+	for _, element := range sorted {
+		totalElements += element.weight
+	}
+	ret := make([]beam.T, 0, f.State.NumQuantiles)
+	// Rank represents the estimate of how many elements we've seen as we iterate through the sorted list of elements stored in the compactors.
+	// Recall that each element stored in a compactor is also assigned a weight indicating how many elements from the input it represents.
+	rank := float64(0)
+	// Represents the quantile we're currently searching for.
+	currentQuantile := float64(1)
+	for _, element := range sorted {
+		rank += float64(element.weight)
+		if rank/float64(totalElements) >= currentQuantile/float64(f.State.NumQuantiles) {
+			ret = append(ret, element.element)
+			currentQuantile++
+		}
+		if currentQuantile >= float64(f.State.NumQuantiles) {
+			break
+		}
+	}
+	return ret
+}
+
+// approximateQuantilesInputFn combines elements into compactors, but not necessarily the final compactor.
+type approximateQuantilesInputFn approximateQuantilesOutputFn
+
+func (f *approximateQuantilesInputFn) Setup() error {
+	return f.State.setup()
+}
+
+func (f *approximateQuantilesInputFn) CreateAccumulator() *compactors {
+	return f.State.createAccumulator()
+}
+
+func (f *approximateQuantilesInputFn) AddInput(compactors *compactors, element weightedElement) *compactors {
+	compactors.update(element.element, element.weight, f.State.less)
+	return compactors
+}
+
+func (f *approximateQuantilesInputFn) MergeAccumulators(ctx context.Context, a, b *compactors) *compactors {
+	a.merge(b, f.State.less)
+	return a
+}
+
+func (f *approximateQuantilesInputFn) ExtractOutput(ctx context.Context, compactors *compactors) *compactors {
+	for i := range compactors.Compactors {
+		// Sort the compactors here so when we're merging them for the final output, they're already sorted and we can merge elements in order.
+		compactors.Compactors[i].sort(f.State.less)
+	}
+	return compactors
+}
+
+// approximateQuantilesMergeOnlyFn combines compactors into smaller compactors, but not necessarily the final compactor.
+type approximateQuantilesMergeOnlyFn approximateQuantilesOutputFn
+
+func (f *approximateQuantilesMergeOnlyFn) Setup() error {
+	return f.State.setup()
+}
+
+func (f *approximateQuantilesMergeOnlyFn) CreateAccumulator() *compactors {
+	return f.State.createAccumulator()
+}
+
+func (f *approximateQuantilesMergeOnlyFn) AddInput(compactors *compactors, element *compactors) *compactors {
+	compactors.merge(element, f.State.less)
+	return compactors
+}
+
+func (f *approximateQuantilesMergeOnlyFn) MergeAccumulators(ctx context.Context, a, b *compactors) *compactors {
+	a.merge(b, f.State.less)
+	return a
+}
+
+func (f *approximateQuantilesMergeOnlyFn) ExtractOutput(ctx context.Context, compactors *compactors) *compactors {
+	for i := range compactors.Compactors {
+		// Sort the compactors here so when we're merging them for the final output, they're already sorted and we can merge elements in order.
+		compactors.Compactors[i].sort(f.State.less)
+	}
+	return compactors
+}
+
+type shardElementsFn struct {
+	Shards         int              `json:"shards"`
+	T              beam.EncodedType `json:"t"`
+	elementEncoder beam.ElementEncoder
+}
+
+func (s *shardElementsFn) Setup() {
+	s.elementEncoder = beam.NewElementEncoder(s.T.T)
+}
+
+func (s *shardElementsFn) ProcessElement(element beam.T) (int, beam.T) {
+	h := crc32.NewIEEE()
+	s.elementEncoder.Encode(element, h)
+	return int(h.Sum32()) % s.Shards, element
+}
+
+func makeWeightedElement(weight int, element beam.T) weightedElement {
+	return weightedElement{weight: weight, element: element}
+}
+
+// ApproximateQuantiles computes approximate quantiles for the input PCollection<T>.
+//
+// The output PCollection contains a single element: a list of numQuantiles - 1 elements approximately splitting up the input collection into numQuantiles separate quantiles.
+// For example, if numQuantiles = 2, the returned list would contain a single element such that approximately half of the input would be less than that element and half would be greater.
+func ApproximateQuantiles(s beam.Scope, pc beam.PCollection, less interface{}, opts Opts) beam.PCollection {
+	return ApproximateWeightedQuantiles(s, beam.ParDo(s, func(e beam.T) (int, beam.T) { return 1, e }, pc), less, opts)
+}
+
+// reduce takes a PCollection<weightedElementWrapper> and returns a PCollection<*compactors>. The output PCollection may have at most shardSizes[len(shardSizes) - 1] compactors.
+func reduce(s beam.Scope, weightedElements beam.PCollection, state approximateQuantilesCombineFnState, shardSizes []int) beam.PCollection {
+	if len(shardSizes) == 0 {
+		shardSizes = []int{1}
+	}
+	shardedCompactors := beam.DropKey(s,

Review comment:
       I strongly recommend unrolling this. Nesting calls to the beam PTransform functions make the code much harder to read, vs having the variables spelled out. Nesting means that order of execution is much harder to follow, as it ends up reversed.
   
   Remember, code only needs to be written a small number of times, compared to how often it'll be read.

##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles_test.go
##########
@@ -0,0 +1,291 @@
+// 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 stats
+
+import (
+	"reflect"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/passert"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/ptest"
+)
+
+func init() {
+	beam.RegisterFunction(weightedElementToKv)
+
+	// In practice, this runs faster than plain reflection.
+	reflectx.RegisterFunc(reflect.ValueOf(less).Type(), func(_ interface{}) reflectx.Func {
+		return newIntLess()
+	})
+}
+
+type intLess struct {
+	name string
+	t    reflect.Type
+}
+
+func newIntLess() *intLess {
+	return &intLess{
+		name: reflectx.FunctionName(reflect.ValueOf(less).Interface()),
+		t:    reflect.ValueOf(less).Type(),
+	}
+}
+
+func (i *intLess) Name() string {
+	return i.name
+}
+func (i *intLess) Type() reflect.Type {
+	return i.t
+}
+func (i *intLess) Call(args []interface{}) []interface{} {
+	return []interface{}{args[0].(int) < args[1].(int)}
+}
+
+func less(a, b int) bool {
+	return a < b
+}
+
+func TestLargeQuantiles(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := 0; i < numElements; i++ {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{10006, 19973}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestLargeQuantilesReversed(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := numElements - 1; i >= 0; i-- {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{9985, 19959}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestBasicQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	tests := []struct {
+		Input    []int
+		Expected [][]int
+	}{
+		{[]int{}, [][]int{}},
+		{[]int{1}, [][]int{[]int{1}}},
+		{[]int{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20}, [][]int{[]int{6, 13}}},
+	}
+
+	for _, test := range tests {
+		p, s, in, exp := ptest.CreateList2(test.Input, test.Expected)
+		quantiles := ApproximateQuantiles(s, in, less, Opts{
+			K:            3,
+			NumQuantiles: 3,
+		})
+		passert.Equals(s, quantiles, exp)
+
+		if err := ptest.Run(p); err != nil {
+			t.Errorf("ApproximateQuantiles(%v) != %v: %v", test.Input, test.Expected, err)
+		}
+	}
+}
+
+func weightedElementToKv(e testWeightedElement) (int, int) {
+	return e.Weight, e.Element
+}
+
+type testWeightedElement struct {
+	Weight  int
+	Element int
+}
+
+func TestWeightedQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	input := []testWeightedElement{
+		{Weight: 1, Element: 1},
+		{Weight: 10, Element: 2},
+		{Weight: 1, Element: 3},
+		{Weight: 10, Element: 4}}
+	expected := []int{2, 4}
+	p, s, in, exp := ptest.CreateList2(input, [][]int{expected})
+	quantiles := ApproximateWeightedQuantiles(
+		s,
+		beam.ParDo(
+			s,
+			weightedElementToKv,
+			in,
+			beam.TypeDefinition{Var: beam.TType, T: reflect.TypeOf(1)}),
+		less,
+		Opts{
+			K:            3,
+			NumQuantiles: 3,
+		})
+	passert.Equals(
+		s,
+		quantiles,
+		exp)
+
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles(%v) != %v: %v", input, expected, err)
+	}
+}
+
+func TestWeightedQuantilesWithInternalSharding(t *testing.T) {
+	// Test shard reduction.
+	input := []testWeightedElement{
+		{Weight: 1, Element: 1},
+		{Weight: 10, Element: 2},
+		{Weight: 1, Element: 3},
+		{Weight: 10, Element: 4}}
+	expected := []int{2, 4}
+	p, s, in, exp := ptest.CreateList2(input, [][]int{expected})
+	quantiles := ApproximateWeightedQuantiles(
+		s,
+		beam.ParDo(
+			s,
+			weightedElementToKv,
+			in,
+			beam.TypeDefinition{Var: beam.TType, T: reflect.TypeOf(1)}),
+		less,
+		Opts{
+			K:                3,
+			NumQuantiles:     3,
+			InternalSharding: []int{4, 3, 2},
+		})
+	passert.Equals(
+		s,
+		quantiles,
+		exp)
+
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles(%v) != %v: %v", input, expected, err)
+	}
+}
+
+func TestMerging(t *testing.T) {
+	compactors1 := compactors{
+		K:                   3,
+		NumberOfCompactions: 1,
+		Compactors: []compactor{{
+			sorted:   [][]beam.T{[]beam.T{1}, []beam.T{2}, []beam.T{3}},
+			unsorted: []beam.T{6, 5, 4},
+			capacity: 4,
+		}},
+	}
+
+	compactors2 := compactors{
+		K:                   3,
+		NumberOfCompactions: 1,
+		Compactors: []compactor{
+			{
+				sorted:   [][]beam.T{[]beam.T{7}, []beam.T{8}, []beam.T{9}},
+				unsorted: []beam.T{12, 11, 10},
+				capacity: 4},
+		},
+	}
+
+	compactors1.merge(&compactors2, reflectx.MakeFunc2x1(less))
+
+	expectedCompactors := compactors{
+		K:                   3,
+		NumberOfCompactions: 3,
+		Compactors: []compactor{
+			{capacity: 4},
+			{
+				sorted:   [][]beam.T{[]beam.T{1, 3, 5, 7, 9, 11}},
+				capacity: 4,
+			},
+		},
+	}
+	if !reflect.DeepEqual(expectedCompactors, compactors1) {

Review comment:
       Instead of DeepEqual, consider using `cmp.Diff` for more readable test outputs when trying to spot the differences. Adding new packages to the Go SDK is a pain right now, until we get to Go Modules, but we already depend on it so it should work out. Here and elsewhere DeepEqual is being used.

##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles.go
##########
@@ -0,0 +1,702 @@
+// 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 stats
+
+// Approximate quantiles is implemented based on https://arxiv.org/pdf/1907.00236.pdf.
+
+import (
+	"bytes"
+	"container/heap"
+	"context"
+	"encoding/gob"
+	"encoding/json"
+	"hash/crc32"
+	"io"
+	"math"
+	"reflect"
+	"sort"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+)
+
+func init() {
+	compactorsType := reflect.TypeOf((**compactors)(nil)).Elem()
+	weightedElementType := reflect.TypeOf((*weightedElement)(nil)).Elem()
+	beam.RegisterType(compactorsType)
+	beam.RegisterType(weightedElementType)
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesInputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesMergeOnlyFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesOutputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*shardElementsFn)(nil)).Elem())
+	beam.RegisterCoder(compactorsType, encodeCompactors, decodeCompactors)
+	beam.RegisterCoder(weightedElementType, encodeWeightedElement, decodeWeightedElement)
+}
+
+// Opts contains settings used to configure how approximate quantiles are computed.
+type Opts struct {
+	// Controls the memory used and approximation error (difference between the quantile returned and the true quantile.)
+	K int
+	// Number of quantiles to return. The algorithm will return NumQuantiles - 1 numbers
+	NumQuantiles int
+	// For extremely large datasets, the Go SDK has some problems holding on to data and might OOM.
+	// If ApproximateQuantiles is running out of memory, you can use this option to tune how the data is sharded internally.
+	// This parameter is optional. If unspecified, Beam will compact all elements into a single compactor at once.
+	InternalSharding []int
+}
+
+// The paper suggests reducing the size of the lower-level compactors as we grow.
+// We reduce the capacity at this rate.
+// The paper suggests 1/sqrt(2) is ideal. That's approximately 0.7.
+const capacityCoefficient float64 = 0.7
+
+type sortListHeap struct {
+	data [][]beam.T
+	less reflectx.Func2x1
+}
+
+func (s sortListHeap) Len() int            { return len(s.data) }
+func (s sortListHeap) Less(i, j int) bool  { return s.less.Call2x1(s.data[i][0], s.data[j][0]).(bool) }
+func (s sortListHeap) Swap(i, j int)       { s.data[i], s.data[j] = s.data[j], s.data[i] }
+func (s *sortListHeap) Push(x interface{}) { s.data = append(s.data, x.([]beam.T)) }
+func (s *sortListHeap) Pop() interface{} {
+	var x beam.T
+	x, s.data = s.data[len(s.data)-1], s.data[:len(s.data)-1]
+	return x
+}
+
+// compactor contains elements to be compacted.
+type compactor struct {
+	// Compaction needs to sort elements before compacting. Thus in practice, we should often have some pre-sorted data.
+	// We want to keep it separate so we can sort only the unsorted data and merge the two sorted lists together.
+	// If we're only receiving elements of weight 1, only level 0 will ever contain unsorted data and the rest of the levels will always remain sorted.
+	// To prevent repeated allocation/copying, we keep multiple sorted lists and then merge them together
+	sorted   [][]beam.T
+	unsorted []beam.T
+	// How many items should be stored in this compactor before it should get compacted.
+	// Note that this is not a hard limit.
+	// The paper suggests implementing lazy compaction which would allow
+	// compactors to temporarily exceed their capacity as long as the total
+	// elements in all compactors doesn't exceed the total capacity in all
+	// compactors. In other words, compactors can temporarily borrow capacity
+	// from each other.
+	// In the paper, this is referred to as the variable k_h.
+	capacity int
+}
+
+type compactorAsGob struct {
+	Sorted            [][]byte
+	Unsorted          []byte
+	EncodedTypeAsJSON []byte
+}
+
+func encodeElements(enc beam.ElementEncoder, elements []beam.T) ([]byte, error) {
+	var buf bytes.Buffer
+	for _, e := range elements {
+		if err := enc.Encode(e, &buf); err != nil {
+			return nil, err
+		}
+	}
+	return buf.Bytes(), nil
+}
+
+func (c *compactor) getElementType() reflect.Type {
+	for _, e := range c.sorted {
+		for _, e2 := range e {
+			return reflect.TypeOf(e2)
+		}
+	}
+	for _, e := range c.unsorted {
+		return reflect.TypeOf(e)
+	}
+	return nil
+}
+
+func (c *compactor) MarshalBinary() ([]byte, error) {
+	t := c.getElementType()
+	var buf bytes.Buffer
+	if t == nil {
+		enc := gob.NewEncoder(&buf)
+		if err := enc.Encode(compactorAsGob{}); err != nil {
+			return nil, err
+		}
+		return buf.Bytes(), nil
+	}
+	enc := beam.NewElementEncoder(t)
+	encodedSorted := make([][]byte, 0, len(c.sorted))
+	for _, sorted := range c.sorted {
+		encoded, err := encodeElements(enc, sorted)
+		if err != nil {
+			return nil, err
+		}
+		encodedSorted = append(encodedSorted, encoded)
+	}
+	encodedUnsorted, err := encodeElements(enc, c.unsorted)
+	if err != nil {
+		return nil, err
+	}
+	tAsJSON, err := beam.EncodedType{T: t}.MarshalJSON()
+	if err != nil {
+		return nil, err
+	}
+	gobEnc := gob.NewEncoder(&buf)
+	if err = gobEnc.Encode(compactorAsGob{
+		Sorted:            encodedSorted,
+		Unsorted:          encodedUnsorted,
+		EncodedTypeAsJSON: tAsJSON,
+	}); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func decodeElements(dec beam.ElementDecoder, data []byte) ([]beam.T, error) {
+	buf := bytes.NewBuffer(data)
+	ret := []beam.T{}

Review comment:
       If there are more than a single digit worth of elements, consider adding a varint encoded prefix for how many elements are ended in encodeElements, this would allow you to pre-allocate the right number of elements ahead of time. graphx/coder has EncodeVarInt and DecodeVarInt functions.
   
   However, given we don't know the static type, we don't get the benefit of avoiding the per-element heap allocations, so it's probably moot in this case, until generics arrive. The Length Prefix would avoid repeated allocating and copying of the contents of the slice as it expands, even though the values are interfaces, which are about 24 bytes each IIRC.

##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles_test.go
##########
@@ -0,0 +1,291 @@
+// 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 stats
+
+import (
+	"reflect"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/passert"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/ptest"
+)
+
+func init() {
+	beam.RegisterFunction(weightedElementToKv)
+
+	// In practice, this runs faster than plain reflection.
+	reflectx.RegisterFunc(reflect.ValueOf(less).Type(), func(_ interface{}) reflectx.Func {
+		return newIntLess()
+	})
+}
+
+type intLess struct {
+	name string
+	t    reflect.Type
+}
+
+func newIntLess() *intLess {
+	return &intLess{
+		name: reflectx.FunctionName(reflect.ValueOf(less).Interface()),
+		t:    reflect.ValueOf(less).Type(),
+	}
+}
+
+func (i *intLess) Name() string {
+	return i.name
+}
+func (i *intLess) Type() reflect.Type {
+	return i.t
+}
+func (i *intLess) Call(args []interface{}) []interface{} {
+	return []interface{}{args[0].(int) < args[1].(int)}
+}
+
+func less(a, b int) bool {
+	return a < b
+}
+
+func TestLargeQuantiles(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := 0; i < numElements; i++ {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{10006, 19973}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestLargeQuantilesReversed(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := numElements - 1; i >= 0; i-- {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{9985, 19959}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestBasicQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	tests := []struct {
+		Input    []int
+		Expected [][]int
+	}{
+		{[]int{}, [][]int{}},
+		{[]int{1}, [][]int{[]int{1}}},
+		{[]int{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20}, [][]int{[]int{6, 13}}},
+	}
+
+	for _, test := range tests {
+		p, s, in, exp := ptest.CreateList2(test.Input, test.Expected)
+		quantiles := ApproximateQuantiles(s, in, less, Opts{
+			K:            3,
+			NumQuantiles: 3,
+		})
+		passert.Equals(s, quantiles, exp)
+
+		if err := ptest.Run(p); err != nil {
+			t.Errorf("ApproximateQuantiles(%v) != %v: %v", test.Input, test.Expected, err)
+		}
+	}
+}
+
+func weightedElementToKv(e testWeightedElement) (int, int) {
+	return e.Weight, e.Element
+}
+
+type testWeightedElement struct {
+	Weight  int
+	Element int
+}
+
+func TestWeightedQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	input := []testWeightedElement{
+		{Weight: 1, Element: 1},
+		{Weight: 10, Element: 2},
+		{Weight: 1, Element: 3},
+		{Weight: 10, Element: 4}}
+	expected := []int{2, 4}
+	p, s, in, exp := ptest.CreateList2(input, [][]int{expected})
+	quantiles := ApproximateWeightedQuantiles(
+		s,
+		beam.ParDo(
+			s,
+			weightedElementToKv,
+			in,
+			beam.TypeDefinition{Var: beam.TType, T: reflect.TypeOf(1)}),
+		less,
+		Opts{
+			K:            3,
+			NumQuantiles: 3,
+		})
+	passert.Equals(
+		s,
+		quantiles,
+		exp)
+
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles(%v) != %v: %v", input, expected, err)
+	}
+}
+
+func TestWeightedQuantilesWithInternalSharding(t *testing.T) {
+	// Test shard reduction.
+	input := []testWeightedElement{
+		{Weight: 1, Element: 1},
+		{Weight: 10, Element: 2},
+		{Weight: 1, Element: 3},
+		{Weight: 10, Element: 4}}
+	expected := []int{2, 4}
+	p, s, in, exp := ptest.CreateList2(input, [][]int{expected})
+	quantiles := ApproximateWeightedQuantiles(
+		s,
+		beam.ParDo(
+			s,
+			weightedElementToKv,
+			in,
+			beam.TypeDefinition{Var: beam.TType, T: reflect.TypeOf(1)}),
+		less,
+		Opts{
+			K:                3,
+			NumQuantiles:     3,
+			InternalSharding: []int{4, 3, 2},
+		})

Review comment:
       Unnest please.

##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles.go
##########
@@ -0,0 +1,702 @@
+// 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 stats
+
+// Approximate quantiles is implemented based on https://arxiv.org/pdf/1907.00236.pdf.
+
+import (
+	"bytes"
+	"container/heap"
+	"context"
+	"encoding/gob"
+	"encoding/json"
+	"hash/crc32"
+	"io"
+	"math"
+	"reflect"
+	"sort"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+)
+
+func init() {
+	compactorsType := reflect.TypeOf((**compactors)(nil)).Elem()
+	weightedElementType := reflect.TypeOf((*weightedElement)(nil)).Elem()
+	beam.RegisterType(compactorsType)
+	beam.RegisterType(weightedElementType)
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesInputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesMergeOnlyFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesOutputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*shardElementsFn)(nil)).Elem())
+	beam.RegisterCoder(compactorsType, encodeCompactors, decodeCompactors)
+	beam.RegisterCoder(weightedElementType, encodeWeightedElement, decodeWeightedElement)
+}
+
+// Opts contains settings used to configure how approximate quantiles are computed.
+type Opts struct {
+	// Controls the memory used and approximation error (difference between the quantile returned and the true quantile.)
+	K int
+	// Number of quantiles to return. The algorithm will return NumQuantiles - 1 numbers
+	NumQuantiles int
+	// For extremely large datasets, the Go SDK has some problems holding on to data and might OOM.
+	// If ApproximateQuantiles is running out of memory, you can use this option to tune how the data is sharded internally.
+	// This parameter is optional. If unspecified, Beam will compact all elements into a single compactor at once.
+	InternalSharding []int
+}
+
+// The paper suggests reducing the size of the lower-level compactors as we grow.
+// We reduce the capacity at this rate.
+// The paper suggests 1/sqrt(2) is ideal. That's approximately 0.7.
+const capacityCoefficient float64 = 0.7
+
+type sortListHeap struct {
+	data [][]beam.T
+	less reflectx.Func2x1
+}
+
+func (s sortListHeap) Len() int            { return len(s.data) }
+func (s sortListHeap) Less(i, j int) bool  { return s.less.Call2x1(s.data[i][0], s.data[j][0]).(bool) }
+func (s sortListHeap) Swap(i, j int)       { s.data[i], s.data[j] = s.data[j], s.data[i] }
+func (s *sortListHeap) Push(x interface{}) { s.data = append(s.data, x.([]beam.T)) }
+func (s *sortListHeap) Pop() interface{} {
+	var x beam.T
+	x, s.data = s.data[len(s.data)-1], s.data[:len(s.data)-1]
+	return x
+}
+
+// compactor contains elements to be compacted.
+type compactor struct {
+	// Compaction needs to sort elements before compacting. Thus in practice, we should often have some pre-sorted data.
+	// We want to keep it separate so we can sort only the unsorted data and merge the two sorted lists together.
+	// If we're only receiving elements of weight 1, only level 0 will ever contain unsorted data and the rest of the levels will always remain sorted.
+	// To prevent repeated allocation/copying, we keep multiple sorted lists and then merge them together
+	sorted   [][]beam.T
+	unsorted []beam.T
+	// How many items should be stored in this compactor before it should get compacted.
+	// Note that this is not a hard limit.
+	// The paper suggests implementing lazy compaction which would allow
+	// compactors to temporarily exceed their capacity as long as the total
+	// elements in all compactors doesn't exceed the total capacity in all
+	// compactors. In other words, compactors can temporarily borrow capacity
+	// from each other.
+	// In the paper, this is referred to as the variable k_h.
+	capacity int
+}
+
+type compactorAsGob struct {
+	Sorted            [][]byte
+	Unsorted          []byte
+	EncodedTypeAsJSON []byte
+}
+
+func encodeElements(enc beam.ElementEncoder, elements []beam.T) ([]byte, error) {
+	var buf bytes.Buffer
+	for _, e := range elements {
+		if err := enc.Encode(e, &buf); err != nil {
+			return nil, err
+		}
+	}
+	return buf.Bytes(), nil
+}
+
+func (c *compactor) getElementType() reflect.Type {
+	for _, e := range c.sorted {
+		for _, e2 := range e {
+			return reflect.TypeOf(e2)
+		}
+	}
+	for _, e := range c.unsorted {
+		return reflect.TypeOf(e)
+	}
+	return nil
+}
+
+func (c *compactor) MarshalBinary() ([]byte, error) {
+	t := c.getElementType()
+	var buf bytes.Buffer
+	if t == nil {
+		enc := gob.NewEncoder(&buf)
+		if err := enc.Encode(compactorAsGob{}); err != nil {
+			return nil, err
+		}
+		return buf.Bytes(), nil
+	}
+	enc := beam.NewElementEncoder(t)
+	encodedSorted := make([][]byte, 0, len(c.sorted))
+	for _, sorted := range c.sorted {
+		encoded, err := encodeElements(enc, sorted)
+		if err != nil {
+			return nil, err
+		}
+		encodedSorted = append(encodedSorted, encoded)
+	}
+	encodedUnsorted, err := encodeElements(enc, c.unsorted)
+	if err != nil {
+		return nil, err
+	}
+	tAsJSON, err := beam.EncodedType{T: t}.MarshalJSON()
+	if err != nil {
+		return nil, err
+	}
+	gobEnc := gob.NewEncoder(&buf)
+	if err = gobEnc.Encode(compactorAsGob{
+		Sorted:            encodedSorted,
+		Unsorted:          encodedUnsorted,
+		EncodedTypeAsJSON: tAsJSON,
+	}); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func decodeElements(dec beam.ElementDecoder, data []byte) ([]beam.T, error) {
+	buf := bytes.NewBuffer(data)
+	ret := []beam.T{}
+	for {
+		element, err := dec.Decode(buf)
+		if err == io.EOF {
+			return ret, nil
+		} else if err != nil {
+			return nil, err
+		}
+		ret = append(ret, element)
+	}
+}
+
+func (c *compactor) UnmarshalBinary(data []byte) error {
+	var g compactorAsGob
+	var err error
+	gobDec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err = gobDec.Decode(&g); err != nil {
+		return err
+	}
+	if len(g.EncodedTypeAsJSON) == 0 {
+		return nil
+	}
+	var t beam.EncodedType
+	if err = json.Unmarshal(g.EncodedTypeAsJSON, &t); err != nil {
+		return err
+	}
+	dec := beam.NewElementDecoder(t.T)
+	decodedSorted := make([][]beam.T, 0, len(g.Sorted))
+	for _, sorted := range g.Sorted {
+		decoded, err := decodeElements(dec, sorted)
+		if err != nil {
+			return err
+		}
+		decodedSorted = append(decodedSorted, decoded)
+	}
+	c.sorted = decodedSorted
+	if c.unsorted, err = decodeElements(dec, g.Unsorted); err != nil {
+		return err
+	}
+	return nil
+}
+
+// update inserts an element into the compactor.
+func (c *compactor) update(element beam.T) {
+	c.unsorted = append(c.unsorted, element)
+}
+
+// size returns the number of elements stored in this compactor.
+func (c *compactor) size() int {
+	size := 0
+	for _, s := range c.sorted {
+		size += len(s)
+	}
+	return len(c.unsorted) + size
+}
+
+type sorter struct {
+	less reflectx.Func2x1
+	data []beam.T
+}
+
+func (s sorter) Len() int           { return len(s.data) }
+func (s sorter) Less(i, j int) bool { return s.less.Call2x1(s.data[i], s.data[j]).(bool) }
+func (s sorter) Swap(i, j int)      { s.data[i], s.data[j] = s.data[j], s.data[i] }
+
+// sort sorts the compactor and returns all the elements in sorted order.
+func (c *compactor) sort(less reflectx.Func2x1) []beam.T {
+	sort.Sort(sorter{data: c.unsorted, less: less})
+	h := sortListHeap{data: c.sorted, less: less}
+	heap.Init(&h)
+	sorted := make([]beam.T, 0, c.size()-len(c.unsorted))
+	for h.Len() > 0 {
+		s := heap.Pop(&h).([]beam.T)
+		sorted = append(sorted, s[0])
+		if len(s) > 1 {
+			heap.Push(&h, s[1:])
+		}
+	}
+	c.sorted = [][]beam.T{mergeSorted(sorted, c.unsorted, func(a, b interface{}) bool { return less.Call2x1(a, b).(bool) })}
+	c.unsorted = nil
+	if len(c.sorted[0]) == 0 {
+		c.sorted = nil
+		return nil
+	}
+	return c.sorted[0]
+}
+
+// Compactors holds the state of the quantile approximation compactors.
+type compactors struct {
+	// References "K" from the paper which influences the amount of memory used.
+	K int
+	// When compacting, we want to alternate between taking elements at even vs odd indices.
+	// The paper suggests using a random variable but we'd prefer to stay deterministic.
+	// Especially when merging two compactors we want to keep track of how often we've selected odds vs evens.
+	NumberOfCompactions int
+
+	// Each compactor takes a sample of elements.
+	// The "height" (also known as the index in this slice) of the compactor determines the weight of its elements.
+	// The weight of a compactor of height h is 2^h.
+	// For example, for h = 3 (which would be compactors[3]), the weight is 2^3 = 8. That means each element in that compactor represents 8 instances of itself.
+	Compactors []compactor
+}
+
+func (c *compactors) totalCapacity() int {
+	totalCapacity := 0
+	for _, compactor := range c.Compactors {
+		totalCapacity += compactor.capacity
+	}
+	return totalCapacity
+}
+
+func (c *compactors) size() int {
+	size := 0
+	for _, compactor := range c.Compactors {
+		size += compactor.size()
+	}
+	return size
+}
+
+// capacity computes the capacity of a compactor at a certain level.
+// The paper suggests decreasing the capacity of lower-leveled compactors as we add more elements.
+func (c *compactors) capacity(compactorLevel int) int {
+	return int(math.Ceil(math.Pow(capacityCoefficient, float64(len(c.Compactors)-compactorLevel-1))*float64(c.K))) + 1
+}
+
+// compact compacts all compactors until the total size is less than the maximum capacity of all compactors.
+func (c *compactors) compact(less reflectx.Func2x1) {
+	for c.size() > c.totalCapacity() {
+		for level, compactor := range c.Compactors {
+			if compactor.size() > compactor.capacity {
+				c.compactLevel(level, less)
+				// Merging compactors can cause us to exceed max capacity in multiple compactors.
+				if c.size() < c.totalCapacity() {
+					// Do lazy compaction as described in the paper.
+					break
+				}
+			}
+		}
+	}
+}
+
+// update inserts the given element into the compactors. If this element causes the compactors to grow too large, we perform the compaction here.
+func (c *compactors) update(element beam.T, weight int, less reflectx.Func2x1) {
+	level := int(math.Log2(float64(weight)))
+	c.growToIncludeLevel(level)
+	c.Compactors[level].update(element)
+	// From the paper, we're using the "Splitting the Input" approach.
+	remainingWeight := weight - (1 << uint(level))
+	// Only attempt compaction if we're doing the last update. Otherwise we'd be compacting too often.
+	if remainingWeight <= 0 {
+		c.compact(less)
+	} else {
+		c.update(element, remainingWeight, less)
+	}
+}
+
+// growToIncludeLevel ensures we have compactors available at the given level.
+func (c *compactors) growToIncludeLevel(level int) {
+	if len(c.Compactors)-1 >= level {
+		return
+	}
+	for i := len(c.Compactors) - 1; i < level; i++ {
+		c.Compactors = append(c.Compactors, compactor{})
+	}
+	for level := range c.Compactors {
+		c.Compactors[level].capacity = c.capacity(level)
+	}
+}
+
+// compact compacts elements in compactors.
+func (c *compactors) compactLevel(level int, less reflectx.Func2x1) {
+	c.growToIncludeLevel(level + 1)
+	jitterIndex := 0
+	// Create a temporary buffer to hold the compacted elements.
+	// Buffering the elements like this makes it easier to call mergeSorted.
+	compacted := make([]beam.T, 0, c.Compactors[level].size()/2)
+	selectEvens := c.NumberOfCompactions%2 == 0
+	c.NumberOfCompactions++
+	for _, element := range c.Compactors[level].sort(less) {
+		if (jitterIndex%2 == 0) == selectEvens {
+			compacted = append(compacted, element)
+		}
+		jitterIndex++
+	}
+	if len(compacted) > 0 {
+		c.Compactors[level+1].sorted = append(c.Compactors[level+1].sorted, compacted)
+	}
+	// Clear out the compactor at this level since we've finished compacting it. The compacted elements have already been moved to the next compactor.
+	c.Compactors[level].sorted = nil
+	c.Compactors[level].unsorted = nil
+}
+
+func encodeCompactors(c *compactors) ([]byte, error) {
+	var buf bytes.Buffer
+	enc := gob.NewEncoder(&buf)
+	if err := enc.Encode(c); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func decodeCompactors(data []byte) (*compactors, error) {
+	var compactors compactors
+	dec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err := dec.Decode(&compactors); err != nil {
+		return nil, err
+	}
+	for level := range compactors.Compactors {
+		compactors.Compactors[level].capacity = compactors.capacity(level)
+	}
+	return &compactors, nil
+}
+
+// mergeSorted takes two slices which are already sorted and returns a new slice containing all elements sorted together.
+func mergeSorted(a, b []beam.T, less func(interface{}, interface{}) bool) []beam.T {
+	output := make([]beam.T, 0, len(a)+len(b))
+	for len(a) > 0 && len(b) > 0 {
+		if less(a[0], b[0]) {
+			output = append(output, a[0])
+			a = a[1:]
+		} else {
+			output = append(output, b[0])
+			b = b[1:]
+		}
+	}
+	if len(a) > 0 {
+		output = append(output, a...)
+	} else {
+		output = append(output, b...)
+	}
+	return output
+}
+
+// mergeSortedWeighted takes two slices which are already sorted and returns a new slice containing all elements sorted together.
+func mergeSortedWeighted(a, b []weightedElement, less func(interface{}, interface{}) bool) []weightedElement {
+	output := make([]weightedElement, 0, len(a)+len(b))
+	for len(a) > 0 && len(b) > 0 {
+		if less(a[0], b[0]) {
+			output = append(output, a[0])
+			a = a[1:]
+		} else {
+			output = append(output, b[0])
+			b = b[1:]
+		}
+	}
+	if len(a) > 0 {
+		output = append(output, a...)
+	} else {
+		output = append(output, b...)
+	}
+	return output
+}
+
+// merge joins two compactors together.
+func (c *compactors) merge(other *compactors, less reflectx.Func2x1) {
+	for level := range c.Compactors {
+		if len(other.Compactors)-1 < level {
+			break
+		}
+		c.Compactors[level].unsorted = append(c.Compactors[level].unsorted, other.Compactors[level].unsorted...)
+		c.Compactors[level].sorted = append(c.Compactors[level].sorted, other.Compactors[level].sorted...)
+	}
+	if len(other.Compactors) > len(c.Compactors) {
+		c.Compactors = append(c.Compactors, other.Compactors[len(c.Compactors):]...)
+	}
+	c.NumberOfCompactions += other.NumberOfCompactions
+	c.compact(less)
+}
+
+// approximateQuantilesCombineFnState contains the payload for the combiners.
+// Ideally this would be a single combine function but in certain cases, runners attempts to do all the merges on a single machine.

Review comment:
       Note: A global combine (rather than a CombinePerKey) will *always* merge everything on a single machine. By implementation, it's a grouping by a single key, and I'm not aware of any beam runners that automatically multi-level merge for GBKs.  One can always split this up manually ahead of time, if it's typically going to be a problem. 
   
   The main trick is having it work efficiently for  smaller datasets where simply having an extra GBK will dominate.
   
   ------
   Edit: Looks like you've taken care of that already. Nice!

##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles.go
##########
@@ -0,0 +1,702 @@
+// 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 stats
+
+// Approximate quantiles is implemented based on https://arxiv.org/pdf/1907.00236.pdf.
+
+import (
+	"bytes"
+	"container/heap"
+	"context"
+	"encoding/gob"
+	"encoding/json"
+	"hash/crc32"
+	"io"
+	"math"
+	"reflect"
+	"sort"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+)
+
+func init() {
+	compactorsType := reflect.TypeOf((**compactors)(nil)).Elem()
+	weightedElementType := reflect.TypeOf((*weightedElement)(nil)).Elem()
+	beam.RegisterType(compactorsType)
+	beam.RegisterType(weightedElementType)
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesInputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesMergeOnlyFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesOutputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*shardElementsFn)(nil)).Elem())
+	beam.RegisterCoder(compactorsType, encodeCompactors, decodeCompactors)
+	beam.RegisterCoder(weightedElementType, encodeWeightedElement, decodeWeightedElement)
+}
+
+// Opts contains settings used to configure how approximate quantiles are computed.
+type Opts struct {
+	// Controls the memory used and approximation error (difference between the quantile returned and the true quantile.)
+	K int
+	// Number of quantiles to return. The algorithm will return NumQuantiles - 1 numbers
+	NumQuantiles int
+	// For extremely large datasets, the Go SDK has some problems holding on to data and might OOM.
+	// If ApproximateQuantiles is running out of memory, you can use this option to tune how the data is sharded internally.
+	// This parameter is optional. If unspecified, Beam will compact all elements into a single compactor at once.
+	InternalSharding []int
+}
+
+// The paper suggests reducing the size of the lower-level compactors as we grow.
+// We reduce the capacity at this rate.
+// The paper suggests 1/sqrt(2) is ideal. That's approximately 0.7.
+const capacityCoefficient float64 = 0.7
+
+type sortListHeap struct {
+	data [][]beam.T
+	less reflectx.Func2x1
+}
+
+func (s sortListHeap) Len() int            { return len(s.data) }
+func (s sortListHeap) Less(i, j int) bool  { return s.less.Call2x1(s.data[i][0], s.data[j][0]).(bool) }
+func (s sortListHeap) Swap(i, j int)       { s.data[i], s.data[j] = s.data[j], s.data[i] }
+func (s *sortListHeap) Push(x interface{}) { s.data = append(s.data, x.([]beam.T)) }
+func (s *sortListHeap) Pop() interface{} {
+	var x beam.T
+	x, s.data = s.data[len(s.data)-1], s.data[:len(s.data)-1]
+	return x
+}
+
+// compactor contains elements to be compacted.
+type compactor struct {
+	// Compaction needs to sort elements before compacting. Thus in practice, we should often have some pre-sorted data.
+	// We want to keep it separate so we can sort only the unsorted data and merge the two sorted lists together.
+	// If we're only receiving elements of weight 1, only level 0 will ever contain unsorted data and the rest of the levels will always remain sorted.
+	// To prevent repeated allocation/copying, we keep multiple sorted lists and then merge them together
+	sorted   [][]beam.T
+	unsorted []beam.T
+	// How many items should be stored in this compactor before it should get compacted.
+	// Note that this is not a hard limit.
+	// The paper suggests implementing lazy compaction which would allow
+	// compactors to temporarily exceed their capacity as long as the total
+	// elements in all compactors doesn't exceed the total capacity in all
+	// compactors. In other words, compactors can temporarily borrow capacity
+	// from each other.
+	// In the paper, this is referred to as the variable k_h.
+	capacity int
+}
+
+type compactorAsGob struct {
+	Sorted            [][]byte
+	Unsorted          []byte
+	EncodedTypeAsJSON []byte
+}
+
+func encodeElements(enc beam.ElementEncoder, elements []beam.T) ([]byte, error) {
+	var buf bytes.Buffer
+	for _, e := range elements {
+		if err := enc.Encode(e, &buf); err != nil {
+			return nil, err
+		}
+	}
+	return buf.Bytes(), nil
+}
+
+func (c *compactor) getElementType() reflect.Type {
+	for _, e := range c.sorted {
+		for _, e2 := range e {
+			return reflect.TypeOf(e2)
+		}
+	}
+	for _, e := range c.unsorted {
+		return reflect.TypeOf(e)
+	}
+	return nil
+}
+
+func (c *compactor) MarshalBinary() ([]byte, error) {
+	t := c.getElementType()
+	var buf bytes.Buffer
+	if t == nil {
+		enc := gob.NewEncoder(&buf)
+		if err := enc.Encode(compactorAsGob{}); err != nil {
+			return nil, err
+		}
+		return buf.Bytes(), nil
+	}
+	enc := beam.NewElementEncoder(t)
+	encodedSorted := make([][]byte, 0, len(c.sorted))
+	for _, sorted := range c.sorted {
+		encoded, err := encodeElements(enc, sorted)
+		if err != nil {
+			return nil, err
+		}
+		encodedSorted = append(encodedSorted, encoded)
+	}
+	encodedUnsorted, err := encodeElements(enc, c.unsorted)
+	if err != nil {
+		return nil, err
+	}
+	tAsJSON, err := beam.EncodedType{T: t}.MarshalJSON()
+	if err != nil {
+		return nil, err
+	}
+	gobEnc := gob.NewEncoder(&buf)
+	if err = gobEnc.Encode(compactorAsGob{
+		Sorted:            encodedSorted,
+		Unsorted:          encodedUnsorted,
+		EncodedTypeAsJSON: tAsJSON,
+	}); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func decodeElements(dec beam.ElementDecoder, data []byte) ([]beam.T, error) {
+	buf := bytes.NewBuffer(data)
+	ret := []beam.T{}
+	for {
+		element, err := dec.Decode(buf)
+		if err == io.EOF {
+			return ret, nil
+		} else if err != nil {
+			return nil, err
+		}
+		ret = append(ret, element)
+	}
+}
+
+func (c *compactor) UnmarshalBinary(data []byte) error {
+	var g compactorAsGob
+	var err error
+	gobDec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err = gobDec.Decode(&g); err != nil {
+		return err
+	}
+	if len(g.EncodedTypeAsJSON) == 0 {
+		return nil
+	}
+	var t beam.EncodedType
+	if err = json.Unmarshal(g.EncodedTypeAsJSON, &t); err != nil {
+		return err
+	}
+	dec := beam.NewElementDecoder(t.T)
+	decodedSorted := make([][]beam.T, 0, len(g.Sorted))
+	for _, sorted := range g.Sorted {
+		decoded, err := decodeElements(dec, sorted)
+		if err != nil {
+			return err
+		}
+		decodedSorted = append(decodedSorted, decoded)
+	}
+	c.sorted = decodedSorted
+	if c.unsorted, err = decodeElements(dec, g.Unsorted); err != nil {
+		return err
+	}
+	return nil
+}
+
+// update inserts an element into the compactor.
+func (c *compactor) update(element beam.T) {
+	c.unsorted = append(c.unsorted, element)
+}
+
+// size returns the number of elements stored in this compactor.
+func (c *compactor) size() int {
+	size := 0
+	for _, s := range c.sorted {
+		size += len(s)
+	}
+	return len(c.unsorted) + size
+}
+
+type sorter struct {
+	less reflectx.Func2x1
+	data []beam.T
+}
+
+func (s sorter) Len() int           { return len(s.data) }
+func (s sorter) Less(i, j int) bool { return s.less.Call2x1(s.data[i], s.data[j]).(bool) }
+func (s sorter) Swap(i, j int)      { s.data[i], s.data[j] = s.data[j], s.data[i] }
+
+// sort sorts the compactor and returns all the elements in sorted order.
+func (c *compactor) sort(less reflectx.Func2x1) []beam.T {
+	sort.Sort(sorter{data: c.unsorted, less: less})
+	h := sortListHeap{data: c.sorted, less: less}
+	heap.Init(&h)
+	sorted := make([]beam.T, 0, c.size()-len(c.unsorted))
+	for h.Len() > 0 {
+		s := heap.Pop(&h).([]beam.T)
+		sorted = append(sorted, s[0])
+		if len(s) > 1 {
+			heap.Push(&h, s[1:])
+		}
+	}
+	c.sorted = [][]beam.T{mergeSorted(sorted, c.unsorted, func(a, b interface{}) bool { return less.Call2x1(a, b).(bool) })}
+	c.unsorted = nil
+	if len(c.sorted[0]) == 0 {
+		c.sorted = nil
+		return nil
+	}
+	return c.sorted[0]
+}
+
+// Compactors holds the state of the quantile approximation compactors.
+type compactors struct {
+	// References "K" from the paper which influences the amount of memory used.
+	K int
+	// When compacting, we want to alternate between taking elements at even vs odd indices.
+	// The paper suggests using a random variable but we'd prefer to stay deterministic.
+	// Especially when merging two compactors we want to keep track of how often we've selected odds vs evens.
+	NumberOfCompactions int
+
+	// Each compactor takes a sample of elements.
+	// The "height" (also known as the index in this slice) of the compactor determines the weight of its elements.
+	// The weight of a compactor of height h is 2^h.
+	// For example, for h = 3 (which would be compactors[3]), the weight is 2^3 = 8. That means each element in that compactor represents 8 instances of itself.
+	Compactors []compactor
+}
+
+func (c *compactors) totalCapacity() int {
+	totalCapacity := 0
+	for _, compactor := range c.Compactors {
+		totalCapacity += compactor.capacity
+	}
+	return totalCapacity
+}
+
+func (c *compactors) size() int {
+	size := 0
+	for _, compactor := range c.Compactors {
+		size += compactor.size()
+	}
+	return size
+}
+
+// capacity computes the capacity of a compactor at a certain level.
+// The paper suggests decreasing the capacity of lower-leveled compactors as we add more elements.
+func (c *compactors) capacity(compactorLevel int) int {
+	return int(math.Ceil(math.Pow(capacityCoefficient, float64(len(c.Compactors)-compactorLevel-1))*float64(c.K))) + 1
+}
+
+// compact compacts all compactors until the total size is less than the maximum capacity of all compactors.
+func (c *compactors) compact(less reflectx.Func2x1) {
+	for c.size() > c.totalCapacity() {
+		for level, compactor := range c.Compactors {
+			if compactor.size() > compactor.capacity {
+				c.compactLevel(level, less)
+				// Merging compactors can cause us to exceed max capacity in multiple compactors.
+				if c.size() < c.totalCapacity() {
+					// Do lazy compaction as described in the paper.
+					break
+				}
+			}
+		}
+	}
+}
+
+// update inserts the given element into the compactors. If this element causes the compactors to grow too large, we perform the compaction here.
+func (c *compactors) update(element beam.T, weight int, less reflectx.Func2x1) {
+	level := int(math.Log2(float64(weight)))
+	c.growToIncludeLevel(level)
+	c.Compactors[level].update(element)
+	// From the paper, we're using the "Splitting the Input" approach.
+	remainingWeight := weight - (1 << uint(level))
+	// Only attempt compaction if we're doing the last update. Otherwise we'd be compacting too often.
+	if remainingWeight <= 0 {
+		c.compact(less)
+	} else {
+		c.update(element, remainingWeight, less)
+	}
+}
+
+// growToIncludeLevel ensures we have compactors available at the given level.
+func (c *compactors) growToIncludeLevel(level int) {
+	if len(c.Compactors)-1 >= level {
+		return
+	}
+	for i := len(c.Compactors) - 1; i < level; i++ {
+		c.Compactors = append(c.Compactors, compactor{})
+	}
+	for level := range c.Compactors {
+		c.Compactors[level].capacity = c.capacity(level)
+	}
+}
+
+// compact compacts elements in compactors.
+func (c *compactors) compactLevel(level int, less reflectx.Func2x1) {
+	c.growToIncludeLevel(level + 1)
+	jitterIndex := 0
+	// Create a temporary buffer to hold the compacted elements.
+	// Buffering the elements like this makes it easier to call mergeSorted.
+	compacted := make([]beam.T, 0, c.Compactors[level].size()/2)
+	selectEvens := c.NumberOfCompactions%2 == 0
+	c.NumberOfCompactions++
+	for _, element := range c.Compactors[level].sort(less) {
+		if (jitterIndex%2 == 0) == selectEvens {
+			compacted = append(compacted, element)
+		}
+		jitterIndex++
+	}
+	if len(compacted) > 0 {
+		c.Compactors[level+1].sorted = append(c.Compactors[level+1].sorted, compacted)
+	}
+	// Clear out the compactor at this level since we've finished compacting it. The compacted elements have already been moved to the next compactor.
+	c.Compactors[level].sorted = nil
+	c.Compactors[level].unsorted = nil
+}
+
+func encodeCompactors(c *compactors) ([]byte, error) {
+	var buf bytes.Buffer
+	enc := gob.NewEncoder(&buf)
+	if err := enc.Encode(c); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func decodeCompactors(data []byte) (*compactors, error) {
+	var compactors compactors
+	dec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err := dec.Decode(&compactors); err != nil {
+		return nil, err
+	}
+	for level := range compactors.Compactors {
+		compactors.Compactors[level].capacity = compactors.capacity(level)
+	}
+	return &compactors, nil
+}
+
+// mergeSorted takes two slices which are already sorted and returns a new slice containing all elements sorted together.
+func mergeSorted(a, b []beam.T, less func(interface{}, interface{}) bool) []beam.T {
+	output := make([]beam.T, 0, len(a)+len(b))
+	for len(a) > 0 && len(b) > 0 {
+		if less(a[0], b[0]) {
+			output = append(output, a[0])
+			a = a[1:]
+		} else {
+			output = append(output, b[0])
+			b = b[1:]
+		}
+	}
+	if len(a) > 0 {
+		output = append(output, a...)
+	} else {
+		output = append(output, b...)
+	}
+	return output
+}
+
+// mergeSortedWeighted takes two slices which are already sorted and returns a new slice containing all elements sorted together.
+func mergeSortedWeighted(a, b []weightedElement, less func(interface{}, interface{}) bool) []weightedElement {
+	output := make([]weightedElement, 0, len(a)+len(b))
+	for len(a) > 0 && len(b) > 0 {
+		if less(a[0], b[0]) {
+			output = append(output, a[0])
+			a = a[1:]
+		} else {
+			output = append(output, b[0])
+			b = b[1:]
+		}
+	}
+	if len(a) > 0 {
+		output = append(output, a...)
+	} else {
+		output = append(output, b...)
+	}
+	return output
+}
+
+// merge joins two compactors together.
+func (c *compactors) merge(other *compactors, less reflectx.Func2x1) {
+	for level := range c.Compactors {
+		if len(other.Compactors)-1 < level {
+			break
+		}
+		c.Compactors[level].unsorted = append(c.Compactors[level].unsorted, other.Compactors[level].unsorted...)
+		c.Compactors[level].sorted = append(c.Compactors[level].sorted, other.Compactors[level].sorted...)
+	}
+	if len(other.Compactors) > len(c.Compactors) {
+		c.Compactors = append(c.Compactors, other.Compactors[len(c.Compactors):]...)
+	}
+	c.NumberOfCompactions += other.NumberOfCompactions
+	c.compact(less)
+}
+
+// approximateQuantilesCombineFnState contains the payload for the combiners.
+// Ideally this would be a single combine function but in certain cases, runners attempts to do all the merges on a single machine.
+// Unfortunately the merges can be slow for extremely large datasets and large values of K. If the merge takes too long, it will get canceled and the job will never complete.
+// Thus we split up the combiners into multiple functions to force the runner to do the work in parallel.
+// This state can be shared across all of the split-up functions.
+type approximateQuantilesCombineFnState struct {
+	// The size of the compactors.
+	// The memory consumed, and the error are controlled by this parameter.
+	K int `json:"k"`
+	// Used to compare elements.
+	LessFunc beam.EncodedFunc
+	// Internally cached instance.
+	less         reflectx.Func2x1
+	NumQuantiles int `json:"numQuantiles"`
+}
+
+func (f *approximateQuantilesCombineFnState) setup() error {
+	f.less = reflectx.ToFunc2x1(f.LessFunc.Fn)
+	return nil
+}
+
+func (f *approximateQuantilesCombineFnState) createAccumulator() *compactors {
+	return &compactors{
+		K:          f.K,
+		Compactors: []compactor{{capacity: f.K}},
+	}
+}
+
+// approximateQuantilesOutputFn extracts the final output containing the quantiles.
+type approximateQuantilesOutputFn struct {
+	State approximateQuantilesCombineFnState `json:"state"`
+}
+
+func (f *approximateQuantilesOutputFn) Setup() error {
+	return f.State.setup()
+}
+
+func (f *approximateQuantilesOutputFn) CreateAccumulator() *compactors {
+	return f.State.createAccumulator()
+}
+
+func (f *approximateQuantilesOutputFn) AddInput(compactors *compactors, element *compactors) *compactors {
+	compactors.merge(element, f.State.less)
+	return compactors
+}
+
+func (f *approximateQuantilesOutputFn) MergeAccumulators(ctx context.Context, a, b *compactors) *compactors {
+	a.merge(b, f.State.less)
+	return a
+}
+
+type weightedElementAsGob struct {
+	EncodedTypeAsJSON []byte
+	Weight            int
+	Element           []byte
+}
+
+func encodeWeightedElement(element weightedElement) ([]byte, error) {
+	t := reflect.TypeOf(element.element)
+	enc := beam.NewElementEncoder(t)
+	var buf bytes.Buffer
+	if err := enc.Encode(element.element, &buf); err != nil {
+		return nil, err
+	}
+	tAsJSON, err := beam.EncodedType{T: t}.MarshalJSON()
+	if err != nil {
+		return nil, err
+	}
+	var gobBuf bytes.Buffer
+	if err := gob.NewEncoder(&gobBuf).Encode(weightedElementAsGob{
+		EncodedTypeAsJSON: tAsJSON,
+		Weight:            element.weight,
+		Element:           buf.Bytes(),
+	}); err != nil {
+		return nil, err
+	}
+	return gobBuf.Bytes(), nil
+}
+
+func decodeWeightedElement(data []byte) (weightedElement, error) {
+	var g weightedElementAsGob
+	dec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err := dec.Decode(&g); err != nil {
+		return weightedElement{}, err
+	}
+	var t beam.EncodedType
+	if err := t.UnmarshalJSON(g.EncodedTypeAsJSON); err != nil {
+		return weightedElement{}, err
+	}
+	element, err := beam.NewElementDecoder(t.T).Decode(bytes.NewBuffer(g.Element))
+	if err != nil {
+		return weightedElement{}, err
+	}
+	return weightedElement{
+		weight:  g.Weight,
+		element: element,
+	}, nil
+}
+
+type weightedElement struct {
+	weight  int
+	element beam.T
+}
+
+func toWeightedSlice(compactor compactor, less reflectx.Func2x1, weight int) []weightedElement {
+	sorted := compactor.sort(less)
+	weightedElements := make([]weightedElement, 0, len(sorted))
+	for _, element := range sorted {
+		weightedElements = append(weightedElements, weightedElement{weight: weight, element: element})
+	}
+	return weightedElements
+}
+func (f *approximateQuantilesOutputFn) ExtractOutput(ctx context.Context, compactors *compactors) []beam.T {
+	sorted := toWeightedSlice(compactors.Compactors[0], f.State.less, 1)
+	for level, compactor := range compactors.Compactors[1:] {
+		sorted = mergeSortedWeighted(sorted, toWeightedSlice(compactor, f.State.less, 1<<uint(level)), func(a, b interface{}) bool {
+			return f.State.less.Call2x1(a.(weightedElement).element, b.(weightedElement).element).(bool)
+		})
+	}
+	totalElements := 0
+	for _, element := range sorted {
+		totalElements += element.weight
+	}
+	ret := make([]beam.T, 0, f.State.NumQuantiles)
+	// Rank represents the estimate of how many elements we've seen as we iterate through the sorted list of elements stored in the compactors.
+	// Recall that each element stored in a compactor is also assigned a weight indicating how many elements from the input it represents.
+	rank := float64(0)
+	// Represents the quantile we're currently searching for.
+	currentQuantile := float64(1)
+	for _, element := range sorted {
+		rank += float64(element.weight)
+		if rank/float64(totalElements) >= currentQuantile/float64(f.State.NumQuantiles) {
+			ret = append(ret, element.element)
+			currentQuantile++
+		}
+		if currentQuantile >= float64(f.State.NumQuantiles) {
+			break
+		}
+	}
+	return ret
+}
+
+// approximateQuantilesInputFn combines elements into compactors, but not necessarily the final compactor.
+type approximateQuantilesInputFn approximateQuantilesOutputFn
+
+func (f *approximateQuantilesInputFn) Setup() error {
+	return f.State.setup()
+}
+
+func (f *approximateQuantilesInputFn) CreateAccumulator() *compactors {
+	return f.State.createAccumulator()
+}
+
+func (f *approximateQuantilesInputFn) AddInput(compactors *compactors, element weightedElement) *compactors {
+	compactors.update(element.element, element.weight, f.State.less)
+	return compactors
+}
+
+func (f *approximateQuantilesInputFn) MergeAccumulators(ctx context.Context, a, b *compactors) *compactors {
+	a.merge(b, f.State.less)
+	return a
+}
+
+func (f *approximateQuantilesInputFn) ExtractOutput(ctx context.Context, compactors *compactors) *compactors {
+	for i := range compactors.Compactors {
+		// Sort the compactors here so when we're merging them for the final output, they're already sorted and we can merge elements in order.
+		compactors.Compactors[i].sort(f.State.less)
+	}
+	return compactors
+}
+
+// approximateQuantilesMergeOnlyFn combines compactors into smaller compactors, but not necessarily the final compactor.
+type approximateQuantilesMergeOnlyFn approximateQuantilesOutputFn
+
+func (f *approximateQuantilesMergeOnlyFn) Setup() error {
+	return f.State.setup()
+}
+
+func (f *approximateQuantilesMergeOnlyFn) CreateAccumulator() *compactors {
+	return f.State.createAccumulator()
+}
+
+func (f *approximateQuantilesMergeOnlyFn) AddInput(compactors *compactors, element *compactors) *compactors {
+	compactors.merge(element, f.State.less)
+	return compactors
+}
+
+func (f *approximateQuantilesMergeOnlyFn) MergeAccumulators(ctx context.Context, a, b *compactors) *compactors {
+	a.merge(b, f.State.less)
+	return a
+}
+
+func (f *approximateQuantilesMergeOnlyFn) ExtractOutput(ctx context.Context, compactors *compactors) *compactors {
+	for i := range compactors.Compactors {
+		// Sort the compactors here so when we're merging them for the final output, they're already sorted and we can merge elements in order.
+		compactors.Compactors[i].sort(f.State.less)
+	}
+	return compactors
+}
+
+type shardElementsFn struct {
+	Shards         int              `json:"shards"`
+	T              beam.EncodedType `json:"t"`
+	elementEncoder beam.ElementEncoder
+}
+
+func (s *shardElementsFn) Setup() {
+	s.elementEncoder = beam.NewElementEncoder(s.T.T)
+}
+
+func (s *shardElementsFn) ProcessElement(element beam.T) (int, beam.T) {
+	h := crc32.NewIEEE()
+	s.elementEncoder.Encode(element, h)

Review comment:
       This is a spot where I'm so glad I stuck to the io interfaces for the most part: No allocation overhead other than what the hasher does. Encoding is cheap compared to the allocations on decoding :D. 

##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles.go
##########
@@ -0,0 +1,702 @@
+// 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 stats
+
+// Approximate quantiles is implemented based on https://arxiv.org/pdf/1907.00236.pdf.
+
+import (
+	"bytes"
+	"container/heap"
+	"context"
+	"encoding/gob"
+	"encoding/json"
+	"hash/crc32"
+	"io"
+	"math"
+	"reflect"
+	"sort"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+)
+
+func init() {
+	compactorsType := reflect.TypeOf((**compactors)(nil)).Elem()
+	weightedElementType := reflect.TypeOf((*weightedElement)(nil)).Elem()
+	beam.RegisterType(compactorsType)
+	beam.RegisterType(weightedElementType)
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesInputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesMergeOnlyFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesOutputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*shardElementsFn)(nil)).Elem())
+	beam.RegisterCoder(compactorsType, encodeCompactors, decodeCompactors)
+	beam.RegisterCoder(weightedElementType, encodeWeightedElement, decodeWeightedElement)
+}
+
+// Opts contains settings used to configure how approximate quantiles are computed.
+type Opts struct {
+	// Controls the memory used and approximation error (difference between the quantile returned and the true quantile.)
+	K int
+	// Number of quantiles to return. The algorithm will return NumQuantiles - 1 numbers
+	NumQuantiles int
+	// For extremely large datasets, the Go SDK has some problems holding on to data and might OOM.
+	// If ApproximateQuantiles is running out of memory, you can use this option to tune how the data is sharded internally.
+	// This parameter is optional. If unspecified, Beam will compact all elements into a single compactor at once.
+	InternalSharding []int
+}
+
+// The paper suggests reducing the size of the lower-level compactors as we grow.
+// We reduce the capacity at this rate.
+// The paper suggests 1/sqrt(2) is ideal. That's approximately 0.7.
+const capacityCoefficient float64 = 0.7
+
+type sortListHeap struct {
+	data [][]beam.T
+	less reflectx.Func2x1
+}
+
+func (s sortListHeap) Len() int            { return len(s.data) }
+func (s sortListHeap) Less(i, j int) bool  { return s.less.Call2x1(s.data[i][0], s.data[j][0]).(bool) }
+func (s sortListHeap) Swap(i, j int)       { s.data[i], s.data[j] = s.data[j], s.data[i] }
+func (s *sortListHeap) Push(x interface{}) { s.data = append(s.data, x.([]beam.T)) }
+func (s *sortListHeap) Pop() interface{} {
+	var x beam.T
+	x, s.data = s.data[len(s.data)-1], s.data[:len(s.data)-1]
+	return x
+}
+
+// compactor contains elements to be compacted.
+type compactor struct {
+	// Compaction needs to sort elements before compacting. Thus in practice, we should often have some pre-sorted data.
+	// We want to keep it separate so we can sort only the unsorted data and merge the two sorted lists together.
+	// If we're only receiving elements of weight 1, only level 0 will ever contain unsorted data and the rest of the levels will always remain sorted.
+	// To prevent repeated allocation/copying, we keep multiple sorted lists and then merge them together
+	sorted   [][]beam.T
+	unsorted []beam.T
+	// How many items should be stored in this compactor before it should get compacted.
+	// Note that this is not a hard limit.
+	// The paper suggests implementing lazy compaction which would allow
+	// compactors to temporarily exceed their capacity as long as the total
+	// elements in all compactors doesn't exceed the total capacity in all
+	// compactors. In other words, compactors can temporarily borrow capacity
+	// from each other.
+	// In the paper, this is referred to as the variable k_h.
+	capacity int
+}
+
+type compactorAsGob struct {
+	Sorted            [][]byte
+	Unsorted          []byte
+	EncodedTypeAsJSON []byte
+}
+
+func encodeElements(enc beam.ElementEncoder, elements []beam.T) ([]byte, error) {
+	var buf bytes.Buffer
+	for _, e := range elements {
+		if err := enc.Encode(e, &buf); err != nil {
+			return nil, err
+		}
+	}
+	return buf.Bytes(), nil
+}
+
+func (c *compactor) getElementType() reflect.Type {
+	for _, e := range c.sorted {
+		for _, e2 := range e {
+			return reflect.TypeOf(e2)
+		}
+	}
+	for _, e := range c.unsorted {
+		return reflect.TypeOf(e)
+	}
+	return nil
+}
+
+func (c *compactor) MarshalBinary() ([]byte, error) {
+	t := c.getElementType()
+	var buf bytes.Buffer
+	if t == nil {
+		enc := gob.NewEncoder(&buf)
+		if err := enc.Encode(compactorAsGob{}); err != nil {
+			return nil, err
+		}
+		return buf.Bytes(), nil
+	}
+	enc := beam.NewElementEncoder(t)
+	encodedSorted := make([][]byte, 0, len(c.sorted))
+	for _, sorted := range c.sorted {
+		encoded, err := encodeElements(enc, sorted)
+		if err != nil {
+			return nil, err
+		}
+		encodedSorted = append(encodedSorted, encoded)
+	}
+	encodedUnsorted, err := encodeElements(enc, c.unsorted)
+	if err != nil {
+		return nil, err
+	}
+	tAsJSON, err := beam.EncodedType{T: t}.MarshalJSON()
+	if err != nil {
+		return nil, err
+	}
+	gobEnc := gob.NewEncoder(&buf)
+	if err = gobEnc.Encode(compactorAsGob{
+		Sorted:            encodedSorted,
+		Unsorted:          encodedUnsorted,
+		EncodedTypeAsJSON: tAsJSON,
+	}); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func decodeElements(dec beam.ElementDecoder, data []byte) ([]beam.T, error) {
+	buf := bytes.NewBuffer(data)
+	ret := []beam.T{}
+	for {
+		element, err := dec.Decode(buf)
+		if err == io.EOF {
+			return ret, nil
+		} else if err != nil {
+			return nil, err
+		}
+		ret = append(ret, element)
+	}
+}
+
+func (c *compactor) UnmarshalBinary(data []byte) error {
+	var g compactorAsGob
+	var err error
+	gobDec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err = gobDec.Decode(&g); err != nil {
+		return err
+	}
+	if len(g.EncodedTypeAsJSON) == 0 {
+		return nil
+	}
+	var t beam.EncodedType
+	if err = json.Unmarshal(g.EncodedTypeAsJSON, &t); err != nil {
+		return err
+	}
+	dec := beam.NewElementDecoder(t.T)
+	decodedSorted := make([][]beam.T, 0, len(g.Sorted))
+	for _, sorted := range g.Sorted {
+		decoded, err := decodeElements(dec, sorted)
+		if err != nil {
+			return err
+		}
+		decodedSorted = append(decodedSorted, decoded)
+	}
+	c.sorted = decodedSorted
+	if c.unsorted, err = decodeElements(dec, g.Unsorted); err != nil {
+		return err
+	}
+	return nil
+}
+
+// update inserts an element into the compactor.
+func (c *compactor) update(element beam.T) {
+	c.unsorted = append(c.unsorted, element)
+}
+
+// size returns the number of elements stored in this compactor.
+func (c *compactor) size() int {
+	size := 0
+	for _, s := range c.sorted {
+		size += len(s)
+	}
+	return len(c.unsorted) + size
+}
+
+type sorter struct {
+	less reflectx.Func2x1
+	data []beam.T
+}
+
+func (s sorter) Len() int           { return len(s.data) }
+func (s sorter) Less(i, j int) bool { return s.less.Call2x1(s.data[i], s.data[j]).(bool) }
+func (s sorter) Swap(i, j int)      { s.data[i], s.data[j] = s.data[j], s.data[i] }
+
+// sort sorts the compactor and returns all the elements in sorted order.
+func (c *compactor) sort(less reflectx.Func2x1) []beam.T {
+	sort.Sort(sorter{data: c.unsorted, less: less})
+	h := sortListHeap{data: c.sorted, less: less}
+	heap.Init(&h)
+	sorted := make([]beam.T, 0, c.size()-len(c.unsorted))
+	for h.Len() > 0 {
+		s := heap.Pop(&h).([]beam.T)
+		sorted = append(sorted, s[0])
+		if len(s) > 1 {
+			heap.Push(&h, s[1:])
+		}
+	}
+	c.sorted = [][]beam.T{mergeSorted(sorted, c.unsorted, func(a, b interface{}) bool { return less.Call2x1(a, b).(bool) })}
+	c.unsorted = nil
+	if len(c.sorted[0]) == 0 {
+		c.sorted = nil
+		return nil
+	}
+	return c.sorted[0]
+}
+
+// Compactors holds the state of the quantile approximation compactors.
+type compactors struct {
+	// References "K" from the paper which influences the amount of memory used.
+	K int
+	// When compacting, we want to alternate between taking elements at even vs odd indices.
+	// The paper suggests using a random variable but we'd prefer to stay deterministic.
+	// Especially when merging two compactors we want to keep track of how often we've selected odds vs evens.
+	NumberOfCompactions int
+
+	// Each compactor takes a sample of elements.
+	// The "height" (also known as the index in this slice) of the compactor determines the weight of its elements.
+	// The weight of a compactor of height h is 2^h.
+	// For example, for h = 3 (which would be compactors[3]), the weight is 2^3 = 8. That means each element in that compactor represents 8 instances of itself.
+	Compactors []compactor
+}
+
+func (c *compactors) totalCapacity() int {
+	totalCapacity := 0
+	for _, compactor := range c.Compactors {
+		totalCapacity += compactor.capacity
+	}
+	return totalCapacity
+}
+
+func (c *compactors) size() int {
+	size := 0
+	for _, compactor := range c.Compactors {
+		size += compactor.size()
+	}
+	return size
+}
+
+// capacity computes the capacity of a compactor at a certain level.
+// The paper suggests decreasing the capacity of lower-leveled compactors as we add more elements.
+func (c *compactors) capacity(compactorLevel int) int {
+	return int(math.Ceil(math.Pow(capacityCoefficient, float64(len(c.Compactors)-compactorLevel-1))*float64(c.K))) + 1
+}
+
+// compact compacts all compactors until the total size is less than the maximum capacity of all compactors.
+func (c *compactors) compact(less reflectx.Func2x1) {
+	for c.size() > c.totalCapacity() {
+		for level, compactor := range c.Compactors {
+			if compactor.size() > compactor.capacity {
+				c.compactLevel(level, less)
+				// Merging compactors can cause us to exceed max capacity in multiple compactors.
+				if c.size() < c.totalCapacity() {
+					// Do lazy compaction as described in the paper.
+					break
+				}
+			}
+		}
+	}
+}
+
+// update inserts the given element into the compactors. If this element causes the compactors to grow too large, we perform the compaction here.
+func (c *compactors) update(element beam.T, weight int, less reflectx.Func2x1) {
+	level := int(math.Log2(float64(weight)))
+	c.growToIncludeLevel(level)
+	c.Compactors[level].update(element)
+	// From the paper, we're using the "Splitting the Input" approach.
+	remainingWeight := weight - (1 << uint(level))
+	// Only attempt compaction if we're doing the last update. Otherwise we'd be compacting too often.
+	if remainingWeight <= 0 {
+		c.compact(less)
+	} else {
+		c.update(element, remainingWeight, less)
+	}
+}
+
+// growToIncludeLevel ensures we have compactors available at the given level.
+func (c *compactors) growToIncludeLevel(level int) {
+	if len(c.Compactors)-1 >= level {
+		return
+	}
+	for i := len(c.Compactors) - 1; i < level; i++ {
+		c.Compactors = append(c.Compactors, compactor{})
+	}
+	for level := range c.Compactors {
+		c.Compactors[level].capacity = c.capacity(level)
+	}
+}
+
+// compact compacts elements in compactors.
+func (c *compactors) compactLevel(level int, less reflectx.Func2x1) {
+	c.growToIncludeLevel(level + 1)
+	jitterIndex := 0
+	// Create a temporary buffer to hold the compacted elements.
+	// Buffering the elements like this makes it easier to call mergeSorted.
+	compacted := make([]beam.T, 0, c.Compactors[level].size()/2)
+	selectEvens := c.NumberOfCompactions%2 == 0
+	c.NumberOfCompactions++
+	for _, element := range c.Compactors[level].sort(less) {
+		if (jitterIndex%2 == 0) == selectEvens {
+			compacted = append(compacted, element)
+		}
+		jitterIndex++
+	}
+	if len(compacted) > 0 {
+		c.Compactors[level+1].sorted = append(c.Compactors[level+1].sorted, compacted)
+	}
+	// Clear out the compactor at this level since we've finished compacting it. The compacted elements have already been moved to the next compactor.
+	c.Compactors[level].sorted = nil
+	c.Compactors[level].unsorted = nil
+}
+
+func encodeCompactors(c *compactors) ([]byte, error) {
+	var buf bytes.Buffer
+	enc := gob.NewEncoder(&buf)
+	if err := enc.Encode(c); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func decodeCompactors(data []byte) (*compactors, error) {
+	var compactors compactors
+	dec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err := dec.Decode(&compactors); err != nil {
+		return nil, err
+	}
+	for level := range compactors.Compactors {
+		compactors.Compactors[level].capacity = compactors.capacity(level)
+	}
+	return &compactors, nil
+}
+
+// mergeSorted takes two slices which are already sorted and returns a new slice containing all elements sorted together.
+func mergeSorted(a, b []beam.T, less func(interface{}, interface{}) bool) []beam.T {
+	output := make([]beam.T, 0, len(a)+len(b))
+	for len(a) > 0 && len(b) > 0 {
+		if less(a[0], b[0]) {
+			output = append(output, a[0])
+			a = a[1:]
+		} else {
+			output = append(output, b[0])
+			b = b[1:]
+		}
+	}
+	if len(a) > 0 {
+		output = append(output, a...)
+	} else {
+		output = append(output, b...)
+	}
+	return output
+}
+
+// mergeSortedWeighted takes two slices which are already sorted and returns a new slice containing all elements sorted together.
+func mergeSortedWeighted(a, b []weightedElement, less func(interface{}, interface{}) bool) []weightedElement {
+	output := make([]weightedElement, 0, len(a)+len(b))
+	for len(a) > 0 && len(b) > 0 {
+		if less(a[0], b[0]) {
+			output = append(output, a[0])
+			a = a[1:]
+		} else {
+			output = append(output, b[0])
+			b = b[1:]
+		}
+	}
+	if len(a) > 0 {
+		output = append(output, a...)
+	} else {
+		output = append(output, b...)
+	}
+	return output
+}
+
+// merge joins two compactors together.
+func (c *compactors) merge(other *compactors, less reflectx.Func2x1) {
+	for level := range c.Compactors {
+		if len(other.Compactors)-1 < level {
+			break
+		}
+		c.Compactors[level].unsorted = append(c.Compactors[level].unsorted, other.Compactors[level].unsorted...)
+		c.Compactors[level].sorted = append(c.Compactors[level].sorted, other.Compactors[level].sorted...)
+	}
+	if len(other.Compactors) > len(c.Compactors) {
+		c.Compactors = append(c.Compactors, other.Compactors[len(c.Compactors):]...)
+	}
+	c.NumberOfCompactions += other.NumberOfCompactions
+	c.compact(less)
+}
+
+// approximateQuantilesCombineFnState contains the payload for the combiners.
+// Ideally this would be a single combine function but in certain cases, runners attempts to do all the merges on a single machine.
+// Unfortunately the merges can be slow for extremely large datasets and large values of K. If the merge takes too long, it will get canceled and the job will never complete.
+// Thus we split up the combiners into multiple functions to force the runner to do the work in parallel.
+// This state can be shared across all of the split-up functions.
+type approximateQuantilesCombineFnState struct {
+	// The size of the compactors.
+	// The memory consumed, and the error are controlled by this parameter.
+	K int `json:"k"`
+	// Used to compare elements.
+	LessFunc beam.EncodedFunc
+	// Internally cached instance.
+	less         reflectx.Func2x1
+	NumQuantiles int `json:"numQuantiles"`
+}
+
+func (f *approximateQuantilesCombineFnState) setup() error {
+	f.less = reflectx.ToFunc2x1(f.LessFunc.Fn)
+	return nil
+}
+
+func (f *approximateQuantilesCombineFnState) createAccumulator() *compactors {
+	return &compactors{
+		K:          f.K,
+		Compactors: []compactor{{capacity: f.K}},
+	}
+}
+
+// approximateQuantilesOutputFn extracts the final output containing the quantiles.
+type approximateQuantilesOutputFn struct {
+	State approximateQuantilesCombineFnState `json:"state"`
+}
+
+func (f *approximateQuantilesOutputFn) Setup() error {
+	return f.State.setup()
+}
+
+func (f *approximateQuantilesOutputFn) CreateAccumulator() *compactors {
+	return f.State.createAccumulator()
+}
+
+func (f *approximateQuantilesOutputFn) AddInput(compactors *compactors, element *compactors) *compactors {
+	compactors.merge(element, f.State.less)
+	return compactors
+}
+
+func (f *approximateQuantilesOutputFn) MergeAccumulators(ctx context.Context, a, b *compactors) *compactors {
+	a.merge(b, f.State.less)
+	return a
+}
+
+type weightedElementAsGob struct {
+	EncodedTypeAsJSON []byte
+	Weight            int
+	Element           []byte
+}
+
+func encodeWeightedElement(element weightedElement) ([]byte, error) {
+	t := reflect.TypeOf(element.element)
+	enc := beam.NewElementEncoder(t)
+	var buf bytes.Buffer
+	if err := enc.Encode(element.element, &buf); err != nil {
+		return nil, err
+	}
+	tAsJSON, err := beam.EncodedType{T: t}.MarshalJSON()
+	if err != nil {
+		return nil, err
+	}
+	var gobBuf bytes.Buffer
+	if err := gob.NewEncoder(&gobBuf).Encode(weightedElementAsGob{
+		EncodedTypeAsJSON: tAsJSON,
+		Weight:            element.weight,
+		Element:           buf.Bytes(),
+	}); err != nil {
+		return nil, err
+	}
+	return gobBuf.Bytes(), nil
+}
+
+func decodeWeightedElement(data []byte) (weightedElement, error) {
+	var g weightedElementAsGob
+	dec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err := dec.Decode(&g); err != nil {
+		return weightedElement{}, err
+	}
+	var t beam.EncodedType
+	if err := t.UnmarshalJSON(g.EncodedTypeAsJSON); err != nil {
+		return weightedElement{}, err
+	}
+	element, err := beam.NewElementDecoder(t.T).Decode(bytes.NewBuffer(g.Element))
+	if err != nil {
+		return weightedElement{}, err
+	}
+	return weightedElement{
+		weight:  g.Weight,
+		element: element,
+	}, nil
+}
+
+type weightedElement struct {
+	weight  int
+	element beam.T
+}
+
+func toWeightedSlice(compactor compactor, less reflectx.Func2x1, weight int) []weightedElement {
+	sorted := compactor.sort(less)
+	weightedElements := make([]weightedElement, 0, len(sorted))
+	for _, element := range sorted {
+		weightedElements = append(weightedElements, weightedElement{weight: weight, element: element})
+	}
+	return weightedElements
+}
+func (f *approximateQuantilesOutputFn) ExtractOutput(ctx context.Context, compactors *compactors) []beam.T {
+	sorted := toWeightedSlice(compactors.Compactors[0], f.State.less, 1)
+	for level, compactor := range compactors.Compactors[1:] {
+		sorted = mergeSortedWeighted(sorted, toWeightedSlice(compactor, f.State.less, 1<<uint(level)), func(a, b interface{}) bool {
+			return f.State.less.Call2x1(a.(weightedElement).element, b.(weightedElement).element).(bool)
+		})
+	}
+	totalElements := 0
+	for _, element := range sorted {
+		totalElements += element.weight
+	}
+	ret := make([]beam.T, 0, f.State.NumQuantiles)
+	// Rank represents the estimate of how many elements we've seen as we iterate through the sorted list of elements stored in the compactors.
+	// Recall that each element stored in a compactor is also assigned a weight indicating how many elements from the input it represents.
+	rank := float64(0)
+	// Represents the quantile we're currently searching for.
+	currentQuantile := float64(1)
+	for _, element := range sorted {
+		rank += float64(element.weight)
+		if rank/float64(totalElements) >= currentQuantile/float64(f.State.NumQuantiles) {
+			ret = append(ret, element.element)
+			currentQuantile++
+		}
+		if currentQuantile >= float64(f.State.NumQuantiles) {
+			break
+		}
+	}
+	return ret
+}
+
+// approximateQuantilesInputFn combines elements into compactors, but not necessarily the final compactor.
+type approximateQuantilesInputFn approximateQuantilesOutputFn
+
+func (f *approximateQuantilesInputFn) Setup() error {
+	return f.State.setup()
+}
+
+func (f *approximateQuantilesInputFn) CreateAccumulator() *compactors {
+	return f.State.createAccumulator()
+}
+
+func (f *approximateQuantilesInputFn) AddInput(compactors *compactors, element weightedElement) *compactors {
+	compactors.update(element.element, element.weight, f.State.less)
+	return compactors
+}
+
+func (f *approximateQuantilesInputFn) MergeAccumulators(ctx context.Context, a, b *compactors) *compactors {
+	a.merge(b, f.State.less)
+	return a
+}
+
+func (f *approximateQuantilesInputFn) ExtractOutput(ctx context.Context, compactors *compactors) *compactors {
+	for i := range compactors.Compactors {
+		// Sort the compactors here so when we're merging them for the final output, they're already sorted and we can merge elements in order.
+		compactors.Compactors[i].sort(f.State.less)
+	}
+	return compactors
+}
+
+// approximateQuantilesMergeOnlyFn combines compactors into smaller compactors, but not necessarily the final compactor.
+type approximateQuantilesMergeOnlyFn approximateQuantilesOutputFn
+
+func (f *approximateQuantilesMergeOnlyFn) Setup() error {
+	return f.State.setup()
+}
+
+func (f *approximateQuantilesMergeOnlyFn) CreateAccumulator() *compactors {
+	return f.State.createAccumulator()
+}
+
+func (f *approximateQuantilesMergeOnlyFn) AddInput(compactors *compactors, element *compactors) *compactors {
+	compactors.merge(element, f.State.less)
+	return compactors
+}
+
+func (f *approximateQuantilesMergeOnlyFn) MergeAccumulators(ctx context.Context, a, b *compactors) *compactors {
+	a.merge(b, f.State.less)
+	return a
+}
+
+func (f *approximateQuantilesMergeOnlyFn) ExtractOutput(ctx context.Context, compactors *compactors) *compactors {
+	for i := range compactors.Compactors {
+		// Sort the compactors here so when we're merging them for the final output, they're already sorted and we can merge elements in order.
+		compactors.Compactors[i].sort(f.State.less)
+	}
+	return compactors
+}
+
+type shardElementsFn struct {
+	Shards         int              `json:"shards"`
+	T              beam.EncodedType `json:"t"`
+	elementEncoder beam.ElementEncoder
+}
+
+func (s *shardElementsFn) Setup() {
+	s.elementEncoder = beam.NewElementEncoder(s.T.T)
+}
+
+func (s *shardElementsFn) ProcessElement(element beam.T) (int, beam.T) {
+	h := crc32.NewIEEE()
+	s.elementEncoder.Encode(element, h)
+	return int(h.Sum32()) % s.Shards, element
+}
+
+func makeWeightedElement(weight int, element beam.T) weightedElement {
+	return weightedElement{weight: weight, element: element}
+}
+
+// ApproximateQuantiles computes approximate quantiles for the input PCollection<T>.
+//
+// The output PCollection contains a single element: a list of numQuantiles - 1 elements approximately splitting up the input collection into numQuantiles separate quantiles.
+// For example, if numQuantiles = 2, the returned list would contain a single element such that approximately half of the input would be less than that element and half would be greater.
+func ApproximateQuantiles(s beam.Scope, pc beam.PCollection, less interface{}, opts Opts) beam.PCollection {
+	return ApproximateWeightedQuantiles(s, beam.ParDo(s, func(e beam.T) (int, beam.T) { return 1, e }, pc), less, opts)
+}
+
+// reduce takes a PCollection<weightedElementWrapper> and returns a PCollection<*compactors>. The output PCollection may have at most shardSizes[len(shardSizes) - 1] compactors.
+func reduce(s beam.Scope, weightedElements beam.PCollection, state approximateQuantilesCombineFnState, shardSizes []int) beam.PCollection {
+	if len(shardSizes) == 0 {
+		shardSizes = []int{1}
+	}
+	shardedCompactors := beam.DropKey(s,
+		beam.CombinePerKey(s, &approximateQuantilesInputFn{State: state},
+			beam.ParDo(s, &shardElementsFn{Shards: shardSizes[0], T: beam.EncodedType{T: reflect.TypeOf((*weightedElement)(nil)).Elem()}}, weightedElements)))
+	shardSizes = shardSizes[1:]
+	compactorsType := reflect.TypeOf((**compactors)(nil)).Elem()
+	for _, shardSize := range shardSizes {
+		shardedCompactors = beam.DropKey(s,
+			beam.CombinePerKey(s, &approximateQuantilesMergeOnlyFn{State: state},
+				beam.ParDo(s, &shardElementsFn{Shards: shardSize, T: beam.EncodedType{T: compactorsType}}, shardedCompactors)))
+	}
+	return shardedCompactors
+}
+
+// ApproximateWeightedQuantiles computes approximate quantiles for the input PCollection<(weight int, T)>.
+//
+// The output PCollection contains a single element: a list of numQuantiles - 1 elements approximately splitting up the input collection into numQuantiles separate quantiles.
+// For example, if numQuantiles = 2, the returned list would contain a single element such that approximately half of the input would be less than that element and half would be greater.

Review comment:
       Aha! When I started reading this 650+ lines ago, I was taking quantiles to mean the actual sets of elements, but this is actually producing the split points between the quantiles. 
   
   Consider also clarifying if the returned split point value is included in the quantile beneath it, or above it.
   




----------------------------------------------------------------
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 #13889: [BEAM-11611] Add transformation for computing approximate quantiles.

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



##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles_test.go
##########
@@ -0,0 +1,274 @@
+// 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 stats
+
+import (
+	"reflect"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/passert"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/ptest"
+	"github.com/google/go-cmp/cmp"
+)
+
+func init() {
+	beam.RegisterFunction(weightedElementToKv)
+
+	// In practice, this runs faster than plain reflection.
+	reflectx.RegisterFunc(reflect.ValueOf(less).Type(), func(_ interface{}) reflectx.Func {
+		return newIntLess()
+	})
+}
+
+type intLess struct {
+	name string
+	t    reflect.Type
+}
+
+func newIntLess() *intLess {
+	return &intLess{
+		name: reflectx.FunctionName(reflect.ValueOf(less).Interface()),
+		t:    reflect.ValueOf(less).Type(),
+	}
+}
+
+func (i *intLess) Name() string {
+	return i.name
+}
+func (i *intLess) Type() reflect.Type {
+	return i.t
+}
+func (i *intLess) Call(args []interface{}) []interface{} {
+	return []interface{}{args[0].(int) < args[1].(int)}
+}
+
+func less(a, b int) bool {
+	return a < b
+}
+
+func TestLargeQuantiles(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := 0; i < numElements; i++ {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{10006, 19973}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestLargeQuantilesReversed(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := numElements - 1; i >= 0; i-- {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{9985, 19959}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestBasicQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	tests := []struct {
+		Input    []int
+		Expected [][]int
+	}{
+		{[]int{}, [][]int{}},
+		{[]int{1}, [][]int{[]int{1}}},
+		{[]int{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20}, [][]int{[]int{6, 13}}},
+	}
+
+	for _, test := range tests {
+		p, s, in, exp := ptest.CreateList2(test.Input, test.Expected)
+		quantiles := ApproximateQuantiles(s, in, less, Opts{
+			K:            3,
+			NumQuantiles: 3,
+		})
+		passert.Equals(s, quantiles, exp)
+
+		if err := ptest.Run(p); err != nil {
+			t.Errorf("ApproximateQuantiles(%v) != %v: %v", test.Input, test.Expected, err)
+		}
+	}
+}
+
+func weightedElementToKv(e testWeightedElement) (int, int) {
+	return e.Weight, e.Element
+}
+
+type testWeightedElement struct {
+	Weight  int
+	Element int
+}
+
+func TestWeightedQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	input := []testWeightedElement{
+		{Weight: 1, Element: 1},
+		{Weight: 10, Element: 2},
+		{Weight: 1, Element: 3},
+		{Weight: 10, Element: 4}}
+	expected := []int{2, 4}
+	p, s, in, exp := ptest.CreateList2(input, [][]int{expected})
+	elementWeightKvs := beam.ParDo(s, weightedElementToKv, in, beam.TypeDefinition{Var: beam.TType, T: reflect.TypeOf(1)})

Review comment:
       ```suggestion
   	elementWeightKvs := beam.ParDo(s, weightedElementToKv, in, beam.TypeDefinition{Var: beam.TType, T: reflect.TypeOf(int(1))})
   ```
   Numeric constants are untyped in Go, so the actual type of 1 varies by context. In this case, it'll be int. It's a preference to make this explicit.
   
   Alternatively, since you've got reflectx imported, we already have an explict reflect.Type for it that can be used instead.
   
   ```suggestion
   	elementWeightKvs := beam.ParDo(s, weightedElementToKv, in, beam.TypeDefinition{Var: beam.TType, T: reflectx.Int})
   ```
   
   




----------------------------------------------------------------
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] johnedmonds commented on pull request #13889: [BEAM-11611] Add transformation for computing approximate quantiles.

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


   R: @lostluck 


----------------------------------------------------------------
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] johnedmonds commented on a change in pull request #13889: [BEAM-11611] Add transformation for computing approximate quantiles.

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



##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles.go
##########
@@ -0,0 +1,710 @@
+// 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 stats
+
+// Approximate quantiles is implemented based on https://arxiv.org/pdf/1907.00236.pdf.
+
+import (
+	"bytes"
+	"container/heap"
+	"context"
+	"encoding/gob"
+	"encoding/json"
+	"hash/crc32"
+	"io"
+	"math"
+	"reflect"
+	"sort"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+)
+
+func init() {
+	compactorsType := reflect.TypeOf((**compactors)(nil)).Elem()
+	weightedElementType := reflect.TypeOf((*weightedElement)(nil)).Elem()
+	beam.RegisterType(compactorsType)
+	beam.RegisterType(weightedElementType)
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesInputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesMergeOnlyFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesOutputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*shardElementsFn)(nil)).Elem())
+	beam.RegisterCoder(compactorsType, encodeCompactors, decodeCompactors)
+	beam.RegisterCoder(weightedElementType, encodeWeightedElement, decodeWeightedElement)
+}
+
+// Opts contains settings used to configure how approximate quantiles are computed.
+type Opts struct {
+	// Controls the memory used and approximation error (difference between the quantile returned and the true quantile.)
+	K int
+	// Number of quantiles to return. The algorithm will return NumQuantiles - 1 numbers
+	NumQuantiles int
+	// For extremely large datasets, runners may have issues with out of memory errors or taking too long to finish.
+	// If ApproximateQuantiles is failing, you can use this option to tune how the data is sharded internally.
+	// This parameter is optional. If unspecified, Beam will compact all elements into a single compactor at once using a single machine.

Review comment:
       I've added an example. Hopefully this makes sense.




----------------------------------------------------------------
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 #13889: [BEAM-11611] Add transformation for computing approximate quantiles.

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



##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles.go
##########
@@ -0,0 +1,703 @@
+// 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 stats contains transforms for statistical processing.

Review comment:
       Only one file in a package must have a description comment, for this package it's count.go. Please remove.




----------------------------------------------------------------
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 #13889: [BEAM-11611] Add transformation for computing approximate quantiles.

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



##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles.go
##########
@@ -0,0 +1,703 @@
+// 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 stats contains transforms for statistical processing.

Review comment:
       Only one file in a package must have a description comment, for this package it's count.go. Please remove.




----------------------------------------------------------------
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 #13889: [BEAM-11611] Add transformation for computing approximate quantiles.

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



##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles.go
##########
@@ -0,0 +1,710 @@
+// 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 stats
+
+// Approximate quantiles is implemented based on https://arxiv.org/pdf/1907.00236.pdf.
+
+import (
+	"bytes"
+	"container/heap"
+	"context"
+	"encoding/gob"
+	"encoding/json"
+	"hash/crc32"
+	"io"
+	"math"
+	"reflect"
+	"sort"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+)
+
+func init() {
+	compactorsType := reflect.TypeOf((**compactors)(nil)).Elem()
+	weightedElementType := reflect.TypeOf((*weightedElement)(nil)).Elem()
+	beam.RegisterType(compactorsType)
+	beam.RegisterType(weightedElementType)
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesInputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesMergeOnlyFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesOutputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*shardElementsFn)(nil)).Elem())
+	beam.RegisterCoder(compactorsType, encodeCompactors, decodeCompactors)
+	beam.RegisterCoder(weightedElementType, encodeWeightedElement, decodeWeightedElement)
+}
+
+// Opts contains settings used to configure how approximate quantiles are computed.
+type Opts struct {
+	// Controls the memory used and approximation error (difference between the quantile returned and the true quantile.)
+	K int
+	// Number of quantiles to return. The algorithm will return NumQuantiles - 1 numbers
+	NumQuantiles int
+	// For extremely large datasets, runners may have issues with out of memory errors or taking too long to finish.
+	// If ApproximateQuantiles is failing, you can use this option to tune how the data is sharded internally.
+	// This parameter is optional. If unspecified, Beam will compact all elements into a single compactor at once using a single machine.

Review comment:
       I think the only issue I have with this comment is that it doesn't have an intuitive use. Consider explaining or have a small example of what a given slice will do to change the behavior of the DoFns.




----------------------------------------------------------------
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 #13889: [BEAM-11611] Add transformation for computing approximate quantiles.

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



##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles_test.go
##########
@@ -0,0 +1,274 @@
+// 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 stats
+
+import (
+	"reflect"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/passert"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/ptest"
+	"github.com/google/go-cmp/cmp"
+)
+
+func init() {
+	beam.RegisterFunction(weightedElementToKv)
+
+	// In practice, this runs faster than plain reflection.
+	reflectx.RegisterFunc(reflect.ValueOf(less).Type(), func(_ interface{}) reflectx.Func {
+		return newIntLess()
+	})
+}
+
+type intLess struct {
+	name string
+	t    reflect.Type
+}
+
+func newIntLess() *intLess {
+	return &intLess{
+		name: reflectx.FunctionName(reflect.ValueOf(less).Interface()),
+		t:    reflect.ValueOf(less).Type(),
+	}
+}
+
+func (i *intLess) Name() string {
+	return i.name
+}
+func (i *intLess) Type() reflect.Type {
+	return i.t
+}
+func (i *intLess) Call(args []interface{}) []interface{} {
+	return []interface{}{args[0].(int) < args[1].(int)}
+}
+
+func less(a, b int) bool {
+	return a < b
+}
+
+func TestLargeQuantiles(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := 0; i < numElements; i++ {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{10006, 19973}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestLargeQuantilesReversed(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := numElements - 1; i >= 0; i-- {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{9985, 19959}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestBasicQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	tests := []struct {
+		Input    []int
+		Expected [][]int
+	}{
+		{[]int{}, [][]int{}},
+		{[]int{1}, [][]int{[]int{1}}},
+		{[]int{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20}, [][]int{[]int{6, 13}}},
+	}
+
+	for _, test := range tests {
+		p, s, in, exp := ptest.CreateList2(test.Input, test.Expected)
+		quantiles := ApproximateQuantiles(s, in, less, Opts{
+			K:            3,
+			NumQuantiles: 3,
+		})
+		passert.Equals(s, quantiles, exp)
+
+		if err := ptest.Run(p); err != nil {
+			t.Errorf("ApproximateQuantiles(%v) != %v: %v", test.Input, test.Expected, err)
+		}
+	}
+}
+
+func weightedElementToKv(e testWeightedElement) (int, int) {
+	return e.Weight, e.Element
+}
+
+type testWeightedElement struct {
+	Weight  int
+	Element int
+}
+
+func TestWeightedQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	input := []testWeightedElement{
+		{Weight: 1, Element: 1},
+		{Weight: 10, Element: 2},
+		{Weight: 1, Element: 3},
+		{Weight: 10, Element: 4}}
+	expected := []int{2, 4}
+	p, s, in, exp := ptest.CreateList2(input, [][]int{expected})
+	elementWeightKvs := beam.ParDo(s, weightedElementToKv, in, beam.TypeDefinition{Var: beam.TType, T: reflect.TypeOf(1)})

Review comment:
       ```suggestion
   	elementWeightKvs := beam.ParDo(s, weightedElementToKv, in, beam.TypeDefinition{Var: beam.TType, T: reflect.TypeOf(int(1))})
   ```
   Numeric constants are untyped in Go, so the actual type of 1 varies by context. In this case, it'll be int. It's a preference to make this explicit.
   
   Alternatively, since you've got reflectx imported, we already have an explict reflect.Type for it that can be used instead.
   
   ```suggestion
   	elementWeightKvs := beam.ParDo(s, weightedElementToKv, in, beam.TypeDefinition{Var: beam.TType, T: reflectx.Int})
   ```
   
   Here and below too.




----------------------------------------------------------------
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] johnedmonds commented on pull request #13889: [BEAM-11611] Add transformation for computing approximate quantiles.

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


   Thanks for the acknowledgement 👍 and looking forward to your review. Let me know if there's anything I can do to help.


----------------------------------------------------------------
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] johnedmonds commented on a change in pull request #13889: [BEAM-11611] Add transformation for computing approximate quantiles.

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



##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles.go
##########
@@ -0,0 +1,702 @@
+// 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 stats
+
+// Approximate quantiles is implemented based on https://arxiv.org/pdf/1907.00236.pdf.
+
+import (
+	"bytes"
+	"container/heap"
+	"context"
+	"encoding/gob"
+	"encoding/json"
+	"hash/crc32"
+	"io"
+	"math"
+	"reflect"
+	"sort"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+)
+
+func init() {
+	compactorsType := reflect.TypeOf((**compactors)(nil)).Elem()
+	weightedElementType := reflect.TypeOf((*weightedElement)(nil)).Elem()
+	beam.RegisterType(compactorsType)
+	beam.RegisterType(weightedElementType)
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesInputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesMergeOnlyFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesOutputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*shardElementsFn)(nil)).Elem())
+	beam.RegisterCoder(compactorsType, encodeCompactors, decodeCompactors)
+	beam.RegisterCoder(weightedElementType, encodeWeightedElement, decodeWeightedElement)
+}
+
+// Opts contains settings used to configure how approximate quantiles are computed.
+type Opts struct {
+	// Controls the memory used and approximation error (difference between the quantile returned and the true quantile.)
+	K int
+	// Number of quantiles to return. The algorithm will return NumQuantiles - 1 numbers
+	NumQuantiles int
+	// For extremely large datasets, the Go SDK has some problems holding on to data and might OOM.
+	// If ApproximateQuantiles is running out of memory, you can use this option to tune how the data is sharded internally.
+	// This parameter is optional. If unspecified, Beam will compact all elements into a single compactor at once.
+	InternalSharding []int
+}
+
+// The paper suggests reducing the size of the lower-level compactors as we grow.
+// We reduce the capacity at this rate.
+// The paper suggests 1/sqrt(2) is ideal. That's approximately 0.7.
+const capacityCoefficient float64 = 0.7
+
+type sortListHeap struct {
+	data [][]beam.T
+	less reflectx.Func2x1
+}
+
+func (s sortListHeap) Len() int            { return len(s.data) }
+func (s sortListHeap) Less(i, j int) bool  { return s.less.Call2x1(s.data[i][0], s.data[j][0]).(bool) }
+func (s sortListHeap) Swap(i, j int)       { s.data[i], s.data[j] = s.data[j], s.data[i] }
+func (s *sortListHeap) Push(x interface{}) { s.data = append(s.data, x.([]beam.T)) }
+func (s *sortListHeap) Pop() interface{} {
+	var x beam.T
+	x, s.data = s.data[len(s.data)-1], s.data[:len(s.data)-1]
+	return x
+}
+
+// compactor contains elements to be compacted.
+type compactor struct {
+	// Compaction needs to sort elements before compacting. Thus in practice, we should often have some pre-sorted data.
+	// We want to keep it separate so we can sort only the unsorted data and merge the two sorted lists together.
+	// If we're only receiving elements of weight 1, only level 0 will ever contain unsorted data and the rest of the levels will always remain sorted.
+	// To prevent repeated allocation/copying, we keep multiple sorted lists and then merge them together
+	sorted   [][]beam.T
+	unsorted []beam.T
+	// How many items should be stored in this compactor before it should get compacted.
+	// Note that this is not a hard limit.
+	// The paper suggests implementing lazy compaction which would allow
+	// compactors to temporarily exceed their capacity as long as the total
+	// elements in all compactors doesn't exceed the total capacity in all
+	// compactors. In other words, compactors can temporarily borrow capacity
+	// from each other.
+	// In the paper, this is referred to as the variable k_h.
+	capacity int
+}
+
+type compactorAsGob struct {
+	Sorted            [][]byte
+	Unsorted          []byte
+	EncodedTypeAsJSON []byte
+}
+
+func encodeElements(enc beam.ElementEncoder, elements []beam.T) ([]byte, error) {
+	var buf bytes.Buffer
+	for _, e := range elements {
+		if err := enc.Encode(e, &buf); err != nil {
+			return nil, err
+		}
+	}
+	return buf.Bytes(), nil
+}
+
+func (c *compactor) getElementType() reflect.Type {
+	for _, e := range c.sorted {
+		for _, e2 := range e {
+			return reflect.TypeOf(e2)
+		}
+	}
+	for _, e := range c.unsorted {
+		return reflect.TypeOf(e)
+	}
+	return nil
+}
+
+func (c *compactor) MarshalBinary() ([]byte, error) {
+	t := c.getElementType()
+	var buf bytes.Buffer
+	if t == nil {
+		enc := gob.NewEncoder(&buf)
+		if err := enc.Encode(compactorAsGob{}); err != nil {
+			return nil, err
+		}
+		return buf.Bytes(), nil
+	}
+	enc := beam.NewElementEncoder(t)
+	encodedSorted := make([][]byte, 0, len(c.sorted))
+	for _, sorted := range c.sorted {
+		encoded, err := encodeElements(enc, sorted)
+		if err != nil {
+			return nil, err
+		}
+		encodedSorted = append(encodedSorted, encoded)
+	}
+	encodedUnsorted, err := encodeElements(enc, c.unsorted)
+	if err != nil {
+		return nil, err
+	}
+	tAsJSON, err := beam.EncodedType{T: t}.MarshalJSON()
+	if err != nil {
+		return nil, err
+	}
+	gobEnc := gob.NewEncoder(&buf)
+	if err = gobEnc.Encode(compactorAsGob{
+		Sorted:            encodedSorted,
+		Unsorted:          encodedUnsorted,
+		EncodedTypeAsJSON: tAsJSON,
+	}); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func decodeElements(dec beam.ElementDecoder, data []byte) ([]beam.T, error) {
+	buf := bytes.NewBuffer(data)
+	ret := []beam.T{}

Review comment:
       Good point. I've added the size as part of the Gob encoding. 

##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles.go
##########
@@ -0,0 +1,702 @@
+// 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 stats
+
+// Approximate quantiles is implemented based on https://arxiv.org/pdf/1907.00236.pdf.
+
+import (
+	"bytes"
+	"container/heap"
+	"context"
+	"encoding/gob"
+	"encoding/json"
+	"hash/crc32"
+	"io"
+	"math"
+	"reflect"
+	"sort"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+)
+
+func init() {
+	compactorsType := reflect.TypeOf((**compactors)(nil)).Elem()
+	weightedElementType := reflect.TypeOf((*weightedElement)(nil)).Elem()
+	beam.RegisterType(compactorsType)
+	beam.RegisterType(weightedElementType)
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesInputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesMergeOnlyFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesOutputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*shardElementsFn)(nil)).Elem())
+	beam.RegisterCoder(compactorsType, encodeCompactors, decodeCompactors)
+	beam.RegisterCoder(weightedElementType, encodeWeightedElement, decodeWeightedElement)
+}
+
+// Opts contains settings used to configure how approximate quantiles are computed.
+type Opts struct {
+	// Controls the memory used and approximation error (difference between the quantile returned and the true quantile.)
+	K int
+	// Number of quantiles to return. The algorithm will return NumQuantiles - 1 numbers
+	NumQuantiles int
+	// For extremely large datasets, the Go SDK has some problems holding on to data and might OOM.

Review comment:
       Thanks for the explanation. I've learned a lot!
   
   FYI, my use case was attempting to shard leveldb keys. My elements are approximately 1kb-1mb byte strings. I suspect that on smaller machines (e.g. something like n1-standard-1) we run into memory problems. ([FixedReStream](https://github.com/apache/beam/blob/6edc3338667957548216fb9db892bc6c8bcf55b2/sdks/go/pkg/beam/core/runtime/exec/fullvalue.go#L62) has caught my suspicion but I haven't fully confirmed it.) On larger machines (e.g. something like n1-highmem-2) we run into speed problems. I've noticed that for large datasets, the work gets canceled on one machine and started on another after maybe 10-20 minutes. As far as I can tell, giving the option to manually shard seems to solve both memory and speed/timeout problems.
   
   I've tried to rewrite this comment to be more vague about the exact cause and focus more on the symptoms. Hopefully that will make it easier for people running into memory or speed issues to understand how to use this option.

##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles_test.go
##########
@@ -0,0 +1,291 @@
+// 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 stats
+
+import (
+	"reflect"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/passert"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/ptest"
+)
+
+func init() {
+	beam.RegisterFunction(weightedElementToKv)
+
+	// In practice, this runs faster than plain reflection.

Review comment:
       Does that mean I could remove `intLess` (the struct) and the reflection optimizations should "just work" because starcgen already runs on the stats package?

##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles.go
##########
@@ -0,0 +1,702 @@
+// 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 stats
+
+// Approximate quantiles is implemented based on https://arxiv.org/pdf/1907.00236.pdf.
+
+import (
+	"bytes"
+	"container/heap"
+	"context"
+	"encoding/gob"
+	"encoding/json"
+	"hash/crc32"
+	"io"
+	"math"
+	"reflect"
+	"sort"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+)
+
+func init() {
+	compactorsType := reflect.TypeOf((**compactors)(nil)).Elem()
+	weightedElementType := reflect.TypeOf((*weightedElement)(nil)).Elem()
+	beam.RegisterType(compactorsType)
+	beam.RegisterType(weightedElementType)
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesInputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesMergeOnlyFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*approximateQuantilesOutputFn)(nil)).Elem())
+	beam.RegisterType(reflect.TypeOf((*shardElementsFn)(nil)).Elem())
+	beam.RegisterCoder(compactorsType, encodeCompactors, decodeCompactors)
+	beam.RegisterCoder(weightedElementType, encodeWeightedElement, decodeWeightedElement)
+}
+
+// Opts contains settings used to configure how approximate quantiles are computed.
+type Opts struct {
+	// Controls the memory used and approximation error (difference between the quantile returned and the true quantile.)
+	K int
+	// Number of quantiles to return. The algorithm will return NumQuantiles - 1 numbers
+	NumQuantiles int
+	// For extremely large datasets, the Go SDK has some problems holding on to data and might OOM.
+	// If ApproximateQuantiles is running out of memory, you can use this option to tune how the data is sharded internally.
+	// This parameter is optional. If unspecified, Beam will compact all elements into a single compactor at once.
+	InternalSharding []int
+}
+
+// The paper suggests reducing the size of the lower-level compactors as we grow.
+// We reduce the capacity at this rate.
+// The paper suggests 1/sqrt(2) is ideal. That's approximately 0.7.
+const capacityCoefficient float64 = 0.7
+
+type sortListHeap struct {
+	data [][]beam.T
+	less reflectx.Func2x1
+}
+
+func (s sortListHeap) Len() int            { return len(s.data) }
+func (s sortListHeap) Less(i, j int) bool  { return s.less.Call2x1(s.data[i][0], s.data[j][0]).(bool) }
+func (s sortListHeap) Swap(i, j int)       { s.data[i], s.data[j] = s.data[j], s.data[i] }
+func (s *sortListHeap) Push(x interface{}) { s.data = append(s.data, x.([]beam.T)) }
+func (s *sortListHeap) Pop() interface{} {
+	var x beam.T
+	x, s.data = s.data[len(s.data)-1], s.data[:len(s.data)-1]
+	return x
+}
+
+// compactor contains elements to be compacted.
+type compactor struct {
+	// Compaction needs to sort elements before compacting. Thus in practice, we should often have some pre-sorted data.
+	// We want to keep it separate so we can sort only the unsorted data and merge the two sorted lists together.
+	// If we're only receiving elements of weight 1, only level 0 will ever contain unsorted data and the rest of the levels will always remain sorted.
+	// To prevent repeated allocation/copying, we keep multiple sorted lists and then merge them together
+	sorted   [][]beam.T
+	unsorted []beam.T
+	// How many items should be stored in this compactor before it should get compacted.
+	// Note that this is not a hard limit.
+	// The paper suggests implementing lazy compaction which would allow
+	// compactors to temporarily exceed their capacity as long as the total
+	// elements in all compactors doesn't exceed the total capacity in all
+	// compactors. In other words, compactors can temporarily borrow capacity
+	// from each other.
+	// In the paper, this is referred to as the variable k_h.
+	capacity int
+}
+
+type compactorAsGob struct {
+	Sorted            [][]byte
+	Unsorted          []byte
+	EncodedTypeAsJSON []byte
+}
+
+func encodeElements(enc beam.ElementEncoder, elements []beam.T) ([]byte, error) {
+	var buf bytes.Buffer
+	for _, e := range elements {
+		if err := enc.Encode(e, &buf); err != nil {
+			return nil, err
+		}
+	}
+	return buf.Bytes(), nil
+}
+
+func (c *compactor) getElementType() reflect.Type {
+	for _, e := range c.sorted {
+		for _, e2 := range e {
+			return reflect.TypeOf(e2)
+		}
+	}
+	for _, e := range c.unsorted {
+		return reflect.TypeOf(e)
+	}
+	return nil
+}
+
+func (c *compactor) MarshalBinary() ([]byte, error) {
+	t := c.getElementType()
+	var buf bytes.Buffer
+	if t == nil {
+		enc := gob.NewEncoder(&buf)
+		if err := enc.Encode(compactorAsGob{}); err != nil {
+			return nil, err
+		}
+		return buf.Bytes(), nil
+	}
+	enc := beam.NewElementEncoder(t)
+	encodedSorted := make([][]byte, 0, len(c.sorted))
+	for _, sorted := range c.sorted {
+		encoded, err := encodeElements(enc, sorted)
+		if err != nil {
+			return nil, err
+		}
+		encodedSorted = append(encodedSorted, encoded)
+	}
+	encodedUnsorted, err := encodeElements(enc, c.unsorted)
+	if err != nil {
+		return nil, err
+	}
+	tAsJSON, err := beam.EncodedType{T: t}.MarshalJSON()
+	if err != nil {
+		return nil, err
+	}
+	gobEnc := gob.NewEncoder(&buf)
+	if err = gobEnc.Encode(compactorAsGob{
+		Sorted:            encodedSorted,
+		Unsorted:          encodedUnsorted,
+		EncodedTypeAsJSON: tAsJSON,
+	}); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func decodeElements(dec beam.ElementDecoder, data []byte) ([]beam.T, error) {
+	buf := bytes.NewBuffer(data)
+	ret := []beam.T{}
+	for {
+		element, err := dec.Decode(buf)
+		if err == io.EOF {
+			return ret, nil
+		} else if err != nil {
+			return nil, err
+		}
+		ret = append(ret, element)
+	}
+}
+
+func (c *compactor) UnmarshalBinary(data []byte) error {
+	var g compactorAsGob
+	var err error
+	gobDec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err = gobDec.Decode(&g); err != nil {
+		return err
+	}
+	if len(g.EncodedTypeAsJSON) == 0 {
+		return nil
+	}
+	var t beam.EncodedType
+	if err = json.Unmarshal(g.EncodedTypeAsJSON, &t); err != nil {
+		return err
+	}
+	dec := beam.NewElementDecoder(t.T)
+	decodedSorted := make([][]beam.T, 0, len(g.Sorted))
+	for _, sorted := range g.Sorted {
+		decoded, err := decodeElements(dec, sorted)
+		if err != nil {
+			return err
+		}
+		decodedSorted = append(decodedSorted, decoded)
+	}
+	c.sorted = decodedSorted
+	if c.unsorted, err = decodeElements(dec, g.Unsorted); err != nil {
+		return err
+	}
+	return nil
+}
+
+// update inserts an element into the compactor.
+func (c *compactor) update(element beam.T) {
+	c.unsorted = append(c.unsorted, element)
+}
+
+// size returns the number of elements stored in this compactor.
+func (c *compactor) size() int {
+	size := 0
+	for _, s := range c.sorted {
+		size += len(s)
+	}
+	return len(c.unsorted) + size
+}
+
+type sorter struct {
+	less reflectx.Func2x1
+	data []beam.T
+}
+
+func (s sorter) Len() int           { return len(s.data) }
+func (s sorter) Less(i, j int) bool { return s.less.Call2x1(s.data[i], s.data[j]).(bool) }
+func (s sorter) Swap(i, j int)      { s.data[i], s.data[j] = s.data[j], s.data[i] }
+
+// sort sorts the compactor and returns all the elements in sorted order.
+func (c *compactor) sort(less reflectx.Func2x1) []beam.T {
+	sort.Sort(sorter{data: c.unsorted, less: less})
+	h := sortListHeap{data: c.sorted, less: less}
+	heap.Init(&h)
+	sorted := make([]beam.T, 0, c.size()-len(c.unsorted))
+	for h.Len() > 0 {
+		s := heap.Pop(&h).([]beam.T)
+		sorted = append(sorted, s[0])
+		if len(s) > 1 {
+			heap.Push(&h, s[1:])
+		}
+	}
+	c.sorted = [][]beam.T{mergeSorted(sorted, c.unsorted, func(a, b interface{}) bool { return less.Call2x1(a, b).(bool) })}
+	c.unsorted = nil
+	if len(c.sorted[0]) == 0 {
+		c.sorted = nil
+		return nil
+	}
+	return c.sorted[0]
+}
+
+// Compactors holds the state of the quantile approximation compactors.
+type compactors struct {
+	// References "K" from the paper which influences the amount of memory used.
+	K int
+	// When compacting, we want to alternate between taking elements at even vs odd indices.
+	// The paper suggests using a random variable but we'd prefer to stay deterministic.
+	// Especially when merging two compactors we want to keep track of how often we've selected odds vs evens.
+	NumberOfCompactions int
+
+	// Each compactor takes a sample of elements.
+	// The "height" (also known as the index in this slice) of the compactor determines the weight of its elements.
+	// The weight of a compactor of height h is 2^h.
+	// For example, for h = 3 (which would be compactors[3]), the weight is 2^3 = 8. That means each element in that compactor represents 8 instances of itself.
+	Compactors []compactor
+}
+
+func (c *compactors) totalCapacity() int {
+	totalCapacity := 0
+	for _, compactor := range c.Compactors {
+		totalCapacity += compactor.capacity
+	}
+	return totalCapacity
+}
+
+func (c *compactors) size() int {
+	size := 0
+	for _, compactor := range c.Compactors {
+		size += compactor.size()
+	}
+	return size
+}
+
+// capacity computes the capacity of a compactor at a certain level.
+// The paper suggests decreasing the capacity of lower-leveled compactors as we add more elements.
+func (c *compactors) capacity(compactorLevel int) int {
+	return int(math.Ceil(math.Pow(capacityCoefficient, float64(len(c.Compactors)-compactorLevel-1))*float64(c.K))) + 1
+}
+
+// compact compacts all compactors until the total size is less than the maximum capacity of all compactors.
+func (c *compactors) compact(less reflectx.Func2x1) {
+	for c.size() > c.totalCapacity() {
+		for level, compactor := range c.Compactors {
+			if compactor.size() > compactor.capacity {
+				c.compactLevel(level, less)
+				// Merging compactors can cause us to exceed max capacity in multiple compactors.
+				if c.size() < c.totalCapacity() {
+					// Do lazy compaction as described in the paper.
+					break
+				}
+			}
+		}
+	}
+}
+
+// update inserts the given element into the compactors. If this element causes the compactors to grow too large, we perform the compaction here.
+func (c *compactors) update(element beam.T, weight int, less reflectx.Func2x1) {
+	level := int(math.Log2(float64(weight)))
+	c.growToIncludeLevel(level)
+	c.Compactors[level].update(element)
+	// From the paper, we're using the "Splitting the Input" approach.
+	remainingWeight := weight - (1 << uint(level))
+	// Only attempt compaction if we're doing the last update. Otherwise we'd be compacting too often.
+	if remainingWeight <= 0 {
+		c.compact(less)
+	} else {
+		c.update(element, remainingWeight, less)
+	}
+}
+
+// growToIncludeLevel ensures we have compactors available at the given level.
+func (c *compactors) growToIncludeLevel(level int) {
+	if len(c.Compactors)-1 >= level {
+		return
+	}
+	for i := len(c.Compactors) - 1; i < level; i++ {
+		c.Compactors = append(c.Compactors, compactor{})
+	}
+	for level := range c.Compactors {
+		c.Compactors[level].capacity = c.capacity(level)
+	}
+}
+
+// compact compacts elements in compactors.
+func (c *compactors) compactLevel(level int, less reflectx.Func2x1) {
+	c.growToIncludeLevel(level + 1)
+	jitterIndex := 0
+	// Create a temporary buffer to hold the compacted elements.
+	// Buffering the elements like this makes it easier to call mergeSorted.
+	compacted := make([]beam.T, 0, c.Compactors[level].size()/2)
+	selectEvens := c.NumberOfCompactions%2 == 0
+	c.NumberOfCompactions++
+	for _, element := range c.Compactors[level].sort(less) {
+		if (jitterIndex%2 == 0) == selectEvens {
+			compacted = append(compacted, element)
+		}
+		jitterIndex++
+	}
+	if len(compacted) > 0 {
+		c.Compactors[level+1].sorted = append(c.Compactors[level+1].sorted, compacted)
+	}
+	// Clear out the compactor at this level since we've finished compacting it. The compacted elements have already been moved to the next compactor.
+	c.Compactors[level].sorted = nil
+	c.Compactors[level].unsorted = nil
+}
+
+func encodeCompactors(c *compactors) ([]byte, error) {
+	var buf bytes.Buffer
+	enc := gob.NewEncoder(&buf)
+	if err := enc.Encode(c); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func decodeCompactors(data []byte) (*compactors, error) {
+	var compactors compactors
+	dec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err := dec.Decode(&compactors); err != nil {
+		return nil, err
+	}
+	for level := range compactors.Compactors {
+		compactors.Compactors[level].capacity = compactors.capacity(level)
+	}
+	return &compactors, nil
+}
+
+// mergeSorted takes two slices which are already sorted and returns a new slice containing all elements sorted together.
+func mergeSorted(a, b []beam.T, less func(interface{}, interface{}) bool) []beam.T {
+	output := make([]beam.T, 0, len(a)+len(b))
+	for len(a) > 0 && len(b) > 0 {
+		if less(a[0], b[0]) {
+			output = append(output, a[0])
+			a = a[1:]
+		} else {
+			output = append(output, b[0])
+			b = b[1:]
+		}
+	}
+	if len(a) > 0 {
+		output = append(output, a...)
+	} else {
+		output = append(output, b...)
+	}
+	return output
+}
+
+// mergeSortedWeighted takes two slices which are already sorted and returns a new slice containing all elements sorted together.
+func mergeSortedWeighted(a, b []weightedElement, less func(interface{}, interface{}) bool) []weightedElement {
+	output := make([]weightedElement, 0, len(a)+len(b))
+	for len(a) > 0 && len(b) > 0 {
+		if less(a[0], b[0]) {
+			output = append(output, a[0])
+			a = a[1:]
+		} else {
+			output = append(output, b[0])
+			b = b[1:]
+		}
+	}
+	if len(a) > 0 {
+		output = append(output, a...)
+	} else {
+		output = append(output, b...)
+	}
+	return output
+}
+
+// merge joins two compactors together.
+func (c *compactors) merge(other *compactors, less reflectx.Func2x1) {
+	for level := range c.Compactors {
+		if len(other.Compactors)-1 < level {
+			break
+		}
+		c.Compactors[level].unsorted = append(c.Compactors[level].unsorted, other.Compactors[level].unsorted...)
+		c.Compactors[level].sorted = append(c.Compactors[level].sorted, other.Compactors[level].sorted...)
+	}
+	if len(other.Compactors) > len(c.Compactors) {
+		c.Compactors = append(c.Compactors, other.Compactors[len(c.Compactors):]...)
+	}
+	c.NumberOfCompactions += other.NumberOfCompactions
+	c.compact(less)
+}
+
+// approximateQuantilesCombineFnState contains the payload for the combiners.
+// Ideally this would be a single combine function but in certain cases, runners attempts to do all the merges on a single machine.
+// Unfortunately the merges can be slow for extremely large datasets and large values of K. If the merge takes too long, it will get canceled and the job will never complete.
+// Thus we split up the combiners into multiple functions to force the runner to do the work in parallel.
+// This state can be shared across all of the split-up functions.
+type approximateQuantilesCombineFnState struct {
+	// The size of the compactors.
+	// The memory consumed, and the error are controlled by this parameter.
+	K int `json:"k"`
+	// Used to compare elements.
+	LessFunc beam.EncodedFunc
+	// Internally cached instance.
+	less         reflectx.Func2x1
+	NumQuantiles int `json:"numQuantiles"`
+}
+
+func (f *approximateQuantilesCombineFnState) setup() error {
+	f.less = reflectx.ToFunc2x1(f.LessFunc.Fn)
+	return nil
+}
+
+func (f *approximateQuantilesCombineFnState) createAccumulator() *compactors {
+	return &compactors{
+		K:          f.K,
+		Compactors: []compactor{{capacity: f.K}},
+	}
+}
+
+// approximateQuantilesOutputFn extracts the final output containing the quantiles.
+type approximateQuantilesOutputFn struct {
+	State approximateQuantilesCombineFnState `json:"state"`
+}
+
+func (f *approximateQuantilesOutputFn) Setup() error {
+	return f.State.setup()
+}
+
+func (f *approximateQuantilesOutputFn) CreateAccumulator() *compactors {
+	return f.State.createAccumulator()
+}
+
+func (f *approximateQuantilesOutputFn) AddInput(compactors *compactors, element *compactors) *compactors {
+	compactors.merge(element, f.State.less)
+	return compactors
+}
+
+func (f *approximateQuantilesOutputFn) MergeAccumulators(ctx context.Context, a, b *compactors) *compactors {
+	a.merge(b, f.State.less)
+	return a
+}
+
+type weightedElementAsGob struct {
+	EncodedTypeAsJSON []byte
+	Weight            int
+	Element           []byte
+}
+
+func encodeWeightedElement(element weightedElement) ([]byte, error) {
+	t := reflect.TypeOf(element.element)
+	enc := beam.NewElementEncoder(t)
+	var buf bytes.Buffer
+	if err := enc.Encode(element.element, &buf); err != nil {
+		return nil, err
+	}
+	tAsJSON, err := beam.EncodedType{T: t}.MarshalJSON()
+	if err != nil {
+		return nil, err
+	}
+	var gobBuf bytes.Buffer
+	if err := gob.NewEncoder(&gobBuf).Encode(weightedElementAsGob{
+		EncodedTypeAsJSON: tAsJSON,
+		Weight:            element.weight,
+		Element:           buf.Bytes(),
+	}); err != nil {
+		return nil, err
+	}
+	return gobBuf.Bytes(), nil
+}
+
+func decodeWeightedElement(data []byte) (weightedElement, error) {
+	var g weightedElementAsGob
+	dec := gob.NewDecoder(bytes.NewBuffer(data))
+	if err := dec.Decode(&g); err != nil {
+		return weightedElement{}, err
+	}
+	var t beam.EncodedType
+	if err := t.UnmarshalJSON(g.EncodedTypeAsJSON); err != nil {
+		return weightedElement{}, err
+	}
+	element, err := beam.NewElementDecoder(t.T).Decode(bytes.NewBuffer(g.Element))
+	if err != nil {
+		return weightedElement{}, err
+	}
+	return weightedElement{
+		weight:  g.Weight,
+		element: element,
+	}, nil
+}
+
+type weightedElement struct {
+	weight  int
+	element beam.T
+}
+
+func toWeightedSlice(compactor compactor, less reflectx.Func2x1, weight int) []weightedElement {
+	sorted := compactor.sort(less)
+	weightedElements := make([]weightedElement, 0, len(sorted))
+	for _, element := range sorted {
+		weightedElements = append(weightedElements, weightedElement{weight: weight, element: element})
+	}
+	return weightedElements
+}
+func (f *approximateQuantilesOutputFn) ExtractOutput(ctx context.Context, compactors *compactors) []beam.T {
+	sorted := toWeightedSlice(compactors.Compactors[0], f.State.less, 1)
+	for level, compactor := range compactors.Compactors[1:] {
+		sorted = mergeSortedWeighted(sorted, toWeightedSlice(compactor, f.State.less, 1<<uint(level)), func(a, b interface{}) bool {
+			return f.State.less.Call2x1(a.(weightedElement).element, b.(weightedElement).element).(bool)
+		})
+	}
+	totalElements := 0
+	for _, element := range sorted {
+		totalElements += element.weight
+	}
+	ret := make([]beam.T, 0, f.State.NumQuantiles)
+	// Rank represents the estimate of how many elements we've seen as we iterate through the sorted list of elements stored in the compactors.
+	// Recall that each element stored in a compactor is also assigned a weight indicating how many elements from the input it represents.
+	rank := float64(0)
+	// Represents the quantile we're currently searching for.
+	currentQuantile := float64(1)
+	for _, element := range sorted {
+		rank += float64(element.weight)
+		if rank/float64(totalElements) >= currentQuantile/float64(f.State.NumQuantiles) {
+			ret = append(ret, element.element)
+			currentQuantile++
+		}
+		if currentQuantile >= float64(f.State.NumQuantiles) {
+			break
+		}
+	}
+	return ret
+}
+
+// approximateQuantilesInputFn combines elements into compactors, but not necessarily the final compactor.
+type approximateQuantilesInputFn approximateQuantilesOutputFn
+
+func (f *approximateQuantilesInputFn) Setup() error {
+	return f.State.setup()
+}
+
+func (f *approximateQuantilesInputFn) CreateAccumulator() *compactors {
+	return f.State.createAccumulator()
+}
+
+func (f *approximateQuantilesInputFn) AddInput(compactors *compactors, element weightedElement) *compactors {
+	compactors.update(element.element, element.weight, f.State.less)
+	return compactors
+}
+
+func (f *approximateQuantilesInputFn) MergeAccumulators(ctx context.Context, a, b *compactors) *compactors {
+	a.merge(b, f.State.less)
+	return a
+}
+
+func (f *approximateQuantilesInputFn) ExtractOutput(ctx context.Context, compactors *compactors) *compactors {
+	for i := range compactors.Compactors {
+		// Sort the compactors here so when we're merging them for the final output, they're already sorted and we can merge elements in order.
+		compactors.Compactors[i].sort(f.State.less)
+	}
+	return compactors
+}
+
+// approximateQuantilesMergeOnlyFn combines compactors into smaller compactors, but not necessarily the final compactor.
+type approximateQuantilesMergeOnlyFn approximateQuantilesOutputFn
+
+func (f *approximateQuantilesMergeOnlyFn) Setup() error {
+	return f.State.setup()
+}
+
+func (f *approximateQuantilesMergeOnlyFn) CreateAccumulator() *compactors {
+	return f.State.createAccumulator()
+}
+
+func (f *approximateQuantilesMergeOnlyFn) AddInput(compactors *compactors, element *compactors) *compactors {
+	compactors.merge(element, f.State.less)
+	return compactors
+}
+
+func (f *approximateQuantilesMergeOnlyFn) MergeAccumulators(ctx context.Context, a, b *compactors) *compactors {
+	a.merge(b, f.State.less)
+	return a
+}
+
+func (f *approximateQuantilesMergeOnlyFn) ExtractOutput(ctx context.Context, compactors *compactors) *compactors {
+	for i := range compactors.Compactors {
+		// Sort the compactors here so when we're merging them for the final output, they're already sorted and we can merge elements in order.
+		compactors.Compactors[i].sort(f.State.less)
+	}
+	return compactors
+}
+
+type shardElementsFn struct {
+	Shards         int              `json:"shards"`
+	T              beam.EncodedType `json:"t"`
+	elementEncoder beam.ElementEncoder
+}
+
+func (s *shardElementsFn) Setup() {
+	s.elementEncoder = beam.NewElementEncoder(s.T.T)
+}
+
+func (s *shardElementsFn) ProcessElement(element beam.T) (int, beam.T) {
+	h := crc32.NewIEEE()
+	s.elementEncoder.Encode(element, h)
+	return int(h.Sum32()) % s.Shards, element
+}
+
+func makeWeightedElement(weight int, element beam.T) weightedElement {
+	return weightedElement{weight: weight, element: element}
+}
+
+// ApproximateQuantiles computes approximate quantiles for the input PCollection<T>.
+//
+// The output PCollection contains a single element: a list of numQuantiles - 1 elements approximately splitting up the input collection into numQuantiles separate quantiles.
+// For example, if numQuantiles = 2, the returned list would contain a single element such that approximately half of the input would be less than that element and half would be greater.
+func ApproximateQuantiles(s beam.Scope, pc beam.PCollection, less interface{}, opts Opts) beam.PCollection {
+	return ApproximateWeightedQuantiles(s, beam.ParDo(s, func(e beam.T) (int, beam.T) { return 1, e }, pc), less, opts)
+}
+
+// reduce takes a PCollection<weightedElementWrapper> and returns a PCollection<*compactors>. The output PCollection may have at most shardSizes[len(shardSizes) - 1] compactors.
+func reduce(s beam.Scope, weightedElements beam.PCollection, state approximateQuantilesCombineFnState, shardSizes []int) beam.PCollection {
+	if len(shardSizes) == 0 {
+		shardSizes = []int{1}
+	}
+	shardedCompactors := beam.DropKey(s,
+		beam.CombinePerKey(s, &approximateQuantilesInputFn{State: state},
+			beam.ParDo(s, &shardElementsFn{Shards: shardSizes[0], T: beam.EncodedType{T: reflect.TypeOf((*weightedElement)(nil)).Elem()}}, weightedElements)))
+	shardSizes = shardSizes[1:]
+	compactorsType := reflect.TypeOf((**compactors)(nil)).Elem()
+	for _, shardSize := range shardSizes {
+		shardedCompactors = beam.DropKey(s,
+			beam.CombinePerKey(s, &approximateQuantilesMergeOnlyFn{State: state},
+				beam.ParDo(s, &shardElementsFn{Shards: shardSize, T: beam.EncodedType{T: compactorsType}}, shardedCompactors)))
+	}
+	return shardedCompactors
+}
+
+// ApproximateWeightedQuantiles computes approximate quantiles for the input PCollection<(weight int, T)>.
+//
+// The output PCollection contains a single element: a list of numQuantiles - 1 elements approximately splitting up the input collection into numQuantiles separate quantiles.
+// For example, if numQuantiles = 2, the returned list would contain a single element such that approximately half of the input would be less than that element and half would be greater.

Review comment:
       I've updated the comment to be "greater or equal" to indicate that the split points would be approximately in the higher quantile. Although as you can see from the tests, the errors can cause the split points to fall into the lower quantiles too.




----------------------------------------------------------------
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] johnedmonds commented on a change in pull request #13889: [BEAM-11611] Add transformation for computing approximate quantiles.

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



##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles.go
##########
@@ -0,0 +1,703 @@
+// 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 stats contains transforms for statistical processing.

Review comment:
       Done.




----------------------------------------------------------------
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 #13889: [BEAM-11611] Add transformation for computing approximate quantiles.

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



##########
File path: sdks/go/pkg/beam/transforms/stats/quantiles_test.go
##########
@@ -0,0 +1,274 @@
+// 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 stats
+
+import (
+	"reflect"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/util/reflectx"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/passert"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/ptest"
+	"github.com/google/go-cmp/cmp"
+)
+
+func init() {
+	beam.RegisterFunction(weightedElementToKv)
+
+	// In practice, this runs faster than plain reflection.
+	reflectx.RegisterFunc(reflect.ValueOf(less).Type(), func(_ interface{}) reflectx.Func {
+		return newIntLess()
+	})
+}
+
+type intLess struct {
+	name string
+	t    reflect.Type
+}
+
+func newIntLess() *intLess {
+	return &intLess{
+		name: reflectx.FunctionName(reflect.ValueOf(less).Interface()),
+		t:    reflect.ValueOf(less).Type(),
+	}
+}
+
+func (i *intLess) Name() string {
+	return i.name
+}
+func (i *intLess) Type() reflect.Type {
+	return i.t
+}
+func (i *intLess) Call(args []interface{}) []interface{} {
+	return []interface{}{args[0].(int) < args[1].(int)}
+}
+
+func less(a, b int) bool {
+	return a < b
+}
+
+func TestLargeQuantiles(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := 0; i < numElements; i++ {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{10006, 19973}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestLargeQuantilesReversed(t *testing.T) {
+	const numElements int = 30000
+	inputSlice := make([]int, 0, numElements)
+	for i := numElements - 1; i >= 0; i-- {
+		inputSlice = append(inputSlice, i)
+	}
+	p, s, input, expected := ptest.CreateList2(inputSlice, [][]int{[]int{9985, 19959}})
+	quantiles := ApproximateQuantiles(s, input, less, Opts{
+		K:            200,
+		NumQuantiles: 3,
+	})
+	passert.Equals(s, quantiles, expected)
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles failed: %v", err)
+	}
+}
+
+func TestBasicQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	tests := []struct {
+		Input    []int
+		Expected [][]int
+	}{
+		{[]int{}, [][]int{}},
+		{[]int{1}, [][]int{[]int{1}}},
+		{[]int{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20}, [][]int{[]int{6, 13}}},
+	}
+
+	for _, test := range tests {
+		p, s, in, exp := ptest.CreateList2(test.Input, test.Expected)
+		quantiles := ApproximateQuantiles(s, in, less, Opts{
+			K:            3,
+			NumQuantiles: 3,
+		})
+		passert.Equals(s, quantiles, exp)
+
+		if err := ptest.Run(p); err != nil {
+			t.Errorf("ApproximateQuantiles(%v) != %v: %v", test.Input, test.Expected, err)
+		}
+	}
+}
+
+func weightedElementToKv(e testWeightedElement) (int, int) {
+	return e.Weight, e.Element
+}
+
+type testWeightedElement struct {
+	Weight  int
+	Element int
+}
+
+func TestWeightedQuantiles(t *testing.T) {
+	// Test asking for 3 quantiles for k=3.
+	input := []testWeightedElement{
+		{Weight: 1, Element: 1},
+		{Weight: 10, Element: 2},
+		{Weight: 1, Element: 3},
+		{Weight: 10, Element: 4}}
+	expected := []int{2, 4}
+	p, s, in, exp := ptest.CreateList2(input, [][]int{expected})
+	elementWeightKvs := beam.ParDo(s, weightedElementToKv, in, beam.TypeDefinition{Var: beam.TType, T: reflect.TypeOf(1)})
+	opts := Opts{
+		K:            3,
+		NumQuantiles: 3,
+	}
+	quantiles := ApproximateWeightedQuantiles(s, elementWeightKvs, less, opts)
+	passert.Equals(s, quantiles, exp)
+
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles(%v) != %v: %v", input, expected, err)
+	}
+}
+
+func TestWeightedQuantilesWithInternalSharding(t *testing.T) {
+	// Test shard reduction.
+	input := []testWeightedElement{
+		{Weight: 1, Element: 1},
+		{Weight: 10, Element: 2},
+		{Weight: 1, Element: 3},
+		{Weight: 10, Element: 4}}
+	expected := []int{2, 4}
+	p, s, in, exp := ptest.CreateList2(input, [][]int{expected})
+	elementWeightKvs := beam.ParDo(s, weightedElementToKv, in, beam.TypeDefinition{Var: beam.TType, T: reflect.TypeOf(1)})
+	opts := Opts{
+		K:                3,
+		NumQuantiles:     3,
+		InternalSharding: []int{4, 3, 2},
+	}
+	quantiles := ApproximateWeightedQuantiles(s, elementWeightKvs, less, opts)
+	passert.Equals(s, quantiles, exp)
+
+	if err := ptest.Run(p); err != nil {
+		t.Errorf("ApproximateQuantiles(%v) != %v: %v", input, expected, err)
+	}
+}
+
+func TestMerging(t *testing.T) {
+	compactors1 := compactors{
+		K:                   3,
+		NumberOfCompactions: 1,
+		Compactors: []compactor{{
+			sorted:   [][]beam.T{[]beam.T{1}, []beam.T{2}, []beam.T{3}},
+			unsorted: []beam.T{6, 5, 4},
+			capacity: 4,
+		}},
+	}
+
+	compactors2 := compactors{
+		K:                   3,
+		NumberOfCompactions: 1,
+		Compactors: []compactor{
+			{
+				sorted:   [][]beam.T{[]beam.T{7}, []beam.T{8}, []beam.T{9}},
+				unsorted: []beam.T{12, 11, 10},
+				capacity: 4},
+		},
+	}
+
+	compactors1.merge(&compactors2, reflectx.MakeFunc2x1(less))
+
+	expectedCompactors := compactors{
+		K:                   3,
+		NumberOfCompactions: 3,
+		Compactors: []compactor{
+			{capacity: 4},
+			{
+				sorted:   [][]beam.T{[]beam.T{1, 3, 5, 7, 9, 11}},
+				capacity: 4,
+			},
+		},
+	}
+	if d := cmp.Diff(expectedCompactors, compactors1, cmp.AllowUnexported(compactor{})); d != "" {
+		t.Errorf("Failed. Expected %v, was %v, diff: %v", expectedCompactors, compactors1, d)
+	}
+}
+
+func TestCompactorsEncoding(t *testing.T) {
+	compactors := compactors{
+		K:                   3,
+		NumberOfCompactions: 1,
+		Compactors: []compactor{
+			{
+				capacity: 4,
+				sorted:   [][]beam.T{[]beam.T{1, 2}},
+				unsorted: []beam.T{3, 4},
+			},
+			{
+				capacity: 4,
+				sorted:   [][]beam.T{[]beam.T{5, 6}},
+				unsorted: []beam.T{7, 8},
+			},
+		},
+	}
+	compactors.Compactors[0].update(1)
+	data, err := encodeCompactors(&compactors)
+	if err != nil {
+		t.Errorf("Failed to encode, %v", err)
+	}
+	decodedCompactors, err := decodeCompactors(data)
+	if err != nil {
+		t.Errorf("Failed to decode, %v", err)
+	}
+	// We want to use cmp.Diff which makes a distinction between empty and nil slices.
+	// So we need to clean up empty slices to be nil.
+	for i := range decodedCompactors.Compactors {
+		if len(decodedCompactors.Compactors[i].sorted) == 0 {
+			decodedCompactors.Compactors[i].sorted = nil
+		}
+		if len(decodedCompactors.Compactors[i].unsorted) == 0 {
+			decodedCompactors.Compactors[i].unsorted = nil
+		}
+	}
+	if d := cmp.Diff(&compactors, decodedCompactors, cmp.AllowUnexported(compactor{})); d != "" {
+		t.Errorf("Invalid coder. Wanted %v, got %v, diff: %v", &compactors, decodedCompactors, d)
+	}
+}
+
+func TestWeightedElementEncoding(t *testing.T) {
+	w := weightedElement{
+		weight:  10,
+		element: 1,
+	}
+	data, err := encodeWeightedElement(w)
+	if err != nil {
+		t.Errorf("Failed to encode %v", err)
+	}
+	decoded, err := decodeWeightedElement(data)
+	if err != nil {
+		t.Errorf("Failed to decode %v", err)
+	}
+	if !reflect.DeepEqual(w, decoded) {

Review comment:
       Was this deep equal missed or was cmp intentionally not used here?




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