Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
36 commits
Select commit Hold shift + click to select a range
dcdbe16
grpc 1.26 and 1.35 in parallel
suztomo Feb 19, 2021
867c5cb
Enriching README with install intruction
suztomo Feb 20, 2021
21e6bcf
netty versions to match gRPC 1.35
suztomo Feb 20, 2021
74ec8e0
Merge remote-tracking branch 'origin/master' into BEAM-11227_parallel
suztomo Mar 2, 2021
4cf462a
vendored gRPC 1.36.0
suztomo Mar 2, 2021
5ee37dd
vendored gRPC 1.36 part of the build
suztomo Mar 2, 2021
f89cd60
Updating readme
suztomo Mar 2, 2021
a92d463
replaced the usage of 1.26.0 to 1.36.0
suztomo Mar 3, 2021
3f08e25
replaced org.apache.beam.vendor.grpc.v1p26p0 to v1p36p0
suztomo Mar 3, 2021
b8ea472
protobuf-version from latest
suztomo Mar 3, 2021
c1144fc
installVendoredGrpc task
suztomo Mar 3, 2021
5eac9b7
a hack to install vendored gRPC
suztomo Mar 3, 2021
ce1e425
removing new flag for mvn
suztomo Mar 3, 2021
4cdba24
mvn full path and info options
suztomo Mar 3, 2021
5fa1bc1
/home/jenkins/tools/maven/apache-maven-3.5.4/bin as PATH
suztomo Mar 3, 2021
0b1ae99
PATH environment before invoking Gradle
suztomo Mar 4, 2021
d8892b8
sdks/java/testing/tpcds to use mavenLocal repository
suztomo Mar 4, 2021
eeab0de
MAVE_HOME environment variable to specify mvn
suztomo Mar 4, 2021
93e2d39
specifying full path to mvn command when available
suztomo Mar 4, 2021
9c9ff37
removing PATH environment setup
suztomo Mar 4, 2021
baadcee
Merge remote-tracking branch 'origin/master' into BEAM-11227_parallel
suztomo Mar 8, 2021
5e117d2
gradlew.bat for Windows
suztomo Mar 8, 2021
d053d7b
path for Windows build
suztomo Mar 9, 2021
0022d43
mvn.cmd for Windows
suztomo Mar 9, 2021
25b9f0a
removing unnecessary dependencies
suztomo Mar 10, 2021
1a413e0
removed unnecessary dependencies
suztomo Mar 10, 2021
4bc873c
unexcluding io.grpc.testing
suztomo Mar 11, 2021
f5e7273
Merge remote-tracking branch 'origin/master' into BEAM-11227_parallel
suztomo Mar 11, 2021
84622c1
Merge remote-tracking branch 'origin/master' into BEAM-11227_parallel
suztomo Mar 11, 2021
dd2df58
renaming vendored_grpc_1_26_0 to 1.36.0
suztomo Mar 11, 2021
16159a3
Merge remote-tracking branch 'origin/master' into BEAM-11227_parallel
suztomo Mar 12, 2021
3c64ccf
Added explanation on expected linkage errors
suztomo Mar 12, 2021
89cb6fc
Merge remote-tracking branch 'origin/master' into BEAM-11227_parallel
suztomo Mar 15, 2021
c69916e
removed local test code
suztomo Mar 15, 2021
4f5c13c
Adding the test code to confirm SQL Postcommit
suztomo Mar 15, 2021
dc4ac01
removing test hack that runs checks with unpublished artifacts
suztomo Mar 15, 2021
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -626,7 +626,7 @@ class BeamModulePlugin implements Plugin<Project> {
testcontainers_postgresql : "org.testcontainers:postgresql:$testcontainers_version",
testcontainers_gcloud : "org.testcontainers:gcloud:$testcontainers_version",
vendored_bytebuddy_1_10_8 : "org.apache.beam:beam-vendor-bytebuddy-1_10_8:0.1",
vendored_grpc_1_26_0 : "org.apache.beam:beam-vendor-grpc-1_26_0:0.3",
vendored_grpc_1_36_0 : "org.apache.beam:beam-vendor-grpc-1_36_0:0.1",
vendored_guava_26_0_jre : "org.apache.beam:beam-vendor-guava-26_0-jre:0.1",
vendored_calcite_1_20_0 : "org.apache.beam:beam-vendor-calcite-1_20_0:0.1",
woodstox_core_asl : "org.codehaus.woodstox:woodstox-core-asl:4.4.1",
Expand Down Expand Up @@ -1845,10 +1845,10 @@ class BeamModulePlugin implements Plugin<Project> {
archivesBaseName: configuration.archivesBaseName,
automaticModuleName: configuration.automaticModuleName,
shadowJarValidationExcludes: it.shadowJarValidationExcludes,
shadowClosure: GrpcVendoring_1_26_0.shadowClosure() << {
shadowClosure: GrpcVendoring_1_36_0.shadowClosure() << {
// We perform all the code relocations but don't include
// any of the actual dependencies since they will be supplied
// by org.apache.beam:beam-vendor-grpc-v1p26p0:0.1
// by org.apache.beam:beam-vendor-grpc-v1p36p0:0.1
dependencies {
include(dependency { return false })
}
Expand All @@ -1865,14 +1865,14 @@ class BeamModulePlugin implements Plugin<Project> {
project.protobuf {
protoc {
// The artifact spec for the Protobuf Compiler
artifact = "com.google.protobuf:protoc:${GrpcVendoring_1_26_0.protobuf_version}" }
artifact = "com.google.protobuf:protoc:${GrpcVendoring_1_36_0.protobuf_version}" }

// Configure the codegen plugins
plugins {
// An artifact spec for a protoc plugin, with "grpc" as
// the identifier, which can be referred to in the "plugins"
// container of the "generateProtoTasks" closure.
grpc { artifact = "io.grpc:protoc-gen-grpc-java:${GrpcVendoring_1_26_0.grpc_version}" }
grpc { artifact = "io.grpc:protoc-gen-grpc-java:${GrpcVendoring_1_36_0.grpc_version}" }
}

generateProtoTasks {
Expand All @@ -1886,7 +1886,7 @@ class BeamModulePlugin implements Plugin<Project> {
}
}

project.dependencies GrpcVendoring_1_26_0.dependenciesClosure() << { shadow project.ext.library.java.vendored_grpc_1_26_0 }
project.dependencies GrpcVendoring_1_36_0.dependenciesClosure() << { shadow project.ext.library.java.vendored_grpc_1_36_0 }
}

/** ***********************************************************************************************/
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,196 @@
/*
* 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 org.apache.beam.gradle

/**
* Utilities for working with our vendored version of gRPC.
*/
class GrpcVendoring_1_36_0 {

static def guava_version = "30.1-jre"
static def protobuf_version = "3.15.3"
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is the latest available version of protobuf-java. We need a version higher than 3.14.0 otherwise it hits Java 8 incompatible problem in protobuf-java 3.13.0. (comment in the ticket)

static def grpc_version = "1.36.0"
static def gson_version = "2.8.6"
// tcnative version from https://github.com/grpc/grpc-java/blob/master/SECURITY.md#netty
static def netty_version = "4.1.52.Final"
// google-auth-library version from https://search.maven.org/artifact/io.grpc/grpc-auth/1.36.0/jar
static def google_auth_version = "0.22.2"
static def proto_google_common_protos_version = "2.0.1"
static def opencensus_version = "0.28.0"
static def conscrypt_version = "2.5.1"

/** Returns the list of compile time dependencies. */
static List<String> dependencies() {
return [
"com.google.guava:guava:$guava_version",
"com.google.protobuf:protobuf-java:$protobuf_version",
"com.google.protobuf:protobuf-java-util:$protobuf_version",
"com.google.code.gson:gson:$gson_version",
"io.grpc:grpc-auth:$grpc_version",
"io.grpc:grpc-core:$grpc_version",
"io.grpc:grpc-context:$grpc_version",
"io.grpc:grpc-netty:$grpc_version",
"io.grpc:grpc-protobuf:$grpc_version",
"io.grpc:grpc-stub:$grpc_version",
"io.netty:netty-transport-native-epoll:$netty_version",
// tcnative version from https://github.com/grpc/grpc-java/blob/master/SECURITY.md#netty
"io.netty:netty-tcnative-boringssl-static:2.0.34.Final",
"com.google.auth:google-auth-library-credentials:$google_auth_version",
"io.grpc:grpc-testing:$grpc_version",
"com.google.api.grpc:proto-google-common-protos:$proto_google_common_protos_version",
"io.opencensus:opencensus-api:$opencensus_version",
"io.opencensus:opencensus-contrib-grpc-metrics:$opencensus_version",
]
}

/**
* Returns the list of runtime time dependencies that should be exported as runtime
* dependencies within the vendored jar.
*/
static List<String> runtimeDependencies() {
return [
'com.google.errorprone:error_prone_annotations:2.4.0',
'commons-logging:commons-logging:1.2',
'org.apache.logging.log4j:log4j-api:2.6.2',
'org.slf4j:slf4j-api:1.7.30',
// TODO(BEAM-9288): Enable relocation for conscrypt
"org.conscrypt:conscrypt-openjdk-uber:$conscrypt_version"
]
}

/**
* Returns the list of test dependencies.
*/
static List<String> testDependencies() {
return [
'junit:junit:4.12',
]
}

static Map<String, String> relocations() {
// The relocation paths below specifically use gRPC and the full version string as
// the code relocation prefix. See https://lists.apache.org/thread.html/4c12db35b40a6d56e170cd6fc8bb0ac4c43a99aa3cb7dbae54176815@%3Cdev.beam.apache.org%3E
// for further details.

// To produce the list of necessary relocations, one needs to start with a set of target
// packages that one wants to vendor, find all necessary transitive dependencies of that
// set and provide relocations for each such that all necessary packages and their
// dependencies are relocated. Any optional dependency that doesn't need relocation
// must be excluded via an 'exclude' rule. There is additional complexity of libraries that use
// JNI or reflection and have to be handled on case by case basis by learning whether
// they support relocation and how would one go about doing it by reading any documentation
// those libraries may provide. The 'validateShadedJarDoesntLeakNonOrgApacheBeamClasses'
// ensures that there are no classes outside of the 'org.apache.beam' namespace.

String version = "v1p36p0";
String prefix = "org.apache.beam.vendor.grpc.${version}";
List<String> packagesToRelocate = [
// guava uses the com.google.common and com.google.thirdparty package namespaces
"com.google.common",
"com.google.thirdparty",
"com.google.protobuf",
"com.google.gson",
"com.google.auth",
"com.google.api",
"com.google.cloud",
"com.google.logging",
"com.google.longrunning",
"com.google.rpc",
"com.google.type",
"io.grpc",
"io.netty",
"io.opencensus",
]

return packagesToRelocate.collectEntries {
[ (it): "${prefix}.${it}" ]
} + [
// Adapted from https://github.com/grpc/grpc-java/blob/e283f70ad91f99c7fee8b31b605ef12a4f9b1690/netty/shaded/build.gradle#L41
// We "io.netty": "${prefix}.io.netty",have to be careful with these replacements as they must not match any
// string in NativeLibraryLoader, else they cause corruption. Note that
// this includes concatenation of string literals and constants.
'META-INF/native/libnetty': "META-INF/native/liborg_apache_beam_vendor_grpc_${version}_netty",
'META-INF/native/netty': "META-INF/native/org_apache_beam_vendor_grpc_${version}_netty",
]
}

/** Returns the list of shading exclusions. */
static List<String> exclusions() {
return [
// Don't include android annotations, errorprone, checkerframework, JDK8 annotations, objenesis, junit,
// commons-logging, log4j, slf4j and mockito in the vendored jar
"android/annotation/**/",
"com/google/errorprone/**",
"com/google/instrumentation/**",
"com/google/j2objc/annotations/**",
"io/netty/handler/codec/marshalling/**",
"io/netty/handler/codec/spdy/**",
"io/netty/handler/codec/compression/JZlib*",
"io/netty/handler/codec/compression/Lz4*",
"io/netty/handler/codec/compression/Lzf*",
"io/netty/handler/codec/compression/Lzma*",
"io/netty/handler/codec/protobuf/Protobuf*Nano.class",
"io/netty/util/internal/logging/CommonsLogger*",
"io/netty/util/internal/logging/LocationAwareSlf4JLogger*",
"io/netty/util/internal/logging/Log4JLogger*",
"io/netty/util/internal/logging/Log4J2Logger*",
"javax/annotation/**",
"junit/**",
"module-info.class",
"org/apache/commons/logging/**",
"org/apache/log/**",
"org/apache/log4j/**",
"org/apache/logging/log4j/**",
"org/checkerframework/**",
"org/codehaus/mojo/animal_sniffer/**",
"org/conscrypt/**",
"META-INF/native/libconscrypt**",
"META-INF/native/conscrypt**",
"org/hamcrest/**",
"org/junit/**",
"org/mockito/**",
"org/objenesis/**",
"org/slf4j/**",
]
}

/**
* Returns a closure contaning the dependencies map used for shading gRPC within the main
* Apache Beam project.
*/
static Object dependenciesClosure() {
return {
dependencies().each { compile it }
runtimeDependencies().each { shadow it }
}
}

/**
* Returns a closure with the code relocation configuration for shading gRPC within the main
* Apache Beam project.
*/
static Object shadowClosure() {
return {
relocations().each { srcNamespace, destNamespace ->
relocate srcNamespace, destNamespace
}
exclusions().each { exclude it }
}
}
}
2 changes: 1 addition & 1 deletion examples/java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,7 @@ dependencies {
compile library.java.joda_time
compile library.java.proto_google_cloud_datastore_v1
compile library.java.slf4j_api
compile library.java.vendored_grpc_1_26_0
compile library.java.vendored_grpc_1_36_0
compile library.java.vendored_guava_26_0_jre
compile "com.google.api.grpc:proto-google-cloud-language-v1:1.81.4"
compile ("io.confluent:kafka-avro-serializer:5.3.2") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,8 @@
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.beam.examples.complete.kafkatopubsub.options.KafkaToPubsubOptions;
import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.JsonObject;
import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.JsonParser;
import org.apache.beam.vendor.grpc.v1p36p0.com.google.gson.JsonObject;
import org.apache.beam.vendor.grpc.v1p36p0.com.google.gson.JsonParser;
import org.apache.http.HttpResponse;
import org.apache.http.client.HttpClient;
import org.apache.http.client.methods.HttpGet;
Expand Down
12 changes: 6 additions & 6 deletions examples/notebooks/get-started/try-apache-beam-java.ipynb
Original file line number Diff line number Diff line change
Expand Up @@ -593,8 +593,8 @@
"\n",
"> Task :runShadow\n",
"WARNING: An illegal reflective access operation has occurred\n",
"WARNING: Illegal reflective access by org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.UnsafeUtil (file:/content/build/install/content-shadow/lib/WordCount.jar) to field java.nio.Buffer.address\n",
"WARNING: Please consider reporting this to the maintainers of org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.UnsafeUtil\n",
"WARNING: Illegal reflective access by org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.UnsafeUtil (file:/content/build/install/content-shadow/lib/WordCount.jar) to field java.nio.Buffer.address\n",
"WARNING: Please consider reporting this to the maintainers of org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.UnsafeUtil\n",
"WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations\n",
"WARNING: All illegal access operations will be denied in a future release\n",
"Mar 04, 2019 11:00:24 PM org.apache.beam.sdk.io.FileBasedSource getEstimatedSizeBytes\n",
Expand Down Expand Up @@ -735,8 +735,8 @@
"\n",
">> java -jar WordCount.jar\n",
"WARNING: An illegal reflective access operation has occurred\n",
"WARNING: Illegal reflective access by org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.UnsafeUtil (file:/content/WordCount.jar) to field java.nio.Buffer.address\n",
"WARNING: Please consider reporting this to the maintainers of org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.UnsafeUtil\n",
"WARNING: Illegal reflective access by org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.UnsafeUtil (file:/content/WordCount.jar) to field java.nio.Buffer.address\n",
"WARNING: Please consider reporting this to the maintainers of org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.UnsafeUtil\n",
"WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations\n",
"WARNING: All illegal access operations will be denied in a future release\n",
"Mar 04, 2019 11:00:49 PM org.apache.beam.sdk.io.FileBasedSource getEstimatedSizeBytes\n",
Expand Down Expand Up @@ -981,8 +981,8 @@
"\n",
"> Task :runShadow\n",
"WARNING: An illegal reflective access operation has occurred\n",
"WARNING: Illegal reflective access by org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.UnsafeUtil (file:/content/build/install/content-shadow/lib/WordCount.jar) to field java.nio.Buffer.address\n",
"WARNING: Please consider reporting this to the maintainers of org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.UnsafeUtil\n",
"WARNING: Illegal reflective access by org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.UnsafeUtil (file:/content/build/install/content-shadow/lib/WordCount.jar) to field java.nio.Buffer.address\n",
"WARNING: Please consider reporting this to the maintainers of org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.UnsafeUtil\n",
"WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations\n",
"WARNING: All illegal access operations will be denied in a future release\n",
"Mar 04, 2019 11:01:26 PM org.apache.beam.sdk.io.FileBasedSource getEstimatedSizeBytes\n",
Expand Down
2 changes: 1 addition & 1 deletion runners/core-construction-java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ dependencies {
compile project(path: ":model:job-management", configuration: "shadow")
compile project(path: ":sdks:java:core", configuration: "shadow")
compile project(path: ":sdks:java:fn-execution")
compile library.java.vendored_grpc_1_26_0
compile library.java.vendored_grpc_1_36_0
compile library.java.vendored_guava_26_0_jre
compile library.java.classgraph
compile library.java.jackson_core
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.beam.runners.core.construction;

import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ProtocolMessageEnum;
import org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.ProtocolMessageEnum;

/** Returns the standard URN of a given enum annotated with [(standard_urn)]. */
public class BeamUrns {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.util.SerializableUtils;
import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
import org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.ByteString;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.BiMap;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableBiMap;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@
import org.apache.beam.sdk.util.ShardedKey;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;

/** {@link CoderTranslator} implementations for known coder types. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@
import org.apache.beam.sdk.util.SerializableUtils;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
import org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.ByteString;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@
import org.apache.beam.sdk.util.SerializableUtils;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionView;
import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
import org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.ByteString;

/**
* Utility methods for translating a {@link View} transforms to and from {@link RunnerApi}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@
import org.apache.beam.model.expansion.v1.ExpansionApi;
import org.apache.beam.model.expansion.v1.ExpansionServiceGrpc;
import org.apache.beam.model.pipeline.v1.Endpoints;
import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
import org.apache.beam.vendor.grpc.v1p36p0.io.grpc.ManagedChannel;

/** Default factory for ExpansionServiceClient used by External transform. */
public class DefaultExpansionServiceClientFactory implements ExpansionServiceClientFactory {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.model.pipeline.v1.RunnerApi.StandardDisplayData;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
import org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.ByteString;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;

Expand Down
Loading