diff --git a/.travis.yml b/.travis.yml index 21735e2a42e1..2a89d2c825d9 100644 --- a/.travis.yml +++ b/.travis.yml @@ -27,27 +27,43 @@ dist: trusty cache: directories: - $HOME/.m2/repository + - $HOME/.thrift services: - docker +before_install: + - | + if [[ ! -e $HOME/.thrift/bin/thrift ]]; then + sudo apt-get install libboost-dev libboost-test-dev libboost-program-options-dev libboost-filesystem-dev libboost-thread-dev libevent-dev automake libtool flex bison pkg-config g++ libssl-dev + wget https://www.apache.org/dist/thrift/0.9.3/thrift-0.9.3.tar.gz + tar xfz thrift-0.9.3.tar.gz + cd thrift-0.9.3 && ./configure --without-cpp --without-c_glib --without-python --without-ruby --without-php --without-erlang --without-go --without-nodejs -q --prefix=$HOME/.thrift + sudo make install > thrift_make_install.log + cd .. + fi + - | + if [[ ! -e /usr/local/bin/thrift ]]; then + sudo ln -s $HOME/.thrift/bin/thrift /usr/local/bin/thrift + fi + install: - ./mvnw -v - | if [[ -v TEST_SPECIFIC_MODULES ]]; then - ./mvnw install $MAVEN_FAST_INSTALL -pl $TEST_SPECIFIC_MODULES -am + ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl $TEST_SPECIFIC_MODULES -am fi - | if [[ -v TEST_OTHER_MODULES ]]; then - ./mvnw install $MAVEN_FAST_INSTALL -pl '!presto-docs,!presto-server,!presto-server-rpm' + ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl '!presto-docs,!presto-server,!presto-server-rpm' fi - | if [[ -v PRODUCT_TESTS_BASIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then - ./mvnw install $MAVEN_FAST_INSTALL -pl '!presto-docs,!presto-server-rpm' + ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl '!presto-docs,!presto-server-rpm' fi - | if [[ -v HIVE_TESTS ]]; then - ./mvnw install $MAVEN_FAST_INSTALL -pl presto-hive-hadoop2 -am + ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl presto-hive-hadoop2 -am fi before_script: @@ -60,30 +76,40 @@ before_script: script: - | if [[ -v MAVEN_CHECKS ]]; then - ./mvnw install -DskipTests -B -T C1 -P ci + ./mvnw install -DskipTests -B -T C1 -P 'ci,!twitter-modules' fi - | if [[ -v TEST_SPECIFIC_MODULES ]]; then - ./mvnw test $MAVEN_SKIP_CHECKS_AND_DOCS -B -pl $TEST_SPECIFIC_MODULES $TEST_FLAGS + ./mvnw test $MAVEN_SKIP_CHECKS_AND_DOCS -P !twitter-modules -B -pl $TEST_SPECIFIC_MODULES $TEST_FLAGS fi - | if [[ -v TEST_OTHER_MODULES ]]; then - ./mvnw test $MAVEN_SKIP_CHECKS_AND_DOCS -B -pl $TEST_OTHER_MODULES + ./mvnw test $MAVEN_SKIP_CHECKS_AND_DOCS -P !twitter-modules -B -pl $TEST_OTHER_MODULES fi - | if [[ -v PRODUCT_TESTS_BASIC_ENVIRONMENT ]]; then presto-product-tests/bin/run_on_docker.sh \ - multinode -x quarantine,big_query,storage_formats,profile_specific_tests,tpcds + multinode -x quarantine,big_query,storage_formats,profile_specific_tests,tpcds,cli,hive_connector + fi + - | + if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then + presto-product-tests/bin/run_on_docker.sh \ + singlenode -g hdfs_no_impersonation + fi + - | + if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then + presto-product-tests/bin/run_on_docker.sh \ + singlenode-hdfs-impersonation -g storage_formats,hdfs_impersonation fi - | if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then presto-product-tests/bin/run_on_docker.sh \ - singlenode-kerberos-hdfs-impersonation -g storage_formats,cli,hdfs_impersonation,authorization + singlenode-kerberos-hdfs-impersonation -g storage_formats,hdfs_impersonation,authorization fi - | if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then presto-product-tests/bin/run_on_docker.sh \ - singlenode-ldap -g ldap -x simba_jdbc + singlenode-ldap -g ldap -x simba_jdbc,ldap_cli fi # SQL server image sporadically hangs during the startup # TODO: Uncomment it once issue is fixed @@ -96,7 +122,7 @@ script: - | if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then presto-product-tests/bin/run_on_docker.sh \ - multinode-tls -g smoke,cli,group-by,join,tls + multinode-tls -g smoke,group-by,join,tls fi - | if [[ -v HIVE_TESTS ]]; then @@ -108,8 +134,11 @@ before_cache: - rm -rf $HOME/.m2/repository/com/facebook notifications: + hipchat: + rooms: + secure: peNh1KxwlxIpFyb60S8AMvaJThgh1LsjE+Whf1rYkJalVd2wUrqBIoyDKVSueyHD01hQ06gT7rBV6Pu/QcBMR1a9BbMCjERfxLZFUAheuC2Rsb+p1c4dyvBcFUGacgW7XWKCaVYGDGxuUvb0I3Z8cR6KxhK2xi88tHiqBGVGV2yI6zzOTpWVknMfFBtn+ONU1Ob2P6trclXaDyFd4MxubULri6CQdl35eQAq/VnmR3SZOgyVu3V30MGKwI3zhSli+3VqmW0JmaDGoHN6gznM1+VqABLgmIq0P+n+r5gdZWRCorq10NZCFMhVQ8U6rQHcL7sAniYJJsC/yRt6+pjyzIF4N+LSzZ7T+FLxQqT7k/1ukNgrujLDfTpn76Mo9eYTZmfAdzbm1QKJDACwr8Slqhq1jGzcrFMHunvXhVqjOs24R+JAHblY0O9PXvv7aR29GOQWDCvD7nV5QBUr8Xz5q7ozbLqHTI+yH02Jj4EaZ+azWYdRmnr9wDBxWMYBEgOdj4pII9b298XEDB72TxA3KpLTpdLxBTR+gIk/LjJqb/wb84xUv8gPXkaXccltGd5YI90c84cX8isbzNkAylzyfF2Eyueh0XbnMHfpFqBS7qaVM0/D+UxZkU0WNJ0x7G9XJvkiq49bZz2q1KLE4XuvVnTZSSjVSUAS8RtHfwUV33c= slack: - secure: V5eyoGShxFoCcYJcp858vf/T6gC9KeMxL0C1EElcpZRcKBrIVZzvhek3HLHxZOxlghqnvNVsyDtU3u5orkEaAXeXj5c2dN+4XBsAB9oeN5MtQ0Z3VLAhZDqKIW1LzcXrq4DpzM0PkGhjfjum/P94/qFYk0UckPtB6a341AuYRo8= + secure: E7XVlbdwIdKxnr6Tk1rmCefufs1w8h4nCWz79Uh6wMma8gC7x5ChKFqwvLRJ0WUpmPS+Ng1xeTv+wmb8TMDv2X8snmht9420/TFRy9wi1aLWNJXQUveNBzn83sCS40jFi6gd9xqKawd68R84UVH3PeNhksDtDnKAblx71miwbKmLwHc1KFoLMEnaaWEg5NgFl8/UadYDvsLD44v6YDza8eYrLp3aGK8v9ewBDySHE16IHAfpteTRaU0kG/H1kvVvFdH/h/sSPfimehd51b4i3mm/nRrjJ/VSLc7p9w5FkHUECtA0N6zcytRxN6MrbhrxJ8XG3vte3KSRSFCqfgOSRM2NWcca4CtBP2V0SwrAYMo5jim6fr921lfcbUTWTSnvMYLC17QrAxoclVrgK05GjGoLgSH42UPGf3QNkqXzyueNzaLJ+KSlgwFblIQKp6WGZYSRorL0F7s50pIoqMVoebcrnB0ObK/CcE2ywS/HeTgoSkWSDSmKBsO+cmtv1yAamy9DlmgRGZlxIxdBELXtHRkQ2B6Z2QdiQU4MHiFBc/IESJbnCait4odn+oJUjehZg+b9vjCoWwVw3zNMIJhokyxO8SiyKJmbO0z1g2L/BykWGI1DQu8HkeQzO+CmNUV3AOrxDG3amL/tkB/06fyQtnYMDhUhvX64uWSaE36sYL4= before_deploy: - mkdir /tmp/artifacts diff --git a/README.md b/README.md index f7428dba837d..9325bc3edef5 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,4 @@ -# Presto [![Build Status](https://travis-ci.org/prestodb/presto.svg?branch=master)](https://travis-ci.org/prestodb/presto) +# Presto [![Build Status](https://travis-ci.org/twitter-forks/presto.svg?branch=twitter-master)](https://travis-ci.org/twitter-forks/presto) Presto is a distributed SQL query engine for big data. diff --git a/pom.xml b/pom.xml index 7d252c16fbe6..f6439e93d43e 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 pom presto-root @@ -28,9 +28,9 @@ - scm:git:git://github.com/facebook/presto.git - https://github.com/facebook/presto - 0.196 + scm:git:git://github.com/twitter-forks/presto.git + https://github.com/twitter-forks/presto + 0.196-tw-0.49 @@ -118,6 +118,7 @@ presto-thrift-testing-server presto-thrift-connector presto-matching + presto-twitter-functions presto-memory-context @@ -819,6 +820,12 @@ 3.6.1 + + org.apache.commons + commons-pool2 + 2.4.2 + + commons-codec commons-codec @@ -895,6 +902,54 @@ 1.1.1.7 + + org.apache.curator + curator-recipes + 4.0.0 + + + org.apache.zookeeper + zookeeper + + + + + + org.apache.curator + curator-framework + 4.0.0 + + + org.apache.zookeeper + zookeeper + + + + + + org.apache.curator + curator-client + 4.0.0 + + + org.apache.zookeeper + zookeeper + + + + + + org.apache.curator + curator-test + 2.12.0 + + + org.apache.zookeeper + zookeeper + + + + org.apache.zookeeper zookeeper @@ -918,7 +973,7 @@ com.101tec zkclient - 0.8 + 0.10 log4j @@ -928,6 +983,10 @@ org.slf4j slf4j-log4j12 + + org.apache.zookeeper + zookeeper + @@ -1195,4 +1254,18 @@ + + + + twitter-modules + + true + + + presto-kafka07 + twitter-eventlistener-plugin + presto-twitter-server + + + diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index bff7399ac8ae..a4b0d420d557 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index d004ff1dd13d..7b1f87723fc7 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 8bf490655dff..8949fbad0d2e 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 4f5fa6d563bf..cdaaf93b6089 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index a1211ba8ef4e..d53cf602b0ad 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index a008359b6794..50fdf6cbb283 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.196 + 0.196-tw-0.49 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 3ad7af423377..2de935090da1 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 09d55354885c..2ea62a29b01e 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 79593f8e48e5..7225b49fe70b 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 7c11b654c66a..b2c3312797f5 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-cli diff --git a/presto-cli/src/main/java/com/facebook/presto/cli/ClientOptions.java b/presto-cli/src/main/java/com/facebook/presto/cli/ClientOptions.java index 81baf40cb1ef..d8e27d0448bd 100644 --- a/presto-cli/src/main/java/com/facebook/presto/cli/ClientOptions.java +++ b/presto-cli/src/main/java/com/facebook/presto/cli/ClientOptions.java @@ -18,6 +18,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.net.HostAndPort; +import com.sun.security.auth.module.UnixSystem; import io.airlift.airline.Option; import io.airlift.units.Duration; @@ -74,15 +75,16 @@ public class ClientOptions @Option(name = "--keystore-password", title = "keystore password", description = "Keystore password") public String keystorePassword; + // Pick the user name for the logged in user. + // Do not let it be overridden by users. + public String user = new UnixSystem().getUsername(); + @Option(name = "--truststore-path", title = "truststore path", description = "Truststore path") public String truststorePath; @Option(name = "--truststore-password", title = "truststore password", description = "Truststore password") public String truststorePassword; - @Option(name = "--user", title = "user", description = "Username") - public String user = System.getProperty("user.name"); - @Option(name = "--password", title = "password", description = "Prompt for password") public boolean password; diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 6fd61d4fd059..2390e74dacca 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-client diff --git a/presto-client/src/main/java/com/facebook/presto/client/SpnegoHandler.java b/presto-client/src/main/java/com/facebook/presto/client/SpnegoHandler.java index 85f144a6dfca..45f3ef5ca4ac 100644 --- a/presto-client/src/main/java/com/facebook/presto/client/SpnegoHandler.java +++ b/presto-client/src/main/java/com/facebook/presto/client/SpnegoHandler.java @@ -75,6 +75,7 @@ public class SpnegoHandler private static final Oid KERBEROS_OID = createOid("1.2.840.113554.1.2.2"); private final String remoteServiceName; + private final boolean isCompleteServicePrinciple; private final boolean useCanonicalHostname; private final Optional principal; private final Optional keytab; @@ -92,6 +93,7 @@ public SpnegoHandler( Optional credentialCache) { this.remoteServiceName = requireNonNull(remoteServiceName, "remoteServiceName is null"); + this.isCompleteServicePrinciple = remoteServiceName.contains("@"); this.useCanonicalHostname = useCanonicalHostname; this.principal = requireNonNull(principal, "principal is null"); this.keytab = requireNonNull(keytab, "keytab is null"); @@ -133,7 +135,7 @@ private static boolean isNegotiate(String value) private Request authenticate(Request request) { String hostName = request.url().host(); - String principal = makeServicePrincipal(remoteServiceName, hostName, useCanonicalHostname); + String principal = isCompleteServicePrinciple ? remoteServiceName : makeServicePrincipal(remoteServiceName, hostName, useCanonicalHostname); byte[] token = generateToken(principal); String credential = format("%s %s", NEGOTIATE, Base64.getEncoder().encodeToString(token)); @@ -149,7 +151,7 @@ private byte[] generateToken(String servicePrincipal) Session session = getSession(); context = doAs(session.getLoginContext().getSubject(), () -> { GSSContext result = GSS_MANAGER.createContext( - GSS_MANAGER.createName(servicePrincipal, NT_HOSTBASED_SERVICE), + GSS_MANAGER.createName(servicePrincipal, isCompleteServicePrinciple ? NT_USER_NAME : NT_HOSTBASED_SERVICE), SPNEGO_OID, session.getClientCredential(), INDEFINITE_LIFETIME); diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 9a987e4b3c00..28e82f8eb187 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-docs diff --git a/presto-docs/src/main/sphinx/functions.rst b/presto-docs/src/main/sphinx/functions.rst index 14f6a1904fb0..3a978b0c631b 100644 --- a/presto-docs/src/main/sphinx/functions.rst +++ b/presto-docs/src/main/sphinx/functions.rst @@ -26,3 +26,4 @@ Functions and Operators functions/geospatial functions/color functions/teradata + functions/twitter diff --git a/presto-docs/src/main/sphinx/functions/twitter.rst b/presto-docs/src/main/sphinx/functions/twitter.rst new file mode 100644 index 000000000000..03bec3f9247d --- /dev/null +++ b/presto-docs/src/main/sphinx/functions/twitter.rst @@ -0,0 +1,51 @@ +================== +Twitter Functions +================== + +These functions provide some convenience functionality commonly used at Twitter. + +String Functions +----------------- + +.. function:: split_every(string) -> array + + Splits ``string`` on every character and returns an array. + +.. function:: split_every(string, length) -> array + + Splits ``string`` on every ``length`` characters and returns an array. + length must be a positive number. + +.. function:: split_every(string, length, limit) -> array + + Splits ``string`` on every ``length`` characters and returns an array of size at most ``limit``. + The last element in the array always contains everything left in the string. ``limit`` must be a positive number. + +Twitter IDs(Snowflake) Functions +--------------------------------- + +The utility functions for `Twitter IDs(Snowflake) `_. + +.. function:: is_snowflake(id) -> boolean + + Return if a bigint is a snowflake ID (true/false). + +.. function:: first_snowflake_for(timestamp) -> bigint + + Return the first snowflake ID given a timestamp. + +.. function:: timestamp_from_snowflake(id) -> timestamp + + Return the timestamp given a snowflake ID. + +.. function:: cluster_id_from_snowflake(id) -> bigint + + Return the cluster ID given a snowflake ID. + +.. function:: instance_id_from_snowflake(id) -> bigint + + Return the instance ID given a snowflake ID. + +.. function:: sequence_num_from_snowflake(id) -> bigint + + Return the sequence number given a snowflake ID. diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index f77d1888a579..b3807f898f01 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 980220593aec..c5f727d2b1fd 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index c3ee3eaba84b..cc92231d711b 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-geospatial diff --git a/presto-hive-hadoop2/bin/run_on_docker.sh b/presto-hive-hadoop2/bin/run_on_docker.sh index 3ef216e034e2..1d3380e19653 100755 --- a/presto-hive-hadoop2/bin/run_on_docker.sh +++ b/presto-hive-hadoop2/bin/run_on_docker.sh @@ -112,7 +112,7 @@ fi # run product tests pushd $PROJECT_ROOT set +e -./mvnw -pl presto-hive-hadoop2 test -P test-hive-hadoop2 \ +./mvnw -pl presto-hive-hadoop2 test -P test-hive-hadoop2,!twitter-modules \ -Dhive.hadoop2.timeZone=UTC \ -DHADOOP_USER_NAME=hive \ -Dhive.hadoop2.metastoreHost=hadoop-master \ diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 5394b686b579..95935c1c95b2 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 8d2ac7b9d7f6..45986141ea91 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-hive @@ -53,11 +53,37 @@ hive-apache + + org.apache.curator + curator-recipes + + + + org.apache.curator + curator-framework + + + + org.apache.curator + curator-client + + + + org.apache.zookeeper + zookeeper + runtime + + org.apache.thrift libthrift + + org.apache.commons + commons-pool2 + + io.airlift aircompressor @@ -98,6 +124,12 @@ configuration + + com.googlecode.json-simple + json-simple + 1.1 + + com.google.guava guava @@ -188,6 +220,41 @@ jackson-databind + + com.twitter.elephantbird + elephant-bird-core + 4.14 + thrift9 + + + commons-logging + commons-logging + + + com.hadoop.gplcompression + hadoop-lzo + + + + + + commons-io + commons-io + 2.5 + runtime + + + + commons-lang + commons-lang + 2.4 + + + + org.anarres.lzo + lzo-hadoop + + io.airlift @@ -217,6 +284,18 @@ + + org.apache.curator + curator-test + test + + + + com.101tec + zkclient + test + + com.facebook.presto presto-main @@ -265,12 +344,6 @@ test - - org.anarres.lzo - lzo-hadoop - test - - com.facebook.presto diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientModule.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientModule.java index 5a6c9e82926d..5defc24637a4 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientModule.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientModule.java @@ -26,6 +26,9 @@ import com.facebook.presto.spi.connector.ConnectorPageSourceProvider; import com.facebook.presto.spi.connector.ConnectorSplitManager; import com.facebook.presto.spi.type.TypeManager; +import com.facebook.presto.twitter.hive.thrift.HiveThriftFieldIdResolverFactory; +import com.facebook.presto.twitter.hive.thrift.ThriftFieldIdResolverFactory; +import com.facebook.presto.twitter.hive.thrift.ThriftHiveRecordCursorProvider; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; @@ -81,13 +84,14 @@ public void configure(Binder binder) binder.bind(NamenodeStats.class).in(Scopes.SINGLETON); newExporter(binder).export(NamenodeStats.class).as(generatedNameOf(NamenodeStats.class, connectorId)); - binder.bind(NodeManager.class).toInstance(nodeManager); binder.bind(TypeManager.class).toInstance(typeManager); binder.bind(PageIndexerFactory.class).toInstance(pageIndexerFactory); + binder.bind(ThriftFieldIdResolverFactory.class).toInstance(new HiveThriftFieldIdResolverFactory()); Multibinder recordCursorProviderBinder = newSetBinder(binder, HiveRecordCursorProvider.class); recordCursorProviderBinder.addBinding().to(ParquetRecordCursorProvider.class).in(Scopes.SINGLETON); + recordCursorProviderBinder.addBinding().to(ThriftHiveRecordCursorProvider.class).in(Scopes.SINGLETON); recordCursorProviderBinder.addBinding().to(GenericHiveRecordCursorProvider.class).in(Scopes.SINGLETON); binder.bind(HiveWriterStats.class).in(Scopes.SINGLETON); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveStorageFormat.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveStorageFormat.java index 4e4523d04185..5a9edf5b383c 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveStorageFormat.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveStorageFormat.java @@ -14,6 +14,7 @@ package com.facebook.presto.hive; import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.twitter.hive.thrift.ThriftGeneralInputFormat; import io.airlift.units.DataSize; import io.airlift.units.DataSize.Unit; import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat; @@ -32,6 +33,7 @@ import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe; import org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe; import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe; +import org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory; import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; @@ -93,6 +95,11 @@ public enum HiveStorageFormat LazySimpleSerDe.class.getName(), TextInputFormat.class.getName(), HiveIgnoreKeyTextOutputFormat.class.getName(), + new DataSize(8, Unit.MEGABYTE)), + THRIFTBINARY( + LazyBinarySerDe.class.getName(), + ThriftGeneralInputFormat.class.getName(), + HiveIgnoreKeyTextOutputFormat.class.getName(), new DataSize(8, Unit.MEGABYTE)); private final String serde; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java index 1047acb411f5..a422d9085ca1 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java @@ -27,6 +27,7 @@ import com.facebook.presto.spi.type.StandardTypes; import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.VarcharType; +import com.facebook.presto.twitter.hive.thrift.ThriftGeneralInputFormat; import com.google.common.base.Joiner; import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; @@ -38,6 +39,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hive.common.JavaUtils; import org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat; import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat; @@ -112,6 +114,7 @@ import static com.google.common.collect.Iterables.filter; import static com.google.common.collect.Lists.newArrayList; import static com.google.common.collect.Lists.transform; +import static com.hadoop.compression.lzo.LzoIndex.LZO_INDEX_SUFFIX; import static java.lang.Byte.parseByte; import static java.lang.Double.parseDouble; import static java.lang.Float.floatToRawIntBits; @@ -148,6 +151,22 @@ public final class HiveUtil private static final String BIG_DECIMAL_POSTFIX = "BD"; + private static final PathFilter LZOP_DEFAULT_SUFFIX_FILTER = new PathFilter() { + @Override + public boolean accept(Path path) + { + return path.toString().endsWith(".lzo"); + } + }; + + private static final PathFilter LZOP_INDEX_DEFAULT_SUFFIX_FILTER = new PathFilter() { + @Override + public boolean accept(Path path) + { + return path.toString().endsWith(".lzo.index"); + } + }; + static { DateTimeParser[] timestampWithoutTimeZoneParser = { DateTimeFormat.forPattern("yyyy-M-d").getParser(), @@ -180,7 +199,7 @@ private HiveUtil() // propagate serialization configuration to getRecordReader schema.stringPropertyNames().stream() - .filter(name -> name.startsWith("serialization.")) + .filter(name -> name.startsWith("serialization.") || name.startsWith("elephantbird.")) .forEach(name -> jobConf.set(name, schema.getProperty(name))); // add Airlift LZO and LZOP to head of codecs list so as to not override existing entries @@ -242,6 +261,11 @@ public static void setReadColumns(Configuration configuration, List rea return MapredParquetInputFormat.class; } + // Remove this after https://github.com/twitter/elephant-bird/pull/481 is included in a release + if ("com.twitter.elephantbird.mapred.input.HiveMultiInputFormat".equals(inputFormatName)) { + return ThriftGeneralInputFormat.class; + } + Class clazz = conf.getClassByName(inputFormatName); return (Class>) clazz.asSubclass(InputFormat.class); } @@ -294,6 +318,21 @@ public static boolean isSplittable(InputFormat inputFormat, FileSystem fil } } + public static boolean isLzopCompressedFile(Path filePath) + { + return LZOP_DEFAULT_SUFFIX_FILTER.accept(filePath); + } + + public static boolean isLzopIndexFile(Path filePath) + { + return LZOP_INDEX_DEFAULT_SUFFIX_FILTER.accept(filePath); + } + + public static Path getLzopIndexPath(Path lzoPath) + { + return lzoPath.suffix(LZO_INDEX_SUFFIX); + } + public static StructObjectInspector getTableObjectInspector(Properties schema) { return getTableObjectInspector(getDeserializer(schema)); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java index 4c35ce9f9105..1e3416d0f805 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java @@ -499,6 +499,17 @@ public static boolean isViewFileSystem(HdfsContext context, HdfsEnvironment hdfs } } + public static boolean isHDFSCompatibleViewFileSystem(HdfsContext context, HdfsEnvironment hdfsEnvironment, Path path) + { + try { + return getRawFileSystem(hdfsEnvironment.getFileSystem(context, path)) + .getClass().getName().equals("org.apache.hadoop.fs.viewfs.HDFSCompatibleViewFileSystem"); + } + catch (IOException e) { + throw new PrestoException(HIVE_FILESYSTEM_ERROR, "Failed checking path: " + path, e); + } + } + private static FileSystem getRawFileSystem(FileSystem fileSystem) { if (fileSystem instanceof FilterFileSystem) { @@ -527,6 +538,11 @@ public static Path createTemporaryPath(HdfsContext context, HdfsEnvironment hdfs temporaryPrefix = ".hive-staging"; } + // use relative temporary directory on HDFSCompatibleViewFileSystem + if (isHDFSCompatibleViewFileSystem(context, hdfsEnvironment, targetPath)) { + temporaryPrefix = "../.hive-staging"; + } + // create a temporary directory on the same filesystem Path temporaryRoot = new Path(targetPath, temporaryPrefix); Path temporaryPath = new Path(temporaryRoot, randomUUID().toString()); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/RetryDriver.java b/presto-hive/src/main/java/com/facebook/presto/hive/RetryDriver.java index 911ee8f3712b..22e1a8127817 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/RetryDriver.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/RetryDriver.java @@ -137,16 +137,18 @@ public V run(String callableName, Callable callable) return callable.call(); } catch (Exception e) { + log.debug("Failed on executing %s with attempt %d, Exception: %s", callableName, attempt, e.getMessage()); e = exceptionMapper.apply(e); for (Class clazz : exceptionWhiteList) { if (clazz.isInstance(e)) { + log.debug("Exception is in whitelist."); throw e; } } if (attempt >= maxAttempts || Duration.nanosSince(startTime).compareTo(maxRetryTime) >= 0) { + log.debug("Maximum attempts or maximum retry time reached. attempt: %d, maxAttempts: %d, duration: [%s] maxRetryTime: [%s]", attempt, maxAttempts, Duration.nanosSince(startTime).toString(), maxRetryTime.toString()); throw e; } - log.debug("Failed on executing %s with attempt %d, will retry. Exception: %s", callableName, attempt, e.getMessage()); int delayInMs = (int) Math.min(minSleepTime.toMillis() * Math.pow(scaleFactor, attempt - 1), maxSleepTime.toMillis()); int jitter = ThreadLocalRandom.current().nextInt(Math.max(1, (int) (delayInMs * 0.1))); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/SimpleHadoopAuthentication.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/SimpleHadoopAuthentication.java index f4e718763626..94bf3f068002 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/SimpleHadoopAuthentication.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/SimpleHadoopAuthentication.java @@ -25,7 +25,7 @@ public class SimpleHadoopAuthentication public UserGroupInformation getUserGroupInformation() { try { - return UserGroupInformation.getCurrentUser(); + return UserGroupInformation.getLoginUser(); } catch (IOException e) { throw new UncheckedIOException(e); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftMetastoreModule.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftMetastoreModule.java index 62d94b3875ba..1ba8ea19281d 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftMetastoreModule.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftMetastoreModule.java @@ -16,17 +16,21 @@ import com.facebook.presto.hive.ForCachingHiveMetastore; import com.facebook.presto.hive.metastore.CachingHiveMetastore; import com.facebook.presto.hive.metastore.ExtendedHiveMetastore; +import com.facebook.presto.twitter.hive.MetastoreStaticClusterModule; +import com.facebook.presto.twitter.hive.MetastoreZkDiscoveryBasedModule; +import com.facebook.presto.twitter.hive.PooledHiveMetastoreClientFactory; +import com.facebook.presto.twitter.hive.ZookeeperServersetMetastoreConfig; import com.google.inject.Binder; -import com.google.inject.Module; import com.google.inject.Scopes; +import io.airlift.configuration.AbstractConfigurationAwareModule; -import static io.airlift.configuration.ConfigBinder.configBinder; +import static io.airlift.configuration.ConditionalModule.installModuleIf; import static java.util.Objects.requireNonNull; import static org.weakref.jmx.ObjectNames.generatedNameOf; import static org.weakref.jmx.guice.ExportBinder.newExporter; public class ThriftMetastoreModule - implements Module + extends AbstractConfigurationAwareModule { private final String connectorId; @@ -36,11 +40,11 @@ public ThriftMetastoreModule(String connectorId) } @Override - public void configure(Binder binder) + public void setup(Binder binder) { binder.bind(HiveMetastoreClientFactory.class).in(Scopes.SINGLETON); - binder.bind(HiveCluster.class).to(StaticHiveCluster.class).in(Scopes.SINGLETON); - configBinder(binder).bindConfig(StaticMetastoreConfig.class); + binder.bind(PooledHiveMetastoreClientFactory.class).in(Scopes.SINGLETON); + bindMetastoreClusterModule(); binder.bind(HiveMetastore.class).to(ThriftHiveMetastore.class).in(Scopes.SINGLETON); binder.bind(ExtendedHiveMetastore.class).annotatedWith(ForCachingHiveMetastore.class).to(BridgingHiveMetastore.class).in(Scopes.SINGLETON); @@ -50,4 +54,16 @@ public void configure(Binder binder) newExporter(binder).export(ExtendedHiveMetastore.class) .as(generatedNameOf(CachingHiveMetastore.class, connectorId)); } + + private void bindMetastoreClusterModule() + { + install(installModuleIf( + ZookeeperServersetMetastoreConfig.class, + zkMetastoreConfig -> zkMetastoreConfig.getZookeeperServerHostAndPort() == null, + new MetastoreStaticClusterModule())); + install(installModuleIf( + ZookeeperServersetMetastoreConfig.class, + zkMetastoreConfig -> zkMetastoreConfig.getZookeeperServerHostAndPort() != null, + new MetastoreZkDiscoveryBasedModule())); + } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HadoopLzoDecompressor.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HadoopLzoDecompressor.java new file mode 100644 index 000000000000..a87b648ca14b --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HadoopLzoDecompressor.java @@ -0,0 +1,69 @@ +/* + * Licensed 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 com.facebook.presto.hive.parquet; + +import com.hadoop.compression.lzo.LzoCodec; +import io.airlift.compress.Decompressor; +import io.airlift.compress.MalformedInputException; +import org.apache.hadoop.conf.Configuration; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import static com.google.common.base.Preconditions.checkArgument; + +// HadoopLzoDecompressor from aircompressor +public class HadoopLzoDecompressor + implements Decompressor +{ + private static final Configuration HADOOP_CONF = new Configuration(); + + private final org.apache.hadoop.io.compress.Decompressor decompressor; + + public HadoopLzoDecompressor() + { + LzoCodec codec = new LzoCodec(); + codec.setConf(HADOOP_CONF); + decompressor = codec.createDecompressor(); + } + + @Override + public int decompress(byte[] input, int inputOffset, int inputLength, byte[] output, int outputOffset, int maxOutputLength) + throws MalformedInputException + { + decompressor.reset(); + decompressor.setInput(input, inputOffset, inputLength); + + int offset = outputOffset; + int outputLimit = outputOffset + maxOutputLength; + while (!decompressor.finished() && offset < outputLimit) { + try { + offset += decompressor.decompress(output, offset, outputLimit - offset); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + checkArgument(decompressor.getRemaining() == 0); + return offset - outputOffset; + } + + @Override + public void decompress(ByteBuffer input, ByteBuffer output) + throws MalformedInputException + { + throw new UnsupportedOperationException("Not supported in HadoopLzoDecompressor"); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetCompressionUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetCompressionUtils.java index ef9a300a4acd..963f432e408b 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetCompressionUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetCompressionUtils.java @@ -14,7 +14,6 @@ package com.facebook.presto.hive.parquet; import io.airlift.compress.Decompressor; -import io.airlift.compress.lzo.LzoDecompressor; import io.airlift.compress.snappy.SnappyDecompressor; import io.airlift.slice.DynamicSliceOutput; import io.airlift.slice.Slice; @@ -88,7 +87,7 @@ private static Slice decompressGzip(Slice input, int uncompressedSize) private static Slice decompressLZO(Slice input, int uncompressedSize) { - LzoDecompressor lzoDecompressor = new LzoDecompressor(); + Decompressor lzoDecompressor = new HadoopLzoDecompressor(); long totalDecompressedCount = 0; // over allocate buffer which makes decompression easier byte[] output = new byte[uncompressedSize + SIZE_OF_LONG]; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java index 57a2ff2939a0..d07a818c3fee 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java @@ -40,8 +40,8 @@ import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveErrorCode.HIVE_BAD_DATA; import static com.facebook.presto.hive.HiveErrorCode.HIVE_CURSOR_ERROR; +import static com.facebook.presto.hive.parquet.ParquetTypeUtils.findFieldIndexByName; import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getDescriptor; -import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getFieldIndex; import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getParquetType; import static com.facebook.presto.spi.type.StandardTypes.ARRAY; import static com.facebook.presto.spi.type.StandardTypes.MAP; @@ -170,7 +170,7 @@ public Page getNextPage() Type type = types.get(fieldId); int fieldIndex; if (useParquetColumnNames) { - fieldIndex = getFieldIndex(fileSchema, columnNames.get(fieldId)); + fieldIndex = findFieldIndexByName(fileSchema, columnNames.get(fieldId)); } else { fieldIndex = hiveColumnIndexes[fieldId]; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java index d8dd063c902f..3bf89764040b 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java @@ -138,7 +138,7 @@ public static int getFieldIndex(MessageType fileSchema, String name) public static parquet.schema.Type getParquetType(HiveColumnHandle column, MessageType messageType, boolean useParquetColumnNames) { if (useParquetColumnNames) { - return getParquetTypeByName(column.getName(), messageType); + return findParquetTypeByName(column, messageType); } if (column.getHiveColumnIndex() < messageType.getFieldCount()) { @@ -147,6 +147,43 @@ public static parquet.schema.Type getParquetType(HiveColumnHandle column, Messag return null; } + /** + * Find the column type by name using returning the first match with the following logic: + *
    + *
  • direct match
  • + *
  • case-insensitive match
  • + *
  • if the name ends with _, remove it and direct match
  • + *
  • if the name ends with _, remove it and case-insensitive match
  • + *
+ */ + private static parquet.schema.Type findParquetTypeByName(HiveColumnHandle column, MessageType messageType) + { + String name = column.getName(); + parquet.schema.Type type = getParquetTypeByName(name, messageType); + + // when a parquet field is a hive keyword we append an _ to it in hive. When doing + // a name-based lookup, we need to strip it off again if we didn't get a direct match. + if (type == null && name.endsWith("_")) { + type = getParquetTypeByName(name.substring(0, name.length() - 1), messageType); + } + return type; + } + + // Find the column index by name following the same logic as findParquetTypeByName + public static int findFieldIndexByName(MessageType fileSchema, String name) + { + // direct match and case-insensitive match + int fieldIndex = getFieldIndex(fileSchema, name); + + // when a parquet field is a hive keyword we append an _ to it in hive. + // try remove _ and direct match / case-insensitive match again + if (fieldIndex == -1 && name.endsWith("_")) { + fieldIndex = getFieldIndex(fileSchema, name.substring(0, name.length() - 1)); + } + + return fieldIndex; + } + public static ParquetEncoding getParquetEncoding(Encoding encoding) { switch (encoding) { diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/util/HiveFileIterator.java b/presto-hive/src/main/java/com/facebook/presto/hive/util/HiveFileIterator.java index 94205aaa3ec9..6a2610930888 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/util/HiveFileIterator.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/util/HiveFileIterator.java @@ -98,7 +98,14 @@ protected LocatedFileStatus computeNext() if (paths.isEmpty()) { return endOfData(); } - remoteIterator = getLocatedFileStatusRemoteIterator(paths.removeFirst()); + try { + remoteIterator = getLocatedFileStatusRemoteIterator(paths.removeFirst()); + } + catch (PrestoException e) { + if (!e.getErrorCode().equals(HIVE_FILE_NOT_FOUND.toErrorCode())) { + throw e; + } + } } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/util/InternalHiveSplitFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/util/InternalHiveSplitFactory.java index 8e0eae84eb0a..6781791625dd 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/util/InternalHiveSplitFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/util/InternalHiveSplitFactory.java @@ -40,6 +40,7 @@ import java.util.Properties; import static com.facebook.presto.hive.HiveColumnHandle.isPathColumnHandle; +import static com.facebook.presto.hive.HiveUtil.isLzopIndexFile; import static com.facebook.presto.hive.HiveUtil.isSplittable; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableList.toImmutableList; @@ -233,6 +234,10 @@ private static Optional getPathDomain(TupleDomain effe private static boolean pathMatchesPredicate(Optional pathDomain, String path) { + if (isLzopIndexFile(new Path(path))) { + return false; + } + if (!pathDomain.isPresent()) { return true; } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/MetastoreStaticClusterModule.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/MetastoreStaticClusterModule.java new file mode 100644 index 000000000000..8751b5bc474f --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/MetastoreStaticClusterModule.java @@ -0,0 +1,34 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive; + +import com.facebook.presto.hive.metastore.thrift.HiveCluster; +import com.facebook.presto.hive.metastore.thrift.StaticHiveCluster; +import com.facebook.presto.hive.metastore.thrift.StaticMetastoreConfig; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; + +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class MetastoreStaticClusterModule + implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(HiveCluster.class).to(StaticHiveCluster.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(StaticMetastoreConfig.class); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/MetastoreZkDiscoveryBasedModule.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/MetastoreZkDiscoveryBasedModule.java new file mode 100644 index 000000000000..5459eca55305 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/MetastoreZkDiscoveryBasedModule.java @@ -0,0 +1,32 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive; + +import com.facebook.presto.hive.metastore.thrift.HiveCluster; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; + +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class MetastoreZkDiscoveryBasedModule + implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(HiveCluster.class).to(ZookeeperServersetHiveCluster.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(ZookeeperServersetMetastoreConfig.class); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java new file mode 100644 index 000000000000..270e313e7511 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java @@ -0,0 +1,92 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive; + +import com.facebook.presto.hive.HiveClientConfig; +import com.facebook.presto.hive.authentication.HiveMetastoreAuthentication; +import com.facebook.presto.hive.metastore.thrift.HiveMetastoreClient; +import com.facebook.presto.hive.metastore.thrift.ThriftHiveMetastoreClient; +import com.facebook.presto.twitter.hive.util.PooledTTransportFactory; +import com.facebook.presto.twitter.hive.util.TTransportPool; +import com.google.common.net.HostAndPort; +import io.airlift.units.Duration; +import org.apache.commons.pool2.impl.GenericObjectPoolConfig; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; + +import javax.annotation.Nullable; +import javax.inject.Inject; + +import static java.lang.Math.toIntExact; +import static java.util.Objects.requireNonNull; + +public class PooledHiveMetastoreClientFactory +{ + private final HostAndPort socksProxy; + private final int timeoutMillis; + private final HiveMetastoreAuthentication metastoreAuthentication; + private final TTransportPool transportPool; + + public PooledHiveMetastoreClientFactory(@Nullable HostAndPort socksProxy, + Duration timeout, + HiveMetastoreAuthentication metastoreAuthentication, + int maxTransport, + long idleTimeout, + long transportEvictInterval, + int evictNumTests) + { + this.socksProxy = socksProxy; + this.timeoutMillis = toIntExact(timeout.toMillis()); + this.metastoreAuthentication = requireNonNull(metastoreAuthentication, "metastoreAuthentication is null"); + GenericObjectPoolConfig poolConfig = new GenericObjectPoolConfig(); + poolConfig.setMaxIdle(maxTransport); + poolConfig.setMaxTotal(maxTransport); + poolConfig.setMinEvictableIdleTimeMillis(idleTimeout); + poolConfig.setTimeBetweenEvictionRunsMillis(transportEvictInterval); + poolConfig.setNumTestsPerEvictionRun(evictNumTests); + this.transportPool = new TTransportPool(poolConfig); + } + + @Inject + public PooledHiveMetastoreClientFactory(HiveClientConfig config, + ZookeeperServersetMetastoreConfig zkConfig, + HiveMetastoreAuthentication metastoreAuthentication) + { + this(config.getMetastoreSocksProxy(), + config.getMetastoreTimeout(), + metastoreAuthentication, + zkConfig.getMaxTransport(), + zkConfig.getTransportIdleTimeout(), + zkConfig.getTransportEvictInterval(), + zkConfig.getTransportEvictNumTests()); + } + + public HiveMetastoreClient create(String host, int port) + throws TTransportException + { + try { + TTransport transport = transportPool.borrowObject(host, port); + if (transport == null) { + transport = transportPool.borrowObject(host, port, + new PooledTTransportFactory(transportPool, + host, port, socksProxy, + timeoutMillis, metastoreAuthentication)); + } + return new ThriftHiveMetastoreClient(transport); + } + catch (Exception e) { + throw new TTransportException(String.format("%s: %s", host, e.getMessage()), e.getCause()); + } + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperMetastoreMonitor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperMetastoreMonitor.java new file mode 100644 index 000000000000..cf8df021c1a1 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperMetastoreMonitor.java @@ -0,0 +1,119 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive; + +import com.google.common.net.HostAndPort; +import io.airlift.log.Logger; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.recipes.cache.PathChildrenCache; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.curator.utils.ZKPaths; +import org.json.simple.JSONObject; +import org.json.simple.JSONValue; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.stream.Collectors; + +public class ZookeeperMetastoreMonitor + implements PathChildrenCacheListener +{ + public static final Logger log = Logger.get(ZookeeperMetastoreMonitor.class); + private CuratorFramework client; + private PathChildrenCache cache; + private ConcurrentMap servers; // (Node_Name->HostAndPort) + + public ZookeeperMetastoreMonitor(String zkServer, String watchPath, int maxRetries, int retrySleepTime) + throws Exception + { + client = CuratorFrameworkFactory.newClient(zkServer, new ExponentialBackoffRetry(retrySleepTime, maxRetries)); + client.start(); + + cache = new PathChildrenCache(client, watchPath, true); // true indicating cache node contents in addition to the stat + try { + cache.start(); + } + catch (Exception ex) { + throw new RuntimeException("Curator PathCache Creation failed: " + ex.getMessage()); + } + + cache.getListenable().addListener(this); + servers = new ConcurrentHashMap<>(); + } + + public void close() + { + client.close(); + + try { + cache.close(); + } + catch (IOException ex) { + // do nothing + } + } + + public List getServers() + { + return servers.values().stream().collect(Collectors.toList()); + } + + private HostAndPort deserialize(byte[] bytes) + { + String serviceEndpoint = "serviceEndpoint"; + JSONObject data = (JSONObject) JSONValue.parse(new String(bytes)); + if (data != null && data.containsKey(serviceEndpoint)) { + Map hostPortMap = (Map) data.get(serviceEndpoint); + String host = hostPortMap.get("host").toString(); + int port = Integer.parseInt(hostPortMap.get("port").toString()); + return HostAndPort.fromParts(host, port); + } + else { + log.warn("failed to deserialize child node data"); + throw new IllegalArgumentException("No host:port found"); + } + } + + @Override + public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception + { + switch (event.getType()) { + case CHILD_ADDED: + case CHILD_UPDATED: { + HostAndPort hostPort = deserialize(event.getData().getData()); + String node = ZKPaths.getNodeFromPath(event.getData().getPath()); + log.info("child updated: " + node + ": " + hostPort); + servers.put(node, hostPort); + break; + } + + case CHILD_REMOVED: { + String node = ZKPaths.getNodeFromPath(event.getData().getPath()); + log.info("child removed: " + node); + servers.remove(node); + break; + } + + default: + log.info("connection state changed: " + event.getType()); + break; + } + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java new file mode 100644 index 000000000000..351df92d93d2 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java @@ -0,0 +1,67 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive; + +import com.facebook.presto.hive.metastore.thrift.HiveCluster; +import com.facebook.presto.hive.metastore.thrift.HiveMetastoreClient; +import com.google.common.net.HostAndPort; +import io.airlift.log.Logger; +import org.apache.thrift.transport.TTransportException; + +import javax.inject.Inject; + +import java.util.Collections; +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class ZookeeperServersetHiveCluster + implements HiveCluster +{ + private static final Logger log = Logger.get(ZookeeperServersetHiveCluster.class); + private final PooledHiveMetastoreClientFactory clientFactory; + private ZookeeperMetastoreMonitor zkMetastoreMonitor; + + @Inject + public ZookeeperServersetHiveCluster(ZookeeperServersetMetastoreConfig config, PooledHiveMetastoreClientFactory clientFactory) + throws Exception + { + String zkServerHostAndPort = requireNonNull(config.getZookeeperServerHostAndPort(), "zkServerHostAndPort is null"); + String zkMetastorePath = requireNonNull(config.getZookeeperMetastorePath(), "zkMetastorePath is null"); + int zkRetries = requireNonNull(config.getZookeeperMaxRetries(), "zkMaxRetried is null"); + int zkRetrySleepTime = requireNonNull(config.getZookeeperRetrySleepTime(), "zkRetrySleepTime is null"); + this.clientFactory = requireNonNull(clientFactory, "clientFactory is null"); + this.zkMetastoreMonitor = new ZookeeperMetastoreMonitor(zkServerHostAndPort, zkMetastorePath, zkRetries, zkRetrySleepTime); + } + + @Override + public HiveMetastoreClient createMetastoreClient() + { + List metastores = zkMetastoreMonitor.getServers(); + Collections.shuffle(metastores); + TTransportException lastException = null; + for (HostAndPort metastore : metastores) { + try { + log.info("Connecting to metastore at: %s", metastore.toString()); + return clientFactory.create(metastore.getHostText(), metastore.getPort()); + } + catch (TTransportException e) { + log.debug("Failed connecting to Hive metastore at: %s", metastore.toString()); + lastException = e; + } + } + + throw new RuntimeException("Failed connecting to Hive metastore.", lastException); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetMetastoreConfig.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetMetastoreConfig.java new file mode 100644 index 000000000000..65b424b6c437 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetMetastoreConfig.java @@ -0,0 +1,136 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive; + +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; + +import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; + +public class ZookeeperServersetMetastoreConfig +{ + private String zookeeperServerHostAndPort; + private String zookeeperMetastorePath; + private int zookeeperRetrySleepTime = 500; // ms + private int zookeeperMaxRetries = 3; + private int maxTransport = 128; + private long transportIdleTimeout = 300_000L; + private long transportEvictInterval = 10_000L; + private int transportEvictNumTests = 3; + + public String getZookeeperServerHostAndPort() + { + return zookeeperServerHostAndPort; + } + + @Config("hive.metastore.zookeeper.uri") + @ConfigDescription("Zookeeper Host and Port") + public ZookeeperServersetMetastoreConfig setZookeeperServerHostAndPort(String zookeeperServerHostAndPort) + { + this.zookeeperServerHostAndPort = zookeeperServerHostAndPort; + return this; + } + + public String getZookeeperMetastorePath() + { + return zookeeperMetastorePath; + } + + @Config("hive.metastore.zookeeper.path") + @ConfigDescription("Hive metastore Zookeeper path") + public ZookeeperServersetMetastoreConfig setZookeeperMetastorePath(String zkPath) + { + this.zookeeperMetastorePath = zkPath; + return this; + } + + @NotNull + public int getZookeeperRetrySleepTime() + { + return zookeeperRetrySleepTime; + } + + @Config("hive.metastore.zookeeper.retry.sleeptime") + @ConfigDescription("Zookeeper sleep time between reties") + public ZookeeperServersetMetastoreConfig setZookeeperRetrySleepTime(int zookeeperRetrySleepTime) + { + this.zookeeperRetrySleepTime = zookeeperRetrySleepTime; + return this; + } + + @Min(1) + public int getZookeeperMaxRetries() + { + return zookeeperMaxRetries; + } + + @Config("hive.metastore.zookeeper.max.retries") + @ConfigDescription("Zookeeper max reties") + public ZookeeperServersetMetastoreConfig setZookeeperMaxRetries(int zookeeperMaxRetries) + { + this.zookeeperMaxRetries = zookeeperMaxRetries; + return this; + } + + @Min(1) + public int getMaxTransport() + { + return maxTransport; + } + + @Config("hive.metastore.max-transport-num") + public ZookeeperServersetMetastoreConfig setMaxTransport(int maxTransport) + { + this.maxTransport = maxTransport; + return this; + } + + public long getTransportIdleTimeout() + { + return transportIdleTimeout; + } + + @Config("hive.metastore.transport-idle-timeout") + public ZookeeperServersetMetastoreConfig setTransportIdleTimeout(long transportIdleTimeout) + { + this.transportIdleTimeout = transportIdleTimeout; + return this; + } + + public long getTransportEvictInterval() + { + return transportEvictInterval; + } + + @Config("hive.metastore.transport-eviction-interval") + public ZookeeperServersetMetastoreConfig setTransportEvictInterval(long transportEvictInterval) + { + this.transportEvictInterval = transportEvictInterval; + return this; + } + + @Min(0) + public int getTransportEvictNumTests() + { + return transportEvictNumTests; + } + + @Config("hive.metastore.transport-eviction-num-tests") + public ZookeeperServersetMetastoreConfig setTransportEvictNumTests(int transportEvictNumTests) + { + this.transportEvictNumTests = transportEvictNumTests; + return this; + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java new file mode 100644 index 000000000000..077a409fd2a2 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java @@ -0,0 +1,130 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive.thrift; + +import com.fasterxml.jackson.databind.JsonNode; + +import java.util.HashMap; +import java.util.Map; + +import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; +import static com.facebook.presto.hive.HiveUtil.checkCondition; +import static com.google.common.base.MoreObjects.toStringHelper; + +/** + * Resolve the translation of continuous hive ids to discontinuous thrift ids by using a json property. + * Example: + * We have the thrift definition: + * + * struct Name { + * 1: string first, + * 2: string last + * } + * struct Person { + * 1: Name name, + * 3: String phone + * } + * + * Hive table for Person: + * + * +---------+-------------+----------------------------------+-----------------+ + * | hive id | column name | type | thrift field id | + * +---------+-------------+----------------------------------+-----------------+ + * | 0 | name | struct | 1 | + * +---------+-------------+----------------------------------+-----------------+ + * | 1 | phone | string | 3 | + * +---------+-------------+----------------------------------+-----------------+ + * + * The corresponding id mapping object is: + * + * x = { + * '0': { + * '0': 1, + * '1': 2, + * }, + * '1': 3 + * } + * + * The json property is: + * + * {"0":{"0":1,"1":2},"1":3} + */ +public class HiveThriftFieldIdResolver + implements ThriftFieldIdResolver +{ + private final JsonNode root; + private final Map nestedResolvers = new HashMap<>(); + private final Map thriftIds = new HashMap<>(); + + public HiveThriftFieldIdResolver(JsonNode root) + { + this.root = root; + } + + @Override + public short getThriftId(int hiveIndex) + { + if (root == null) { + return (short) (hiveIndex + 1); + } + + Short thriftId = thriftIds.get(hiveIndex); + if (thriftId != null) { + return thriftId; + } + else { + JsonNode child = root.get(String.valueOf(hiveIndex)); + checkCondition(child != null, HIVE_INVALID_METADATA, "Missed json value for hiveIndex: %s, root: %s", hiveIndex, root); + if (child.isNumber()) { + thriftId = (short) child.asInt(); + } + else { + checkCondition(child.get("id") != null, HIVE_INVALID_METADATA, "Missed id for hiveIndex: %s, root: %s", hiveIndex, root); + thriftId = (short) child.get("id").asInt(); + } + thriftIds.put(hiveIndex, thriftId); + return thriftId; + } + } + + @Override + public ThriftFieldIdResolver getNestedResolver(int hiveIndex) + { + if (root == null) { + return this; + } + + ThriftFieldIdResolver nestedResolver = nestedResolvers.get(hiveIndex); + if (nestedResolver != null) { + return nestedResolver; + } + else { + JsonNode child = root.get(String.valueOf(hiveIndex)); + checkCondition(child != null, HIVE_INVALID_METADATA, "Missed json value for hiveIndex: %s, root: %s", hiveIndex, root); + nestedResolver = new HiveThriftFieldIdResolver(child); + nestedResolvers.put(hiveIndex, nestedResolver); + return nestedResolver; + } + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("root", root) + .add("nestedResolvers", nestedResolvers) + .add("thriftIds", thriftIds) + .toString(); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolverFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolverFactory.java new file mode 100644 index 000000000000..c1c736a9f453 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolverFactory.java @@ -0,0 +1,49 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive.thrift; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.airlift.log.Logger; + +import java.io.IOException; +import java.util.Properties; + +public class HiveThriftFieldIdResolverFactory + implements ThriftFieldIdResolverFactory +{ + private static final Logger log = Logger.get(HiveThriftFieldIdResolverFactory.class); + private static final ObjectMapper objectMapper = new ObjectMapper(); + public static final String THRIFT_FIELD_ID_JSON = "thrift.field.id.json"; + // The default resolver which returns thrift id as hive id plus one + public static final ThriftFieldIdResolver HIVE_THRIFT_FIELD_ID_DEFAULT_RESOLVER = new HiveThriftFieldIdResolver(null); + + public ThriftFieldIdResolver createResolver(Properties schema) + { + String jsonData = schema.getProperty(THRIFT_FIELD_ID_JSON); + if (jsonData == null) { + return HIVE_THRIFT_FIELD_ID_DEFAULT_RESOLVER; + } + + try { + JsonNode root = objectMapper.readTree(jsonData); + return new HiveThriftFieldIdResolver(root); + } + catch (IOException e) { + log.debug(e, "Failed to create an optimized thrift id resolver, json string: %s, schema: %s. Will use a default resolver.", jsonData, schema); + } + + return HIVE_THRIFT_FIELD_ID_DEFAULT_RESOLVER; + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolver.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolver.java new file mode 100644 index 000000000000..83047e1d953f --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolver.java @@ -0,0 +1,20 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive.thrift; + +public interface ThriftFieldIdResolver +{ + ThriftFieldIdResolver getNestedResolver(int hiveIndex); + short getThriftId(int hiveIndex); +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolverFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolverFactory.java new file mode 100644 index 000000000000..034308aaa569 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolverFactory.java @@ -0,0 +1,21 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive.thrift; + +import java.util.Properties; + +public interface ThriftFieldIdResolverFactory +{ + ThriftFieldIdResolver createResolver(Properties schema); +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java new file mode 100644 index 000000000000..001994f7a7e4 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java @@ -0,0 +1,50 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive.thrift; + +import com.twitter.elephantbird.mapreduce.io.ThriftWritable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.Writable; +import org.apache.thrift.TException; + +import java.util.Properties; + +import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; +import static com.facebook.presto.hive.HiveErrorCode.HIVE_UNKNOWN_ERROR; +import static com.facebook.presto.hive.HiveUtil.checkCondition; +import static org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS; + +public class ThriftGeneralDeserializer +{ + private static final String REQUIRED_SERIALIZATION_CLASS = ThriftGenericRow.class.getName(); + public ThriftGeneralDeserializer(Configuration conf, Properties properties) + { + String thriftClassName = properties.getProperty(SERIALIZATION_CLASS, null); + checkCondition(thriftClassName != null, HIVE_INVALID_METADATA, "Table or partition is missing Hive deserializer property: %s", SERIALIZATION_CLASS); + checkCondition(thriftClassName.equals(REQUIRED_SERIALIZATION_CLASS), HIVE_INVALID_METADATA, SERIALIZATION_CLASS + thriftClassName + " cannot match " + REQUIRED_SERIALIZATION_CLASS); + } + + public ThriftGenericRow deserialize(Writable writable, short[] thriftIds) + { + checkCondition(writable instanceof ThriftWritable, HIVE_UNKNOWN_ERROR, "Not an instance of ThriftWritable: " + writable); + ThriftGenericRow row = (ThriftGenericRow) ((ThriftWritable) writable).get(); + try { + row.parse(thriftIds); + } + catch (TException e) { + throw new IllegalStateException("ThriftGenericRow failed to parse values", e); + } + return row; + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java new file mode 100644 index 000000000000..1f9690036dbf --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java @@ -0,0 +1,95 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive.thrift; + +import com.facebook.presto.spi.PrestoException; +import com.twitter.elephantbird.mapred.input.DeprecatedFileInputFormatWrapper; +import com.twitter.elephantbird.mapreduce.input.MultiInputFormat; +import com.twitter.elephantbird.mapreduce.io.BinaryWritable; +import com.twitter.elephantbird.util.TypeRef; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapred.Reporter; + +import java.io.IOException; + +import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; +import static com.facebook.presto.hive.HiveUtil.checkCondition; +import static com.facebook.presto.hive.HiveUtil.getLzopIndexPath; +import static com.facebook.presto.hive.HiveUtil.isLzopCompressedFile; +import static java.lang.String.format; +import static org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS; + +/** + * Mirror of com.twitter.elephantbird.mapred.input.HiveMultiInputFormat allows to pass the thriftClassName + * directly as a property of JobConfig and check lzo index existence when check splitability. + * PR for twitter/elephant-bird: + * https://github.com/twitter/elephant-bird/pull/481 + * https://github.com/twitter/elephant-bird/pull/485 + * Remove the class once #481 is included in a release + */ +@SuppressWarnings("deprecation") +public class ThriftGeneralInputFormat + extends DeprecatedFileInputFormatWrapper +{ + public ThriftGeneralInputFormat() + { + super(new MultiInputFormat()); + } + + private void initialize(FileSplit split, JobConf job) throws IOException + { + String thriftClassName = job.get(SERIALIZATION_CLASS); + checkCondition(thriftClassName != null, HIVE_INVALID_METADATA, "Table or partition is missing Hive deserializer property: %s", SERIALIZATION_CLASS); + + try { + Class thriftClass = job.getClassByName(thriftClassName); + setInputFormatInstance(new MultiInputFormat(new TypeRef(thriftClass) {})); + } + catch (ClassNotFoundException e) { + throw new PrestoException(HIVE_INVALID_METADATA, format("Failed getting class for %s", thriftClassName)); + } + } + + @Override + public boolean isSplitable(FileSystem fs, Path filename) + { + if (isLzopCompressedFile(filename)) { + Path indexFile = getLzopIndexPath(filename); + try { + return fs.exists(indexFile); + } + catch (IOException e) { + return false; + } + } + return super.isSplitable(fs, filename); + } + + @Override + public RecordReader getRecordReader( + InputSplit split, + JobConf job, + Reporter reporter) + throws IOException + { + initialize((FileSplit) split, job); + return super.getRecordReader(split, job, reporter); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java new file mode 100644 index 000000000000..5b961f7d813a --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java @@ -0,0 +1,241 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive.thrift; + +import io.airlift.log.Logger; +import org.apache.commons.lang.ArrayUtils; +import org.apache.thrift.TBase; +import org.apache.thrift.TException; +import org.apache.thrift.TFieldIdEnum; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TField; +import org.apache.thrift.protocol.TList; +import org.apache.thrift.protocol.TMap; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.protocol.TProtocolUtil; +import org.apache.thrift.protocol.TSet; +import org.apache.thrift.protocol.TType; +import org.apache.thrift.transport.TMemoryInputTransport; +import org.apache.thrift.transport.TTransport; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class ThriftGenericRow + implements TBase +{ + private static final Logger log = Logger.get(ThriftGenericRow.class); + private final Map values = new HashMap<>(); + private byte[] buf; + private int off; + private int len; + + public ThriftGenericRow() + { + } + + public ThriftGenericRow(Map values) + { + this.values.putAll(values); + } + + public class Fields + implements TFieldIdEnum + { + private final short thriftId; + private final String fieldName; + + Fields(short thriftId, String fieldName) + { + this.thriftId = thriftId; + this.fieldName = fieldName; + } + + public short getThriftFieldId() + { + return thriftId; + } + + public String getFieldName() + { + return fieldName; + } + } + + public void read(TProtocol iprot) + throws TException + { + TTransport trans = iprot.getTransport(); + buf = trans.getBuffer(); + off = trans.getBufferPosition(); + TProtocolUtil.skip(iprot, TType.STRUCT); + len = trans.getBufferPosition() - off; + } + + public void parse() + throws TException + { + parse(null); + } + + public void parse(short[] thriftIds) + throws TException + { + Set idSet = thriftIds == null ? null : new HashSet(Arrays.asList(ArrayUtils.toObject(thriftIds))); + TMemoryInputTransport trans = new TMemoryInputTransport(buf, off, len); + TBinaryProtocol iprot = new TBinaryProtocol(trans); + TField field; + iprot.readStructBegin(); + while (true) { + field = iprot.readFieldBegin(); + if (field.type == TType.STOP) { + break; + } + if (idSet != null && !idSet.remove(Short.valueOf(field.id))) { + TProtocolUtil.skip(iprot, field.type); + } + else { + values.put(field.id, readElem(iprot, field.type)); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + } + + private Object readElem(TProtocol iprot, byte type) + throws TException + { + switch (type) { + case TType.BOOL: + return iprot.readBool(); + case TType.BYTE: + return iprot.readByte(); + case TType.I16: + return iprot.readI16(); + case TType.ENUM: + case TType.I32: + return iprot.readI32(); + case TType.I64: + return iprot.readI64(); + case TType.DOUBLE: + return iprot.readDouble(); + case TType.STRING: + return iprot.readString(); + case TType.STRUCT: + return readStruct(iprot); + case TType.LIST: + return readList(iprot); + case TType.SET: + return readSet(iprot); + case TType.MAP: + return readMap(iprot); + default: + TProtocolUtil.skip(iprot, type); + return null; + } + } + + private Object readStruct(TProtocol iprot) + throws TException + { + ThriftGenericRow elem = new ThriftGenericRow(); + elem.read(iprot); + elem.parse(); + return elem; + } + + private Object readList(TProtocol iprot) + throws TException + { + TList ilist = iprot.readListBegin(); + List listValue = new ArrayList<>(); + for (int i = 0; i < ilist.size; i++) { + listValue.add(readElem(iprot, ilist.elemType)); + } + iprot.readListEnd(); + return listValue; + } + + private Object readSet(TProtocol iprot) + throws TException + { + TSet iset = iprot.readSetBegin(); + List setValue = new ArrayList<>(); + for (int i = 0; i < iset.size; i++) { + setValue.add(readElem(iprot, iset.elemType)); + } + iprot.readSetEnd(); + return setValue; + } + + private Object readMap(TProtocol iprot) + throws TException + { + TMap imap = iprot.readMapBegin(); + Map mapValue = new HashMap<>(); + for (int i = 0; i < imap.size; i++) { + mapValue.put(readElem(iprot, imap.keyType), readElem(iprot, imap.valueType)); + } + iprot.readMapEnd(); + return mapValue; + } + + public Object getFieldValueForThriftId(short thriftId) + { + return values.get(thriftId); + } + + public ThriftGenericRow deepCopy() + { + return new ThriftGenericRow(values); + } + + public void clear() {} + + public Fields fieldForId(int fieldId) + { + return new Fields((short) fieldId, "dummy"); + } + + public Object getFieldValue(Fields field) + { + return values.get(field.thriftId); + } + + public boolean isSet(Fields field) + { + return values.containsKey(field.getThriftFieldId()); + } + + public void setFieldValue(Fields field, Object value) + { + values.put(field.getThriftFieldId(), value); + } + + public void write(TProtocol oprot) + throws TException + { + throw new UnsupportedOperationException("ThriftGenericRow.write is not supported."); + } + + public int compareTo(ThriftGenericRow other) + { + throw new UnsupportedOperationException("ThriftGenericRow.compareTo is not supported."); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java new file mode 100644 index 000000000000..6cd1ba00443f --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -0,0 +1,721 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive.thrift; + +import com.facebook.presto.hive.HiveColumnHandle; +import com.facebook.presto.hive.HiveType; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.RecordCursor; +import com.facebook.presto.spi.block.Block; +import com.facebook.presto.spi.block.BlockBuilder; +import com.facebook.presto.spi.block.BlockBuilderStatus; +import com.facebook.presto.spi.type.DecimalType; +import com.facebook.presto.spi.type.Decimals; +import com.facebook.presto.spi.type.Type; +import com.facebook.presto.spi.type.TypeManager; +import io.airlift.log.Logger; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.type.HiveChar; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.common.type.HiveVarchar; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.RecordReader; +import org.joda.time.DateTimeZone; + +import java.io.IOException; +import java.math.BigInteger; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.HiveErrorCode.HIVE_CURSOR_ERROR; +import static com.facebook.presto.hive.HiveUtil.closeWithSuppression; +import static com.facebook.presto.hive.HiveUtil.isArrayType; +import static com.facebook.presto.hive.HiveUtil.isMapType; +import static com.facebook.presto.hive.HiveUtil.isRowType; +import static com.facebook.presto.hive.HiveUtil.isStructuralType; +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; +import static com.facebook.presto.spi.type.Chars.isCharType; +import static com.facebook.presto.spi.type.Chars.truncateToLengthAndTrimSpaces; +import static com.facebook.presto.spi.type.DateType.DATE; +import static com.facebook.presto.spi.type.Decimals.rescale; +import static com.facebook.presto.spi.type.DoubleType.DOUBLE; +import static com.facebook.presto.spi.type.IntegerType.INTEGER; +import static com.facebook.presto.spi.type.RealType.REAL; +import static com.facebook.presto.spi.type.SmallintType.SMALLINT; +import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP; +import static com.facebook.presto.spi.type.TinyintType.TINYINT; +import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY; +import static com.facebook.presto.spi.type.Varchars.isVarcharType; +import static com.facebook.presto.spi.type.Varchars.truncateToLength; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static java.lang.Float.floatToRawIntBits; +import static java.lang.Math.max; +import static java.lang.Math.min; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +class ThriftHiveRecordCursor + implements RecordCursor +{ + private static final Logger log = Logger.get(ThriftHiveRecordCursor.class); + private static final short NON_EXISTED_THRIFT_ID = (short) -1; + private final RecordReader recordReader; + private final K key; + private final V value; + + private final ThriftGeneralDeserializer deserializer; + + private final Type[] types; + private final HiveType[] hiveTypes; + private final int[] hiveIndexs; + private final short[] thriftIds; + + private final boolean[] loaded; + private final boolean[] booleans; + private final long[] longs; + private final double[] doubles; + private final Slice[] slices; + private final Object[] objects; + private final boolean[] nulls; + + private final Path path; + private final long start; + private final long totalBytes; + private final DateTimeZone hiveStorageTimeZone; + + private final ThriftFieldIdResolver thriftFieldIdResolver; + + private long completedBytes; + private ThriftGenericRow rowData; + private boolean closed; + + public ThriftHiveRecordCursor( + RecordReader recordReader, + Path path, + long start, + long totalBytes, + Properties splitSchema, + List columns, + DateTimeZone hiveStorageTimeZone, + TypeManager typeManager, + ThriftFieldIdResolver thriftFieldIdResolver) + { + requireNonNull(recordReader, "recordReader is null"); + requireNonNull(path, "path is null"); + checkArgument(start >= 0, "start is negative"); + checkArgument(totalBytes >= 0, "totalBytes is negative"); + requireNonNull(splitSchema, "splitSchema is null"); + requireNonNull(columns, "columns is null"); + requireNonNull(hiveStorageTimeZone, "hiveStorageTimeZone is null"); + requireNonNull(thriftFieldIdResolver, "thriftFieldIdResolver is null"); + + this.recordReader = recordReader; + this.path = path; + this.start = start; + this.totalBytes = totalBytes; + this.key = recordReader.createKey(); + this.value = recordReader.createValue(); + this.hiveStorageTimeZone = hiveStorageTimeZone; + this.thriftFieldIdResolver = thriftFieldIdResolver; + + this.deserializer = new ThriftGeneralDeserializer(new Configuration(false), splitSchema); + + int size = columns.size(); + + this.types = new Type[size]; + this.hiveTypes = new HiveType[size]; + this.hiveIndexs = new int[size]; + this.thriftIds = new short[size]; + + this.loaded = new boolean[size]; + this.booleans = new boolean[size]; + this.longs = new long[size]; + this.doubles = new double[size]; + this.slices = new Slice[size]; + this.objects = new Object[size]; + this.nulls = new boolean[size]; + + // initialize data columns + for (int i = 0; i < columns.size(); i++) { + HiveColumnHandle column = columns.get(i); + checkState(column.getColumnType() == REGULAR, "column type must be regular"); + + types[i] = typeManager.getType(column.getTypeSignature()); + hiveTypes[i] = column.getHiveType(); + hiveIndexs[i] = column.getHiveColumnIndex(); + thriftIds[i] = getThriftIdWithFailOver(thriftFieldIdResolver, hiveIndexs[i]); + } + + // close immediately if the number of totalBytes is zero + if (totalBytes == 0) { + close(); + } + } + + @Override + public long getCompletedBytes() + { + if (!closed) { + updateCompletedBytes(); + } + return completedBytes; + } + + @Override + public long getReadTimeNanos() + { + return 0; + } + + private void updateCompletedBytes() + { + try { + long newCompletedBytes = (long) (totalBytes * recordReader.getProgress()); + completedBytes = min(totalBytes, max(completedBytes, newCompletedBytes)); + } + catch (IOException ignored) { + } + } + + @Override + public Type getType(int field) + { + return types[field]; + } + + @Override + public boolean advanceNextPosition() + { + try { + if (closed || !recordReader.next(key, value)) { + close(); + return false; + } + + // reset loaded flags + Arrays.fill(loaded, false); + + // decode value + rowData = deserializer.deserialize(value, thriftIds); + + return true; + } + catch (IOException | RuntimeException e) { + closeWithSuppression(this, e); + throw new PrestoException(HIVE_CURSOR_ERROR, + format("Failed to read split: %s %s:%s, total bytes: %s, completed bytes: %s", + path, start, start + totalBytes, totalBytes, completedBytes), + e); + } + } + + @Override + public boolean getBoolean(int fieldId) + { + checkState(!closed, "Cursor is closed"); + + validateType(fieldId, boolean.class); + if (!loaded[fieldId]) { + parseBooleanColumn(fieldId); + } + return booleans[fieldId]; + } + + private void parseBooleanColumn(int column) + { + loaded[column] = true; + + Object fieldValue = rowData.getFieldValueForThriftId(thriftIds[column]); + + if (fieldValue == null) { + nulls[column] = true; + } + else { + booleans[column] = (Boolean) fieldValue; + nulls[column] = false; + } + } + + @Override + public long getLong(int fieldId) + { + checkState(!closed, "Cursor is closed"); + + validateType(fieldId, long.class); + if (!loaded[fieldId]) { + parseLongColumn(fieldId); + } + return longs[fieldId]; + } + + private void parseLongColumn(int column) + { + loaded[column] = true; + + Object fieldValue = rowData.getFieldValueForThriftId(thriftIds[column]); + + if (fieldValue == null) { + nulls[column] = true; + } + else { + longs[column] = getLongExpressedValue(fieldValue, hiveStorageTimeZone); + nulls[column] = false; + } + } + + private static long getLongExpressedValue(Object value, DateTimeZone hiveTimeZone) + { + if (value instanceof Date) { + long storageTime = ((Date) value).getTime(); + // convert date from VM current time zone to UTC + long utcMillis = storageTime + DateTimeZone.getDefault().getOffset(storageTime); + return TimeUnit.MILLISECONDS.toDays(utcMillis); + } + if (value instanceof Timestamp) { + // The Hive SerDe parses timestamps using the default time zone of + // this JVM, but the data might have been written using a different + // time zone. We need to convert it to the configured time zone. + + // the timestamp that Hive parsed using the JVM time zone + long parsedJvmMillis = ((Timestamp) value).getTime(); + + // remove the JVM time zone correction from the timestamp + DateTimeZone jvmTimeZone = DateTimeZone.getDefault(); + long hiveMillis = jvmTimeZone.convertUTCToLocal(parsedJvmMillis); + + // convert to UTC using the real time zone for the underlying data + long utcMillis = hiveTimeZone.convertLocalToUTC(hiveMillis, false); + + return utcMillis; + } + if (value instanceof Float) { + return floatToRawIntBits(((Float) value)); + } + return ((Number) value).longValue(); + } + + @Override + public double getDouble(int fieldId) + { + checkState(!closed, "Cursor is closed"); + + validateType(fieldId, double.class); + if (!loaded[fieldId]) { + parseDoubleColumn(fieldId); + } + return doubles[fieldId]; + } + + private void parseDoubleColumn(int column) + { + loaded[column] = true; + + Object fieldValue = rowData.getFieldValueForThriftId(thriftIds[column]); + + if (fieldValue == null) { + nulls[column] = true; + } + else { + doubles[column] = ((Number) fieldValue).doubleValue(); + nulls[column] = false; + } + } + + @Override + public Slice getSlice(int fieldId) + { + checkState(!closed, "Cursor is closed"); + + validateType(fieldId, Slice.class); + if (!loaded[fieldId]) { + parseStringColumn(fieldId); + } + return slices[fieldId]; + } + + private void parseStringColumn(int column) + { + loaded[column] = true; + + Object fieldValue = rowData.getFieldValueForThriftId(thriftIds[column]); + + if (fieldValue == null) { + nulls[column] = true; + } + else { + slices[column] = getSliceExpressedValue(fieldValue, types[column]); + nulls[column] = false; + } + } + + private static Slice getSliceExpressedValue(Object value, Type type) + { + Slice sliceValue; + if (value instanceof String) { + sliceValue = Slices.utf8Slice((String) value); + } + else if (value instanceof byte[]) { + sliceValue = Slices.wrappedBuffer((byte[]) value); + } + else if (value instanceof HiveVarchar) { + sliceValue = Slices.utf8Slice(((HiveVarchar) value).getValue()); + } + else if (value instanceof HiveChar) { + sliceValue = Slices.utf8Slice(((HiveChar) value).getValue()); + } + else if (value instanceof Integer) { + sliceValue = Slices.utf8Slice(value.toString()); + } + else { + throw new IllegalStateException("unsupported string field type: " + value.getClass().getName()); + } + if (isVarcharType(type)) { + sliceValue = truncateToLength(sliceValue, type); + } + if (isCharType(type)) { + sliceValue = truncateToLengthAndTrimSpaces(sliceValue, type); + } + + return sliceValue; + } + + private void parseDecimalColumn(int column) + { + loaded[column] = true; + + Object fieldValue = rowData.getFieldValueForThriftId(thriftIds[column]); + + if (fieldValue == null) { + nulls[column] = true; + } + else { + HiveDecimal decimal = (HiveDecimal) fieldValue; + DecimalType columnType = (DecimalType) types[column]; + BigInteger unscaledDecimal = rescale(decimal.unscaledValue(), decimal.scale(), columnType.getScale()); + + if (columnType.isShort()) { + longs[column] = unscaledDecimal.longValue(); + } + else { + slices[column] = Decimals.encodeUnscaledValue(unscaledDecimal); + } + nulls[column] = false; + } + } + + @Override + public Object getObject(int fieldId) + { + checkState(!closed, "Cursor is closed"); + + validateType(fieldId, Block.class); + if (!loaded[fieldId]) { + parseObjectColumn(fieldId); + } + return objects[fieldId]; + } + + private void parseObjectColumn(int column) + { + loaded[column] = true; + + Object fieldValue = rowData.getFieldValueForThriftId(thriftIds[column]); + + if (fieldValue == null) { + nulls[column] = true; + } + else { + ThriftFieldIdResolver resolver = thriftFieldIdResolver.getNestedResolver(hiveIndexs[column]); + objects[column] = getBlockObject(types[column], resolver, fieldValue, hiveStorageTimeZone); + nulls[column] = false; + } + } + + @Override + public boolean isNull(int fieldId) + { + checkState(!closed, "Cursor is closed"); + + if (!loaded[fieldId]) { + parseColumn(fieldId); + } + return nulls[fieldId]; + } + + private void parseColumn(int column) + { + Type type = types[column]; + if (BOOLEAN.equals(type)) { + parseBooleanColumn(column); + } + else if (BIGINT.equals(type)) { + parseLongColumn(column); + } + else if (INTEGER.equals(type)) { + parseLongColumn(column); + } + else if (SMALLINT.equals(type)) { + parseLongColumn(column); + } + else if (TINYINT.equals(type)) { + parseLongColumn(column); + } + else if (REAL.equals(type)) { + parseLongColumn(column); + } + else if (DOUBLE.equals(type)) { + parseDoubleColumn(column); + } + else if (isVarcharType(type) || VARBINARY.equals(type)) { + parseStringColumn(column); + } + else if (isCharType(type)) { + parseStringColumn(column); + } + else if (isStructuralType(hiveTypes[column])) { + parseObjectColumn(column); + } + else if (DATE.equals(type)) { + parseLongColumn(column); + } + else if (TIMESTAMP.equals(type)) { + parseLongColumn(column); + } + else if (type instanceof DecimalType) { + parseDecimalColumn(column); + } + else { + throw new UnsupportedOperationException("Unsupported column type: " + type); + } + } + + private void validateType(int fieldId, Class type) + { + if (!types[fieldId].getJavaType().equals(type)) { + // we don't use Preconditions.checkArgument because it requires boxing fieldId, which affects inner loop performance + throw new IllegalArgumentException(String.format("Expected field to be %s, actual %s (field %s)", type, types[fieldId], fieldId)); + } + } + + @Override + public void close() + { + // some hive input formats are broken and bad things can happen if you close them multiple times + if (closed) { + return; + } + closed = true; + + updateCompletedBytes(); + + try { + recordReader.close(); + } + catch (IOException e) { + throw new RuntimeException("Error closing thrift record reader", e); + } + } + + private static Block getBlockObject(Type type, ThriftFieldIdResolver resolver, Object object, DateTimeZone hiveStorageTimeZone) + { + return requireNonNull(serializeObject(type, resolver, null, object, hiveStorageTimeZone), "serialized result is null"); + } + + private static Block serializeObject(Type type, ThriftFieldIdResolver resolver, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) + { + if (object == null) { + requireNonNull(builder, "parent builder is null").appendNull(); + return null; + } + if (!isStructuralType(type)) { + serializePrimitive(type, resolver, builder, object, hiveStorageTimeZone); + return null; + } + else if (isArrayType(type)) { + return serializeList(type, resolver, builder, object, hiveStorageTimeZone); + } + else if (isMapType(type)) { + return serializeMap(type, resolver, builder, object, hiveStorageTimeZone); + } + else if (isRowType(type)) { + return serializeStruct(type, resolver, builder, object, hiveStorageTimeZone); + } + throw new RuntimeException("Unknown object type: " + type); + } + + private static Block serializeList(Type type, ThriftFieldIdResolver resolver, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) + { + List list = (List) requireNonNull(object, "object is null"); + List typeParameters = type.getTypeParameters(); + checkArgument(typeParameters.size() == 1, "list must have exactly 1 type parameter"); + Type elementType = typeParameters.get(0); + ThriftFieldIdResolver elementResolver = resolver.getNestedResolver(0); + BlockBuilder currentBuilder; + if (builder != null) { + currentBuilder = builder.beginBlockEntry(); + } + else { + currentBuilder = elementType.createBlockBuilder(new BlockBuilderStatus(), list.size()); + } + + for (Object element : list) { + serializeObject(elementType, elementResolver, currentBuilder, element, hiveStorageTimeZone); + } + + if (builder != null) { + builder.closeEntry(); + return null; + } + else { + Block resultBlock = currentBuilder.build(); + return resultBlock; + } + } + + private static Block serializeMap(Type type, ThriftFieldIdResolver resolver, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) + { + Map map = (Map) requireNonNull(object, "object is null"); + List typeParameters = type.getTypeParameters(); + checkArgument(typeParameters.size() == 2, "map must have exactly 2 type parameter"); + Type keyType = typeParameters.get(0); + Type valueType = typeParameters.get(1); + ThriftFieldIdResolver keyResolver = resolver.getNestedResolver(0); + ThriftFieldIdResolver valueResolver = resolver.getNestedResolver(1); + boolean builderSynthesized = false; + if (builder == null) { + builderSynthesized = true; + builder = type.createBlockBuilder(new BlockBuilderStatus(), 1); + } + BlockBuilder currentBuilder = builder.beginBlockEntry(); + + for (Map.Entry entry : map.entrySet()) { + // Hive skips map entries with null keys + if (entry.getKey() != null) { + serializeObject(keyType, keyResolver, currentBuilder, entry.getKey(), hiveStorageTimeZone); + serializeObject(valueType, valueResolver, currentBuilder, entry.getValue(), hiveStorageTimeZone); + } + } + + builder.closeEntry(); + if (builderSynthesized) { + return (Block) type.getObject(builder, 0); + } + else { + return null; + } + } + + private static Block serializeStruct(Type type, ThriftFieldIdResolver resolver, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) + { + ThriftGenericRow structData = (ThriftGenericRow) requireNonNull(object, "object is null"); + List typeParameters = type.getTypeParameters(); + + boolean builderSynthesized = false; + if (builder == null) { + builderSynthesized = true; + builder = type.createBlockBuilder(new BlockBuilderStatus(), 1); + } + BlockBuilder currentBuilder = builder.beginBlockEntry(); + + for (int i = 0; i < typeParameters.size(); i++) { + Object fieldValue = structData.getFieldValueForThriftId(getThriftIdWithFailOver(resolver, i)); + if (fieldValue == null) { + currentBuilder.appendNull(); + } + else { + serializeObject(typeParameters.get(i), resolver.getNestedResolver(i), currentBuilder, fieldValue, hiveStorageTimeZone); + } + } + + builder.closeEntry(); + if (builderSynthesized) { + return (Block) type.getObject(builder, 0); + } + else { + return null; + } + } + + private static void serializePrimitive(Type type, ThriftFieldIdResolver resolver, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) + { + requireNonNull(builder, "parent builder is null"); + requireNonNull(object, "object is null"); + + if (BOOLEAN.equals(type)) { + BOOLEAN.writeBoolean(builder, (Boolean) object); + } + else if (BIGINT.equals(type)) { + BIGINT.writeLong(builder, getLongExpressedValue(object, hiveStorageTimeZone)); + } + else if (INTEGER.equals(type)) { + INTEGER.writeLong(builder, getLongExpressedValue(object, hiveStorageTimeZone)); + } + else if (SMALLINT.equals(type)) { + SMALLINT.writeLong(builder, getLongExpressedValue(object, hiveStorageTimeZone)); + } + else if (TINYINT.equals(type)) { + TINYINT.writeLong(builder, getLongExpressedValue(object, hiveStorageTimeZone)); + } + else if (REAL.equals(type)) { + REAL.writeLong(builder, getLongExpressedValue(object, hiveStorageTimeZone)); + } + else if (DOUBLE.equals(type)) { + DOUBLE.writeDouble(builder, ((Number) object).doubleValue()); + } + else if (isVarcharType(type) || VARBINARY.equals(type) || isCharType(type)) { + type.writeSlice(builder, getSliceExpressedValue(object, type)); + } + else if (DATE.equals(type)) { + DATE.writeLong(builder, getLongExpressedValue(object, hiveStorageTimeZone)); + } + else if (TIMESTAMP.equals(type)) { + TIMESTAMP.writeLong(builder, getLongExpressedValue(object, hiveStorageTimeZone)); + } + else if (type instanceof DecimalType) { + HiveDecimal decimal = (HiveDecimal) object; + DecimalType decimalType = (DecimalType) type; + BigInteger unscaledDecimal = rescale(decimal.unscaledValue(), decimal.scale(), decimalType.getScale()); + if (decimalType.isShort()) { + decimalType.writeLong(builder, unscaledDecimal.longValue()); + } + else { + decimalType.writeSlice(builder, Decimals.encodeUnscaledValue(unscaledDecimal)); + } + } + else { + throw new UnsupportedOperationException("Unsupported primitive type: " + type); + } + } + + private static short getThriftIdWithFailOver(ThriftFieldIdResolver thriftFieldIdResolver, int hiveIndex) + { + try { + return thriftFieldIdResolver.getThriftId(hiveIndex); + } + catch (PrestoException e) { + return NON_EXISTED_THRIFT_ID; + } + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursorProvider.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursorProvider.java new file mode 100644 index 000000000000..8c10e78f5f33 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursorProvider.java @@ -0,0 +1,163 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive.thrift; + +import com.facebook.presto.hive.HdfsEnvironment; +import com.facebook.presto.hive.HiveColumnHandle; +import com.facebook.presto.hive.HiveRecordCursorProvider; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.RecordCursor; +import com.facebook.presto.spi.predicate.TupleDomain; +import com.facebook.presto.spi.type.TypeManager; +import com.google.common.collect.ImmutableSet; +import com.hadoop.compression.lzo.LzoIndex; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.RecordReader; +import org.joda.time.DateTimeZone; + +import javax.inject.Inject; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; + +import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; +import static com.facebook.presto.hive.HiveStorageFormat.THRIFTBINARY; +import static com.facebook.presto.hive.HiveUtil.checkCondition; +import static com.facebook.presto.hive.HiveUtil.createRecordReader; +import static com.facebook.presto.hive.HiveUtil.getDeserializerClassName; +import static com.facebook.presto.hive.HiveUtil.getLzopIndexPath; +import static com.facebook.presto.hive.HiveUtil.isLzopCompressedFile; +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS; + +public class ThriftHiveRecordCursorProvider + implements HiveRecordCursorProvider +{ + private static final String THRIFT_GENERIC_ROW = ThriftGenericRow.class.getName(); + private static final Set THRIFT_SERDE_CLASS_NAMES = ImmutableSet.builder() + .add(ThriftGeneralDeserializer.class.getName()) + .add(THRIFTBINARY.getSerDe()) + .build(); + private final HdfsEnvironment hdfsEnvironment; + private final ThriftFieldIdResolverFactory thriftFieldIdResolverFactory; + + @Inject + public ThriftHiveRecordCursorProvider(HdfsEnvironment hdfsEnvironment, ThriftFieldIdResolverFactory thriftFieldIdResolverFactory) + { + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + this.thriftFieldIdResolverFactory = requireNonNull(thriftFieldIdResolverFactory, "thriftFieldIdResolverFactory is null"); + } + + @Override + public Optional createRecordCursor( + Configuration configuration, + ConnectorSession session, + Path path, + long start, + long length, + long fileSize, + Properties schema, + List columns, + TupleDomain effectivePredicate, + DateTimeZone hiveStorageTimeZone, + TypeManager typeManager) + { + if (!THRIFT_SERDE_CLASS_NAMES.contains(getDeserializerClassName(schema))) { + return Optional.empty(); + } + + // We only allow the table which specified its serialization class is compatible to + // our thrift general row, if the SerDe is LazyBinarySerDe. + if (THRIFTBINARY.getSerDe().equals(getDeserializerClassName(schema)) && !THRIFT_GENERIC_ROW.equals(getSerializationClassName(schema))) { + return Optional.empty(); + } + + setPropertyIfUnset(schema, "elephantbird.mapred.input.bad.record.check.only.in.close", Boolean.toString(false)); + setPropertyIfUnset(schema, "elephantbird.mapred.input.bad.record.threshold", Float.toString(0.0f)); + + // re-align split range + if (isLzopCompressedFile(path)) { + LzoIndex index = new LzoIndex(); + try { + index = LzoIndex.readIndex(hdfsEnvironment.getFileSystem(session.getUser(), getLzopIndexPath(path), configuration), path); + } + catch (IOException ignored) { + // ignored + } + + // re-align split start + if (index.isEmpty() && start != 0) { + // empty index and split not start from beginning + start = LzoIndex.NOT_FOUND; + } + if (!index.isEmpty()) { + // align start based on index + start = index.alignSliceStartToIndex(start, start + length); + } + + // re-align split end + if (start == LzoIndex.NOT_FOUND) { + // split start cannot be find, then we should skip this split + start = 0; + length = 0; + } + else { + // if index is empty but start is not NOT_FOUND, read the whole file, otherwise align split end. + length = index.isEmpty() ? fileSize : + (Math.min(index.alignSliceEndToIndex(start + length, fileSize), fileSize) - start); + } + } + + long finalStart = start; + long finalLength = length; + RecordReader recordReader = hdfsEnvironment.doAs(session.getUser(), + () -> createRecordReader(configuration, path, finalStart, finalLength, schema, columns)); + + return Optional.of(new ThriftHiveRecordCursor<>( + genericRecordReader(recordReader), + path, + start, + length, + schema, + columns, + hiveStorageTimeZone, + typeManager, + thriftFieldIdResolverFactory.createResolver(schema))); + } + + @SuppressWarnings("unchecked") + private static RecordReader genericRecordReader(RecordReader recordReader) + { + return (RecordReader) recordReader; + } + + private static void setPropertyIfUnset(Properties schema, String key, String value) + { + if (schema.getProperty(key) == null) { + schema.setProperty(key, value); + } + } + + private static String getSerializationClassName(Properties schema) + { + String name = schema.getProperty(SERIALIZATION_CLASS); + checkCondition(name != null, HIVE_INVALID_METADATA, "Table or partition is missing Hive property: %s", SERIALIZATION_CLASS); + return name; + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java new file mode 100644 index 000000000000..554435e12923 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java @@ -0,0 +1,261 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive.util; + +import com.facebook.presto.hive.authentication.HiveMetastoreAuthentication; +import com.google.common.net.HostAndPort; +import org.apache.commons.pool2.BasePooledObjectFactory; +import org.apache.commons.pool2.PooledObject; +import org.apache.commons.pool2.impl.DefaultPooledObject; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.Proxy; +import java.net.Socket; +import java.net.SocketAddress; +import java.net.SocketException; + +import static java.util.Objects.requireNonNull; + +public class PooledTTransportFactory + extends BasePooledObjectFactory +{ + private final TTransportPool pool; + private final String host; + private final int port; + private final HostAndPort socksProxy; + private final int timeoutMillis; + private final HiveMetastoreAuthentication metastoreAuthentication; + + public PooledTTransportFactory(TTransportPool pool, + String host, + int port, + @Nullable HostAndPort socksProxy, + int timeoutMillis, + HiveMetastoreAuthentication metastoreAuthentication) + { + this.pool = requireNonNull(pool, "pool is null"); + this.host = requireNonNull(host, "host is null"); + this.port = port; + this.socksProxy = socksProxy; + this.timeoutMillis = timeoutMillis; + this.metastoreAuthentication = requireNonNull(metastoreAuthentication, "metastoreAuthentication is null"); + } + + @Override + public void activateObject(PooledObject pooledObject) + throws Exception + { + pooledObject.getObject().flush(); + } + + @Override + public boolean validateObject(PooledObject pooledObject) + { + try { + return (pooledObject.getObject().isOpen() && + ((PooledTTransport) pooledObject.getObject()).isReachable(timeoutMillis)); + } + catch (Exception e) { + return false; + } + } + + @Override + public TTransport create() + throws Exception + { + TTransport transport; + if (socksProxy == null) { + transport = new TSocket(host, port, timeoutMillis); + } + else { + SocketAddress address = InetSocketAddress.createUnresolved(socksProxy.getHostText(), + socksProxy.getPort()); + Socket socket = new Socket(new Proxy(Proxy.Type.SOCKS, address)); + try { + socket.connect(InetSocketAddress.createUnresolved(host, port), timeoutMillis); + socket.setSoTimeout(timeoutMillis); + transport = new TSocket(socket); + } + catch (SocketException e) { + if (socket.isConnected()) { + try { + socket.close(); + } + catch (IOException ioException) { + // ignored + } + } + throw e; + } + } + TTransport authenticatedTransport = metastoreAuthentication.authenticate(transport, host); + if (!authenticatedTransport.isOpen()) { + authenticatedTransport.open(); + } + + return new PooledTTransport(authenticatedTransport, pool, + HostAndPort.fromParts(host, port).toString()); + } + + @Override + public void destroyObject(PooledObject pooledObject) + { + try { + ((PooledTTransport) pooledObject.getObject()).getTTransport().close(); + } + catch (ClassCastException e) { + // ignore + } + pooledObject.invalidate(); + } + + @Override + public PooledObject wrap(TTransport transport) + { + return new DefaultPooledObject(transport); + } + + @Override + public void passivateObject(PooledObject pooledObject) + { + try { + pooledObject.getObject().flush(); + } + catch (TTransportException e) { + destroyObject(pooledObject); + } + } + + private static class PooledTTransport + extends TTransport + { + private final String remote; + private final TTransportPool pool; + private final TTransport transport; + + public PooledTTransport(TTransport transport, TTransportPool pool, String remote) + { + this.transport = transport; + this.pool = pool; + this.remote = remote; + } + + public TTransport getTTransport() + { + return transport; + } + + public boolean isReachable(int timeoutMillis) + throws ClassCastException, IOException + { + return ((TSocket) transport).getSocket().getInetAddress().isReachable(timeoutMillis); + } + + @Override + public void close() + { + try { + pool.returnObject(remote, this, transport); + } + catch (Exception e) { + transport.close(); + } + } + + @Override + public boolean isOpen() + { + return transport.isOpen(); + } + + @Override + public boolean peek() + { + return transport.peek(); + } + + @Override + public byte[] getBuffer() + { + return transport.getBuffer(); + } + + @Override + public int getBufferPosition() + { + return transport.getBufferPosition(); + } + + @Override + public int getBytesRemainingInBuffer() + { + return transport.getBytesRemainingInBuffer(); + } + + @Override + public void consumeBuffer(int len) + { + transport.consumeBuffer(len); + } + + @Override + public void open() + throws TTransportException + { + transport.open(); + } + + @Override + public int readAll(byte[] bytes, int off, int len) + throws TTransportException + { + return transport.readAll(bytes, off, len); + } + + @Override + public int read(byte[] bytes, int off, int len) + throws TTransportException + { + return transport.read(bytes, off, len); + } + + @Override + public void write(byte[] bytes) + throws TTransportException + { + transport.write(bytes); + } + + @Override + public void write(byte[] bytes, int off, int len) + throws TTransportException + { + transport.write(bytes, off, len); + } + + @Override + public void flush() + throws TTransportException + { + transport.flush(); + } + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java new file mode 100644 index 000000000000..7d2a3b5af23b --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java @@ -0,0 +1,93 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive.util; + +import com.google.common.net.HostAndPort; +import org.apache.commons.pool2.ObjectPool; +import org.apache.commons.pool2.PooledObjectFactory; +import org.apache.commons.pool2.impl.GenericObjectPool; +import org.apache.commons.pool2.impl.GenericObjectPoolConfig; +import org.apache.thrift.transport.TTransport; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +public class TTransportPool +{ + private final ConcurrentMap> pools = new ConcurrentHashMap(); + private GenericObjectPoolConfig poolConfig; + + public TTransportPool(GenericObjectPoolConfig poolConfig) + { + this.poolConfig = poolConfig; + } + + protected synchronized void add(String remote, PooledObjectFactory transportFactory) + { + pools.putIfAbsent(remote, new GenericObjectPool(transportFactory, poolConfig)); + } + + protected TTransport get(String remote, PooledObjectFactory transportFactory) + throws Exception + { + add(remote, transportFactory); + return get(remote); + } + + protected TTransport get(String remote) + throws Exception + { + ObjectPool pool = pools.get(remote); + if (pool == null) { + return null; + } + return pool.borrowObject(); + } + + public TTransport borrowObject(String host, int port, PooledObjectFactory transportFactory) + throws Exception + { + return get(HostAndPort.fromParts(host, port).toString(), transportFactory); + } + + public TTransport borrowObject(String host, int port) + throws Exception + { + return get(HostAndPort.fromParts(host, port).toString()); + } + + public void returnObject(String remote, TTransport pooledTransport, TTransport transport) + { + if (remote == null) { + transport.close(); + return; + } + ObjectPool pool = pools.get(remote); + if (pool == null) { + transport.close(); + return; + } + try { + pool.returnObject(pooledTransport); + } + catch (Exception e) { + transport.close(); + } + } + + public void returnObject(TTransport transport) + { + transport.close(); + } +} diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java index 3c09636808d6..1ddb7a1ec847 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java @@ -142,6 +142,7 @@ import static com.facebook.presto.hive.HiveStorageFormat.RCTEXT; import static com.facebook.presto.hive.HiveStorageFormat.SEQUENCEFILE; import static com.facebook.presto.hive.HiveStorageFormat.TEXTFILE; +import static com.facebook.presto.hive.HiveStorageFormat.THRIFTBINARY; import static com.facebook.presto.hive.HiveTableProperties.BUCKETED_BY_PROPERTY; import static com.facebook.presto.hive.HiveTableProperties.BUCKET_COUNT_PROPERTY; import static com.facebook.presto.hive.HiveTableProperties.PARTITIONED_BY_PROPERTY; @@ -382,7 +383,7 @@ private static RowType toRowType(List columns) }).collect(toList())) .build(); - protected Set createTableFormats = difference(ImmutableSet.copyOf(HiveStorageFormat.values()), ImmutableSet.of(AVRO)); + protected Set createTableFormats = difference(ImmutableSet.copyOf(HiveStorageFormat.values()), ImmutableSet.of(AVRO, THRIFTBINARY)); private static final JoinCompiler JOIN_COMPILER = new JoinCompiler(); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java index d0e53b397e4d..14bf8df53f3a 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java @@ -28,6 +28,9 @@ import com.facebook.presto.spi.type.ArrayType; import com.facebook.presto.spi.type.RowType; import com.facebook.presto.testing.TestingConnectorSession; +import com.facebook.presto.twitter.hive.thrift.HiveThriftFieldIdResolverFactory; +import com.facebook.presto.twitter.hive.thrift.ThriftGenericRow; +import com.facebook.presto.twitter.hive.thrift.ThriftHiveRecordCursorProvider; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -70,6 +73,7 @@ import static com.facebook.presto.hive.HiveStorageFormat.RCTEXT; import static com.facebook.presto.hive.HiveStorageFormat.SEQUENCEFILE; import static com.facebook.presto.hive.HiveStorageFormat.TEXTFILE; +import static com.facebook.presto.hive.HiveStorageFormat.THRIFTBINARY; import static com.facebook.presto.hive.HiveTestUtils.HDFS_ENVIRONMENT; import static com.facebook.presto.hive.HiveTestUtils.SESSION; import static com.facebook.presto.hive.HiveTestUtils.TYPE_MANAGER; @@ -86,6 +90,7 @@ import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.toList; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.FILE_INPUT_FORMAT; +import static org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS; import static org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_LIB; import static org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.getStandardListObjectInspector; import static org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.getStandardStructObjectInspector; @@ -492,6 +497,44 @@ public void testParquetThrift(int rowCount) testCursorProvider(cursorProvider, split, PARQUET, testColumns, 1); } + @Test(dataProvider = "rowCount") + public void testLZOThrift(int rowCount) + throws Exception + { + RowType nameType = new RowType(ImmutableList.of(createUnboundedVarcharType(), createUnboundedVarcharType()), Optional.empty()); + RowType phoneType = new RowType(ImmutableList.of(createUnboundedVarcharType(), createUnboundedVarcharType()), Optional.empty()); + RowType personType = new RowType(ImmutableList.of(nameType, INTEGER, createUnboundedVarcharType(), new ArrayType(phoneType)), Optional.empty()); + + List testColumns = ImmutableList.of( + new TestColumn( + "persons", + getStandardListObjectInspector( + getStandardStructObjectInspector( + ImmutableList.of("name", "id", "email", "phones"), + ImmutableList.of( + getStandardStructObjectInspector( + ImmutableList.of("first_name", "last_name"), + ImmutableList.of(javaStringObjectInspector, javaStringObjectInspector)), + javaIntObjectInspector, + javaStringObjectInspector, + getStandardListObjectInspector( + getStandardStructObjectInspector( + ImmutableList.of("number", "type"), + ImmutableList.of(javaStringObjectInspector, javaStringObjectInspector)))))), + null, + arrayBlockOf(personType, + rowBlockOf(ImmutableList.of(nameType, INTEGER, createUnboundedVarcharType(), new ArrayType(phoneType)), + rowBlockOf(ImmutableList.of(createUnboundedVarcharType(), createUnboundedVarcharType()), "Bob", "Roberts"), + 0, + "bob.roberts@example.com", + arrayBlockOf(phoneType, rowBlockOf(ImmutableList.of(createUnboundedVarcharType(), createUnboundedVarcharType()), "1234567890", null)))))); + + File file = new File(this.getClass().getClassLoader().getResource("addressbook.thrift.lzo").getPath()); + FileSplit split = new FileSplit(new Path(file.getAbsolutePath()), 0, file.length(), new String[0]); + HiveRecordCursorProvider cursorProvider = new ThriftHiveRecordCursorProvider(HDFS_ENVIRONMENT, new HiveThriftFieldIdResolverFactory()); + testCursorProvider(cursorProvider, split, THRIFTBINARY, testColumns, 1); + } + @Test(dataProvider = "rowCount") public void testDwrf(int rowCount) throws Exception @@ -654,7 +697,9 @@ private void testCursorProvider(HiveRecordCursorProvider cursorProvider, splitProperties.setProperty(SERIALIZATION_LIB, storageFormat.getSerDe()); splitProperties.setProperty("columns", Joiner.on(',').join(transform(filter(testColumns, not(TestColumn::isPartitionKey)), TestColumn::getName))); splitProperties.setProperty("columns.types", Joiner.on(',').join(transform(filter(testColumns, not(TestColumn::isPartitionKey)), TestColumn::getType))); - + if (storageFormat.equals(THRIFTBINARY)) { + splitProperties.setProperty(SERIALIZATION_CLASS, ThriftGenericRow.class.getName()); + } List partitionKeys = testColumns.stream() .filter(TestColumn::isPartitionKey) .map(input -> new HivePartitionKey(input.getName(), (String) input.getWriteValue())) @@ -662,6 +707,9 @@ private void testCursorProvider(HiveRecordCursorProvider cursorProvider, Configuration configuration = new Configuration(); configuration.set("io.compression.codecs", LzoCodec.class.getName() + "," + LzopCodec.class.getName()); + if (storageFormat.equals(THRIFTBINARY)) { + configuration.set("io.compression.codecs", "com.hadoop.compression.lzo.LzoCodec,com.hadoop.compression.lzo.LzopCodec"); + } Optional pageSource = HivePageSourceProvider.createHivePageSource( ImmutableSet.of(cursorProvider), ImmutableSet.of(), diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java index 0acceeca42e1..c5d96e585791 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java @@ -2149,6 +2149,9 @@ private List getAllTestingHiveStorageFormat() Session session = getSession(); ImmutableList.Builder formats = ImmutableList.builder(); for (HiveStorageFormat hiveStorageFormat : HiveStorageFormat.values()) { + if (hiveStorageFormat.equals(HiveStorageFormat.THRIFTBINARY)) { + continue; + } formats.add(new TestingHiveStorageFormat(session, hiveStorageFormat)); } formats.add(new TestingHiveStorageFormat( diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java index eed92b361835..f7a39eb7e7e3 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java @@ -97,6 +97,9 @@ public void testAllFormats() try { ExtendedHiveMetastore metastore = new TestingHiveMetastore(new File(tempDir, "metastore")); for (HiveStorageFormat format : HiveStorageFormat.values()) { + if (format.equals(HiveStorageFormat.THRIFTBINARY)) { + continue; + } config.setHiveStorageFormat(format); config.setHiveCompressionCodec(NONE); long uncompressedLength = writeTestFile(config, metastore, makeFileName(tempDir, config)); diff --git a/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperMetastoreMonitor.java b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperMetastoreMonitor.java new file mode 100644 index 000000000000..60b20d5baab9 --- /dev/null +++ b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperMetastoreMonitor.java @@ -0,0 +1,157 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive; + +import com.facebook.presto.twitter.hive.util.TestUtils; +import com.google.common.collect.ImmutableList; +import com.google.common.net.HostAndPort; +import io.airlift.log.Logger; +import org.I0Itec.zkclient.ZkClient; +import org.I0Itec.zkclient.exception.ZkMarshallingError; +import org.I0Itec.zkclient.serialize.ZkSerializer; +import org.apache.curator.test.TestingServer; +import org.json.simple.JSONObject; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeTest; +import org.testng.annotations.Test; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.testng.Assert.assertTrue; + +public class TestZookeeperMetastoreMonitor +{ + private static final Logger log = Logger.get(TestZookeeperMetastoreMonitor.class); + + private ZookeeperMetastoreMonitor zkMetastoreMonitor; + private TestingServer zkServer; + private ZkClient zkClient; + private final String zkPath = "/metastores"; + + public TestZookeeperMetastoreMonitor() + throws Exception + { + zkServer = new TestingServer(TestUtils.findUnusedPort()); + zkClient = new ZkClient(zkServer.getConnectString(), 30_000, 30_000); + + // Set the serializer + zkClient.setZkSerializer(new ZkSerializer() { + @Override + public byte[] serialize(Object o) throws ZkMarshallingError + { + try { + return o.toString().getBytes(StandardCharsets.UTF_8); + } + catch (Exception e) { + log.warn("Exception in serializing " + e); + } + return "".getBytes(); + } + + @Override + public Object deserialize(byte[] bytes) throws ZkMarshallingError + { + return null; + } + }); + } + + @AfterClass + public void destroy() + throws IOException + { + zkMetastoreMonitor.close(); + zkClient.close(); + zkServer.close(); + } + + @BeforeTest + public void setUp() + throws Exception + { + log.info("Cleaning up zookeeper"); + zkClient.getChildren("/").stream() + .filter(child -> !child.equals("zookeeper")) + .forEach(child -> zkClient.deleteRecursive("/" + child)); + + zkClient.unsubscribeAll(); + + zkClient.createPersistent(zkPath); + zkMetastoreMonitor = new ZookeeperMetastoreMonitor(zkServer.getConnectString(), zkPath, 3, 500); + } + + @Test + public void testGetServers() throws Exception + { + List servers; + List expected; + assertTrue(zkMetastoreMonitor.getServers().isEmpty()); + + addServerToZk("nameNode1", "host1", 10001); + // Sleep for some time so that event can be propagated. + TimeUnit.MILLISECONDS.sleep(1000); + servers = zkMetastoreMonitor.getServers(); + expected = ImmutableList.of(HostAndPort.fromParts("host1", 10001)); + assertTrue(servers.containsAll(expected) && expected.containsAll(servers)); + + addServerToZk("nameNode2", "host2", 10002); + // Sleep for some time so that event can be propagated. + TimeUnit.MILLISECONDS.sleep(1000); + servers = zkMetastoreMonitor.getServers(); + expected = ImmutableList.of(HostAndPort.fromParts("host1", 10001), HostAndPort.fromParts("host2", 10002)); + assertTrue(servers.containsAll(expected) && expected.containsAll(servers)); + + // Change value of an existing name node + addServerToZk("nameNode2", "host2", 10003); + // Sleep for some time so that event can be propagated. + TimeUnit.MILLISECONDS.sleep(1000); + servers = zkMetastoreMonitor.getServers(); + expected = ImmutableList.of(HostAndPort.fromParts("host1", 10001), HostAndPort.fromParts("host2", 10003)); + assertTrue(servers.containsAll(expected) && expected.containsAll(servers)); + + // Delete an existing name node + zkClient.delete(getPathForNameNode("nameNode1")); + // Sleep for some time so that event can be propagated. + TimeUnit.MILLISECONDS.sleep(1000); + servers = zkMetastoreMonitor.getServers(); + expected = ImmutableList.of(HostAndPort.fromParts("host2", 10003)); + assertTrue(servers.containsAll(expected) && expected.containsAll(servers), servers.toString()); + } + + private void addServerToZk(String nameNode, String host, int port) + { + JSONObject serviceEndpoint = new JSONObject(); + serviceEndpoint.put("host", host); + serviceEndpoint.put("port", port); + JSONObject jsonObject = new JSONObject(); + jsonObject.put("serviceEndpoint", serviceEndpoint); + + String path = getPathForNameNode(nameNode); + + if (!zkClient.exists(path)) { + zkClient.createPersistent(path, jsonObject.toJSONString()); + } + else { + zkClient.writeData(path, jsonObject.toJSONString()); + } + } + + private String getPathForNameNode(String nameNode) + { + return zkPath + "/" + nameNode; + } +} diff --git a/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperServersetMetastoreConfig.java b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperServersetMetastoreConfig.java new file mode 100644 index 000000000000..b839a8d15404 --- /dev/null +++ b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperServersetMetastoreConfig.java @@ -0,0 +1,67 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive; + +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; + +public class TestZookeeperServersetMetastoreConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(ZookeeperServersetMetastoreConfig.class) + .setZookeeperMaxRetries(3) + .setZookeeperRetrySleepTime(500) + .setZookeeperMetastorePath(null) + .setZookeeperServerHostAndPort(null) + .setMaxTransport(128) + .setTransportIdleTimeout(300_000L) + .setTransportEvictInterval(10_000L) + .setTransportEvictNumTests(3)); + } + + @Test + public void testExplicitPropertyMappingsSingleMetastore() + { + Map properties = new ImmutableMap.Builder() + .put("hive.metastore.zookeeper.uri", "localhost:2181") + .put("hive.metastore.zookeeper.path", "/zookeeper/path/") + .put("hive.metastore.zookeeper.retry.sleeptime", "200") + .put("hive.metastore.zookeeper.max.retries", "2") + .put("hive.metastore.max-transport-num", "64") + .put("hive.metastore.transport-idle-timeout", "100000") + .put("hive.metastore.transport-eviction-interval", "1000") + .put("hive.metastore.transport-eviction-num-tests", "10") + .build(); + + ZookeeperServersetMetastoreConfig expected = new ZookeeperServersetMetastoreConfig() + .setZookeeperServerHostAndPort("localhost:2181") + .setZookeeperMetastorePath("/zookeeper/path/") + .setZookeeperRetrySleepTime(200) + .setZookeeperMaxRetries(2) + .setMaxTransport(64) + .setTransportIdleTimeout(100_000L) + .setTransportEvictInterval(1_000L) + .setTransportEvictNumTests(10); + + assertFullMapping(properties, expected); + } +} diff --git a/presto-hive/src/test/java/com/facebook/presto/twitter/hive/thrift/TestHiveThriftFieldIdResolver.java b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/thrift/TestHiveThriftFieldIdResolver.java new file mode 100644 index 000000000000..6342a0c088e9 --- /dev/null +++ b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/thrift/TestHiveThriftFieldIdResolver.java @@ -0,0 +1,111 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive.thrift; + +import com.facebook.presto.spi.PrestoException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import java.util.Map; +import java.util.Properties; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertThrows; + +@Test +public class TestHiveThriftFieldIdResolver +{ + private static final Map STRUCT_FIELD_ID_AS_MAP = ImmutableMap.of( + "0", (short) 1, + "1", (short) 2, + "id", (short) 4); + + private static final Map LIST_FIELD_ID_AS_MAP = ImmutableMap.of( + "0", STRUCT_FIELD_ID_AS_MAP, + "id", (short) 5); + + private static final Map VERBOSE_PRIMARY_FIELD_ID_AS_MAP = ImmutableMap.of( + "id", (short) 6); + + private static final Map THRIFT_FIELD_ID_JSON_AS_MAP = ImmutableMap.builder() + .put("0", (short) 1) + .put("1", (short) 3) + .put("2", STRUCT_FIELD_ID_AS_MAP) + .put("3", LIST_FIELD_ID_AS_MAP) + .put("4", VERBOSE_PRIMARY_FIELD_ID_AS_MAP) + .build(); + + private final ObjectMapper objectMapper = new ObjectMapper(); + private final ThriftFieldIdResolverFactory resolverFactory = new HiveThriftFieldIdResolverFactory(); + + @Test + public void testDefaultResolver() + throws Exception + { + ThriftFieldIdResolver defaultResolver = resolverFactory.createResolver(new Properties()); + + for (int i = 0; i <= 5; ++i) { + assertEquals(defaultResolver.getThriftId(i), i + 1); + assertEquals(defaultResolver.getNestedResolver(i), defaultResolver); + } + for (int i = 5; i >= 0; --i) { + assertEquals(defaultResolver.getThriftId(i), i + 1); + assertEquals(defaultResolver.getNestedResolver(i), defaultResolver); + } + } + + @Test + public void testOptimizedResolver() + throws Exception + { + String json = objectMapper.writeValueAsString(THRIFT_FIELD_ID_JSON_AS_MAP); + Properties schema = new Properties(); + schema.setProperty(HiveThriftFieldIdResolverFactory.THRIFT_FIELD_ID_JSON, json); + ThriftFieldIdResolver resolver = resolverFactory.createResolver(schema); + + // primary field + assertEquals(resolver.getThriftId(0), THRIFT_FIELD_ID_JSON_AS_MAP.get("0")); + // discrete field + assertEquals(resolver.getThriftId(1), THRIFT_FIELD_ID_JSON_AS_MAP.get("1")); + + // nested field + ThriftFieldIdResolver nestedResolver = resolver.getNestedResolver(2); + Map field = (Map) THRIFT_FIELD_ID_JSON_AS_MAP.get("2"); + assertEquals(resolver.getThriftId(2), field.get("id")); + assertEquals(nestedResolver.getThriftId(0), field.get("0")); + assertEquals(nestedResolver.getThriftId(1), field.get("1")); + + // non-nested non-primary field + nestedResolver = resolver.getNestedResolver(3); + field = (Map) THRIFT_FIELD_ID_JSON_AS_MAP.get("3"); + assertEquals(resolver.getThriftId(3), field.get("id")); + + // non-primary nested field + nestedResolver = resolver.getNestedResolver(3); + field = (Map) THRIFT_FIELD_ID_JSON_AS_MAP.get("3"); + nestedResolver = nestedResolver.getNestedResolver(0); + field = (Map) field.get("0"); + assertEquals(nestedResolver.getThriftId(0), field.get("0")); + assertEquals(nestedResolver.getThriftId(1), field.get("1")); + + // verbose primary field + field = (Map) THRIFT_FIELD_ID_JSON_AS_MAP.get("4"); + assertEquals(resolver.getThriftId(4), field.get("id")); + + // non-existing field + assertThrows(PrestoException.class, () -> resolver.getThriftId(5)); + assertThrows(PrestoException.class, () -> resolver.getNestedResolver(5)); + } +} diff --git a/presto-hive/src/test/java/com/facebook/presto/twitter/hive/util/TestUtils.java b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/util/TestUtils.java new file mode 100644 index 000000000000..379ad3877e32 --- /dev/null +++ b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/util/TestUtils.java @@ -0,0 +1,30 @@ +/* + * Licensed 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 com.facebook.presto.twitter.hive.util; + +import java.io.IOException; +import java.net.ServerSocket; + +public final class TestUtils +{ + private TestUtils() {} + + public static int findUnusedPort() + throws IOException + { + try (ServerSocket socket = new ServerSocket(0)) { + return socket.getLocalPort(); + } + } +} diff --git a/presto-hive/src/test/resources/addressbook.thrift.lzo b/presto-hive/src/test/resources/addressbook.thrift.lzo new file mode 100644 index 000000000000..bc982749d3ff Binary files /dev/null and b/presto-hive/src/test/resources/addressbook.thrift.lzo differ diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index da80bbeaf83f..e8c0a0a22c22 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 1905b875133e..be4947bdabf0 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 640f0ad39c59..dbbbfe72c4b3 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml new file mode 100644 index 000000000000..3ebad0746851 --- /dev/null +++ b/presto-kafka07/pom.xml @@ -0,0 +1,249 @@ + + + 4.0.0 + + + com.facebook.presto + presto-root + 0.196-tw-0.49 + + + presto-kafka07 + Presto - Kafka Connector for ver0.7 + presto-plugin + + + ${project.parent.basedir} + + + true + + + + + io.airlift + bootstrap + + + + io.airlift + json + + + + io.airlift + log + + + + io.airlift + configuration + + + + com.facebook.presto + presto-record-decoder + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + com.google.inject.extensions + guice-multibindings + + + + javax.validation + validation-api + + + + com.twitter + rosette-kafka_2.11 + 0.7.2-21 + + + jsr305 + com.google.code.findbugs + + + zookeeper + org.apache.zookeeper + + + finagle-ostrich4_2.10 + com.twitter + + + commons-lang + commons-lang + + + + + + org.apache.zookeeper + zookeeper + + + + + joda-time + joda-time + + + + org.scala-lang + scala-library + 2.11.7 + + + + javax.annotation + javax.annotation-api + + + + javax.inject + javax.inject + + + + com.fasterxml.jackson.core + jackson-databind + + + + + com.facebook.presto + presto-spi + provided + + + + io.airlift + slice + provided + + + + io.airlift + units + provided + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + + io.airlift + log-manager + runtime + + + + + org.testng + testng + test + + + + io.airlift + testing + test + + + + com.facebook.presto + presto-main + test + + + + com.facebook.presto + presto-tpch + test + + + + com.facebook.presto + presto-client + test + + + + com.facebook.presto + presto-tests + test + + + + io.airlift.tpch + tpch + test + + + + com.github.sgroschupf + zkclient + 0.1 + + + log4j + log4j + + + + + + org.jetbrains + annotations + test + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + + **/TestKafkaDistributed.java + + + + + + + + + ci + + + + org.apache.maven.plugins + maven-surefire-plugin + + + + + + + + + diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaColumnHandle.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaColumnHandle.java new file mode 100644 index 000000000000..b8ec023b2401 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaColumnHandle.java @@ -0,0 +1,224 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.spi.ColumnMetadata; +import com.facebook.presto.spi.type.Type; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Kafka specific connector column handle. + */ +public final class KafkaColumnHandle + implements DecoderColumnHandle, Comparable +{ + private final String connectorId; + private final int ordinalPosition; + + /** + * Column Name + */ + private final String name; + + /** + * Column type + */ + private final Type type; + + /** + * Mapping hint for the decoder. Can be null. + */ + private final String mapping; + + /** + * Data format to use (selects the decoder). Can be null. + */ + private final String dataFormat; + + /** + * Additional format hint for the selected decoder. Selects a decoder subtype (e.g. which timestamp decoder). + */ + private final String formatHint; + + /** + * True if the key decoder should be used, false if the message decoder should be used. + */ + private final boolean keyDecoder; + + /** + * True if the column should be hidden. + */ + private final boolean hidden; + + /** + * True if the column is internal to the connector and not defined by a topic definition. + */ + private final boolean internal; + + @JsonCreator + public KafkaColumnHandle( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("ordinalPosition") int ordinalPosition, + @JsonProperty("name") String name, + @JsonProperty("type") Type type, + @JsonProperty("mapping") String mapping, + @JsonProperty("dataFormat") String dataFormat, + @JsonProperty("formatHint") String formatHint, + @JsonProperty("keyDecoder") boolean keyDecoder, + @JsonProperty("hidden") boolean hidden, + @JsonProperty("internal") boolean internal) + + { + this.connectorId = requireNonNull(connectorId, "connectorId is null"); + this.ordinalPosition = ordinalPosition; + this.name = requireNonNull(name, "name is null"); + this.type = requireNonNull(type, "type is null"); + this.mapping = mapping; + this.dataFormat = dataFormat; + this.formatHint = formatHint; + this.keyDecoder = keyDecoder; + this.hidden = hidden; + this.internal = internal; + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public int getOrdinalPosition() + { + return ordinalPosition; + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @Override + @JsonProperty + public Type getType() + { + return type; + } + + @Override + @JsonProperty + public String getMapping() + { + return mapping; + } + + @Override + @JsonProperty + public String getDataFormat() + { + return dataFormat; + } + + @Override + @JsonProperty + public String getFormatHint() + { + return formatHint; + } + + @JsonProperty + public boolean isKeyDecoder() + { + return keyDecoder; + } + + @JsonProperty + public boolean isHidden() + { + return hidden; + } + + @Override + @JsonProperty + public boolean isInternal() + { + return internal; + } + + ColumnMetadata getColumnMetadata() + { + return new ColumnMetadata(name, type, null, hidden); + } + + @Override + public int hashCode() + { + return Objects.hash(connectorId, ordinalPosition, name, type, mapping, dataFormat, formatHint, keyDecoder, hidden, internal); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + KafkaColumnHandle other = (KafkaColumnHandle) obj; + return Objects.equals(this.connectorId, other.connectorId) && + Objects.equals(this.ordinalPosition, other.ordinalPosition) && + Objects.equals(this.name, other.name) && + Objects.equals(this.type, other.type) && + Objects.equals(this.mapping, other.mapping) && + Objects.equals(this.dataFormat, other.dataFormat) && + Objects.equals(this.formatHint, other.formatHint) && + Objects.equals(this.keyDecoder, other.keyDecoder) && + Objects.equals(this.hidden, other.hidden) && + Objects.equals(this.internal, other.internal); + } + + @Override + public int compareTo(KafkaColumnHandle otherHandle) + { + return Integer.compare(this.getOrdinalPosition(), otherHandle.getOrdinalPosition()); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("connectorId", connectorId) + .add("ordinalPosition", ordinalPosition) + .add("name", name) + .add("type", type) + .add("mapping", mapping) + .add("dataFormat", dataFormat) + .add("formatHint", formatHint) + .add("keyDecoder", keyDecoder) + .add("hidden", hidden) + .add("internal", internal) + .toString(); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnector.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnector.java new file mode 100644 index 000000000000..69cfe62e8737 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnector.java @@ -0,0 +1,92 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.spi.connector.Connector; +import com.facebook.presto.spi.connector.ConnectorMetadata; +import com.facebook.presto.spi.connector.ConnectorRecordSetProvider; +import com.facebook.presto.spi.connector.ConnectorSplitManager; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.facebook.presto.spi.transaction.IsolationLevel; +import io.airlift.bootstrap.LifeCycleManager; +import io.airlift.log.Logger; + +import javax.inject.Inject; + +import static com.facebook.presto.spi.transaction.IsolationLevel.READ_COMMITTED; +import static com.facebook.presto.spi.transaction.IsolationLevel.checkConnectorSupports; +import static java.util.Objects.requireNonNull; + +/** + * Kafka specific implementation of the Presto Connector SPI. This is a read only connector. + */ +public class KafkaConnector + implements Connector +{ + private static final Logger log = Logger.get(KafkaConnector.class); + + private final LifeCycleManager lifeCycleManager; + private final KafkaMetadata metadata; + private final KafkaSplitManager splitManager; + private final KafkaRecordSetProvider recordSetProvider; + + @Inject + public KafkaConnector( + LifeCycleManager lifeCycleManager, + KafkaMetadata metadata, + KafkaSplitManager splitManager, + KafkaRecordSetProvider recordSetProvider) + { + this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); + this.metadata = requireNonNull(metadata, "metadata is null"); + this.splitManager = requireNonNull(splitManager, "splitManager is null"); + this.recordSetProvider = requireNonNull(recordSetProvider, "recordSetProvider is null"); + } + + @Override + public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly) + { + checkConnectorSupports(READ_COMMITTED, isolationLevel); + return KafkaTransactionHandle.INSTANCE; + } + + @Override + public ConnectorMetadata getMetadata(ConnectorTransactionHandle transactionHandle) + { + return metadata; + } + + @Override + public ConnectorSplitManager getSplitManager() + { + return splitManager; + } + + @Override + public ConnectorRecordSetProvider getRecordSetProvider() + { + return recordSetProvider; + } + + @Override + public final void shutdown() + { + try { + lifeCycleManager.stop(); + } + catch (Exception e) { + log.error(e, "Error shutting down connector"); + } + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorConfig.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorConfig.java new file mode 100644 index 000000000000..de5615de2167 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorConfig.java @@ -0,0 +1,204 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.spi.HostAddress; +import com.google.common.base.Splitter; +import com.google.common.collect.ImmutableSet; +import io.airlift.configuration.Config; +import io.airlift.units.DataSize; +import io.airlift.units.DataSize.Unit; +import io.airlift.units.Duration; +import io.airlift.units.MinDuration; + +import javax.validation.constraints.NotNull; + +import java.io.File; +import java.util.Set; + +import static com.google.common.collect.Iterables.transform; + +public class KafkaConnectorConfig +{ + private static final int KAFKA_DEFAULT_PORT = 9092; + + /** + * Seed nodes for Kafka cluster. At least one must exist. + */ + private Set nodes = ImmutableSet.of(); + + /** + * Timeout to connect to Kafka. + */ + private Duration kafkaConnectTimeout = Duration.valueOf("10s"); + + /** + * Buffer size for connecting to Kafka. + */ + private DataSize kafkaBufferSize = new DataSize(64, Unit.KILOBYTE); + + /** + * The schema name to use in the connector. + */ + private String defaultSchema = "default"; + + /** + * Set of tables known to this connector. For each table, a description file may be present in the catalog folder which describes columns for the given topic. + */ + private Set tableNames = ImmutableSet.of(); + + /** + * Folder holding the JSON description files for Kafka topics. + */ + private File tableDescriptionDir = new File("etc/kafka07/"); + + /** + * Whether internal columns are shown in table metadata or not. Default is no. + */ + private boolean hideInternalColumns = true; + + /** + * ZK endpoint for getting broker list + */ + private String zkEndpoint = ""; + + /** + * Fetch size + */ + private int fetchSize = 10 * 1024 * 1024; + + @NotNull + public File getTableDescriptionDir() + { + return tableDescriptionDir; + } + + @Config("kafka.table-description-dir") + public KafkaConnectorConfig setTableDescriptionDir(File tableDescriptionDir) + { + this.tableDescriptionDir = tableDescriptionDir; + return this; + } + + @NotNull + public Set getTableNames() + { + return tableNames; + } + + @Config("kafka.table-names") + public KafkaConnectorConfig setTableNames(String tableNames) + { + this.tableNames = ImmutableSet.copyOf(Splitter.on(',').omitEmptyStrings().trimResults().split(tableNames)); + return this; + } + + @NotNull + public String getDefaultSchema() + { + return defaultSchema; + } + + @Config("kafka.default-schema") + public KafkaConnectorConfig setDefaultSchema(String defaultSchema) + { + this.defaultSchema = defaultSchema; + return this; + } + + public Set getNodes() + { + return nodes; + } + + @Config("kafka.nodes") + public KafkaConnectorConfig setNodes(String nodes) + { + this.nodes = (nodes == null) ? null : parseNodes(nodes); + return this; + } + + @MinDuration("1s") + public Duration getKafkaConnectTimeout() + { + return kafkaConnectTimeout; + } + + @Config("kafka.connect-timeout") + public KafkaConnectorConfig setKafkaConnectTimeout(String kafkaConnectTimeout) + { + this.kafkaConnectTimeout = Duration.valueOf(kafkaConnectTimeout); + return this; + } + + public DataSize getKafkaBufferSize() + { + return kafkaBufferSize; + } + + @Config("kafka.buffer-size") + public KafkaConnectorConfig setKafkaBufferSize(String kafkaBufferSize) + { + this.kafkaBufferSize = DataSize.valueOf(kafkaBufferSize); + return this; + } + + public boolean isHideInternalColumns() + { + return hideInternalColumns; + } + + @Config("kafka.hide-internal-columns") + public KafkaConnectorConfig setHideInternalColumns(boolean hideInternalColumns) + { + this.hideInternalColumns = hideInternalColumns; + return this; + } + + @NotNull + public String getZkEndpoint() + { + return zkEndpoint; + } + + @Config("kafka.zk-endpoint") + public KafkaConnectorConfig setZkEndpoint(String zkEndpoint) + { + this.zkEndpoint = zkEndpoint; + return this; + } + + public int getFetchSize() + { + return fetchSize; + } + + @Config("kafka.fetch-size") + public KafkaConnectorConfig setFetchSize(int fetchSize) + { + this.fetchSize = fetchSize; + return this; + } + + public static ImmutableSet parseNodes(String nodes) + { + Splitter splitter = Splitter.on(',').omitEmptyStrings().trimResults(); + return ImmutableSet.copyOf(transform(splitter.split(nodes), KafkaConnectorConfig::toHostAddress)); + } + + private static HostAddress toHostAddress(String value) + { + return HostAddress.fromString(value).withDefaultPort(KAFKA_DEFAULT_PORT); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java new file mode 100644 index 000000000000..b9721f0459f8 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java @@ -0,0 +1,95 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.spi.ConnectorHandleResolver; +import com.facebook.presto.spi.NodeManager; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.connector.Connector; +import com.facebook.presto.spi.connector.ConnectorContext; +import com.facebook.presto.spi.connector.ConnectorFactory; +import com.facebook.presto.spi.type.TypeManager; +import com.google.common.base.Throwables; +import com.google.inject.Injector; +import com.google.inject.Scopes; +import com.google.inject.TypeLiteral; +import io.airlift.bootstrap.Bootstrap; +import io.airlift.json.JsonModule; + +import java.util.Map; +import java.util.Optional; +import java.util.function.Supplier; + +import static java.util.Objects.requireNonNull; + +/** + * Creates Kafka Connectors based off connectorId and specific configuration. + */ +public class KafkaConnectorFactory + implements ConnectorFactory +{ + private final Optional>> tableDescriptionSupplier; + + KafkaConnectorFactory(Optional>> tableDescriptionSupplier) + { + this.tableDescriptionSupplier = requireNonNull(tableDescriptionSupplier, "tableDescriptionSupplier is null"); + } + + @Override + public String getName() + { + return "kafka07"; + } + + @Override + public ConnectorHandleResolver getHandleResolver() + { + return new KafkaHandleResolver(); + } + + @Override + public Connector create(String connectorId, Map config, ConnectorContext context) + { + requireNonNull(connectorId, "connectorId is null"); + requireNonNull(config, "config is null"); + + try { + Bootstrap app = new Bootstrap( + new JsonModule(), + new KafkaConnectorModule(), + binder -> { + binder.bind(KafkaConnectorId.class).toInstance(new KafkaConnectorId(connectorId)); + binder.bind(TypeManager.class).toInstance(context.getTypeManager()); + binder.bind(NodeManager.class).toInstance(context.getNodeManager()); + + if (tableDescriptionSupplier.isPresent()) { + binder.bind(new TypeLiteral>>() {}).toInstance(tableDescriptionSupplier.get()); + } + else { + binder.bind(new TypeLiteral>>() {}).to(KafkaTableDescriptionSupplier.class).in(Scopes.SINGLETON); + } + }); + + Injector injector = app.strictConfig() + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + + return injector.getInstance(KafkaConnector.class); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorId.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorId.java new file mode 100644 index 000000000000..3470980df073 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorId.java @@ -0,0 +1,53 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import java.util.Objects; + +import static java.util.Objects.requireNonNull; + +public class KafkaConnectorId +{ + private final String connectorId; + + public KafkaConnectorId(String connectorId) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null"); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + KafkaConnectorId other = (KafkaConnectorId) obj; + return Objects.equals(this.connectorId, other.connectorId); + } + + @Override + public int hashCode() + { + return Objects.hash(connectorId); + } + + @Override + public String toString() + { + return connectorId; + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorModule.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorModule.java new file mode 100644 index 000000000000..e12e70567dc0 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorModule.java @@ -0,0 +1,92 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.decoder.DecoderModule; +import com.facebook.presto.spi.type.Type; +import com.facebook.presto.spi.type.TypeManager; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; +import com.google.inject.multibindings.Multibinder; + +import javax.inject.Inject; + +import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; +import static com.google.common.base.Preconditions.checkArgument; +import static io.airlift.configuration.ConfigBinder.configBinder; +import static io.airlift.json.JsonBinder.jsonBinder; +import static io.airlift.json.JsonCodecBinder.jsonCodecBinder; +import static java.util.Objects.requireNonNull; + +/** + * Guice module for the Apache Kafka connector. + */ +public class KafkaConnectorModule + implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(KafkaConnector.class).in(Scopes.SINGLETON); + + binder.bind(KafkaMetadata.class).in(Scopes.SINGLETON); + binder.bind(KafkaSplitManager.class).in(Scopes.SINGLETON); + binder.bind(KafkaRecordSetProvider.class).in(Scopes.SINGLETON); + + binder.bind(KafkaSimpleConsumerManager.class).in(Scopes.SINGLETON); + + configBinder(binder).bindConfig(KafkaConnectorConfig.class); + + jsonBinder(binder).addDeserializerBinding(Type.class).to(TypeDeserializer.class); + jsonCodecBinder(binder).bindJsonCodec(KafkaTopicDescription.class); + + binder.install(new DecoderModule()); + + for (KafkaInternalFieldDescription internalFieldDescription : KafkaInternalFieldDescription.getInternalFields()) { + bindInternalColumn(binder, internalFieldDescription); + } + } + + private static void bindInternalColumn(Binder binder, KafkaInternalFieldDescription fieldDescription) + { + Multibinder fieldDescriptionBinder = Multibinder.newSetBinder(binder, KafkaInternalFieldDescription.class); + fieldDescriptionBinder.addBinding().toInstance(fieldDescription); + } + + public static final class TypeDeserializer + extends FromStringDeserializer + { + private static final long serialVersionUID = 1L; + + private final TypeManager typeManager; + + @Inject + public TypeDeserializer(TypeManager typeManager) + { + super(Type.class); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + @Override + protected Type _deserialize(String value, DeserializationContext context) + { + Type type = typeManager.getType(parseTypeSignature(value)); + checkArgument(type != null, "Unknown type %s", value); + return type; + } + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaErrorCode.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaErrorCode.java new file mode 100644 index 000000000000..9338b9849211 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaErrorCode.java @@ -0,0 +1,42 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.spi.ErrorCode; +import com.facebook.presto.spi.ErrorCodeSupplier; +import com.facebook.presto.spi.ErrorType; + +import static com.facebook.presto.spi.ErrorType.EXTERNAL; + +/** + * Kafka connector specific error codes. + */ +public enum KafkaErrorCode + implements ErrorCodeSupplier +{ + KAFKA_SPLIT_ERROR(0, EXTERNAL); + + private final ErrorCode errorCode; + + KafkaErrorCode(int code, ErrorType type) + { + errorCode = new ErrorCode(code + 0x0102_0000, name(), type); + } + + @Override + public ErrorCode toErrorCode() + { + return errorCode; + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaHandleResolver.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaHandleResolver.java new file mode 100644 index 000000000000..539b08270129 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaHandleResolver.java @@ -0,0 +1,89 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ConnectorHandleResolver; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +/** + * Kafka specific {@link com.facebook.presto.spi.ConnectorHandleResolver} implementation. + */ +public class KafkaHandleResolver + implements ConnectorHandleResolver +{ + @Override + public Class getTableHandleClass() + { + return KafkaTableHandle.class; + } + + @Override + public Class getColumnHandleClass() + { + return KafkaColumnHandle.class; + } + + @Override + public Class getSplitClass() + { + return KafkaSplit.class; + } + + @Override + public Class getTableLayoutHandleClass() + { + return KafkaTableLayoutHandle.class; + } + + @Override + public Class getTransactionHandleClass() + { + return KafkaTransactionHandle.class; + } + + static KafkaTableHandle convertTableHandle(ConnectorTableHandle tableHandle) + { + requireNonNull(tableHandle, "tableHandle is null"); + checkArgument(tableHandle instanceof KafkaTableHandle, "tableHandle is not an instance of KafkaTableHandle"); + return (KafkaTableHandle) tableHandle; + } + + static KafkaColumnHandle convertColumnHandle(ColumnHandle columnHandle) + { + requireNonNull(columnHandle, "columnHandle is null"); + checkArgument(columnHandle instanceof KafkaColumnHandle, "columnHandle is not an instance of KafkaColumnHandle"); + return (KafkaColumnHandle) columnHandle; + } + + static KafkaSplit convertSplit(ConnectorSplit split) + { + requireNonNull(split, "split is null"); + checkArgument(split instanceof KafkaSplit, "split is not an instance of KafkaSplit"); + return (KafkaSplit) split; + } + + static KafkaTableLayoutHandle convertLayout(ConnectorTableLayoutHandle layout) + { + requireNonNull(layout, "layout is null"); + checkArgument(layout instanceof KafkaTableLayoutHandle, "layout is not an instance of KafkaTableLayoutHandle"); + return (KafkaTableLayoutHandle) layout; + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaInternalFieldDescription.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaInternalFieldDescription.java new file mode 100644 index 000000000000..5bcad3d565d4 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaInternalFieldDescription.java @@ -0,0 +1,286 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.FieldValueProvider; +import com.facebook.presto.spi.ColumnMetadata; +import com.facebook.presto.spi.type.BigintType; +import com.facebook.presto.spi.type.BooleanType; +import com.facebook.presto.spi.type.Type; +import com.google.common.collect.ImmutableSet; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; + +import java.util.Objects; +import java.util.Set; + +import static com.facebook.presto.spi.type.VarcharType.createUnboundedVarcharType; +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static java.util.Objects.requireNonNull; + +/** + * Describes an internal (managed by the connector) field which is added to each table row. The definition itself makes the row + * show up in the tables (the columns are hidden by default, so they must be explicitly selected) but unless the field is hooked in using the + * forBooleanValue/forLongValue/forBytesValue methods and the resulting FieldValueProvider is then passed into the appropriate row decoder, the fields + * will be null. Most values are assigned in the {@link com.facebook.presto.kafka.KafkaRecordSet}. + */ +public class KafkaInternalFieldDescription +{ + /** + * _partition_id - Kafka partition id. + */ + public static final KafkaInternalFieldDescription PARTITION_ID_FIELD = new KafkaInternalFieldDescription("_partition_id", BigintType.BIGINT, "Partition Id"); + + /** + * _partition_offset - The current offset of the message in the partition. + */ + public static final KafkaInternalFieldDescription PARTITION_OFFSET_FIELD = new KafkaInternalFieldDescription("_partition_offset", BigintType.BIGINT, "Offset for the message within the partition"); + + /** + * _segment_start - Kafka start offset for the segment which contains the current message. This is per-partition. + */ + public static final KafkaInternalFieldDescription SEGMENT_START_FIELD = new KafkaInternalFieldDescription("_segment_start", BigintType.BIGINT, "Segment start offset"); + + /** + * _segment_end - Kafka end offset for the segment which contains the current message. This is per-partition. The end offset is the first offset that is *not* in the segment. + */ + public static final KafkaInternalFieldDescription SEGMENT_END_FIELD = new KafkaInternalFieldDescription("_segment_end", BigintType.BIGINT, "Segment end offset"); + + /** + * _segment_count - Running count of messages in a segment. + */ + public static final KafkaInternalFieldDescription SEGMENT_COUNT_FIELD = new KafkaInternalFieldDescription("_segment_count", BigintType.BIGINT, "Running message count per segment"); + + /** + * _message_corrupt - True if the row converter could not read the a message. May be null if the row converter does not set a value (e.g. the dummy row converter does not). + */ + public static final KafkaInternalFieldDescription MESSAGE_CORRUPT_FIELD = new KafkaInternalFieldDescription("_message_corrupt", BooleanType.BOOLEAN, "Message data is corrupt"); + + /** + * _message - Represents the full topic as a text column. Format is UTF-8 which may be wrong for some topics. TODO: make charset configurable. + */ + public static final KafkaInternalFieldDescription MESSAGE_FIELD = new KafkaInternalFieldDescription("_message", createUnboundedVarcharType(), "Message text"); + + /** + * _message_length - length in bytes of the message. + */ + public static final KafkaInternalFieldDescription MESSAGE_LENGTH_FIELD = new KafkaInternalFieldDescription("_message_length", BigintType.BIGINT, "Total number of message bytes"); + + /** + * _key_corrupt - True if the row converter could not read the a key. May be null if the row converter does not set a value (e.g. the dummy row converter does not). + */ + public static final KafkaInternalFieldDescription KEY_CORRUPT_FIELD = new KafkaInternalFieldDescription("_key_corrupt", BooleanType.BOOLEAN, "Key data is corrupt"); + + /** + * _key - Represents the key as a text column. Format is UTF-8 which may be wrong for topics. TODO: make charset configurable. + */ + public static final KafkaInternalFieldDescription KEY_FIELD = new KafkaInternalFieldDescription("_key", createUnboundedVarcharType(), "Key text"); + + /** + * _key_length - length in bytes of the key. + */ + public static final KafkaInternalFieldDescription KEY_LENGTH_FIELD = new KafkaInternalFieldDescription("_key_length", BigintType.BIGINT, "Total number of key bytes"); + + /** + * _timestamp - offset timestamp, used to narrow scan range + */ + public static final KafkaInternalFieldDescription OFFSET_TIMESTAMP_FIELD = new KafkaInternalFieldDescription("_timestamp", BigintType.BIGINT, "Offset Timestamp"); + + public static Set getInternalFields() + { + return ImmutableSet.of(PARTITION_ID_FIELD, PARTITION_OFFSET_FIELD, + SEGMENT_START_FIELD, SEGMENT_END_FIELD, SEGMENT_COUNT_FIELD, + KEY_FIELD, KEY_CORRUPT_FIELD, KEY_LENGTH_FIELD, + MESSAGE_FIELD, MESSAGE_CORRUPT_FIELD, MESSAGE_LENGTH_FIELD, OFFSET_TIMESTAMP_FIELD); + } + + private final String name; + private final Type type; + private final String comment; + + KafkaInternalFieldDescription( + String name, + Type type, + String comment) + { + checkArgument(!isNullOrEmpty(name), "name is null or is empty"); + this.name = name; + this.type = requireNonNull(type, "type is null"); + this.comment = requireNonNull(comment, "comment is null"); + } + + public String getName() + { + return name; + } + + public Type getType() + { + return type; + } + + KafkaColumnHandle getColumnHandle(String connectorId, int index, boolean hidden) + { + return new KafkaColumnHandle(connectorId, + index, + getName(), + getType(), + null, + null, + null, + false, + hidden, + true); + } + + ColumnMetadata getColumnMetadata(boolean hidden) + { + return new ColumnMetadata(name, type, comment, hidden); + } + + public FieldValueProvider forBooleanValue(boolean value) + { + return new BooleanKafkaFieldValueProvider(value); + } + + public FieldValueProvider forLongValue(long value) + { + return new LongKafkaFieldValueProvider(value); + } + + public FieldValueProvider forByteValue(byte[] value) + { + return new BytesKafkaFieldValueProvider(value); + } + + @Override + public int hashCode() + { + return Objects.hash(name, type); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + KafkaInternalFieldDescription other = (KafkaInternalFieldDescription) obj; + return Objects.equals(this.name, other.name) && + Objects.equals(this.type, other.type); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("name", name) + .add("type", type) + .toString(); + } + + public class BooleanKafkaFieldValueProvider + extends FieldValueProvider + { + private final boolean value; + + private BooleanKafkaFieldValueProvider(boolean value) + { + this.value = value; + } + + @Override + public boolean accept(DecoderColumnHandle columnHandle) + { + return columnHandle.getName().equals(name); + } + + @Override + public boolean getBoolean() + { + return value; + } + + @Override + public boolean isNull() + { + return false; + } + } + + public class LongKafkaFieldValueProvider + extends FieldValueProvider + { + private final long value; + + private LongKafkaFieldValueProvider(long value) + { + this.value = value; + } + + @Override + public boolean accept(DecoderColumnHandle columnHandle) + { + return columnHandle.getName().equals(name); + } + + @Override + public long getLong() + { + return value; + } + + @Override + public boolean isNull() + { + return false; + } + } + + public class BytesKafkaFieldValueProvider + extends FieldValueProvider + { + private final byte[] value; + + private BytesKafkaFieldValueProvider(byte[] value) + { + this.value = value; + } + + @Override + public boolean accept(DecoderColumnHandle columnHandle) + { + return columnHandle.getName().equals(name); + } + + @Override + public Slice getSlice() + { + return isNull() ? Slices.EMPTY_SLICE : Slices.wrappedBuffer(value); + } + + @Override + public boolean isNull() + { + return value == null || value.length == 0; + } + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java new file mode 100644 index 000000000000..e9644a4c1471 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java @@ -0,0 +1,274 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.decoder.dummy.DummyRowDecoder; +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ColumnMetadata; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.ConnectorTableLayout; +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.facebook.presto.spi.ConnectorTableLayoutResult; +import com.facebook.presto.spi.ConnectorTableMetadata; +import com.facebook.presto.spi.Constraint; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.SchemaTablePrefix; +import com.facebook.presto.spi.TableNotFoundException; +import com.facebook.presto.spi.connector.ConnectorMetadata; +import com.facebook.presto.spi.predicate.Domain; +import com.facebook.presto.spi.predicate.Marker; +import com.facebook.presto.spi.predicate.Range; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.airlift.log.Logger; + +import javax.inject.Inject; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.Supplier; + +import static com.facebook.presto.kafka.KafkaHandleResolver.convertColumnHandle; +import static com.facebook.presto.kafka.KafkaHandleResolver.convertTableHandle; +import static java.util.Objects.requireNonNull; + +/** + * Manages the Kafka connector specific metadata information. The Connector provides an additional set of columns + * for each table that are created as hidden columns. See {@link KafkaInternalFieldDescription} for a list + * of per-topic additional columns. + */ +public class KafkaMetadata + implements ConnectorMetadata +{ + private static final Logger log = Logger.get(KafkaMetadata.class); + + private final String connectorId; + private final boolean hideInternalColumns; + private final Map tableDescriptions; + private final Set internalFieldDescriptions; + + @Inject + public KafkaMetadata( + KafkaConnectorId connectorId, + KafkaConnectorConfig kafkaConnectorConfig, + Supplier> kafkaTableDescriptionSupplier, + Set internalFieldDescriptions) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + + requireNonNull(kafkaConnectorConfig, "kafkaConfig is null"); + this.hideInternalColumns = kafkaConnectorConfig.isHideInternalColumns(); + + requireNonNull(kafkaTableDescriptionSupplier, "kafkaTableDescriptionSupplier is null"); + this.tableDescriptions = kafkaTableDescriptionSupplier.get(); + this.internalFieldDescriptions = requireNonNull(internalFieldDescriptions, "internalFieldDescriptions is null"); + } + + @Override + public List listSchemaNames(ConnectorSession session) + { + ImmutableSet.Builder builder = ImmutableSet.builder(); + for (SchemaTableName tableName : tableDescriptions.keySet()) { + builder.add(tableName.getSchemaName()); + } + return ImmutableList.copyOf(builder.build()); + } + + @Override + public KafkaTableHandle getTableHandle(ConnectorSession session, SchemaTableName schemaTableName) + { + KafkaTopicDescription table = tableDescriptions.get(schemaTableName); + if (table == null) { + return null; + } + + return new KafkaTableHandle(connectorId, + schemaTableName.getSchemaName(), + schemaTableName.getTableName(), + table.getTopicName(), + getDataFormat(table.getKey()), + getDataFormat(table.getMessage())); + } + + private static String getDataFormat(KafkaTopicFieldGroup fieldGroup) + { + return (fieldGroup == null) ? DummyRowDecoder.NAME : fieldGroup.getDataFormat(); + } + + @Override + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle tableHandle) + { + return getTableMetadata(convertTableHandle(tableHandle).toSchemaTableName()); + } + + @Override + public List listTables(ConnectorSession session, String schemaNameOrNull) + { + ImmutableList.Builder builder = ImmutableList.builder(); + for (SchemaTableName tableName : tableDescriptions.keySet()) { + if (schemaNameOrNull == null || tableName.getSchemaName().equals(schemaNameOrNull)) { + builder.add(tableName); + } + } + + return builder.build(); + } + + @SuppressWarnings("ValueOfIncrementOrDecrementUsed") + @Override + public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) + { + KafkaTableHandle kafkaTableHandle = convertTableHandle(tableHandle); + + KafkaTopicDescription kafkaTopicDescription = tableDescriptions.get(kafkaTableHandle.toSchemaTableName()); + if (kafkaTopicDescription == null) { + throw new TableNotFoundException(kafkaTableHandle.toSchemaTableName()); + } + + ImmutableMap.Builder columnHandles = ImmutableMap.builder(); + + int index = 0; + KafkaTopicFieldGroup key = kafkaTopicDescription.getKey(); + if (key != null) { + List fields = key.getFields(); + if (fields != null) { + for (KafkaTopicFieldDescription kafkaTopicFieldDescription : fields) { + columnHandles.put(kafkaTopicFieldDescription.getName(), kafkaTopicFieldDescription.getColumnHandle(connectorId, true, index++)); + } + } + } + + KafkaTopicFieldGroup message = kafkaTopicDescription.getMessage(); + if (message != null) { + List fields = message.getFields(); + if (fields != null) { + for (KafkaTopicFieldDescription kafkaTopicFieldDescription : fields) { + columnHandles.put(kafkaTopicFieldDescription.getName(), kafkaTopicFieldDescription.getColumnHandle(connectorId, false, index++)); + } + } + } + + for (KafkaInternalFieldDescription kafkaInternalFieldDescription : internalFieldDescriptions) { + columnHandles.put(kafkaInternalFieldDescription.getName(), kafkaInternalFieldDescription.getColumnHandle(connectorId, index++, hideInternalColumns)); + } + + return columnHandles.build(); + } + + @Override + public Map> listTableColumns(ConnectorSession session, SchemaTablePrefix prefix) + { + requireNonNull(prefix, "prefix is null"); + + ImmutableMap.Builder> columns = ImmutableMap.builder(); + + List tableNames = prefix.getSchemaName() == null ? listTables(session, null) : ImmutableList.of(new SchemaTableName(prefix.getSchemaName(), prefix.getTableName())); + + for (SchemaTableName tableName : tableNames) { + ConnectorTableMetadata tableMetadata = getTableMetadata(tableName); + // table can disappear during listing operation + if (tableMetadata != null) { + columns.put(tableName, tableMetadata.getColumns()); + } + } + return columns.build(); + } + + @Override + public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle) + { + convertTableHandle(tableHandle); + return convertColumnHandle(columnHandle).getColumnMetadata(); + } + + @Override + public List getTableLayouts(ConnectorSession session, ConnectorTableHandle table, Constraint constraint, Optional> desiredColumns) + { + KafkaTableHandle handle = convertTableHandle(table); + + Long startTs = null; + Long endTs = null; + Optional> domains = constraint.getSummary().getDomains(); + log.info(constraint.getSummary().toString(null)); + if (domains.isPresent()) { + Map columnHandleDomainMap = domains.get(); + for (Map.Entry entry : columnHandleDomainMap.entrySet()) { + if (entry.getKey() instanceof KafkaColumnHandle && ((KafkaColumnHandle) entry.getKey()).getName().equals(KafkaInternalFieldDescription.OFFSET_TIMESTAMP_FIELD.getName())) { + Range span = entry.getValue().getValues().getRanges().getSpan(); + Marker low = span.getLow(); + Marker high = span.getHigh(); + if (!low.isLowerUnbounded()) { + startTs = (Long) low.getValue(); + } + if (!high.isUpperUnbounded()) { + endTs = (Long) high.getValue(); + } + } + + log.info("K: %s\tV: %s", entry.getKey().toString(), entry.getValue().toString()); + } + } + + log.info("startTs: %s, endTs: %s", startTs, endTs); + ConnectorTableLayout layout = new ConnectorTableLayout(new KafkaTableLayoutHandle(handle, startTs, endTs)); + return ImmutableList.of(new ConnectorTableLayoutResult(layout, constraint.getSummary())); + } + + @Override + public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTableLayoutHandle handle) + { + return new ConnectorTableLayout(handle); + } + + @SuppressWarnings("ValueOfIncrementOrDecrementUsed") + private ConnectorTableMetadata getTableMetadata(SchemaTableName schemaTableName) + { + KafkaTopicDescription table = tableDescriptions.get(schemaTableName); + if (table == null) { + throw new TableNotFoundException(schemaTableName); + } + + ImmutableList.Builder builder = ImmutableList.builder(); + + KafkaTopicFieldGroup key = table.getKey(); + if (key != null) { + List fields = key.getFields(); + if (fields != null) { + for (KafkaTopicFieldDescription fieldDescription : fields) { + builder.add(fieldDescription.getColumnMetadata()); + } + } + } + + KafkaTopicFieldGroup message = table.getMessage(); + if (message != null) { + List fields = message.getFields(); + if (fields != null) { + for (KafkaTopicFieldDescription fieldDescription : fields) { + builder.add(fieldDescription.getColumnMetadata()); + } + } + } + + for (KafkaInternalFieldDescription fieldDescription : internalFieldDescriptions) { + builder.add(fieldDescription.getColumnMetadata(hideInternalColumns)); + } + + return new ConnectorTableMetadata(schemaTableName, builder.build()); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaPlugin.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaPlugin.java new file mode 100644 index 000000000000..284424a9a7fa --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaPlugin.java @@ -0,0 +1,47 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.spi.Plugin; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.connector.ConnectorFactory; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; + +import java.util.Map; +import java.util.Optional; +import java.util.function.Supplier; + +import static java.util.Objects.requireNonNull; + +/** + * Presto plugin to use Apache Kafka as a data source. + */ +public class KafkaPlugin + implements Plugin +{ + private Optional>> tableDescriptionSupplier = Optional.empty(); + + @VisibleForTesting + public synchronized void setTableDescriptionSupplier(Supplier> tableDescriptionSupplier) + { + this.tableDescriptionSupplier = Optional.of(requireNonNull(tableDescriptionSupplier, "tableDescriptionSupplier is null")); + } + + @Override + public synchronized Iterable getConnectorFactories() + { + return ImmutableList.of(new KafkaConnectorFactory(tableDescriptionSupplier)); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java new file mode 100644 index 000000000000..34283d277cef --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java @@ -0,0 +1,346 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.FieldDecoder; +import com.facebook.presto.decoder.FieldValueProvider; +import com.facebook.presto.decoder.RowDecoder; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.RecordCursor; +import com.facebook.presto.spi.RecordSet; +import com.facebook.presto.spi.block.Block; +import com.facebook.presto.spi.type.Type; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import io.airlift.log.Logger; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import kafka.api.FetchRequest; +import kafka.api.OffsetRequest; +import kafka.common.ErrorMapping; +import kafka.common.OffsetOutOfRangeException; +import kafka.javaapi.consumer.SimpleConsumer; +import kafka.javaapi.message.ByteBufferMessageSet; +import kafka.message.MessageAndOffset; + +import java.nio.ByteBuffer; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.facebook.presto.kafka.KafkaErrorCode.KAFKA_SPLIT_ERROR; +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +/** + * Kafka specific record set. Returns a cursor for a topic which iterates over a Kafka partition segment. + */ +public class KafkaRecordSet + implements RecordSet +{ + private static final Logger log = Logger.get(KafkaRecordSet.class); + + private static final byte[] EMPTY_BYTE_ARRAY = new byte[0]; + + private final KafkaSplit split; + private final KafkaSimpleConsumerManager consumerManager; + private final int fetchSize; + + private final RowDecoder keyDecoder; + private final RowDecoder messageDecoder; + private final Map> keyFieldDecoders; + private final Map> messageFieldDecoders; + + private final List columnHandles; + private final List columnTypes; + + private final Set globalInternalFieldValueProviders; + + KafkaRecordSet(KafkaSplit split, + KafkaSimpleConsumerManager consumerManager, + List columnHandles, + RowDecoder keyDecoder, + RowDecoder messageDecoder, + Map> keyFieldDecoders, + Map> messageFieldDecoders, + int fetchSize) + { + this.split = requireNonNull(split, "split is null"); + + this.globalInternalFieldValueProviders = ImmutableSet.of( + KafkaInternalFieldDescription.PARTITION_ID_FIELD.forLongValue(split.getPartitionId()), + KafkaInternalFieldDescription.SEGMENT_START_FIELD.forLongValue(split.getStart()), + KafkaInternalFieldDescription.SEGMENT_END_FIELD.forLongValue(split.getEnd())); + + this.consumerManager = requireNonNull(consumerManager, "consumerManager is null"); + + this.keyDecoder = requireNonNull(keyDecoder, "rowDecoder is null"); + this.messageDecoder = requireNonNull(messageDecoder, "rowDecoder is null"); + this.keyFieldDecoders = requireNonNull(keyFieldDecoders, "keyFieldDecoders is null"); + this.messageFieldDecoders = requireNonNull(messageFieldDecoders, "messageFieldDecoders is null"); + + this.columnHandles = requireNonNull(columnHandles, "columnHandles is null"); + + ImmutableList.Builder typeBuilder = ImmutableList.builder(); + + for (DecoderColumnHandle handle : columnHandles) { + typeBuilder.add(handle.getType()); + } + + this.columnTypes = typeBuilder.build(); + this.fetchSize = fetchSize; + } + + @Override + public List getColumnTypes() + { + return columnTypes; + } + + @Override + public RecordCursor cursor() + { + return new KafkaRecordCursor(split.getStartTs(), split.getEndTs()); + } + + public class KafkaRecordCursor + implements RecordCursor + { + private long totalBytes; + private long totalMessages; + private long cursorOffset = split.getStart(); + private Iterator messageAndOffsetIterator; + private long fetchedSize; + private final AtomicBoolean reported = new AtomicBoolean(); + + private final long startTs; + private final long endTs; + + private FieldValueProvider[] fieldValueProviders; + + KafkaRecordCursor(long startTs, long endTs) + { + this.startTs = startTs; + this.endTs = endTs; + } + + @Override + public long getCompletedBytes() + { + return totalBytes; + } + + @Override + public long getReadTimeNanos() + { + return 0; + } + + @Override + public Type getType(int field) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + return columnHandles.get(field).getType(); + } + + @Override + public boolean advanceNextPosition() + { + while (true) { + if (cursorOffset >= split.getEnd()) { + return endOfData(1); // Split end is exclusive. + } + + try { + // Create a fetch request + openFetchRequest(); + if (cursorOffset >= split.getEnd()) { + return endOfData(2); // Split end is exclusive. + } + if (messageAndOffsetIterator.hasNext()) { + MessageAndOffset currentMessageAndOffset = messageAndOffsetIterator.next(); + return nextRow(currentMessageAndOffset); + } + } + catch (OffsetOutOfRangeException e) { + e.printStackTrace(); + return endOfData(4); + } + messageAndOffsetIterator = null; + } + } + + private boolean endOfData(int from) + { + if (!reported.getAndSet(true)) { + log.info("Found (from %d) a total of %d messages with %d bytes (%d compressed bytes expected). Last Offset: %d (%d, %d)", + from, totalMessages, totalBytes, split.getEnd() - split.getStart(), + cursorOffset, split.getStart(), split.getEnd()); + } + return false; + } + + private boolean nextRow(MessageAndOffset messageAndOffset) + { + totalBytes += messageAndOffset.message().payloadSize(); + totalMessages++; + + byte[] keyData = EMPTY_BYTE_ARRAY; + byte[] messageData = EMPTY_BYTE_ARRAY; + + ByteBuffer message = messageAndOffset.message().payload(); + if (message != null) { + messageData = new byte[message.remaining()]; + message.get(messageData); + } + + Set fieldValueProviders = new HashSet<>(); + + fieldValueProviders.addAll(globalInternalFieldValueProviders); + fieldValueProviders.add(KafkaInternalFieldDescription.SEGMENT_COUNT_FIELD.forLongValue(totalMessages)); + fieldValueProviders.add(KafkaInternalFieldDescription.PARTITION_OFFSET_FIELD.forLongValue(messageAndOffset.offset())); + fieldValueProviders.add(KafkaInternalFieldDescription.MESSAGE_FIELD.forByteValue(messageData)); + fieldValueProviders.add(KafkaInternalFieldDescription.MESSAGE_LENGTH_FIELD.forLongValue(messageData.length)); + fieldValueProviders.add(KafkaInternalFieldDescription.KEY_FIELD.forByteValue(keyData)); + fieldValueProviders.add(KafkaInternalFieldDescription.KEY_LENGTH_FIELD.forLongValue(keyData.length)); + fieldValueProviders.add(KafkaInternalFieldDescription.KEY_CORRUPT_FIELD.forBooleanValue(keyDecoder.decodeRow(keyData, null, fieldValueProviders, columnHandles, keyFieldDecoders))); + fieldValueProviders.add(KafkaInternalFieldDescription.MESSAGE_CORRUPT_FIELD.forBooleanValue(messageDecoder.decodeRow(messageData, null, fieldValueProviders, columnHandles, messageFieldDecoders))); + fieldValueProviders.add(KafkaInternalFieldDescription.OFFSET_TIMESTAMP_FIELD.forLongValue(populateOffsetTimestamp(startTs, endTs))); + + this.fieldValueProviders = new FieldValueProvider[columnHandles.size()]; + + // If a value provider for a requested internal column is present, assign the + // value to the internal cache. It is possible that an internal column is present + // where no value provider exists (e.g. the '_corrupt' column with the DummyRowDecoder). + // In that case, the cache is null (and the column is reported as null). + for (int i = 0; i < columnHandles.size(); i++) { + for (FieldValueProvider fieldValueProvider : fieldValueProviders) { + if (fieldValueProvider.accept(columnHandles.get(i))) { + this.fieldValueProviders[i] = fieldValueProvider; + break; // for(InternalColumnProvider... + } + } + } + + return true; // Advanced successfully. + } + + private void openFetchRequest() + { + if (messageAndOffsetIterator == null) { + log.info("Fetching %d bytes from partition %d @offset %d (%d - %d) -- %d messages read so far", + fetchSize, split.getPartitionId(), cursorOffset, split.getStart(), split.getEnd(), totalMessages); + cursorOffset += fetchedSize; + if (cursorOffset < split.getEnd()) { + FetchRequest req = new FetchRequest(split.getTopicName(), split.getPartitionId(), cursorOffset, fetchSize); + SimpleConsumer consumer = consumerManager.getConsumer(split.getLeader()); + + ByteBufferMessageSet fetch = consumer.fetch(req); + log.debug("\t...fetched %s bytes, validBytes=%s, initialOffset=%s", fetch.sizeInBytes(), fetch.validBytes(), fetch.getInitialOffset()); + int errorCode = fetch.getErrorCode(); + if (errorCode != ErrorMapping.NoError() && errorCode != ErrorMapping.OffsetOutOfRangeCode()) { + log.warn("Fetch response has error: %d", errorCode); + throw new PrestoException(KAFKA_SPLIT_ERROR, "could not fetch data from Kafka, error code is '" + errorCode + "'"); + } + + fetchedSize = fetch.validBytes(); + messageAndOffsetIterator = fetch.iterator(); + } + } + } + + private long populateOffsetTimestamp(long startTs, long endTs) + { + if (startTs == OffsetRequest.EarliestTime()) { + startTs = 0; + } + + if (endTs == OffsetRequest.LatestTime()) { + endTs = System.currentTimeMillis(); + } + + return startTs + (endTs - startTs) / 2; + } + + @SuppressWarnings("SimplifiableConditionalExpression") + @Override + public boolean getBoolean(int field) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + + checkFieldType(field, boolean.class); + return isNull(field) ? false : fieldValueProviders[field].getBoolean(); + } + + @Override + public long getLong(int field) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + + checkFieldType(field, long.class); + return isNull(field) ? 0L : fieldValueProviders[field].getLong(); + } + + @Override + public double getDouble(int field) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + + checkFieldType(field, double.class); + return isNull(field) ? 0.0d : fieldValueProviders[field].getDouble(); + } + + @Override + public Slice getSlice(int field) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + + checkFieldType(field, Slice.class); + return isNull(field) ? Slices.EMPTY_SLICE : fieldValueProviders[field].getSlice(); + } + + @Override + public Object getObject(int field) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + + checkFieldType(field, Block.class); + + return isNull(field) ? null : fieldValueProviders[field].getBlock(); + } + + @Override + public boolean isNull(int field) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + + return fieldValueProviders[field] == null || fieldValueProviders[field].isNull(); + } + + private void checkFieldType(int field, Class expected) + { + Class actual = getType(field).getJavaType(); + checkArgument(actual == expected, "Expected field %s to be type %s but is %s", field, expected, actual); + } + + @Override + public void close() + { + } + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSetProvider.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSetProvider.java new file mode 100644 index 000000000000..ba7b2b9e11a6 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSetProvider.java @@ -0,0 +1,97 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.DecoderRegistry; +import com.facebook.presto.decoder.FieldDecoder; +import com.facebook.presto.decoder.RowDecoder; +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.RecordSet; +import com.facebook.presto.spi.connector.ConnectorRecordSetProvider; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import javax.inject.Inject; + +import java.util.List; + +import static com.facebook.presto.kafka.KafkaHandleResolver.convertColumnHandle; +import static com.facebook.presto.kafka.KafkaHandleResolver.convertSplit; +import static java.util.Objects.requireNonNull; + +/** + * Factory for Kafka specific {@link RecordSet} instances. + */ +public class KafkaRecordSetProvider + implements ConnectorRecordSetProvider +{ + private final KafkaSimpleConsumerManager consumerManager; + private final DecoderRegistry registry; + private final KafkaConnectorConfig config; + + @Inject + public KafkaRecordSetProvider(DecoderRegistry registry, KafkaSimpleConsumerManager consumerManager, KafkaConnectorConfig config) + { + this.registry = requireNonNull(registry, "registry is null"); + this.consumerManager = requireNonNull(consumerManager, "consumerManager is null"); + this.config = requireNonNull(config, "config is null"); + } + + @Override + public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorSplit split, List columns) + { + KafkaSplit kafkaSplit = convertSplit(split); + + ImmutableList.Builder handleBuilder = ImmutableList.builder(); + ImmutableMap.Builder> keyFieldDecoderBuilder = ImmutableMap.builder(); + ImmutableMap.Builder> messageFieldDecoderBuilder = ImmutableMap.builder(); + + RowDecoder keyDecoder = registry.getRowDecoder(kafkaSplit.getKeyDataFormat()); + RowDecoder messageDecoder = registry.getRowDecoder(kafkaSplit.getMessageDataFormat()); + + for (ColumnHandle handle : columns) { + KafkaColumnHandle columnHandle = convertColumnHandle(handle); + handleBuilder.add(columnHandle); + + if (!columnHandle.isInternal()) { + if (columnHandle.isKeyDecoder()) { + FieldDecoder fieldDecoder = registry.getFieldDecoder( + kafkaSplit.getKeyDataFormat(), + columnHandle.getType().getJavaType(), + columnHandle.getDataFormat()); + + keyFieldDecoderBuilder.put(columnHandle, fieldDecoder); + } + else { + FieldDecoder fieldDecoder = registry.getFieldDecoder( + kafkaSplit.getMessageDataFormat(), + columnHandle.getType().getJavaType(), + columnHandle.getDataFormat()); + + messageFieldDecoderBuilder.put(columnHandle, fieldDecoder); + } + } + } + + ImmutableList handles = handleBuilder.build(); + ImmutableMap> keyFieldDecoders = keyFieldDecoderBuilder.build(); + ImmutableMap> messageFieldDecoders = messageFieldDecoderBuilder.build(); + + return new KafkaRecordSet(kafkaSplit, consumerManager, handles, keyDecoder, messageDecoder, keyFieldDecoders, messageFieldDecoders, config.getFetchSize()); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSimpleConsumerManager.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSimpleConsumerManager.java new file mode 100644 index 000000000000..53756c614efa --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSimpleConsumerManager.java @@ -0,0 +1,103 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.spi.HostAddress; +import com.facebook.presto.spi.NodeManager; +import com.google.common.base.Throwables; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import io.airlift.log.Logger; +import kafka.javaapi.consumer.SimpleConsumer; + +import javax.annotation.PreDestroy; +import javax.inject.Inject; + +import java.util.Map; +import java.util.concurrent.ExecutionException; + +import static java.lang.Math.toIntExact; +import static java.util.Objects.requireNonNull; + +/** + * Manages connections to the Kafka nodes. A worker may connect to multiple Kafka nodes depending on the segments and partitions + * it needs to process. According to the Kafka source code, a Kafka {@link kafka.javaapi.consumer.SimpleConsumer} is thread-safe. + */ +public class KafkaSimpleConsumerManager +{ + private static final Logger log = Logger.get(KafkaSimpleConsumerManager.class); + + private final LoadingCache consumerCache; + + private final String connectorId; + private final NodeManager nodeManager; + private final int connectTimeoutMillis; + private final int bufferSizeBytes; + + @Inject + public KafkaSimpleConsumerManager( + KafkaConnectorId connectorId, + KafkaConnectorConfig kafkaConnectorConfig, + NodeManager nodeManager) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); + + requireNonNull(kafkaConnectorConfig, "kafkaConfig is null"); + this.connectTimeoutMillis = toIntExact(kafkaConnectorConfig.getKafkaConnectTimeout().toMillis()); + this.bufferSizeBytes = toIntExact(kafkaConnectorConfig.getKafkaBufferSize().toBytes()); + + this.consumerCache = CacheBuilder.newBuilder().build(new SimpleConsumerCacheLoader()); + } + + @PreDestroy + public void tearDown() + { + for (Map.Entry entry : consumerCache.asMap().entrySet()) { + try { + entry.getValue().close(); + } + catch (Exception e) { + log.warn(e, "While closing consumer %s:", entry.getKey()); + } + } + } + + public SimpleConsumer getConsumer(HostAddress host) + { + requireNonNull(host, "host is null"); + try { + return consumerCache.get(host); + } + catch (ExecutionException e) { + throw Throwables.propagate(e.getCause()); + } + } + + private class SimpleConsumerCacheLoader + extends CacheLoader + { + @Override + public SimpleConsumer load(HostAddress host) + throws Exception + { + log.debug("Creating new Consumer for %s", host); + return new SimpleConsumer(host.getHostText(), + host.getPort(), + connectTimeoutMillis, + bufferSizeBytes); + } + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplit.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplit.java new file mode 100644 index 000000000000..17edf7d47e6e --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplit.java @@ -0,0 +1,165 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.HostAddress; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Represents a kafka specific {@link ConnectorSplit}. Each split is mapped to a segment file on disk (based off the segment offset start() and end() values) so that + * a partition can be processed by reading segment files from partition leader. Otherwise, a Kafka topic could only be processed along partition boundaries. + *

+ * When planning to process a Kafka topic with Presto, using smaller than the recommended segment size (default is 1G) allows Presto to optimize early and process a topic + * with more workers in parallel. + */ +public class KafkaSplit + implements ConnectorSplit +{ + private final String connectorId; + private final String topicName; + private final String keyDataFormat; + private final String messageDataFormat; + private final int partitionId; + private final long start; + private final long end; + private final HostAddress leader; + private final long startTs; + private final long endTs; + + @JsonCreator + public KafkaSplit( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("topicName") String topicName, + @JsonProperty("keyDataFormat") String keyDataFormat, + @JsonProperty("messageDataFormat") String messageDataFormat, + @JsonProperty("partitionId") int partitionId, + @JsonProperty("start") long start, + @JsonProperty("end") long end, + @JsonProperty("leader") HostAddress leader, + @JsonProperty("startTs") long startTs, + @JsonProperty("endTs") long endTs) + { + this.connectorId = requireNonNull(connectorId, "connector id is null"); + this.topicName = requireNonNull(topicName, "topicName is null"); + this.keyDataFormat = requireNonNull(keyDataFormat, "dataFormat is null"); + this.messageDataFormat = requireNonNull(messageDataFormat, "messageDataFormat is null"); + this.partitionId = partitionId; + this.start = start; + this.end = end; + this.leader = requireNonNull(leader, "leader address is null"); + this.startTs = startTs; + this.endTs = endTs; + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public long getStart() + { + return start; + } + + @JsonProperty + public long getEnd() + { + return end; + } + + @JsonProperty + public String getTopicName() + { + return topicName; + } + + @JsonProperty + public String getKeyDataFormat() + { + return keyDataFormat; + } + + @JsonProperty + public String getMessageDataFormat() + { + return messageDataFormat; + } + + @JsonProperty + public int getPartitionId() + { + return partitionId; + } + + @JsonProperty + public HostAddress getLeader() + { + return leader; + } + + @JsonProperty + public long getStartTs() + { + return startTs; + } + + @JsonProperty + public long getEndTs() + { + return endTs; + } + + @Override + public boolean isRemotelyAccessible() + { + return true; + } + + @Override + public List getAddresses() + { + return ImmutableList.of(leader); + } + + @Override + public Object getInfo() + { + return this; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("connectorId", connectorId) + .add("topicName", topicName) + .add("keyDataFormat", keyDataFormat) + .add("messageDataFormat", messageDataFormat) + .add("partitionId", partitionId) + .add("start", start) + .add("end", end) + .add("leader", leader) + .toString(); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java new file mode 100644 index 000000000000..056bfea7e2b2 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java @@ -0,0 +1,153 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.ConnectorSplitSource; +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.facebook.presto.spi.FixedSplitSource; +import com.facebook.presto.spi.HostAddress; +import com.facebook.presto.spi.connector.ConnectorSplitManager; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.google.common.collect.ImmutableList; +import io.airlift.log.Logger; +import kafka.api.OffsetRequest; +import kafka.cluster.Broker; +import kafka.cluster.Cluster; +import kafka.cluster.Partition; +import kafka.javaapi.consumer.SimpleConsumer; +import kafka.utils.ZkUtils; +import org.I0Itec.zkclient.ZkClient; + +import javax.inject.Inject; + +import java.util.List; + +import static com.facebook.presto.kafka.KafkaHandleResolver.convertLayout; +import static java.util.Objects.requireNonNull; + +/** + * Kafka specific implementation of {@link ConnectorSplitManager}. + */ +public class KafkaSplitManager + implements ConnectorSplitManager +{ + private static final Logger log = Logger.get(KafkaSplitManager.class); + + private final String connectorId; + private final KafkaSimpleConsumerManager consumerManager; + private final KafkaConnectorConfig config; + + @Inject + public KafkaSplitManager( + KafkaConnectorId connectorId, + KafkaConnectorConfig kafkaConnectorConfig, + KafkaSimpleConsumerManager consumerManager) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + this.consumerManager = requireNonNull(consumerManager, "consumerManager is null"); + + requireNonNull(kafkaConnectorConfig, "kafkaConfig is null"); + this.config = kafkaConnectorConfig; + } + + @Override + public ConnectorSplitSource getSplits(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableLayoutHandle layout, SplitSchedulingStrategy splitSchedulingStrategy) + { + KafkaTableHandle kafkaTableHandle = convertLayout(layout).getTable(); + ZkClient zkClient = KafkaUtil.newZkClient(config.getZkEndpoint()); + + ImmutableList.Builder splits = ImmutableList.builder(); + Cluster cluster = ZkUtils.getCluster(zkClient); + List partitions = KafkaUtil.getPartitionsForTopic(zkClient, kafkaTableHandle.getTopicName()); + + long estimatedTotalSize = 0L; + + for (Partition part : partitions) { + log.debug("Adding Partition %s/%s from broker %s", kafkaTableHandle.getTopicName(), part.partId(), part.brokerId()); + Broker leader = cluster.getBroker(part.brokerId()).get(); + + if (leader == null) { // Leader election going on... + log.error("No leader for partition %s/%s found!", kafkaTableHandle.getTopicName(), part.partId()); + continue; + } + + HostAddress partitionLeader = HostAddress.fromParts(leader.host(), leader.port()); + + SimpleConsumer leaderConsumer = consumerManager.getConsumer(partitionLeader); + // Kafka contains a reverse list of "end - start" pairs for the splits + + KafkaTableLayoutHandle layoutHandle = (KafkaTableLayoutHandle) layout; + long startTs = layoutHandle.getOffsetStartTs(); + long endTs = layoutHandle.getOffsetEndTs(); + + long[] offsets = findAllOffsets(leaderConsumer, kafkaTableHandle.getTopicName(), part.partId(), startTs, endTs); + for (int i = offsets.length - 1; i > 0; i--) { + KafkaSplit split = new KafkaSplit( + connectorId, + kafkaTableHandle.getTopicName(), + kafkaTableHandle.getKeyDataFormat(), + kafkaTableHandle.getMessageDataFormat(), + part.partId(), + offsets[i], + offsets[i - 1], + partitionLeader, + startTs, + endTs); + splits.add(split); + + long splitSize = (split.getEnd() - split.getStart()) / 1024 / 1024; + log.debug("Split summarize: %s-%s (%sMB)", split.getStart(), split.getEnd(), splitSize); + estimatedTotalSize += splitSize; + } + } + + ImmutableList builtSplits = splits.build(); + log.info("Built " + builtSplits.size() + " splits"); + + log.info("EstimatedTotalSize: %s", estimatedTotalSize); + return new FixedSplitSource(builtSplits); + } + + private static long[] findAllOffsets(SimpleConsumer consumer, String topicName, int partitionId, long startTs, long endTs) + { + // startTs: start timestamp, or -2/null as earliest + // endTs: end timestamp, or -1/null as latest + if (startTs >= endTs && endTs != OffsetRequest.LatestTime()) { + throw new IllegalArgumentException(String.format("Invalid Kafka Offset start/end pair: %s - %s", startTs, endTs)); + } + + long[] offsetsBeforeStartTs = consumer.getOffsetsBefore(topicName, partitionId, startTs, Integer.MAX_VALUE); + long[] offsetsBeforeEndTs = consumer.getOffsetsBefore(topicName, partitionId, endTs, Integer.MAX_VALUE); + log.debug("NumOffsetsBeforeStartTs=%s, NumOffsetsBeforeEndTs=%s", offsetsBeforeStartTs.length, offsetsBeforeEndTs.length); + + if (offsetsBeforeStartTs.length == 0) { + return offsetsBeforeEndTs; + } + + long[] offsets = new long[offsetsBeforeEndTs.length - offsetsBeforeStartTs.length + 1]; + long startOffset = offsetsBeforeStartTs[0]; + + for (int i = 0; i < offsetsBeforeEndTs.length; i++) { + if (offsetsBeforeEndTs[i] == startOffset) { + offsets[i] = startOffset; + break; + } + offsets[i] = offsetsBeforeEndTs[i]; + } + + return offsets; + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableDescriptionSupplier.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableDescriptionSupplier.java new file mode 100644 index 000000000000..2fcdbe539ef4 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableDescriptionSupplier.java @@ -0,0 +1,138 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.decoder.dummy.DummyRowDecoder; +import com.facebook.presto.spi.SchemaTableName; +import com.google.common.base.Splitter; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.airlift.json.JsonCodec; +import io.airlift.log.Logger; + +import javax.inject.Inject; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Supplier; + +import static com.google.common.base.MoreObjects.firstNonNull; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static java.nio.file.Files.readAllBytes; +import static java.util.Arrays.asList; +import static java.util.Objects.requireNonNull; + +public class KafkaTableDescriptionSupplier + implements Supplier> +{ + private static final Logger log = Logger.get(KafkaTableDescriptionSupplier.class); + + private final JsonCodec topicDescriptionCodec; + private final File tableDescriptionDir; + private final String defaultSchema; + private final Set tableNames; + + @Inject + KafkaTableDescriptionSupplier(KafkaConnectorConfig kafkaConnectorConfig, + JsonCodec topicDescriptionCodec) + { + this.topicDescriptionCodec = requireNonNull(topicDescriptionCodec, "topicDescriptionCodec is null"); + + requireNonNull(kafkaConnectorConfig, "kafkaConfig is null"); + this.tableDescriptionDir = kafkaConnectorConfig.getTableDescriptionDir(); + this.defaultSchema = kafkaConnectorConfig.getDefaultSchema(); + this.tableNames = ImmutableSet.copyOf(kafkaConnectorConfig.getTableNames()); + } + + @Override + public Map get() + { + ImmutableMap.Builder builder = ImmutableMap.builder(); + + log.debug("Loading kafka table definitions from %s", tableDescriptionDir.getAbsolutePath()); + + try { + for (File file : listFiles(tableDescriptionDir)) { + if (file.isFile() && file.getName().endsWith(".json")) { + KafkaTopicDescription table = topicDescriptionCodec.fromJson(readAllBytes(file.toPath())); + String schemaName = firstNonNull(table.getSchemaName(), defaultSchema); + log.debug("Kafka table %s.%s: %s", schemaName, table.getTableName(), table); + builder.put(new SchemaTableName(schemaName, table.getTableName()), table); + } + } + + Map tableDefinitions = builder.build(); + + log.debug("Loaded Table definitions: %s", tableDefinitions.keySet()); + + builder = ImmutableMap.builder(); + for (String definedTable : tableNames) { + SchemaTableName tableName; + try { + tableName = parseTableName(definedTable); + } + catch (IllegalArgumentException iae) { + tableName = new SchemaTableName(defaultSchema, definedTable); + } + + if (tableDefinitions.containsKey(tableName)) { + KafkaTopicDescription kafkaTable = tableDefinitions.get(tableName); + log.debug("Found Table definition for %s: %s", tableName, kafkaTable); + builder.put(tableName, kafkaTable); + } + else { + // A dummy table definition only supports the internal columns. + log.debug("Created dummy Table definition for %s", tableName); + builder.put(tableName, new KafkaTopicDescription(tableName.getTableName(), + tableName.getSchemaName(), + definedTable, + new KafkaTopicFieldGroup(DummyRowDecoder.NAME, ImmutableList.of()), + new KafkaTopicFieldGroup(DummyRowDecoder.NAME, ImmutableList.of()))); + } + } + + return builder.build(); + } + catch (IOException e) { + log.warn(e, "Error: "); + throw Throwables.propagate(e); + } + } + + private static List listFiles(File dir) + { + if ((dir != null) && dir.isDirectory()) { + File[] files = dir.listFiles(); + if (files != null) { + log.debug("Considering files: %s", asList(files)); + return ImmutableList.copyOf(files); + } + } + return ImmutableList.of(); + } + + private static SchemaTableName parseTableName(String schemaTableName) + { + checkArgument(!isNullOrEmpty(schemaTableName), "schemaTableName is null or is empty"); + List parts = Splitter.on('.').splitToList(schemaTableName); + checkArgument(parts.size() == 2, "Invalid schemaTableName: %s", schemaTableName); + return new SchemaTableName(parts.get(0), parts.get(1)); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableHandle.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableHandle.java new file mode 100644 index 000000000000..8475b2c68c98 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableHandle.java @@ -0,0 +1,151 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.SchemaTableName; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Kafka specific {@link ConnectorTableHandle}. + */ +public final class KafkaTableHandle + implements ConnectorTableHandle +{ + /** + * connector id + */ + private final String connectorId; + + /** + * The schema name for this table. Is set through configuration and read + * using {@link KafkaConnectorConfig#getDefaultSchema()}. Usually 'default'. + */ + private final String schemaName; + + /** + * The table name used by presto. + */ + private final String tableName; + + /** + * The topic name that is read from Kafka. + */ + private final String topicName; + + private final String keyDataFormat; + private final String messageDataFormat; + + @JsonCreator + public KafkaTableHandle( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("schemaName") String schemaName, + @JsonProperty("tableName") String tableName, + @JsonProperty("topicName") String topicName, + @JsonProperty("keyDataFormat") String keyDataFormat, + @JsonProperty("messageDataFormat") String messageDataFormat) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null"); + this.schemaName = requireNonNull(schemaName, "schemaName is null"); + this.tableName = requireNonNull(tableName, "tableName is null"); + this.topicName = requireNonNull(topicName, "topicName is null"); + this.keyDataFormat = requireNonNull(keyDataFormat, "keyDataFormat is null"); + this.messageDataFormat = requireNonNull(messageDataFormat, "messageDataFormat is null"); + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public String getSchemaName() + { + return schemaName; + } + + @JsonProperty + public String getTableName() + { + return tableName; + } + + @JsonProperty + public String getTopicName() + { + return topicName; + } + + @JsonProperty + public String getKeyDataFormat() + { + return keyDataFormat; + } + + @JsonProperty + public String getMessageDataFormat() + { + return messageDataFormat; + } + + public SchemaTableName toSchemaTableName() + { + return new SchemaTableName(schemaName, tableName); + } + + @Override + public int hashCode() + { + return Objects.hash(connectorId, schemaName, tableName, topicName, keyDataFormat, messageDataFormat); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + KafkaTableHandle other = (KafkaTableHandle) obj; + return Objects.equals(this.connectorId, other.connectorId) + && Objects.equals(this.schemaName, other.schemaName) + && Objects.equals(this.tableName, other.tableName) + && Objects.equals(this.topicName, other.topicName) + && Objects.equals(this.keyDataFormat, other.keyDataFormat) + && Objects.equals(this.messageDataFormat, other.messageDataFormat); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("connectorId", connectorId) + .add("schemaName", schemaName) + .add("tableName", tableName) + .add("topicName", topicName) + .add("keyDataFormat", keyDataFormat) + .add("messageDataFormat", messageDataFormat) + .toString(); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java new file mode 100644 index 000000000000..57e6ffc6e615 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java @@ -0,0 +1,64 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import kafka.api.OffsetRequest; + +import static java.util.Objects.requireNonNull; + +public class KafkaTableLayoutHandle + implements ConnectorTableLayoutHandle +{ + private final KafkaTableHandle table; + private final long offsetStartTs; + private final long offsetEndTs; + + @JsonCreator + public KafkaTableLayoutHandle( + @JsonProperty("table") KafkaTableHandle table, + @JsonProperty("offset_start_ts") Long offsetStartTs, + @JsonProperty("offset_end_ts") Long offsetEndTs) + { + this.table = requireNonNull(table, "table is null"); + this.offsetStartTs = offsetStartTs == null ? OffsetRequest.EarliestTime() : offsetStartTs; + this.offsetEndTs = offsetEndTs == null ? OffsetRequest.LatestTime() : offsetEndTs; + } + + @JsonProperty + public KafkaTableHandle getTable() + { + return table; + } + + @JsonProperty + public long getOffsetStartTs() + { + return offsetStartTs; + } + + @JsonProperty + public long getOffsetEndTs() + { + return offsetEndTs; + } + + @Override + public String toString() + { + return table.toString(); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicDescription.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicDescription.java new file mode 100644 index 000000000000..443bb6828940 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicDescription.java @@ -0,0 +1,92 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static java.util.Objects.requireNonNull; + +/** + * Json description to parse a row on a Kafka topic. A row contains a message and an optional key. See the documentation for the exact JSON syntax. + */ +public class KafkaTopicDescription +{ + private final String tableName; + private final String topicName; + private final String schemaName; + private final KafkaTopicFieldGroup key; + private final KafkaTopicFieldGroup message; + + @JsonCreator + public KafkaTopicDescription( + @JsonProperty("tableName") String tableName, + @JsonProperty("schemaName") String schemaName, + @JsonProperty("topicName") String topicName, + @JsonProperty("key") KafkaTopicFieldGroup key, + @JsonProperty("message") KafkaTopicFieldGroup message) + { + checkArgument(!isNullOrEmpty(tableName), "tableName is null or is empty"); + this.tableName = tableName; + this.topicName = requireNonNull(topicName, "topicName is null"); + this.schemaName = schemaName; + this.key = key; + this.message = message; + } + + @JsonProperty + public String getTableName() + { + return tableName; + } + + @JsonProperty + public String getTopicName() + { + return topicName; + } + + @JsonProperty + public String getSchemaName() + { + return schemaName; + } + + @JsonProperty + public KafkaTopicFieldGroup getKey() + { + return key; + } + + @JsonProperty + public KafkaTopicFieldGroup getMessage() + { + return message; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("tableName", tableName) + .add("topicName", topicName) + .add("schemaName", schemaName) + .add("key", key) + .add("message", message) + .toString(); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldDescription.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldDescription.java new file mode 100644 index 000000000000..38c9861ea5f5 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldDescription.java @@ -0,0 +1,159 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.spi.ColumnMetadata; +import com.facebook.presto.spi.type.Type; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static java.util.Objects.requireNonNull; + +/** + * Json description to parse a single field from a Kafka topic message. See {@link com.facebook.presto.kafka.KafkaTopicDescription} for more details. + */ +public final class KafkaTopicFieldDescription +{ + private final String name; + private final Type type; + private final String mapping; + private final String comment; + private final String dataFormat; + private final String formatHint; + private final boolean hidden; + + @JsonCreator + public KafkaTopicFieldDescription( + @JsonProperty("name") String name, + @JsonProperty("type") Type type, + @JsonProperty("mapping") String mapping, + @JsonProperty("comment") String comment, + @JsonProperty("dataFormat") String dataFormat, + @JsonProperty("formatHint") String formatHint, + @JsonProperty("hidden") boolean hidden) + { + checkArgument(!isNullOrEmpty(name), "name is null or is empty"); + this.name = name; + this.type = requireNonNull(type, "type is null"); + this.mapping = mapping; + this.comment = comment; + this.dataFormat = dataFormat; + this.formatHint = formatHint; + this.hidden = hidden; + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public Type getType() + { + return type; + } + + @JsonProperty + public String getMapping() + { + return mapping; + } + + @JsonProperty + public String getComment() + { + return comment; + } + + @JsonProperty + public String getDataFormat() + { + return dataFormat; + } + + @JsonProperty + public String getFormatHint() + { + return formatHint; + } + + @JsonProperty + public boolean isHidden() + { + return hidden; + } + + KafkaColumnHandle getColumnHandle(String connectorId, boolean keyDecoder, int index) + { + return new KafkaColumnHandle(connectorId, + index, + getName(), + getType(), + getMapping(), + getDataFormat(), + getFormatHint(), + keyDecoder, + isHidden(), + false); + } + + ColumnMetadata getColumnMetadata() + { + return new ColumnMetadata(getName(), getType(), getComment(), isHidden()); + } + + @Override + public int hashCode() + { + return Objects.hash(name, type, mapping, dataFormat, formatHint, hidden); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + KafkaTopicFieldDescription other = (KafkaTopicFieldDescription) obj; + return Objects.equals(this.name, other.name) && + Objects.equals(this.type, other.type) && + Objects.equals(this.mapping, other.mapping) && + Objects.equals(this.dataFormat, other.dataFormat) && + Objects.equals(this.formatHint, other.formatHint) && + Objects.equals(this.hidden, other.hidden); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("name", name) + .add("type", type) + .add("mapping", mapping) + .add("dataFormat", dataFormat) + .add("formatHint", formatHint) + .add("hidden", hidden) + .toString(); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldGroup.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldGroup.java new file mode 100644 index 000000000000..4b5384fe982e --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldGroup.java @@ -0,0 +1,62 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Groups the field descriptions for message or key. + */ +public class KafkaTopicFieldGroup +{ + private final String dataFormat; + private final List fields; + + @JsonCreator + public KafkaTopicFieldGroup( + @JsonProperty("dataFormat") String dataFormat, + @JsonProperty("fields") List fields) + { + this.dataFormat = requireNonNull(dataFormat, "dataFormat is null"); + this.fields = ImmutableList.copyOf(requireNonNull(fields, "fields is null")); + } + + @JsonProperty + public String getDataFormat() + { + return dataFormat; + } + + @JsonProperty + public List getFields() + { + return fields; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("dataFormat", dataFormat) + .add("fields", fields) + .toString(); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTransactionHandle.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTransactionHandle.java new file mode 100644 index 000000000000..f777be380ef2 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTransactionHandle.java @@ -0,0 +1,22 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; + +public enum KafkaTransactionHandle + implements ConnectorTransactionHandle +{ + INSTANCE +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaUtil.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaUtil.java new file mode 100644 index 000000000000..4edf4f6da06f --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaUtil.java @@ -0,0 +1,159 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import kafka.cluster.Broker; +import kafka.cluster.Partition; +import kafka.consumer.ConsumerConfig; +import kafka.consumer.SimpleConsumer; +import kafka.utils.ZKConfig; +import kafka.utils.ZkUtils; +import org.I0Itec.zkclient.ZkClient; +import scala.collection.JavaConversions; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * A collection of utility methods for accessing Kafka. + * + * @author Raghu Angadi + */ +public final class KafkaUtil +{ + public static final ConsumerConfig DEFAULT_CONSUMER_CONFIG; + + static { + Properties properties = new Properties(); + properties.setProperty("groupid", "this-should-not-be-used"); + DEFAULT_CONSUMER_CONFIG = new ConsumerConfig(properties); + } + + private KafkaUtil() + { + } + + /** + * create ZkClient with default options. + */ + public static ZkClient newZkClient(String zkConnect) + { + // get defaults from ZkConfig. + ZKConfig config = new ZKConfig(new Properties()); + + return new ZkClient(zkConnect, + config.zkSessionTimeoutMs(), + config.zkConnectionTimeoutMs(), + kafka.utils.ZKStringSerializer$.MODULE$); + } + + /** + * Returns partitions for given topic. An empty list if the topic is not + * found. + */ + public static List getPartitionsForTopic(ZkClient zkClient, + String topic) + { + // handle scala <-> java conversions. + scala.collection.Iterator topics = + JavaConversions.asScalaIterator(Iterators.forArray(topic)); + Map> map = + JavaConversions.mapAsJavaMap(ZkUtils.getPartitionsForTopics(zkClient, topics)); + + // since we are asking for just one topic, map's size is 0 or 1. + if (map.size() > 0) { + List partitions = JavaConversions.seqAsJavaList( + map.values().iterator().next()); + // transform string to Partition object + return Lists.newArrayList( + Lists.transform(partitions, + input -> Partition.parse(input))); + } + + return new ArrayList<>(); + } + + /** + * Returns latest offset before the given timestamp. If there is no offset + * avaliable, returns the earliest available. An offset before the timestamp + * may not be available if the messages are already rotated. + */ + public static long getBeforeOrEarliestOffset(SimpleConsumer consumer, + String topic, + int partId, + long time) + { + long[] offsets = consumer.getOffsetsBefore(topic, partId, time, 1); + if (offsets.length == 0) { + // then the earliest offset + offsets = consumer.getOffsetsBefore(topic, partId, -2, 1); + } + + return (offsets.length > 0) ? offsets[0] : 0; + } + + /** + * Returns the topics on given Kafka Server + */ + public static List getTopics(ZkClient zkClient) + { + String topicPath = ZkUtils.BrokerTopicsPath(); + return zkClient.getChildren(topicPath); + } + + /** + * Returns the brokers currently registered + */ + public static List getBrokersIds(ZkClient zkClient) + { + String brokerPath = ZkUtils.BrokerIdsPath(); + List brokers = zkClient.getChildren(brokerPath); + List brokerIds = new ArrayList(); + for (String s : brokers) { + Long l = Long.parseLong(s); + brokerIds.add(l); + } + + return brokerIds; + } + + /** + * Returns the number of partitions for a given topic and broker. + */ + public static Integer getNumPartitions(ZkClient zkClient, String topic, Long broker) + { + String topicPath = ZkUtils.BrokerTopicsPath(); + String partitionPath = topicPath + "/" + topic + "/" + broker.toString(); + + String numPartitions = zkClient.readData(partitionPath, true); + if (numPartitions == null) { + return 0; + } + else { + return Integer.parseInt(numPartitions); + } + } + + public static SimpleConsumer newSimpleConsumer(Broker broker) + { + return new SimpleConsumer(broker.host(), broker.port(), + DEFAULT_CONSUMER_CONFIG.socketTimeoutMs(), + DEFAULT_CONSUMER_CONFIG.socketBufferSize()); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/KafkaQueryRunner.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/KafkaQueryRunner.java new file mode 100644 index 000000000000..1f760f8a4c46 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/KafkaQueryRunner.java @@ -0,0 +1,142 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.Session; +import com.facebook.presto.kafka.util.CodecSupplier; +import com.facebook.presto.kafka.util.EmbeddedKafka; +import com.facebook.presto.kafka.util.TestUtils; +import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.metadata.QualifiedObjectName; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.tests.DistributedQueryRunner; +import com.facebook.presto.tests.TestingPrestoClient; +import com.facebook.presto.tpch.TpchPlugin; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.json.JsonCodec; +import io.airlift.log.Logger; +import io.airlift.log.Logging; +import io.airlift.tpch.TpchTable; + +import java.util.Map; + +import static com.facebook.presto.kafka.util.TestUtils.installKafkaPlugin; +import static com.facebook.presto.kafka.util.TestUtils.loadTpchTopicDescription; +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; +import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; +import static io.airlift.testing.Closeables.closeAllSuppress; +import static io.airlift.units.Duration.nanosSince; +import static java.util.Locale.ENGLISH; +import static java.util.concurrent.TimeUnit.SECONDS; + +public final class KafkaQueryRunner +{ + private KafkaQueryRunner() + { + } + + private static final Logger log = Logger.get("TestQueries"); + private static final String TPCH_SCHEMA = "tpch"; + + public static DistributedQueryRunner createKafkaQueryRunner(EmbeddedKafka embeddedKafka, TpchTable... tables) + throws Exception + { + return createKafkaQueryRunner(embeddedKafka, ImmutableList.copyOf(tables)); + } + + public static DistributedQueryRunner createKafkaQueryRunner(EmbeddedKafka embeddedKafka, Iterable> tables) + throws Exception + { + DistributedQueryRunner queryRunner = null; + try { + queryRunner = new DistributedQueryRunner(createSession(), 2); + + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); + + embeddedKafka.start(); + + for (TpchTable table : tables) { + embeddedKafka.createTopics(kafkaTopicName(table)); + } + + Map topicDescriptions = createTpchTopicDescriptions(queryRunner.getCoordinator().getMetadata(), tables); + + installKafkaPlugin(embeddedKafka, queryRunner, topicDescriptions); + + TestingPrestoClient prestoClient = queryRunner.getClient(); + + log.info("Loading data..."); + long startTime = System.nanoTime(); + for (TpchTable table : tables) { + loadTpchTopic(embeddedKafka, prestoClient, table); + } + log.info("Loading complete in %s", nanosSince(startTime).toString(SECONDS)); + + return queryRunner; + } + catch (Throwable e) { + closeAllSuppress(e, queryRunner, embeddedKafka); + throw e; + } + } + + private static void loadTpchTopic(EmbeddedKafka embeddedKafka, TestingPrestoClient prestoClient, TpchTable table) + { + long start = System.nanoTime(); + log.info("Running import for %s", table.getTableName()); + TestUtils.loadTpchTopic(embeddedKafka, prestoClient, kafkaTopicName(table), new QualifiedObjectName("tpch", TINY_SCHEMA_NAME, table.getTableName().toLowerCase(ENGLISH))); + log.info("Imported %s in %s", 0, table.getTableName(), nanosSince(start).convertToMostSuccinctTimeUnit()); + } + + private static String kafkaTopicName(TpchTable table) + { + return TPCH_SCHEMA + "." + table.getTableName().toLowerCase(ENGLISH); + } + + private static Map createTpchTopicDescriptions(Metadata metadata, Iterable> tables) + throws Exception + { + JsonCodec topicDescriptionJsonCodec = new CodecSupplier<>(KafkaTopicDescription.class, metadata).get(); + + ImmutableMap.Builder topicDescriptions = ImmutableMap.builder(); + for (TpchTable table : tables) { + String tableName = table.getTableName(); + SchemaTableName tpchTable = new SchemaTableName(TPCH_SCHEMA, tableName); + + topicDescriptions.put(loadTpchTopicDescription(topicDescriptionJsonCodec, tpchTable.toString(), tpchTable)); + } + return topicDescriptions.build(); + } + + public static Session createSession() + { + return testSessionBuilder() + .setCatalog("kafka") + .setSchema(TPCH_SCHEMA) + .build(); + } + + public static void main(String[] args) + throws Exception + { + Logging.initialize(); + DistributedQueryRunner queryRunner = createKafkaQueryRunner(EmbeddedKafka.createEmbeddedKafka(), TpchTable.getTables()); + Thread.sleep(10); + Logger log = Logger.get(KafkaQueryRunner.class); + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java new file mode 100644 index 000000000000..e824778a9bee --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java @@ -0,0 +1,68 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.google.common.collect.ImmutableMap; +import io.airlift.configuration.testing.ConfigAssertions; +import org.testng.annotations.Test; + +import java.io.File; +import java.util.Map; + +public class TestKafkaConnectorConfig +{ + @Test + public void testDefaults() + { + ConfigAssertions.assertRecordedDefaults(ConfigAssertions.recordDefaults(KafkaConnectorConfig.class) + .setNodes("") + .setKafkaConnectTimeout("10s") + .setKafkaBufferSize("64kB") + .setDefaultSchema("default") + .setTableNames("") + .setTableDescriptionDir(new File("etc/kafka07/")) + .setHideInternalColumns(true) + .setFetchSize(10485760) + .setZkEndpoint("")); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("kafka.table-description-dir", "/var/lib/kafka") + .put("kafka.table-names", "table1, table2, table3") + .put("kafka.default-schema", "kafka") + .put("kafka.nodes", "localhost:12345,localhost:23456") + .put("kafka.connect-timeout", "1h") + .put("kafka.buffer-size", "1MB") + .put("kafka.hide-internal-columns", "false") + .put("kafka.fetch-size", "10000000") + .put("kafka.zk-endpoint", "localhost:2181") + .build(); + + KafkaConnectorConfig expected = new KafkaConnectorConfig() + .setTableDescriptionDir(new File("/var/lib/kafka")) + .setTableNames("table1, table2, table3") + .setDefaultSchema("kafka") + .setNodes("localhost:12345, localhost:23456") + .setKafkaConnectTimeout("1h") + .setKafkaBufferSize("1MB") + .setHideInternalColumns(false) + .setFetchSize(10000000) + .setZkEndpoint("localhost:2181"); + + ConfigAssertions.assertFullMapping(properties, expected); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaDistributed.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaDistributed.java new file mode 100644 index 000000000000..c31e40f5d62e --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaDistributed.java @@ -0,0 +1,52 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.kafka.util.EmbeddedKafka; +import com.facebook.presto.tests.AbstractTestQueries; +import io.airlift.tpch.TpchTable; +import org.testng.annotations.AfterClass; +import org.testng.annotations.Test; + +import java.io.IOException; + +import static com.facebook.presto.kafka.KafkaQueryRunner.createKafkaQueryRunner; +import static com.facebook.presto.kafka.util.EmbeddedKafka.createEmbeddedKafka; + +@Test +public class TestKafkaDistributed + extends AbstractTestQueries +{ + private final EmbeddedKafka embeddedKafka; + + public TestKafkaDistributed() + throws Exception + { + this(createEmbeddedKafka()); + } + + public TestKafkaDistributed(EmbeddedKafka embeddedKafka) + throws Exception + { + super(() -> createKafkaQueryRunner(embeddedKafka, TpchTable.getTables())); + this.embeddedKafka = embeddedKafka; + } + + @AfterClass(alwaysRun = true) + public void destroy() + throws IOException + { + embeddedKafka.close(); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaIntegrationSmokeTest.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaIntegrationSmokeTest.java new file mode 100644 index 000000000000..faf02f0bfb80 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaIntegrationSmokeTest.java @@ -0,0 +1,52 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.kafka.util.EmbeddedKafka; +import com.facebook.presto.tests.AbstractTestIntegrationSmokeTest; +import org.testng.annotations.AfterClass; +import org.testng.annotations.Test; + +import java.io.IOException; + +import static com.facebook.presto.kafka.KafkaQueryRunner.createKafkaQueryRunner; +import static com.facebook.presto.kafka.util.EmbeddedKafka.createEmbeddedKafka; +import static io.airlift.tpch.TpchTable.ORDERS; + +@Test +public class TestKafkaIntegrationSmokeTest + extends AbstractTestIntegrationSmokeTest +{ + private final EmbeddedKafka embeddedKafka; + + public TestKafkaIntegrationSmokeTest() + throws Exception + { + this(createEmbeddedKafka()); + } + + public TestKafkaIntegrationSmokeTest(EmbeddedKafka embeddedKafka) + throws Exception + { + super(() -> createKafkaQueryRunner(embeddedKafka, ORDERS)); + this.embeddedKafka = embeddedKafka; + } + + @AfterClass(alwaysRun = true) + public void destroy() + throws IOException + { + embeddedKafka.close(); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaPlugin.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaPlugin.java new file mode 100644 index 000000000000..ad5e19b0eef4 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaPlugin.java @@ -0,0 +1,46 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.spi.connector.Connector; +import com.facebook.presto.spi.connector.ConnectorFactory; +import com.facebook.presto.testing.TestingConnectorContext; +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import static com.google.common.collect.Iterables.getOnlyElement; +import static io.airlift.testing.Assertions.assertInstanceOf; +import static org.testng.Assert.assertNotNull; + +@Test +public class TestKafkaPlugin +{ + @Test + public void testSpinup() + { + KafkaPlugin plugin = new KafkaPlugin(); + + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + assertInstanceOf(factory, KafkaConnectorFactory.class); + + Connector c = factory.create( + "test-connector", + ImmutableMap.builder() + .put("kafka.table-names", "test") + .put("kafka.nodes", "localhost:9092") + .build(), + new TestingConnectorContext()); + assertNotNull(c); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestManySegments.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestManySegments.java new file mode 100644 index 000000000000..56c39553fe76 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestManySegments.java @@ -0,0 +1,118 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.Session; +import com.facebook.presto.kafka.util.EmbeddedKafka; +import com.facebook.presto.kafka.util.TestUtils; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.type.BigintType; +import com.facebook.presto.testing.MaterializedResult; +import com.facebook.presto.tests.StandaloneQueryRunner; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import kafka.javaapi.producer.ProducerData; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.util.Collections; +import java.util.Properties; +import java.util.UUID; + +import static com.facebook.presto.kafka.util.EmbeddedKafka.CloseableProducer; +import static com.facebook.presto.kafka.util.TestUtils.createEmptyTopicDescription; +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; +import static com.facebook.presto.testing.assertions.Assert.assertEquals; + +@Test(singleThreaded = true) +public class TestManySegments +{ + private static final Session SESSION = testSessionBuilder() + .setCatalog("kafka") + .setSchema("default") + .build(); + + private EmbeddedKafka embeddedKafka; + private String topicName; + private StandaloneQueryRunner queryRunner; + + @BeforeClass + public void startKafka() + throws Exception + { + embeddedKafka = EmbeddedKafka.createEmbeddedKafka(); + embeddedKafka.start(); + + topicName = "test_" + UUID.randomUUID().toString().replaceAll("-", "_"); + + Properties topicProperties = new Properties(); + topicProperties.setProperty("segment.bytes", "1048576"); + + embeddedKafka.createTopics(1, 1, topicProperties, topicName); + + try (CloseableProducer producer = embeddedKafka.createProducer()) { + int jMax = 10_000; + int iMax = 100_000 / jMax; + for (long i = 0; i < iMax; i++) { + ImmutableList.Builder> builder = ImmutableList.builder(); + for (long j = 0; j < jMax; j++) { + builder.add(new ProducerData<>(topicName, i, Collections.singletonList(ImmutableMap.of("id", String.format("%05d", i * jMax + j), "value", UUID.randomUUID().toString())))); + } + producer.send(builder.build()); + } + } + } + + @AfterClass(alwaysRun = true) + public void stopKafka() + throws Exception + { + embeddedKafka.close(); + } + + @BeforeMethod + public void spinUp() + throws Exception + { + this.queryRunner = new StandaloneQueryRunner(SESSION); + + TestUtils.installKafkaPlugin(embeddedKafka, queryRunner, + ImmutableMap.builder() + .put(createEmptyTopicDescription(topicName, new SchemaTableName("default", topicName))) + .build()); + } + + @AfterMethod(alwaysRun = true) + public void tearDown() + throws Exception + { + queryRunner.close(); + } + + @Test + public void testManySegments() + throws Exception + { + MaterializedResult result = queryRunner.execute("SELECT count(_message) from " + topicName); + + MaterializedResult expected = MaterializedResult.resultBuilder(SESSION, BigintType.BIGINT) + .row(100000L) + .build(); + + assertEquals(result, expected); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestMinimalFunctionality.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestMinimalFunctionality.java new file mode 100644 index 000000000000..d26e77b4f9dc --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestMinimalFunctionality.java @@ -0,0 +1,147 @@ +/* + * Licensed 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 com.facebook.presto.kafka; + +import com.facebook.presto.Session; +import com.facebook.presto.kafka.util.EmbeddedKafka; +import com.facebook.presto.kafka.util.TestUtils; +import com.facebook.presto.metadata.QualifiedObjectName; +import com.facebook.presto.metadata.TableHandle; +import com.facebook.presto.security.AllowAllAccessControl; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.type.BigintType; +import com.facebook.presto.testing.MaterializedResult; +import com.facebook.presto.tests.StandaloneQueryRunner; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import kafka.javaapi.producer.ProducerData; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.util.Collections; +import java.util.Optional; +import java.util.Properties; +import java.util.UUID; + +import static com.facebook.presto.kafka.util.EmbeddedKafka.CloseableProducer; +import static com.facebook.presto.kafka.util.TestUtils.createEmptyTopicDescription; +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; +import static com.facebook.presto.testing.assertions.Assert.assertEquals; +import static com.facebook.presto.transaction.TransactionBuilder.transaction; +import static org.testng.Assert.assertTrue; + +@Test(singleThreaded = true) +public class TestMinimalFunctionality +{ + private static final Session SESSION = testSessionBuilder() + .setCatalog("kafka") + .setSchema("default") + .build(); + + private EmbeddedKafka embeddedKafka; + private String topicName; + private StandaloneQueryRunner queryRunner; + + @BeforeClass + public void startKafka() + throws Exception + { + embeddedKafka = EmbeddedKafka.createEmbeddedKafka(); + embeddedKafka.start(); + } + + @AfterClass + public void stopKafka() + throws Exception + { + embeddedKafka.close(); + } + + @BeforeMethod + public void spinUp() + throws Exception + { + this.topicName = "test_" + UUID.randomUUID().toString().replaceAll("-", "_"); + + Properties topicProperties = new Properties(); + embeddedKafka.createTopics(2, 1, topicProperties, topicName); + + this.queryRunner = new StandaloneQueryRunner(SESSION); + + TestUtils.installKafkaPlugin(embeddedKafka, queryRunner, + ImmutableMap.builder() + .put(createEmptyTopicDescription(topicName, new SchemaTableName("default", topicName))) + .build()); + } + + @AfterMethod + public void tearDown() + throws Exception + { + queryRunner.close(); + } + + private void createMessages(String topicName, int count) + { + try (CloseableProducer producer = embeddedKafka.createProducer()) { + ImmutableList.Builder> builder = ImmutableList.builder(); + for (long i = 0; i < count; i++) { + Object message = ImmutableMap.of("id", Long.toString(i), "value", UUID.randomUUID().toString()); + builder.add(new ProducerData<>(topicName, i, Collections.singletonList(message))); + } + producer.send(builder.build()); + } + } + + @Test + public void testTopicExists() + throws Exception + { + QualifiedObjectName name = new QualifiedObjectName("kafka", "default", topicName); + + transaction(queryRunner.getTransactionManager(), new AllowAllAccessControl()) + .singleStatement() + .execute(SESSION, session -> { + Optional handle = queryRunner.getServer().getMetadata().getTableHandle(session, name); + assertTrue(handle.isPresent()); + }); + } + + @Test + public void testTopicHasData() + throws Exception + { + MaterializedResult result = queryRunner.execute("SELECT count(1) from " + topicName); + + MaterializedResult expected = MaterializedResult.resultBuilder(SESSION, BigintType.BIGINT) + .row(0L) + .build(); + + assertEquals(result, expected); + + int count = 1000; + createMessages(topicName, count); + Thread.sleep(10_000); + result = queryRunner.execute("SELECT count(1) from " + topicName); + + expected = MaterializedResult.resultBuilder(SESSION, BigintType.BIGINT) + .row((long) count) + .build(); + + assertEquals(result, expected); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java new file mode 100644 index 000000000000..05f7a035869f --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java @@ -0,0 +1,71 @@ +/* + * Licensed 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 com.facebook.presto.kafka.util; + +import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.spi.type.Type; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; +import com.google.common.collect.ImmutableMap; +import io.airlift.json.JsonCodec; +import io.airlift.json.JsonCodecFactory; +import io.airlift.json.ObjectMapperProvider; + +import java.util.function.Supplier; + +import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; + +public final class CodecSupplier + implements Supplier> +{ + private final Metadata metadata; + private final JsonCodecFactory codecFactory; + private final Class clazz; + + public CodecSupplier(Class clazz, Metadata metadata) + { + this.clazz = clazz; + this.metadata = metadata; + ObjectMapperProvider objectMapperProvider = new ObjectMapperProvider(); + objectMapperProvider.setJsonDeserializers(ImmutableMap.of(Type.class, new TypeDeserializer())); + this.codecFactory = new JsonCodecFactory(objectMapperProvider); + } + + @Override + public JsonCodec get() + { + return codecFactory.jsonCodec(clazz); + } + + private class TypeDeserializer + extends FromStringDeserializer + { + private static final long serialVersionUID = 1L; + + public TypeDeserializer() + { + super(Type.class); + } + + @Override + protected Type _deserialize(String value, DeserializationContext context) + { + Type type = metadata.getType(parseTypeSignature(value)); + if (type == null) { + throw new IllegalArgumentException(String.valueOf("Unknown type " + value)); + } + return type; + } + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java new file mode 100644 index 000000000000..2ea3a57f3b48 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java @@ -0,0 +1,166 @@ +/* + * Licensed 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 com.facebook.presto.kafka.util; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import com.google.common.io.Files; +import com.google.common.io.MoreFiles; +import kafka.javaapi.producer.Producer; +import kafka.producer.ProducerConfig; +import kafka.server.KafkaConfig; +import kafka.server.KafkaServerStartable; +import scala.Option; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.facebook.presto.kafka.util.TestUtils.findUnusedPort; +import static com.facebook.presto.kafka.util.TestUtils.toProperties; +import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static java.util.Objects.requireNonNull; + +public class EmbeddedKafka + implements Closeable +{ + private final EmbeddedZookeeper zookeeper; + private final int port; + private final File kafkaDataDir; + private final KafkaServerStartable kafka; + + private final AtomicBoolean started = new AtomicBoolean(); + private final AtomicBoolean stopped = new AtomicBoolean(); + + public static EmbeddedKafka createEmbeddedKafka() + throws IOException + { + return new EmbeddedKafka(new EmbeddedZookeeper(), new Properties()); + } + + public static EmbeddedKafka createEmbeddedKafka(Properties overrideProperties) + throws IOException + { + return new EmbeddedKafka(new EmbeddedZookeeper(), overrideProperties); + } + + EmbeddedKafka(EmbeddedZookeeper zookeeper, Properties overrideProperties) + throws IOException + { + this.zookeeper = requireNonNull(zookeeper, "zookeeper is null"); + requireNonNull(overrideProperties, "overrideProperties is null"); + + this.port = findUnusedPort(); + this.kafkaDataDir = Files.createTempDir(); + + Map properties = ImmutableMap.builder() + .put("brokerid", "0") + .put("regionid", "1") + .put("hostname", "localhost") + .put("num.partitions", "2") + .put("log.flush.interval", "10000") + .put("log.default.flush.interval.ms", "1000") + .put("log.retention.hours", "1") + .put("log.segment.bytes", "10248576") + .put("log.file.size", "1048576") + .put("auto.create.topics.enable", "false") + .put("zookeeper.connection.timeout.ms", "1000000") + .put("port", Integer.toString(port)) + .put("log.dir", kafkaDataDir.getAbsolutePath()) + .put("zookeeper.connect", zookeeper.getConnectString()) + .put("zk.connect", zookeeper.getConnectString()) + .putAll(Maps.fromProperties(overrideProperties)) + .build(); + + KafkaConfig config = new KafkaConfig(toProperties(properties)); + this.kafka = new KafkaServerStartable(config, Option.empty()); + } + + public void start() + throws InterruptedException, IOException + { + if (!started.getAndSet(true)) { + zookeeper.start(); + kafka.startup(); + } + } + + @Override + public void close() + throws IOException + { + if (started.get() && !stopped.getAndSet(true)) { + kafka.shutdown(); + kafka.awaitShutdown(); + zookeeper.close(); + MoreFiles.deleteRecursively(kafkaDataDir.toPath(), ALLOW_INSECURE); + } + } + + public void createTopics(String... topics) + { + createTopics(2, 1, new Properties(), topics); + } + + public void createTopics(int partitions, int replication, Properties topicProperties, String... topics) + { + // noop + } + + public CloseableProducer createProducer() + { + Map properties = ImmutableMap.builder() + .put("broker.list", String.format("0:%s", getConnectString())) + .put("serializer.class", JsonEncoder.class.getName()) + .put("key.serializer.class", NumberEncoder.class.getName()) + .put("request.required.acks", "1") + .build(); + + ProducerConfig producerConfig = new ProducerConfig(toProperties(properties)); + return new CloseableProducer<>(producerConfig); + } + + public static class CloseableProducer + extends Producer + implements AutoCloseable + { + public CloseableProducer(ProducerConfig config) + { + super(config); + } + } + + public int getZookeeperPort() + { + return zookeeper.getPort(); + } + + public int getPort() + { + return port; + } + + public String getConnectString() + { + return "localhost:" + Integer.toString(port); + } + + public String getZookeeperConnectString() + { + return zookeeper.getConnectString(); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java new file mode 100644 index 000000000000..d312179421fd --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java @@ -0,0 +1,101 @@ +/* + * Licensed 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 com.facebook.presto.kafka.util; + +import com.google.common.io.Files; +import com.google.common.io.MoreFiles; +import org.apache.zookeeper.server.NIOServerCnxnFactory; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.apache.zookeeper.server.persistence.FileTxnSnapLog; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; + +public class EmbeddedZookeeper + implements Closeable +{ + private final int port; + private final File zkDataDir; + private final ZooKeeperServer zkServer; + private final ServerCnxnFactory cnxnFactory; + + private final AtomicBoolean started = new AtomicBoolean(); + private final AtomicBoolean stopped = new AtomicBoolean(); + + public EmbeddedZookeeper() + throws IOException + { + this(TestUtils.findUnusedPort()); + } + + public EmbeddedZookeeper(int port) + throws IOException + { + this.port = port; + zkDataDir = Files.createTempDir(); + zkServer = new ZooKeeperServer(); + + FileTxnSnapLog ftxn = new FileTxnSnapLog(zkDataDir, zkDataDir); + zkServer.setTxnLogFactory(ftxn); + zkServer.setMinSessionTimeout(60000); + zkServer.setMaxSessionTimeout(120000); + + cnxnFactory = NIOServerCnxnFactory.createFactory(new InetSocketAddress(port), 300); + } + + public void start() + throws InterruptedException, IOException + { + if (!started.getAndSet(true)) { + cnxnFactory.startup(zkServer); + } + } + + @Override + public void close() + throws IOException + { + if (started.get() && !stopped.getAndSet(true)) { + cnxnFactory.shutdown(); + try { + cnxnFactory.join(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + + if (zkServer.isRunning()) { + zkServer.shutdown(); + } + + MoreFiles.deleteRecursively(zkDataDir.toPath(), ALLOW_INSECURE); + } + } + + public String getConnectString() + { + return "127.0.0.1:" + Integer.toString(port); + } + + public int getPort() + { + return port; + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/JsonEncoder.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/JsonEncoder.java new file mode 100644 index 000000000000..65e54985c894 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/JsonEncoder.java @@ -0,0 +1,39 @@ +/* + * Licensed 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 com.facebook.presto.kafka.util; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import kafka.message.Message; +import kafka.serializer.Encoder; + +import java.io.IOException; + +public class JsonEncoder + implements Encoder +{ + private final ObjectMapper objectMapper = new ObjectMapper(); + + @Override + public Message toMessage(Object o) + { + try { + return new Message(objectMapper.writeValueAsBytes(o)); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java new file mode 100644 index 000000000000..f72f650be757 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java @@ -0,0 +1,158 @@ +/* + * Licensed 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 com.facebook.presto.kafka.util; + +import com.facebook.presto.Session; +import com.facebook.presto.client.Column; +import com.facebook.presto.client.QueryData; +import com.facebook.presto.client.QueryStatusInfo; +import com.facebook.presto.server.testing.TestingPrestoServer; +import com.facebook.presto.spi.type.TimeZoneKey; +import com.facebook.presto.spi.type.Type; +import com.facebook.presto.spi.type.Varchars; +import com.facebook.presto.tests.AbstractTestingPrestoClient; +import com.facebook.presto.tests.ResultsSession; +import com.google.common.collect.ImmutableMap; +import kafka.javaapi.producer.Producer; +import kafka.javaapi.producer.ProducerData; +import org.joda.time.format.DateTimeFormatter; +import org.joda.time.format.ISODateTimeFormat; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; +import static com.facebook.presto.spi.type.DateTimeEncoding.unpackMillisUtc; +import static com.facebook.presto.spi.type.DateType.DATE; +import static com.facebook.presto.spi.type.DoubleType.DOUBLE; +import static com.facebook.presto.spi.type.IntegerType.INTEGER; +import static com.facebook.presto.spi.type.TimeType.TIME; +import static com.facebook.presto.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE; +import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP; +import static com.facebook.presto.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; +import static com.facebook.presto.util.DateTimeUtils.parseTime; +import static com.facebook.presto.util.DateTimeUtils.parseTimestampWithTimeZone; +import static com.facebook.presto.util.DateTimeUtils.parseTimestampWithoutTimeZone; +import static com.google.common.base.Preconditions.checkState; +import static java.util.Objects.requireNonNull; + +public class KafkaLoader + extends AbstractTestingPrestoClient +{ + private static final DateTimeFormatter ISO8601_FORMATTER = ISODateTimeFormat.dateTime(); + + private final String topicName; + private final Producer producer; + private final AtomicLong count = new AtomicLong(); + + public KafkaLoader(Producer producer, + String topicName, + TestingPrestoServer prestoServer, + Session defaultSession) + { + super(prestoServer, defaultSession); + + this.topicName = topicName; + this.producer = producer; + } + + @Override + public ResultsSession getResultSession(Session session) + { + requireNonNull(session, "session is null"); + return new KafkaLoadingSession(session); + } + + private class KafkaLoadingSession + implements ResultsSession + { + private final AtomicReference> types = new AtomicReference<>(); + + private final TimeZoneKey timeZoneKey; + + private KafkaLoadingSession(Session session) + { + this.timeZoneKey = session.getTimeZoneKey(); + } + + @Override + public void addResults(QueryStatusInfo statusInfo, QueryData data) + { + if (types.get() == null && statusInfo.getColumns() != null) { + types.set(getTypes(statusInfo.getColumns())); + } + + if (data.getData() != null) { + checkState(types.get() != null, "Data without types received!"); + List columns = statusInfo.getColumns(); + for (List fields : data.getData()) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (int i = 0; i < fields.size(); i++) { + Type type = types.get().get(i); + Object value = convertValue(fields.get(i), type); + if (value != null) { + builder.put(columns.get(i).getName(), value); + } + } + + producer.send(new ProducerData<>(topicName, count.getAndIncrement(), Collections.singletonList(builder.build()))); + } + } + } + + @Override + public Void build(Map setSessionProperties, Set resetSessionProperties) + { + return null; + } + + private Object convertValue(Object value, Type type) + { + if (value == null) { + return null; + } + + if (BOOLEAN.equals(type) || Varchars.isVarcharType(type)) { + return value; + } + if (BIGINT.equals(type)) { + return ((Number) value).longValue(); + } + if (INTEGER.equals(type)) { + return ((Number) value).intValue(); + } + if (DOUBLE.equals(type)) { + return ((Number) value).doubleValue(); + } + if (DATE.equals(type)) { + return value; + } + if (TIME.equals(type)) { + return ISO8601_FORMATTER.print(parseTime(timeZoneKey, (String) value)); + } + if (TIMESTAMP.equals(type)) { + return ISO8601_FORMATTER.print(parseTimestampWithoutTimeZone(timeZoneKey, (String) value)); + } + if (TIME_WITH_TIME_ZONE.equals(type) || TIMESTAMP_WITH_TIME_ZONE.equals(type)) { + return ISO8601_FORMATTER.print(unpackMillisUtc(parseTimestampWithTimeZone(timeZoneKey, (String) value))); + } + throw new AssertionError("unhandled type: " + type); + } + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberEncoder.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberEncoder.java new file mode 100644 index 000000000000..07b627b4cff9 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberEncoder.java @@ -0,0 +1,31 @@ +/* + * Licensed 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 com.facebook.presto.kafka.util; + +import kafka.message.Message; +import kafka.serializer.Encoder; + +import java.nio.ByteBuffer; + +public class NumberEncoder + implements Encoder +{ + @Override + public Message toMessage(Number value) + { + ByteBuffer buf = ByteBuffer.allocate(8); + buf.putLong(value == null ? 0L : value.longValue()); + return new Message(buf.array()); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java new file mode 100644 index 000000000000..a1225b358813 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java @@ -0,0 +1,33 @@ +/* + * Licensed 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 com.facebook.presto.kafka.util; + +import kafka.cluster.Partition; +import kafka.producer.Partitioner; +import scala.collection.Seq; + +import static java.lang.Math.toIntExact; + +public class NumberPartitioner + implements Partitioner +{ + @Override + public int partition(Object key, Seq partitions) + { + if (key instanceof Number) { + return toIntExact(((Number) key).longValue() % partitions.size()); + } + return 0; + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java new file mode 100644 index 000000000000..5e194df60de6 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java @@ -0,0 +1,96 @@ +/* + * Licensed 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 com.facebook.presto.kafka.util; + +import com.facebook.presto.kafka.KafkaPlugin; +import com.facebook.presto.kafka.KafkaTopicDescription; +import com.facebook.presto.metadata.QualifiedObjectName; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.testing.QueryRunner; +import com.facebook.presto.tests.TestingPrestoClient; +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.ByteStreams; +import io.airlift.json.JsonCodec; + +import java.io.IOException; +import java.net.ServerSocket; +import java.util.AbstractMap; +import java.util.Map; +import java.util.Properties; + +import static com.facebook.presto.kafka.util.EmbeddedKafka.CloseableProducer; +import static java.lang.String.format; + +public final class TestUtils +{ + private TestUtils() {} + + public static int findUnusedPort() + throws IOException + { + try (ServerSocket socket = new ServerSocket(0)) { + return socket.getLocalPort(); + } + } + + public static Properties toProperties(Map map) + { + Properties properties = new Properties(); + for (Map.Entry entry : map.entrySet()) { + properties.setProperty(entry.getKey(), entry.getValue()); + } + return properties; + } + + public static void installKafkaPlugin(EmbeddedKafka embeddedKafka, QueryRunner queryRunner, Map topicDescriptions) + { + KafkaPlugin kafkaPlugin = new KafkaPlugin(); + kafkaPlugin.setTableDescriptionSupplier(() -> topicDescriptions); + queryRunner.installPlugin(kafkaPlugin); + + Map kafkaConfig = ImmutableMap.of( + "kafka.nodes", embeddedKafka.getConnectString(), + "kafka.table-names", Joiner.on(",").join(topicDescriptions.keySet()), + "kafka.connect-timeout", "120s", + "kafka.default-schema", "default", + "kafka.zk-endpoint", embeddedKafka.getZookeeperConnectString()); + queryRunner.createCatalog("kafka", "kafka07", kafkaConfig); + } + + public static void loadTpchTopic(EmbeddedKafka embeddedKafka, TestingPrestoClient prestoClient, String topicName, QualifiedObjectName tpchTableName) + { + try (CloseableProducer producer = embeddedKafka.createProducer(); + KafkaLoader tpchLoader = new KafkaLoader(producer, topicName, prestoClient.getServer(), prestoClient.getDefaultSession())) { + tpchLoader.execute(format("SELECT * from %s", tpchTableName)); + } + } + + public static Map.Entry loadTpchTopicDescription(JsonCodec topicDescriptionJsonCodec, String topicName, SchemaTableName schemaTableName) + throws IOException + { + KafkaTopicDescription tpchTemplate = topicDescriptionJsonCodec.fromJson(ByteStreams.toByteArray(TestUtils.class.getResourceAsStream(format("/tpch/%s.json", schemaTableName.getTableName())))); + + return new AbstractMap.SimpleImmutableEntry<>( + schemaTableName, + new KafkaTopicDescription(schemaTableName.getTableName(), schemaTableName.getSchemaName(), topicName, tpchTemplate.getKey(), tpchTemplate.getMessage())); + } + + public static Map.Entry createEmptyTopicDescription(String topicName, SchemaTableName schemaTableName) + { + return new AbstractMap.SimpleImmutableEntry<>( + schemaTableName, + new KafkaTopicDescription(schemaTableName.getTableName(), schemaTableName.getSchemaName(), topicName, null, null)); + } +} diff --git a/presto-kafka07/src/test/resources/tpch/customer.json b/presto-kafka07/src/test/resources/tpch/customer.json new file mode 100644 index 000000000000..cf800fc0a9bb --- /dev/null +++ b/presto-kafka07/src/test/resources/tpch/customer.json @@ -0,0 +1,61 @@ +{ + "tableName": "customer", + "schemaName": "tpch", + "topicName": "tpch.customer", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "true" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "custkey", + "mapping": "custkey", + "type": "BIGINT" + }, + { + "name": "name", + "mapping": "name", + "type": "VARCHAR(25)" + }, + { + "name": "address", + "mapping": "address", + "type": "VARCHAR(40)" + }, + { + "name": "nationkey", + "mapping": "nationkey", + "type": "BIGINT" + }, + { + "name": "phone", + "mapping": "phone", + "type": "VARCHAR(15)" + }, + { + "name": "acctbal", + "mapping": "acctbal", + "type": "DOUBLE" + }, + { + "name": "mktsegment", + "mapping": "mktsegment", + "type": "VARCHAR(10)" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(117)" + } + ] + } +} diff --git a/presto-kafka07/src/test/resources/tpch/lineitem.json b/presto-kafka07/src/test/resources/tpch/lineitem.json new file mode 100644 index 000000000000..2250fddf7771 --- /dev/null +++ b/presto-kafka07/src/test/resources/tpch/lineitem.json @@ -0,0 +1,104 @@ +{ + "tableName": "lineitem", + "schemaName": "tpch", + "topicName": "tpch.lineitem", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "true" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "orderkey", + "mapping": "orderkey", + "type": "BIGINT" + }, + { + "name": "partkey", + "mapping": "partkey", + "type": "BIGINT" + }, + { + "name": "suppkey", + "mapping": "suppkey", + "type": "BIGINT" + }, + { + "name": "linenumber", + "mapping": "linenumber", + "type": "INTEGER" + }, + { + "name": "quantity", + "mapping": "quantity", + "type": "DOUBLE" + }, + { + "name": "extendedprice", + "mapping": "extendedprice", + "type": "DOUBLE" + }, + { + "name": "discount", + "mapping": "discount", + "type": "DOUBLE" + }, + { + "name": "tax", + "mapping": "tax", + "type": "DOUBLE" + }, + { + "name": "returnflag", + "mapping": "returnflag", + "type": "VARCHAR(1)" + }, + { + "name": "linestatus", + "mapping": "linestatus", + "type": "VARCHAR(1)" + }, + { + "name": "shipdate", + "mapping": "shipdate", + "type": "DATE", + "dataFormat": "iso8601" + }, + { + "name": "commitdate", + "mapping": "commitdate", + "type": "DATE", + "dataFormat": "iso8601" + }, + { + "name": "receiptdate", + "mapping": "receiptdate", + "type": "DATE", + "dataFormat": "iso8601" + }, + { + "name": "shipinstruct", + "mapping": "shipinstruct", + "type": "VARCHAR(25)" + }, + { + "name": "shipmode", + "mapping": "shipmode", + "type": "VARCHAR(7)" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(44)" + } + ] + } +} diff --git a/presto-kafka07/src/test/resources/tpch/nation.json b/presto-kafka07/src/test/resources/tpch/nation.json new file mode 100644 index 000000000000..0749a1249d76 --- /dev/null +++ b/presto-kafka07/src/test/resources/tpch/nation.json @@ -0,0 +1,41 @@ +{ + "tableName": "nation", + "schemaName": "tpch", + "topicName": "tpch.nation", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "true" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "nationkey", + "mapping": "nationkey", + "type": "BIGINT" + }, + { + "name": "name", + "mapping": "name", + "type": "VARCHAR(25)" + }, + { + "name": "regionkey", + "mapping": "regionkey", + "type": "BIGINT" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(152)" + } + ] + } +} diff --git a/presto-kafka07/src/test/resources/tpch/orders.json b/presto-kafka07/src/test/resources/tpch/orders.json new file mode 100644 index 000000000000..f5eb594083a4 --- /dev/null +++ b/presto-kafka07/src/test/resources/tpch/orders.json @@ -0,0 +1,67 @@ +{ + "tableName": "orders", + "schemaName": "tpch", + "topicName": "tpch.orders", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "true" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "orderkey", + "mapping": "orderkey", + "type": "BIGINT" + }, + { + "name": "custkey", + "mapping": "custkey", + "type": "BIGINT" + }, + { + "name": "orderstatus", + "mapping": "orderstatus", + "type": "VARCHAR(1)" + }, + { + "name": "totalprice", + "mapping": "totalprice", + "type": "DOUBLE" + }, + { + "name": "orderdate", + "mapping": "orderdate", + "type": "DATE", + "dataFormat": "iso8601" + }, + { + "name": "orderpriority", + "mapping": "orderpriority", + "type": "VARCHAR(15)" + }, + { + "name": "clerk", + "mapping": "clerk", + "type": "VARCHAR(15)" + }, + { + "name": "shippriority", + "mapping": "shippriority", + "type": "INTEGER" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(79)" + } + ] + } +} diff --git a/presto-kafka07/src/test/resources/tpch/part.json b/presto-kafka07/src/test/resources/tpch/part.json new file mode 100644 index 000000000000..3c56b27955f1 --- /dev/null +++ b/presto-kafka07/src/test/resources/tpch/part.json @@ -0,0 +1,66 @@ +{ + "tableName": "part", + "schemaName": "tpch", + "topicName": "tpch.part", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "true" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "partkey", + "mapping": "partkey", + "type": "BIGINT" + }, + { + "name": "name", + "mapping": "name", + "type": "VARCHAR(55)" + }, + { + "name": "mfgr", + "mapping": "mfgr", + "type": "VARCHAR(25)" + }, + { + "name": "brand", + "mapping": "brand", + "type": "VARCHAR(10)" + }, + { + "name": "type", + "mapping": "type", + "type": "VARCHAR(25)" + }, + { + "name": "size", + "mapping": "size", + "type": "INTEGER" + }, + { + "name": "container", + "mapping": "container", + "type": "VARCHAR(10)" + }, + { + "name": "retailprice", + "mapping": "retailprice", + "type": "DOUBLE" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(23)" + } + ] + } +} diff --git a/presto-kafka07/src/test/resources/tpch/partsupp.json b/presto-kafka07/src/test/resources/tpch/partsupp.json new file mode 100644 index 000000000000..cd6d6364f87f --- /dev/null +++ b/presto-kafka07/src/test/resources/tpch/partsupp.json @@ -0,0 +1,46 @@ +{ + "tableName": "partsupp", + "schemaName": "tpch", + "topicName": "tpch.partsupp", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "true" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "partkey", + "mapping": "partkey", + "type": "BIGINT" + }, + { + "name": "suppkey", + "mapping": "suppkey", + "type": "BIGINT" + }, + { + "name": "availqty", + "mapping": "availqty", + "type": "INTEGER" + }, + { + "name": "supplycost", + "mapping": "supplycost", + "type": "DOUBLE" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(199)" + } + ] + } +} diff --git a/presto-kafka07/src/test/resources/tpch/region.json b/presto-kafka07/src/test/resources/tpch/region.json new file mode 100644 index 000000000000..4aca016e22cc --- /dev/null +++ b/presto-kafka07/src/test/resources/tpch/region.json @@ -0,0 +1,36 @@ +{ + "tableName": "region", + "schemaName": "tpch", + "topicName": "tpch.region", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "true" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "regionkey", + "mapping": "regionkey", + "type": "BIGINT" + }, + { + "name": "name", + "mapping": "name", + "type": "VARCHAR(25)" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(152)" + } + ] + } +} diff --git a/presto-kafka07/src/test/resources/tpch/supplier.json b/presto-kafka07/src/test/resources/tpch/supplier.json new file mode 100644 index 000000000000..af9d24c86a83 --- /dev/null +++ b/presto-kafka07/src/test/resources/tpch/supplier.json @@ -0,0 +1,56 @@ +{ + "tableName": "supplier", + "schemaName": "tpch", + "topicName": "tpch.supplier", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "true" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "suppkey", + "mapping": "suppkey", + "type": "BIGINT" + }, + { + "name": "name", + "mapping": "name", + "type": "VARCHAR(25)" + }, + { + "name": "address", + "mapping": "address", + "type": "VARCHAR(40)" + }, + { + "name": "nationkey", + "mapping": "nationkey", + "type": "BIGINT" + }, + { + "name": "phone", + "mapping": "phone", + "type": "VARCHAR(15)" + }, + { + "name": "acctbal", + "mapping": "acctbal", + "type": "DOUBLE" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(101)" + } + ] + } +} diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index ecaf297fd20f..9d1d78f4c539 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-local-file diff --git a/presto-main/etc/config.properties b/presto-main/etc/config.properties index da917b58e4d5..105c4ce6fe2d 100644 --- a/presto-main/etc/config.properties +++ b/presto-main/etc/config.properties @@ -34,6 +34,7 @@ plugin.bundles=\ ../presto-hive-hadoop2/pom.xml,\ ../presto-example-http/pom.xml,\ ../presto-kafka/pom.xml, \ + ../presto-kafka07/pom.xml, \ ../presto-tpch/pom.xml, \ ../presto-local-file/pom.xml, \ ../presto-mysql/pom.xml,\ diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 515c06c772a8..6b4a5540b9f2 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-main diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/DiscoveryNodeManager.java b/presto-main/src/main/java/com/facebook/presto/metadata/DiscoveryNodeManager.java index ca55deb9f1d6..42ac7b711d61 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/DiscoveryNodeManager.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/DiscoveryNodeManager.java @@ -133,12 +133,18 @@ public void startPollingNodeStates() Set deadNodes = difference(nodeStates.keySet(), aliveNodeIds).immutableCopy(); nodeStates.keySet().removeAll(deadNodes); + if (deadNodes.size() > 0) { + log.warn("Dead nodes: %s", deadNodes); + } + // Add new nodes for (Node node : aliveNodes) { nodeStates.putIfAbsent(node.getNodeIdentifier(), new RemoteNodeState(httpClient, uriBuilderFrom(node.getHttpUri()).appendPath("/v1/info/state").build())); } + log.debug("Number of alive nodes: %d", nodeStates.size()); + // Schedule refresh nodeStates.values().forEach(RemoteNodeState::asyncRefresh); }, 1, 5, TimeUnit.SECONDS); diff --git a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java index bd5916c4b323..013e6750c3de 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java @@ -107,8 +107,6 @@ public void run() new ServerMainModule(sqlParserOptions), new GracefulShutdownModule()); - modules.addAll(getAdditionalModules()); - Bootstrap app = new Bootstrap(modules.build()); try { @@ -141,11 +139,6 @@ public void run() } } - protected Iterable getAdditionalModules() - { - return ImmutableList.of(); - } - private static void updateConnectorIds(Announcer announcer, CatalogManager metadata, ServerConfig serverConfig, NodeSchedulerConfig schedulerConfig) { // get existing announcement diff --git a/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java b/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java index 8ab70cb1dbef..241f9c4643bb 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java @@ -42,10 +42,10 @@ public static void verifyJvmRequirements() { String vendor = StandardSystemProperty.JAVA_VENDOR.value(); if (!"Oracle Corporation".equals(vendor)) { - failRequirement("Presto requires an Oracle or OpenJDK JVM (found %s)", vendor); + warnRequirement("Presto requires an Oracle or OpenJDK JVM (found %s)", vendor); } - verifyJavaVersion(); + //verifyJavaVersion(); String dataModel = System.getProperty("sun.arch.data.model"); if (!"64".equals(dataModel)) { diff --git a/presto-main/src/main/java/com/facebook/presto/server/protocol/StatementResource.java b/presto-main/src/main/java/com/facebook/presto/server/protocol/StatementResource.java index 4381e8ff4eb9..378ea4413de8 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/protocol/StatementResource.java +++ b/presto-main/src/main/java/com/facebook/presto/server/protocol/StatementResource.java @@ -68,6 +68,7 @@ import static com.facebook.presto.client.PrestoHeaders.PRESTO_STARTED_TRANSACTION_ID; import static com.facebook.presto.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; import static com.google.common.base.Strings.isNullOrEmpty; +import static com.google.common.base.Strings.nullToEmpty; import static io.airlift.concurrent.Threads.threadsNamed; import static io.airlift.http.server.AsyncResponseHandler.bindAsyncResponse; import static java.util.Objects.requireNonNull; @@ -132,6 +133,23 @@ public void createQuery( .build()); } + // The Teradata Presto ODBC Driver checks node version to decide the Presto's statement + // protocol and sends test queries about PREPARE statement. + // Rewrite the statement so that Presto always returns version for the compatible protocol. + // Ban the statement which will never be consumed by the driver. + if (nullToEmpty(servletRequest.getHeader("User-Agent")).equals("Teradata Presto ODBC Driver")) { + if (statement.equals("select node_version from system.runtime.nodes where coordinator=true")) { + statement = "select '0.148' as node_version"; + } + else if (statement.equals("DESCRIBE OUTPUT prepare_test_stmt")) { + throw new WebApplicationException(Response + .status(Status.BAD_REQUEST) + .type(MediaType.TEXT_PLAIN) + .entity("SQL statement is known, and wouldn't be consumed by this driver") + .build()); + } + } + SessionContext sessionContext = new HttpRequestSessionContext(servletRequest); ExchangeClient exchangeClient = exchangeClientSupplier.get(new SimpleLocalMemoryContext(newSimpleAggregatedMemoryContext())); diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java b/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java index ec62c724dea8..d8ff99dfd4ca 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java @@ -44,11 +44,13 @@ public class AuthenticationFilter implements Filter { private final List authenticators; + private final String httpAuthenticationPathRegex; @Inject - public AuthenticationFilter(List authenticators) + public AuthenticationFilter(Set authenticators, SecurityConfig securityConfig) { this.authenticators = ImmutableList.copyOf(authenticators); + this.httpAuthenticationPathRegex = requireNonNull(securityConfig.getHttpAuthenticationPathRegex(), "httpAuthenticationPathRegex is null"); } @Override @@ -64,8 +66,8 @@ public void doFilter(ServletRequest servletRequest, ServletResponse servletRespo HttpServletRequest request = (HttpServletRequest) servletRequest; HttpServletResponse response = (HttpServletResponse) servletResponse; - // skip authentication if non-secure or not configured - if (!request.isSecure() || authenticators.isEmpty()) { + // skip authentication if (not configured) or (non-secure and not match httpAuthenticationPathRegex) + if (authenticators.isEmpty() || (!request.isSecure() && !request.getPathInfo().matches(httpAuthenticationPathRegex))) { nextFilter.doFilter(request, response); return; } diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/KerberosAuthenticator.java b/presto-main/src/main/java/com/facebook/presto/server/security/KerberosAuthenticator.java index 6ecd91a6f1dd..581b86ff6ac2 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/KerberosAuthenticator.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/KerberosAuthenticator.java @@ -64,8 +64,9 @@ public KerberosAuthenticator(KerberosConfig config) System.setProperty("java.security.krb5.conf", config.getKerberosConfig().getAbsolutePath()); try { + boolean isCompleteServicePrinciple = config.getServiceName().contains("@"); String hostname = InetAddress.getLocalHost().getCanonicalHostName().toLowerCase(Locale.US); - String servicePrincipal = config.getServiceName() + "/" + hostname; + String servicePrincipal = isCompleteServicePrinciple ? config.getServiceName() : config.getServiceName() + "/" + hostname; loginContext = new LoginContext("", null, null, new Configuration() { @Override @@ -91,7 +92,7 @@ public AppConfigurationEntry[] getAppConfigurationEntry(String name) loginContext.login(); serverCredential = doAs(loginContext.getSubject(), () -> gssManager.createCredential( - gssManager.createName(config.getServiceName() + "@" + hostname, GSSName.NT_HOSTBASED_SERVICE), + isCompleteServicePrinciple ? gssManager.createName(config.getServiceName(), GSSName.NT_USER_NAME) : gssManager.createName(config.getServiceName() + "@" + hostname, GSSName.NT_HOSTBASED_SERVICE), INDEFINITE_LIFETIME, new Oid[] { new Oid("1.2.840.113554.1.2.2"), // kerberos 5 diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java b/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java index 9af432966374..ce869cc2e116 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java @@ -33,6 +33,8 @@ public class SecurityConfig private List authenticationTypes = ImmutableList.of(); + private String httpAuthenticationPathRegex = "^\b$"; + public enum AuthenticationType { CERTIFICATE, @@ -66,4 +68,18 @@ public SecurityConfig setAuthenticationTypes(String types) .collect(toImmutableList()); return this; } + + @NotNull + public String getHttpAuthenticationPathRegex() + { + return httpAuthenticationPathRegex; + } + + @Config("http-server.http.authentication.path.regex") + @ConfigDescription("Regex of path that needs to be authenticated for non-secured http request") + public SecurityConfig setHttpAuthenticationPathRegex(String regex) + { + httpAuthenticationPathRegex = regex; + return this; + } } diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/ServerSecurityModule.java b/presto-main/src/main/java/com/facebook/presto/server/security/ServerSecurityModule.java index 8350a42ed9fa..28949cd1482a 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/ServerSecurityModule.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/ServerSecurityModule.java @@ -41,6 +41,7 @@ protected void setup(Binder binder) { newSetBinder(binder, Filter.class, TheServlet.class).addBinding() .to(AuthenticationFilter.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(SecurityConfig.class); binder.bind(PasswordAuthenticatorManager.class).in(Scopes.SINGLETON); diff --git a/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java b/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java index 173983ee6fde..46e5c02ff8dc 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java +++ b/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java @@ -46,7 +46,9 @@ public String getName() @Override public Type createType(TypeManager typeManager, List parameters) { - checkArgument(!parameters.isEmpty(), "Row type must have at least one parameter"); + if (parameters.isEmpty()) { + parameters.add(TypeParameter.of(new NamedType(UnknownType.NAME, UnknownType.UNKNOWN))); + } checkArgument( parameters.stream().allMatch(parameter -> parameter.getKind() == ParameterKind.NAMED_TYPE), "Expected only named types as a parameters, got %s", diff --git a/presto-main/src/main/resources/webapp/tableau/presto-client.js b/presto-main/src/main/resources/webapp/tableau/presto-client.js index 3bc357dd1b65..14de90193895 100644 --- a/presto-main/src/main/resources/webapp/tableau/presto-client.js +++ b/presto-main/src/main/resources/webapp/tableau/presto-client.js @@ -17,6 +17,8 @@ function StatementClient(connectionData, headerCallback, dataCallback, errorCall this.currentResults = null; this.valid = true; + this.isHttps = window.location.protocol === "https:" + if (!(connectionData.sessionParameters === undefined)) { var parameterMap = JSON.parse(connectionData.sessionParameters); for (var name in parameterMap) { @@ -72,7 +74,7 @@ StatementClient.prototype.advance = function(lastRecordNumber) { var statementClient = this; $.ajax({ type: "GET", - url: this.currentResults.nextUri, + url: this.isHttps ? this.currentResults.nextUri.replace(/^http:/, 'https:') : this.currentResults.nextUri, headers: this.headers, dataType: 'json', // FIXME having problems when async: true diff --git a/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java b/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java index 96beefb828f7..52064739490e 100644 --- a/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java +++ b/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java @@ -29,7 +29,8 @@ public class TestSecurityConfig public void testDefaults() { ConfigAssertions.assertRecordedDefaults(ConfigAssertions.recordDefaults(SecurityConfig.class) - .setAuthenticationTypes("")); + .setAuthenticationTypes("") + .setHttpAuthenticationPathRegex("^\b$")); } @Test @@ -37,10 +38,12 @@ public void testExplicitPropertyMappings() { Map properties = new ImmutableMap.Builder() .put("http-server.authentication.type", "KERBEROS,PASSWORD") + .put("http-server.http.authentication.path.regex", "^/v1/statement") .build(); SecurityConfig expected = new SecurityConfig() - .setAuthenticationTypes(ImmutableList.of(KERBEROS, PASSWORD)); + .setAuthenticationTypes(ImmutableList.of(KERBEROS, PASSWORD)) + .setHttpAuthenticationPathRegex("^/v1/statement"); ConfigAssertions.assertFullMapping(properties, expected); } diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 068761506a79..fadf9f267538 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.196 + 0.196-tw-0.49 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index 53fcee84cc2d..c09ad7f9f66b 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 7b9ce0472e59..bfc0f6bdded6 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 1aed1cfd810d..d9aef95f7900 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index f4f610700932..7f54cf4c1657 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 197d01fac3f3..c8705a60d091 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 194d92bde3df..71cbd040baab 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index c2c8cc6b4dca..b25eae864507 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index ebf840a3c111..c5a8cb7b1a47 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 39826b6f3a8a..6fcb128ae8d5 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 56bac316f97e..f124f6f6347d 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-postgresql diff --git a/presto-product-tests/conf/presto/etc/singlenode-kerberized.properties b/presto-product-tests/conf/presto/etc/singlenode-kerberized.properties index b012c831a5a7..43ea74e648db 100644 --- a/presto-product-tests/conf/presto/etc/singlenode-kerberized.properties +++ b/presto-product-tests/conf/presto/etc/singlenode-kerberized.properties @@ -18,7 +18,7 @@ discovery.uri=https://presto-master.docker.cluster:7778 http.authentication.krb5.config=/etc/krb5.conf http-server.authentication.type=KERBEROS,CERTIFICATE -http.server.authentication.krb5.service-name=presto-server +http.server.authentication.krb5.service-name=presto-server/presto-master.docker.cluster@LABS.TERADATA.COM http-server.http.enabled=false http-server.https.enabled=true http-server.https.port=7778 diff --git a/presto-product-tests/conf/tempto/tempto-configuration-for-docker-kerberos.yaml b/presto-product-tests/conf/tempto/tempto-configuration-for-docker-kerberos.yaml index 54e2d063630d..16ef4773805b 100644 --- a/presto-product-tests/conf/tempto/tempto-configuration-for-docker-kerberos.yaml +++ b/presto-product-tests/conf/tempto/tempto-configuration-for-docker-kerberos.yaml @@ -37,7 +37,7 @@ databases: cli_kerberos_principal: presto-client/presto-master.docker.cluster@LABS.TERADATA.COM cli_kerberos_keytab: /etc/presto/conf/presto-client.keytab cli_kerberos_config_path: /etc/krb5.conf - cli_kerberos_service_name: presto-server + cli_kerberos_service_name: presto-server/presto-master.docker.cluster@LABS.TERADATA.COM cli_kerberos_use_canonical_hostname: false configured_hdfs_user: hdfs diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 7ab38d399136..c82ab7963b4a 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.196 + 0.196-tw-0.49 presto-product-tests diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/ImpersonationTests.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/ImpersonationTests.java index 84bfa0a41a33..eb55984b9890 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/ImpersonationTests.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/ImpersonationTests.java @@ -15,16 +15,18 @@ import com.google.inject.Inject; import com.google.inject.name.Named; +import io.prestodb.tempto.BeforeTestWithContext; import io.prestodb.tempto.ProductTest; import io.prestodb.tempto.Requires; import io.prestodb.tempto.fulfillment.table.hive.tpch.ImmutableTpchTablesRequirements.ImmutableNationTable; import io.prestodb.tempto.hadoop.hdfs.HdfsClient; +import io.prestodb.tempto.query.QueryExecutor; import org.testng.annotations.Test; import static com.facebook.presto.tests.TestGroups.HDFS_IMPERSONATION; import static com.facebook.presto.tests.TestGroups.HDFS_NO_IMPERSONATION; import static com.facebook.presto.tests.TestGroups.PROFILE_SPECIFIC_TESTS; -import static io.prestodb.tempto.query.QueryExecutor.query; +import static com.facebook.presto.tests.utils.QueryExecutors.connectToPresto; import static java.lang.String.format; import static org.testng.Assert.assertEquals; @@ -32,12 +34,14 @@ public class ImpersonationTests extends ProductTest { + private QueryExecutor aliceExecutor; + @Inject private HdfsClient hdfsClient; @Inject - @Named("databases.presto.jdbc_user") - private String prestoJdbcUser; + @Named("databases.alice@presto.jdbc_user") + private String aliceJdbcUser; // The value for configuredHdfsUser is profile dependent // For non-Kerberos environments this variable will be equal to -DHADOOP_USER_NAME as set in jvm.config @@ -50,18 +54,25 @@ public class ImpersonationTests @Named("databases.hive.warehouse_directory_path") private String warehouseDirectoryPath; + @BeforeTestWithContext + public void setup() + { + aliceExecutor = connectToPresto("alice@presto"); + QueryExecutor.query(format("GRANT SELECT ON NATION TO %s", aliceJdbcUser)); + } + @Test(groups = {HDFS_NO_IMPERSONATION, PROFILE_SPECIFIC_TESTS}) public void testHdfsImpersonationDisabled() { String tableName = "check_hdfs_impersonation_disabled"; - checkTableOwner(tableName, configuredHdfsUser); + checkTableOwner(tableName, configuredHdfsUser, aliceExecutor); } @Test(groups = {HDFS_IMPERSONATION, PROFILE_SPECIFIC_TESTS}) public void testHdfsImpersonationEnabled() { String tableName = "check_hdfs_impersonation_enabled"; - checkTableOwner(tableName, prestoJdbcUser); + checkTableOwner(tableName, aliceJdbcUser, aliceExecutor); } private String getTableLocation(String tableName) @@ -69,13 +80,13 @@ private String getTableLocation(String tableName) return warehouseDirectoryPath + '/' + tableName; } - private void checkTableOwner(String tableName, String expectedOwner) + private void checkTableOwner(String tableName, String expectedOwner, QueryExecutor executor) { - query(format("DROP TABLE IF EXISTS %s", tableName)); - query(format("CREATE TABLE %s AS SELECT * FROM NATION", tableName)); + executor.executeQuery(format("DROP TABLE IF EXISTS %s", tableName)); + executor.executeQuery(format("CREATE TABLE %s AS SELECT * FROM NATION", tableName)); String tableLocation = getTableLocation(tableName); String owner = hdfsClient.getOwner(tableLocation); assertEquals(owner, expectedOwner); - query(format("DROP TABLE IF EXISTS %s", tableName)); + executor.executeQuery(format("DROP TABLE IF EXISTS %s", tableName)); } } diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveCoercion.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveCoercion.java index 94c49329b053..fabccd65ab48 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveCoercion.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveCoercion.java @@ -70,6 +70,7 @@ public class TestHiveCoercion extends ProductTest { private static String tableNameFormat = "%s_hive_coercion"; + private static String dummyTableNameFormat = "%s_dummy"; public static final HiveTableDefinition HIVE_COERCION_TEXTFILE = tableDefinitionBuilder("TEXTFILE", Optional.empty(), Optional.of("DELIMITED FIELDS TERMINATED BY '|'")) .setNoData() @@ -260,7 +261,6 @@ public void testHiveCoercionAvro() private void doTestHiveCoercion(HiveTableDefinition tableDefinition) { String tableName = mutableTableInstanceOf(tableDefinition).getNameInDatabase(); - String floatToDoubleType = tableName.toLowerCase(Locale.ENGLISH).contains("parquet") ? "DOUBLE" : "REAL"; query(format( diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java index 40346aebcc36..8a052c3728ee 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java @@ -46,11 +46,14 @@ public static Object[][] storageFormats() { return new StorageFormat[][] { {storageFormat("ORC")}, + {storageFormat("ORC", ImmutableMap.of("hive.orc_optimized_writer_enabled", "true", "hive.orc_optimized_writer_validate", "true"))}, {storageFormat("DWRF")}, {storageFormat("PARQUET")}, {storageFormat("PARQUET", ImmutableMap.of("hive.parquet_optimized_reader_enabled", "true"))}, - {storageFormat("RCBINARY")}, - {storageFormat("RCTEXT")}, + {storageFormat("RCBINARY", ImmutableMap.of("hive.rcfile_optimized_writer_enabled", "false", "hive.rcfile_optimized_writer_validate", "false"))}, + {storageFormat("RCBINARY", ImmutableMap.of("hive.rcfile_optimized_writer_enabled", "true", "hive.rcfile_optimized_writer_validate", "true"))}, + {storageFormat("RCTEXT", ImmutableMap.of("hive.rcfile_optimized_writer_enabled", "false", "hive.rcfile_optimized_writer_validate", "false"))}, + {storageFormat("RCTEXT", ImmutableMap.of("hive.rcfile_optimized_writer_enabled", "true", "hive.rcfile_optimized_writer_validate", "true"))}, {storageFormat("SEQUENCEFILE")}, {storageFormat("TEXTFILE")}, {storageFormat("AVRO")} diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index a2fba282e2d0..50c763de1b8d 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 13b32deb7141..2efc684adb0f 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 45b6514625c9..bf7812b94219 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-record-decoder @@ -83,6 +83,18 @@ jackson-annotations + + + org.apache.thrift + libthrift + + + + commons-lang + commons-lang + 2.5 + + org.testng @@ -95,5 +107,53 @@ testing test + + com.facebook.presto + presto-main + test + + + javax.servlet-api + javax.servlet + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.8 + 1.8 + + + + org.apache.thrift.tools + maven-thrift-plugin + 0.1.11 + + /usr/local/bin/thrift + + + + thrift-sources + generate-sources + + compile + + + + thrift-test-sources + generate-test-sources + + testCompile + + + + + + diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/DecoderModule.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/DecoderModule.java index 2947f0bf0a5e..5bfe6b6a2303 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/DecoderModule.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/DecoderModule.java @@ -17,6 +17,7 @@ import com.facebook.presto.decoder.dummy.DummyDecoderModule; import com.facebook.presto.decoder.json.JsonDecoderModule; import com.facebook.presto.decoder.raw.RawDecoderModule; +import com.facebook.presto.decoder.thrift.ThriftDecoderModule; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Scopes; @@ -38,6 +39,7 @@ public void configure(Binder binder) binder.install(new CsvDecoderModule()); binder.install(new JsonDecoderModule()); binder.install(new RawDecoderModule()); + binder.install(new ThriftDecoderModule()); } public static void bindRowDecoder(Binder binder, Class decoderClass) diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/FieldValueProvider.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/FieldValueProvider.java index cb5f4f37e7a9..27c8a7a8ba0e 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/FieldValueProvider.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/FieldValueProvider.java @@ -14,6 +14,7 @@ package com.facebook.presto.decoder; import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.block.Block; import io.airlift.slice.Slice; /** @@ -44,4 +45,9 @@ public Slice getSlice() } public abstract boolean isNull(); + + public Block getBlock() + { + throw new PrestoException(DecoderErrorCode.DECODER_CONVERSION_NOT_SUPPORTED, "conversion to block not supported"); + } } diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftDecoderModule.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftDecoderModule.java new file mode 100644 index 000000000000..2f919dce9b4b --- /dev/null +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftDecoderModule.java @@ -0,0 +1,32 @@ +/* + * Licensed 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 com.facebook.presto.decoder.thrift; + +import com.google.inject.Binder; +import com.google.inject.Module; + +import static com.facebook.presto.decoder.DecoderModule.bindFieldDecoder; +import static com.facebook.presto.decoder.DecoderModule.bindRowDecoder; + +public class ThriftDecoderModule + implements Module +{ + @Override + public void configure(Binder binder) + { + bindRowDecoder(binder, ThriftRowDecoder.class); + bindFieldDecoder(binder, ThriftFieldDecoder.class); + } +} diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftFieldDecoder.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftFieldDecoder.java new file mode 100644 index 000000000000..c8be21f7c736 --- /dev/null +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftFieldDecoder.java @@ -0,0 +1,368 @@ +/* + * Licensed 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 com.facebook.presto.decoder.thrift; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.FieldDecoder; +import com.facebook.presto.decoder.FieldValueProvider; +import com.facebook.presto.spi.block.Block; +import com.facebook.presto.spi.block.BlockBuilder; +import com.facebook.presto.spi.block.BlockBuilderStatus; +import com.facebook.presto.spi.type.StandardTypes; +import com.facebook.presto.spi.type.Type; +import com.google.common.collect.ImmutableSet; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import org.joda.time.DateTimeZone; + +import java.sql.Date; +import java.sql.Timestamp; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; +import static com.facebook.presto.spi.type.Chars.isCharType; +import static com.facebook.presto.spi.type.Chars.truncateToLengthAndTrimSpaces; +import static com.facebook.presto.spi.type.DateType.DATE; +import static com.facebook.presto.spi.type.DoubleType.DOUBLE; +import static com.facebook.presto.spi.type.IntegerType.INTEGER; +import static com.facebook.presto.spi.type.RealType.REAL; +import static com.facebook.presto.spi.type.SmallintType.SMALLINT; +import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP; +import static com.facebook.presto.spi.type.TinyintType.TINYINT; +import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY; +import static com.facebook.presto.spi.type.Varchars.isVarcharType; +import static com.facebook.presto.spi.type.Varchars.truncateToLength; +import static com.google.common.base.Preconditions.checkArgument; +import static io.airlift.slice.Slices.EMPTY_SLICE; +import static java.lang.Float.floatToRawIntBits; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class ThriftFieldDecoder + implements FieldDecoder +{ + @Override + public Set> getJavaTypes() + { + return ImmutableSet.of(boolean.class, long.class, double.class, Slice.class, Block.class); + } + + @Override + public final String getRowDecoderName() + { + return ThriftRowDecoder.NAME; + } + + @Override + public String getFieldDecoderName() + { + return FieldDecoder.DEFAULT_FIELD_DECODER_NAME; + } + + @Override + public FieldValueProvider decode(Object value, DecoderColumnHandle columnHandle) + { + requireNonNull(columnHandle, "columnHandle is null"); + return new ObjectValueProvider(value, columnHandle); + } + + @Override + public String toString() + { + return format("FieldDecoder[%s/%s]", getRowDecoderName(), getFieldDecoderName()); + } + + public static class ObjectValueProvider + extends FieldValueProvider + { + protected final Object value; + protected final DecoderColumnHandle columnHandle; + + public ObjectValueProvider(Object value, DecoderColumnHandle columnHandle) + { + this.columnHandle = requireNonNull(columnHandle, "columnHandle is null"); + this.value = value; + } + + @Override + public final boolean accept(DecoderColumnHandle columnHandle) + { + return this.columnHandle.equals(columnHandle); + } + + @Override + public final boolean isNull() + { + return value == null; + } + + @Override + public boolean getBoolean() + { + return isNull() ? false : (Boolean) value; + } + + @Override + public long getLong() + { + return isNull() ? 0L : getLongExpressedValue(value); + } + + private static long getLongExpressedValue(Object value) + { + if (value instanceof Date) { + long storageTime = ((Date) value).getTime(); + // convert date from VM current time zone to UTC + long utcMillis = storageTime + DateTimeZone.getDefault().getOffset(storageTime); + return TimeUnit.MILLISECONDS.toDays(utcMillis); + } + if (value instanceof Timestamp) { + long parsedJvmMillis = ((Timestamp) value).getTime(); + DateTimeZone jvmTimeZone = DateTimeZone.getDefault(); + long convertedMillis = jvmTimeZone.convertUTCToLocal(parsedJvmMillis); + + return convertedMillis; + } + if (value instanceof Float) { + return floatToRawIntBits(((Float) value)); + } + return ((Number) value).longValue(); + } + + @Override + public double getDouble() + { + return isNull() ? 0.0d : (Double) value; + } + + @Override + public Slice getSlice() + { + return isNull() ? EMPTY_SLICE : getSliceExpressedValue(value, columnHandle.getType()); + } + + private static Slice getSliceExpressedValue(Object value, Type type) + { + Slice sliceValue; + if (value instanceof String) { + sliceValue = Slices.utf8Slice((String) value); + } + else if (value instanceof byte[]) { + sliceValue = Slices.wrappedBuffer((byte[]) value); + } + else if (value instanceof Integer) { + sliceValue = Slices.utf8Slice(value.toString()); + } + else { + throw new IllegalStateException("unsupported string field type: " + value.getClass().getName()); + } + if (isVarcharType(type)) { + sliceValue = truncateToLength(sliceValue, type); + } + if (isCharType(type)) { + sliceValue = truncateToLengthAndTrimSpaces(sliceValue, type); + } + + return sliceValue; + } + + @Override + public Block getBlock() + { + if (isNull()) { + return null; + } + + Type type = columnHandle.getType(); + return serializeObject(type, null, value); + } + + private static Block serializeObject(Type type, BlockBuilder builder, Object object) + { + if (!isStructuralType(type)) { + serializePrimitive(type, builder, object); + return null; + } + else if (isArrayType(type)) { + return serializeList(type, builder, object); + } + else if (isMapType(type)) { + return serializeMap(type, builder, object); + } + else if (isRowType(type)) { + return serializeStruct(type, builder, object); + } + throw new RuntimeException("Unknown object type: " + type); + } + + private static Block serializeList(Type type, BlockBuilder builder, Object object) + { + List list = (List) object; + if (list == null) { + requireNonNull(builder, "parent builder is null").appendNull(); + return null; + } + + List typeParameters = type.getTypeParameters(); + checkArgument(typeParameters.size() == 1, "list must have exactly 1 type parameter"); + Type elementType = typeParameters.get(0); + + BlockBuilder currentBuilder; + if (builder != null) { + currentBuilder = builder.beginBlockEntry(); + } + else { + currentBuilder = elementType.createBlockBuilder(new BlockBuilderStatus(), list.size()); + } + + for (Object element : list) { + serializeObject(elementType, currentBuilder, element); + } + + if (builder != null) { + builder.closeEntry(); + return null; + } + else { + Block resultBlock = currentBuilder.build(); + return resultBlock; + } + } + + private static Block serializeMap(Type type, BlockBuilder builder, Object object) + { + Map map = (Map) object; + if (map == null) { + requireNonNull(builder, "parent builder is null").appendNull(); + return null; + } + + List typeParameters = type.getTypeParameters(); + checkArgument(typeParameters.size() == 2, "map must have exactly 2 type parameter"); + Type keyType = typeParameters.get(0); + Type valueType = typeParameters.get(1); + + BlockBuilder currentBuilder; + if (builder != null) { + currentBuilder = builder.beginBlockEntry(); + } + else { + currentBuilder = type.createBlockBuilder(new BlockBuilderStatus(), map.size()); + } + + for (Map.Entry entry : map.entrySet()) { + // Hive skips map entries with null keys + if (entry.getKey() != null) { + serializeObject(keyType, currentBuilder, entry.getKey()); + serializeObject(valueType, currentBuilder, entry.getValue()); + } + } + + if (builder != null) { + builder.closeEntry(); + return null; + } + else { + Block resultBlock = currentBuilder.build(); + return resultBlock; + } + } + + private static Block serializeStruct(Type type, BlockBuilder builder, Object object) + { + if (object == null) { + requireNonNull(builder, "parent builder is null").appendNull(); + return null; + } + + List typeParameters = type.getTypeParameters(); + ThriftGenericRow structData = (ThriftGenericRow) object; + BlockBuilder currentBuilder; + if (builder != null) { + currentBuilder = builder.beginBlockEntry(); + } + else { + currentBuilder = type.createBlockBuilder(new BlockBuilderStatus(), typeParameters.size()); + } + + for (int i = 0; i < typeParameters.size(); i++) { + // TODO: Handle cases where ids are not consecutive + Object fieldValue = structData.getFieldValueForThriftId((short) (i + 1)); + serializeObject(typeParameters.get(i), currentBuilder, fieldValue); + } + + if (builder != null) { + builder.closeEntry(); + return null; + } + else { + Block resultBlock = currentBuilder.build(); + return resultBlock; + } + } + + private static void serializePrimitive(Type type, BlockBuilder builder, Object object) + { + requireNonNull(builder, "parent builder is null"); + + if (object == null) { + builder.appendNull(); + return; + } + + if (BOOLEAN.equals(type)) { + BOOLEAN.writeBoolean(builder, (Boolean) object); + } + else if (BIGINT.equals(type) || INTEGER.equals(type) || SMALLINT.equals(type) || TINYINT.equals(type) + || REAL.equals(type) || DATE.equals(type) || TIMESTAMP.equals(type)) { + type.writeLong(builder, getLongExpressedValue(object)); + } + else if (DOUBLE.equals(type)) { + DOUBLE.writeDouble(builder, ((Number) object).doubleValue()); + } + else if (isVarcharType(type) || VARBINARY.equals(type) || isCharType(type)) { + type.writeSlice(builder, getSliceExpressedValue(object, type)); + } + else { + throw new UnsupportedOperationException("Unsupported primitive type: " + type); + } + } + + public static boolean isArrayType(Type type) + { + return type.getTypeSignature().getBase().equals(StandardTypes.ARRAY); + } + + public static boolean isMapType(Type type) + { + return type.getTypeSignature().getBase().equals(StandardTypes.MAP); + } + + public static boolean isRowType(Type type) + { + return type.getTypeSignature().getBase().equals(StandardTypes.ROW); + } + + public static boolean isStructuralType(Type type) + { + String baseName = type.getTypeSignature().getBase(); + return baseName.equals(StandardTypes.MAP) || baseName.equals(StandardTypes.ARRAY) || baseName.equals(StandardTypes.ROW); + } + } +} diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftGenericRow.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftGenericRow.java new file mode 100644 index 000000000000..f3f356983bed --- /dev/null +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftGenericRow.java @@ -0,0 +1,248 @@ +/* + * Licensed 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 com.facebook.presto.decoder.thrift; + +import io.airlift.log.Logger; +import org.apache.commons.lang.ArrayUtils; +import org.apache.thrift.TBase; +import org.apache.thrift.TException; +import org.apache.thrift.TFieldIdEnum; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TField; +import org.apache.thrift.protocol.TList; +import org.apache.thrift.protocol.TMap; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.protocol.TProtocolUtil; +import org.apache.thrift.protocol.TSet; +import org.apache.thrift.protocol.TType; +import org.apache.thrift.transport.TMemoryInputTransport; +import org.apache.thrift.transport.TTransport; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class ThriftGenericRow + implements TBase +{ + private static final Logger log = Logger.get(ThriftGenericRow.class); + private final Map values = new HashMap<>(); + private byte[] buf; + private int off; + private int len; + + public ThriftGenericRow() + { + } + + public ThriftGenericRow(Map values) + { + this.values.putAll(values); + } + + public class Fields + implements TFieldIdEnum + { + private final short thriftId; + private final String fieldName; + + Fields(short thriftId, String fieldName) + { + this.thriftId = thriftId; + this.fieldName = fieldName; + } + + public short getThriftFieldId() + { + return thriftId; + } + + public String getFieldName() + { + return fieldName; + } + } + + public void read(TProtocol iprot) + throws TException + { + TTransport trans = iprot.getTransport(); + buf = trans.getBuffer(); + off = trans.getBufferPosition(); + TProtocolUtil.skip(iprot, TType.STRUCT); + len = trans.getBufferPosition() - off; + } + + public void parse() + throws TException + { + parse(null); + } + + public void parse(short[] thriftIds) + throws TException + { + Set idSet = thriftIds == null ? null : new HashSet(Arrays.asList(ArrayUtils.toObject(thriftIds))); + TMemoryInputTransport trans = new TMemoryInputTransport(buf, off, len); + TBinaryProtocol iprot = new TBinaryProtocol(trans); + TField field; + iprot.readStructBegin(); + while (true) { + field = iprot.readFieldBegin(); + if (field.type == TType.STOP) { + break; + } + if (idSet != null && !idSet.remove(Short.valueOf(field.id))) { + TProtocolUtil.skip(iprot, field.type); + } + else { + values.put(field.id, readElem(iprot, field.type)); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + } + + private Object readElem(TProtocol iprot, byte type) + throws TException + { + switch (type) { + case TType.BOOL: + return iprot.readBool(); + case TType.BYTE: + return iprot.readByte(); + case TType.I16: + return iprot.readI16(); + case TType.ENUM: + case TType.I32: + return iprot.readI32(); + case TType.I64: + return iprot.readI64(); + case TType.DOUBLE: + return iprot.readDouble(); + case TType.STRING: + return iprot.readString(); + case TType.STRUCT: + return readStruct(iprot); + case TType.LIST: + return readList(iprot); + case TType.SET: + return readSet(iprot); + case TType.MAP: + return readMap(iprot); + default: + TProtocolUtil.skip(iprot, type); + return null; + } + } + + private Object readStruct(TProtocol iprot) + throws TException + { + ThriftGenericRow elem = new ThriftGenericRow(); + elem.read(iprot); + elem.parse(); + return elem; + } + + private Object readList(TProtocol iprot) + throws TException + { + TList ilist = iprot.readListBegin(); + List listValue = new ArrayList<>(); + for (int i = 0; i < ilist.size; i++) { + listValue.add(readElem(iprot, ilist.elemType)); + } + iprot.readListEnd(); + return listValue; + } + + private Object readSet(TProtocol iprot) + throws TException + { + TSet iset = iprot.readSetBegin(); + List setValue = new ArrayList<>(); + for (int i = 0; i < iset.size; i++) { + setValue.add(readElem(iprot, iset.elemType)); + } + iprot.readSetEnd(); + return setValue; + } + + private Object readMap(TProtocol iprot) + throws TException + { + TMap imap = iprot.readMapBegin(); + Map mapValue = new HashMap<>(); + for (int i = 0; i < imap.size; i++) { + mapValue.put(readElem(iprot, imap.keyType), readElem(iprot, imap.valueType)); + } + iprot.readMapEnd(); + return mapValue; + } + + public Object getFieldValueForThriftId(short thriftId) + { + return values.get(thriftId); + } + + public ThriftGenericRow deepCopy() + { + return new ThriftGenericRow(values); + } + + public void clear() + { + } + + public Fields fieldForId(int fieldId) + { + return new Fields((short) fieldId, "dummy"); + } + + public Object getFieldValue(Fields field) + { + return values.get(field.thriftId); + } + + public boolean isSet(Fields field) + { + return values.containsKey(field.getThriftFieldId()); + } + + public void setFieldValue(Fields field, Object value) + { + values.put(field.getThriftFieldId(), value); + } + + public void write(TProtocol oprot) + throws TException + { + throw new UnsupportedOperationException("ThriftGenericRow.write is not supported."); + } + + public Map getValues() + { + return values; + } + + public int compareTo(ThriftGenericRow other) + { + throw new UnsupportedOperationException("ThriftGenericRow.compareTo is not supported."); + } +} diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftRowDecoder.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftRowDecoder.java new file mode 100644 index 000000000000..27dbff67c1b1 --- /dev/null +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftRowDecoder.java @@ -0,0 +1,110 @@ +/* + * Licensed 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 com.facebook.presto.decoder.thrift; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.FieldDecoder; +import com.facebook.presto.decoder.FieldValueProvider; +import com.facebook.presto.decoder.RowDecoder; +import com.google.common.base.Splitter; +import org.apache.thrift.TDeserializer; +import org.apache.thrift.TException; + +import javax.inject.Inject; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Thrift specific row decoder + */ +public class ThriftRowDecoder + implements RowDecoder +{ + public static final String NAME = "thrift"; + + @Inject + public ThriftRowDecoder() + { + } + + @Override + public String getName() + { + return NAME; + } + + @Override + public boolean decodeRow(byte[] data, + Map dataMap, + Set fieldValueProviders, + List columnHandles, + Map> fieldDecoders) + { + ThriftGenericRow row = new ThriftGenericRow(); + try { + TDeserializer deser = new TDeserializer(); + deser.deserialize(row, data); + row.parse(); + } + catch (TException e) { + return true; + } + + for (DecoderColumnHandle columnHandle : columnHandles) { + if (columnHandle.isInternal()) { + continue; + } + + @SuppressWarnings("unchecked") + FieldDecoder decoder = (FieldDecoder) fieldDecoders.get(columnHandle); + + if (decoder != null) { + Object node = locateNode(row.getValues(), columnHandle); + fieldValueProviders.add(decoder.decode(node, columnHandle)); + } + } + return false; + } + + private static Object locateNode(Map map, DecoderColumnHandle columnHandle) + { + Map currentLevel = map; + Object val = null; + + Iterator it = Splitter.on('/').omitEmptyStrings().split(columnHandle.getMapping()).iterator(); + while (it.hasNext()) { + String pathElement = it.next(); + Short key = Short.valueOf(pathElement); + val = currentLevel.get(key); + + // could be because of optional fields + if (val == null) { + return null; + } + + if (val instanceof ThriftGenericRow) { + currentLevel = ((ThriftGenericRow) val).getValues(); + } + else if (it.hasNext()) { + throw new IllegalStateException("Invalid thrift field schema"); + } + } + + return val; + } +} diff --git a/presto-record-decoder/src/test/java/com/facebook/presto/decoder/thrift/TestThriftDecoder.java b/presto-record-decoder/src/test/java/com/facebook/presto/decoder/thrift/TestThriftDecoder.java new file mode 100644 index 000000000000..02f8a133606e --- /dev/null +++ b/presto-record-decoder/src/test/java/com/facebook/presto/decoder/thrift/TestThriftDecoder.java @@ -0,0 +1,113 @@ +/* + * Licensed 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 com.facebook.presto.decoder.thrift; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.DecoderTestColumnHandle; +import com.facebook.presto.decoder.FieldDecoder; +import com.facebook.presto.decoder.FieldValueProvider; +import com.facebook.presto.decoder.thrift.tweep.Location; +import com.facebook.presto.decoder.thrift.tweep.Tweet; +import com.facebook.presto.decoder.thrift.tweep.TweetType; +import com.facebook.presto.spi.type.BigintType; +import com.facebook.presto.spi.type.BooleanType; +import com.facebook.presto.spi.type.DoubleType; +import com.facebook.presto.spi.type.IntegerType; +import com.facebook.presto.spi.type.SmallintType; +import com.facebook.presto.spi.type.TinyintType; +import com.facebook.presto.spi.type.VarbinaryType; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.transport.TMemoryBuffer; +import org.testng.annotations.Test; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static com.facebook.presto.decoder.util.DecoderTestUtil.checkValue; +import static com.facebook.presto.spi.type.VarcharType.createVarcharType; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; + +public class TestThriftDecoder +{ + private static final ThriftFieldDecoder DEFAULT_FIELD_DECODER = new ThriftFieldDecoder(); + + private static Map> buildMap(List columns) + { + ImmutableMap.Builder> map = ImmutableMap.builder(); + for (DecoderColumnHandle column : columns) { + map.put(column, DEFAULT_FIELD_DECODER); + } + return map.build(); + } + + @Test + public void testSimple() + throws Exception + { + Tweet tweet = new Tweet(1, "newUser", "hello world") + .setLoc(new Location(1234, 5678)) + .setAge((short) 26) + .setB((byte) 10) + .setIsDeleted(false) + .setTweetType(TweetType.REPLY) + .setFullId(1234567) + .setPic("abc".getBytes()) + .setAttr(ImmutableMap.of("a", "a")); + + ThriftRowDecoder rowDecoder = new ThriftRowDecoder(); + + // schema + DecoderTestColumnHandle col1 = new DecoderTestColumnHandle("", 1, "user_id", IntegerType.INTEGER, "1", "thrift", null, false, false, false); + DecoderTestColumnHandle col2 = new DecoderTestColumnHandle("", 2, "username", createVarcharType(100), "2", "thrift", null, false, false, false); + DecoderTestColumnHandle col3 = new DecoderTestColumnHandle("", 3, "text", createVarcharType(100), "3", "thrift", null, false, false, false); + DecoderTestColumnHandle col4 = new DecoderTestColumnHandle("", 4, "loc.latitude", DoubleType.DOUBLE, "4/1", "thrift", null, false, false, false); + DecoderTestColumnHandle col5 = new DecoderTestColumnHandle("", 5, "loc.longitude", DoubleType.DOUBLE, "4/2", "thrift", null, false, false, false); + DecoderTestColumnHandle col6 = new DecoderTestColumnHandle("", 6, "tweet_type", BigintType.BIGINT, "5", "thrift", null, false, false, false); + DecoderTestColumnHandle col7 = new DecoderTestColumnHandle("", 7, "is_deleted", BooleanType.BOOLEAN, "6", "thrift", null, false, false, false); + DecoderTestColumnHandle col8 = new DecoderTestColumnHandle("", 8, "b", TinyintType.TINYINT, "7", "thrift", null, false, false, false); + DecoderTestColumnHandle col9 = new DecoderTestColumnHandle("", 9, "age", SmallintType.SMALLINT, "8", "thrift", null, false, false, false); + DecoderTestColumnHandle col10 = new DecoderTestColumnHandle("", 10, "full_id", BigintType.BIGINT, "9", "thrift", null, false, false, false); + DecoderTestColumnHandle col11 = new DecoderTestColumnHandle("", 11, "pic", VarbinaryType.VARBINARY, "10", "thrift", null, false, false, false); + DecoderTestColumnHandle col12 = new DecoderTestColumnHandle("", 12, "language", createVarcharType(100), "16", "thrift", null, false, false, false); + + List columns = ImmutableList.of(col1, col2, col3, col4, col5, col6, col7, col8, col9, col10, col11, col12); + Set providers = new HashSet<>(); + + TMemoryBuffer transport = new TMemoryBuffer(4096); + TBinaryProtocol protocol = new TBinaryProtocol(transport); + tweet.write(protocol); + + boolean corrupt = rowDecoder.decodeRow(transport.getArray(), null, providers, columns, buildMap(columns)); + assertFalse(corrupt); + assertEquals(providers.size(), columns.size()); + + checkValue(providers, col1, 1); + checkValue(providers, col2, "newUser"); + checkValue(providers, col3, "hello world"); + checkValue(providers, col4, 1234); + checkValue(providers, col5, 5678); + checkValue(providers, col6, TweetType.REPLY.getValue()); + checkValue(providers, col7, false); + checkValue(providers, col8, 10); + checkValue(providers, col9, 26); + checkValue(providers, col10, 1234567); + checkValue(providers, col11, "abc"); + checkValue(providers, col12, "english"); + } +} diff --git a/presto-record-decoder/src/test/thrift/tweep.thrift b/presto-record-decoder/src/test/thrift/tweep.thrift new file mode 100644 index 000000000000..57bcc7c317e7 --- /dev/null +++ b/presto-record-decoder/src/test/thrift/tweep.thrift @@ -0,0 +1,43 @@ +namespace java com.facebook.presto.decoder.thrift.tweep + +enum TweetType { + TWEET, + RETWEET = 2, + DM = 0xa, + REPLY +} + +struct Location { + 1: required double latitude; + 2: required double longitude; +} + +struct Tweet { + 1: required i32 userId; + 2: required string userName; + 3: required string text; + 4: optional Location loc; + 5: optional TweetType tweetType = TweetType.TWEET; + 6: optional bool isDeleted = false; + 7: optional byte b; + 8: optional i16 age; + 9: optional i64 fullId; + 10: optional binary pic; + 11: optional map attr; + 12: optional list items; + 16: optional string language = "english"; +} + +typedef list TweetList +typedef set TweetSet + +struct TweetSearchResult { + 1: TweetList tweetList; + 2: TweetSet tweetSet; +} + +exception TwitterUnavailable { + 1: string message; +} + +const i32 MAX_RESULTS = 100; diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 3421dbaf44bb..e4553afc4421 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index ab7dd6f956d1..124e1570799b 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 0a178b3239ae..7ee81b1524a0 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 7b59da2a926c..852cb08830a4 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index ebd88abecf0e..eb0e0a6ffce3 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index ca5e0f8ba489..4598b523abae 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-spi diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/type/TypeSignature.java b/presto-spi/src/main/java/com/facebook/presto/spi/type/TypeSignature.java index 1570d045bbb0..74a613ff4b1c 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/type/TypeSignature.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/type/TypeSignature.java @@ -42,6 +42,7 @@ public class TypeSignature static { BASE_NAME_ALIAS_TO_CANONICAL.put("int", StandardTypes.INTEGER); + BASE_NAME_ALIAS_TO_CANONICAL.put("", "unknown"); } public TypeSignature(String base, TypeSignatureParameter... parameters) diff --git a/presto-spi/src/test/java/com/facebook/presto/spi/type/TestTypeSignature.java b/presto-spi/src/test/java/com/facebook/presto/spi/type/TestTypeSignature.java index 091cced4e1ba..8b7ab83b533d 100644 --- a/presto-spi/src/test/java/com/facebook/presto/spi/type/TestTypeSignature.java +++ b/presto-spi/src/test/java/com/facebook/presto/spi/type/TestTypeSignature.java @@ -78,6 +78,11 @@ public void parseRowSignature() ImmutableSet.of("p1", "s1", "p2", "s2"), rowSignature(namedParameter("a", decimal("p1", "s1")), namedParameter("b", decimal("p2", "s2")))); assertEquals(parseTypeSignature("row(a Int(p1))"), parseTypeSignature("row(a integer(p1))")); + assertRowSignature( + "row()", + "row", + ImmutableList.of("field0 unknown"), + "row(field0 unknown)"); // TODO: remove the following tests when the old style row type has been completely dropped assertOldRowSignature( @@ -174,9 +179,9 @@ public void parseSignature() "map", ImmutableList.of("bigint", "map(bigint,map(varchar,bigint))")); - assertSignatureFail("blah()"); - assertSignatureFail("array()"); - assertSignatureFail("map()"); + assertSignature("blah()", "blah", ImmutableList.of("unknown"), "blah(unknown)"); + assertSignature("array()", "array", ImmutableList.of("unknown"), "array(unknown)"); + assertSignature("map()", "map", ImmutableList.of("unknown"), "map(unknown)"); assertSignatureFail("x", ImmutableSet.of("x")); // ensure this is not treated as a row type diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 39e57a4f08e0..00ebd689dd05 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.196 + 0.196-tw-0.49 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index db25c6e73fdd..d3027a7dc7d6 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index c7298620b767..ddbcb9fad9c3 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 55a120133344..8b8c91cae7c7 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.196 + 0.196-tw-0.49 presto-tests @@ -120,6 +120,8 @@ com.google.guava guava + + 21.0 diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 003792555c49..7987bcdad12b 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 7bc59918bc6b..670ad0c7d4a4 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index c73ae23af78d..b56a9d2ac497 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 694f3f7b0f5f..55c3f5f2fabd 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index f33edf2a5d7f..70131dfb7a62 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml new file mode 100644 index 000000000000..ec79cee209b6 --- /dev/null +++ b/presto-twitter-functions/pom.xml @@ -0,0 +1,65 @@ + + + 4.0.0 + + + com.facebook.presto + presto-root + 0.196-tw-0.49 + + + presto-twitter-functions + Twitter's specific functions for Presto + presto-plugin + + + ${project.parent.basedir} + + + + + com.google.guava + guava + + + + + com.facebook.presto + presto-spi + provided + + + + io.airlift + slice + provided + + + + + org.testng + testng + test + + + + com.facebook.presto + presto-tests + test + + + + com.facebook.presto + presto-main + test + + + + com.facebook.presto + presto-main + test-jar + test + + + + diff --git a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/SnowflakeFunctions.java b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/SnowflakeFunctions.java new file mode 100644 index 000000000000..979cb95a8504 --- /dev/null +++ b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/SnowflakeFunctions.java @@ -0,0 +1,129 @@ +/* + * Licensed 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 com.facebook.presto.twitter.functions; + +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.function.Description; +import com.facebook.presto.spi.function.ScalarFunction; +import com.facebook.presto.spi.function.SqlType; +import com.facebook.presto.spi.type.StandardTypes; + +import java.util.concurrent.TimeUnit; + +import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; + +/* size in + * bits: 1 41 5 5 12 + * +-+-----------------------------------------+-----+-----+------------+ + * |0| milliseconds |clstr|instc| sequence | + * |0| since twepoch | id | id | number | + * +-+-----------------------------------------+-----+-----+------------+ + * | + * +- Most significant bit + */ +public class SnowflakeFunctions +{ + private static final long SequenceNumBits = 12L; + private static final long MaxSequenceNum = (1L << SequenceNumBits) - 1; + private static final long SequenceNumMask = MaxSequenceNum; + + private static final long InstanceIdBits = 5L; + private static final long MaxInstanceId = (1L << InstanceIdBits) - 1; + private static final long InstanceIdShift = SequenceNumBits; + private static final long InstanceIdMask = MaxInstanceId << InstanceIdShift; + + private static final long ClusterIdBits = 5L; + private static final long MaxClusterId = (1L << ClusterIdBits) - 1; + private static final long ClusterIdShift = InstanceIdShift + InstanceIdBits; + private static final long ClusterIdMask = MaxClusterId << ClusterIdShift; + + private static final long TimestampBits = 41L; + private static final long MaxTimestamp = (1L << TimestampBits) - 1; + private static final long TimestampShift = ClusterIdShift + ClusterIdBits; + private static final long TimestampMask = MaxTimestamp << TimestampShift; + + /* Twepoch is 2010-11-04T01:42:54Z. + * Value is in millis since Unix Epoch 1970-01-01T00:00:00Z. + */ + private static final long Twepoch = 1288834974657L; + private static final long FirstSnowflakeIdUnixTime = Twepoch + TimeUnit.DAYS.toMillis(1); // 1 day after Twepoch. + private static final long FirstSnowflakeId = firstSnowflakeIdFor(FirstSnowflakeIdUnixTime); + + private SnowflakeFunctions() + { + } + + @ScalarFunction("is_snowflake") + @Description("Check if a BIGINT is a Snowflake ID") + @SqlType(StandardTypes.BOOLEAN) + public static boolean isSnowflakeId(@SqlType(StandardTypes.BIGINT) long id) + { + return id >= FirstSnowflakeId; + } + + @ScalarFunction("first_snowflake_for") + @Description("Return the first snowflake ID given a timestamp") + @SqlType(StandardTypes.BIGINT) + public static long firstSnowflakeIdFor(@SqlType(StandardTypes.TIMESTAMP) long timestamp) + { + if (timestamp < FirstSnowflakeIdUnixTime) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Invalid UnixTimeMillis: UnixTimeMillis[" + timestamp + "] >= FirstSnowflakeIdUnixTime"); + } + return ((timestamp - Twepoch) << TimestampShift); + } + + @ScalarFunction("timestamp_from_snowflake") + @Description("Return the timestamp given a snowflake ID") + @SqlType(StandardTypes.TIMESTAMP) + public static long timestampFromSnowflakeId(@SqlType(StandardTypes.BIGINT) long id) + { + if (!isSnowflakeId(id)) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Not a Snowflake Id: " + id); + } + return ((id & TimestampMask) >> TimestampShift) + Twepoch; + } + + @ScalarFunction("cluster_id_from_snowflake") + @Description("Return the cluster id given a snowflake ID") + @SqlType(StandardTypes.BIGINT) + public static long clusterIdFromSnowflakeId(@SqlType(StandardTypes.BIGINT) long id) + { + if (!isSnowflakeId(id)) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Not a Snowflake Id: " + id); + } + return (id & ClusterIdMask) >> ClusterIdShift; + } + + @ScalarFunction("instance_id_from_snowflake") + @Description("Return the instance id given a snowflake ID") + @SqlType(StandardTypes.BIGINT) + public static long instanceIdFromSnowflakeId(@SqlType(StandardTypes.BIGINT) long id) + { + if (!isSnowflakeId(id)) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Not a Snowflake Id: " + id); + } + return (id & InstanceIdMask) >> InstanceIdShift; + } + + @ScalarFunction("sequence_num_from_snowflake") + @Description("Return the sequence number given a snowflake ID") + @SqlType(StandardTypes.BIGINT) + public static long sequenceNumFromSnowflakeId(@SqlType(StandardTypes.BIGINT) long id) + { + if (!isSnowflakeId(id)) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Not a Snowflake Id: " + id); + } + return id & SequenceNumMask; + } +} diff --git a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterFunctionsPlugin.java b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterFunctionsPlugin.java new file mode 100644 index 000000000000..6a9a491e9306 --- /dev/null +++ b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterFunctionsPlugin.java @@ -0,0 +1,32 @@ +/* + * Licensed 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 com.facebook.presto.twitter.functions; + +import com.facebook.presto.spi.Plugin; +import com.google.common.collect.ImmutableSet; + +import java.util.Set; + +public class TwitterFunctionsPlugin + implements Plugin +{ + @Override + public Set> getFunctions() + { + return ImmutableSet.>builder() + .add(TwitterStringFunctions.class) + .add(SnowflakeFunctions.class) + .build(); + } +} diff --git a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java new file mode 100644 index 000000000000..dac9cf846a9a --- /dev/null +++ b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java @@ -0,0 +1,103 @@ +/* + * Licensed 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 com.facebook.presto.twitter.functions; + +import com.facebook.presto.spi.ErrorCodeSupplier; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.block.Block; +import com.facebook.presto.spi.block.BlockBuilder; +import com.facebook.presto.spi.block.BlockBuilderStatus; +import com.facebook.presto.spi.function.Description; +import com.facebook.presto.spi.function.LiteralParameters; +import com.facebook.presto.spi.function.ScalarFunction; +import com.facebook.presto.spi.function.SqlType; +import com.facebook.presto.spi.type.StandardTypes; +import com.google.common.primitives.Ints; +import io.airlift.slice.Slice; + +import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; +import static com.facebook.presto.spi.type.VarcharType.VARCHAR; +import static io.airlift.slice.SliceUtf8.offsetOfCodePoint; +import static java.lang.String.format; + +public class TwitterStringFunctions +{ + private TwitterStringFunctions() + { + } + + @ScalarFunction("split_every") + @Description("Splits the string on every character and returns an array") + @LiteralParameters({"x"}) + @SqlType("array(varchar(x))") + public static Block str2array(@SqlType("varchar(x)") Slice utf8) + { + return str2array(utf8, 1, utf8.length() + 1); + } + + @ScalarFunction("split_every") + @Description("Splits the string on every given length of characters and returns an array") + @LiteralParameters({"x"}) + @SqlType("array(varchar(x))") + public static Block str2array(@SqlType("varchar(x)") Slice utf8, @SqlType(StandardTypes.BIGINT) long length) + { + return str2array(utf8, length, utf8.length() / length + 1); + } + + @ScalarFunction("split_every") + @Description("Splits the string on every given length of characters and returns an array with the size at most of the given limit") + @LiteralParameters({"x"}) + @SqlType("array(varchar(x))") + public static Block str2array(@SqlType("varchar(x)") Slice utf8, @SqlType(StandardTypes.BIGINT) long length, @SqlType(StandardTypes.BIGINT) long limit) + { + checkCondition(limit > 0, INVALID_FUNCTION_ARGUMENT, "Limit must be positive"); + checkCondition(limit <= Integer.MAX_VALUE, INVALID_FUNCTION_ARGUMENT, "Limit is too large"); + checkCondition(length > 0, INVALID_FUNCTION_ARGUMENT, "Length must be positive"); + checkCondition(length <= Integer.MAX_VALUE, INVALID_FUNCTION_ARGUMENT, "Length is too large"); + BlockBuilder parts = VARCHAR.createBlockBuilder(new BlockBuilderStatus(), 1, Ints.saturatedCast(length)); + // If limit is one, the last and only element is the complete string + if (limit == 1) { + VARCHAR.writeSlice(parts, utf8); + return parts.build(); + } + + int index = offsetOfCodePoint(utf8, 0); + while (index < utf8.length()) { + int splitIndex = offsetOfCodePoint(utf8, index, Ints.saturatedCast(length)); + // Enough remaining string? + if (splitIndex < 0) { + break; + } + // Add the part from current index to found split + VARCHAR.writeSlice(parts, utf8, index, splitIndex - index); + // Continue after current end + index = splitIndex; + // Reached limit-1 parts so we can stop + if (parts.getPositionCount() == limit - 1) { + break; + } + } + // Rest of string + VARCHAR.writeSlice(parts, utf8, index, utf8.length() - index); + + return parts.build(); + } + + private static void checkCondition(boolean condition, ErrorCodeSupplier errorCode, String formatString, Object... args) + { + if (!condition) { + throw new PrestoException(errorCode, format(formatString, args)); + } + } +} diff --git a/presto-twitter-functions/src/test/java/com/facebook/presto/twitter/functions/TestTwitterFunctions.java b/presto-twitter-functions/src/test/java/com/facebook/presto/twitter/functions/TestTwitterFunctions.java new file mode 100644 index 000000000000..4c75f59a88f5 --- /dev/null +++ b/presto-twitter-functions/src/test/java/com/facebook/presto/twitter/functions/TestTwitterFunctions.java @@ -0,0 +1,82 @@ +/* + * Licensed 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 com.facebook.presto.twitter.functions; + +import com.facebook.presto.operator.scalar.AbstractTestFunctions; +import com.facebook.presto.spi.type.ArrayType; +import com.facebook.presto.spi.type.SqlTimestamp; +import com.google.common.collect.ImmutableList; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static com.facebook.presto.metadata.FunctionExtractor.extractFunctions; +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; +import static com.facebook.presto.spi.type.TimeZoneKey.UTC_KEY; +import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP; +import static com.facebook.presto.spi.type.VarcharType.createVarcharType; + +public class TestTwitterFunctions + extends AbstractTestFunctions +{ + @BeforeClass + public void setUp() + { + functionAssertions.addFunctions(extractFunctions(new TwitterFunctionsPlugin().getFunctions())); + } + + @Test + public void testStr2Array() + { + assertFunction("SPLIT_EVERY('')", new ArrayType(createVarcharType(0)), ImmutableList.of("")); + assertFunction("SPLIT_EVERY('abc')", new ArrayType(createVarcharType(3)), ImmutableList.of("a", "b", "c")); + assertFunction("SPLIT_EVERY('a.b.c')", new ArrayType(createVarcharType(5)), ImmutableList.of("a", ".", "b", ".", "c")); + assertFunction("SPLIT_EVERY('...')", new ArrayType(createVarcharType(3)), ImmutableList.of(".", ".", ".")); + // Test str_to_array for non-ASCII + assertFunction("SPLIT_EVERY('\u4FE1\u5FF5,\u7231,\u5E0C\u671B')", new ArrayType(createVarcharType(7)), ImmutableList.of("\u4FE1", "\u5FF5", ",", "\u7231", ",", "\u5E0C", "\u671B")); + // Test argument length + assertFunction("SPLIT_EVERY('a.b.c', 2)", new ArrayType(createVarcharType(5)), ImmutableList.of("a.", "b.", "c")); + // Test argument limit + assertFunction("SPLIT_EVERY('a.b.c', 2, 1)", new ArrayType(createVarcharType(5)), ImmutableList.of("a.b.c")); + assertFunction("SPLIT_EVERY('a.b.c', 2, 2)", new ArrayType(createVarcharType(5)), ImmutableList.of("a.", "b.c")); + } + + private static SqlTimestamp toTimestampUTC(long millis) + { + return new SqlTimestamp(millis, UTC_KEY); + } + + @Test + public void testSnowflake() + { + assertFunction("IS_SNOWFLAKE(1000)", BOOLEAN, false); + assertFunction("IS_SNOWFLAKE(265605588183052288)", BOOLEAN, true); + assertFunction("IS_SNOWFLAKE(-265605588183052288)", BOOLEAN, false); + + assertFunction("FIRST_SNOWFLAKE_FOR(from_unixtime(1352160281.593))", BIGINT, 265605588182892544L); + assertInvalidFunction("FIRST_SNOWFLAKE_FOR(from_unixtime(1000))", "Invalid UnixTimeMillis: UnixTimeMillis[1000000] >= FirstSnowflakeIdUnixTime"); + + assertFunction("TIMESTAMP_FROM_SNOWFLAKE(265605588183052288)", TIMESTAMP, toTimestampUTC(1352160281593L)); + assertInvalidFunction("TIMESTAMP_FROM_SNOWFLAKE(1000)", "Not a Snowflake Id: 1000"); + + assertFunction("CLUSTER_ID_FROM_SNOWFLAKE(265605588183052288)", BIGINT, 1L); + assertInvalidFunction("CLUSTER_ID_FROM_SNOWFLAKE(1000)", "Not a Snowflake Id: 1000"); + + assertFunction("INSTANCE_ID_FROM_SNOWFLAKE(265605588183052288)", BIGINT, 7L); + assertInvalidFunction("INSTANCE_ID_FROM_SNOWFLAKE(1000)", "Not a Snowflake Id: 1000"); + + assertFunction("SEQUENCE_NUM_FROM_SNOWFLAKE(265605588183052288)", BIGINT, 0L); + assertInvalidFunction("SEQUENCE_NUM_FROM_SNOWFLAKE(1000)", "Not a Snowflake Id: 1000"); + } +} diff --git a/presto-twitter-server/NOTICE b/presto-twitter-server/NOTICE new file mode 100644 index 000000000000..6182bbb1148d --- /dev/null +++ b/presto-twitter-server/NOTICE @@ -0,0 +1,2836 @@ +THE FOLLOWING IS SOFTWARE LICENSED BY THIRD PARTIES UNDER OPEN SOURCE LICENSES THAT MAY BE USED BY THIS PRODUCT. + +----- + +The following software may be included in this product: aether. The source code is available at http://eclipse.org/aether/download/. You may also request a copy of the source code by sending a request to opensource@fb.com. This software contains the following license and notice below: + +Eclipse Public License - v 1.0 + +THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. + +1. DEFINITIONS + +"Contribution" means: + +a) in the case of the initial Contributor, the initial code and documentation distributed under this Agreement, and + +b) in the case of each subsequent Contributor: + +i) changes to the Program, and + +ii) additions to the Program; + +where such changes and/or additions to the Program originate from and are distributed by that particular Contributor. A Contribution 'originates' from a Contributor if it was added to the Program by such Contributor itself or anyone acting on such Contributor's behalf. Contributions do not include additions to the Program which: (i) are separate modules of software distributed in conjunction with the Program under their own license agreement, and (ii) are not derivative works of the Program. + +"Contributor" means any person or entity that distributes the Program. + +"Licensed Patents" mean patent claims licensable by a Contributor which are necessarily infringed by the use or sale of its Contribution alone or when combined with the Program. + +"Program" means the Contributions distributed in accordance with this Agreement. + +"Recipient" means anyone who receives the Program under this Agreement, including all Contributors. + +2. GRANT OF RIGHTS + +a) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free copyright license to reproduce, prepare derivative works of, publicly display, publicly perform, distribute and sublicense the Contribution of such Contributor, if any, and such derivative works, in source code and object code form. + +b) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed Patents to make, use, sell, offer to sell, import and otherwise transfer the Contribution of such Contributor, if any, in source code and object code form. This patent license shall apply to the combination of the Contribution and the Program if, at the time the Contribution is added by the Contributor, such addition of the Contribution causes such combination to be covered by the Licensed Patents. The patent license shall not apply to any other combinations which include the Contribution. No hardware per se is licensed hereunder. + +c) Recipient understands that although each Contributor grants the licenses to its Contributions set forth herein, no assurances are provided by any Contributor that the Program does not infringe the patent or other intellectual property rights of any other entity. Each Contributor disclaims any liability to Recipient for claims brought by any other entity based on infringement of intellectual property rights or otherwise. As a condition to exercising the rights and licenses granted hereunder, each Recipient hereby assumes sole responsibility to secure any other intellectual property rights needed, if any. For example, if a third party patent license is required to allow Recipient to distribute the Program, it is Recipient's responsibility to acquire that license before distributing the Program. + +d) Each Contributor represents that to its knowledge it has sufficient copyright rights in its Contribution, if any, to grant the copyright license set forth in this Agreement. + +3. REQUIREMENTS + +A Contributor may choose to distribute the Program in object code form under its own license agreement, provided that: + +a) it complies with the terms and conditions of this Agreement; and + +b) its license agreement: + +i) effectively disclaims on behalf of all Contributors all warranties and conditions, express and implied, including warranties or conditions of title and non-infringement, and implied warranties or conditions of merchantability and fitness for a particular purpose; + +ii) effectively excludes on behalf of all Contributors all liability for damages, including direct, indirect, special, incidental and consequential damages, such as lost profits; + +iii) states that any provisions which differ from this Agreement are offered by that Contributor alone and not by any other party; and + +iv) states that source code for the Program is available from such Contributor, and informs licensees how to obtain it in a reasonable manner on or through a medium customarily used for software exchange. + +When the Program is made available in source code form: + +a) it must be made available under this Agreement; and + +b) a copy of this Agreement must be included with each copy of the Program. + +Contributors may not remove or alter any copyright notices contained within the Program. + +Each Contributor must identify itself as the originator of its Contribution, if any, in a manner that reasonably allows subsequent Recipients to identify the originator of the Contribution. + +4. COMMERCIAL DISTRIBUTION + +Commercial distributors of software may accept certain responsibilities with respect to end users, business partners and the like. While this license is intended to facilitate the commercial use of the Program, the Contributor who includes the Program in a commercial product offering should do so in a manner which does not create potential liability for other Contributors. Therefore, if a Contributor includes the Program in a commercial product offering, such Contributor ("Commercial Contributor") hereby agrees to defend and indemnify every other Contributor ("Indemnified Contributor") against any losses, damages and costs (collectively "Losses") arising from claims, lawsuits and other legal actions brought by a third party against the Indemnified Contributor to the extent caused by the acts or omissions of such Commercial Contributor in connection with its distribution of the Program in a commercial product offering. The obligations in this section do not apply to any claims or Losses relating to any actual or alleged intellectual property infringement. In order to qualify, an Indemnified Contributor must: a) promptly notify the Commercial Contributor in writing of such claim, and b) allow the Commercial Contributor to control, and cooperate with the Commercial Contributor in, the defense and any related settlement negotiations. The Indemnified Contributor may participate in any such claim at its own expense. + +For example, a Contributor might include the Program in a commercial product offering, Product X. That Contributor is then a Commercial Contributor. If that Commercial Contributor then makes performance claims, or offers warranties related to Product X, those performance claims and warranties are such Commercial Contributor's responsibility alone. Under this section, the Commercial Contributor would have to defend claims against the other Contributors related to those performance claims and warranties, and if a court requires any other Contributor to pay any damages as a result, the Commercial Contributor must pay those damages. + +5. NO WARRANTY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each Recipient is solely responsible for determining the appropriateness of using and distributing the Program and assumes all risks associated with its exercise of rights under this Agreement , including but not limited to the risks and costs of program errors, compliance with applicable laws, damage to or loss of data, programs or equipment, and unavailability or interruption of operations. + +6. DISCLAIMER OF LIABILITY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + +7. GENERAL + +If any provision of this Agreement is invalid or unenforceable under applicable law, it shall not affect the validity or enforceability of the remainder of the terms of this Agreement, and without further action by the parties hereto, such provision shall be reformed to the minimum extent necessary to make such provision valid and enforceable. + +If Recipient institutes patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Program itself (excluding combinations of the Program with other software or hardware) infringes such Recipient's patent(s), then such Recipient's rights granted under Section 2(b) shall terminate as of the date such litigation is filed. + +All Recipient's rights under this Agreement shall terminate if it fails to comply with any of the material terms or conditions of this Agreement and does not cure such failure in a reasonable period of time after becoming aware of such noncompliance. If all Recipient's rights under this Agreement terminate, Recipient agrees to cease use and distribution of the Program as soon as reasonably practicable. However, Recipient's obligations under this Agreement and any licenses granted by Recipient relating to the Program shall continue and survive. + +Everyone is permitted to copy and distribute copies of this Agreement, but in order to avoid inconsistency the Agreement is copyrighted and may only be modified in the following manner. The Agreement Steward reserves the right to publish new versions (including revisions) of this Agreement from time to time. No one other than the Agreement Steward has the right to modify this Agreement. The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation may assign the responsibility to serve as the Agreement Steward to a suitable separate entity. Each new version of the Agreement will be given a distinguishing version number. The Program (including Contributions) may always be distributed subject to the version of the Agreement under which it was received. In addition, after a new version of the Agreement is published, Contributor may elect to distribute the Program (including its Contributions) under the new version. Except as expressly stated in Sections 2(a) and 2(b) above, Recipient receives no rights or licenses to the intellectual property of any Contributor under this Agreement, whether expressly, by implication, estoppel or otherwise. All rights in the Program not expressly granted under this Agreement are reserved. + +This Agreement is governed by the laws of the State of New York and the intellectual property laws of the United States of America. No party to this Agreement will bring a legal action under this Agreement more than one year after the cause of action arose. Each party waives its rights to a jury trial in any resulting litigation. + +----- + +The following software may be included in this product: antlr stringtemplate4, antlr runtime. This software contains the following license and notice below: + +[The "BSD license"] +Copyright (c) 2011-2013 Terence Parr +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + + 1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + 3. The name of the author may not be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR +IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES +OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. +IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT +NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF +THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +----- + +The following software may be included in this product: Apache Avro, Apache BVal, Apache Commons BeanUtils Core, Apache Commons CLI, Apache Commons Codec, Apache Commons Configuration, Apache Commons IO, Apache Commons Lang, Apache Commons Logging, Apache Hadoop, Apache Hive, Apache HttpClient, Apache Maven, Apache Thrift, Apache XBean, Bean Validation API, Code Generation Library, Guava, Jackson, Jetty, Joda time, Log4j Implemented Over SLF4J, Ning Asynchronous Http Client, Plexus, Tableau Web Data Connector, airlift, airlift resolver, airlift slice, fastutil, jDBI, javax.inject, jmxutils, jQuery, opencsv, snappy, vis.js. +This software contains the following license and notice below: + + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + +TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + +1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + +2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + +3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + +4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + +5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + +6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + +7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + +8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + +9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + +END OF TERMS AND CONDITIONS + +APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + +Copyright [yyyy] [name of copyright owner] + +Licensed 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. + +----- + +The following software may be included in this product: asm. This software contains the following license and notice below: + +Copyright (c) 2000-2011 INRIA, France Telecom +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + +3. Neither the name of the copyright holders nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF +THE POSSIBILITY OF SUCH DAMAGE. + +----- + +The following software may be included in this product: findbugs. The source code is available at http://code.google.com/p/findbugs/. You may also request a copy of the source code by sending a request to opensource@fb.com. This software contains the following license and notice below: + + GNU LESSER GENERAL PUBLIC LICENSE + Version 2.1, February 1999 + + Copyright (C) 1991, 1999 Free Software Foundation, Inc. + 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA + Everyone is permitted to copy and distribute verbatim copies + of this license document, but changing it is not allowed. + +[This is the first released version of the Lesser GPL. It also counts + as the successor of the GNU Library Public License, version 2, hence + the version number 2.1.] + + Preamble + + The licenses for most software are designed to take away your +freedom to share and change it. By contrast, the GNU General Public +Licenses are intended to guarantee your freedom to share and change +free software--to make sure the software is free for all its users. + + This license, the Lesser General Public License, applies to some +specially designated software packages--typically libraries--of the +Free Software Foundation and other authors who decide to use it. You +can use it too, but we suggest you first think carefully about whether +this license or the ordinary General Public License is the better +strategy to use in any particular case, based on the explanations below. + + When we speak of free software, we are referring to freedom of use, +not price. Our General Public Licenses are designed to make sure that +you have the freedom to distribute copies of free software (and charge +for this service if you wish); that you receive source code or can get +it if you want it; that you can change the software and use pieces of +it in new free programs; and that you are informed that you can do +these things. + + To protect your rights, we need to make restrictions that forbid +distributors to deny you these rights or to ask you to surrender these +rights. These restrictions translate to certain responsibilities for +you if you distribute copies of the library or if you modify it. + + For example, if you distribute copies of the library, whether gratis +or for a fee, you must give the recipients all the rights that we gave +you. You must make sure that they, too, receive or can get the source +code. If you link other code with the library, you must provide +complete object files to the recipients, so that they can relink them +with the library after making changes to the library and recompiling +it. And you must show them these terms so they know their rights. + + We protect your rights with a two-step method: (1) we copyright the +library, and (2) we offer you this license, which gives you legal +permission to copy, distribute and/or modify the library. + + To protect each distributor, we want to make it very clear that +there is no warranty for the free library. Also, if the library is +modified by someone else and passed on, the recipients should know +that what they have is not the original version, so that the original +author's reputation will not be affected by problems that might be +introduced by others. + + Finally, software patents pose a constant threat to the existence of +any free program. We wish to make sure that a company cannot +effectively restrict the users of a free program by obtaining a +restrictive license from a patent holder. Therefore, we insist that +any patent license obtained for a version of the library must be +consistent with the full freedom of use specified in this license. + + Most GNU software, including some libraries, is covered by the +ordinary GNU General Public License. This license, the GNU Lesser +General Public License, applies to certain designated libraries, and +is quite different from the ordinary General Public License. We use +this license for certain libraries in order to permit linking those +libraries into non-free programs. + + When a program is linked with a library, whether statically or using +a shared library, the combination of the two is legally speaking a +combined work, a derivative of the original library. The ordinary +General Public License therefore permits such linking only if the +entire combination fits its criteria of freedom. The Lesser General +Public License permits more lax criteria for linking other code with +the library. + + We call this license the "Lesser" General Public License because it +does Less to protect the user's freedom than the ordinary General +Public License. It also provides other free software developers Less +of an advantage over competing non-free programs. These disadvantages +are the reason we use the ordinary General Public License for many +libraries. However, the Lesser license provides advantages in certain +special circumstances. + + For example, on rare occasions, there may be a special need to +encourage the widest possible use of a certain library, so that it becomes +a de-facto standard. To achieve this, non-free programs must be +allowed to use the library. A more frequent case is that a free +library does the same job as widely used non-free libraries. In this +case, there is little to gain by limiting the free library to free +software only, so we use the Lesser General Public License. + + In other cases, permission to use a particular library in non-free +programs enables a greater number of people to use a large body of +free software. For example, permission to use the GNU C Library in +non-free programs enables many more people to use the whole GNU +operating system, as well as its variant, the GNU/Linux operating +system. + + Although the Lesser General Public License is Less protective of the +users' freedom, it does ensure that the user of a program that is +linked with the Library has the freedom and the wherewithal to run +that program using a modified version of the Library. + + The precise terms and conditions for copying, distribution and +modification follow. Pay close attention to the difference between a +"work based on the library" and a "work that uses the library". The +former contains code derived from the library, whereas the latter must +be combined with the library in order to run. + + GNU LESSER GENERAL PUBLIC LICENSE + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. This License Agreement applies to any software library or other +program which contains a notice placed by the copyright holder or +other authorized party saying it may be distributed under the terms of +this Lesser General Public License (also called "this License"). +Each licensee is addressed as "you". + + A "library" means a collection of software functions and/or data +prepared so as to be conveniently linked with application programs +(which use some of those functions and data) to form executables. + + The "Library", below, refers to any such software library or work +which has been distributed under these terms. A "work based on the +Library" means either the Library or any derivative work under +copyright law: that is to say, a work containing the Library or a +portion of it, either verbatim or with modifications and/or translated +straightforwardly into another language. (Hereinafter, translation is +included without limitation in the term "modification".) + + "Source code" for a work means the preferred form of the work for +making modifications to it. For a library, complete source code means +all the source code for all modules it contains, plus any associated +interface definition files, plus the scripts used to control compilation +and installation of the library. + + Activities other than copying, distribution and modification are not +covered by this License; they are outside its scope. The act of +running a program using the Library is not restricted, and output from +such a program is covered only if its contents constitute a work based +on the Library (independent of the use of the Library in a tool for +writing it). Whether that is true depends on what the Library does +and what the program that uses the Library does. + + 1. You may copy and distribute verbatim copies of the Library's +complete source code as you receive it, in any medium, provided that +you conspicuously and appropriately publish on each copy an +appropriate copyright notice and disclaimer of warranty; keep intact +all the notices that refer to this License and to the absence of any +warranty; and distribute a copy of this License along with the +Library. + + You may charge a fee for the physical act of transferring a copy, +and you may at your option offer warranty protection in exchange for a +fee. + + 2. You may modify your copy or copies of the Library or any portion +of it, thus forming a work based on the Library, and copy and +distribute such modifications or work under the terms of Section 1 +above, provided that you also meet all of these conditions: + + a) The modified work must itself be a software library. + + b) You must cause the files modified to carry prominent notices + stating that you changed the files and the date of any change. + + c) You must cause the whole of the work to be licensed at no + charge to all third parties under the terms of this License. + + d) If a facility in the modified Library refers to a function or a + table of data to be supplied by an application program that uses + the facility, other than as an argument passed when the facility + is invoked, then you must make a good faith effort to ensure that, + in the event an application does not supply such function or + table, the facility still operates, and performs whatever part of + its purpose remains meaningful. + + (For example, a function in a library to compute square roots has + a purpose that is entirely well-defined independent of the + application. Therefore, Subsection 2d requires that any + application-supplied function or table used by this function must + be optional: if the application does not supply it, the square + root function must still compute square roots.) + +These requirements apply to the modified work as a whole. If +identifiable sections of that work are not derived from the Library, +and can be reasonably considered independent and separate works in +themselves, then this License, and its terms, do not apply to those +sections when you distribute them as separate works. But when you +distribute the same sections as part of a whole which is a work based +on the Library, the distribution of the whole must be on the terms of +this License, whose permissions for other licensees extend to the +entire whole, and thus to each and every part regardless of who wrote +it. + +Thus, it is not the intent of this section to claim rights or contest +your rights to work written entirely by you; rather, the intent is to +exercise the right to control the distribution of derivative or +collective works based on the Library. + +In addition, mere aggregation of another work not based on the Library +with the Library (or with a work based on the Library) on a volume of +a storage or distribution medium does not bring the other work under +the scope of this License. + + 3. You may opt to apply the terms of the ordinary GNU General Public +License instead of this License to a given copy of the Library. To do +this, you must alter all the notices that refer to this License, so +that they refer to the ordinary GNU General Public License, version 2, +instead of to this License. (If a newer version than version 2 of the +ordinary GNU General Public License has appeared, then you can specify +that version instead if you wish.) Do not make any other change in +these notices. + + Once this change is made in a given copy, it is irreversible for +that copy, so the ordinary GNU General Public License applies to all +subsequent copies and derivative works made from that copy. + + This option is useful when you wish to copy part of the code of +the Library into a program that is not a library. + + 4. You may copy and distribute the Library (or a portion or +derivative of it, under Section 2) in object code or executable form +under the terms of Sections 1 and 2 above provided that you accompany +it with the complete corresponding machine-readable source code, which +must be distributed under the terms of Sections 1 and 2 above on a +medium customarily used for software interchange. + + If distribution of object code is made by offering access to copy +from a designated place, then offering equivalent access to copy the +source code from the same place satisfies the requirement to +distribute the source code, even though third parties are not +compelled to copy the source along with the object code. + + 5. A program that contains no derivative of any portion of the +Library, but is designed to work with the Library by being compiled or +linked with it, is called a "work that uses the Library". Such a +work, in isolation, is not a derivative work of the Library, and +therefore falls outside the scope of this License. + + However, linking a "work that uses the Library" with the Library +creates an executable that is a derivative of the Library (because it +contains portions of the Library), rather than a "work that uses the +library". The executable is therefore covered by this License. +Section 6 states terms for distribution of such executables. + + When a "work that uses the Library" uses material from a header file +that is part of the Library, the object code for the work may be a +derivative work of the Library even though the source code is not. +Whether this is true is especially significant if the work can be +linked without the Library, or if the work is itself a library. The +threshold for this to be true is not precisely defined by law. + + If such an object file uses only numerical parameters, data +structure layouts and accessors, and small macros and small inline +functions (ten lines or less in length), then the use of the object +file is unrestricted, regardless of whether it is legally a derivative +work. (Executables containing this object code plus portions of the +Library will still fall under Section 6.) + + Otherwise, if the work is a derivative of the Library, you may +distribute the object code for the work under the terms of Section 6. +Any executables containing that work also fall under Section 6, +whether or not they are linked directly with the Library itself. + + 6. As an exception to the Sections above, you may also combine or +link a "work that uses the Library" with the Library to produce a +work containing portions of the Library, and distribute that work +under terms of your choice, provided that the terms permit +modification of the work for the customer's own use and reverse +engineering for debugging such modifications. + + You must give prominent notice with each copy of the work that the +Library is used in it and that the Library and its use are covered by +this License. You must supply a copy of this License. If the work +during execution displays copyright notices, you must include the +copyright notice for the Library among them, as well as a reference +directing the user to the copy of this License. Also, you must do one +of these things: + + a) Accompany the work with the complete corresponding + machine-readable source code for the Library including whatever + changes were used in the work (which must be distributed under + Sections 1 and 2 above); and, if the work is an executable linked + with the Library, with the complete machine-readable "work that + uses the Library", as object code and/or source code, so that the + user can modify the Library and then relink to produce a modified + executable containing the modified Library. (It is understood + that the user who changes the contents of definitions files in the + Library will not necessarily be able to recompile the application + to use the modified definitions.) + + b) Use a suitable shared library mechanism for linking with the + Library. A suitable mechanism is one that (1) uses at run time a + copy of the library already present on the user's computer system, + rather than copying library functions into the executable, and (2) + will operate properly with a modified version of the library, if + the user installs one, as long as the modified version is + interface-compatible with the version that the work was made with. + + c) Accompany the work with a written offer, valid for at + least three years, to give the same user the materials + specified in Subsection 6a, above, for a charge no more + than the cost of performing this distribution. + + d) If distribution of the work is made by offering access to copy + from a designated place, offer equivalent access to copy the above + specified materials from the same place. + + e) Verify that the user has already received a copy of these + materials or that you have already sent this user a copy. + + For an executable, the required form of the "work that uses the +Library" must include any data and utility programs needed for +reproducing the executable from it. However, as a special exception, +the materials to be distributed need not include anything that is +normally distributed (in either source or binary form) with the major +components (compiler, kernel, and so on) of the operating system on +which the executable runs, unless that component itself accompanies +the executable. + + It may happen that this requirement contradicts the license +restrictions of other proprietary libraries that do not normally +accompany the operating system. Such a contradiction means you cannot +use both them and the Library together in an executable that you +distribute. + + 7. You may place library facilities that are a work based on the +Library side-by-side in a single library together with other library +facilities not covered by this License, and distribute such a combined +library, provided that the separate distribution of the work based on +the Library and of the other library facilities is otherwise +permitted, and provided that you do these two things: + + a) Accompany the combined library with a copy of the same work + based on the Library, uncombined with any other library + facilities. This must be distributed under the terms of the + Sections above. + + b) Give prominent notice with the combined library of the fact + that part of it is a work based on the Library, and explaining + where to find the accompanying uncombined form of the same work. + + 8. You may not copy, modify, sublicense, link with, or distribute +the Library except as expressly provided under this License. Any +attempt otherwise to copy, modify, sublicense, link with, or +distribute the Library is void, and will automatically terminate your +rights under this License. However, parties who have received copies, +or rights, from you under this License will not have their licenses +terminated so long as such parties remain in full compliance. + + 9. You are not required to accept this License, since you have not +signed it. However, nothing else grants you permission to modify or +distribute the Library or its derivative works. These actions are +prohibited by law if you do not accept this License. Therefore, by +modifying or distributing the Library (or any work based on the +Library), you indicate your acceptance of this License to do so, and +all its terms and conditions for copying, distributing or modifying +the Library or works based on it. + + 10. Each time you redistribute the Library (or any work based on the +Library), the recipient automatically receives a license from the +original licensor to copy, distribute, link with or modify the Library +subject to these terms and conditions. You may not impose any further +restrictions on the recipients' exercise of the rights granted herein. +You are not responsible for enforcing compliance by third parties with +this License. + + 11. If, as a consequence of a court judgment or allegation of patent +infringement or for any other reason (not limited to patent issues), +conditions are imposed on you (whether by court order, agreement or +otherwise) that contradict the conditions of this License, they do not +excuse you from the conditions of this License. If you cannot +distribute so as to satisfy simultaneously your obligations under this +License and any other pertinent obligations, then as a consequence you +may not distribute the Library at all. For example, if a patent +license would not permit royalty-free redistribution of the Library by +all those who receive copies directly or indirectly through you, then +the only way you could satisfy both it and this License would be to +refrain entirely from distribution of the Library. + +If any portion of this section is held invalid or unenforceable under any +particular circumstance, the balance of the section is intended to apply, +and the section as a whole is intended to apply in other circumstances. + +It is not the purpose of this section to induce you to infringe any +patents or other property right claims or to contest validity of any +such claims; this section has the sole purpose of protecting the +integrity of the free software distribution system which is +implemented by public license practices. Many people have made +generous contributions to the wide range of software distributed +through that system in reliance on consistent application of that +system; it is up to the author/donor to decide if he or she is willing +to distribute software through any other system and a licensee cannot +impose that choice. + +This section is intended to make thoroughly clear what is believed to +be a consequence of the rest of this License. + + 12. If the distribution and/or use of the Library is restricted in +certain countries either by patents or by copyrighted interfaces, the +original copyright holder who places the Library under this License may add +an explicit geographical distribution limitation excluding those countries, +so that distribution is permitted only in or among countries not thus +excluded. In such case, this License incorporates the limitation as if +written in the body of this License. + + 13. The Free Software Foundation may publish revised and/or new +versions of the Lesser General Public License from time to time. +Such new versions will be similar in spirit to the present version, +but may differ in detail to address new problems or concerns. + +Each version is given a distinguishing version number. If the Library +specifies a version number of this License which applies to it and +"any later version", you have the option of following the terms and +conditions either of that version or of any later version published by +the Free Software Foundation. If the Library does not specify a +license version number, you may choose any version ever published by +the Free Software Foundation. + + 14. If you wish to incorporate parts of the Library into other free +programs whose distribution conditions are incompatible with these, +write to the author to ask for permission. For software which is +copyrighted by the Free Software Foundation, write to the Free +Software Foundation; we sometimes make exceptions for this. Our +decision will be guided by the two goals of preserving the free status +of all derivatives of our free software and of promoting the sharing +and reuse of software generally. + + NO WARRANTY + + 15. BECAUSE THE LIBRARY IS LICENSED FREE OF CHARGE, THERE IS NO +WARRANTY FOR THE LIBRARY, TO THE EXTENT PERMITTED BY APPLICABLE LAW. +EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR +OTHER PARTIES PROVIDE THE LIBRARY "AS IS" WITHOUT WARRANTY OF ANY +KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR +PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE +LIBRARY IS WITH YOU. SHOULD THE LIBRARY PROVE DEFECTIVE, YOU ASSUME +THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION. + + 16. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN +WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY +AND/OR REDISTRIBUTE THE LIBRARY AS PERMITTED ABOVE, BE LIABLE TO YOU +FOR DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR +CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE +LIBRARY (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING +RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A +FAILURE OF THE LIBRARY TO OPERATE WITH ANY OTHER SOFTWARE), EVEN IF +SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH +DAMAGES. + + END OF TERMS AND CONDITIONS + + How to Apply These Terms to Your New Libraries + + If you develop a new library, and you want it to be of the greatest +possible use to the public, we recommend making it free software that +everyone can redistribute and change. You can do so by permitting +redistribution under these terms (or, alternatively, under the terms of the +ordinary General Public License). + + To apply these terms, attach the following notices to the library. It is +safest to attach them to the start of each source file to most effectively +convey the exclusion of warranty; and each file should have at least the +"copyright" line and a pointer to where the full notice is found. + + + Copyright (C) + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library; if not, write to the Free Software + Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA + +Also add information on how to contact you by electronic and paper mail. + +You should also get your employer (if you work as a programmer) or your +school, if any, to sign a "copyright disclaimer" for the library, if +necessary. Here is a sample; alter the names: + + Yoyodyne, Inc., hereby disclaims all copyright interest in the + library `Frob' (a library for tweaking knobs) written by James Random Hacker. + + , 1 April 1990 + Ty Coon, President of Vice + +That's all there is to it! + +-- From LICENSE-ASM.txt: + +Copyright (c) 2000-2005 INRIA, France Telecom +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + +1. Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright +notice, this list of conditions and the following disclaimer in the +documentation and/or other materials provided with the distribution. + +3. Neither the name of the copyright holders nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF +THE POSSIBILITY OF SUCH DAMAGE. + +-- From LICENSE-AppleJavaExtensions.txt: + +AppleJavaExtensions +v 1.2 + +This is a pluggable jar of stub classes representing the new Apple eAWT and eIO APIs for Java 1.4 on Mac OS X. The purpose of these stubs is to allow for compilation of eAWT- or eIO-referencing code on platforms other than Mac OS X. The jar file is enclosed in a zip archive for easy expansion on other platforms. + +These stubs are not intended for the runtime classpath on non-Mac platforms. Please see the OSXAdapter sample for how to write cross-platform code that uses eAWT. + +Disclaimer: IMPORTANT: This Apple software is supplied to you by Apple +Computer, Inc. ("Apple") in consideration of your agreement to the +following terms, and your use, installation, modification or +redistribution of this Apple software constitutes acceptance of these +terms. If you do not agree with these terms, please do not use, +install, modify or redistribute this Apple software. + +In consideration of your agreement to abide by the following terms, and +subject to these terms, Apple grants you a personal, non-exclusive +license, under Apple's copyrights in this original Apple software (the +"Apple Software"), to use, reproduce, modify and redistribute the Apple +Software, with or without modifications, in source and/or binary forms; +provided that if you redistribute the Apple Software in its entirety and +without modifications, you must retain this notice and the following +text and disclaimers in all such redistributions of the Apple Software. +Neither the name, trademarks, service marks or logos of Apple Computer, +Inc. may be used to endorse or promote products derived from the Apple +Software without specific prior written permission from Apple. Except +as expressly stated in this notice, no other rights or licenses, express +or implied, are granted by Apple herein, including but not limited to +any patent rights that may be infringed by your derivative works or by +other works in which the Apple Software may be incorporated. + +The Apple Software is provided by Apple on an "AS IS" basis. APPLE +MAKES NO WARRANTIES, EXPRESS OR IMPLIED, INCLUDING WITHOUT LIMITATION +THE IMPLIED WARRANTIES OF NON-INFRINGEMENT, MERCHANTABILITY AND FITNESS +FOR A PARTICULAR PURPOSE, REGARDING THE APPLE SOFTWARE OR ITS USE AND +OPERATION ALONE OR IN COMBINATION WITH YOUR PRODUCTS. + +IN NO EVENT SHALL APPLE BE LIABLE FOR ANY SPECIAL, INDIRECT, INCIDENTAL +OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) ARISING IN ANY WAY OUT OF THE USE, REPRODUCTION, +MODIFICATION AND/OR DISTRIBUTION OF THE APPLE SOFTWARE, HOWEVER CAUSED +AND WHETHER UNDER THEORY OF CONTRACT, TORT (INCLUDING NEGLIGENCE), +STRICT LIABILITY OR OTHERWISE, EVEN IF APPLE HAS BEEN ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. + +Copyright © 2003-2006 Apple Computer, Inc., All Rights Reserved + +-- From LICENSE-bcel.txt: + +/* + * Apache License + * Version 2.0, January 2004 + * http://www.apache.org/licenses/ + * + * TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + * + * 1. Definitions. + * + * "License" shall mean the terms and conditions for use, reproduction, + * and distribution as defined by Sections 1 through 9 of this document. + * + * "Licensor" shall mean the copyright owner or entity authorized by + * the copyright owner that is granting the License. + * + * "Legal Entity" shall mean the union of the acting entity and all + * other entities that control, are controlled by, or are under common + * control with that entity. For the purposes of this definition, + * "control" means (i) the power, direct or indirect, to cause the + * direction or management of such entity, whether by contract or + * otherwise, or (ii) ownership of fifty percent (50%) or more of the + * outstanding shares, or (iii) beneficial ownership of such entity. + * + * "You" (or "Your") shall mean an individual or Legal Entity + * exercising permissions granted by this License. + * + * "Source" form shall mean the preferred form for making modifications, + * including but not limited to software source code, documentation + * source, and configuration files. + * + * "Object" form shall mean any form resulting from mechanical + * transformation or translation of a Source form, including but + * not limited to compiled object code, generated documentation, + * and conversions to other media types. + * + * "Work" shall mean the work of authorship, whether in Source or + * Object form, made available under the License, as indicated by a + * copyright notice that is included in or attached to the work + * (an example is provided in the Appendix below). + * + * "Derivative Works" shall mean any work, whether in Source or Object + * form, that is based on (or derived from) the Work and for which the + * editorial revisions, annotations, elaborations, or other modifications + * represent, as a whole, an original work of authorship. For the purposes + * of this License, Derivative Works shall not include works that remain + * separable from, or merely link (or bind by name) to the interfaces of, + * the Work and Derivative Works thereof. + * + * "Contribution" shall mean any work of authorship, including + * the original version of the Work and any modifications or additions + * to that Work or Derivative Works thereof, that is intentionally + * submitted to Licensor for inclusion in the Work by the copyright owner + * or by an individual or Legal Entity authorized to submit on behalf of + * the copyright owner. For the purposes of this definition, "submitted" + * means any form of electronic, verbal, or written communication sent + * to the Licensor or its representatives, including but not limited to + * communication on electronic mailing lists, source code control systems, + * and issue tracking systems that are managed by, or on behalf of, the + * Licensor for the purpose of discussing and improving the Work, but + * excluding communication that is conspicuously marked or otherwise + * designated in writing by the copyright owner as "Not a Contribution." + * + * "Contributor" shall mean Licensor and any individual or Legal Entity + * on behalf of whom a Contribution has been received by Licensor and + * subsequently incorporated within the Work. + * + * 2. Grant of Copyright License. Subject to the terms and conditions of + * this License, each Contributor hereby grants to You a perpetual, + * worldwide, non-exclusive, no-charge, royalty-free, irrevocable + * copyright license to reproduce, prepare Derivative Works of, + * publicly display, publicly perform, sublicense, and distribute the + * Work and such Derivative Works in Source or Object form. + * + * 3. Grant of Patent License. Subject to the terms and conditions of + * this License, each Contributor hereby grants to You a perpetual, + * worldwide, non-exclusive, no-charge, royalty-free, irrevocable + * (except as stated in this section) patent license to make, have made, + * use, offer to sell, sell, import, and otherwise transfer the Work, + * where such license applies only to those patent claims licensable + * by such Contributor that are necessarily infringed by their + * Contribution(s) alone or by combination of their Contribution(s) + * with the Work to which such Contribution(s) was submitted. If You + * institute patent litigation against any entity (including a + * cross-claim or counterclaim in a lawsuit) alleging that the Work + * or a Contribution incorporated within the Work constitutes direct + * or contributory patent infringement, then any patent licenses + * granted to You under this License for that Work shall terminate + * as of the date such litigation is filed. + * + * 4. Redistribution. You may reproduce and distribute copies of the + * Work or Derivative Works thereof in any medium, with or without + * modifications, and in Source or Object form, provided that You + * meet the following conditions: + * + * (a) You must give any other recipients of the Work or + * Derivative Works a copy of this License; and + * + * (b) You must cause any modified files to carry prominent notices + * stating that You changed the files; and + * + * (c) You must retain, in the Source form of any Derivative Works + * that You distribute, all copyright, patent, trademark, and + * attribution notices from the Source form of the Work, + * excluding those notices that do not pertain to any part of + * the Derivative Works; and + * + * (d) If the Work includes a "NOTICE" text file as part of its + * distribution, then any Derivative Works that You distribute must + * include a readable copy of the attribution notices contained + * within such NOTICE file, excluding those notices that do not + * pertain to any part of the Derivative Works, in at least one + * of the following places: within a NOTICE text file distributed + * as part of the Derivative Works; within the Source form or + * documentation, if provided along with the Derivative Works; or, + * within a display generated by the Derivative Works, if and + * wherever such third-party notices normally appear. The contents + * of the NOTICE file are for informational purposes only and + * do not modify the License. You may add Your own attribution + * notices within Derivative Works that You distribute, alongside + * or as an addendum to the NOTICE text from the Work, provided + * that such additional attribution notices cannot be construed + * as modifying the License. + * + * You may add Your own copyright statement to Your modifications and + * may provide additional or different license terms and conditions + * for use, reproduction, or distribution of Your modifications, or + * for any such Derivative Works as a whole, provided Your use, + * reproduction, and distribution of the Work otherwise complies with + * the conditions stated in this License. + * + * 5. Submission of Contributions. Unless You explicitly state otherwise, + * any Contribution intentionally submitted for inclusion in the Work + * by You to the Licensor shall be under the terms and conditions of + * this License, without any additional terms or conditions. + * Notwithstanding the above, nothing herein shall supersede or modify + * the terms of any separate license agreement you may have executed + * with Licensor regarding such Contributions. + * + * 6. Trademarks. This License does not grant permission to use the trade + * names, trademarks, service marks, or product names of the Licensor, + * except as required for reasonable and customary use in describing the + * origin of the Work and reproducing the content of the NOTICE file. + * + * 7. Disclaimer of Warranty. Unless required by applicable law or + * agreed to in writing, Licensor provides the Work (and each + * Contributor provides its Contributions) on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied, including, without limitation, any warranties or conditions + * of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + * PARTICULAR PURPOSE. You are solely responsible for determining the + * appropriateness of using or redistributing the Work and assume any + * risks associated with Your exercise of permissions under this License. + * + * 8. Limitation of Liability. In no event and under no legal theory, + * whether in tort (including negligence), contract, or otherwise, + * unless required by applicable law (such as deliberate and grossly + * negligent acts) or agreed to in writing, shall any Contributor be + * liable to You for damages, including any direct, indirect, special, + * incidental, or consequential damages of any character arising as a + * result of this License or out of the use or inability to use the + * Work (including but not limited to damages for loss of goodwill, + * work stoppage, computer failure or malfunction, or any and all + * other commercial damages or losses), even if such Contributor + * has been advised of the possibility of such damages. + * + * 9. Accepting Warranty or Additional Liability. While redistributing + * the Work or Derivative Works thereof, You may choose to offer, + * and charge a fee for, acceptance of support, warranty, indemnity, + * or other liability obligations and/or rights consistent with this + * License. However, in accepting such obligations, You may act only + * on Your own behalf and on Your sole responsibility, not on behalf + * of any other Contributor, and only if You agree to indemnify, + * defend, and hold each Contributor harmless for any liability + * incurred by, or claims asserted against, such Contributor by reason + * of your accepting any such warranty or additional liability. + * + * END OF TERMS AND CONDITIONS + * + * APPENDIX: How to apply the Apache License to your work. + * + * To apply the Apache License to your work, attach the following + * boilerplate notice, with the fields enclosed by brackets "[]" + * replaced with your own identifying information. (Don't include + * the brackets!) The text should be enclosed in the appropriate + * comment syntax for the file format. We also recommend that a + * file or class name and description of purpose be included on the + * same "printed page" as the copyright notice for easier + * identification within third-party archives. + * + * Copyright [yyyy] [name of copyright owner] + * + * Licensed 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. + */ + +-- From LICENSE-commons-lang.txt: + + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + +TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + +1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + +2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + +3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + +4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + +5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + +6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + +7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + +8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + +9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + +END OF TERMS AND CONDITIONS + +APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + +Copyright [yyyy] [name of copyright owner] + +Licensed 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. + +-- From LICENSE-docbook.txt: + + + +-- From LICENSE-dom4j.txt: + +BSD style license + +Redistribution and use of this software and associated documentation +("Software"), with or without modification, are permitted provided that +the following conditions are met: + +1. Redistributions of source code must retain copyright statements +and notices. Redistributions must also contain a copy of this +document. + +2. Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following +disclaimer in the documentation and/or other materials provided +with the distribution. + +3. The name "DOM4J" must not be used to endorse or promote +products derived from this Software without prior written +permission of MetaStuff, Ltd. For written permission, please +contact dom4j-info@metastuff.com. + +4. Products derived from this Software may not be called "DOM4J" +nor may "DOM4J" appear in their names without prior written +permission of MetaStuff, Ltd. DOM4J is a registered trademark of +MetaStuff, Ltd. + +5. Due credit should be given to the DOM4J Project +(http://dom4j.org/). + +THIS SOFTWARE IS PROVIDED BY METASTUFF, LTD. AND CONTRIBUTORS ``AS IS'' +AND ANY EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, +THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR +PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL METASTUFF, LTD. OR ITS +CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Copyright 2001 (C) MetaStuff, Ltd. All Rights Reserved. + +-- From LICENSE-jFormatString.txt: + +The GNU General Public License (GPL) + +Version 2, June 1991 + +Copyright (C) 1989, 1991 Free Software Foundation, Inc. +59 Temple Place, Suite 330, Boston, MA 02111-1307 USA + +Everyone is permitted to copy and distribute verbatim copies of this license +document, but changing it is not allowed. + +Preamble + +The licenses for most software are designed to take away your freedom to share +and change it. By contrast, the GNU General Public License is intended to +guarantee your freedom to share and change free software--to make sure the +software is free for all its users. This General Public License applies to +most of the Free Software Foundation's software and to any other program whose +authors commit to using it. (Some other Free Software Foundation software is +covered by the GNU Library General Public License instead.) You can apply it to +your programs, too. + +When we speak of free software, we are referring to freedom, not price. Our +General Public Licenses are designed to make sure that you have the freedom to +distribute copies of free software (and charge for this service if you wish), +that you receive source code or can get it if you want it, that you can change +the software or use pieces of it in new free programs; and that you know you +can do these things. + +To protect your rights, we need to make restrictions that forbid anyone to deny +you these rights or to ask you to surrender the rights. These restrictions +translate to certain responsibilities for you if you distribute copies of the +software, or if you modify it. + +For example, if you distribute copies of such a program, whether gratis or for +a fee, you must give the recipients all the rights that you have. You must +make sure that they, too, receive or can get the source code. And you must +show them these terms so they know their rights. + +We protect your rights with two steps: (1) copyright the software, and (2) +offer you this license which gives you legal permission to copy, distribute +and/or modify the software. + +Also, for each author's protection and ours, we want to make certain that +everyone understands that there is no warranty for this free software. If the +software is modified by someone else and passed on, we want its recipients to +know that what they have is not the original, so that any problems introduced +by others will not reflect on the original authors' reputations. + +Finally, any free program is threatened constantly by software patents. We +wish to avoid the danger that redistributors of a free program will +individually obtain patent licenses, in effect making the program proprietary. +To prevent this, we have made it clear that any patent must be licensed for +everyone's free use or not licensed at all. + +The precise terms and conditions for copying, distribution and modification +follow. + +TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + +0. This License applies to any program or other work which contains a notice +placed by the copyright holder saying it may be distributed under the terms of +this General Public License. The "Program", below, refers to any such program +or work, and a "work based on the Program" means either the Program or any +derivative work under copyright law: that is to say, a work containing the +Program or a portion of it, either verbatim or with modifications and/or +translated into another language. (Hereinafter, translation is included +without limitation in the term "modification".) Each licensee is addressed as +"you". + +Activities other than copying, distribution and modification are not covered by +this License; they are outside its scope. The act of running the Program is +not restricted, and the output from the Program is covered only if its contents +constitute a work based on the Program (independent of having been made by +running the Program). Whether that is true depends on what the Program does. + +1. You may copy and distribute verbatim copies of the Program's source code as +you receive it, in any medium, provided that you conspicuously and +appropriately publish on each copy an appropriate copyright notice and +disclaimer of warranty; keep intact all the notices that refer to this License +and to the absence of any warranty; and give any other recipients of the +Program a copy of this License along with the Program. + +You may charge a fee for the physical act of transferring a copy, and you may +at your option offer warranty protection in exchange for a fee. + +2. You may modify your copy or copies of the Program or any portion of it, thus +forming a work based on the Program, and copy and distribute such modifications +or work under the terms of Section 1 above, provided that you also meet all of +these conditions: + +a) You must cause the modified files to carry prominent notices stating +that you changed the files and the date of any change. + +b) You must cause any work that you distribute or publish, that in whole or +in part contains or is derived from the Program or any part thereof, to be +licensed as a whole at no charge to all third parties under the terms of +this License. + +c) If the modified program normally reads commands interactively when run, +you must cause it, when started running for such interactive use in the +most ordinary way, to print or display an announcement including an +appropriate copyright notice and a notice that there is no warranty (or +else, saying that you provide a warranty) and that users may redistribute +the program under these conditions, and telling the user how to view a copy +of this License. (Exception: if the Program itself is interactive but does +not normally print such an announcement, your work based on the Program is +not required to print an announcement.) + +These requirements apply to the modified work as a whole. If identifiable +sections of that work are not derived from the Program, and can be reasonably +considered independent and separate works in themselves, then this License, and +its terms, do not apply to those sections when you distribute them as separate +works. But when you distribute the same sections as part of a whole which is a +work based on the Program, the distribution of the whole must be on the terms +of this License, whose permissions for other licensees extend to the entire +whole, and thus to each and every part regardless of who wrote it. + +Thus, it is not the intent of this section to claim rights or contest your +rights to work written entirely by you; rather, the intent is to exercise the +right to control the distribution of derivative or collective works based on +the Program. + +In addition, mere aggregation of another work not based on the Program with the +Program (or with a work based on the Program) on a volume of a storage or +distribution medium does not bring the other work under the scope of this +License. + +3. You may copy and distribute the Program (or a work based on it, under +Section 2) in object code or executable form under the terms of Sections 1 and +2 above provided that you also do one of the following: + +a) Accompany it with the complete corresponding machine-readable source +code, which must be distributed under the terms of Sections 1 and 2 above +on a medium customarily used for software interchange; or, + +b) Accompany it with a written offer, valid for at least three years, to +give any third party, for a charge no more than your cost of physically +performing source distribution, a complete machine-readable copy of the +corresponding source code, to be distributed under the terms of Sections 1 +and 2 above on a medium customarily used for software interchange; or, + +c) Accompany it with the information you received as to the offer to +distribute corresponding source code. (This alternative is allowed only +for noncommercial distribution and only if you received the program in +object code or executable form with such an offer, in accord with +Subsection b above.) + +The source code for a work means the preferred form of the work for making +modifications to it. For an executable work, complete source code means all +the source code for all modules it contains, plus any associated interface +definition files, plus the scripts used to control compilation and installation +of the executable. However, as a special exception, the source code +distributed need not include anything that is normally distributed (in either +source or binary form) with the major components (compiler, kernel, and so on) +of the operating system on which the executable runs, unless that component +itself accompanies the executable. + +If distribution of executable or object code is made by offering access to copy +from a designated place, then offering equivalent access to copy the source +code from the same place counts as distribution of the source code, even though +third parties are not compelled to copy the source along with the object code. + +4. You may not copy, modify, sublicense, or distribute the Program except as +expressly provided under this License. Any attempt otherwise to copy, modify, +sublicense or distribute the Program is void, and will automatically terminate +your rights under this License. However, parties who have received copies, or +rights, from you under this License will not have their licenses terminated so +long as such parties remain in full compliance. + +5. You are not required to accept this License, since you have not signed it. +However, nothing else grants you permission to modify or distribute the Program +or its derivative works. These actions are prohibited by law if you do not +accept this License. Therefore, by modifying or distributing the Program (or +any work based on the Program), you indicate your acceptance of this License to +do so, and all its terms and conditions for copying, distributing or modifying +the Program or works based on it. + +6. Each time you redistribute the Program (or any work based on the Program), +the recipient automatically receives a license from the original licensor to +copy, distribute or modify the Program subject to these terms and conditions. +You may not impose any further restrictions on the recipients' exercise of the +rights granted herein. You are not responsible for enforcing compliance by +third parties to this License. + +7. If, as a consequence of a court judgment or allegation of patent +infringement or for any other reason (not limited to patent issues), conditions +are imposed on you (whether by court order, agreement or otherwise) that +contradict the conditions of this License, they do not excuse you from the +conditions of this License. If you cannot distribute so as to satisfy +simultaneously your obligations under this License and any other pertinent +obligations, then as a consequence you may not distribute the Program at all. +For example, if a patent license would not permit royalty-free redistribution +of the Program by all those who receive copies directly or indirectly through +you, then the only way you could satisfy both it and this License would be to +refrain entirely from distribution of the Program. + +If any portion of this section is held invalid or unenforceable under any +particular circumstance, the balance of the section is intended to apply and +the section as a whole is intended to apply in other circumstances. + +It is not the purpose of this section to induce you to infringe any patents or +other property right claims or to contest validity of any such claims; this +section has the sole purpose of protecting the integrity of the free software +distribution system, which is implemented by public license practices. Many +people have made generous contributions to the wide range of software +distributed through that system in reliance on consistent application of that +system; it is up to the author/donor to decide if he or she is willing to +distribute software through any other system and a licensee cannot impose that +choice. + +This section is intended to make thoroughly clear what is believed to be a +consequence of the rest of this License. + +8. If the distribution and/or use of the Program is restricted in certain +countries either by patents or by copyrighted interfaces, the original +copyright holder who places the Program under this License may add an explicit +geographical distribution limitation excluding those countries, so that +distribution is permitted only in or among countries not thus excluded. In +such case, this License incorporates the limitation as if written in the body +of this License. + +9. The Free Software Foundation may publish revised and/or new versions of the +General Public License from time to time. Such new versions will be similar in +spirit to the present version, but may differ in detail to address new problems +or concerns. + +Each version is given a distinguishing version number. If the Program +specifies a version number of this License which applies to it and "any later +version", you have the option of following the terms and conditions either of +that version or of any later version published by the Free Software Foundation. +If the Program does not specify a version number of this License, you may +choose any version ever published by the Free Software Foundation. + +10. If you wish to incorporate parts of the Program into other free programs +whose distribution conditions are different, write to the author to ask for +permission. For software which is copyrighted by the Free Software Foundation, +write to the Free Software Foundation; we sometimes make exceptions for this. +Our decision will be guided by the two goals of preserving the free status of +all derivatives of our free software and of promoting the sharing and reuse of +software generally. + +NO WARRANTY + +11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY FOR +THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN OTHERWISE +STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES PROVIDE THE +PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND +FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND +PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, +YOU ASSUME THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION. + +12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING WILL +ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR REDISTRIBUTE THE +PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY +GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR +INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA +BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A +FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER +OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + +END OF TERMS AND CONDITIONS + +How to Apply These Terms to Your New Programs + +If you develop a new program, and you want it to be of the greatest possible +use to the public, the best way to achieve this is to make it free software +which everyone can redistribute and change under these terms. + +To do so, attach the following notices to the program. It is safest to attach +them to the start of each source file to most effectively convey the exclusion +of warranty; and each file should have at least the "copyright" line and a +pointer to where the full notice is found. + +One line to give the program's name and a brief idea of what it does. + +Copyright (C) + +This program is free software; you can redistribute it and/or modify it +under the terms of the GNU General Public License as published by the Free +Software Foundation; either version 2 of the License, or (at your option) +any later version. + +This program is distributed in the hope that it will be useful, but WITHOUT +ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or +FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License for +more details. + +You should have received a copy of the GNU General Public License along +with this program; if not, write to the Free Software Foundation, Inc., 59 +Temple Place, Suite 330, Boston, MA 02111-1307 USA + +Also add information on how to contact you by electronic and paper mail. + +If the program is interactive, make it output a short notice like this when it +starts in an interactive mode: + +Gnomovision version 69, Copyright (C) year name of author Gnomovision comes +with ABSOLUTELY NO WARRANTY; for details type 'show w'. This is free +software, and you are welcome to redistribute it under certain conditions; +type 'show c' for details. + +The hypothetical commands 'show w' and 'show c' should show the appropriate +parts of the General Public License. Of course, the commands you use may be +called something other than 'show w' and 'show c'; they could even be +mouse-clicks or menu items--whatever suits your program. + +You should also get your employer (if you work as a programmer) or your school, +if any, to sign a "copyright disclaimer" for the program, if necessary. Here +is a sample; alter the names: + +Yoyodyne, Inc., hereby disclaims all copyright interest in the program +'Gnomovision' (which makes passes at compilers) written by James Hacker. + +signature of Ty Coon, 1 April 1989 + +Ty Coon, President of Vice + +This General Public License does not permit incorporating your program into +proprietary programs. If your program is a subroutine library, you may +consider it more useful to permit linking proprietary applications with the +library. If this is what you want to do, use the GNU Library General Public +License instead of this License. + + +"CLASSPATH" EXCEPTION TO THE GPL + +Certain source files distributed by Sun Microsystems, Inc. are subject to +the following clarification and special exception to the GPL, but only where +Sun has expressly included in the particular source file's header the words +"Sun designates this particular file as subject to the "Classpath" exception +as provided by Sun in the LICENSE file that accompanied this code." + +Linking this library statically or dynamically with other modules is making +a combined work based on this library. Thus, the terms and conditions of +the GNU General Public License cover the whole combination. + +As a special exception, the copyright holders of this library give you +permission to link this library with independent modules to produce an +executable, regardless of the license terms of these independent modules, +and to copy and distribute the resulting executable under terms of your +choice, provided that you also meet, for each linked independent module, +the terms and conditions of the license of that module. An independent +module is a module which is not derived from or based on this library. If +you modify this library, you may extend this exception to your version of +the library, but you are not obligated to do so. If you do not wish to do +so, delete this exception statement from your version. + +-- From LICENSE-jaxen.txt: + +/* +$Id: LICENSE-jaxen.txt,v 1.1 2008/06/18 18:54:23 wpugh Exp $ + +Copyright 2003-2006 The Werken Company. All Rights Reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + +* Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright +notice, this list of conditions and the following disclaimer in the +documentation and/or other materials provided with the distribution. + +* Neither the name of the Jaxen Project nor the names of its +contributors may be used to endorse or promote products derived +from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS +IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED +TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A +PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER +OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +*/ + +-- From LICENSE-jcip.txt: + +The Java code in the package net.jcip.annotations +is copyright (c) 2005 Brian Goetz +and is released under the Creative Commons Attribution License +(http://creativecommons.org/licenses/by/2.5) +Official home: http://www.jcip.net + +-- From LICENSE-jdepend.txt: + +The jdepend library (lib/jdepend-2.9.jar) is distributed under the terms of the BSD license: +http://www.clarkware.com/software/JDepend.html#license +http://www.clarkware.com/software/license.txt + +Copyright (C) 2001 Clarkware Consulting, Inc. +All Rights Reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + +1. Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright +notice, this list of conditions and the following disclaimer in the +documentation and/or other materials provided with the distribution. + +3. Neither the name of Clarkware Consulting, Inc. nor the names of its +contributors may be used to endorse or promote products derived +from this software without prior written permission. For written +permission, please contact clarkware@clarkware.com. + +THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND +FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL +CLARKWARE CONSULTING OR ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, +OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, +EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-- From LICENSE-jsr305.txt: + +The JSR-305 reference implementation (lib/jsr305.jar) is +distributed under the terms of the New BSD license: + +http://www.opensource.org/licenses/bsd-license.php + +See the JSR-305 home page for more information: + +http://code.google.com/p/jsr-305/ + +----- + +The following software may be included in this product: floatingdecimal. The source code is available at https://github.com/airlift/floatingdecimal. You may also request a copy of the source code by sending a request to opensource@fb.com. This software contains the following license and notice below: + +/* +* Copyright (c) 1996, 2011, Oracle and/or its affiliates. All rights reserved. +* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. +* +* This code is free software; you can redistribute it and/or modify it +* under the terms of the GNU General Public License version 2 only, as +* published by the Free Software Foundation. Oracle designates this +* particular file as subject to the "Classpath" exception as provided +* by Oracle in the LICENSE file that accompanied this code. +* +* This code is distributed in the hope that it will be useful, but WITHOUT +* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or +* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License +* version 2 for more details (a copy is included in the LICENSE file that +* accompanied this code). +* +* You should have received a copy of the GNU General Public License version +* 2 along with this work; if not, write to the Free Software Foundation, +* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA. +* +* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA +* or visit www.oracle.com if you need additional information or have any +* questions. +*/ + + GNU GENERAL PUBLIC LICENSE + Version 2, June 1991 + + Copyright (C) 1989, 1991 Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA + Everyone is permitted to copy and distribute verbatim copies + of this license document, but changing it is not allowed. + + Preamble + + The licenses for most software are designed to take away your +freedom to share and change it. By contrast, the GNU General Public +License is intended to guarantee your freedom to share and change free +software--to make sure the software is free for all its users. This +General Public License applies to most of the Free Software +Foundation's software and to any other program whose authors commit to +using it. (Some other Free Software Foundation software is covered by +the GNU Lesser General Public License instead.) You can apply it to +your programs, too. + + When we speak of free software, we are referring to freedom, not +price. Our General Public Licenses are designed to make sure that you +have the freedom to distribute copies of free software (and charge for +this service if you wish), that you receive source code or can get it +if you want it, that you can change the software or use pieces of it +in new free programs; and that you know you can do these things. + + To protect your rights, we need to make restrictions that forbid +anyone to deny you these rights or to ask you to surrender the rights. +These restrictions translate to certain responsibilities for you if you +distribute copies of the software, or if you modify it. + + For example, if you distribute copies of such a program, whether +gratis or for a fee, you must give the recipients all the rights that +you have. You must make sure that they, too, receive or can get the +source code. And you must show them these terms so they know their +rights. + + We protect your rights with two steps: (1) copyright the software, and +(2) offer you this license which gives you legal permission to copy, +distribute and/or modify the software. + + Also, for each author's protection and ours, we want to make certain +that everyone understands that there is no warranty for this free +software. If the software is modified by someone else and passed on, we +want its recipients to know that what they have is not the original, so +that any problems introduced by others will not reflect on the original +authors' reputations. + + Finally, any free program is threatened constantly by software +patents. We wish to avoid the danger that redistributors of a free +program will individually obtain patent licenses, in effect making the +program proprietary. To prevent this, we have made it clear that any +patent must be licensed for everyone's free use or not licensed at all. + + The precise terms and conditions for copying, distribution and +modification follow. + + GNU GENERAL PUBLIC LICENSE + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. This License applies to any program or other work which contains +a notice placed by the copyright holder saying it may be distributed +under the terms of this General Public License. The "Program", below, +refers to any such program or work, and a "work based on the Program" +means either the Program or any derivative work under copyright law: +that is to say, a work containing the Program or a portion of it, +either verbatim or with modifications and/or translated into another +language. (Hereinafter, translation is included without limitation in +the term "modification".) Each licensee is addressed as "you". + +Activities other than copying, distribution and modification are not +covered by this License; they are outside its scope. The act of +running the Program is not restricted, and the output from the Program +is covered only if its contents constitute a work based on the +Program (independent of having been made by running the Program). +Whether that is true depends on what the Program does. + + 1. You may copy and distribute verbatim copies of the Program's +source code as you receive it, in any medium, provided that you +conspicuously and appropriately publish on each copy an appropriate +copyright notice and disclaimer of warranty; keep intact all the +notices that refer to this License and to the absence of any warranty; +and give any other recipients of the Program a copy of this License +along with the Program. + +You may charge a fee for the physical act of transferring a copy, and +you may at your option offer warranty protection in exchange for a fee. + + 2. You may modify your copy or copies of the Program or any portion +of it, thus forming a work based on the Program, and copy and +distribute such modifications or work under the terms of Section 1 +above, provided that you also meet all of these conditions: + + a) You must cause the modified files to carry prominent notices + stating that you changed the files and the date of any change. + + b) You must cause any work that you distribute or publish, that in + whole or in part contains or is derived from the Program or any + part thereof, to be licensed as a whole at no charge to all third + parties under the terms of this License. + + c) If the modified program normally reads commands interactively + when run, you must cause it, when started running for such + interactive use in the most ordinary way, to print or display an + announcement including an appropriate copyright notice and a + notice that there is no warranty (or else, saying that you provide + a warranty) and that users may redistribute the program under + these conditions, and telling the user how to view a copy of this + License. (Exception: if the Program itself is interactive but + does not normally print such an announcement, your work based on + the Program is not required to print an announcement.) + +These requirements apply to the modified work as a whole. If +identifiable sections of that work are not derived from the Program, +and can be reasonably considered independent and separate works in +themselves, then this License, and its terms, do not apply to those +sections when you distribute them as separate works. But when you +distribute the same sections as part of a whole which is a work based +on the Program, the distribution of the whole must be on the terms of +this License, whose permissions for other licensees extend to the +entire whole, and thus to each and every part regardless of who wrote it. + +Thus, it is not the intent of this section to claim rights or contest +your rights to work written entirely by you; rather, the intent is to +exercise the right to control the distribution of derivative or +collective works based on the Program. + +In addition, mere aggregation of another work not based on the Program +with the Program (or with a work based on the Program) on a volume of +a storage or distribution medium does not bring the other work under +the scope of this License. + + 3. You may copy and distribute the Program (or a work based on it, +under Section 2) in object code or executable form under the terms of +Sections 1 and 2 above provided that you also do one of the following: + + a) Accompany it with the complete corresponding machine-readable + source code, which must be distributed under the terms of Sections + 1 and 2 above on a medium customarily used for software interchange; or, + + b) Accompany it with a written offer, valid for at least three + years, to give any third party, for a charge no more than your + cost of physically performing source distribution, a complete + machine-readable copy of the corresponding source code, to be + distributed under the terms of Sections 1 and 2 above on a medium + customarily used for software interchange; or, + + c) Accompany it with the information you received as to the offer + to distribute corresponding source code. (This alternative is + allowed only for noncommercial distribution and only if you + received the program in object code or executable form with such + an offer, in accord with Subsection b above.) + +The source code for a work means the preferred form of the work for +making modifications to it. For an executable work, complete source +code means all the source code for all modules it contains, plus any +associated interface definition files, plus the scripts used to +control compilation and installation of the executable. However, as a +special exception, the source code distributed need not include +anything that is normally distributed (in either source or binary +form) with the major components (compiler, kernel, and so on) of the +operating system on which the executable runs, unless that component +itself accompanies the executable. + +If distribution of executable or object code is made by offering +access to copy from a designated place, then offering equivalent +access to copy the source code from the same place counts as +distribution of the source code, even though third parties are not +compelled to copy the source along with the object code. + + 4. You may not copy, modify, sublicense, or distribute the Program +except as expressly provided under this License. Any attempt +otherwise to copy, modify, sublicense or distribute the Program is +void, and will automatically terminate your rights under this License. +However, parties who have received copies, or rights, from you under +this License will not have their licenses terminated so long as such +parties remain in full compliance. + + 5. You are not required to accept this License, since you have not +signed it. However, nothing else grants you permission to modify or +distribute the Program or its derivative works. These actions are +prohibited by law if you do not accept this License. Therefore, by +modifying or distributing the Program (or any work based on the +Program), you indicate your acceptance of this License to do so, and +all its terms and conditions for copying, distributing or modifying +the Program or works based on it. + + 6. Each time you redistribute the Program (or any work based on the +Program), the recipient automatically receives a license from the +original licensor to copy, distribute or modify the Program subject to +these terms and conditions. You may not impose any further +restrictions on the recipients' exercise of the rights granted herein. +You are not responsible for enforcing compliance by third parties to +this License. + + 7. If, as a consequence of a court judgment or allegation of patent +infringement or for any other reason (not limited to patent issues), +conditions are imposed on you (whether by court order, agreement or +otherwise) that contradict the conditions of this License, they do not +excuse you from the conditions of this License. If you cannot +distribute so as to satisfy simultaneously your obligations under this +License and any other pertinent obligations, then as a consequence you +may not distribute the Program at all. For example, if a patent +license would not permit royalty-free redistribution of the Program by +all those who receive copies directly or indirectly through you, then +the only way you could satisfy both it and this License would be to +refrain entirely from distribution of the Program. + +If any portion of this section is held invalid or unenforceable under +any particular circumstance, the balance of the section is intended to +apply and the section as a whole is intended to apply in other +circumstances. + +It is not the purpose of this section to induce you to infringe any +patents or other property right claims or to contest validity of any +such claims; this section has the sole purpose of protecting the +integrity of the free software distribution system, which is +implemented by public license practices. Many people have made +generous contributions to the wide range of software distributed +through that system in reliance on consistent application of that +system; it is up to the author/donor to decide if he or she is willing +to distribute software through any other system and a licensee cannot +impose that choice. + +This section is intended to make thoroughly clear what is believed to +be a consequence of the rest of this License. + + 8. If the distribution and/or use of the Program is restricted in +certain countries either by patents or by copyrighted interfaces, the +original copyright holder who places the Program under this License +may add an explicit geographical distribution limitation excluding +those countries, so that distribution is permitted only in or among +countries not thus excluded. In such case, this License incorporates +the limitation as if written in the body of this License. + + 9. The Free Software Foundation may publish revised and/or new versions +of the General Public License from time to time. Such new versions will +be similar in spirit to the present version, but may differ in detail to +address new problems or concerns. + +Each version is given a distinguishing version number. If the Program +specifies a version number of this License which applies to it and "any +later version", you have the option of following the terms and conditions +either of that version or of any later version published by the Free +Software Foundation. If the Program does not specify a version number of +this License, you may choose any version ever published by the Free Software +Foundation. + + 10. If you wish to incorporate parts of the Program into other free +programs whose distribution conditions are different, write to the author +to ask for permission. For software which is copyrighted by the Free +Software Foundation, write to the Free Software Foundation; we sometimes +make exceptions for this. Our decision will be guided by the two goals +of preserving the free status of all derivatives of our free software and +of promoting the sharing and reuse of software generally. + + NO WARRANTY + + 11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY +FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN +OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES +PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED +OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS +TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE +PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL NECESSARY SERVICING, +REPAIR OR CORRECTION. + + 12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING +WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR +REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, +INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING +OUT OF THE USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED +TO LOSS OF DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY +YOU OR THIRD PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER +PROGRAMS), EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE +POSSIBILITY OF SUCH DAMAGES. + + END OF TERMS AND CONDITIONS + + How to Apply These Terms to Your New Programs + + If you develop a new program, and you want it to be of the greatest +possible use to the public, the best way to achieve this is to make it +free software which everyone can redistribute and change under these terms. + + To do so, attach the following notices to the program. It is safest +to attach them to the start of each source file to most effectively +convey the exclusion of warranty; and each file should have at least +the "copyright" line and a pointer to where the full notice is found. + + + Copyright (C) + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. + +Also add information on how to contact you by electronic and paper mail. + +If the program is interactive, make it output a short notice like this +when it starts in an interactive mode: + + Gnomovision version 69, Copyright (C) year name of author + Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type `show w'. + This is free software, and you are welcome to redistribute it + under certain conditions; type `show c' for details. + +The hypothetical commands `show w' and `show c' should show the appropriate +parts of the General Public License. Of course, the commands you use may +be called something other than `show w' and `show c'; they could even be +mouse-clicks or menu items--whatever suits your program. + +You should also get your employer (if you work as a programmer) or your +school, if any, to sign a "copyright disclaimer" for the program, if +necessary. Here is a sample; alter the names: + + Yoyodyne, Inc., hereby disclaims all copyright interest in the program + `Gnomovision' (which makes passes at compilers) written by James Hacker. + + , 1 April 1989 + Ty Coon, President of Vice + +This General Public License does not permit incorporating your program into +proprietary programs. If your program is a subroutine library, you may +consider it more useful to permit linking proprietary applications with the +library. If this is what you want to do, use the GNU Lesser General +Public License instead of this License. + +----- + +The following software may be included in this product: H2 Database Engine. The source code is available at http://www.h2database.com/html/download.html. You may also request a copy of the source code by sending a request to opensource@fb.com. This software contains the following license and notice below: + +Eclipse Public License - v 1.0 + +THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. + +1. DEFINITIONS + +"Contribution" means: + +a) in the case of the initial Contributor, the initial code and documentation distributed under this Agreement, and + +b) in the case of each subsequent Contributor: + +i) changes to the Program, and + +ii) additions to the Program; + +where such changes and/or additions to the Program originate from and are distributed by that particular Contributor. A Contribution 'originates' from a Contributor if it was added to the Program by such Contributor itself or anyone acting on such Contributor's behalf. Contributions do not include additions to the Program which: (i) are separate modules of software distributed in conjunction with the Program under their own license agreement, and (ii) are not derivative works of the Program. + +"Contributor" means any person or entity that distributes the Program. + +"Licensed Patents" mean patent claims licensable by a Contributor which are necessarily infringed by the use or sale of its Contribution alone or when combined with the Program. + +"Program" means the Contributions distributed in accordance with this Agreement. + +"Recipient" means anyone who receives the Program under this Agreement, including all Contributors. + +2. GRANT OF RIGHTS + +a) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free copyright license to reproduce, prepare derivative works of, publicly display, publicly perform, distribute and sublicense the Contribution of such Contributor, if any, and such derivative works, in source code and object code form. + +b) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed Patents to make, use, sell, offer to sell, import and otherwise transfer the Contribution of such Contributor, if any, in source code and object code form. This patent license shall apply to the combination of the Contribution and the Program if, at the time the Contribution is added by the Contributor, such addition of the Contribution causes such combination to be covered by the Licensed Patents. The patent license shall not apply to any other combinations which include the Contribution. No hardware per se is licensed hereunder. + +c) Recipient understands that although each Contributor grants the licenses to its Contributions set forth herein, no assurances are provided by any Contributor that the Program does not infringe the patent or other intellectual property rights of any other entity. Each Contributor disclaims any liability to Recipient for claims brought by any other entity based on infringement of intellectual property rights or otherwise. As a condition to exercising the rights and licenses granted hereunder, each Recipient hereby assumes sole responsibility to secure any other intellectual property rights needed, if any. For example, if a third party patent license is required to allow Recipient to distribute the Program, it is Recipient's responsibility to acquire that license before distributing the Program. + +d) Each Contributor represents that to its knowledge it has sufficient copyright rights in its Contribution, if any, to grant the copyright license set forth in this Agreement. + +3. REQUIREMENTS + +A Contributor may choose to distribute the Program in object code form under its own license agreement, provided that: + +a) it complies with the terms and conditions of this Agreement; and + +b) its license agreement: + +i) effectively disclaims on behalf of all Contributors all warranties and conditions, express and implied, including warranties or conditions of title and non-infringement, and implied warranties or conditions of merchantability and fitness for a particular purpose; + +ii) effectively excludes on behalf of all Contributors all liability for damages, including direct, indirect, special, incidental and consequential damages, such as lost profits; + +iii) states that any provisions which differ from this Agreement are offered by that Contributor alone and not by any other party; and + +iv) states that source code for the Program is available from such Contributor, and informs licensees how to obtain it in a reasonable manner on or through a medium customarily used for software exchange. + +When the Program is made available in source code form: + +a) it must be made available under this Agreement; and + +b) a copy of this Agreement must be included with each copy of the Program. + +Contributors may not remove or alter any copyright notices contained within the Program. + +Each Contributor must identify itself as the originator of its Contribution, if any, in a manner that reasonably allows subsequent Recipients to identify the originator of the Contribution. + +4. COMMERCIAL DISTRIBUTION + +Commercial distributors of software may accept certain responsibilities with respect to end users, business partners and the like. While this license is intended to facilitate the commercial use of the Program, the Contributor who includes the Program in a commercial product offering should do so in a manner which does not create potential liability for other Contributors. Therefore, if a Contributor includes the Program in a commercial product offering, such Contributor ("Commercial Contributor") hereby agrees to defend and indemnify every other Contributor ("Indemnified Contributor") against any losses, damages and costs (collectively "Losses") arising from claims, lawsuits and other legal actions brought by a third party against the Indemnified Contributor to the extent caused by the acts or omissions of such Commercial Contributor in connection with its distribution of the Program in a commercial product offering. The obligations in this section do not apply to any claims or Losses relating to any actual or alleged intellectual property infringement. In order to qualify, an Indemnified Contributor must: a) promptly notify the Commercial Contributor in writing of such claim, and b) allow the Commercial Contributor to control, and cooperate with the Commercial Contributor in, the defense and any related settlement negotiations. The Indemnified Contributor may participate in any such claim at its own expense. + +For example, a Contributor might include the Program in a commercial product offering, Product X. That Contributor is then a Commercial Contributor. If that Commercial Contributor then makes performance claims, or offers warranties related to Product X, those performance claims and warranties are such Commercial Contributor's responsibility alone. Under this section, the Commercial Contributor would have to defend claims against the other Contributors related to those performance claims and warranties, and if a court requires any other Contributor to pay any damages as a result, the Commercial Contributor must pay those damages. + +5. NO WARRANTY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each Recipient is solely responsible for determining the appropriateness of using and distributing the Program and assumes all risks associated with its exercise of rights under this Agreement , including but not limited to the risks and costs of program errors, compliance with applicable laws, damage to or loss of data, programs or equipment, and unavailability or interruption of operations. + +6. DISCLAIMER OF LIABILITY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + +7. GENERAL + +If any provision of this Agreement is invalid or unenforceable under applicable law, it shall not affect the validity or enforceability of the remainder of the terms of this Agreement, and without further action by the parties hereto, such provision shall be reformed to the minimum extent necessary to make such provision valid and enforceable. + +If Recipient institutes patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Program itself (excluding combinations of the Program with other software or hardware) infringes such Recipient's patent(s), then such Recipient's rights granted under Section 2(b) shall terminate as of the date such litigation is filed. + +All Recipient's rights under this Agreement shall terminate if it fails to comply with any of the material terms or conditions of this Agreement and does not cure such failure in a reasonable period of time after becoming aware of such noncompliance. If all Recipient's rights under this Agreement terminate, Recipient agrees to cease use and distribution of the Program as soon as reasonably practicable. However, Recipient's obligations under this Agreement and any licenses granted by Recipient relating to the Program shall continue and survive. + +Everyone is permitted to copy and distribute copies of this Agreement, but in order to avoid inconsistency the Agreement is copyrighted and may only be modified in the following manner. The Agreement Steward reserves the right to publish new versions (including revisions) of this Agreement from time to time. No one other than the Agreement Steward has the right to modify this Agreement. The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation may assign the responsibility to serve as the Agreement Steward to a suitable separate entity. Each new version of the Agreement will be given a distinguishing version number. The Program (including Contributions) may always be distributed subject to the version of the Agreement under which it was received. In addition, after a new version of the Agreement is published, Contributor may elect to distribute the Program (including its Contributions) under the new version. Except as expressly stated in Sections 2(a) and 2(b) above, Recipient receives no rights or licenses to the intellectual property of any Contributor under this Agreement, whether expressly, by implication, estoppel or otherwise. All rights in the Program not expressly granted under this Agreement are reserved. + +This Agreement is governed by the laws of the State of New York and the intellectual property laws of the United States of America. No party to this Agreement will bring a legal action under this Agreement more than one year after the cause of action arose. Each party waives its rights to a jury trial in any resulting litigation. + +----- + +The following software may be included in this product: java servlet api. The source code is available at https://java.net/projects/servlet-spec/. You may also request a copy of the source code by sending a request to opensource@fb.com. This software contains the following license and notice below: + +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL - Version 1.1) +1. Definitions. + + 1.1. “Contributor” means each individual or entity that creates or contributes to the creation of Modifications. + + 1.2. “Contributor Version” means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor. + + 1.3. “Covered Software” means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof. + + 1.4. “Executable” means the Covered Software in any form other than Source Code. + + 1.5. “Initial Developer” means the individual or entity that first makes Original Software available under this License. + + 1.6. “Larger Work” means a work which combines Covered Software or portions thereof with code not governed by the terms of this License. + + 1.7. “License” means this document. + + 1.8. “Licensable” means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein. + + 1.9. “Modifications” means the Source Code and Executable form of any of the following: + + A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; + + B. Any new file that contains any part of the Original Software or previous Modification; or + + C. Any new file that is contributed or otherwise made available under the terms of this License. + + 1.10. “Original Software” means the Source Code and Executable form of computer software code that is originally released under this License. + + 1.11. “Patent Claims” means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor. + + 1.12. “Source Code” means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code. + + 1.13. “You” (or “Your”) means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, “You” includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, “control” means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity. + +2. License Grants. + + 2.1. The Initial Developer Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof). + + (c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License. + + (d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices. + + 2.2. Contributor Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination). + + (c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party. + + (d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor. + +3. Distribution Obligations. + + 3.1. Availability of Source Code. + + Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange. + + 3.2. Modifications. + + The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License. + + 3.3. Required Notices. + + You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer. + + 3.4. Application of Additional Terms. + + You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients’ rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer. + + 3.5. Distribution of Executable Versions. + + You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient’s rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. + + 3.6. Larger Works. + + You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software. + +4. Versions of the License. + + 4.1. New Versions. + + Oracle is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License. + + 4.2. Effect of New Versions. + + You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward. + + 4.3. Modified Versions. + + When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License. + +5. DISCLAIMER OF WARRANTY. + +COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN “AS IS” BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. +6. TERMINATION. + + 6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. + + 6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as “Participant”) alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant. + + 6.3. If You assert a patent infringement claim against Participant alleging that the Participant Software directly or indirectly infringes any patent where such claim is resolved (such as by license or settlement) prior to the initiation of patent infringement litigation, then the reasonable value of the licenses granted by such Participant under Sections 2.1 or 2.2 shall be taken into account in determining the amount or value of any payment or license. + + 6.4. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. + +UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY’S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. +8. U.S. GOVERNMENT END USERS. + +The Covered Software is a “commercial item,” as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of “commercial computer software” (as that term is defined at 48 C.F.R. § 252.227-7014(a)(1)) and “commercial computer software documentation” as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License. +9. MISCELLANEOUS. + +This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction’s conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys’ fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software. +10. RESPONSIBILITY FOR CLAIMS. + +As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability. + +----- + +The following software may be included in this product: JCodings. This software contains the following license and notice below: + +Permission is hereby granted, free of charge, to any person obtaining a copy of +this software and associated documentation files (the "Software"), to deal in +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +of the Software, and to permit persons to whom the Software is furnished to do +so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +----- + +The following software may be included in this product: jersey. The source code is available at https://jersey.java.net/download.html. You may also request a copy of the source code by sending a request to opensource@fb.com. This software contains the following license and notice below: + +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL - Version 1.1) +1. Definitions. + + 1.1. “Contributor” means each individual or entity that creates or contributes to the creation of Modifications. + + 1.2. “Contributor Version” means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor. + + 1.3. “Covered Software” means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof. + + 1.4. “Executable” means the Covered Software in any form other than Source Code. + + 1.5. “Initial Developer” means the individual or entity that first makes Original Software available under this License. + + 1.6. “Larger Work” means a work which combines Covered Software or portions thereof with code not governed by the terms of this License. + + 1.7. “License” means this document. + + 1.8. “Licensable” means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein. + + 1.9. “Modifications” means the Source Code and Executable form of any of the following: + + A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; + + B. Any new file that contains any part of the Original Software or previous Modification; or + + C. Any new file that is contributed or otherwise made available under the terms of this License. + + 1.10. “Original Software” means the Source Code and Executable form of computer software code that is originally released under this License. + + 1.11. “Patent Claims” means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor. + + 1.12. “Source Code” means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code. + + 1.13. “You” (or “Your”) means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, “You” includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, “control” means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity. + +2. License Grants. + + 2.1. The Initial Developer Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof). + + (c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License. + + (d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices. + + 2.2. Contributor Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination). + + (c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party. + + (d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor. + +3. Distribution Obligations. + + 3.1. Availability of Source Code. + + Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange. + + 3.2. Modifications. + + The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License. + + 3.3. Required Notices. + + You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer. + + 3.4. Application of Additional Terms. + + You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients’ rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer. + + 3.5. Distribution of Executable Versions. + + You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient’s rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. + + 3.6. Larger Works. + + You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software. + +4. Versions of the License. + + 4.1. New Versions. + + Oracle is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License. + + 4.2. Effect of New Versions. + + You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward. + + 4.3. Modified Versions. + + When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License. + +5. DISCLAIMER OF WARRANTY. + +COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN “AS IS” BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. +6. TERMINATION. + + 6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. + + 6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as “Participant”) alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant. + + 6.3. If You assert a patent infringement claim against Participant alleging that the Participant Software directly or indirectly infringes any patent where such claim is resolved (such as by license or settlement) prior to the initiation of patent infringement litigation, then the reasonable value of the licenses granted by such Participant under Sections 2.1 or 2.2 shall be taken into account in determining the amount or value of any payment or license. + + 6.4. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. + +UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY’S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. +8. U.S. GOVERNMENT END USERS. + +The Covered Software is a “commercial item,” as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of “commercial computer software” (as that term is defined at 48 C.F.R. § 252.227-7014(a)(1)) and “commercial computer software documentation” as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License. +9. MISCELLANEOUS. + +This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction’s conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys’ fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software. +10. RESPONSIBILITY FOR CLAIMS. + +As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability. +NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) + +The code released under the CDDL shall be governed by the laws of the State of California (excluding conflict-of-law provisions). Any litigation relating to this License shall be subject to the jurisdiction of the Federal Courts of the Northern District of California and the state courts of the State of California, with venue lying in Santa Clara County, California. + +----- + +The following software may be included in this product: JLine. This software contains the following license and notice below: + +/* + * Copyright (c) 2002-2007, Marc Prud'hommeaux. All rights reserved. + * + * This software is distributable under the BSD license. See the terms of the + * BSD license in the documentation provided with this software. + */ + +----- + +The following software may be included in this product: Joni. This software contains the following license and notice below: + +/* +* Permission is hereby granted, free of charge, to any person obtaining a copy of +* this software and associated documentation files (the "Software"), to deal in +* the Software without restriction, including without limitation the rights to +* use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +* of the Software, and to permit persons to whom the Software is furnished to do +* so, subject to the following conditions: +* +* The above copyright notice and this permission notice shall be included in all +* copies or substantial portions of the Software. +* +* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +* IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +* FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +* AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +* LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +* OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +* SOFTWARE. +*/ + +----- + +The following software may be included in this product: leveldb. This software contains the following license and notice below: + +Copyright (c) 2011 The LevelDB Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + +* Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. +* Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. +* Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +----- + +The following software may be included in this product: logback. The source code is available at http://logback.qos.ch/download.html. You may also request a copy of the source code by sending a request to opensource@fb.com. This software contains the following license and notice below: + +Eclipse Public License - v 1.0 + +THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. + +1. DEFINITIONS + +"Contribution" means: + +a) in the case of the initial Contributor, the initial code and documentation distributed under this Agreement, and + +b) in the case of each subsequent Contributor: + +i) changes to the Program, and + +ii) additions to the Program; + +where such changes and/or additions to the Program originate from and are distributed by that particular Contributor. A Contribution 'originates' from a Contributor if it was added to the Program by such Contributor itself or anyone acting on such Contributor's behalf. Contributions do not include additions to the Program which: (i) are separate modules of software distributed in conjunction with the Program under their own license agreement, and (ii) are not derivative works of the Program. + +"Contributor" means any person or entity that distributes the Program. + +"Licensed Patents" mean patent claims licensable by a Contributor which are necessarily infringed by the use or sale of its Contribution alone or when combined with the Program. + +"Program" means the Contributions distributed in accordance with this Agreement. + +"Recipient" means anyone who receives the Program under this Agreement, including all Contributors. + +2. GRANT OF RIGHTS + +a) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free copyright license to reproduce, prepare derivative works of, publicly display, publicly perform, distribute and sublicense the Contribution of such Contributor, if any, and such derivative works, in source code and object code form. + +b) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed Patents to make, use, sell, offer to sell, import and otherwise transfer the Contribution of such Contributor, if any, in source code and object code form. This patent license shall apply to the combination of the Contribution and the Program if, at the time the Contribution is added by the Contributor, such addition of the Contribution causes such combination to be covered by the Licensed Patents. The patent license shall not apply to any other combinations which include the Contribution. No hardware per se is licensed hereunder. + +c) Recipient understands that although each Contributor grants the licenses to its Contributions set forth herein, no assurances are provided by any Contributor that the Program does not infringe the patent or other intellectual property rights of any other entity. Each Contributor disclaims any liability to Recipient for claims brought by any other entity based on infringement of intellectual property rights or otherwise. As a condition to exercising the rights and licenses granted hereunder, each Recipient hereby assumes sole responsibility to secure any other intellectual property rights needed, if any. For example, if a third party patent license is required to allow Recipient to distribute the Program, it is Recipient's responsibility to acquire that license before distributing the Program. + +d) Each Contributor represents that to its knowledge it has sufficient copyright rights in its Contribution, if any, to grant the copyright license set forth in this Agreement. + +3. REQUIREMENTS + +A Contributor may choose to distribute the Program in object code form under its own license agreement, provided that: + +a) it complies with the terms and conditions of this Agreement; and + +b) its license agreement: + +i) effectively disclaims on behalf of all Contributors all warranties and conditions, express and implied, including warranties or conditions of title and non-infringement, and implied warranties or conditions of merchantability and fitness for a particular purpose; + +ii) effectively excludes on behalf of all Contributors all liability for damages, including direct, indirect, special, incidental and consequential damages, such as lost profits; + +iii) states that any provisions which differ from this Agreement are offered by that Contributor alone and not by any other party; and + +iv) states that source code for the Program is available from such Contributor, and informs licensees how to obtain it in a reasonable manner on or through a medium customarily used for software exchange. + +When the Program is made available in source code form: + +a) it must be made available under this Agreement; and + +b) a copy of this Agreement must be included with each copy of the Program. + +Contributors may not remove or alter any copyright notices contained within the Program. + +Each Contributor must identify itself as the originator of its Contribution, if any, in a manner that reasonably allows subsequent Recipients to identify the originator of the Contribution. + +4. COMMERCIAL DISTRIBUTION + +Commercial distributors of software may accept certain responsibilities with respect to end users, business partners and the like. While this license is intended to facilitate the commercial use of the Program, the Contributor who includes the Program in a commercial product offering should do so in a manner which does not create potential liability for other Contributors. Therefore, if a Contributor includes the Program in a commercial product offering, such Contributor ("Commercial Contributor") hereby agrees to defend and indemnify every other Contributor ("Indemnified Contributor") against any losses, damages and costs (collectively "Losses") arising from claims, lawsuits and other legal actions brought by a third party against the Indemnified Contributor to the extent caused by the acts or omissions of such Commercial Contributor in connection with its distribution of the Program in a commercial product offering. The obligations in this section do not apply to any claims or Losses relating to any actual or alleged intellectual property infringement. In order to qualify, an Indemnified Contributor must: a) promptly notify the Commercial Contributor in writing of such claim, and b) allow the Commercial Contributor to control, and cooperate with the Commercial Contributor in, the defense and any related settlement negotiations. The Indemnified Contributor may participate in any such claim at its own expense. + +For example, a Contributor might include the Program in a commercial product offering, Product X. That Contributor is then a Commercial Contributor. If that Commercial Contributor then makes performance claims, or offers warranties related to Product X, those performance claims and warranties are such Commercial Contributor's responsibility alone. Under this section, the Commercial Contributor would have to defend claims against the other Contributors related to those performance claims and warranties, and if a court requires any other Contributor to pay any damages as a result, the Commercial Contributor must pay those damages. + +5. NO WARRANTY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each Recipient is solely responsible for determining the appropriateness of using and distributing the Program and assumes all risks associated with its exercise of rights under this Agreement , including but not limited to the risks and costs of program errors, compliance with applicable laws, damage to or loss of data, programs or equipment, and unavailability or interruption of operations. + +6. DISCLAIMER OF LIABILITY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + +7. GENERAL + +If any provision of this Agreement is invalid or unenforceable under applicable law, it shall not affect the validity or enforceability of the remainder of the terms of this Agreement, and without further action by the parties hereto, such provision shall be reformed to the minimum extent necessary to make such provision valid and enforceable. + +If Recipient institutes patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Program itself (excluding combinations of the Program with other software or hardware) infringes such Recipient's patent(s), then such Recipient's rights granted under Section 2(b) shall terminate as of the date such litigation is filed. + +All Recipient's rights under this Agreement shall terminate if it fails to comply with any of the material terms or conditions of this Agreement and does not cure such failure in a reasonable period of time after becoming aware of such noncompliance. If all Recipient's rights under this Agreement terminate, Recipient agrees to cease use and distribution of the Program as soon as reasonably practicable. However, Recipient's obligations under this Agreement and any licenses granted by Recipient relating to the Program shall continue and survive. + +Everyone is permitted to copy and distribute copies of this Agreement, but in order to avoid inconsistency the Agreement is copyrighted and may only be modified in the following manner. The Agreement Steward reserves the right to publish new versions (including revisions) of this Agreement from time to time. No one other than the Agreement Steward has the right to modify this Agreement. The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation may assign the responsibility to serve as the Agreement Steward to a suitable separate entity. Each new version of the Agreement will be given a distinguishing version number. The Program (including Contributions) may always be distributed subject to the version of the Agreement under which it was received. In addition, after a new version of the Agreement is published, Contributor may elect to distribute the Program (including its Contributions) under the new version. Except as expressly stated in Sections 2(a) and 2(b) above, Recipient receives no rights or licenses to the intellectual property of any Contributor under this Agreement, whether expressly, by implication, estoppel or otherwise. All rights in the Program not expressly granted under this Agreement are reserved. + +This Agreement is governed by the laws of the State of New York and the intellectual property laws of the United States of America. No party to this Agreement will bring a legal action under this Agreement more than one year after the cause of action arose. Each party waives its rights to a jury trial in any resulting litigation. + +----- + +The following software may be included in this product: protobuf. This software contains the following license and notice below: + +Copyright 2008, Google Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + +* Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. +* Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. +* Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. + +----- + +The following software may be included in this product: slf4j. This software contains the following license and notice below: + +Copyright (c) 2004-2013 QOS.ch +All rights reserved. + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/presto-twitter-server/README.txt b/presto-twitter-server/README.txt new file mode 100644 index 000000000000..46017e937c7b --- /dev/null +++ b/presto-twitter-server/README.txt @@ -0,0 +1,5 @@ +Presto is a distributed SQL query engine. + +Please see the website for installation instructions: + +https://prestodb.io/ diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml new file mode 100644 index 000000000000..8ef9a1160fc9 --- /dev/null +++ b/presto-twitter-server/pom.xml @@ -0,0 +1,26 @@ + + + 4.0.0 + + + com.facebook.presto + presto-root + 0.196-tw-0.49 + + + presto-twitter-server + presto-twitter-server + provisio + + + ${project.parent.basedir} + true + true + true + true + + + com.facebook.presto.server.PrestoServer + ${project.artifactId} + + diff --git a/presto-twitter-server/src/main/provisio/presto.xml b/presto-twitter-server/src/main/provisio/presto.xml new file mode 100644 index 000000000000..7a448a544048 --- /dev/null +++ b/presto-twitter-server/src/main/provisio/presto.xml @@ -0,0 +1,76 @@ + + + + + + + + NOTICE + README.txt + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/presto-twitter-server/src/main/provisio/twitter.xml b/presto-twitter-server/src/main/provisio/twitter.xml new file mode 100644 index 000000000000..e1da085284c0 --- /dev/null +++ b/presto-twitter-server/src/main/provisio/twitter.xml @@ -0,0 +1,22 @@ + + + + + + + + + + + + + + + + + + + + + + diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 00a6f5869099..591446214b77 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml new file mode 100644 index 000000000000..d78b0cc2f35c --- /dev/null +++ b/twitter-eventlistener-plugin/pom.xml @@ -0,0 +1,115 @@ + + + 4.0.0 + + com.facebook.presto + presto-root + 0.196-tw-0.49 + + + twitter-eventlistener-plugin + Twitter Event Listener - scribes QueryCompletedEvent + presto-plugin + + + ${project.parent.basedir} + + + + + com.facebook.presto + presto-spi + 0.196-tw-0.49 + provided + + + io.airlift + log + + + io.airlift + units + provided + + + com.google.guava + guava + + + org.glassfish + javax.json + 1.0.4 + + + + + com.twitter + presto-thrift-java + 0.0.5 + + + com.twitter + util-core_2.11 + + + com.twitter + util-core-java + + + com.twitter + util-function_2.10 + + + com.twitter + util-function-java + + + commons-logging + commons-logging + + + org.scala-lang.modules + scala-parser-combinators_2.11 + + + com.twitter + scrooge-core + + + org.scala-lang + scala-library + + + org.scala-lang + scala-reflect + + + + + com.twitter + util-logging_2.10 + 6.34.0 + + + commons-logging + commons-logging + + + + + org.apache.thrift + libthrift + + + org.scala-lang + scala-library + 2.10.6 + + + commons-logging + commons-logging + + + + + diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java new file mode 100644 index 000000000000..e6bfd714ad36 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -0,0 +1,114 @@ +/* + * Licensed 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 com.twitter.presto.plugin.eventlistener; + +import com.facebook.presto.spi.eventlistener.QueryCompletedEvent; +import com.facebook.presto.spi.eventlistener.QueryContext; +import com.facebook.presto.spi.eventlistener.QueryFailureInfo; +import com.facebook.presto.spi.eventlistener.QueryMetadata; +import com.facebook.presto.spi.eventlistener.QueryStatistics; +import com.twitter.presto.thriftjava.QueryCompletionEvent; +import com.twitter.presto.thriftjava.QueryState; +import io.airlift.log.Logger; +import org.apache.thrift.TException; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Class that scribes query completion events + */ +public class QueryCompletedEventScriber +{ + private static final String DASH = "-"; + private static final Logger log = Logger.get(QueryCompletedEventScriber.class); + + private TwitterScriber scriber = new TwitterScriber("presto_query_completion"); + + public void handle(QueryCompletedEvent event) + { + try { + scriber.scribe(toThriftQueryCompletionEvent(event)); + } + catch (TException e) { + log.warn(e, + String.format("Could not serialize thrift object of Query(id=%s, user=%s, env=%s, schema=%s.%s)", + event.getMetadata().getQueryId(), + event.getContext().getUser(), + event.getContext().getEnvironment(), + event.getContext().getCatalog().orElse(DASH), + event.getContext().getSchema().orElse(DASH))); + } + } + + private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedEvent event) + { + QueryMetadata eventMetadata = event.getMetadata(); + QueryContext eventContext = event.getContext(); + QueryStatistics eventStat = event.getStatistics(); + + QueryCompletionEvent thriftEvent = + new com.twitter.presto.thriftjava.QueryCompletionEvent(); + + thriftEvent.query_id = eventMetadata.getQueryId(); + thriftEvent.transaction_id = eventMetadata.getTransactionId().orElse(DASH); + thriftEvent.user = eventContext.getUser(); + thriftEvent.principal = eventContext.getPrincipal().orElse(DASH); + thriftEvent.source = eventContext.getSource().orElse(DASH); + thriftEvent.server_version = eventContext.getServerVersion(); + thriftEvent.environment = eventContext.getEnvironment(); + thriftEvent.catalog = eventContext.getCatalog().orElse(DASH); + thriftEvent.schema = eventContext.getSchema().orElse(DASH); + Map> queriedColumnsByTable = new HashMap>(); + event.getIoMetadata().getInputs().forEach(input -> queriedColumnsByTable.put(String.format("%s.%s", input.getSchema(), input.getTable()), input.getColumns())); + thriftEvent.queried_columns_by_table = queriedColumnsByTable; + thriftEvent.remote_client_address = eventContext.getRemoteClientAddress().orElse(DASH); + thriftEvent.user_agent = eventContext.getUserAgent().orElse(DASH); + thriftEvent.query_state = QueryState.valueOf(eventMetadata.getQueryState()); + thriftEvent.uri = eventMetadata.getUri().toString(); + thriftEvent.query = eventMetadata.getQuery(); + thriftEvent.create_time_ms = event.getCreateTime().toEpochMilli(); + thriftEvent.execution_start_time_ms = event.getExecutionStartTime().toEpochMilli(); + thriftEvent.end_time_ms = event.getEndTime().toEpochMilli(); + thriftEvent.queued_time_ms = eventStat.getQueuedTime().toMillis(); + thriftEvent.query_wall_time_ms = eventStat.getWallTime().toMillis(); + thriftEvent.cumulative_memory_bytesecond = eventStat.getCumulativeMemory(); + thriftEvent.peak_memory_bytes = eventStat.getPeakTotalNonRevocableMemoryBytes(); + thriftEvent.cpu_time_ms = eventStat.getCpuTime().toMillis(); + if (eventStat.getAnalysisTime().isPresent()) { + thriftEvent.analysis_time_ms = eventStat.getAnalysisTime().get().toMillis(); + } + if (eventStat.getDistributedPlanningTime().isPresent()) { + thriftEvent.distributed_planning_time_ms = eventStat.getDistributedPlanningTime().get().toMillis(); + } + thriftEvent.total_bytes = eventStat.getTotalBytes(); + thriftEvent.query_stages = QueryStatsHelper.getQueryStages(eventMetadata); + thriftEvent.operator_summaries = QueryStatsHelper.getOperatorSummaries(eventStat); + thriftEvent.total_rows = eventStat.getTotalRows(); + thriftEvent.splits = eventStat.getCompletedSplits(); + if (event.getFailureInfo().isPresent()) { + QueryFailureInfo eventFailureInfo = event.getFailureInfo().get(); + thriftEvent.error_code_id = eventFailureInfo.getErrorCode().getCode(); + thriftEvent.error_code_name = eventFailureInfo.getErrorCode().getName(); + thriftEvent.failure_type = eventFailureInfo.getFailureType().orElse(DASH); + thriftEvent.failure_message = eventFailureInfo.getFailureMessage().orElse(DASH); + thriftEvent.failure_task = eventFailureInfo.getFailureTask().orElse(DASH); + thriftEvent.failure_host = eventFailureInfo.getFailureHost().orElse(DASH); + thriftEvent.failures_json = eventFailureInfo.getFailuresJson(); + } + + return thriftEvent; + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java new file mode 100644 index 000000000000..e57cedda3ff0 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java @@ -0,0 +1,217 @@ +/* + * Licensed 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 com.twitter.presto.plugin.eventlistener; + +import com.facebook.presto.spi.eventlistener.QueryMetadata; +import com.facebook.presto.spi.eventlistener.QueryStatistics; +import com.twitter.presto.thriftjava.OperatorStats; +import com.twitter.presto.thriftjava.QueryStageInfo; +import io.airlift.log.Logger; +import io.airlift.units.DataSize; +import io.airlift.units.Duration; + +import javax.json.Json; +import javax.json.JsonObject; +import javax.json.JsonReader; +import javax.json.JsonValue.ValueType; + +import java.io.StringReader; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Queue; +import java.util.stream.Collectors; + +public class QueryStatsHelper +{ + private static final Logger log = Logger.get(QueryStatsHelper.class); + + private QueryStatsHelper() + { + throw new AssertionError(); + } + + private static long getBytesOrNegativeOne(String strVal) + { + try { + return DataSize.valueOf(strVal).toBytes(); + } + catch (IllegalArgumentException e) { + log.warn(e, + String.format("Failed to parse io.airlift.units.DataSize '%s', returning -1", strVal)); + return -1; + } + } + + private static long getMillisOrNegativeOne(String strVal) + { + try { + return Duration.valueOf(strVal).toMillis(); + } + catch (IllegalArgumentException e) { + log.warn(e, + String.format("Failed to parse io.airlift.units.Duration '%s', returning -1", strVal)); + return -1; + } + } + + private static QueryStageInfo getQueryStageInfo(int stageId, JsonObject stage) + { + QueryStageInfo stageInfo = new QueryStageInfo(); + + stageInfo.stage_id = stageId; + try { + JsonObject stageStats = stage.getJsonObject("stageStats"); + stageInfo.raw_input_data_size_bytes = getBytesOrNegativeOne(stageStats.getString("rawInputDataSize")); + stageInfo.output_data_size_bytes = getBytesOrNegativeOne(stageStats.getString("outputDataSize")); + stageInfo.completed_tasks = stageStats.getInt("completedTasks"); + stageInfo.completed_drivers = stageStats.getInt("completedDrivers"); + stageInfo.cumulative_memory = stageStats.getJsonNumber("cumulativeMemory").doubleValue(); + stageInfo.peak_memory_reservation_bytes = getBytesOrNegativeOne(stageStats.getString("peakMemoryReservation")); + stageInfo.total_scheduled_time_millis = getMillisOrNegativeOne(stageStats.getString("totalScheduledTime")); + stageInfo.total_cpu_time_millis = getMillisOrNegativeOne(stageStats.getString("totalCpuTime")); + stageInfo.total_user_time_millis = getMillisOrNegativeOne(stageStats.getString("totalUserTime")); + stageInfo.total_blocked_time_millis = getMillisOrNegativeOne(stageStats.getString("totalBlockedTime")); + } + catch (Exception e) { + log.error(e, String.format("Error retrieving stage stats for stage %d", stageId)); + return null; + } + + return stageInfo; + } + + private static OperatorStats getOperatorStat(String operatorSummaryStr) + { + try { + JsonReader jsonReader = Json.createReader(new StringReader(operatorSummaryStr)); + return getOperatorStat(jsonReader.readObject()); + } + catch (Exception e) { + log.error(e, String.format("Error retrieving operator stats from string:\n%s\n", operatorSummaryStr)); + } + + return null; + } + + private static OperatorStats getOperatorStat(JsonObject obj) + { + OperatorStats operatorStats = new OperatorStats(); + + try { + operatorStats.pipeline_id = obj.getInt("pipelineId"); + operatorStats.operator_id = obj.getInt("operatorId"); + operatorStats.plan_node_id = obj.getString("planNodeId"); + operatorStats.operator_type = obj.getString("operatorType"); + operatorStats.total_drivers = obj.getJsonNumber("totalDrivers").longValue(); + operatorStats.add_input_calls = obj.getJsonNumber("addInputCalls").longValue(); + operatorStats.add_input_wall_millis = getMillisOrNegativeOne(obj.getString("addInputWall")); + operatorStats.add_input_cpu_millis = getMillisOrNegativeOne(obj.getString("addInputCpu")); + operatorStats.add_input_user_millis = getMillisOrNegativeOne(obj.getString("addInputUser")); + operatorStats.input_data_size_bytes = getBytesOrNegativeOne(obj.getString("inputDataSize")); + operatorStats.input_positions = obj.getJsonNumber("inputPositions").longValue(); + operatorStats.sum_squared_input_positions = obj.getJsonNumber("sumSquaredInputPositions").doubleValue(); + operatorStats.get_output_calls = obj.getJsonNumber("getOutputCalls").longValue(); + operatorStats.get_output_wall_millis = getMillisOrNegativeOne(obj.getString("getOutputWall")); + operatorStats.get_output_cpu_millis = getMillisOrNegativeOne(obj.getString("getOutputCpu")); + operatorStats.get_output_user_millis = getMillisOrNegativeOne(obj.getString("getOutputUser")); + operatorStats.output_data_size_bytes = getBytesOrNegativeOne(obj.getString("outputDataSize")); + operatorStats.output_positions = obj.getJsonNumber("outputPositions").longValue(); + operatorStats.blocked_wall_millis = getMillisOrNegativeOne(obj.getString("blockedWall")); + operatorStats.finish_calls = obj.getJsonNumber("finishCalls").longValue(); + operatorStats.finish_wall_millis = getMillisOrNegativeOne(obj.getString("finishWall")); + operatorStats.finish_cpu_millis = getMillisOrNegativeOne(obj.getString("finishCpu")); + operatorStats.finish_user_millis = getMillisOrNegativeOne(obj.getString("finishUser")); + operatorStats.memory_reservation_bytes = getBytesOrNegativeOne(obj.getString("memoryReservation")); + operatorStats.system_memory_reservation_bytes = getBytesOrNegativeOne(obj.getString("systemMemoryReservation")); + } + catch (Exception e) { + log.error(e, String.format("Error retrieving operator stats from JsonObject:\n%s\n", obj.toString())); + return null; + } + + return operatorStats; + } + + public static Map getQueryStages(QueryMetadata eventMetadata) + { + if (!eventMetadata.getPayload().isPresent()) { + return null; + } + + String payload = eventMetadata.getPayload().get(); + Queue stageJsonObjs = new LinkedList(); + try { + JsonReader jsonReader = Json.createReader(new StringReader(payload)); + stageJsonObjs.add(jsonReader.readObject()); + } + catch (Exception e) { + log.error(e, + String.format("getQueryStages - Unable to extract JsonObject out of following blob:\n%s\n", payload)); + return null; + } + + Map stages = new HashMap(); + while (!stageJsonObjs.isEmpty()) { + JsonObject cur = stageJsonObjs.poll(); + String stageIdStr = "Unknown"; + try { + stageIdStr = cur.getString("stageId"); + int stageId = Integer.parseInt(stageIdStr.split("\\.")[1]); + QueryStageInfo curStage = getQueryStageInfo(stageId, cur); + if (curStage != null) { + stages.put(stageId, getQueryStageInfo(stageId, cur)); + } + } + catch (Exception e) { + log.error(e, + String.format("Failed to parse QueryStageInfo from JsonObject:\n%s\n", cur.toString())); + return null; + } + + try { + cur.getJsonArray("subStages") + .stream() + .filter(val -> val.getValueType() == ValueType.OBJECT) + .forEach(val -> stageJsonObjs.add((JsonObject) val)); + } + catch (Exception e) { + log.error(e, + String.format("Failed to get subStages for stage %s, treating as no subStages", stageIdStr)); + } + } + + return stages; + } + + public static List getOperatorSummaries(QueryStatistics eventStat) + { + try { + return eventStat.getOperatorSummaries() + .stream() + .filter(val -> val != null && !val.isEmpty()) + .map(QueryStatsHelper::getOperatorStat) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + } + catch (Exception e) { + log.error(e, + String.format("Error converting List to List:\n%s\n", eventStat.getOperatorSummaries().toString())); + } + + return null; + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java new file mode 100644 index 000000000000..6df4eb29c54b --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java @@ -0,0 +1,43 @@ +/* + * Licensed 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 com.twitter.presto.plugin.eventlistener; + +import com.facebook.presto.spi.eventlistener.EventListener; +import com.facebook.presto.spi.eventlistener.QueryCompletedEvent; +import com.facebook.presto.spi.eventlistener.QueryCreatedEvent; +import com.facebook.presto.spi.eventlistener.SplitCompletedEvent; +import io.airlift.log.Logger; + +public class TwitterEventListener + implements EventListener +{ + private static final Logger log = Logger.get(TwitterEventListener.class); + private final QueryCompletedEventScriber scriber = new QueryCompletedEventScriber(); + + @Override + public void queryCreated(QueryCreatedEvent queryCreatedEvent) + { + } + + @Override + public void queryCompleted(QueryCompletedEvent queryCompletedEvent) + { + scriber.handle(queryCompletedEvent); + } + + @Override + public void splitCompleted(SplitCompletedEvent splitCompletedEvent) + { + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java new file mode 100644 index 000000000000..a22762c9ec32 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java @@ -0,0 +1,35 @@ +/* + * Licensed 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 com.twitter.presto.plugin.eventlistener; + +import com.facebook.presto.spi.eventlistener.EventListener; +import com.facebook.presto.spi.eventlistener.EventListenerFactory; + +import java.util.Map; + +public class TwitterEventListenerFactory + implements EventListenerFactory +{ + @Override + public String getName() + { + return "twitter-event-listener"; + } + + @Override + public EventListener create(Map config) + { + return new TwitterEventListener(); + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java new file mode 100644 index 000000000000..1b77b1decce7 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java @@ -0,0 +1,28 @@ +/* + * Licensed 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 com.twitter.presto.plugin.eventlistener; + +import com.facebook.presto.spi.Plugin; +import com.facebook.presto.spi.eventlistener.EventListenerFactory; +import com.google.common.collect.ImmutableList; + +public class TwitterEventListenerPlugin + implements Plugin +{ + @Override + public Iterable getEventListenerFactories() + { + return ImmutableList.of(new TwitterEventListenerFactory()); + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java new file mode 100644 index 000000000000..8f4088da6aeb --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java @@ -0,0 +1,79 @@ +/* + * Licensed 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 com.twitter.presto.plugin.eventlistener; + +import com.twitter.logging.BareFormatter$; +import com.twitter.logging.Level; +import com.twitter.logging.QueueingHandler; +import com.twitter.logging.ScribeHandler; +import org.apache.thrift.TBase; +import org.apache.thrift.TException; +import org.apache.thrift.TSerializer; + +import java.util.Base64; +import java.util.logging.LogRecord; + +public class TwitterScriber +{ + private static final String DASH = "-"; + private static final int MAX_QUEUE_SIZE = 1000; + + private QueueingHandler queueingHandler; + + // TSerializer is not thread safe + private final ThreadLocal serializer = new ThreadLocal() { + @Override + protected TSerializer initialValue() + { + return new TSerializer(); + } + }; + + public TwitterScriber(String scribeCategory) + { + ScribeHandler scribeHandler = new ScribeHandler( + ScribeHandler.DefaultHostname(), + ScribeHandler.DefaultPort(), + scribeCategory, + ScribeHandler.DefaultBufferTime(), + ScribeHandler.DefaultConnectBackoff(), + ScribeHandler.DefaultMaxMessagesPerTransaction(), + ScribeHandler.DefaultMaxMessagesToBuffer(), + BareFormatter$.MODULE$, + scala.Option.apply((Level) null)); + queueingHandler = new QueueingHandler(scribeHandler, MAX_QUEUE_SIZE); + } + + public void scribe(TBase thriftMessage) + throws TException + { + scribe(serializeThriftToString(thriftMessage)); + } + + /** + * Serialize a thrift object to bytes, compress, then encode as a base64 string. + * Throws TException + */ + private String serializeThriftToString(TBase thriftMessage) + throws TException + { + return Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage)); + } + + private void scribe(String message) + { + LogRecord logRecord = new LogRecord(Level.ALL, message); + queueingHandler.publish(logRecord); + } +}