diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 1c0f50328ee72..9b7026eeca4c8 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -17,7 +17,8 @@ jobs: # Build: build Spark and run the tests for specified modules. build: name: "Build modules: ${{ matrix.modules }} ${{ matrix.comment }} (JDK ${{ matrix.java }}, ${{ matrix.hadoop }}, ${{ matrix.hive }})" - runs-on: ubuntu-latest + # Ubuntu 20.04 is the latest LTS. The next LTS is 22.04. + runs-on: ubuntu-20.04 strategy: fail-fast: false matrix: @@ -154,12 +155,11 @@ jobs: - name: Install Python packages (Python 3.6 and PyPy3) if: contains(matrix.modules, 'pyspark') # PyArrow is not supported in PyPy yet, see ARROW-2651. - # TODO(SPARK-32247): scipy installation with PyPy fails for an unknown reason. run: | python3.6 -m pip install numpy pyarrow pandas scipy xmlrunner python3.6 -m pip list # PyPy does not have xmlrunner - pypy3 -m pip install numpy pandas + pypy3 -m pip install numpy pandas scipy pypy3 -m pip list - name: Install Python packages (Python 3.8) if: contains(matrix.modules, 'pyspark') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) @@ -168,12 +168,10 @@ jobs: python3.8 -m pip list # SparkR - name: Install R 4.0 + uses: r-lib/actions/setup-r@v1 if: contains(matrix.modules, 'sparkr') - run: | - sudo sh -c "echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran40/' >> /etc/apt/sources.list" - curl -sL "https://keyserver.ubuntu.com/pks/lookup?op=get&search=0xE298A3A825C0D65DFD57CBB651716619E084DAB9" | sudo apt-key add - sudo apt-get update - sudo apt-get install -y r-base r-base-dev libcurl4-openssl-dev + with: + r-version: 4.0 - name: Install R packages if: contains(matrix.modules, 'sparkr') run: | @@ -206,7 +204,7 @@ jobs: # Static analysis, and documentation build lint: name: Linters, licenses, dependencies and documentation generation - runs-on: ubuntu-latest + runs-on: ubuntu-20.04 steps: - name: Checkout Spark repository uses: actions/checkout@v2 @@ -232,11 +230,9 @@ jobs: # See also https://github.com/sphinx-doc/sphinx/issues/7551. pip3 install flake8 'sphinx<3.1.0' numpy pydata_sphinx_theme ipython nbsphinx - name: Install R 4.0 - run: | - sudo sh -c "echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran40/' >> /etc/apt/sources.list" - curl -sL "https://keyserver.ubuntu.com/pks/lookup?op=get&search=0xE298A3A825C0D65DFD57CBB651716619E084DAB9" | sudo apt-key add - sudo apt-get update - sudo apt-get install -y r-base r-base-dev libcurl4-openssl-dev + uses: r-lib/actions/setup-r@v1 + with: + r-version: 4.0 - name: Install R linter dependencies and SparkR run: | sudo apt-get install -y libcurl4-openssl-dev @@ -275,7 +271,7 @@ jobs: java11: name: Java 11 build - runs-on: ubuntu-latest + runs-on: ubuntu-20.04 steps: - name: Checkout Spark repository uses: actions/checkout@v2 @@ -297,3 +293,25 @@ jobs: mkdir -p ~/.m2 ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -Phadoop-cloud -Djava.version=11 install rm -rf ~/.m2/repository/org/apache/spark + + scala-213: + name: Scala 2.13 build + runs-on: ubuntu-20.04 + steps: + - name: Checkout Spark repository + uses: actions/checkout@v2 + - name: Cache Ivy local repository + uses: actions/cache@v2 + with: + path: ~/.ivy2/cache + key: scala-213-ivy-${{ hashFiles('**/pom.xml', '**/plugins.sbt') }} + restore-keys: | + scala-213-ivy- + - name: Install Java 11 + uses: actions/setup-java@v1 + with: + java-version: 11 + - name: Build with SBT + run: | + ./dev/change-scala-version.sh 2.13 + ./build/sbt -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -Phadoop-cloud -Pkinesis-asl -Djava.version=11 -Pscala-2.13 compile test:compile diff --git a/.github/workflows/test_report.yml b/.github/workflows/test_report.yml index 93cdb86687261..060a8795b6a77 100644 --- a/.github/workflows/test_report.yml +++ b/.github/workflows/test_report.yml @@ -15,7 +15,16 @@ jobs: github_token: ${{ secrets.GITHUB_TOKEN }} workflow: ${{ github.event.workflow_run.workflow_id }} commit: ${{ github.event.workflow_run.head_commit.id }} + - name: Check if JUnit report XML files exist + run: | + if ls **/target/test-reports/*.xml > /dev/null 2>&1; then + echo '::set-output name=FILE_EXISTS::true' + else + echo '::set-output name=FILE_EXISTS::false' + fi + id: check-junit-file - name: Publish test report + if: steps.check-junit-file.outputs.FILE_EXISTS == 'true' uses: scacap/action-surefire-report@v1 with: check_name: Report test results diff --git a/.sbtopts b/.sbtopts new file mode 100644 index 0000000000000..9afbdca6db1c7 --- /dev/null +++ b/.sbtopts @@ -0,0 +1,17 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +-J-Xmx4G +-J-Xss4m diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 53a0b7856567e..a9cca4bf6f6fc 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -230,6 +230,7 @@ exportMethods("%<=>%", "asc", "ascii", "asin", + "assert_true", "atan", "atan2", "avg", @@ -272,6 +273,7 @@ exportMethods("%<=>%", "degrees", "dense_rank", "desc", + "dropFields", "element_at", "encode", "endsWith", @@ -348,6 +350,7 @@ exportMethods("%<=>%", "negate", "next_day", "not", + "nth_value", "ntile", "otherwise", "over", @@ -359,6 +362,7 @@ exportMethods("%<=>%", "posexplode_outer", "quarter", "radians", + "raise_error", "rand", "randn", "rank", @@ -405,6 +409,7 @@ exportMethods("%<=>%", "sumDistinct", "tan", "tanh", + "timestamp_seconds", "toDegrees", "toRadians", "to_csv", @@ -425,9 +430,11 @@ exportMethods("%<=>%", "variance", "var_pop", "var_samp", + "vector_to_array", "weekofyear", "when", "window", + "withField", "xxhash64", "year") diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 089e1f26b7d3b..2ce53782d9af0 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2863,11 +2863,18 @@ setMethod("unionAll", #' \code{UNION ALL} and \code{UNION DISTINCT} in SQL as column positions are not taken #' into account. Input SparkDataFrames can have different data types in the schema. #' +#' When the parameter allowMissingColumns is `TRUE`, the set of column names +#' in x and y can differ; missing columns will be filled as null. +#' Further, the missing columns of x will be added at the end +#' in the schema of the union result. +#' #' Note: This does not remove duplicate rows across the two SparkDataFrames. #' This function resolves columns by name (not by position). #' #' @param x A SparkDataFrame #' @param y A SparkDataFrame +#' @param allowMissingColumns logical +#' @param ... further arguments to be passed to or from other methods. #' @return A SparkDataFrame containing the result of the union. #' @family SparkDataFrame functions #' @rdname unionByName @@ -2880,12 +2887,15 @@ setMethod("unionAll", #' df1 <- select(createDataFrame(mtcars), "carb", "am", "gear") #' df2 <- select(createDataFrame(mtcars), "am", "gear", "carb") #' head(unionByName(df1, df2)) +#' +#' df3 <- select(createDataFrame(mtcars), "carb") +#' head(unionByName(df1, df3, allowMissingColumns = TRUE)) #' } #' @note unionByName since 2.3.0 setMethod("unionByName", signature(x = "SparkDataFrame", y = "SparkDataFrame"), - function(x, y) { - unioned <- callJMethod(x@sdf, "unionByName", y@sdf) + function(x, y, allowMissingColumns=FALSE) { + unioned <- callJMethod(x@sdf, "unionByName", y@sdf, allowMissingColumns) dataFrame(unioned) }) diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index 7926a9a2467ee..c5fcfaff94029 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -356,3 +356,103 @@ setMethod("%<=>%", #' } #' @note ! since 2.3.0 setMethod("!", signature(x = "Column"), function(x) not(x)) + +#' withField +#' +#' Adds/replaces field in a struct \code{Column} by name. +#' +#' @param x a Column +#' @param fieldName a character +#' @param col a Column expression +#' +#' @rdname withField +#' @aliases withField withField,Column-method +#' @examples +#' \dontrun{ +#' df <- withColumn( +#' createDataFrame(iris), +#' "sepal", +#' struct(column("Sepal_Width"), column("Sepal_Length")) +#' ) +#' +#' head(select( +#' df, +#' withField(df$sepal, "product", df$Sepal_Length * df$Sepal_Width) +#' )) +#' } +#' @note withField since 3.1.0 +setMethod("withField", + signature(x = "Column", fieldName = "character", col = "Column"), + function(x, fieldName, col) { + jc <- callJMethod(x@jc, "withField", fieldName, col@jc) + column(jc) + }) + +#' dropFields +#' +#' Drops fields in a struct \code{Column} by name. +#' +#' @param x a Column +#' @param ... names of the fields to be dropped. +#' +#' @rdname dropFields +#' @aliases dropFields dropFields,Column-method +#' @examples +#' \dontrun{ +#' df <- select( +#' createDataFrame(iris), +#' alias( +#' struct( +#' column("Sepal_Width"), column("Sepal_Length"), +#' alias( +#' struct( +#' column("Petal_Width"), column("Petal_Length"), +#' alias( +#' column("Petal_Width") * column("Petal_Length"), +#' "Petal_Product" +#' ) +#' ), +#' "Petal" +#' ) +#' ), +#' "dimensions" +#' ) +#' ) +#' head(withColumn(df, "dimensions", dropFields(df$dimensions, "Petal"))) +#' +#' head( +#' withColumn( +#' df, "dimensions", +#' dropFields(df$dimensions, "Sepal_Width", "Sepal_Length") +#' ) +#' ) +#' +#' # This method supports dropping multiple nested fields directly e.g. +#' head( +#' withColumn( +#' df, "dimensions", +#' dropFields(df$dimensions, "Petal.Petal_Width", "Petal.Petal_Length") +#' ) +#' ) +#' +#' # However, if you are going to add/replace multiple nested fields, +#' # it is preffered to extract out the nested struct before +#' # adding/replacing multiple fields e.g. +#' head( +#' withColumn( +#' df, "dimensions", +#' withField( +#' column("dimensions"), +#' "Petal", +#' dropFields(column("dimensions.Petal"), "Petal_Width", "Petal_Length") +#' ) +#' ) +#' ) +#' } +#' @note dropFields since 3.1.0 +setMethod("dropFields", + signature(x = "Column"), + function(x, ...) { + jc <- callJMethod(x@jc, "dropFields", list(...)) + column(jc) + }) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 5d9c8e8124d9a..bcd798a8c31e2 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -338,12 +338,29 @@ NULL #' tmp <- mutate(df, dist = over(cume_dist(), ws), dense_rank = over(dense_rank(), ws), #' lag = over(lag(df$mpg), ws), lead = over(lead(df$mpg, 1), ws), #' percent_rank = over(percent_rank(), ws), -#' rank = over(rank(), ws), row_number = over(row_number(), ws)) +#' rank = over(rank(), ws), row_number = over(row_number(), ws), +#' nth_value = over(nth_value(df$mpg, 3), ws)) #' # Get ntile group id (1-4) for hp #' tmp <- mutate(tmp, ntile = over(ntile(4), ws)) #' head(tmp)} NULL +#' ML functions for Column operations +#' +#' ML functions defined for \code{Column}. +#' +#' @param x Column to compute on. +#' @param ... additional argument(s). +#' @name column_ml_functions +#' @rdname column_ml_functions +#' @family ml functions +#' @examples +#' \dontrun{ +#' df <- read.df("data/mllib/sample_libsvm_data.txt", source = "libsvm") +#' head(select(df, vector_to_array(df$features))) +#' } +NULL + #' @details #' \code{lit}: A new Column is created to represent the literal value. #' If the parameter is a Column, it is returned unchanged. @@ -809,6 +826,57 @@ setMethod("xxhash64", column(jc) }) +#' @details +#' \code{assert_true}: Returns null if the input column is true; throws an exception +#' with the provided error message otherwise. +#' +#' @param errMsg (optional) The error message to be thrown. +#' +#' @rdname column_misc_functions +#' @aliases assert_true assert_true,Column-method +#' @examples +#' \dontrun{ +#' tmp <- mutate(df, v1 = assert_true(df$vs < 2), +#' v2 = assert_true(df$vs < 2, "custom error message"), +#' v3 = assert_true(df$vs < 2, df$vs)) +#' head(tmp)} +#' @note assert_true since 3.1.0 +setMethod("assert_true", + signature(x = "Column"), + function(x, errMsg = NULL) { + jc <- if (is.null(errMsg)) { + callJStatic("org.apache.spark.sql.functions", "assert_true", x@jc) + } else { + if (is.character(errMsg)) { + stopifnot(length(errMsg) == 1) + errMsg <- lit(errMsg) + } + callJStatic("org.apache.spark.sql.functions", "assert_true", x@jc, errMsg@jc) + } + column(jc) + }) + +#' @details +#' \code{raise_error}: Throws an exception with the provided error message. +#' +#' @rdname column_misc_functions +#' @aliases raise_error raise_error,characterOrColumn-method +#' @examples +#' \dontrun{ +#' tmp <- mutate(df, v1 = raise_error("error message")) +#' head(tmp)} +#' @note raise_error since 3.1.0 +setMethod("raise_error", + signature(x = "characterOrColumn"), + function(x) { + if (is.character(x)) { + stopifnot(length(x) == 1) + x <- lit(x) + } + jc <- callJStatic("org.apache.spark.sql.functions", "raise_error", x@jc) + column(jc) + }) + #' @details #' \code{dayofmonth}: Extracts the day of the month as an integer from a #' given date/timestamp/string. @@ -1417,8 +1485,10 @@ setMethod("quarter", }) #' @details -#' \code{percentile_approx} Returns the approximate percentile value of -#' numeric column at the given percentage. +#' \code{percentile_approx} Returns the approximate \code{percentile} of the numeric column +#' \code{col} which is the smallest value in the ordered \code{col} values (sorted from least to +#' greatest) such that no more than \code{percentage} of \code{col} values is less than the value +#' or equal to that value. #' #' @param percentage Numeric percentage at which percentile should be computed #' All values should be between 0 and 1. @@ -3296,6 +3366,37 @@ setMethod("lead", column(jc) }) +#' @details +#' \code{nth_value}: Window function: returns the value that is the \code{offset}th +#' row of the window frame# (counting from 1), and \code{null} if the size of window +#' frame is less than \code{offset} rows. +#' +#' @param offset a numeric indicating number of row to use as the value +#' @param na.rm a logical which indicates that the Nth value should skip null in the +#' determination of which row to use +#' +#' @rdname column_window_functions +#' @aliases nth_value nth_value,characterOrColumn-method +#' @note nth_value since 3.1.0 +setMethod("nth_value", + signature(x = "characterOrColumn", offset = "numeric"), + function(x, offset, na.rm = FALSE) { + x <- if (is.character(x)) { + column(x) + } else { + x + } + offset <- as.integer(offset) + jc <- callJStatic( + "org.apache.spark.sql.functions", + "nth_value", + x@jc, + offset, + na.rm + ) + column(jc) + }) + #' @details #' \code{ntile}: Returns the ntile group id (from 1 to n inclusive) in an ordered window #' partition. For example, if n is 4, the first quarter of the rows will get value 1, the second @@ -4380,7 +4481,8 @@ setMethod("date_trunc", }) #' @details -#' \code{current_date}: Returns the current date as a date column. +#' \code{current_date}: Returns the current date at the start of query evaluation as a date column. +#' All calls of current_date within the same query return the same value. #' #' @rdname column_datetime_functions #' @aliases current_date current_date,missing-method @@ -4396,7 +4498,8 @@ setMethod("current_date", }) #' @details -#' \code{current_timestamp}: Returns the current timestamp as a timestamp column. +#' \code{current_timestamp}: Returns the current timestamp at the start of query evaluation as +#' a timestamp column. All calls of current_timestamp within the same query return the same value. #' #' @rdname column_datetime_functions #' @aliases current_timestamp current_timestamp,missing-method @@ -4407,3 +4510,40 @@ setMethod("current_timestamp", jc <- callJStatic("org.apache.spark.sql.functions", "current_timestamp") column(jc) }) + +#' @details +#' \code{timestamp_seconds}: Creates timestamp from the number of seconds since UTC epoch. +#' +#' @rdname column_datetime_functions +#' @aliases timestamp_seconds timestamp_seconds,Column-method +#' @note timestamp_seconds since 3.1.0 +setMethod("timestamp_seconds", + signature(x = "Column"), + function(x) { + jc <- callJStatic( + "org.apache.spark.sql.functions", "timestamp_seconds", x@jc + ) + column(jc) + }) + +#' @details +#' \code{vector_to_array} Converts a column of MLlib sparse/dense vectors into +#' a column of dense arrays. +#' +#' @param dtype The data type of the output array. Valid values: "float64" or "float32". +#' +#' @rdname column_ml_functions +#' @aliases vector_to_array vector_to_array,Column-method +#' @note vector_to_array since 3.1.0 +setMethod("vector_to_array", + signature(x = "Column"), + function(x, dtype = c("float64", "float32")) { + dtype <- match.arg(dtype) + jc <- callJStatic( + "org.apache.spark.ml.functions", + "vector_to_array", + x@jc, + dtype + ) + column(jc) + }) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 839c00cf21aeb..6b732e594cd3f 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -638,7 +638,7 @@ setGeneric("union", function(x, y) { standardGeneric("union") }) setGeneric("unionAll", function(x, y) { standardGeneric("unionAll") }) #' @rdname unionByName -setGeneric("unionByName", function(x, y) { standardGeneric("unionByName") }) +setGeneric("unionByName", function(x, y, ...) { standardGeneric("unionByName") }) #' @rdname unpersist setGeneric("unpersist", function(x, ...) { standardGeneric("unpersist") }) @@ -729,6 +729,12 @@ setGeneric("over", function(x, window) { standardGeneric("over") }) #' @rdname eq_null_safe setGeneric("%<=>%", function(x, value) { standardGeneric("%<=>%") }) +#' @rdname withField +setGeneric("withField", function(x, fieldName, col) { standardGeneric("withField") }) + +#' @rdname dropFields +setGeneric("dropFields", function(x, ...) { standardGeneric("dropFields") }) + ###################### WindowSpec Methods ########################## #' @rdname partitionBy @@ -844,6 +850,10 @@ setGeneric("arrays_zip_with", function(x, y, f) { standardGeneric("arrays_zip_wi #' @name NULL setGeneric("ascii", function(x) { standardGeneric("ascii") }) +#' @rdname column_misc_functions +#' @name NULL +setGeneric("assert_true", function(x, errMsg = NULL) { standardGeneric("assert_true") }) + #' @param x Column to compute on or a GroupedData object. #' @param ... additional argument(s) when \code{x} is a GroupedData object. #' @rdname avg @@ -1161,6 +1171,10 @@ setGeneric("months_between", function(y, x, ...) { standardGeneric("months_betwe #' @rdname count setGeneric("n", function(x) { standardGeneric("n") }) +#' @rdname column_window_functions +#' @name NULL +setGeneric("nth_value", function(x, offset, ...) { standardGeneric("nth_value") }) + #' @rdname column_nonaggregate_functions #' @name NULL setGeneric("nanvl", function(y, x) { standardGeneric("nanvl") }) @@ -1213,6 +1227,10 @@ setGeneric("posexplode_outer", function(x) { standardGeneric("posexplode_outer") #' @name NULL setGeneric("quarter", function(x) { standardGeneric("quarter") }) +#' @rdname column_misc_functions +#' @name NULL +setGeneric("raise_error", function(x) { standardGeneric("raise_error") }) + #' @rdname column_nonaggregate_functions #' @name NULL setGeneric("rand", function(seed) { standardGeneric("rand") }) @@ -1354,6 +1372,10 @@ setGeneric("substring_index", function(x, delim, count) { standardGeneric("subst #' @name NULL setGeneric("sumDistinct", function(x) { standardGeneric("sumDistinct") }) +#' @rdname column_datetime_functions +#' @name timestamp_seconds +setGeneric("timestamp_seconds", function(x) { standardGeneric("timestamp_seconds") }) + #' @rdname column_collection_functions #' @name NULL setGeneric("transform_keys", function(x, f) { standardGeneric("transform_keys") }) @@ -1438,6 +1460,10 @@ setGeneric("var_pop", function(x) { standardGeneric("var_pop") }) #' @name NULL setGeneric("var_samp", function(x) { standardGeneric("var_samp") }) +#' @rdname column_ml_functions +#' @name NULL +setGeneric("vector_to_array", function(x, ...) { standardGeneric("vector_to_array") }) + #' @rdname column_datetime_functions #' @name NULL setGeneric("weekofyear", function(x) { standardGeneric("weekofyear") }) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index e008bc5bbd7d9..077dfc6770d94 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -1424,6 +1424,10 @@ test_that("column functions", { date_trunc("quarter", c) + current_date() + current_timestamp() c25 <- overlay(c1, c2, c3, c3) + overlay(c1, c2, c3) + overlay(c1, c2, 1) + overlay(c1, c2, 3, 4) + c26 <- timestamp_seconds(c1) + vector_to_array(c) + + vector_to_array(c, "float32") + vector_to_array(c, "float64") + c27 <- nth_value("x", 1L) + nth_value("y", 2, TRUE) + + nth_value(column("v"), 3) + nth_value(column("z"), 4L, FALSE) # Test if base::is.nan() is exposed expect_equal(is.nan(c("a", "b")), c(FALSE, FALSE)) @@ -1803,6 +1807,36 @@ test_that("column functions", { ) expect_equal(actual, expected) + + # Test withField + lines <- c("{\"Person\": {\"name\":\"Bob\", \"age\":24, \"height\": 170}}") + jsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") + writeLines(lines, jsonPath) + df <- read.df(jsonPath, "json") + result <- collect( + select( + select(df, alias(withField(df$Person, "dummy", lit(42)), "Person")), + "Person.dummy" + ) + ) + expect_equal(result, data.frame(dummy = 42)) + + # Test dropFields + expect_setequal( + colnames(select( + withColumn(df, "Person", dropFields(df$Person, "age")), + column("Person.*") + )), + c("name", "height") + ) + + expect_equal( + colnames(select( + withColumn(df, "Person", dropFields(df$Person, "height", "name")), + column("Person.*") + )), + "age" + ) }) test_that("column binary mathfunctions", { @@ -2113,7 +2147,7 @@ test_that("group by, agg functions", { df3 <- agg(gd, age = "stddev") expect_is(df3, "SparkDataFrame") df3_local <- collect(df3) - expect_true(is.nan(df3_local[df3_local$name == "Andy", ][1, 2])) + expect_true(is.na(df3_local[df3_local$name == "Andy", ][1, 2])) df4 <- agg(gd, sumAge = sum(df$age)) expect_is(df4, "SparkDataFrame") @@ -2144,7 +2178,7 @@ test_that("group by, agg functions", { df7 <- agg(gd2, value = "stddev") df7_local <- collect(df7) expect_true(abs(df7_local[df7_local$name == "ID1", ][1, 2] - 6.928203) < 1e-6) - expect_true(is.nan(df7_local[df7_local$name == "ID2", ][1, 2])) + expect_true(is.na(df7_local[df7_local$name == "ID2", ][1, 2])) mockLines3 <- c("{\"name\":\"Andy\", \"age\":30}", "{\"name\":\"Andy\", \"age\":30}", @@ -2696,6 +2730,19 @@ test_that("union(), unionByName(), rbind(), except(), and intersect() on a DataF expect_error(rbind(df, df2, df3), "Names of input data frames are different.") + + df4 <- unionByName(df2, select(df2, "age"), TRUE) + + expect_equal( + sum(collect( + select(df4, alias(isNull(df4$name), "missing_name") + ))$missing_name), + 3 + ) + + testthat::expect_error(unionByName(df2, select(df2, "age"), FALSE)) + testthat::expect_error(unionByName(df2, select(df2, "age"))) + excepted <- arrange(except(df, df2), desc(df$age)) expect_is(unioned, "SparkDataFrame") expect_equal(count(excepted), 2) @@ -3898,6 +3945,24 @@ test_that("catalog APIs, listTables, listColumns, listFunctions", { dropTempView("cars") }) +test_that("assert_true, raise_error", { + df <- read.json(jsonPath) + filtered <- filter(df, "age < 20") + + expect_equal(collect(select(filtered, assert_true(filtered$age < 20)))$age, c(NULL)) + expect_equal(collect(select(filtered, assert_true(filtered$age < 20, "error message")))$age, + c(NULL)) + expect_equal(collect(select(filtered, assert_true(filtered$age < 20, filtered$name)))$age, + c(NULL)) + expect_error(collect(select(df, assert_true(df$age < 20))), "is not true!") + expect_error(collect(select(df, assert_true(df$age < 20, "error message"))), + "error message") + expect_error(collect(select(df, assert_true(df$age < 20, df$name))), "Michael") + + expect_error(collect(select(filtered, raise_error("error message"))), "error message") + expect_error(collect(select(filtered, raise_error(filtered$name))), "Justin") +}) + compare_list <- function(list1, list2) { # get testthat to show the diff by first making the 2 lists equal in length expect_equal(length(list1), length(list2)) diff --git a/bin/find-spark-home.cmd b/bin/find-spark-home.cmd index f795d146d49c7..3149d05039ba4 100644 --- a/bin/find-spark-home.cmd +++ b/bin/find-spark-home.cmd @@ -55,6 +55,6 @@ if "x%SPARK_HOME%"=="x" ( set SPARK_HOME=%~dp0.. ) else ( rem We are pip installed, use the Python script to resolve a reasonable SPARK_HOME - for /f "delims=" %%i in ('%PYTHON_RUNNER% %FIND_SPARK_HOME_PYTHON_SCRIPT%') do set SPARK_HOME=%%i + for /f "delims=" %%i in ('%PYTHON_RUNNER% "%FIND_SPARK_HOME_PYTHON_SCRIPT%"') do set SPARK_HOME=%%i ) ) diff --git a/bin/load-spark-env.cmd b/bin/load-spark-env.cmd index fe725a4e1a368..5692af529fb66 100644 --- a/bin/load-spark-env.cmd +++ b/bin/load-spark-env.cmd @@ -24,7 +24,7 @@ rem conf\ subdirectory. if not defined SPARK_ENV_LOADED ( set SPARK_ENV_LOADED=1 - if [%SPARK_CONF_DIR%] == [] ( + if not defined SPARK_CONF_DIR ( set SPARK_CONF_DIR=%~dp0..\conf ) @@ -36,8 +36,8 @@ rem Setting SPARK_SCALA_VERSION if not already set. set SCALA_VERSION_1=2.13 set SCALA_VERSION_2=2.12 -set ASSEMBLY_DIR1=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_1% -set ASSEMBLY_DIR2=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_2% +set ASSEMBLY_DIR1="%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_1%" +set ASSEMBLY_DIR2="%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_2%" set ENV_VARIABLE_DOC=https://spark.apache.org/docs/latest/configuration.html#environment-variables if not defined SPARK_SCALA_VERSION ( diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd old mode 100644 new mode 100755 index 34d04c9856d2c..68b271d1d05d9 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -30,12 +30,12 @@ if "x%1"=="x" ( rem Find Spark jars. if exist "%SPARK_HOME%\jars" ( - set SPARK_JARS_DIR="%SPARK_HOME%\jars" + set SPARK_JARS_DIR=%SPARK_HOME%\jars ) else ( - set SPARK_JARS_DIR="%SPARK_HOME%\assembly\target\scala-%SPARK_SCALA_VERSION%\jars" + set SPARK_JARS_DIR=%SPARK_HOME%\assembly\target\scala-%SPARK_SCALA_VERSION%\jars ) -if not exist "%SPARK_JARS_DIR%"\ ( +if not exist "%SPARK_JARS_DIR%" ( echo Failed to find Spark jars directory. echo You need to build Spark before running this program. exit /b 1 diff --git a/build/sbt-launch-lib.bash b/build/sbt-launch-lib.bash index 162bfbf2257c7..423ba3b766e61 100755 --- a/build/sbt-launch-lib.bash +++ b/build/sbt-launch-lib.bash @@ -39,7 +39,7 @@ dlog () { acquire_sbt_jar () { SBT_VERSION=`awk -F "=" '/sbt\.version/ {print $2}' ./project/build.properties` - URL1=https://dl.bintray.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar + URL1=https://repo1.maven.org/maven2/org/scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch-${SBT_VERSION}.jar JAR=build/sbt-launch-${SBT_VERSION}.jar sbt_jar=$JAR diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java index 42e090bc83ed1..431c7e42774e4 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java @@ -164,8 +164,9 @@ public void clear() { } /** - * An alias class for the type "ConcurrentHashMap, Boolean>", which is used - * as a concurrent hashset for storing natural keys and the boolean value doesn't matter. + * An alias class for the type "{@literal ConcurrentHashMap, Boolean>}", + * which is used as a concurrent hashset for storing natural keys + * and the boolean value doesn't matter. */ private static class NaturalKeys extends ConcurrentHashMap, Boolean> {} diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 9d5bc9aae0719..d328a7de0a762 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -91,6 +91,10 @@ org.apache.commons commons-crypto + + org.roaringbitmap + RoaringBitmap + diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java index 490915f6de4b3..4fa191b3917e3 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java @@ -17,9 +17,11 @@ package org.apache.spark.network.protocol; +import java.io.IOException; import java.nio.charset.StandardCharsets; import io.netty.buffer.ByteBuf; +import org.roaringbitmap.RoaringBitmap; /** Provides a canonical set of Encoders for simple types. */ public class Encoders { @@ -44,6 +46,40 @@ public static String decode(ByteBuf buf) { } } + /** Bitmaps are encoded with their serialization length followed by the serialization bytes. */ + public static class Bitmaps { + public static int encodedLength(RoaringBitmap b) { + // Compress the bitmap before serializing it. Note that since BlockTransferMessage + // needs to invoke encodedLength first to figure out the length for the ByteBuf, it + // guarantees that the bitmap will always be compressed before being serialized. + b.trim(); + b.runOptimize(); + return b.serializedSizeInBytes(); + } + + public static void encode(ByteBuf buf, RoaringBitmap b) { + int encodedLength = b.serializedSizeInBytes(); + // RoaringBitmap requires nio ByteBuffer for serde. We expose the netty ByteBuf as a nio + // ByteBuffer. Here, we need to explicitly manage the index so we can write into the + // ByteBuffer, and the write is reflected in the underneath ByteBuf. + b.serialize(buf.nioBuffer(buf.writerIndex(), encodedLength)); + buf.writerIndex(buf.writerIndex() + encodedLength); + } + + public static RoaringBitmap decode(ByteBuf buf) { + RoaringBitmap bitmap = new RoaringBitmap(); + try { + bitmap.deserialize(buf.nioBuffer()); + // RoaringBitmap deserialize does not advance the reader index of the underlying ByteBuf. + // Manually update the index here. + buf.readerIndex(buf.readerIndex() + bitmap.serializedSizeInBytes()); + } catch (IOException e) { + throw new RuntimeException("Exception while decoding bitmap", e); + } + return bitmap; + } + } + /** Byte arrays are encoded with their length followed by bytes. */ public static class ByteArrays { public static int encodedLength(byte[] arr) { @@ -135,4 +171,31 @@ public static long[] decode(ByteBuf buf) { return longs; } } + + /** Bitmap arrays are encoded with the number of bitmaps followed by per-Bitmap encoding. */ + public static class BitmapArrays { + public static int encodedLength(RoaringBitmap[] bitmaps) { + int totalLength = 4; + for (RoaringBitmap b : bitmaps) { + totalLength += Bitmaps.encodedLength(b); + } + return totalLength; + } + + public static void encode(ByteBuf buf, RoaringBitmap[] bitmaps) { + buf.writeInt(bitmaps.length); + for (RoaringBitmap b : bitmaps) { + Bitmaps.encode(buf, b); + } + } + + public static RoaringBitmap[] decode(ByteBuf buf) { + int numBitmaps = buf.readInt(); + RoaringBitmap[] bitmaps = new RoaringBitmap[numBitmaps]; + for (int i = 0; i < bitmaps.length; i ++) { + bitmaps[i] = Bitmaps.decode(buf); + } + return bitmaps; + } + } } diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 00f1defbb0093..a4a1ff92ef9a0 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -57,6 +57,10 @@ com.google.guava guava + + org.roaringbitmap + RoaringBitmap + @@ -93,6 +97,11 @@ mockito-core test + + commons-io + commons-io + test + diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java index e762bd2071632..37befcd4b67fa 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java @@ -29,6 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.client.TransportClientFactory; @@ -135,4 +136,24 @@ public void onFailure(Throwable t) { hostLocalDirsCompletable.completeExceptionally(e); } } + + /** + * Push a sequence of shuffle blocks in a best-effort manner to a remote node asynchronously. + * These shuffle blocks, along with blocks pushed by other clients, will be merged into + * per-shuffle partition merged shuffle files on the destination node. + * + * @param host the host of the remote node. + * @param port the port of the remote node. + * @param blockIds block ids to be pushed + * @param buffers buffers to be pushed + * @param listener the listener to receive block push status. + */ + public void pushBlocks( + String host, + int port, + String[] blockIds, + ManagedBuffer[] buffers, + BlockFetchingListener listener) { + throw new UnsupportedOperationException(); + } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java new file mode 100644 index 0000000000000..308b0b7a6b33b --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.net.ConnectException; + +import com.google.common.base.Throwables; + +/** + * Plugs into {@link RetryingBlockFetcher} to further control when an exception should be retried + * and logged. + * Note: {@link RetryingBlockFetcher} will delegate the exception to this handler only when + * - remaining retries < max retries + * - exception is an IOException + */ + +public interface ErrorHandler { + + boolean shouldRetryError(Throwable t); + + default boolean shouldLogError(Throwable t) { + return true; + } + + /** + * A no-op error handler instance. + */ + ErrorHandler NOOP_ERROR_HANDLER = t -> true; + + /** + * The error handler for pushing shuffle blocks to remote shuffle services. + */ + class BlockPushErrorHandler implements ErrorHandler { + /** + * String constant used for generating exception messages indicating a block to be merged + * arrives too late on the server side, and also for later checking such exceptions on the + * client side. When we get a block push failure because of the block arrives too late, we + * will not retry pushing the block nor log the exception on the client side. + */ + public static final String TOO_LATE_MESSAGE_SUFFIX = + "received after merged shuffle is finalized"; + + /** + * String constant used for generating exception messages indicating the server couldn't + * append a block after all available attempts due to collision with other blocks belonging + * to the same shuffle partition, and also for later checking such exceptions on the client + * side. When we get a block push failure because of the block couldn't be written due to + * this reason, we will not log the exception on the client side. + */ + public static final String BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX = + "Couldn't find an opportunity to write block"; + + @Override + public boolean shouldRetryError(Throwable t) { + // If it is a connection time out or a connection closed exception, no need to retry. + if (t.getCause() != null && t.getCause() instanceof ConnectException) { + return false; + } + // If the block is too late, there is no need to retry it + return !Throwables.getStackTraceAsString(t).contains(TOO_LATE_MESSAGE_SUFFIX); + } + + @Override + public boolean shouldLogError(Throwable t) { + String errorStackTrace = Throwables.getStackTraceAsString(t); + return !errorStackTrace.contains(BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX) && + !errorStackTrace.contains(TOO_LATE_MESSAGE_SUFFIX); + } + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index 33865a21ea914..321b25305c504 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -32,6 +32,7 @@ import com.codahale.metrics.Timer; import com.codahale.metrics.Counter; import com.google.common.annotations.VisibleForTesting; +import org.apache.spark.network.client.StreamCallbackWithID; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,11 +62,21 @@ public class ExternalBlockHandler extends RpcHandler { final ExternalShuffleBlockResolver blockManager; private final OneForOneStreamManager streamManager; private final ShuffleMetrics metrics; + private final MergedShuffleFileManager mergeManager; public ExternalBlockHandler(TransportConf conf, File registeredExecutorFile) throws IOException { this(new OneForOneStreamManager(), - new ExternalShuffleBlockResolver(conf, registeredExecutorFile)); + new ExternalShuffleBlockResolver(conf, registeredExecutorFile), + new NoOpMergedShuffleFileManager()); + } + + public ExternalBlockHandler( + TransportConf conf, + File registeredExecutorFile, + MergedShuffleFileManager mergeManager) throws IOException { + this(new OneForOneStreamManager(), + new ExternalShuffleBlockResolver(conf, registeredExecutorFile), mergeManager); } @VisibleForTesting @@ -78,9 +89,19 @@ public ExternalShuffleBlockResolver getBlockResolver() { public ExternalBlockHandler( OneForOneStreamManager streamManager, ExternalShuffleBlockResolver blockManager) { + this(streamManager, blockManager, new NoOpMergedShuffleFileManager()); + } + + /** Enables mocking out the StreamManager, BlockManager, and MergeManager. */ + @VisibleForTesting + public ExternalBlockHandler( + OneForOneStreamManager streamManager, + ExternalShuffleBlockResolver blockManager, + MergedShuffleFileManager mergeManager) { this.metrics = new ShuffleMetrics(); this.streamManager = streamManager; this.blockManager = blockManager; + this.mergeManager = mergeManager; } @Override @@ -89,6 +110,21 @@ public void receive(TransportClient client, ByteBuffer message, RpcResponseCallb handleMessage(msgObj, client, callback); } + @Override + public StreamCallbackWithID receiveStream( + TransportClient client, + ByteBuffer messageHeader, + RpcResponseCallback callback) { + BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(messageHeader); + if (msgObj instanceof PushBlockStream) { + PushBlockStream message = (PushBlockStream) msgObj; + checkAuth(client, message.appId); + return mergeManager.receiveBlockDataAsStream(message); + } else { + throw new UnsupportedOperationException("Unexpected message with #receiveStream: " + msgObj); + } + } + protected void handleMessage( BlockTransferMessage msgObj, TransportClient client, @@ -139,6 +175,7 @@ protected void handleMessage( RegisterExecutor msg = (RegisterExecutor) msgObj; checkAuth(client, msg.appId); blockManager.registerExecutor(msg.appId, msg.execId, msg.executorInfo); + mergeManager.registerExecutor(msg.appId, msg.executorInfo.localDirs); callback.onSuccess(ByteBuffer.wrap(new byte[0])); } finally { responseDelayContext.stop(); @@ -156,6 +193,20 @@ protected void handleMessage( Map localDirs = blockManager.getLocalDirs(msg.appId, msg.execIds); callback.onSuccess(new LocalDirsForExecutors(localDirs).toByteBuffer()); + } else if (msgObj instanceof FinalizeShuffleMerge) { + final Timer.Context responseDelayContext = + metrics.finalizeShuffleMergeLatencyMillis.time(); + FinalizeShuffleMerge msg = (FinalizeShuffleMerge) msgObj; + try { + checkAuth(client, msg.appId); + MergeStatuses statuses = mergeManager.finalizeShuffleMerge(msg); + callback.onSuccess(statuses.toByteBuffer()); + } catch(IOException e) { + throw new RuntimeException(String.format("Error while finalizing shuffle merge " + + "for application %s shuffle %d", msg.appId, msg.shuffleId), e); + } finally { + responseDelayContext.stop(); + } } else { throw new UnsupportedOperationException("Unexpected message: " + msgObj); } @@ -225,6 +276,8 @@ public class ShuffleMetrics implements MetricSet { private final Timer openBlockRequestLatencyMillis = new Timer(); // Time latency for executor registration latency in ms private final Timer registerExecutorRequestLatencyMillis = new Timer(); + // Time latency for processing finalize shuffle merge request latency in ms + private final Timer finalizeShuffleMergeLatencyMillis = new Timer(); // Block transfer rate in byte per second private final Meter blockTransferRateBytes = new Meter(); // Number of active connections to the shuffle service @@ -236,6 +289,7 @@ public ShuffleMetrics() { allMetrics = new HashMap<>(); allMetrics.put("openBlockRequestLatencyMillis", openBlockRequestLatencyMillis); allMetrics.put("registerExecutorRequestLatencyMillis", registerExecutorRequestLatencyMillis); + allMetrics.put("finalizeShuffleMergeLatencyMillis", finalizeShuffleMergeLatencyMillis); allMetrics.put("blockTransferRateBytes", blockTransferRateBytes); allMetrics.put("registeredExecutorsSize", (Gauge) () -> blockManager.getRegisteredExecutorsSize()); @@ -373,6 +427,54 @@ public ManagedBuffer next() { } } + /** + * Dummy implementation of merged shuffle file manager. Suitable for when push-based shuffle + * is not enabled. + */ + private static class NoOpMergedShuffleFileManager implements MergedShuffleFileManager { + + @Override + public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { + throw new UnsupportedOperationException("Cannot handle shuffle block merge"); + } + + @Override + public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException { + throw new UnsupportedOperationException("Cannot handle shuffle block merge"); + } + + @Override + public void registerApplication(String appId, String user) { + // No-op. Do nothing. + } + + @Override + public void registerExecutor(String appId, String[] localDirs) { + // No-Op. Do nothing. + } + + @Override + public void applicationRemoved(String appId, boolean cleanupLocalDirs) { + throw new UnsupportedOperationException("Cannot handle shuffle block merge"); + } + + @Override + public ManagedBuffer getMergedBlockData( + String appId, int shuffleId, int reduceId, int chunkId) { + throw new UnsupportedOperationException("Cannot handle shuffle block merge"); + } + + @Override + public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) { + throw new UnsupportedOperationException("Cannot handle shuffle block merge"); + } + + @Override + public String[] getMergedBlockDirs(String appId) { + throw new UnsupportedOperationException("Cannot handle shuffle block merge"); + } + } + @Override public void channelActive(TransportClient client) { metrics.activeConnections.inc(); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java index 76e23e7c69d2d..eca35ed290467 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java @@ -20,21 +20,24 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import com.codahale.metrics.MetricSet; import com.google.common.collect.Lists; + +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.client.TransportClientBootstrap; -import org.apache.spark.network.shuffle.protocol.*; - -import org.apache.spark.network.TransportContext; import org.apache.spark.network.crypto.AuthClientBootstrap; import org.apache.spark.network.sasl.SecretKeyHolder; import org.apache.spark.network.server.NoOpRpcHandler; +import org.apache.spark.network.shuffle.protocol.*; import org.apache.spark.network.util.TransportConf; /** @@ -43,6 +46,8 @@ * (via BlockTransferService), which has the downside of losing the data if we lose the executors. */ public class ExternalBlockStoreClient extends BlockStoreClient { + private static final ErrorHandler PUSH_ERROR_HANDLER = new ErrorHandler.BlockPushErrorHandler(); + private final TransportConf conf; private final boolean authEnabled; private final SecretKeyHolder secretKeyHolder; @@ -90,12 +95,12 @@ public void fetchBlocks( try { int maxRetries = conf.maxIORetries(); RetryingBlockFetcher.BlockFetchStarter blockFetchStarter = - (blockIds1, listener1) -> { + (inputBlockId, inputListener) -> { // Unless this client is closed. if (clientFactory != null) { TransportClient client = clientFactory.createClient(host, port, maxRetries > 0); new OneForOneBlockFetcher(client, appId, execId, - blockIds1, listener1, conf, downloadFileManager).start(); + inputBlockId, inputListener, conf, downloadFileManager).start(); } else { logger.info("This clientFactory was closed. Skipping further block fetch retries."); } @@ -116,6 +121,43 @@ public void fetchBlocks( } } + @Override + public void pushBlocks( + String host, + int port, + String[] blockIds, + ManagedBuffer[] buffers, + BlockFetchingListener listener) { + checkInit(); + assert blockIds.length == buffers.length : "Number of block ids and buffers do not match."; + + Map buffersWithId = new HashMap<>(); + for (int i = 0; i < blockIds.length; i++) { + buffersWithId.put(blockIds[i], buffers[i]); + } + logger.debug("Push {} shuffle blocks to {}:{}", blockIds.length, host, port); + try { + RetryingBlockFetcher.BlockFetchStarter blockPushStarter = + (inputBlockId, inputListener) -> { + TransportClient client = clientFactory.createClient(host, port); + new OneForOneBlockPusher(client, appId, inputBlockId, inputListener, buffersWithId) + .start(); + }; + int maxRetries = conf.maxIORetries(); + if (maxRetries > 0) { + new RetryingBlockFetcher( + conf, blockPushStarter, blockIds, listener, PUSH_ERROR_HANDLER).start(); + } else { + blockPushStarter.createAndStart(blockIds, listener); + } + } catch (Exception e) { + logger.error("Exception while beginning pushBlocks", e); + for (String blockId : blockIds) { + listener.onBlockFetchFailure(blockId, e); + } + } + } + @Override public MetricSet shuffleMetrics() { checkInit(); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index a6bcbb8850566..a095bf2723418 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -92,10 +92,6 @@ public class ExternalShuffleBlockResolver { @VisibleForTesting final DB db; - private final List knownManagers = Arrays.asList( - "org.apache.spark.shuffle.sort.SortShuffleManager", - "org.apache.spark.shuffle.unsafe.UnsafeShuffleManager"); - public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorFile) throws IOException { this(conf, registeredExecutorFile, Executors.newSingleThreadExecutor( @@ -148,10 +144,6 @@ public void registerExecutor( ExecutorShuffleInfo executorInfo) { AppExecId fullId = new AppExecId(appId, execId); logger.info("Registered executor {} with {}", fullId, executorInfo); - if (!knownManagers.contains(executorInfo.shuffleManager)) { - throw new UnsupportedOperationException( - "Unsupported shuffle manager of executor: " + executorInfo); - } try { if (db != null) { byte[] key = dbAppExecKey(fullId); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlockMeta.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlockMeta.java new file mode 100644 index 0000000000000..e9d9e53495469 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlockMeta.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import com.google.common.base.Preconditions; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.roaringbitmap.RoaringBitmap; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.protocol.Encoders; + +/** + * Contains meta information for a merged block. Currently this information constitutes: + * 1. Number of chunks in a merged shuffle block. + * 2. Bitmaps for each chunk in the merged block. A chunk bitmap contains all the mapIds that were + * merged to that merged block chunk. + */ +public class MergedBlockMeta { + private final int numChunks; + private final ManagedBuffer chunksBitmapBuffer; + + public MergedBlockMeta(int numChunks, ManagedBuffer chunksBitmapBuffer) { + this.numChunks = numChunks; + this.chunksBitmapBuffer = Preconditions.checkNotNull(chunksBitmapBuffer); + } + + public int getNumChunks() { + return numChunks; + } + + public ManagedBuffer getChunksBitmapBuffer() { + return chunksBitmapBuffer; + } + + public RoaringBitmap[] readChunkBitmaps() throws IOException { + ByteBuf buf = Unpooled.wrappedBuffer(chunksBitmapBuffer.nioByteBuffer()); + List bitmaps = new ArrayList<>(); + while(buf.isReadable()) { + bitmaps.add(Encoders.Bitmaps.decode(buf)); + } + assert (bitmaps.size() == numChunks); + return bitmaps.toArray(new RoaringBitmap[0]); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java new file mode 100644 index 0000000000000..ef4dbb2bd0059 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.IOException; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.client.StreamCallbackWithID; +import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge; +import org.apache.spark.network.shuffle.protocol.MergeStatuses; +import org.apache.spark.network.shuffle.protocol.PushBlockStream; + + +/** + * The MergedShuffleFileManager is used to process push based shuffle when enabled. It works + * along side {@link ExternalBlockHandler} and serves as an RPCHandler for + * {@link org.apache.spark.network.server.RpcHandler#receiveStream}, where it processes the + * remotely pushed streams of shuffle blocks to merge them into merged shuffle files. Right + * now, support for push based shuffle is only implemented for external shuffle service in + * YARN mode. + */ +public interface MergedShuffleFileManager { + /** + * Provides the stream callback used to process a remotely pushed block. The callback is + * used by the {@link org.apache.spark.network.client.StreamInterceptor} installed on the + * channel to process the block data in the channel outside of the message frame. + * + * @param msg metadata of the remotely pushed blocks. This is processed inside the message frame + * @return A stream callback to process the block data in streaming fashion as it arrives + */ + StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg); + + /** + * Handles the request to finalize shuffle merge for a given shuffle. + * + * @param msg contains appId and shuffleId to uniquely identify a shuffle to be finalized + * @return The statuses of the merged shuffle partitions for the given shuffle on this + * shuffle service + * @throws IOException + */ + MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException; + + /** + * Registers an application when it starts. It also stores the username which is necessary + * for generating the host local directories for merged shuffle files. + * Right now, this is invoked by YarnShuffleService. + * + * @param appId application ID + * @param user username + */ + void registerApplication(String appId, String user); + + /** + * Registers an executor with its local dir list when it starts. This provides the specific path + * so MergedShuffleFileManager knows where to store and look for shuffle data for a + * given application. It is invoked by the RPC call when executor tries to register with the + * local shuffle service. + * + * @param appId application ID + * @param localDirs The list of local dirs that this executor gets granted from NodeManager + */ + void registerExecutor(String appId, String[] localDirs); + + /** + * Invoked when an application finishes. This cleans up any remaining metadata associated with + * this application, and optionally deletes the application specific directory path. + * + * @param appId application ID + * @param cleanupLocalDirs flag indicating whether MergedShuffleFileManager should handle + * deletion of local dirs itself. + */ + void applicationRemoved(String appId, boolean cleanupLocalDirs); + + /** + * Get the buffer for a given merged shuffle chunk when serving merged shuffle to reducers + * + * @param appId application ID + * @param shuffleId shuffle ID + * @param reduceId reducer ID + * @param chunkId merged shuffle file chunk ID + * @return The {@link ManagedBuffer} for the given merged shuffle chunk + */ + ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId); + + /** + * Get the meta information of a merged block. + * + * @param appId application ID + * @param shuffleId shuffle ID + * @param reduceId reducer ID + * @return meta information of a merged block + */ + MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId); + + /** + * Get the local directories which stores the merged shuffle files. + * + * @param appId application ID + */ + String[] getMergedBlockDirs(String appId); +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java new file mode 100644 index 0000000000000..407b248170a46 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.shuffle.protocol.PushBlockStream; + +/** + * Similar to {@link OneForOneBlockFetcher}, but for pushing blocks to remote shuffle service to + * be merged instead of for fetching them from remote shuffle services. This is used by + * ShuffleWriter when the block push process is initiated. The supplied BlockFetchingListener + * is used to handle the success or failure in pushing each blocks. + */ +public class OneForOneBlockPusher { + private static final Logger logger = LoggerFactory.getLogger(OneForOneBlockPusher.class); + private static final ErrorHandler PUSH_ERROR_HANDLER = new ErrorHandler.BlockPushErrorHandler(); + + private final TransportClient client; + private final String appId; + private final String[] blockIds; + private final BlockFetchingListener listener; + private final Map buffers; + + public OneForOneBlockPusher( + TransportClient client, + String appId, + String[] blockIds, + BlockFetchingListener listener, + Map buffers) { + this.client = client; + this.appId = appId; + this.blockIds = blockIds; + this.listener = listener; + this.buffers = buffers; + } + + private class BlockPushCallback implements RpcResponseCallback { + + private int index; + private String blockId; + + BlockPushCallback(int index, String blockId) { + this.index = index; + this.blockId = blockId; + } + + @Override + public void onSuccess(ByteBuffer response) { + // On receipt of a successful block push + listener.onBlockFetchSuccess(blockId, new NioManagedBuffer(ByteBuffer.allocate(0))); + } + + @Override + public void onFailure(Throwable e) { + // Since block push is best effort, i.e., if we encountered a block push failure that's not + // retriable or exceeding the max retires, we should not fail all remaining block pushes. + // The best effort nature makes block push tolerable of a partial completion. Thus, we only + // fail the block that's actually failed. Not that, on the RetryingBlockFetcher side, once + // retry is initiated, it would still invalidate the previous active retry listener, and + // retry all outstanding blocks. We are preventing forwarding unnecessary block push failures + // to the parent listener of the retry listener. The only exceptions would be if the block + // push failure is due to block arriving on the server side after merge finalization, or the + // client fails to establish connection to the server side. In both cases, we would fail all + // remaining blocks. + if (PUSH_ERROR_HANDLER.shouldRetryError(e)) { + String[] targetBlockId = Arrays.copyOfRange(blockIds, index, index + 1); + failRemainingBlocks(targetBlockId, e); + } else { + String[] targetBlockId = Arrays.copyOfRange(blockIds, index, blockIds.length); + failRemainingBlocks(targetBlockId, e); + } + } + } + + private void failRemainingBlocks(String[] failedBlockIds, Throwable e) { + for (String blockId : failedBlockIds) { + try { + listener.onBlockFetchFailure(blockId, e); + } catch (Exception e2) { + logger.error("Error in block push failure callback", e2); + } + } + } + + /** + * Begins the block pushing process, calling the listener with every block pushed. + */ + public void start() { + logger.debug("Start pushing {} blocks", blockIds.length); + for (int i = 0; i < blockIds.length; i++) { + assert buffers.containsKey(blockIds[i]) : "Could not find the block buffer for block " + + blockIds[i]; + ByteBuffer header = new PushBlockStream(appId, blockIds[i], i).toByteBuffer(); + client.uploadStream(new NioManagedBuffer(header), buffers.get(blockIds[i]), + new BlockPushCallback(i, blockIds[i])); + } + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java index 6bf3da94030d4..43bde1610e41e 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java @@ -99,11 +99,14 @@ void createAndStart(String[] blockIds, BlockFetchingListener listener) */ private RetryingBlockFetchListener currentListener; + private final ErrorHandler errorHandler; + public RetryingBlockFetcher( TransportConf conf, RetryingBlockFetcher.BlockFetchStarter fetchStarter, String[] blockIds, - BlockFetchingListener listener) { + BlockFetchingListener listener, + ErrorHandler errorHandler) { this.fetchStarter = fetchStarter; this.listener = listener; this.maxRetries = conf.maxIORetries(); @@ -111,6 +114,15 @@ public RetryingBlockFetcher( this.outstandingBlocksIds = Sets.newLinkedHashSet(); Collections.addAll(outstandingBlocksIds, blockIds); this.currentListener = new RetryingBlockFetchListener(); + this.errorHandler = errorHandler; + } + + public RetryingBlockFetcher( + TransportConf conf, + BlockFetchStarter fetchStarter, + String[] blockIds, + BlockFetchingListener listener) { + this(conf, fetchStarter, blockIds, listener, ErrorHandler.NOOP_ERROR_HANDLER); } /** @@ -178,7 +190,7 @@ private synchronized boolean shouldRetry(Throwable e) { boolean isIOException = e instanceof IOException || (e.getCause() != null && e.getCause() instanceof IOException); boolean hasRemainingRetries = retryCount < maxRetries; - return isIOException && hasRemainingRetries; + return isIOException && hasRemainingRetries && errorHandler.shouldRetryError(e); } /** @@ -215,8 +227,15 @@ public void onBlockFetchFailure(String blockId, Throwable exception) { if (shouldRetry(exception)) { initiateRetry(); } else { - logger.error(String.format("Failed to fetch block %s, and will not retry (%s retries)", - blockId, retryCount), exception); + if (errorHandler.shouldLogError(exception)) { + logger.error( + String.format("Failed to fetch block %s, and will not retry (%s retries)", + blockId, retryCount), exception); + } else { + logger.debug( + String.format("Failed to fetch block %s, and will not retry (%s retries)", + blockId, retryCount), exception); + } outstandingBlocksIds.remove(blockId); shouldForwardFailure = true; } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java index 89d8dfe8716b8..7f5058124988f 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java @@ -47,7 +47,8 @@ public abstract class BlockTransferMessage implements Encodable { public enum Type { OPEN_BLOCKS(0), UPLOAD_BLOCK(1), REGISTER_EXECUTOR(2), STREAM_HANDLE(3), REGISTER_DRIVER(4), HEARTBEAT(5), UPLOAD_BLOCK_STREAM(6), REMOVE_BLOCKS(7), BLOCKS_REMOVED(8), - FETCH_SHUFFLE_BLOCKS(9), GET_LOCAL_DIRS_FOR_EXECUTORS(10), LOCAL_DIRS_FOR_EXECUTORS(11); + FETCH_SHUFFLE_BLOCKS(9), GET_LOCAL_DIRS_FOR_EXECUTORS(10), LOCAL_DIRS_FOR_EXECUTORS(11), + PUSH_BLOCK_STREAM(12), FINALIZE_SHUFFLE_MERGE(13), MERGE_STATUSES(14); private final byte id; @@ -78,6 +79,9 @@ public static BlockTransferMessage fromByteBuffer(ByteBuffer msg) { case 9: return FetchShuffleBlocks.decode(buf); case 10: return GetLocalDirsForExecutors.decode(buf); case 11: return LocalDirsForExecutors.decode(buf); + case 12: return PushBlockStream.decode(buf); + case 13: return FinalizeShuffleMerge.decode(buf); + case 14: return MergeStatuses.decode(buf); default: throw new IllegalArgumentException("Unknown message type: " + type); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java new file mode 100644 index 0000000000000..9058575df57ef --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.protocol.Encoders; + +/** + * Request to finalize merge for a given shuffle. + * Returns {@link MergeStatuses} + */ +public class FinalizeShuffleMerge extends BlockTransferMessage { + public final String appId; + public final int shuffleId; + + public FinalizeShuffleMerge( + String appId, + int shuffleId) { + this.appId = appId; + this.shuffleId = shuffleId; + } + + @Override + protected BlockTransferMessage.Type type() { + return Type.FINALIZE_SHUFFLE_MERGE; + } + + @Override + public int hashCode() { + return Objects.hashCode(appId, shuffleId); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("shuffleId", shuffleId) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof FinalizeShuffleMerge) { + FinalizeShuffleMerge o = (FinalizeShuffleMerge) other; + return Objects.equal(appId, o.appId) + && shuffleId == o.shuffleId; + } + return false; + } + + @Override + public int encodedLength() { + return Encoders.Strings.encodedLength(appId) + 4; + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, appId); + buf.writeInt(shuffleId); + } + + public static FinalizeShuffleMerge decode(ByteBuf buf) { + String appId = Encoders.Strings.decode(buf); + int shuffleId = buf.readInt(); + return new FinalizeShuffleMerge(appId, shuffleId); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java new file mode 100644 index 0000000000000..f57e8b326e5e2 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle.protocol; + +import java.util.Arrays; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; +import org.roaringbitmap.RoaringBitmap; + +import org.apache.spark.network.protocol.Encoders; + +/** + * Result returned by an ExternalShuffleService to the DAGScheduler. This represents the result + * of all the remote shuffle block merge operations performed by an ExternalShuffleService + * for a given shuffle ID. It includes the shuffle ID, an array of bitmaps each representing + * the set of mapper partition blocks that are merged for a given reducer partition, an array + * of reducer IDs, and an array of merged shuffle partition sizes. The 3 arrays list information + * about all the reducer partitions merged by the ExternalShuffleService in the same order. + */ +public class MergeStatuses extends BlockTransferMessage { + /** Shuffle ID **/ + public final int shuffleId; + /** + * Array of bitmaps tracking the set of mapper partition blocks merged for each + * reducer partition + */ + public final RoaringBitmap[] bitmaps; + /** Array of reducer IDs **/ + public final int[] reduceIds; + /** + * Array of merged shuffle partition block size. Each represents the total size of all + * merged shuffle partition blocks for one reducer partition. + * **/ + public final long[] sizes; + + public MergeStatuses( + int shuffleId, + RoaringBitmap[] bitmaps, + int[] reduceIds, + long[] sizes) { + this.shuffleId = shuffleId; + this.bitmaps = bitmaps; + this.reduceIds = reduceIds; + this.sizes = sizes; + } + + @Override + protected Type type() { + return Type.MERGE_STATUSES; + } + + @Override + public int hashCode() { + int objectHashCode = Objects.hashCode(shuffleId); + return (objectHashCode * 41 + Arrays.hashCode(reduceIds) * 41 + + Arrays.hashCode(bitmaps) * 41 + Arrays.hashCode(sizes)); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("shuffleId", shuffleId) + .add("reduceId size", reduceIds.length) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof MergeStatuses) { + MergeStatuses o = (MergeStatuses) other; + return Objects.equal(shuffleId, o.shuffleId) + && Arrays.equals(bitmaps, o.bitmaps) + && Arrays.equals(reduceIds, o.reduceIds) + && Arrays.equals(sizes, o.sizes); + } + return false; + } + + @Override + public int encodedLength() { + return 4 // int + + Encoders.BitmapArrays.encodedLength(bitmaps) + + Encoders.IntArrays.encodedLength(reduceIds) + + Encoders.LongArrays.encodedLength(sizes); + } + + @Override + public void encode(ByteBuf buf) { + buf.writeInt(shuffleId); + Encoders.BitmapArrays.encode(buf, bitmaps); + Encoders.IntArrays.encode(buf, reduceIds); + Encoders.LongArrays.encode(buf, sizes); + } + + public static MergeStatuses decode(ByteBuf buf) { + int shuffleId = buf.readInt(); + RoaringBitmap[] bitmaps = Encoders.BitmapArrays.decode(buf); + int[] reduceIds = Encoders.IntArrays.decode(buf); + long[] sizes = Encoders.LongArrays.decode(buf); + return new MergeStatuses(shuffleId, bitmaps, reduceIds, sizes); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java new file mode 100644 index 0000000000000..7eab5a644783c --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.protocol.Encoders; + +// Needed by ScalaDoc. See SPARK-7726 +import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; + + +/** + * Request to push a block to a remote shuffle service to be merged in push based shuffle. + * The remote shuffle service will also include this message when responding the push requests. + */ +public class PushBlockStream extends BlockTransferMessage { + public final String appId; + public final String blockId; + // Similar to the chunkIndex in StreamChunkId, indicating the index of a block in a batch of + // blocks to be pushed. + public final int index; + + public PushBlockStream(String appId, String blockId, int index) { + this.appId = appId; + this.blockId = blockId; + this.index = index; + } + + @Override + protected Type type() { + return Type.PUSH_BLOCK_STREAM; + } + + @Override + public int hashCode() { + return Objects.hashCode(appId, blockId, index); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("blockId", blockId) + .add("index", index) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof PushBlockStream) { + PushBlockStream o = (PushBlockStream) other; + return Objects.equal(appId, o.appId) + && Objects.equal(blockId, o.blockId) + && index == o.index; + } + return false; + } + + @Override + public int encodedLength() { + return Encoders.Strings.encodedLength(appId) + + Encoders.Strings.encodedLength(blockId) + 4; + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, appId); + Encoders.Strings.encode(buf, blockId); + buf.writeInt(index); + } + + public static PushBlockStream decode(ByteBuf buf) { + String appId = Encoders.Strings.decode(buf); + String blockId = Encoders.Strings.decode(buf); + int index = buf.readInt(); + return new PushBlockStream(appId, blockId, index); + } +} diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java new file mode 100644 index 0000000000000..992e7762c5a54 --- /dev/null +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.net.ConnectException; + +import org.junit.Test; + +import static org.junit.Assert.*; + +/** + * Test suite for {@link ErrorHandler} + */ +public class ErrorHandlerSuite { + + @Test + public void testPushErrorRetry() { + ErrorHandler.BlockPushErrorHandler handler = new ErrorHandler.BlockPushErrorHandler(); + assertFalse(handler.shouldRetryError(new RuntimeException(new IllegalArgumentException( + ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))); + assertFalse(handler.shouldRetryError(new RuntimeException(new ConnectException()))); + assertTrue(handler.shouldRetryError(new RuntimeException(new IllegalArgumentException( + ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX)))); + assertTrue(handler.shouldRetryError(new Throwable())); + } + + @Test + public void testPushErrorLogging() { + ErrorHandler.BlockPushErrorHandler handler = new ErrorHandler.BlockPushErrorHandler(); + assertFalse(handler.shouldLogError(new RuntimeException(new IllegalArgumentException( + ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))); + assertFalse(handler.shouldLogError(new RuntimeException(new IllegalArgumentException( + ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX)))); + assertTrue(handler.shouldLogError(new Throwable())); + } +} diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java index 455351fcf767c..680b8d74a2eea 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java @@ -17,6 +17,7 @@ package org.apache.spark.network.shuffle; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.Iterator; @@ -25,6 +26,7 @@ import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; +import org.roaringbitmap.RoaringBitmap; import static org.junit.Assert.*; import static org.mockito.ArgumentMatchers.any; @@ -39,6 +41,8 @@ import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.apache.spark.network.shuffle.protocol.FetchShuffleBlocks; +import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge; +import org.apache.spark.network.shuffle.protocol.MergeStatuses; import org.apache.spark.network.shuffle.protocol.OpenBlocks; import org.apache.spark.network.shuffle.protocol.RegisterExecutor; import org.apache.spark.network.shuffle.protocol.StreamHandle; @@ -50,6 +54,7 @@ public class ExternalBlockHandlerSuite { OneForOneStreamManager streamManager; ExternalShuffleBlockResolver blockResolver; RpcHandler handler; + MergedShuffleFileManager mergedShuffleManager; ManagedBuffer[] blockMarkers = { new NioManagedBuffer(ByteBuffer.wrap(new byte[3])), new NioManagedBuffer(ByteBuffer.wrap(new byte[7])) @@ -59,17 +64,20 @@ public class ExternalBlockHandlerSuite { public void beforeEach() { streamManager = mock(OneForOneStreamManager.class); blockResolver = mock(ExternalShuffleBlockResolver.class); - handler = new ExternalBlockHandler(streamManager, blockResolver); + mergedShuffleManager = mock(MergedShuffleFileManager.class); + handler = new ExternalBlockHandler(streamManager, blockResolver, mergedShuffleManager); } @Test public void testRegisterExecutor() { RpcResponseCallback callback = mock(RpcResponseCallback.class); - ExecutorShuffleInfo config = new ExecutorShuffleInfo(new String[] {"/a", "/b"}, 16, "sort"); + String[] localDirs = new String[] {"/a", "/b"}; + ExecutorShuffleInfo config = new ExecutorShuffleInfo(localDirs, 16, "sort"); ByteBuffer registerMessage = new RegisterExecutor("app0", "exec1", config).toByteBuffer(); handler.receive(client, registerMessage, callback); verify(blockResolver, times(1)).registerExecutor("app0", "exec1", config); + verify(mergedShuffleManager, times(1)).registerExecutor("app0", localDirs); verify(callback, times(1)).onSuccess(any(ByteBuffer.class)); verify(callback, never()).onFailure(any(Throwable.class)); @@ -222,4 +230,32 @@ public void testBadMessages() { verify(callback, never()).onSuccess(any(ByteBuffer.class)); verify(callback, never()).onFailure(any(Throwable.class)); } + + @Test + public void testFinalizeShuffleMerge() throws IOException { + RpcResponseCallback callback = mock(RpcResponseCallback.class); + + FinalizeShuffleMerge req = new FinalizeShuffleMerge("app0", 0); + RoaringBitmap bitmap = RoaringBitmap.bitmapOf(0, 1, 2); + MergeStatuses statuses = new MergeStatuses(0, new RoaringBitmap[]{bitmap}, + new int[]{3}, new long[]{30}); + when(mergedShuffleManager.finalizeShuffleMerge(req)).thenReturn(statuses); + + ByteBuffer reqBuf = req.toByteBuffer(); + handler.receive(client, reqBuf, callback); + verify(mergedShuffleManager, times(1)).finalizeShuffleMerge(req); + ArgumentCaptor response = ArgumentCaptor.forClass(ByteBuffer.class); + verify(callback, times(1)).onSuccess(response.capture()); + verify(callback, never()).onFailure(any()); + + MergeStatuses mergeStatuses = + (MergeStatuses) BlockTransferMessage.Decoder.fromByteBuffer(response.getValue()); + assertEquals(mergeStatuses, statuses); + + Timer finalizeShuffleMergeLatencyMillis = (Timer) ((ExternalBlockHandler) handler) + .getAllMetrics() + .getMetrics() + .get("finalizeShuffleMergeLatencyMillis"); + assertEquals(1, finalizeShuffleMergeLatencyMillis.getCount()); + } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index 88bcf43c2371f..04d4bdf92bae7 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -71,15 +71,6 @@ public void testBadRequests() throws IOException { assertTrue("Bad error message: " + e, e.getMessage().contains("not registered")); } - // Invalid shuffle manager - try { - resolver.registerExecutor("app0", "exec2", dataContext.createExecutorInfo("foobar")); - resolver.getBlockData("app0", "exec2", 1, 1, 0); - fail("Should have failed"); - } catch (UnsupportedOperationException e) { - // pass - } - // Nonexistent shuffle block resolver.registerExecutor("app0", "exec3", dataContext.createExecutorInfo(SORT_MANAGER)); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 9d398e372056b..49d02e5dc6fb4 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -233,9 +233,9 @@ public void testFetchThreeSort() throws Exception { exec0Fetch.releaseBuffers(); } - @Test (expected = RuntimeException.class) - public void testRegisterInvalidExecutor() throws Exception { - registerExecutor("exec-1", dataContext0.createExecutorInfo("unknown sort manager")); + @Test + public void testRegisterWithCustomShuffleManager() throws Exception { + registerExecutor("exec-1", dataContext0.createExecutorInfo("custom shuffle manager")); } @Test diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java new file mode 100644 index 0000000000000..ebcdba72aa1a8 --- /dev/null +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.Map; + +import com.google.common.collect.Maps; +import io.netty.buffer.Unpooled; +import org.junit.Test; + +import static org.junit.Assert.*; +import static org.mockito.AdditionalMatchers.*; +import static org.mockito.Mockito.*; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NettyManagedBuffer; +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; +import org.apache.spark.network.shuffle.protocol.PushBlockStream; + + +public class OneForOneBlockPusherSuite { + + @Test + public void testPushOne() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[1]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = pushBlocks( + blocks, + blockIds, + Arrays.asList(new PushBlockStream("app-id", "shuffle_0_0_0", 0))); + + verify(listener).onBlockFetchSuccess(eq("shuffle_0_0_0"), any()); + } + + @Test + public void testPushThree() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("b1", new NioManagedBuffer(ByteBuffer.wrap(new byte[23]))); + blocks.put("b2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[23]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = pushBlocks( + blocks, + blockIds, + Arrays.asList(new PushBlockStream("app-id", "b0", 0), + new PushBlockStream("app-id", "b1", 1), + new PushBlockStream("app-id", "b2", 2))); + + for (int i = 0; i < 3; i ++) { + verify(listener, times(1)).onBlockFetchSuccess(eq("b" + i), any()); + } + } + + @Test + public void testServerFailures() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("b1", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); + blocks.put("b2", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = pushBlocks( + blocks, + blockIds, + Arrays.asList(new PushBlockStream("app-id", "b0", 0), + new PushBlockStream("app-id", "b1", 1), + new PushBlockStream("app-id", "b2", 2))); + + verify(listener, times(1)).onBlockFetchSuccess(eq("b0"), any()); + verify(listener, times(1)).onBlockFetchFailure(eq("b1"), any()); + verify(listener, times(1)).onBlockFetchFailure(eq("b2"), any()); + } + + @Test + public void testHandlingRetriableFailures() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("b1", null); + blocks.put("b2", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = pushBlocks( + blocks, + blockIds, + Arrays.asList(new PushBlockStream("app-id", "b0", 0), + new PushBlockStream("app-id", "b1", 1), + new PushBlockStream("app-id", "b2", 2))); + + verify(listener, times(1)).onBlockFetchSuccess(eq("b0"), any()); + verify(listener, times(0)).onBlockFetchSuccess(not(eq("b0")), any()); + verify(listener, times(0)).onBlockFetchFailure(eq("b0"), any()); + verify(listener, times(1)).onBlockFetchFailure(eq("b1"), any()); + verify(listener, times(2)).onBlockFetchFailure(eq("b2"), any()); + } + + /** + * Begins a push on the given set of blocks by mocking the response from server side. + * If a block is an empty byte, a server side retriable exception will be thrown. + * If a block is null, a non-retriable exception will be thrown. + */ + private static BlockFetchingListener pushBlocks( + LinkedHashMap blocks, + String[] blockIds, + Iterable expectMessages) { + TransportClient client = mock(TransportClient.class); + BlockFetchingListener listener = mock(BlockFetchingListener.class); + OneForOneBlockPusher pusher = + new OneForOneBlockPusher(client, "app-id", blockIds, listener, blocks); + + Iterator> blockIterator = blocks.entrySet().iterator(); + Iterator msgIterator = expectMessages.iterator(); + doAnswer(invocation -> { + ByteBuffer header = ((ManagedBuffer) invocation.getArguments()[0]).nioByteBuffer(); + BlockTransferMessage message = BlockTransferMessage.Decoder.fromByteBuffer(header); + RpcResponseCallback callback = (RpcResponseCallback) invocation.getArguments()[2]; + Map.Entry entry = blockIterator.next(); + ManagedBuffer block = entry.getValue(); + if (block != null && block.nioByteBuffer().capacity() > 0) { + callback.onSuccess(header); + } else if (block != null) { + callback.onFailure(new RuntimeException("Failed " + entry.getKey() + + ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX)); + } else { + callback.onFailure(new RuntimeException("Quick fail " + entry.getKey() + + ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)); + } + assertEquals(msgIterator.next(), message); + return null; + }).when(client).uploadStream(any(ManagedBuffer.class), any(), any(RpcResponseCallback.class)); + + pusher.start(); + return listener; + } +} diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/Murmur3_x86_32.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/Murmur3_x86_32.java index e83b331391e39..61cd2cec1a34b 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/Murmur3_x86_32.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/Murmur3_x86_32.java @@ -17,12 +17,16 @@ package org.apache.spark.util.sketch; +import java.nio.ByteOrder; + /** * 32-bit Murmur3 hasher. This is based on Guava's Murmur3_32HashFunction. */ // This class is duplicated from `org.apache.spark.unsafe.hash.Murmur3_x86_32` to make sure // spark-sketch has no external dependencies. final class Murmur3_x86_32 { + private static final boolean isBigEndian = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); + private static final int C1 = 0xcc9e2d51; private static final int C2 = 0x1b873593; @@ -92,8 +96,10 @@ private static int hashBytesByInt(Object base, long offset, int lengthInBytes, i int h1 = seed; for (int i = 0; i < lengthInBytes; i += 4) { int halfWord = Platform.getInt(base, offset + i); - int k1 = mixK1(halfWord); - h1 = mixH1(h1, k1); + if (isBigEndian) { + halfWord = Integer.reverseBytes(halfWord); + } + h1 = mixH1(h1, mixK1(halfWord)); } return h1; } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index d239de6083ad0..0b9d9ced312a1 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -17,12 +17,16 @@ package org.apache.spark.unsafe.hash; +import java.nio.ByteOrder; + import org.apache.spark.unsafe.Platform; /** * 32-bit Murmur3 hasher. This is based on Guava's Murmur3_32HashFunction. */ public final class Murmur3_x86_32 { + private static final boolean isBigEndian = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); + private static final int C1 = 0xcc9e2d51; private static final int C2 = 0x1b873593; @@ -92,8 +96,10 @@ private static int hashBytesByInt(Object base, long offset, int lengthInBytes, i int h1 = seed; for (int i = 0; i < lengthInBytes; i += 4) { int halfWord = Platform.getInt(base, offset + i); - int k1 = mixK1(halfWord); - h1 = mixH1(h1, k1); + if (isBigEndian) { + halfWord = Integer.reverseBytes(halfWord); + } + h1 = mixH1(h1, mixK1(halfWord)); } return h1; } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 43bd7976c5d33..b8dda22240042 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -563,7 +563,7 @@ public UTF8String trim() { } /** - * Trims whitespaces (<= ASCII 32) from both ends of this string. + * Trims whitespaces ({@literal <=} ASCII 32) from both ends of this string. * * Note that, this method is the same as java's {@link String#trim}, and different from * {@link UTF8String#trim()} which remove only spaces(= ASCII 32) from both ends. diff --git a/core/src/main/java/org/apache/spark/shuffle/api/metadata/ShuffleOutputTracker.java b/core/src/main/java/org/apache/spark/shuffle/api/metadata/ShuffleOutputTracker.java index bff143da574e1..55b8457398ff1 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/metadata/ShuffleOutputTracker.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/metadata/ShuffleOutputTracker.java @@ -17,8 +17,6 @@ package org.apache.spark.shuffle.api.metadata; -import org.apache.spark.shuffle.api.ShuffleDriverComponents; - /** * :: Private :: * diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index f98cff46338ea..d3f0feee377a2 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -273,7 +273,7 @@ private MapOutputCommitMessage mergeSpills(SpillInfo[] spills) throws IOExceptio if (spills.length == 0) { final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents .createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions()); - return mapWriter.commitAllPartitions(); + mapOutputCommitMessage = mapWriter.commitAllPartitions(); } else if (spills.length == 1) { Optional maybeSingleFileWriter = shuffleExecutorComponents.createSingleFileMapOutputWriter(shuffleId, mapId); @@ -281,7 +281,7 @@ private MapOutputCommitMessage mergeSpills(SpillInfo[] spills) throws IOExceptio // Here, we don't need to perform any metrics updates because the bytes written to this // output file would have already been counted as shuffle bytes written. long[] partitionLengths = spills[0].partitionLengths; - logger.debug("Merge shuffle spills for mapId {} with length {}", mapId, + logger.debug("Transfer shuffle spills for mapId {} with length {}", mapId, partitionLengths.length); mapOutputCommitMessage = maybeSingleFileWriter.get().transferMapSpillFile( spills[0].file, partitionLengths); diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 8eea9db393aff..d7940fc08e1a5 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -808,12 +808,21 @@ public boolean append(Object kbase, long koff, int klen, Object vbase, long voff longArray.set(pos * 2 + 1, keyHashcode); isDefined = true; - // We use two array entries per key, so the array size is twice the capacity. - // We should compare the current capacity of the array, instead of its size. - if (numKeys >= growthThreshold && longArray.size() / 2 < MAX_CAPACITY) { - try { - growAndRehash(); - } catch (SparkOutOfMemoryError oom) { + // If the map has reached its growth threshold, try to grow it. + if (numKeys >= growthThreshold) { + // We use two array entries per key, so the array size is twice the capacity. + // We should compare the current capacity of the array, instead of its size. + if (longArray.size() / 2 < MAX_CAPACITY) { + try { + growAndRehash(); + } catch (SparkOutOfMemoryError oom) { + canGrowArray = false; + } + } else { + // The map is already at MAX_CAPACITY and cannot grow. Instead, we prevent it from + // accepting any more new elements to make sure we don't exceed the load factor. If we + // need to spill later, this allows UnsafeKVExternalSorter to reuse the array for + // sorting. canGrowArray = false; } } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 55e4e609c3c7b..dda8ed4c239ae 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -203,6 +203,10 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { } if (inMemSorter == null || inMemSorter.numRecords() <= 0) { + // There could still be some memory allocated when there are no records in the in-memory + // sorter. We will not spill it however, to ensure that we can always process at least one + // record before spilling. See the comments in `allocateMemoryForRecordIfNecessary` for why + // this is necessary. return 0L; } @@ -224,7 +228,7 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { // Note that this is more-or-less going to be a multiple of the page size, so wasted space in // pages will currently be counted as memory spilled even though that space isn't actually // written to disk. This also counts the space needed to store the sorter's pointer array. - inMemSorter.reset(); + inMemSorter.freeMemory(); // Reset the in-memory sorter's pointer array only after freeing up the memory pages holding the // records. Otherwise, if the task is over allocated memory, then without freeing the memory // pages, we might not be able to get memory for the pointer array. @@ -325,7 +329,7 @@ public void cleanupResources() { deleteSpillFiles(); freeMemory(); if (inMemSorter != null) { - inMemSorter.free(); + inMemSorter.freeMemory(); inMemSorter = null; } } @@ -339,40 +343,53 @@ public void cleanupResources() { private void growPointerArrayIfNecessary() throws IOException { assert(inMemSorter != null); if (!inMemSorter.hasSpaceForAnotherRecord()) { + if (inMemSorter.numRecords() <= 0) { + // Spilling was triggered just before this method was called. The pointer array was freed + // during the spill, so a new pointer array needs to be allocated here. + LongArray array = allocateArray(inMemSorter.getInitialSize()); + inMemSorter.expandPointerArray(array); + return; + } + long used = inMemSorter.getMemoryUsage(); - LongArray array; + LongArray array = null; try { // could trigger spilling array = allocateArray(used / 8 * 2); } catch (TooLargePageException e) { // The pointer array is too big to fix in a single page, spill. spill(); - return; } catch (SparkOutOfMemoryError e) { - // should have trigger spilling - if (!inMemSorter.hasSpaceForAnotherRecord()) { + if (inMemSorter.numRecords() > 0) { logger.error("Unable to grow the pointer array"); throw e; } - return; + // The new array could not be allocated, but that is not an issue as it is longer needed, + // as all records were spilled. } - // check if spilling is triggered or not - if (inMemSorter.hasSpaceForAnotherRecord()) { - freeArray(array); - } else { - inMemSorter.expandPointerArray(array); + + if (inMemSorter.numRecords() <= 0) { + // Spilling was triggered while trying to allocate the new array. + if (array != null) { + // We succeeded in allocating the new array, but, since all records were spilled, a + // smaller array would also suffice. + freeArray(array); + } + // The pointer array was freed during the spill, so a new pointer array needs to be + // allocated here. + array = allocateArray(inMemSorter.getInitialSize()); } + inMemSorter.expandPointerArray(array); } } /** - * Allocates more memory in order to insert an additional record. This will request additional - * memory from the memory manager and spill if the requested memory can not be obtained. + * Allocates an additional page in order to insert an additional record. This will request + * additional memory from the memory manager and spill if the requested memory can not be + * obtained. * * @param required the required space in the data page, in bytes, including space for storing - * the record size. This must be less than or equal to the page size (records - * that exceed the page size are handled via a different code path which uses - * special overflow pages). + * the record size. */ private void acquireNewPageIfNecessary(int required) { if (currentPage == null || @@ -384,6 +401,37 @@ private void acquireNewPageIfNecessary(int required) { } } + /** + * Allocates more memory in order to insert an additional record. This will request additional + * memory from the memory manager and spill if the requested memory can not be obtained. + * + * @param required the required space in the data page, in bytes, including space for storing + * the record size. + */ + private void allocateMemoryForRecordIfNecessary(int required) throws IOException { + // Step 1: + // Ensure that the pointer array has space for another record. This may cause a spill. + growPointerArrayIfNecessary(); + // Step 2: + // Ensure that the last page has space for another record. This may cause a spill. + acquireNewPageIfNecessary(required); + // Step 3: + // The allocation in step 2 could have caused a spill, which would have freed the pointer + // array allocated in step 1. Therefore we need to check again whether we have to allocate + // a new pointer array. + // + // If the allocation in this step causes a spill event then it will not cause the page + // allocated in the previous step to be freed. The function `spill` only frees memory if at + // least one record has been inserted in the in-memory sorter. This will not be the case if + // we have spilled in the previous step. + // + // If we did not spill in the previous step then `growPointerArrayIfNecessary` will be a + // no-op that does not allocate any memory, and therefore can't cause a spill event. + // + // Thus there is no need to call `acquireNewPageIfNecessary` again after this step. + growPointerArrayIfNecessary(); + } + /** * Write a record to the sorter. */ @@ -398,11 +446,10 @@ public void insertRecord( spill(); } - growPointerArrayIfNecessary(); - int uaoSize = UnsafeAlignedOffset.getUaoSize(); + final int uaoSize = UnsafeAlignedOffset.getUaoSize(); // Need 4 or 8 bytes to store the record length. final int required = length + uaoSize; - acquireNewPageIfNecessary(required); + allocateMemoryForRecordIfNecessary(required); final Object base = currentPage.getBaseObject(); final long recordAddress = taskMemoryManager.encodePageNumberAndOffset(currentPage, pageCursor); @@ -425,10 +472,9 @@ public void insertKVRecord(Object keyBase, long keyOffset, int keyLen, Object valueBase, long valueOffset, int valueLen, long prefix, boolean prefixIsNull) throws IOException { - growPointerArrayIfNecessary(); - int uaoSize = UnsafeAlignedOffset.getUaoSize(); + final int uaoSize = UnsafeAlignedOffset.getUaoSize(); final int required = keyLen + valueLen + (2 * uaoSize); - acquireNewPageIfNecessary(required); + allocateMemoryForRecordIfNecessary(required); final Object base = currentPage.getBaseObject(); final long recordAddress = taskMemoryManager.encodePageNumberAndOffset(currentPage, pageCursor); @@ -501,10 +547,14 @@ private static void spillIterator(UnsafeSorterIterator inMemIterator, */ class SpillableIterator extends UnsafeSorterIterator { private UnsafeSorterIterator upstream; - private UnsafeSorterIterator nextUpstream = null; private MemoryBlock lastPage = null; private boolean loaded = false; - private int numRecords = 0; + private int numRecords; + + private Object currentBaseObject; + private long currentBaseOffset; + private int currentRecordLength; + private long currentKeyPrefix; SpillableIterator(UnsafeSorterIterator inMemIterator) { this.upstream = inMemIterator; @@ -516,23 +566,32 @@ public int getNumRecords() { return numRecords; } + @Override + public long getCurrentPageNumber() { + throw new UnsupportedOperationException(); + } + public long spill() throws IOException { synchronized (this) { - if (!(upstream instanceof UnsafeInMemorySorter.SortedIterator && nextUpstream == null - && numRecords > 0)) { + if (inMemSorter == null) { return 0L; } - UnsafeInMemorySorter.SortedIterator inMemIterator = - ((UnsafeInMemorySorter.SortedIterator) upstream).clone(); + long currentPageNumber = upstream.getCurrentPageNumber(); - ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); - // Iterate over the records that have not been returned and spill them. - final UnsafeSorterSpillWriter spillWriter = - new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, writeMetrics, numRecords); - spillIterator(inMemIterator, spillWriter); - spillWriters.add(spillWriter); - nextUpstream = spillWriter.getReader(serializerManager); + ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); + if (numRecords > 0) { + // Iterate over the records that have not been returned and spill them. + final UnsafeSorterSpillWriter spillWriter = new UnsafeSorterSpillWriter( + blockManager, fileBufferSizeBytes, writeMetrics, numRecords); + spillIterator(upstream, spillWriter); + spillWriters.add(spillWriter); + upstream = spillWriter.getReader(serializerManager); + } else { + // Nothing to spill as all records have been read already, but do not return yet, as the + // memory still has to be freed. + upstream = null; + } long released = 0L; synchronized (UnsafeExternalSorter.this) { @@ -540,8 +599,7 @@ public long spill() throws IOException { // is accessing the current record. We free this page in that caller's next loadNext() // call. for (MemoryBlock page : allocatedPages) { - if (!loaded || page.pageNumber != - ((UnsafeInMemorySorter.SortedIterator)upstream).getCurrentPageNumber()) { + if (!loaded || page.pageNumber != currentPageNumber) { released += page.size(); freePage(page); } else { @@ -549,13 +607,18 @@ public long spill() throws IOException { } } allocatedPages.clear(); + if (lastPage != null) { + // Add the last page back to the list of allocated pages to make sure it gets freed in + // case loadNext() never gets called again. + allocatedPages.add(lastPage); + } } // in-memory sorter will not be used after spilling assert(inMemSorter != null); released += inMemSorter.getMemoryUsage(); totalSortTimeNanos += inMemSorter.getSortTimeNanos(); - inMemSorter.free(); + inMemSorter.freeMemory(); inMemSorter = null; taskContext.taskMetrics().incMemoryBytesSpilled(released); taskContext.taskMetrics().incDiskBytesSpilled(writeMetrics.bytesWritten()); @@ -571,26 +634,32 @@ public boolean hasNext() { @Override public void loadNext() throws IOException { + assert upstream != null; MemoryBlock pageToFree = null; try { synchronized (this) { loaded = true; - if (nextUpstream != null) { - // Just consumed the last record from in memory iterator - if(lastPage != null) { - // Do not free the page here, while we are locking `SpillableIterator`. The `freePage` - // method locks the `TaskMemoryManager`, and it's a bad idea to lock 2 objects in - // sequence. We may hit dead lock if another thread locks `TaskMemoryManager` and - // `SpillableIterator` in sequence, which may happen in - // `TaskMemoryManager.acquireExecutionMemory`. - pageToFree = lastPage; - lastPage = null; - } - upstream = nextUpstream; - nextUpstream = null; + // Just consumed the last record from the in-memory iterator. + if (lastPage != null) { + // Do not free the page here, while we are locking `SpillableIterator`. The `freePage` + // method locks the `TaskMemoryManager`, and it's a bad idea to lock 2 objects in + // sequence. We may hit dead lock if another thread locks `TaskMemoryManager` and + // `SpillableIterator` in sequence, which may happen in + // `TaskMemoryManager.acquireExecutionMemory`. + pageToFree = lastPage; + allocatedPages.clear(); + lastPage = null; } numRecords--; upstream.loadNext(); + + // Keep track of the current base object, base offset, record length, and key prefix, + // so that the current record can still be read in case a spill is triggered and we + // switch to the spill writer's iterator. + currentBaseObject = upstream.getBaseObject(); + currentBaseOffset = upstream.getBaseOffset(); + currentRecordLength = upstream.getRecordLength(); + currentKeyPrefix = upstream.getKeyPrefix(); } } finally { if (pageToFree != null) { @@ -601,22 +670,22 @@ public void loadNext() throws IOException { @Override public Object getBaseObject() { - return upstream.getBaseObject(); + return currentBaseObject; } @Override public long getBaseOffset() { - return upstream.getBaseOffset(); + return currentBaseOffset; } @Override public int getRecordLength() { - return upstream.getRecordLength(); + return currentRecordLength; } @Override public long getKeyPrefix() { - return upstream.getKeyPrefix(); + return currentKeyPrefix; } } @@ -646,7 +715,7 @@ public UnsafeSorterIterator getIterator(int startIndex) throws IOException { } i += spillWriter.recordsSpilled(); } - if (inMemSorter != null) { + if (inMemSorter != null && inMemSorter.numRecords() > 0) { UnsafeSorterIterator iter = inMemSorter.getSortedIterator(); moveOver(iter, startIndex - i); queue.add(iter); @@ -693,6 +762,11 @@ public int getNumRecords() { return numRecords; } + @Override + public long getCurrentPageNumber() { + return current.getCurrentPageNumber(); + } + @Override public boolean hasNext() { while (!current.hasNext() && !iterators.isEmpty()) { diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index 660eb790a550b..33be899b6b438 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -159,32 +159,26 @@ private int getUsableCapacity() { return (int) (array.size() / (radixSortSupport != null ? 2 : 1.5)); } + public long getInitialSize() { + return initialSize; + } + /** * Free the memory used by pointer array. */ - public void free() { + public void freeMemory() { if (consumer != null) { if (array != null) { consumer.freeArray(array); } - array = null; - } - } - public void reset() { - if (consumer != null) { - consumer.freeArray(array); - // the call to consumer.allocateArray may trigger a spill which in turn access this instance - // and eventually re-enter this method and try to free the array again. by setting the array - // to null and its length to 0 we effectively make the spill code-path a no-op. setting the - // array to null also indicates that it has already been de-allocated which prevents a double - // de-allocation in free(). + // Set the array to null instead of allocating a new array. Allocating an array could have + // triggered another spill and this method already is called from UnsafeExternalSorter when + // spilling. Attempting to allocate while spilling is dangerous, as we could be holding onto + // a large partially complete allocation, which may prevent other memory from being allocated. + // Instead we will allocate the new array when it is necessary. array = null; usableCapacity = 0; - pos = 0; - nullBoundaryPos = 0; - array = consumer.allocateArray(initialSize); - usableCapacity = getUsableCapacity(); } pos = 0; nullBoundaryPos = 0; @@ -217,18 +211,20 @@ public boolean hasSpaceForAnotherRecord() { } public void expandPointerArray(LongArray newArray) { - if (newArray.size() < array.size()) { - // checkstyle.off: RegexpSinglelineJava - throw new SparkOutOfMemoryError("Not enough memory to grow pointer array"); - // checkstyle.on: RegexpSinglelineJava + if (array != null) { + if (newArray.size() < array.size()) { + // checkstyle.off: RegexpSinglelineJava + throw new SparkOutOfMemoryError("Not enough memory to grow pointer array"); + // checkstyle.on: RegexpSinglelineJava + } + Platform.copyMemory( + array.getBaseObject(), + array.getBaseOffset(), + newArray.getBaseObject(), + newArray.getBaseOffset(), + pos * 8L); + consumer.freeArray(array); } - Platform.copyMemory( - array.getBaseObject(), - array.getBaseOffset(), - newArray.getBaseObject(), - newArray.getBaseOffset(), - pos * 8L); - consumer.freeArray(array); array = newArray; usableCapacity = getUsableCapacity(); } @@ -330,6 +326,7 @@ public void loadNext() { @Override public long getBaseOffset() { return baseOffset; } + @Override public long getCurrentPageNumber() { return currentPageNumber; } @@ -346,6 +343,11 @@ public long getCurrentPageNumber() { * {@code next()} will return the same mutable object. */ public UnsafeSorterIterator getSortedIterator() { + if (numRecords() == 0) { + // `array` might be null, so make sure that it is not accessed by returning early. + return new SortedIterator(0, 0); + } + int offset = 0; long start = System.nanoTime(); if (sortComparator != null) { diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterIterator.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterIterator.java index 1b3167fcc250c..d9f22311d07c2 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterIterator.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterIterator.java @@ -34,4 +34,6 @@ public abstract class UnsafeSorterIterator { public abstract long getKeyPrefix(); public abstract int getNumRecords(); + + public abstract long getCurrentPageNumber(); } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java index ab800288dcb43..f8603c5799e9b 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java @@ -70,6 +70,11 @@ public int getNumRecords() { return numRecords; } + @Override + public long getCurrentPageNumber() { + throw new UnsupportedOperationException(); + } + @Override public boolean hasNext() { return !priorityQueue.isEmpty() || (spillReader != null && spillReader.hasNext()); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java index a524c4790407d..db79efd008530 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java @@ -89,6 +89,11 @@ public int getNumRecords() { return numRecords; } + @Override + public long getCurrentPageNumber() { + throw new UnsupportedOperationException(); + } + @Override public boolean hasNext() { return (numRecordsRemaining > 0); diff --git a/core/src/main/resources/org/apache/spark/ui/static/jquery-3.4.1.min.js b/core/src/main/resources/org/apache/spark/ui/static/jquery-3.4.1.min.js deleted file mode 100644 index 07c00cd227da0..0000000000000 --- a/core/src/main/resources/org/apache/spark/ui/static/jquery-3.4.1.min.js +++ /dev/null @@ -1,2 +0,0 @@ -/*! jQuery v3.4.1 | (c) JS Foundation and other contributors | jquery.org/license */ -!function(e,t){"use strict";"object"==typeof module&&"object"==typeof module.exports?module.exports=e.document?t(e,!0):function(e){if(!e.document)throw new Error("jQuery requires a window with a document");return t(e)}:t(e)}("undefined"!=typeof window?window:this,function(C,e){"use strict";var t=[],E=C.document,r=Object.getPrototypeOf,s=t.slice,g=t.concat,u=t.push,i=t.indexOf,n={},o=n.toString,v=n.hasOwnProperty,a=v.toString,l=a.call(Object),y={},m=function(e){return"function"==typeof e&&"number"!=typeof e.nodeType},x=function(e){return null!=e&&e===e.window},c={type:!0,src:!0,nonce:!0,noModule:!0};function b(e,t,n){var r,i,o=(n=n||E).createElement("script");if(o.text=e,t)for(r in c)(i=t[r]||t.getAttribute&&t.getAttribute(r))&&o.setAttribute(r,i);n.head.appendChild(o).parentNode.removeChild(o)}function w(e){return null==e?e+"":"object"==typeof e||"function"==typeof e?n[o.call(e)]||"object":typeof e}var f="3.4.1",k=function(e,t){return new k.fn.init(e,t)},p=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g;function d(e){var t=!!e&&"length"in e&&e.length,n=w(e);return!m(e)&&!x(e)&&("array"===n||0===t||"number"==typeof t&&0+~]|"+M+")"+M+"*"),U=new RegExp(M+"|>"),X=new RegExp($),V=new RegExp("^"+I+"$"),G={ID:new RegExp("^#("+I+")"),CLASS:new RegExp("^\\.("+I+")"),TAG:new RegExp("^("+I+"|[*])"),ATTR:new RegExp("^"+W),PSEUDO:new RegExp("^"+$),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+M+"*(even|odd|(([+-]|)(\\d*)n|)"+M+"*(?:([+-]|)"+M+"*(\\d+)|))"+M+"*\\)|)","i"),bool:new RegExp("^(?:"+R+")$","i"),needsContext:new RegExp("^"+M+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+M+"*((?:-\\d)?\\d*)"+M+"*\\)|)(?=[^-]|$)","i")},Y=/HTML$/i,Q=/^(?:input|select|textarea|button)$/i,J=/^h\d$/i,K=/^[^{]+\{\s*\[native \w/,Z=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,ee=/[+~]/,te=new RegExp("\\\\([\\da-f]{1,6}"+M+"?|("+M+")|.)","ig"),ne=function(e,t,n){var r="0x"+t-65536;return r!=r||n?t:r<0?String.fromCharCode(r+65536):String.fromCharCode(r>>10|55296,1023&r|56320)},re=/([\0-\x1f\x7f]|^-?\d)|^-$|[^\0-\x1f\x7f-\uFFFF\w-]/g,ie=function(e,t){return t?"\0"===e?"\ufffd":e.slice(0,-1)+"\\"+e.charCodeAt(e.length-1).toString(16)+" ":"\\"+e},oe=function(){T()},ae=be(function(e){return!0===e.disabled&&"fieldset"===e.nodeName.toLowerCase()},{dir:"parentNode",next:"legend"});try{H.apply(t=O.call(m.childNodes),m.childNodes),t[m.childNodes.length].nodeType}catch(e){H={apply:t.length?function(e,t){L.apply(e,O.call(t))}:function(e,t){var n=e.length,r=0;while(e[n++]=t[r++]);e.length=n-1}}}function se(t,e,n,r){var i,o,a,s,u,l,c,f=e&&e.ownerDocument,p=e?e.nodeType:9;if(n=n||[],"string"!=typeof t||!t||1!==p&&9!==p&&11!==p)return n;if(!r&&((e?e.ownerDocument||e:m)!==C&&T(e),e=e||C,E)){if(11!==p&&(u=Z.exec(t)))if(i=u[1]){if(9===p){if(!(a=e.getElementById(i)))return n;if(a.id===i)return n.push(a),n}else if(f&&(a=f.getElementById(i))&&y(e,a)&&a.id===i)return n.push(a),n}else{if(u[2])return H.apply(n,e.getElementsByTagName(t)),n;if((i=u[3])&&d.getElementsByClassName&&e.getElementsByClassName)return H.apply(n,e.getElementsByClassName(i)),n}if(d.qsa&&!A[t+" "]&&(!v||!v.test(t))&&(1!==p||"object"!==e.nodeName.toLowerCase())){if(c=t,f=e,1===p&&U.test(t)){(s=e.getAttribute("id"))?s=s.replace(re,ie):e.setAttribute("id",s=k),o=(l=h(t)).length;while(o--)l[o]="#"+s+" "+xe(l[o]);c=l.join(","),f=ee.test(t)&&ye(e.parentNode)||e}try{return H.apply(n,f.querySelectorAll(c)),n}catch(e){A(t,!0)}finally{s===k&&e.removeAttribute("id")}}}return g(t.replace(B,"$1"),e,n,r)}function ue(){var r=[];return function e(t,n){return r.push(t+" ")>b.cacheLength&&delete e[r.shift()],e[t+" "]=n}}function le(e){return e[k]=!0,e}function ce(e){var t=C.createElement("fieldset");try{return!!e(t)}catch(e){return!1}finally{t.parentNode&&t.parentNode.removeChild(t),t=null}}function fe(e,t){var n=e.split("|"),r=n.length;while(r--)b.attrHandle[n[r]]=t}function pe(e,t){var n=t&&e,r=n&&1===e.nodeType&&1===t.nodeType&&e.sourceIndex-t.sourceIndex;if(r)return r;if(n)while(n=n.nextSibling)if(n===t)return-1;return e?1:-1}function de(t){return function(e){return"input"===e.nodeName.toLowerCase()&&e.type===t}}function he(n){return function(e){var t=e.nodeName.toLowerCase();return("input"===t||"button"===t)&&e.type===n}}function ge(t){return function(e){return"form"in e?e.parentNode&&!1===e.disabled?"label"in e?"label"in e.parentNode?e.parentNode.disabled===t:e.disabled===t:e.isDisabled===t||e.isDisabled!==!t&&ae(e)===t:e.disabled===t:"label"in e&&e.disabled===t}}function ve(a){return le(function(o){return o=+o,le(function(e,t){var n,r=a([],e.length,o),i=r.length;while(i--)e[n=r[i]]&&(e[n]=!(t[n]=e[n]))})})}function ye(e){return e&&"undefined"!=typeof e.getElementsByTagName&&e}for(e in d=se.support={},i=se.isXML=function(e){var t=e.namespaceURI,n=(e.ownerDocument||e).documentElement;return!Y.test(t||n&&n.nodeName||"HTML")},T=se.setDocument=function(e){var t,n,r=e?e.ownerDocument||e:m;return r!==C&&9===r.nodeType&&r.documentElement&&(a=(C=r).documentElement,E=!i(C),m!==C&&(n=C.defaultView)&&n.top!==n&&(n.addEventListener?n.addEventListener("unload",oe,!1):n.attachEvent&&n.attachEvent("onunload",oe)),d.attributes=ce(function(e){return e.className="i",!e.getAttribute("className")}),d.getElementsByTagName=ce(function(e){return e.appendChild(C.createComment("")),!e.getElementsByTagName("*").length}),d.getElementsByClassName=K.test(C.getElementsByClassName),d.getById=ce(function(e){return a.appendChild(e).id=k,!C.getElementsByName||!C.getElementsByName(k).length}),d.getById?(b.filter.ID=function(e){var t=e.replace(te,ne);return function(e){return e.getAttribute("id")===t}},b.find.ID=function(e,t){if("undefined"!=typeof t.getElementById&&E){var n=t.getElementById(e);return n?[n]:[]}}):(b.filter.ID=function(e){var n=e.replace(te,ne);return function(e){var t="undefined"!=typeof e.getAttributeNode&&e.getAttributeNode("id");return t&&t.value===n}},b.find.ID=function(e,t){if("undefined"!=typeof t.getElementById&&E){var n,r,i,o=t.getElementById(e);if(o){if((n=o.getAttributeNode("id"))&&n.value===e)return[o];i=t.getElementsByName(e),r=0;while(o=i[r++])if((n=o.getAttributeNode("id"))&&n.value===e)return[o]}return[]}}),b.find.TAG=d.getElementsByTagName?function(e,t){return"undefined"!=typeof t.getElementsByTagName?t.getElementsByTagName(e):d.qsa?t.querySelectorAll(e):void 0}:function(e,t){var n,r=[],i=0,o=t.getElementsByTagName(e);if("*"===e){while(n=o[i++])1===n.nodeType&&r.push(n);return r}return o},b.find.CLASS=d.getElementsByClassName&&function(e,t){if("undefined"!=typeof t.getElementsByClassName&&E)return t.getElementsByClassName(e)},s=[],v=[],(d.qsa=K.test(C.querySelectorAll))&&(ce(function(e){a.appendChild(e).innerHTML="",e.querySelectorAll("[msallowcapture^='']").length&&v.push("[*^$]="+M+"*(?:''|\"\")"),e.querySelectorAll("[selected]").length||v.push("\\["+M+"*(?:value|"+R+")"),e.querySelectorAll("[id~="+k+"-]").length||v.push("~="),e.querySelectorAll(":checked").length||v.push(":checked"),e.querySelectorAll("a#"+k+"+*").length||v.push(".#.+[+~]")}),ce(function(e){e.innerHTML="";var t=C.createElement("input");t.setAttribute("type","hidden"),e.appendChild(t).setAttribute("name","D"),e.querySelectorAll("[name=d]").length&&v.push("name"+M+"*[*^$|!~]?="),2!==e.querySelectorAll(":enabled").length&&v.push(":enabled",":disabled"),a.appendChild(e).disabled=!0,2!==e.querySelectorAll(":disabled").length&&v.push(":enabled",":disabled"),e.querySelectorAll("*,:x"),v.push(",.*:")})),(d.matchesSelector=K.test(c=a.matches||a.webkitMatchesSelector||a.mozMatchesSelector||a.oMatchesSelector||a.msMatchesSelector))&&ce(function(e){d.disconnectedMatch=c.call(e,"*"),c.call(e,"[s!='']:x"),s.push("!=",$)}),v=v.length&&new RegExp(v.join("|")),s=s.length&&new RegExp(s.join("|")),t=K.test(a.compareDocumentPosition),y=t||K.test(a.contains)?function(e,t){var n=9===e.nodeType?e.documentElement:e,r=t&&t.parentNode;return e===r||!(!r||1!==r.nodeType||!(n.contains?n.contains(r):e.compareDocumentPosition&&16&e.compareDocumentPosition(r)))}:function(e,t){if(t)while(t=t.parentNode)if(t===e)return!0;return!1},D=t?function(e,t){if(e===t)return l=!0,0;var n=!e.compareDocumentPosition-!t.compareDocumentPosition;return n||(1&(n=(e.ownerDocument||e)===(t.ownerDocument||t)?e.compareDocumentPosition(t):1)||!d.sortDetached&&t.compareDocumentPosition(e)===n?e===C||e.ownerDocument===m&&y(m,e)?-1:t===C||t.ownerDocument===m&&y(m,t)?1:u?P(u,e)-P(u,t):0:4&n?-1:1)}:function(e,t){if(e===t)return l=!0,0;var n,r=0,i=e.parentNode,o=t.parentNode,a=[e],s=[t];if(!i||!o)return e===C?-1:t===C?1:i?-1:o?1:u?P(u,e)-P(u,t):0;if(i===o)return pe(e,t);n=e;while(n=n.parentNode)a.unshift(n);n=t;while(n=n.parentNode)s.unshift(n);while(a[r]===s[r])r++;return r?pe(a[r],s[r]):a[r]===m?-1:s[r]===m?1:0}),C},se.matches=function(e,t){return se(e,null,null,t)},se.matchesSelector=function(e,t){if((e.ownerDocument||e)!==C&&T(e),d.matchesSelector&&E&&!A[t+" "]&&(!s||!s.test(t))&&(!v||!v.test(t)))try{var n=c.call(e,t);if(n||d.disconnectedMatch||e.document&&11!==e.document.nodeType)return n}catch(e){A(t,!0)}return 0":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(e){return e[1]=e[1].replace(te,ne),e[3]=(e[3]||e[4]||e[5]||"").replace(te,ne),"~="===e[2]&&(e[3]=" "+e[3]+" "),e.slice(0,4)},CHILD:function(e){return e[1]=e[1].toLowerCase(),"nth"===e[1].slice(0,3)?(e[3]||se.error(e[0]),e[4]=+(e[4]?e[5]+(e[6]||1):2*("even"===e[3]||"odd"===e[3])),e[5]=+(e[7]+e[8]||"odd"===e[3])):e[3]&&se.error(e[0]),e},PSEUDO:function(e){var t,n=!e[6]&&e[2];return G.CHILD.test(e[0])?null:(e[3]?e[2]=e[4]||e[5]||"":n&&X.test(n)&&(t=h(n,!0))&&(t=n.indexOf(")",n.length-t)-n.length)&&(e[0]=e[0].slice(0,t),e[2]=n.slice(0,t)),e.slice(0,3))}},filter:{TAG:function(e){var t=e.replace(te,ne).toLowerCase();return"*"===e?function(){return!0}:function(e){return e.nodeName&&e.nodeName.toLowerCase()===t}},CLASS:function(e){var t=p[e+" "];return t||(t=new RegExp("(^|"+M+")"+e+"("+M+"|$)"))&&p(e,function(e){return t.test("string"==typeof e.className&&e.className||"undefined"!=typeof e.getAttribute&&e.getAttribute("class")||"")})},ATTR:function(n,r,i){return function(e){var t=se.attr(e,n);return null==t?"!="===r:!r||(t+="","="===r?t===i:"!="===r?t!==i:"^="===r?i&&0===t.indexOf(i):"*="===r?i&&-1:\x20\t\r\n\f]*)[\x20\t\r\n\f]*\/?>(?:<\/\1>|)$/i;function j(e,n,r){return m(n)?k.grep(e,function(e,t){return!!n.call(e,t,e)!==r}):n.nodeType?k.grep(e,function(e){return e===n!==r}):"string"!=typeof n?k.grep(e,function(e){return-1)[^>]*|#([\w-]+))$/;(k.fn.init=function(e,t,n){var r,i;if(!e)return this;if(n=n||q,"string"==typeof e){if(!(r="<"===e[0]&&">"===e[e.length-1]&&3<=e.length?[null,e,null]:L.exec(e))||!r[1]&&t)return!t||t.jquery?(t||n).find(e):this.constructor(t).find(e);if(r[1]){if(t=t instanceof k?t[0]:t,k.merge(this,k.parseHTML(r[1],t&&t.nodeType?t.ownerDocument||t:E,!0)),D.test(r[1])&&k.isPlainObject(t))for(r in t)m(this[r])?this[r](t[r]):this.attr(r,t[r]);return this}return(i=E.getElementById(r[2]))&&(this[0]=i,this.length=1),this}return e.nodeType?(this[0]=e,this.length=1,this):m(e)?void 0!==n.ready?n.ready(e):e(k):k.makeArray(e,this)}).prototype=k.fn,q=k(E);var H=/^(?:parents|prev(?:Until|All))/,O={children:!0,contents:!0,next:!0,prev:!0};function P(e,t){while((e=e[t])&&1!==e.nodeType);return e}k.fn.extend({has:function(e){var t=k(e,this),n=t.length;return this.filter(function(){for(var e=0;e\x20\t\r\n\f]*)/i,he=/^$|^module$|\/(?:java|ecma)script/i,ge={option:[1,""],thead:[1,"","
"],col:[2,"","
"],tr:[2,"","
"],td:[3,"","
"],_default:[0,"",""]};function ve(e,t){var n;return n="undefined"!=typeof e.getElementsByTagName?e.getElementsByTagName(t||"*"):"undefined"!=typeof e.querySelectorAll?e.querySelectorAll(t||"*"):[],void 0===t||t&&A(e,t)?k.merge([e],n):n}function ye(e,t){for(var n=0,r=e.length;nx",y.noCloneChecked=!!me.cloneNode(!0).lastChild.defaultValue;var Te=/^key/,Ce=/^(?:mouse|pointer|contextmenu|drag|drop)|click/,Ee=/^([^.]*)(?:\.(.+)|)/;function ke(){return!0}function Se(){return!1}function Ne(e,t){return e===function(){try{return E.activeElement}catch(e){}}()==("focus"===t)}function Ae(e,t,n,r,i,o){var a,s;if("object"==typeof t){for(s in"string"!=typeof n&&(r=r||n,n=void 0),t)Ae(e,s,n,r,t[s],o);return e}if(null==r&&null==i?(i=n,r=n=void 0):null==i&&("string"==typeof n?(i=r,r=void 0):(i=r,r=n,n=void 0)),!1===i)i=Se;else if(!i)return e;return 1===o&&(a=i,(i=function(e){return k().off(e),a.apply(this,arguments)}).guid=a.guid||(a.guid=k.guid++)),e.each(function(){k.event.add(this,t,i,r,n)})}function De(e,i,o){o?(Q.set(e,i,!1),k.event.add(e,i,{namespace:!1,handler:function(e){var t,n,r=Q.get(this,i);if(1&e.isTrigger&&this[i]){if(r.length)(k.event.special[i]||{}).delegateType&&e.stopPropagation();else if(r=s.call(arguments),Q.set(this,i,r),t=o(this,i),this[i](),r!==(n=Q.get(this,i))||t?Q.set(this,i,!1):n={},r!==n)return e.stopImmediatePropagation(),e.preventDefault(),n.value}else r.length&&(Q.set(this,i,{value:k.event.trigger(k.extend(r[0],k.Event.prototype),r.slice(1),this)}),e.stopImmediatePropagation())}})):void 0===Q.get(e,i)&&k.event.add(e,i,ke)}k.event={global:{},add:function(t,e,n,r,i){var o,a,s,u,l,c,f,p,d,h,g,v=Q.get(t);if(v){n.handler&&(n=(o=n).handler,i=o.selector),i&&k.find.matchesSelector(ie,i),n.guid||(n.guid=k.guid++),(u=v.events)||(u=v.events={}),(a=v.handle)||(a=v.handle=function(e){return"undefined"!=typeof k&&k.event.triggered!==e.type?k.event.dispatch.apply(t,arguments):void 0}),l=(e=(e||"").match(R)||[""]).length;while(l--)d=g=(s=Ee.exec(e[l])||[])[1],h=(s[2]||"").split(".").sort(),d&&(f=k.event.special[d]||{},d=(i?f.delegateType:f.bindType)||d,f=k.event.special[d]||{},c=k.extend({type:d,origType:g,data:r,handler:n,guid:n.guid,selector:i,needsContext:i&&k.expr.match.needsContext.test(i),namespace:h.join(".")},o),(p=u[d])||((p=u[d]=[]).delegateCount=0,f.setup&&!1!==f.setup.call(t,r,h,a)||t.addEventListener&&t.addEventListener(d,a)),f.add&&(f.add.call(t,c),c.handler.guid||(c.handler.guid=n.guid)),i?p.splice(p.delegateCount++,0,c):p.push(c),k.event.global[d]=!0)}},remove:function(e,t,n,r,i){var o,a,s,u,l,c,f,p,d,h,g,v=Q.hasData(e)&&Q.get(e);if(v&&(u=v.events)){l=(t=(t||"").match(R)||[""]).length;while(l--)if(d=g=(s=Ee.exec(t[l])||[])[1],h=(s[2]||"").split(".").sort(),d){f=k.event.special[d]||{},p=u[d=(r?f.delegateType:f.bindType)||d]||[],s=s[2]&&new RegExp("(^|\\.)"+h.join("\\.(?:.*\\.|)")+"(\\.|$)"),a=o=p.length;while(o--)c=p[o],!i&&g!==c.origType||n&&n.guid!==c.guid||s&&!s.test(c.namespace)||r&&r!==c.selector&&("**"!==r||!c.selector)||(p.splice(o,1),c.selector&&p.delegateCount--,f.remove&&f.remove.call(e,c));a&&!p.length&&(f.teardown&&!1!==f.teardown.call(e,h,v.handle)||k.removeEvent(e,d,v.handle),delete u[d])}else for(d in u)k.event.remove(e,d+t[l],n,r,!0);k.isEmptyObject(u)&&Q.remove(e,"handle events")}},dispatch:function(e){var t,n,r,i,o,a,s=k.event.fix(e),u=new Array(arguments.length),l=(Q.get(this,"events")||{})[s.type]||[],c=k.event.special[s.type]||{};for(u[0]=s,t=1;t\x20\t\r\n\f]*)[^>]*)\/>/gi,qe=/\s*$/g;function Oe(e,t){return A(e,"table")&&A(11!==t.nodeType?t:t.firstChild,"tr")&&k(e).children("tbody")[0]||e}function Pe(e){return e.type=(null!==e.getAttribute("type"))+"/"+e.type,e}function Re(e){return"true/"===(e.type||"").slice(0,5)?e.type=e.type.slice(5):e.removeAttribute("type"),e}function Me(e,t){var n,r,i,o,a,s,u,l;if(1===t.nodeType){if(Q.hasData(e)&&(o=Q.access(e),a=Q.set(t,o),l=o.events))for(i in delete a.handle,a.events={},l)for(n=0,r=l[i].length;n")},clone:function(e,t,n){var r,i,o,a,s,u,l,c=e.cloneNode(!0),f=oe(e);if(!(y.noCloneChecked||1!==e.nodeType&&11!==e.nodeType||k.isXMLDoc(e)))for(a=ve(c),r=0,i=(o=ve(e)).length;r").attr(n.scriptAttrs||{}).prop({charset:n.scriptCharset,src:n.url}).on("load error",i=function(e){r.remove(),i=null,e&&t("error"===e.type?404:200,e.type)}),E.head.appendChild(r[0])},abort:function(){i&&i()}}});var Vt,Gt=[],Yt=/(=)\?(?=&|$)|\?\?/;k.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var e=Gt.pop()||k.expando+"_"+kt++;return this[e]=!0,e}}),k.ajaxPrefilter("json jsonp",function(e,t,n){var r,i,o,a=!1!==e.jsonp&&(Yt.test(e.url)?"url":"string"==typeof e.data&&0===(e.contentType||"").indexOf("application/x-www-form-urlencoded")&&Yt.test(e.data)&&"data");if(a||"jsonp"===e.dataTypes[0])return r=e.jsonpCallback=m(e.jsonpCallback)?e.jsonpCallback():e.jsonpCallback,a?e[a]=e[a].replace(Yt,"$1"+r):!1!==e.jsonp&&(e.url+=(St.test(e.url)?"&":"?")+e.jsonp+"="+r),e.converters["script json"]=function(){return o||k.error(r+" was not called"),o[0]},e.dataTypes[0]="json",i=C[r],C[r]=function(){o=arguments},n.always(function(){void 0===i?k(C).removeProp(r):C[r]=i,e[r]&&(e.jsonpCallback=t.jsonpCallback,Gt.push(r)),o&&m(i)&&i(o[0]),o=i=void 0}),"script"}),y.createHTMLDocument=((Vt=E.implementation.createHTMLDocument("").body).innerHTML="
",2===Vt.childNodes.length),k.parseHTML=function(e,t,n){return"string"!=typeof e?[]:("boolean"==typeof t&&(n=t,t=!1),t||(y.createHTMLDocument?((r=(t=E.implementation.createHTMLDocument("")).createElement("base")).href=E.location.href,t.head.appendChild(r)):t=E),o=!n&&[],(i=D.exec(e))?[t.createElement(i[1])]:(i=we([e],t,o),o&&o.length&&k(o).remove(),k.merge([],i.childNodes)));var r,i,o},k.fn.load=function(e,t,n){var r,i,o,a=this,s=e.indexOf(" ");return-1").append(k.parseHTML(e)).find(r):e)}).always(n&&function(e,t){a.each(function(){n.apply(this,o||[e.responseText,t,e])})}),this},k.each(["ajaxStart","ajaxStop","ajaxComplete","ajaxError","ajaxSuccess","ajaxSend"],function(e,t){k.fn[t]=function(e){return this.on(t,e)}}),k.expr.pseudos.animated=function(t){return k.grep(k.timers,function(e){return t===e.elem}).length},k.offset={setOffset:function(e,t,n){var r,i,o,a,s,u,l=k.css(e,"position"),c=k(e),f={};"static"===l&&(e.style.position="relative"),s=c.offset(),o=k.css(e,"top"),u=k.css(e,"left"),("absolute"===l||"fixed"===l)&&-1<(o+u).indexOf("auto")?(a=(r=c.position()).top,i=r.left):(a=parseFloat(o)||0,i=parseFloat(u)||0),m(t)&&(t=t.call(e,n,k.extend({},s))),null!=t.top&&(f.top=t.top-s.top+a),null!=t.left&&(f.left=t.left-s.left+i),"using"in t?t.using.call(e,f):c.css(f)}},k.fn.extend({offset:function(t){if(arguments.length)return void 0===t?this:this.each(function(e){k.offset.setOffset(this,t,e)});var e,n,r=this[0];return r?r.getClientRects().length?(e=r.getBoundingClientRect(),n=r.ownerDocument.defaultView,{top:e.top+n.pageYOffset,left:e.left+n.pageXOffset}):{top:0,left:0}:void 0},position:function(){if(this[0]){var e,t,n,r=this[0],i={top:0,left:0};if("fixed"===k.css(r,"position"))t=r.getBoundingClientRect();else{t=this.offset(),n=r.ownerDocument,e=r.offsetParent||n.documentElement;while(e&&(e===n.body||e===n.documentElement)&&"static"===k.css(e,"position"))e=e.parentNode;e&&e!==r&&1===e.nodeType&&((i=k(e).offset()).top+=k.css(e,"borderTopWidth",!0),i.left+=k.css(e,"borderLeftWidth",!0))}return{top:t.top-i.top-k.css(r,"marginTop",!0),left:t.left-i.left-k.css(r,"marginLeft",!0)}}},offsetParent:function(){return this.map(function(){var e=this.offsetParent;while(e&&"static"===k.css(e,"position"))e=e.offsetParent;return e||ie})}}),k.each({scrollLeft:"pageXOffset",scrollTop:"pageYOffset"},function(t,i){var o="pageYOffset"===i;k.fn[t]=function(e){return _(this,function(e,t,n){var r;if(x(e)?r=e:9===e.nodeType&&(r=e.defaultView),void 0===n)return r?r[i]:e[t];r?r.scrollTo(o?r.pageXOffset:n,o?n:r.pageYOffset):e[t]=n},t,e,arguments.length)}}),k.each(["top","left"],function(e,n){k.cssHooks[n]=ze(y.pixelPosition,function(e,t){if(t)return t=_e(e,n),$e.test(t)?k(e).position()[n]+"px":t})}),k.each({Height:"height",Width:"width"},function(a,s){k.each({padding:"inner"+a,content:s,"":"outer"+a},function(r,o){k.fn[o]=function(e,t){var n=arguments.length&&(r||"boolean"!=typeof e),i=r||(!0===e||!0===t?"margin":"border");return _(this,function(e,t,n){var r;return x(e)?0===o.indexOf("outer")?e["inner"+a]:e.document.documentElement["client"+a]:9===e.nodeType?(r=e.documentElement,Math.max(e.body["scroll"+a],r["scroll"+a],e.body["offset"+a],r["offset"+a],r["client"+a])):void 0===n?k.css(e,t,i):k.style(e,t,n,i)},s,n?e:void 0,n)}})}),k.each("blur focus focusin focusout resize scroll click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup contextmenu".split(" "),function(e,n){k.fn[n]=function(e,t){return 0+~]|"+M+")"+M+"*"),U=new RegExp(M+"|>"),X=new RegExp(F),V=new RegExp("^"+I+"$"),G={ID:new RegExp("^#("+I+")"),CLASS:new RegExp("^\\.("+I+")"),TAG:new RegExp("^("+I+"|[*])"),ATTR:new RegExp("^"+W),PSEUDO:new RegExp("^"+F),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+M+"*(even|odd|(([+-]|)(\\d*)n|)"+M+"*(?:([+-]|)"+M+"*(\\d+)|))"+M+"*\\)|)","i"),bool:new RegExp("^(?:"+R+")$","i"),needsContext:new RegExp("^"+M+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+M+"*((?:-\\d)?\\d*)"+M+"*\\)|)(?=[^-]|$)","i")},Y=/HTML$/i,Q=/^(?:input|select|textarea|button)$/i,J=/^h\d$/i,K=/^[^{]+\{\s*\[native \w/,Z=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,ee=/[+~]/,te=new RegExp("\\\\[\\da-fA-F]{1,6}"+M+"?|\\\\([^\\r\\n\\f])","g"),ne=function(e,t){var n="0x"+e.slice(1)-65536;return t||(n<0?String.fromCharCode(n+65536):String.fromCharCode(n>>10|55296,1023&n|56320))},re=/([\0-\x1f\x7f]|^-?\d)|^-$|[^\0-\x1f\x7f-\uFFFF\w-]/g,ie=function(e,t){return t?"\0"===e?"\ufffd":e.slice(0,-1)+"\\"+e.charCodeAt(e.length-1).toString(16)+" ":"\\"+e},oe=function(){T()},ae=be(function(e){return!0===e.disabled&&"fieldset"===e.nodeName.toLowerCase()},{dir:"parentNode",next:"legend"});try{H.apply(t=O.call(p.childNodes),p.childNodes),t[p.childNodes.length].nodeType}catch(e){H={apply:t.length?function(e,t){L.apply(e,O.call(t))}:function(e,t){var n=e.length,r=0;while(e[n++]=t[r++]);e.length=n-1}}}function se(t,e,n,r){var i,o,a,s,u,l,c,f=e&&e.ownerDocument,p=e?e.nodeType:9;if(n=n||[],"string"!=typeof t||!t||1!==p&&9!==p&&11!==p)return n;if(!r&&(T(e),e=e||C,E)){if(11!==p&&(u=Z.exec(t)))if(i=u[1]){if(9===p){if(!(a=e.getElementById(i)))return n;if(a.id===i)return n.push(a),n}else if(f&&(a=f.getElementById(i))&&y(e,a)&&a.id===i)return n.push(a),n}else{if(u[2])return H.apply(n,e.getElementsByTagName(t)),n;if((i=u[3])&&d.getElementsByClassName&&e.getElementsByClassName)return H.apply(n,e.getElementsByClassName(i)),n}if(d.qsa&&!N[t+" "]&&(!v||!v.test(t))&&(1!==p||"object"!==e.nodeName.toLowerCase())){if(c=t,f=e,1===p&&(U.test(t)||z.test(t))){(f=ee.test(t)&&ye(e.parentNode)||e)===e&&d.scope||((s=e.getAttribute("id"))?s=s.replace(re,ie):e.setAttribute("id",s=S)),o=(l=h(t)).length;while(o--)l[o]=(s?"#"+s:":scope")+" "+xe(l[o]);c=l.join(",")}try{return H.apply(n,f.querySelectorAll(c)),n}catch(e){N(t,!0)}finally{s===S&&e.removeAttribute("id")}}}return g(t.replace($,"$1"),e,n,r)}function ue(){var r=[];return function e(t,n){return r.push(t+" ")>b.cacheLength&&delete e[r.shift()],e[t+" "]=n}}function le(e){return e[S]=!0,e}function ce(e){var t=C.createElement("fieldset");try{return!!e(t)}catch(e){return!1}finally{t.parentNode&&t.parentNode.removeChild(t),t=null}}function fe(e,t){var n=e.split("|"),r=n.length;while(r--)b.attrHandle[n[r]]=t}function pe(e,t){var n=t&&e,r=n&&1===e.nodeType&&1===t.nodeType&&e.sourceIndex-t.sourceIndex;if(r)return r;if(n)while(n=n.nextSibling)if(n===t)return-1;return e?1:-1}function de(t){return function(e){return"input"===e.nodeName.toLowerCase()&&e.type===t}}function he(n){return function(e){var t=e.nodeName.toLowerCase();return("input"===t||"button"===t)&&e.type===n}}function ge(t){return function(e){return"form"in e?e.parentNode&&!1===e.disabled?"label"in e?"label"in e.parentNode?e.parentNode.disabled===t:e.disabled===t:e.isDisabled===t||e.isDisabled!==!t&&ae(e)===t:e.disabled===t:"label"in e&&e.disabled===t}}function ve(a){return le(function(o){return o=+o,le(function(e,t){var n,r=a([],e.length,o),i=r.length;while(i--)e[n=r[i]]&&(e[n]=!(t[n]=e[n]))})})}function ye(e){return e&&"undefined"!=typeof e.getElementsByTagName&&e}for(e in d=se.support={},i=se.isXML=function(e){var t=e.namespaceURI,n=(e.ownerDocument||e).documentElement;return!Y.test(t||n&&n.nodeName||"HTML")},T=se.setDocument=function(e){var t,n,r=e?e.ownerDocument||e:p;return r!=C&&9===r.nodeType&&r.documentElement&&(a=(C=r).documentElement,E=!i(C),p!=C&&(n=C.defaultView)&&n.top!==n&&(n.addEventListener?n.addEventListener("unload",oe,!1):n.attachEvent&&n.attachEvent("onunload",oe)),d.scope=ce(function(e){return a.appendChild(e).appendChild(C.createElement("div")),"undefined"!=typeof e.querySelectorAll&&!e.querySelectorAll(":scope fieldset div").length}),d.attributes=ce(function(e){return e.className="i",!e.getAttribute("className")}),d.getElementsByTagName=ce(function(e){return e.appendChild(C.createComment("")),!e.getElementsByTagName("*").length}),d.getElementsByClassName=K.test(C.getElementsByClassName),d.getById=ce(function(e){return a.appendChild(e).id=S,!C.getElementsByName||!C.getElementsByName(S).length}),d.getById?(b.filter.ID=function(e){var t=e.replace(te,ne);return function(e){return e.getAttribute("id")===t}},b.find.ID=function(e,t){if("undefined"!=typeof t.getElementById&&E){var n=t.getElementById(e);return n?[n]:[]}}):(b.filter.ID=function(e){var n=e.replace(te,ne);return function(e){var t="undefined"!=typeof e.getAttributeNode&&e.getAttributeNode("id");return t&&t.value===n}},b.find.ID=function(e,t){if("undefined"!=typeof t.getElementById&&E){var n,r,i,o=t.getElementById(e);if(o){if((n=o.getAttributeNode("id"))&&n.value===e)return[o];i=t.getElementsByName(e),r=0;while(o=i[r++])if((n=o.getAttributeNode("id"))&&n.value===e)return[o]}return[]}}),b.find.TAG=d.getElementsByTagName?function(e,t){return"undefined"!=typeof t.getElementsByTagName?t.getElementsByTagName(e):d.qsa?t.querySelectorAll(e):void 0}:function(e,t){var n,r=[],i=0,o=t.getElementsByTagName(e);if("*"===e){while(n=o[i++])1===n.nodeType&&r.push(n);return r}return o},b.find.CLASS=d.getElementsByClassName&&function(e,t){if("undefined"!=typeof t.getElementsByClassName&&E)return t.getElementsByClassName(e)},s=[],v=[],(d.qsa=K.test(C.querySelectorAll))&&(ce(function(e){var t;a.appendChild(e).innerHTML="",e.querySelectorAll("[msallowcapture^='']").length&&v.push("[*^$]="+M+"*(?:''|\"\")"),e.querySelectorAll("[selected]").length||v.push("\\["+M+"*(?:value|"+R+")"),e.querySelectorAll("[id~="+S+"-]").length||v.push("~="),(t=C.createElement("input")).setAttribute("name",""),e.appendChild(t),e.querySelectorAll("[name='']").length||v.push("\\["+M+"*name"+M+"*="+M+"*(?:''|\"\")"),e.querySelectorAll(":checked").length||v.push(":checked"),e.querySelectorAll("a#"+S+"+*").length||v.push(".#.+[+~]"),e.querySelectorAll("\\\f"),v.push("[\\r\\n\\f]")}),ce(function(e){e.innerHTML="";var t=C.createElement("input");t.setAttribute("type","hidden"),e.appendChild(t).setAttribute("name","D"),e.querySelectorAll("[name=d]").length&&v.push("name"+M+"*[*^$|!~]?="),2!==e.querySelectorAll(":enabled").length&&v.push(":enabled",":disabled"),a.appendChild(e).disabled=!0,2!==e.querySelectorAll(":disabled").length&&v.push(":enabled",":disabled"),e.querySelectorAll("*,:x"),v.push(",.*:")})),(d.matchesSelector=K.test(c=a.matches||a.webkitMatchesSelector||a.mozMatchesSelector||a.oMatchesSelector||a.msMatchesSelector))&&ce(function(e){d.disconnectedMatch=c.call(e,"*"),c.call(e,"[s!='']:x"),s.push("!=",F)}),v=v.length&&new RegExp(v.join("|")),s=s.length&&new RegExp(s.join("|")),t=K.test(a.compareDocumentPosition),y=t||K.test(a.contains)?function(e,t){var n=9===e.nodeType?e.documentElement:e,r=t&&t.parentNode;return e===r||!(!r||1!==r.nodeType||!(n.contains?n.contains(r):e.compareDocumentPosition&&16&e.compareDocumentPosition(r)))}:function(e,t){if(t)while(t=t.parentNode)if(t===e)return!0;return!1},D=t?function(e,t){if(e===t)return l=!0,0;var n=!e.compareDocumentPosition-!t.compareDocumentPosition;return n||(1&(n=(e.ownerDocument||e)==(t.ownerDocument||t)?e.compareDocumentPosition(t):1)||!d.sortDetached&&t.compareDocumentPosition(e)===n?e==C||e.ownerDocument==p&&y(p,e)?-1:t==C||t.ownerDocument==p&&y(p,t)?1:u?P(u,e)-P(u,t):0:4&n?-1:1)}:function(e,t){if(e===t)return l=!0,0;var n,r=0,i=e.parentNode,o=t.parentNode,a=[e],s=[t];if(!i||!o)return e==C?-1:t==C?1:i?-1:o?1:u?P(u,e)-P(u,t):0;if(i===o)return pe(e,t);n=e;while(n=n.parentNode)a.unshift(n);n=t;while(n=n.parentNode)s.unshift(n);while(a[r]===s[r])r++;return r?pe(a[r],s[r]):a[r]==p?-1:s[r]==p?1:0}),C},se.matches=function(e,t){return se(e,null,null,t)},se.matchesSelector=function(e,t){if(T(e),d.matchesSelector&&E&&!N[t+" "]&&(!s||!s.test(t))&&(!v||!v.test(t)))try{var n=c.call(e,t);if(n||d.disconnectedMatch||e.document&&11!==e.document.nodeType)return n}catch(e){N(t,!0)}return 0":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(e){return e[1]=e[1].replace(te,ne),e[3]=(e[3]||e[4]||e[5]||"").replace(te,ne),"~="===e[2]&&(e[3]=" "+e[3]+" "),e.slice(0,4)},CHILD:function(e){return e[1]=e[1].toLowerCase(),"nth"===e[1].slice(0,3)?(e[3]||se.error(e[0]),e[4]=+(e[4]?e[5]+(e[6]||1):2*("even"===e[3]||"odd"===e[3])),e[5]=+(e[7]+e[8]||"odd"===e[3])):e[3]&&se.error(e[0]),e},PSEUDO:function(e){var t,n=!e[6]&&e[2];return G.CHILD.test(e[0])?null:(e[3]?e[2]=e[4]||e[5]||"":n&&X.test(n)&&(t=h(n,!0))&&(t=n.indexOf(")",n.length-t)-n.length)&&(e[0]=e[0].slice(0,t),e[2]=n.slice(0,t)),e.slice(0,3))}},filter:{TAG:function(e){var t=e.replace(te,ne).toLowerCase();return"*"===e?function(){return!0}:function(e){return e.nodeName&&e.nodeName.toLowerCase()===t}},CLASS:function(e){var t=m[e+" "];return t||(t=new RegExp("(^|"+M+")"+e+"("+M+"|$)"))&&m(e,function(e){return t.test("string"==typeof e.className&&e.className||"undefined"!=typeof e.getAttribute&&e.getAttribute("class")||"")})},ATTR:function(n,r,i){return function(e){var t=se.attr(e,n);return null==t?"!="===r:!r||(t+="","="===r?t===i:"!="===r?t!==i:"^="===r?i&&0===t.indexOf(i):"*="===r?i&&-1:\x20\t\r\n\f]*)[\x20\t\r\n\f]*\/?>(?:<\/\1>|)$/i;function D(e,n,r){return m(n)?S.grep(e,function(e,t){return!!n.call(e,t,e)!==r}):n.nodeType?S.grep(e,function(e){return e===n!==r}):"string"!=typeof n?S.grep(e,function(e){return-1)[^>]*|#([\w-]+))$/;(S.fn.init=function(e,t,n){var r,i;if(!e)return this;if(n=n||j,"string"==typeof e){if(!(r="<"===e[0]&&">"===e[e.length-1]&&3<=e.length?[null,e,null]:q.exec(e))||!r[1]&&t)return!t||t.jquery?(t||n).find(e):this.constructor(t).find(e);if(r[1]){if(t=t instanceof S?t[0]:t,S.merge(this,S.parseHTML(r[1],t&&t.nodeType?t.ownerDocument||t:E,!0)),N.test(r[1])&&S.isPlainObject(t))for(r in t)m(this[r])?this[r](t[r]):this.attr(r,t[r]);return this}return(i=E.getElementById(r[2]))&&(this[0]=i,this.length=1),this}return e.nodeType?(this[0]=e,this.length=1,this):m(e)?void 0!==n.ready?n.ready(e):e(S):S.makeArray(e,this)}).prototype=S.fn,j=S(E);var L=/^(?:parents|prev(?:Until|All))/,H={children:!0,contents:!0,next:!0,prev:!0};function O(e,t){while((e=e[t])&&1!==e.nodeType);return e}S.fn.extend({has:function(e){var t=S(e,this),n=t.length;return this.filter(function(){for(var e=0;e\x20\t\r\n\f]*)/i,he=/^$|^module$|\/(?:java|ecma)script/i;ce=E.createDocumentFragment().appendChild(E.createElement("div")),(fe=E.createElement("input")).setAttribute("type","radio"),fe.setAttribute("checked","checked"),fe.setAttribute("name","t"),ce.appendChild(fe),y.checkClone=ce.cloneNode(!0).cloneNode(!0).lastChild.checked,ce.innerHTML="",y.noCloneChecked=!!ce.cloneNode(!0).lastChild.defaultValue,ce.innerHTML="",y.option=!!ce.lastChild;var ge={thead:[1,"","
"],col:[2,"","
"],tr:[2,"","
"],td:[3,"","
"],_default:[0,"",""]};function ve(e,t){var n;return n="undefined"!=typeof e.getElementsByTagName?e.getElementsByTagName(t||"*"):"undefined"!=typeof e.querySelectorAll?e.querySelectorAll(t||"*"):[],void 0===t||t&&A(e,t)?S.merge([e],n):n}function ye(e,t){for(var n=0,r=e.length;n",""]);var me=/<|&#?\w+;/;function xe(e,t,n,r,i){for(var o,a,s,u,l,c,f=t.createDocumentFragment(),p=[],d=0,h=e.length;d\s*$/g;function qe(e,t){return A(e,"table")&&A(11!==t.nodeType?t:t.firstChild,"tr")&&S(e).children("tbody")[0]||e}function Le(e){return e.type=(null!==e.getAttribute("type"))+"/"+e.type,e}function He(e){return"true/"===(e.type||"").slice(0,5)?e.type=e.type.slice(5):e.removeAttribute("type"),e}function Oe(e,t){var n,r,i,o,a,s;if(1===t.nodeType){if(Y.hasData(e)&&(s=Y.get(e).events))for(i in Y.remove(t,"handle events"),s)for(n=0,r=s[i].length;n").attr(n.scriptAttrs||{}).prop({charset:n.scriptCharset,src:n.url}).on("load error",i=function(e){r.remove(),i=null,e&&t("error"===e.type?404:200,e.type)}),E.head.appendChild(r[0])},abort:function(){i&&i()}}});var Ut,Xt=[],Vt=/(=)\?(?=&|$)|\?\?/;S.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var e=Xt.pop()||S.expando+"_"+Ct.guid++;return this[e]=!0,e}}),S.ajaxPrefilter("json jsonp",function(e,t,n){var r,i,o,a=!1!==e.jsonp&&(Vt.test(e.url)?"url":"string"==typeof e.data&&0===(e.contentType||"").indexOf("application/x-www-form-urlencoded")&&Vt.test(e.data)&&"data");if(a||"jsonp"===e.dataTypes[0])return r=e.jsonpCallback=m(e.jsonpCallback)?e.jsonpCallback():e.jsonpCallback,a?e[a]=e[a].replace(Vt,"$1"+r):!1!==e.jsonp&&(e.url+=(Et.test(e.url)?"&":"?")+e.jsonp+"="+r),e.converters["script json"]=function(){return o||S.error(r+" was not called"),o[0]},e.dataTypes[0]="json",i=C[r],C[r]=function(){o=arguments},n.always(function(){void 0===i?S(C).removeProp(r):C[r]=i,e[r]&&(e.jsonpCallback=t.jsonpCallback,Xt.push(r)),o&&m(i)&&i(o[0]),o=i=void 0}),"script"}),y.createHTMLDocument=((Ut=E.implementation.createHTMLDocument("").body).innerHTML="
",2===Ut.childNodes.length),S.parseHTML=function(e,t,n){return"string"!=typeof e?[]:("boolean"==typeof t&&(n=t,t=!1),t||(y.createHTMLDocument?((r=(t=E.implementation.createHTMLDocument("")).createElement("base")).href=E.location.href,t.head.appendChild(r)):t=E),o=!n&&[],(i=N.exec(e))?[t.createElement(i[1])]:(i=xe([e],t,o),o&&o.length&&S(o).remove(),S.merge([],i.childNodes)));var r,i,o},S.fn.load=function(e,t,n){var r,i,o,a=this,s=e.indexOf(" ");return-1").append(S.parseHTML(e)).find(r):e)}).always(n&&function(e,t){a.each(function(){n.apply(this,o||[e.responseText,t,e])})}),this},S.expr.pseudos.animated=function(t){return S.grep(S.timers,function(e){return t===e.elem}).length},S.offset={setOffset:function(e,t,n){var r,i,o,a,s,u,l=S.css(e,"position"),c=S(e),f={};"static"===l&&(e.style.position="relative"),s=c.offset(),o=S.css(e,"top"),u=S.css(e,"left"),("absolute"===l||"fixed"===l)&&-1<(o+u).indexOf("auto")?(a=(r=c.position()).top,i=r.left):(a=parseFloat(o)||0,i=parseFloat(u)||0),m(t)&&(t=t.call(e,n,S.extend({},s))),null!=t.top&&(f.top=t.top-s.top+a),null!=t.left&&(f.left=t.left-s.left+i),"using"in t?t.using.call(e,f):("number"==typeof f.top&&(f.top+="px"),"number"==typeof f.left&&(f.left+="px"),c.css(f))}},S.fn.extend({offset:function(t){if(arguments.length)return void 0===t?this:this.each(function(e){S.offset.setOffset(this,t,e)});var e,n,r=this[0];return r?r.getClientRects().length?(e=r.getBoundingClientRect(),n=r.ownerDocument.defaultView,{top:e.top+n.pageYOffset,left:e.left+n.pageXOffset}):{top:0,left:0}:void 0},position:function(){if(this[0]){var e,t,n,r=this[0],i={top:0,left:0};if("fixed"===S.css(r,"position"))t=r.getBoundingClientRect();else{t=this.offset(),n=r.ownerDocument,e=r.offsetParent||n.documentElement;while(e&&(e===n.body||e===n.documentElement)&&"static"===S.css(e,"position"))e=e.parentNode;e&&e!==r&&1===e.nodeType&&((i=S(e).offset()).top+=S.css(e,"borderTopWidth",!0),i.left+=S.css(e,"borderLeftWidth",!0))}return{top:t.top-i.top-S.css(r,"marginTop",!0),left:t.left-i.left-S.css(r,"marginLeft",!0)}}},offsetParent:function(){return this.map(function(){var e=this.offsetParent;while(e&&"static"===S.css(e,"position"))e=e.offsetParent;return e||re})}}),S.each({scrollLeft:"pageXOffset",scrollTop:"pageYOffset"},function(t,i){var o="pageYOffset"===i;S.fn[t]=function(e){return $(this,function(e,t,n){var r;if(x(e)?r=e:9===e.nodeType&&(r=e.defaultView),void 0===n)return r?r[i]:e[t];r?r.scrollTo(o?r.pageXOffset:n,o?n:r.pageYOffset):e[t]=n},t,e,arguments.length)}}),S.each(["top","left"],function(e,n){S.cssHooks[n]=$e(y.pixelPosition,function(e,t){if(t)return t=Be(e,n),Me.test(t)?S(e).position()[n]+"px":t})}),S.each({Height:"height",Width:"width"},function(a,s){S.each({padding:"inner"+a,content:s,"":"outer"+a},function(r,o){S.fn[o]=function(e,t){var n=arguments.length&&(r||"boolean"!=typeof e),i=r||(!0===e||!0===t?"margin":"border");return $(this,function(e,t,n){var r;return x(e)?0===o.indexOf("outer")?e["inner"+a]:e.document.documentElement["client"+a]:9===e.nodeType?(r=e.documentElement,Math.max(e.body["scroll"+a],r["scroll"+a],e.body["offset"+a],r["offset"+a],r["client"+a])):void 0===n?S.css(e,t,i):S.style(e,t,n,i)},s,n?e:void 0,n)}})}),S.each(["ajaxStart","ajaxStop","ajaxComplete","ajaxError","ajaxSuccess","ajaxSend"],function(e,t){S.fn[t]=function(e){return this.on(t,e)}}),S.fn.extend({bind:function(e,t,n){return this.on(e,null,t,n)},unbind:function(e,t){return this.off(e,null,t)},delegate:function(e,t,n,r){return this.on(t,e,n,r)},undelegate:function(e,t,n){return 1===arguments.length?this.off(e,"**"):this.off(t,e||"**",n)},hover:function(e,t){return this.mouseenter(e).mouseleave(t||e)}}),S.each("blur focus focusin focusout resize scroll click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup contextmenu".split(" "),function(e,n){S.fn[n]=function(e,t){return 0 - logInfo(s"Updating map output for ${mapId} to ${bmAddress}") - mapStatus.updateLocation(bmAddress) - invalidateSerializedMapOutputStatusCache() - case None => - logError(s"Asked to update map output ${mapId} for untracked map status.") + try { + val mapStatusOpt = mapStatuses.find(_.mapId == mapId) + mapStatusOpt match { + case Some(mapStatus) => + logInfo(s"Updating map output for ${mapId} to ${bmAddress}") + mapStatus.updateLocation(bmAddress) + invalidateSerializedMapOutputStatusCache() + case None => + logWarning(s"Asked to update map output ${mapId} for untracked map status.") + } + } catch { + case e: java.lang.NullPointerException => + logWarning(s"Unable to update map output for ${mapId}, status removed in-flight") } } diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index dbd89d646ae54..427e98e616515 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -568,7 +568,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria // If spark.executor.heartbeatInterval bigger than spark.network.timeout, // it will almost always cause ExecutorLostFailure. See SPARK-22754. require(executorTimeoutThresholdMs > executorHeartbeatIntervalMs, "The value of " + - s"${networkTimeout}=${executorTimeoutThresholdMs}ms must be no less than the value of " + + s"${networkTimeout}=${executorTimeoutThresholdMs}ms must be greater than the value of " + s"${EXECUTOR_HEARTBEAT_INTERVAL.key}=${executorHeartbeatIntervalMs}ms.") } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1766943703c50..32325d576634c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1882,7 +1882,7 @@ class SparkContext(config: SparkConf) extends Logging { if (!fs.exists(hadoopPath)) { throw new FileNotFoundException(s"Jar ${path} not found") } - if (fs.isDirectory(hadoopPath)) { + if (fs.getFileStatus(hadoopPath).isDirectory) { throw new IllegalArgumentException( s"Directory ${path} is not allowed for addJar") } @@ -1900,7 +1900,7 @@ class SparkContext(config: SparkConf) extends Logging { if (path == null || path.isEmpty) { logWarning("null or empty path specified as parameter to addJar") } else { - val key = if (path.contains("\\")) { + val key = if (path.contains("\\") && Utils.isWindows) { // For local paths with backslashes on Windows, URI throws an exception addLocalJarFile(new File(path)) } else { diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 6947d1c72f12b..bc3644df0ebb5 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -255,6 +255,19 @@ private[spark] object TestUtils { attempt.isSuccess && attempt.get == 0 } + def isPythonVersionAtLeast38(): Boolean = { + val attempt = if (Utils.isWindows) { + Try(Process(Seq("cmd.exe", "/C", "python3 --version")) + .run(ProcessLogger(s => s.startsWith("Python 3.8") || s.startsWith("Python 3.9"))) + .exitValue()) + } else { + Try(Process(Seq("sh", "-c", "python3 --version")) + .run(ProcessLogger(s => s.startsWith("Python 3.8") || s.startsWith("Python 3.9"))) + .exitValue()) + } + attempt.isSuccess && attempt.get == 0 + } + /** * Returns the response code from an HTTP(S) URL. */ diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 77fbbc08c2103..1024d9b5060bc 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -133,22 +133,30 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) if (!blockManager.putSingle(broadcastId, value, MEMORY_AND_DISK, tellMaster = false)) { throw new SparkException(s"Failed to store $broadcastId in BlockManager") } - val blocks = - TorrentBroadcast.blockifyObject(value, blockSize, SparkEnv.get.serializer, compressionCodec) - if (checksumEnabled) { - checksums = new Array[Int](blocks.length) - } - blocks.zipWithIndex.foreach { case (block, i) => + try { + val blocks = + TorrentBroadcast.blockifyObject(value, blockSize, SparkEnv.get.serializer, compressionCodec) if (checksumEnabled) { - checksums(i) = calcChecksum(block) + checksums = new Array[Int](blocks.length) } - val pieceId = BroadcastBlockId(id, "piece" + i) - val bytes = new ChunkedByteBuffer(block.duplicate()) - if (!blockManager.putBytes(pieceId, bytes, MEMORY_AND_DISK_SER, tellMaster = true)) { - throw new SparkException(s"Failed to store $pieceId of $broadcastId in local BlockManager") + blocks.zipWithIndex.foreach { case (block, i) => + if (checksumEnabled) { + checksums(i) = calcChecksum(block) + } + val pieceId = BroadcastBlockId(id, "piece" + i) + val bytes = new ChunkedByteBuffer(block.duplicate()) + if (!blockManager.putBytes(pieceId, bytes, MEMORY_AND_DISK_SER, tellMaster = true)) { + throw new SparkException(s"Failed to store $pieceId of $broadcastId " + + s"in local BlockManager") + } } + blocks.length + } catch { + case t: Throwable => + logError(s"Store broadcast $broadcastId fail, remove all pieces of the broadcast") + blockManager.removeBroadcast(id, tellMaster = true) + throw t } - blocks.length } /** Fetch torrent blocks from the driver and/or other executors. */ diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 1180501e8c738..6f799a542bc1e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -462,6 +462,9 @@ private[spark] object SparkHadoopUtil { for ((key, value) <- conf.getAll if key.startsWith("spark.hadoop.")) { hadoopConf.set(key.substring("spark.hadoop.".length), value) } + if (conf.getOption("spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version").isEmpty) { + hadoopConf.set("mapreduce.fileoutputcommitter.algorithm.version", "1") + } } private def appendSparkHiveConfigs(conf: SparkConf, hadoopConf: Configuration): Unit = { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index fe8be0b3b20d3..168bd1e68a304 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -538,6 +538,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) reader.fileSizeForLastIndex > 0 } catch { case _: FileNotFoundException => false + case NonFatal(e) => + logWarning(s"Error while reading new log ${reader.rootPath}", e) + false } case NonFatal(e) => diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 54b50e6d2fa4a..27addd8fc12e2 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -400,7 +400,9 @@ private[spark] class Executor( // Report executor runtime and JVM gc time Option(task).foreach(t => { t.metrics.setExecutorRunTime(TimeUnit.NANOSECONDS.toMillis( - System.nanoTime() - taskStartTimeNs)) + // SPARK-32898: it's possible that a task is killed when taskStartTimeNs has the initial + // value(=0) still. In this case, the executorRunTime should be considered as 0. + if (taskStartTimeNs > 0) System.nanoTime() - taskStartTimeNs else 0)) t.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime) }) diff --git a/core/src/main/scala/org/apache/spark/internal/config/Tests.scala b/core/src/main/scala/org/apache/spark/internal/config/Tests.scala index a1ebe5ce0ca32..7b8b204bab640 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Tests.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Tests.scala @@ -26,11 +26,11 @@ private[spark] object Tests { .longConf .createWithDefault(Runtime.getRuntime.maxMemory) - val TEST_SCHEDULE_INTERVAL = - ConfigBuilder("spark.testing.dynamicAllocation.scheduleInterval") - .version("2.3.0") - .longConf - .createWithDefault(100) + val TEST_DYNAMIC_ALLOCATION_SCHEDULE_ENABLED = + ConfigBuilder("spark.testing.dynamicAllocation.schedule.enabled") + .version("3.1.0") + .booleanConf + .createWithDefault(true) val IS_TESTING = ConfigBuilder("spark.testing") .version("1.0.1") diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala index 2ed03f7430c32..472401b23fe8e 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala @@ -200,6 +200,16 @@ private[netty] class Inbox(val endpointName: String, val endpoint: RpcEndpoint) * Calls action closure, and calls the endpoint's onError function in the case of exceptions. */ private def safelyCall(endpoint: RpcEndpoint)(action: => Unit): Unit = { + def dealWithFatalError(fatal: Throwable): Unit = { + inbox.synchronized { + assert(numActiveThreads > 0, "The number of active threads should be positive.") + // Should reduce the number of active threads before throw the error. + numActiveThreads -= 1 + } + logError(s"An error happened while processing message in the inbox for $endpointName", fatal) + throw fatal + } + try action catch { case NonFatal(e) => try endpoint.onError(e) catch { @@ -209,8 +219,18 @@ private[netty] class Inbox(val endpointName: String, val endpoint: RpcEndpoint) } else { logError("Ignoring error", ee) } + case fatal: Throwable => + dealWithFatalError(fatal) } + case fatal: Throwable => + dealWithFatalError(fatal) } } + // exposed only for testing + def getNumActiveThreads: Int = { + inbox.synchronized { + inbox.numActiveThreads + } + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala index f2eb4a7047b56..2644d0af2ac50 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala @@ -70,6 +70,7 @@ case class ExecutorProcessLost( * * This is used by the task scheduler to remove state associated with the executor, but * not yet fail any tasks that were running in the executor before the executor is "fully" lost. + * If you update this code make sure to re-run the K8s integration tests. * * @param workerHost it is defined when the worker is decommissioned too */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 2e2851eb9070b..7333b31524f2a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -59,6 +59,8 @@ private[spark] class Pool( } } + override def isSchedulable: Boolean = true + override def addSchedulable(schedulable: Schedulable): Unit = { require(schedulable != null) schedulableQueue.add(schedulable) @@ -105,7 +107,7 @@ private[spark] class Pool( val sortedSchedulableQueue = schedulableQueue.asScala.toSeq.sortWith(taskSetSchedulingAlgorithm.comparator) for (schedulable <- sortedSchedulableQueue) { - sortedTaskSetQueue ++= schedulable.getSortedTaskSetQueue + sortedTaskSetQueue ++= schedulable.getSortedTaskSetQueue.filter(_.isSchedulable) } sortedTaskSetQueue } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala index 8cc239c81d11a..0626f8fb8150a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala @@ -39,6 +39,7 @@ private[spark] trait Schedulable { def stageId: Int def name: String + def isSchedulable: Boolean def addSchedulable(schedulable: Schedulable): Unit def removeSchedulable(schedulable: Schedulable): Unit def getSchedulableByName(name: String): Schedulable diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 107c517ca06bc..2fcf13d5268f8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -535,7 +535,7 @@ private[spark] class TaskSchedulerImpl( val availableResources = shuffledOffers.map(_.resources).toArray val availableCpus = shuffledOffers.map(o => o.cores).toArray val resourceProfileIds = shuffledOffers.map(o => o.resourceProfileId).toArray - val sortedTaskSets = rootPool.getSortedTaskSetQueue.filterNot(_.isZombie) + val sortedTaskSets = rootPool.getSortedTaskSetQueue for (taskSet <- sortedTaskSets) { logDebug("parentName: %s, name: %s, runningTasks: %s".format( taskSet.parent.name, taskSet.name, taskSet.runningTasks)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 673fe4fe27519..78fd412ef154c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -951,6 +951,9 @@ private[spark] class TaskSetManager( null } + override def isSchedulable: Boolean = !isZombie && + (pendingTasks.all.nonEmpty || pendingSpeculatableTasks.all.nonEmpty) + override def addSchedulable(schedulable: Schedulable): Unit = {} override def removeSchedulable(schedulable: Schedulable): Unit = {} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 0f144125af7bf..37ea648d80048 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -191,6 +191,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp executorDataMap.get(executorId).foreach(_.executorEndpoint.send(StopExecutor)) removeExecutor(executorId, reason) + // Do not change this code without running the K8s integration suites case DecommissionExecutor(executorId, decommissionInfo) => logError(s"Received decommission executor message ${executorId}: $decommissionInfo") decommissionExecutor(executorId, decommissionInfo, adjustTargetNumExecutors = false) @@ -272,6 +273,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp removeWorker(workerId, host, message) context.reply(true) + // Do not change this code without running the K8s integration suites case DecommissionExecutor(executorId, decommissionInfo) => logError(s"Received decommission executor message ${executorId}: ${decommissionInfo}.") context.reply(decommissionExecutor(executorId, decommissionInfo, @@ -469,6 +471,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)], adjustTargetNumExecutors: Boolean): Seq[String] = { + // Do not change this code without running the K8s integration suites val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) => CoarseGrainedSchedulerBackend.this.synchronized { // Only bother decommissioning executors which are alive. @@ -491,7 +494,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp }.map(_._1) } - + // Do not change this code without running the K8s integration suites private def doDecommission(executorId: String, decomInfo: ExecutorDecommissionInfo): Boolean = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/SupportsStreamingUpdate.scala b/core/src/main/scala/org/apache/spark/security/SecurityConfigurationLock.scala similarity index 70% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/SupportsStreamingUpdate.scala rename to core/src/main/scala/org/apache/spark/security/SecurityConfigurationLock.scala index 32be74a345c5a..0741a8c1580df 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/SupportsStreamingUpdate.scala +++ b/core/src/main/scala/org/apache/spark/security/SecurityConfigurationLock.scala @@ -15,12 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.internal.connector +package org.apache.spark.security -import org.apache.spark.sql.connector.write.WriteBuilder - -// An internal `WriteBuilder` mixin to support UPDATE streaming output mode. -// TODO: design an official API for streaming output mode UPDATE. -trait SupportsStreamingUpdate extends WriteBuilder { - def update(): WriteBuilder -} +/** + * There are cases when global JVM security configuration must be modified. + * In order to avoid race the modification must be synchronized with this. + */ +object SecurityConfigurationLock diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 400c4526f0114..4e2183451c258 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -24,8 +24,12 @@ import org.apache.spark.{ShuffleDependency, TaskContext} * and on each executor, based on the spark.shuffle.manager setting. The driver registers shuffles * with it, and executors (or tasks running locally in the driver) can ask to read and write data. * - * NOTE: this will be instantiated by SparkEnv so its constructor can take a SparkConf and + * NOTE: + * 1. This will be instantiated by SparkEnv so its constructor can take a SparkConf and * boolean isDriver as parameters. + * 2. This contains a method ShuffleBlockResolver which interacts with External Shuffle Service + * when it is enabled. Need to pay attention to that, if implementing a custom ShuffleManager, to + * make sure the custom ShuffleManager could co-exist with External Shuffle Service. */ private[spark] trait ShuffleManager { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index d207a6023f7f9..5a8cf09e1cba6 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -136,6 +136,10 @@ private[spark] class ExecutorMetricsJsonDeserializer new TypeReference[Option[Map[String, java.lang.Long]]] {}) metricsMap.map(metrics => new ExecutorMetrics(metrics)) } + + override def getNullValue(ctxt: DeserializationContext): Option[ExecutorMetrics] = { + None + } } /** serializer for peakMemoryMetrics: convert ExecutorMetrics to map with metric name as key */ private[spark] class ExecutorMetricsJsonSerializer @@ -144,11 +148,15 @@ private[spark] class ExecutorMetricsJsonSerializer metrics: Option[ExecutorMetrics], jsonGenerator: JsonGenerator, serializerProvider: SerializerProvider): Unit = { - metrics.foreach { m: ExecutorMetrics => - val metricsMap = ExecutorMetricType.metricToOffset.map { case (metric, _) => - metric -> m.getMetricValue(metric) + if (metrics.isEmpty) { + jsonGenerator.writeNull() + } else { + metrics.foreach { m: ExecutorMetrics => + val metricsMap = ExecutorMetricType.metricToOffset.map { case (metric, _) => + metric -> m.getMetricValue(metric) + } + jsonGenerator.writeObject(metricsMap) } - jsonGenerator.writeObject(metricsMap) } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index a3d42348befaa..61a88b4f26c00 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -380,16 +380,22 @@ class BlockManagerMasterEndpoint( * @return Seq of ReplicateBlock */ private def getReplicateInfoForRDDBlocks(blockManagerId: BlockManagerId): Seq[ReplicateBlock] = { - val info = blockManagerInfo(blockManagerId) + try { + val info = blockManagerInfo(blockManagerId) - val rddBlocks = info.blocks.keySet().asScala.filter(_.isRDD) - rddBlocks.map { blockId => - val currentBlockLocations = blockLocations.get(blockId) - val maxReplicas = currentBlockLocations.size + 1 - val remainingLocations = currentBlockLocations.toSeq.filter(bm => bm != blockManagerId) - val replicateMsg = ReplicateBlock(blockId, remainingLocations, maxReplicas) - replicateMsg - }.toSeq + val rddBlocks = info.blocks.keySet().asScala.filter(_.isRDD) + rddBlocks.map { blockId => + val currentBlockLocations = blockLocations.get(blockId) + val maxReplicas = currentBlockLocations.size + 1 + val remainingLocations = currentBlockLocations.toSeq.filter(bm => bm != blockManagerId) + val replicateMsg = ReplicateBlock(blockId, remainingLocations, maxReplicas) + replicateMsg + }.toSeq + } catch { + // If the block manager has already exited, nothing to replicate. + case e: java.util.NoSuchElementException => + Seq.empty[ReplicateBlock] + } } // Remove a block from the workers that have it. This can only be used to remove diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index a070cc9c7b39d..dba6f8e8440cb 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -231,7 +231,7 @@ private[spark] object UIUtils extends Logging { - + diff --git a/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala b/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala new file mode 100644 index 0000000000000..c0a135e04bac5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala @@ -0,0 +1,360 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io.FileNotFoundException + +import scala.collection.mutable + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.apache.hadoop.fs.viewfs.ViewFileSystem +import org.apache.hadoop.hdfs.DistributedFileSystem + +import org.apache.spark._ +import org.apache.spark.annotation.Private +import org.apache.spark.internal.Logging +import org.apache.spark.metrics.source.HiveCatalogMetrics + +/** + * Utility functions to simplify and speed-up file listing. + */ +private[spark] object HadoopFSUtils extends Logging { + /** + * Lists a collection of paths recursively. Picks the listing strategy adaptively depending + * on the number of paths to list. + * + * This may only be called on the driver. + * + * @param sc Spark context used to run parallel listing. + * @param paths Input paths to list + * @param hadoopConf Hadoop configuration + * @param filter Path filter used to exclude leaf files from result + * @param isRootLevel Whether the input paths are at the root level, i.e., they are the root + * paths as opposed to nested paths encountered during recursive calls of this. + * @param ignoreMissingFiles Ignore missing files that happen during recursive listing + * (e.g., due to race conditions) + * @param ignoreLocality Whether to fetch data locality info when listing leaf files. If false, + * this will return `FileStatus` without `BlockLocation` info. + * @param parallelismThreshold The threshold to enable parallelism. If the number of input paths + * is smaller than this value, this will fallback to use + * sequential listing. + * @param parallelismMax The maximum parallelism for listing. If the number of input paths is + * larger than this value, parallelism will be throttled to this value + * to avoid generating too many tasks. + * @param filterFun Optional predicate on the leaf files. Files who failed the check will be + * excluded from the results + * @return for each input path, the set of discovered files for the path + */ + def parallelListLeafFiles( + sc: SparkContext, + paths: Seq[Path], + hadoopConf: Configuration, + filter: PathFilter, + isRootLevel: Boolean, + ignoreMissingFiles: Boolean, + ignoreLocality: Boolean, + parallelismThreshold: Int, + parallelismMax: Int, + filterFun: Option[String => Boolean] = None): Seq[(Path, Seq[FileStatus])] = { + + // Short-circuits parallel listing when serial listing is likely to be faster. + if (paths.size <= parallelismThreshold) { + return paths.map { path => + val leafFiles = listLeafFiles( + path, + hadoopConf, + filter, + Some(sc), + ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, + isRootPath = isRootLevel, + parallelismThreshold = parallelismThreshold, + parallelismMax = parallelismMax, + filterFun = filterFun) + (path, leafFiles) + } + } + + logInfo(s"Listing leaf files and directories in parallel under ${paths.length} paths." + + s" The first several paths are: ${paths.take(10).mkString(", ")}.") + HiveCatalogMetrics.incrementParallelListingJobCount(1) + + val serializableConfiguration = new SerializableConfiguration(hadoopConf) + val serializedPaths = paths.map(_.toString) + + // Set the number of parallelism to prevent following file listing from generating many tasks + // in case of large #defaultParallelism. + val numParallelism = Math.min(paths.size, parallelismMax) + + val previousJobDescription = sc.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION) + val statusMap = try { + val description = paths.size match { + case 0 => + "Listing leaf files and directories 0 paths" + case 1 => + s"Listing leaf files and directories for 1 path:
${paths(0)}" + case s => + s"Listing leaf files and directories for $s paths:
${paths(0)}, ..." + } + sc.setJobDescription(description) + sc + .parallelize(serializedPaths, numParallelism) + .mapPartitions { pathStrings => + val hadoopConf = serializableConfiguration.value + pathStrings.map(new Path(_)).toSeq.map { path => + val leafFiles = listLeafFiles( + path = path, + hadoopConf = hadoopConf, + filter = filter, + contextOpt = None, // Can't execute parallel scans on workers + ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, + isRootPath = isRootLevel, + filterFun = filterFun, + parallelismThreshold = Int.MaxValue, + parallelismMax = 0) + (path, leafFiles) + }.iterator + }.map { case (path, statuses) => + val serializableStatuses = statuses.map { status => + // Turn FileStatus into SerializableFileStatus so we can send it back to the driver + val blockLocations = status match { + case f: LocatedFileStatus => + f.getBlockLocations.map { loc => + SerializableBlockLocation( + loc.getNames, + loc.getHosts, + loc.getOffset, + loc.getLength) + } + + case _ => + Array.empty[SerializableBlockLocation] + } + + SerializableFileStatus( + status.getPath.toString, + status.getLen, + status.isDirectory, + status.getReplication, + status.getBlockSize, + status.getModificationTime, + status.getAccessTime, + blockLocations) + } + (path.toString, serializableStatuses) + }.collect() + } finally { + sc.setJobDescription(previousJobDescription) + } + + // turn SerializableFileStatus back to Status + statusMap.map { case (path, serializableStatuses) => + val statuses = serializableStatuses.map { f => + val blockLocations = f.blockLocations.map { loc => + new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) + } + new LocatedFileStatus( + new FileStatus( + f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, + new Path(f.path)), + blockLocations) + } + (new Path(path), statuses) + } + } + + // scalastyle:off argcount + /** + * Lists a single filesystem path recursively. If a `SparkContext` object is specified, this + * function may launch Spark jobs to parallelize listing based on `parallelismThreshold`. + * + * If sessionOpt is None, this may be called on executors. + * + * @return all children of path that match the specified filter. + */ + private def listLeafFiles( + path: Path, + hadoopConf: Configuration, + filter: PathFilter, + contextOpt: Option[SparkContext], + ignoreMissingFiles: Boolean, + ignoreLocality: Boolean, + isRootPath: Boolean, + filterFun: Option[String => Boolean], + parallelismThreshold: Int, + parallelismMax: Int): Seq[FileStatus] = { + + logTrace(s"Listing $path") + val fs = path.getFileSystem(hadoopConf) + + // Note that statuses only include FileStatus for the files and dirs directly under path, + // and does not include anything else recursively. + val statuses: Array[FileStatus] = try { + fs match { + // DistributedFileSystem overrides listLocatedStatus to make 1 single call to namenode + // to retrieve the file status with the file block location. The reason to still fallback + // to listStatus is because the default implementation would potentially throw a + // FileNotFoundException which is better handled by doing the lookups manually below. + case (_: DistributedFileSystem | _: ViewFileSystem) if !ignoreLocality => + val remoteIter = fs.listLocatedStatus(path) + new Iterator[LocatedFileStatus]() { + def next(): LocatedFileStatus = remoteIter.next + def hasNext(): Boolean = remoteIter.hasNext + }.toArray + case _ => fs.listStatus(path) + } + } catch { + // If we are listing a root path for SQL (e.g. a top level directory of a table), we need to + // ignore FileNotFoundExceptions during this root level of the listing because + // + // (a) certain code paths might construct an InMemoryFileIndex with root paths that + // might not exist (i.e. not all callers are guaranteed to have checked + // path existence prior to constructing InMemoryFileIndex) and, + // (b) we need to ignore deleted root paths during REFRESH TABLE, otherwise we break + // existing behavior and break the ability drop SessionCatalog tables when tables' + // root directories have been deleted (which breaks a number of Spark's own tests). + // + // If we are NOT listing a root path then a FileNotFoundException here means that the + // directory was present in a previous level of file listing but is absent in this + // listing, likely indicating a race condition (e.g. concurrent table overwrite or S3 + // list inconsistency). + // + // The trade-off in supporting existing behaviors / use-cases is that we won't be + // able to detect race conditions involving root paths being deleted during + // InMemoryFileIndex construction. However, it's still a net improvement to detect and + // fail-fast on the non-root cases. For more info see the SPARK-27676 review discussion. + case _: FileNotFoundException if isRootPath || ignoreMissingFiles => + logWarning(s"The directory $path was not found. Was it deleted very recently?") + Array.empty[FileStatus] + } + + def doFilter(statuses: Array[FileStatus]) = filterFun match { + case Some(shouldFilterOut) => + statuses.filterNot(status => shouldFilterOut(status.getPath.getName)) + case None => + statuses + } + + val filteredStatuses = doFilter(statuses) + val allLeafStatuses = { + val (dirs, topLevelFiles) = filteredStatuses.partition(_.isDirectory) + val nestedFiles: Seq[FileStatus] = contextOpt match { + case Some(context) if dirs.size > parallelismThreshold => + parallelListLeafFiles( + context, + dirs.map(_.getPath), + hadoopConf = hadoopConf, + filter = filter, + isRootLevel = false, + ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, + filterFun = filterFun, + parallelismThreshold = parallelismThreshold, + parallelismMax = parallelismMax + ).flatMap(_._2) + case _ => + dirs.flatMap { dir => + listLeafFiles( + path = dir.getPath, + hadoopConf = hadoopConf, + filter = filter, + contextOpt = contextOpt, + ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, + isRootPath = false, + filterFun = filterFun, + parallelismThreshold = parallelismThreshold, + parallelismMax = parallelismMax) + } + } + val allFiles = topLevelFiles ++ nestedFiles + if (filter != null) allFiles.filter(f => filter.accept(f.getPath)) else allFiles + } + + val missingFiles = mutable.ArrayBuffer.empty[String] + val filteredLeafStatuses = doFilter(allLeafStatuses) + val resolvedLeafStatuses = filteredLeafStatuses.flatMap { + case f: LocatedFileStatus => + Some(f) + + // NOTE: + // + // - Although S3/S3A/S3N file system can be quite slow for remote file metadata + // operations, calling `getFileBlockLocations` does no harm here since these file system + // implementations don't actually issue RPC for this method. + // + // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not + // be a big deal since we always use to `parallelListLeafFiles` when the number of + // paths exceeds threshold. + case f if !ignoreLocality => + // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), + // which is very slow on some file system (RawLocalFileSystem, which is launch a + // subprocess and parse the stdout). + try { + val locations = fs.getFileBlockLocations(f, 0, f.getLen).map { loc => + // Store BlockLocation objects to consume less memory + if (loc.getClass == classOf[BlockLocation]) { + loc + } else { + new BlockLocation(loc.getNames, loc.getHosts, loc.getOffset, loc.getLength) + } + } + val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, + f.getModificationTime, 0, null, null, null, null, f.getPath, locations) + if (f.isSymlink) { + lfs.setSymlink(f.getSymlink) + } + Some(lfs) + } catch { + case _: FileNotFoundException if ignoreMissingFiles => + missingFiles += f.getPath.toString + None + } + + case f => Some(f) + } + + if (missingFiles.nonEmpty) { + logWarning( + s"the following files were missing during file scan:\n ${missingFiles.mkString("\n ")}") + } + + resolvedLeafStatuses + } + // scalastyle:on argcount + + /** A serializable variant of HDFS's BlockLocation. */ + private case class SerializableBlockLocation( + names: Array[String], + hosts: Array[String], + offset: Long, + length: Long) + + /** A serializable variant of HDFS's FileStatus. */ + private case class SerializableFileStatus( + path: String, + length: Long, + isDir: Boolean, + blockReplication: Short, + blockSize: Long, + modificationTime: Long, + accessTime: Long, + blockLocations: Array[SerializableBlockLocation]) +} diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java index 43977717f6c97..dc2b4814c8284 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java @@ -23,7 +23,6 @@ import java.util.LinkedList; import java.util.UUID; -import org.hamcrest.Matchers; import scala.Tuple2$; import org.junit.After; @@ -38,7 +37,6 @@ import org.apache.spark.executor.TaskMetrics; import org.apache.spark.internal.config.package$; import org.apache.spark.memory.TestMemoryManager; -import org.apache.spark.memory.SparkOutOfMemoryError; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.serializer.JavaSerializer; import org.apache.spark.serializer.SerializerInstance; @@ -359,6 +357,69 @@ public void forcedSpillingWithReadIterator() throws Exception { assertSpillFilesWereCleanedUp(); } + @Test + public void forcedSpillingNullsWithReadIterator() throws Exception { + final UnsafeExternalSorter sorter = newSorter(); + long[] record = new long[100]; + final int recordSize = record.length * 8; + final int n = (int) pageSizeBytes / recordSize * 3; + for (int i = 0; i < n; i++) { + boolean isNull = i % 2 == 0; + sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0, isNull); + } + assertTrue(sorter.getNumberOfAllocatedPages() >= 2); + + UnsafeExternalSorter.SpillableIterator iter = + (UnsafeExternalSorter.SpillableIterator) sorter.getSortedIterator(); + final int numRecordsToReadBeforeSpilling = n / 3; + for (int i = 0; i < numRecordsToReadBeforeSpilling; i++) { + assertTrue(iter.hasNext()); + iter.loadNext(); + } + + assertTrue(iter.spill() > 0); + assertEquals(0, iter.spill()); + + for (int i = numRecordsToReadBeforeSpilling; i < n; i++) { + assertTrue(iter.hasNext()); + iter.loadNext(); + } + assertFalse(iter.hasNext()); + + sorter.cleanupResources(); + assertSpillFilesWereCleanedUp(); + } + + @Test + public void forcedSpillingWithFullyReadIterator() throws Exception { + final UnsafeExternalSorter sorter = newSorter(); + long[] record = new long[100]; + final int recordSize = record.length * 8; + final int n = (int) pageSizeBytes / recordSize * 3; + for (int i = 0; i < n; i++) { + record[0] = i; + sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0, false); + } + assertTrue(sorter.getNumberOfAllocatedPages() >= 2); + + UnsafeExternalSorter.SpillableIterator iter = + (UnsafeExternalSorter.SpillableIterator) sorter.getSortedIterator(); + for (int i = 0; i < n; i++) { + assertTrue(iter.hasNext()); + iter.loadNext(); + assertEquals(i, Platform.getLong(iter.getBaseObject(), iter.getBaseOffset())); + } + assertFalse(iter.hasNext()); + + assertTrue(iter.spill() > 0); + assertEquals(0, iter.spill()); + assertEquals(n - 1, Platform.getLong(iter.getBaseObject(), iter.getBaseOffset())); + assertFalse(iter.hasNext()); + + sorter.cleanupResources(); + assertSpillFilesWereCleanedUp(); + } + @Test public void forcedSpillingWithNotReadIterator() throws Exception { final UnsafeExternalSorter sorter = newSorter(); @@ -518,40 +579,28 @@ public void testGetIterator() throws Exception { } @Test - public void testOOMDuringSpill() throws Exception { + public void testNoOOMDuringSpill() throws Exception { final UnsafeExternalSorter sorter = newSorter(); - // we assume that given default configuration, - // the size of the data we insert to the sorter (ints) - // and assuming we shouldn't spill before pointers array is exhausted - // (memory manager is not configured to throw at this point) - // - so this loop runs a reasonable number of iterations (<2000). - // test indeed completed within <30ms (on a quad i7 laptop). - for (int i = 0; sorter.hasSpaceForAnotherRecord(); ++i) { + for (int i = 0; i < 100; i++) { insertNumber(sorter, i); } - // we expect the next insert to attempt growing the pointerssArray first - // allocation is expected to fail, then a spill is triggered which - // attempts another allocation which also fails and we expect to see this - // OOM here. the original code messed with a released array within the - // spill code and ended up with a failed assertion. we also expect the - // location of the OOM to be - // org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.reset - memoryManager.markconsequentOOM(2); - try { - insertNumber(sorter, 1024); - fail("expected OutOfMmoryError but it seems operation surprisingly succeeded"); - } - // we expect an SparkOutOfMemoryError here, anything else (i.e the original NPE is a failure) - catch (SparkOutOfMemoryError oom){ - String oomStackTrace = Utils.exceptionString(oom); - assertThat("expected SparkOutOfMemoryError in " + - "org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.reset", - oomStackTrace, - Matchers.containsString( - "org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.reset")); + + // Check that spilling still succeeds when the task is starved for memory. + memoryManager.markconsequentOOM(Integer.MAX_VALUE); + sorter.spill(); + memoryManager.resetConsequentOOM(); + + // Ensure that records can be appended after spilling, i.e. check that the sorter will allocate + // the new pointer array that it could not allocate while spilling. + for (int i = 0; i < 100; ++i) { + insertNumber(sorter, i); } + + sorter.cleanupResources(); + assertSpillFilesWereCleanedUp(); } + private void verifyIntIterator(UnsafeSorterIterator iter, int start, int end) throws IOException { for (int i = start; i < end; i++) { diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java index 2b8a0602730e1..9d4909ddce792 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java @@ -20,6 +20,7 @@ import java.nio.charset.StandardCharsets; import java.util.Arrays; +import org.apache.spark.unsafe.array.LongArray; import org.junit.Assert; import org.junit.Test; @@ -27,7 +28,6 @@ import org.apache.spark.SparkConf; import org.apache.spark.memory.TestMemoryConsumer; import org.apache.spark.memory.TestMemoryManager; -import org.apache.spark.memory.SparkOutOfMemoryError; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.memory.MemoryBlock; @@ -37,7 +37,6 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.isIn; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; public class UnsafeInMemorySorterSuite { @@ -147,7 +146,7 @@ public int compare( } @Test - public void freeAfterOOM() { + public void testNoOOMDuringReset() { final SparkConf sparkConf = new SparkConf(); sparkConf.set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false); @@ -156,12 +155,7 @@ public void freeAfterOOM() { final TaskMemoryManager memoryManager = new TaskMemoryManager( testMemoryManager, 0); final TestMemoryConsumer consumer = new TestMemoryConsumer(memoryManager); - final MemoryBlock dataPage = memoryManager.allocatePage(2048, consumer); - final Object baseObject = dataPage.getBaseObject(); - // Write the records into the data page: - long position = dataPage.getBaseOffset(); - final HashPartitioner hashPartitioner = new HashPartitioner(4); // Use integer comparison for comparing prefixes (which are partition ids, in this case) final PrefixComparator prefixComparator = PrefixComparators.LONG; final RecordComparator recordComparator = new RecordComparator() { @@ -179,18 +173,24 @@ public int compare( UnsafeInMemorySorter sorter = new UnsafeInMemorySorter(consumer, memoryManager, recordComparator, prefixComparator, 100, shouldUseRadixSort()); - testMemoryManager.markExecutionAsOutOfMemoryOnce(); - try { - sorter.reset(); - fail("expected SparkOutOfMemoryError but it seems operation surprisingly succeeded"); - } catch (SparkOutOfMemoryError oom) { - // as expected - } - // [SPARK-21907] this failed on NPE at - // org.apache.spark.memory.MemoryConsumer.freeArray(MemoryConsumer.java:108) - sorter.free(); - // simulate a 'back to back' free. - sorter.free(); + // Ensure that the sorter does not OOM while freeing its memory. + testMemoryManager.markconsequentOOM(Integer.MAX_VALUE); + sorter.freeMemory(); + testMemoryManager.resetConsequentOOM(); + Assert.assertFalse(sorter.hasSpaceForAnotherRecord()); + + // Get the sorter in an usable state again by allocating a new pointer array. + LongArray array = consumer.allocateArray(1000); + sorter.expandPointerArray(array); + + // Ensure that it is safe to call freeMemory() multiple times. + testMemoryManager.markconsequentOOM(Integer.MAX_VALUE); + sorter.freeMemory(); + sorter.freeMemory(); + testMemoryManager.resetConsequentOOM(); + Assert.assertFalse(sorter.hasSpaceForAnotherRecord()); + + assertEquals(0L, memoryManager.cleanUpAllAllocatedMemory()); } } diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 3f8cbf59bf527..6a38bba5dd0e5 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.PrivateMethodTester import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config import org.apache.spark.internal.config.DECOMMISSION_ENABLED -import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL +import org.apache.spark.internal.config.Tests.TEST_DYNAMIC_ALLOCATION_SCHEDULE_ENABLED import org.apache.spark.metrics.MetricsSystem import org.apache.spark.resource._ import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID @@ -1665,9 +1665,10 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { .set(config.DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT.key, s"${executorIdleTimeout.toString}s") .set(config.SHUFFLE_SERVICE_ENABLED, true) .set(config.DYN_ALLOCATION_TESTING, true) - // SPARK-22864: effectively disable the allocation schedule by setting the period to a - // really long value. - .set(TEST_SCHEDULE_INTERVAL, 30000L) + // SPARK-22864/SPARK-32287: effectively disable the allocation schedule for the tests so that + // we won't result in the race condition between thread "spark-dynamic-executor-allocation" + // and thread "pool-1-thread-1-ScalaTest-running". + .set(TEST_DYNAMIC_ALLOCATION_SCHEDULE_ENABLED, false) .set(DECOMMISSION_ENABLED, decommissioningEnabled) sparkConf } diff --git a/core/src/test/scala/org/apache/spark/TempLocalSparkContext.scala b/core/src/test/scala/org/apache/spark/TempLocalSparkContext.scala new file mode 100644 index 0000000000000..6d5fcd1edfb03 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/TempLocalSparkContext.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import _root_.io.netty.util.internal.logging.{InternalLoggerFactory, Slf4JLoggerFactory} +import org.scalatest.BeforeAndAfterAll +import org.scalatest.BeforeAndAfterEach +import org.scalatest.Suite + +import org.apache.spark.internal.Logging +import org.apache.spark.resource.ResourceProfile + +/** + * Manages a local `sc` `SparkContext` variable, correctly stopping it after each test. + * + * Note: this class is a copy of [[LocalSparkContext]]. Why copy it? Reduce conflict. Because + * many test suites use [[LocalSparkContext]] and overwrite some variable or function (e.g. + * sc of LocalSparkContext), there occurs conflict when we refactor the `sc` as a new function. + * After migrating all test suites that use [[LocalSparkContext]] to use + * [[TempLocalSparkContext]], we will delete the original [[LocalSparkContext]] and rename + * [[TempLocalSparkContext]] to [[LocalSparkContext]]. + */ +trait TempLocalSparkContext extends BeforeAndAfterEach + with BeforeAndAfterAll with Logging { self: Suite => + + private var _conf: SparkConf = defaultSparkConf + + @transient private var _sc: SparkContext = _ + + def conf: SparkConf = _conf + + /** + * Currently, we are focusing on the reconstruction of LocalSparkContext, so this method + * was created temporarily. When the migration work is completed, this method will be + * renamed to `sc` and the variable `sc` will be deleted. + */ + def sc: SparkContext = { + if (_sc == null) { + _sc = new SparkContext(_conf) + } + _sc + } + + override def beforeAll(): Unit = { + super.beforeAll() + InternalLoggerFactory.setDefaultFactory(Slf4JLoggerFactory.INSTANCE) + } + + override def afterEach(): Unit = { + try { + resetSparkContext() + } finally { + super.afterEach() + } + } + + def resetSparkContext(): Unit = { + TempLocalSparkContext.stop(_sc) + ResourceProfile.clearDefaultProfile() + _sc = null + _conf = defaultSparkConf + } + + private def defaultSparkConf: SparkConf = new SparkConf() + .setMaster("local[2]").setAppName(s"${this.getClass.getSimpleName}") +} + +object TempLocalSparkContext { + def stop(sc: SparkContext): Unit = { + if (sc != null) { + sc.stop() + } + // To avoid RPC rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.driver.port") + } + + /** Runs `f` by passing in `sc` and ensures that `sc` is stopped. */ + def withSpark[T](sc: SparkContext)(f: SparkContext => T): T = { + try { + f(sc) + } finally { + stop(sc) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceMetricsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceMetricsSuite.scala index d681c13337e0d..ea4d252f0dbae 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceMetricsSuite.scala @@ -61,7 +61,8 @@ class ExternalShuffleServiceMetricsSuite extends SparkFunSuite { "registeredExecutorsSize", "registerExecutorRequestLatencyMillis", "shuffle-server.usedDirectMemory", - "shuffle-server.usedHeapMemory") + "shuffle-server.usedHeapMemory", + "finalizeShuffleMergeLatencyMillis") ) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala index 060b878fb8ef2..e9b739ce7a4c6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala @@ -213,7 +213,7 @@ class SingleEventLogFileWriterSuite extends EventLogFileWritersSuite { compressionCodecShortName) val finalLogPath = new Path(logPath) - assert(fileSystem.exists(finalLogPath) && fileSystem.isFile(finalLogPath)) + assert(fileSystem.exists(finalLogPath) && fileSystem.getFileStatus(finalLogPath).isFile) assert(expectedLines === readLinesFromEventLogFile(finalLogPath, fileSystem)) } } @@ -357,10 +357,10 @@ class RollingEventLogFilesWriterSuite extends EventLogFileWritersSuite { expectedLines: Seq[String]): Unit = { val logDirPath = getAppEventLogDirPath(logBaseDir, appId, appAttemptId) - assert(fileSystem.exists(logDirPath) && fileSystem.isDirectory(logDirPath)) + assert(fileSystem.exists(logDirPath) && fileSystem.getFileStatus(logDirPath).isDirectory) val appStatusFile = getAppStatusFilePath(logDirPath, appId, appAttemptId, inProgress = false) - assert(fileSystem.exists(appStatusFile) && fileSystem.isFile(appStatusFile)) + assert(fileSystem.exists(appStatusFile) && fileSystem.getFileStatus(appStatusFile).isFile) val eventLogFiles = listEventLogFiles(logDirPath) val allLines = mutable.ArrayBuffer[String]() diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 3f8c875f5a552..1578b908b1b55 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -1475,6 +1475,55 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { } } + test("SPARK-33146: don't let one bad rolling log folder prevent loading other applications") { + withTempDir { dir => + val conf = createTestConf(true) + conf.set(HISTORY_LOG_DIR, dir.getAbsolutePath) + val hadoopConf = SparkHadoopUtil.newConfiguration(conf) + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + val provider = new FsHistoryProvider(conf) + + val writer = new RollingEventLogFilesWriter("app", None, dir.toURI, conf, hadoopConf) + writer.start() + + writeEventsToRollingWriter(writer, Seq( + SparkListenerApplicationStart("app", Some("app"), 0, "user", None), + SparkListenerJobStart(1, 0, Seq.empty)), rollFile = false) + provider.checkForLogs() + provider.cleanLogs() + assert(dir.listFiles().size === 1) + assert(provider.getListing.length === 1) + + // Manually delete the appstatus file to make an invalid rolling event log + val appStatusPath = RollingEventLogFilesWriter.getAppStatusFilePath(new Path(writer.logPath), + "app", None, true) + fs.delete(appStatusPath, false) + provider.checkForLogs() + provider.cleanLogs() + assert(provider.getListing.length === 0) + + // Create a new application + val writer2 = new RollingEventLogFilesWriter("app2", None, dir.toURI, conf, hadoopConf) + writer2.start() + writeEventsToRollingWriter(writer2, Seq( + SparkListenerApplicationStart("app2", Some("app2"), 0, "user", None), + SparkListenerJobStart(1, 0, Seq.empty)), rollFile = false) + + // Both folders exist but only one application found + provider.checkForLogs() + provider.cleanLogs() + assert(provider.getListing.length === 1) + assert(dir.listFiles().size === 2) + + // Make sure a new provider sees the valid application + provider.stop() + val newProvider = new FsHistoryProvider(conf) + newProvider.checkForLogs() + assert(newProvider.getListing.length === 1) + } + } + /** * Asks the provider to check for logs and calls a function to perform checks on the updated * app list. Example: diff --git a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala index 60f67699f81be..987f383c9c4fa 100644 --- a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala +++ b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala @@ -119,6 +119,14 @@ class TestMemoryManager(conf: SparkConf) consequentOOM += n } + /** + * Undos the effects of [[markExecutionAsOutOfMemoryOnce]] and [[markconsequentOOM]] and lets + * calls to [[acquireExecutionMemory()]] (if there is enough memory available). + */ + def resetConsequentOOM(): Unit = synchronized { + consequentOOM = 0 + } + def limit(avail: Long): Unit = synchronized { require(avail >= 0) available = avail diff --git a/core/src/test/scala/org/apache/spark/metrics/sink/StatsdSinkSuite.scala b/core/src/test/scala/org/apache/spark/metrics/sink/StatsdSinkSuite.scala index 0e21a36071c42..3d4b8c868d6fc 100644 --- a/core/src/test/scala/org/apache/spark/metrics/sink/StatsdSinkSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/sink/StatsdSinkSuite.scala @@ -35,12 +35,27 @@ class StatsdSinkSuite extends SparkFunSuite { STATSD_KEY_UNIT -> "seconds", STATSD_KEY_HOST -> "127.0.0.1" ) - private val socketTimeout = 30000 // milliseconds - private val socketBufferSize = 8192 + // The maximum size of a single datagram packet payload. Payloads + // larger than this will be truncated. + private val maxPayloadSize = 256 // bytes + + // The receive buffer must be large enough to hold all inflight + // packets. This includes any kernel and protocol overhead. + // This value was determined experimentally and should be + // increased if timeouts are seen. + private val socketMinRecvBufferSize = 16384 // bytes + private val socketTimeout = 30000 // milliseconds private def withSocketAndSink(testCode: (DatagramSocket, StatsdSink) => Any): Unit = { val socket = new DatagramSocket - socket.setReceiveBufferSize(socketBufferSize) + + // Leave the receive buffer size untouched unless it is too + // small. If the receive buffer is too small packets will be + // silently dropped and receive operations will timeout. + if (socket.getReceiveBufferSize() < socketMinRecvBufferSize) { + socket.setReceiveBufferSize(socketMinRecvBufferSize) + } + socket.setSoTimeout(socketTimeout) val props = new Properties defaultProps.foreach(e => props.put(e._1, e._2)) @@ -61,7 +76,7 @@ class StatsdSinkSuite extends SparkFunSuite { sink.registry.register("counter", counter) sink.report() - val p = new DatagramPacket(new Array[Byte](socketBufferSize), socketBufferSize) + val p = new DatagramPacket(new Array[Byte](maxPayloadSize), maxPayloadSize) socket.receive(p) val result = new String(p.getData, 0, p.getLength, UTF_8) @@ -77,7 +92,7 @@ class StatsdSinkSuite extends SparkFunSuite { sink.registry.register("gauge", gauge) sink.report() - val p = new DatagramPacket(new Array[Byte](socketBufferSize), socketBufferSize) + val p = new DatagramPacket(new Array[Byte](maxPayloadSize), maxPayloadSize) socket.receive(p) val result = new String(p.getData, 0, p.getLength, UTF_8) @@ -87,7 +102,7 @@ class StatsdSinkSuite extends SparkFunSuite { test("metrics StatsD sink with Histogram") { withSocketAndSink { (socket, sink) => - val p = new DatagramPacket(new Array[Byte](socketBufferSize), socketBufferSize) + val p = new DatagramPacket(new Array[Byte](maxPayloadSize), maxPayloadSize) val histogram = new Histogram(new UniformReservoir) histogram.update(10) histogram.update(20) @@ -121,7 +136,7 @@ class StatsdSinkSuite extends SparkFunSuite { test("metrics StatsD sink with Timer") { withSocketAndSink { (socket, sink) => - val p = new DatagramPacket(new Array[Byte](socketBufferSize), socketBufferSize) + val p = new DatagramPacket(new Array[Byte](maxPayloadSize), maxPayloadSize) val timer = new Timer() timer.update(1, SECONDS) timer.update(2, SECONDS) diff --git a/core/src/test/scala/org/apache/spark/rpc/netty/InboxSuite.scala b/core/src/test/scala/org/apache/spark/rpc/netty/InboxSuite.scala index c74c728b3e3f3..8b1c602cd8e58 100644 --- a/core/src/test/scala/org/apache/spark/rpc/netty/InboxSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/netty/InboxSuite.scala @@ -136,4 +136,17 @@ class InboxSuite extends SparkFunSuite { endpoint.verifySingleOnNetworkErrorMessage(cause, remoteAddress) } + + test("SPARK-32738: should reduce the number of active threads when fatal error happens") { + val endpoint = mock(classOf[TestRpcEndpoint]) + when(endpoint.receive).thenThrow(new OutOfMemoryError()) + + val dispatcher = mock(classOf[Dispatcher]) + val inbox = new Inbox("name", endpoint) + inbox.post(OneWayMessage(null, "hi")) + intercept[OutOfMemoryError] { + inbox.process(dispatcher) + } + assert(inbox.getNumActiveThreads == 0) + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala index d18ca36f1fa60..e4ec62f8efc5b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala @@ -189,7 +189,7 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext with test("throw exception if the number of barrier() calls are not the same on every task") { initLocalClusterSparkContext() - sc.conf.set("spark.barrier.sync.timeout", "1") + sc.conf.set("spark.barrier.sync.timeout", "5") val rdd = sc.makeRDD(1 to 10, 4) val rdd2 = rdd.barrier().mapPartitions { it => val context = BarrierTaskContext.get() @@ -212,7 +212,7 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext with rdd2.collect() }.getMessage assert(error.contains("The coordinator didn't get all barrier sync requests")) - assert(error.contains("within 1 second(s)")) + assert(error.contains("within 5 second(s)")) } def testBarrierTaskKilled(interruptOnKill: Boolean): Unit = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index fd2d9029c0836..298962d196778 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -126,14 +126,14 @@ class MyRDD( class DAGSchedulerSuiteDummyException extends Exception -class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLimits { +class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with TimeLimits { import DAGSchedulerSuite._ // Necessary to make ScalaTest 3.x interrupt a thread on the JVM like ScalaTest 2.2.x implicit val defaultSignaler: Signaler = ThreadSignaler - val conf = new SparkConf + private var firstInit: Boolean = _ /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() @@ -299,11 +299,19 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi override def beforeEach(): Unit = { super.beforeEach() - init(new SparkConf()) + firstInit = true } - private def init(testConf: SparkConf): Unit = { - sc = new SparkContext("local[2]", "DAGSchedulerSuite", testConf) + override def sc: SparkContext = { + val sc = super.sc + if (firstInit) { + init(sc) + firstInit = false + } + sc + } + + private def init(sc: SparkContext): Unit = { sparkListener = new EventInfoRecordingListener failure = null sc.addSparkListener(sparkListener) @@ -312,11 +320,11 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi cancelledStages.clear() cacheLocations.clear() results.clear() - securityMgr = new SecurityManager(conf) - broadcastManager = new BroadcastManager(true, conf, securityMgr) + securityMgr = new SecurityManager(sc.getConf) + broadcastManager = new BroadcastManager(true, sc.getConf, securityMgr) mapOutputTracker = spy( - new MyMapOutputTrackerMaster(conf, new NoOpShuffleOutputTracker(), broadcastManager)) - blockManagerMaster = spy(new MyBlockManagerMaster(conf)) + new MyMapOutputTrackerMaster(sc.getConf, new NoOpShuffleOutputTracker(), broadcastManager)) + blockManagerMaster = spy(new MyBlockManagerMaster(sc.getConf)) scheduler = new DAGScheduler( sc, taskScheduler, @@ -356,6 +364,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi * DAGScheduler event loop. */ private def runEvent(event: DAGSchedulerEvent): Unit = { + // Ensure the initialization of various components + sc dagEventProcessLoopTester.post(event) } @@ -494,12 +504,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } test("All shuffle files on the storage endpoint should be cleaned up when it is lost") { - // reset the test context with the right shuffle service config - afterEach() - val conf = new SparkConf() conf.set(config.SHUFFLE_SERVICE_ENABLED.key, "true") conf.set("spark.files.fetchFailure.unRegisterOutputOnHost", "true") - init(conf) runEvent(ExecutorAdded("hostA-exec1", "hostA")) runEvent(ExecutorAdded("hostA-exec2", "hostA")) runEvent(ExecutorAdded("hostB-exec", "hostB")) @@ -578,11 +584,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } test("SPARK-32003: All shuffle files for executor should be cleaned up on fetch failure") { - // reset the test context with the right shuffle service config - afterEach() - val conf = new SparkConf() conf.set(config.SHUFFLE_SERVICE_ENABLED.key, "true") - init(conf) val shuffleMapRdd = new MyRDD(sc, 3, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(3)) @@ -874,11 +876,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi "not lost" } test(s"shuffle files $maybeLost when $eventDescription") { - // reset the test context with the right shuffle service config - afterEach() - val conf = new SparkConf() conf.set(config.SHUFFLE_SERVICE_ENABLED.key, shuffleServiceOn.toString) - init(conf) assert(sc.env.blockManager.externalShuffleServiceEnabled == shuffleServiceOn) val shuffleMapRdd = new MyRDD(sc, 2, Nil) @@ -2901,11 +2899,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } test("SPARK-25341: abort stage while using old fetch protocol") { - // reset the test context with using old fetch protocol - afterEach() - val conf = new SparkConf() conf.set(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL.key, "true") - init(conf) // Construct the scenario of indeterminate stage fetch failed. constructIndeterminateStageFetchFailed() // The job should fail because Spark can't rollback the shuffle map stage while @@ -3233,10 +3227,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } test("test 2 resource profile with merge conflict config true") { - afterEach() - val conf = new SparkConf() conf.set(config.RESOURCE_PROFILE_MERGE_CONFLICTS.key, "true") - init(conf) val ereqs = new ExecutorResourceRequests().cores(4) val treqs = new TaskResourceRequests().cpus(1) @@ -3254,10 +3245,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } test("test multiple resource profiles created from merging use same rp") { - afterEach() - val conf = new SparkConf() conf.set(config.RESOURCE_PROFILE_MERGE_CONFLICTS.key, "true") - init(conf) val ereqs = new ExecutorResourceRequests().cores(4) val treqs = new TaskResourceRequests().cpus(1) @@ -3351,10 +3339,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } test("test merge 3 resource profiles") { - afterEach() - val conf = new SparkConf() conf.set(config.RESOURCE_PROFILE_MERGE_CONFLICTS.key, "true") - init(conf) val ereqs = new ExecutorResourceRequests().cores(4) val treqs = new TaskResourceRequests().cpus(1) val rp1 = new ResourceProfile(ereqs.requests, treqs.requests) diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/ExecutorSummarySuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/ExecutorSummarySuite.scala new file mode 100644 index 0000000000000..286911bdfc19a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/status/api/v1/ExecutorSummarySuite.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import java.util.Date + +import com.fasterxml.jackson.core.`type`.TypeReference +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule + +import org.apache.spark.SparkFunSuite + +class ExecutorSummarySuite extends SparkFunSuite { + + test("Check ExecutorSummary serialize and deserialize with empty peakMemoryMetrics") { + val mapper = new ObjectMapper().registerModule(DefaultScalaModule) + val executorSummary = new ExecutorSummary("id", "host:port", true, 1, + 10, 10, 1, 1, 1, + 0, 0, 1, 100, + 1, 100, 100, + 10, false, 20, new Date(1600984336352L), + Option.empty, Option.empty, Map(), Option.empty, Set(), Option.empty, Map(), Map(), 1) + val expectedJson = "{\"id\":\"id\",\"hostPort\":\"host:port\",\"isActive\":true," + + "\"rddBlocks\":1,\"memoryUsed\":10,\"diskUsed\":10,\"totalCores\":1,\"maxTasks\":1," + + "\"activeTasks\":1,\"failedTasks\":0,\"completedTasks\":0,\"totalTasks\":1," + + "\"totalDuration\":100,\"totalGCTime\":1,\"totalInputBytes\":100," + + "\"totalShuffleRead\":100,\"totalShuffleWrite\":10,\"isBlacklisted\":false," + + "\"maxMemory\":20,\"addTime\":1600984336352,\"removeTime\":null,\"removeReason\":null," + + "\"executorLogs\":{},\"memoryMetrics\":null,\"blacklistedInStages\":[]," + + "\"peakMemoryMetrics\":null,\"attributes\":{},\"resources\":{},\"resourceProfileId\":1}" + val json = mapper.writeValueAsString(executorSummary) + assert(expectedJson.equals(json)) + val deserializeExecutorSummary = mapper.readValue(json, new TypeReference[ExecutorSummary] {}) + assert(deserializeExecutorSummary.peakMemoryMetrics == None) + } + +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala index 094b893cdda2e..dcf313f671d5e 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala @@ -69,6 +69,8 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS .set(config.STORAGE_DECOMMISSION_ENABLED, true) .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED, persist) .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED, shuffle) + // Since we use the bus for testing we don't want to drop any messages + .set(config.LISTENER_BUS_EVENT_QUEUE_CAPACITY, 1000000) // Just replicate blocks quickly during testing, there isn't another // workload we need to worry about. .set(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL, 10L) @@ -137,7 +139,7 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS taskEndEvents.add(taskEnd) } - override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { + override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = synchronized { blocksUpdated.append(blockUpdated) } diff --git a/dev/.rat-excludes b/dev/.rat-excludes index df1dd51a7c519..0e892a927906a 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -25,7 +25,7 @@ bootstrap.bundle.min.js bootstrap.bundle.min.js.map bootstrap.min.css bootstrap.min.css.map -jquery-3.4.1.min.js +jquery-3.5.1.min.js d3.min.js dagre-d3.min.js graphlib-dot.min.js @@ -124,3 +124,4 @@ GangliaReporter.java application_1578436911597_0052 config.properties app-20200706201101-0003 +py.typed diff --git a/dev/appveyor-install-dependencies.ps1 b/dev/appveyor-install-dependencies.ps1 index e344a7fc23191..fb4cc22de35f4 100644 --- a/dev/appveyor-install-dependencies.ps1 +++ b/dev/appveyor-install-dependencies.ps1 @@ -19,7 +19,7 @@ $CRAN = "https://cloud.r-project.org" Function InstallR { if ( -not(Test-Path Env:\R_ARCH) ) { - $arch = "i386" + $arch = "x64" } Else { $arch = $env:R_ARCH @@ -68,7 +68,7 @@ Function InstallRtools { $gccPath = $env:GCC_PATH } $env:PATH = $RtoolsDrive + '\Rtools40\bin;' + $RtoolsDrive + '\Rtools40\mingw64\bin;' + $RtoolsDrive + '\Rtools40\' + $gccPath + '\bin;' + $env:PATH - $env:BINPREF=$RtoolsDrive + '/Rtools40/mingw64/bin/' + $env:BINPREF=$RtoolsDrive + '/Rtools40/mingw$(WIN)/bin/' } # create tools directory outside of Spark directory diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 31633456a6590..c7fee13d39c6b 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -275,6 +275,9 @@ if [[ "$1" == "package" ]]; then # In dry run mode, only build the first one. The keys in BINARY_PKGS_ARGS are used as the # list of packages to be built, so it's ok for things to be missing in BINARY_PKGS_EXTRA. + # NOTE: Don't forget to update the valid combinations of distributions at + # 'python/pyspark/install.py' and 'python/docs/source/getting_started/install.rst' + # if you're changing them. declare -A BINARY_PKGS_ARGS BINARY_PKGS_ARGS["hadoop3.2"]="-Phadoop-3.2 $HIVE_PROFILES" if ! is_dry_run; then @@ -282,13 +285,12 @@ if [[ "$1" == "package" ]]; then if [[ $SPARK_VERSION < "3.0." ]]; then BINARY_PKGS_ARGS["hadoop2.6"]="-Phadoop-2.6 $HIVE_PROFILES" else - BINARY_PKGS_ARGS["hadoop2.7-hive1.2"]="-Phadoop-2.7 -Phive-1.2 $HIVE_PROFILES" BINARY_PKGS_ARGS["hadoop2.7"]="-Phadoop-2.7 $HIVE_PROFILES" fi fi declare -A BINARY_PKGS_EXTRA - BINARY_PKGS_EXTRA["hadoop2.7"]="withpip,withr" + BINARY_PKGS_EXTRA["hadoop3.2"]="withpip,withr" if [[ $PUBLISH_SCALA_2_11 = 1 ]]; then key="without-hadoop-scala-2.11" diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 deleted file mode 100644 index 5a5da2bd624e3..0000000000000 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ /dev/null @@ -1,212 +0,0 @@ -JLargeArrays/1.5//JLargeArrays-1.5.jar -JTransforms/3.1//JTransforms-3.1.jar -JavaEWAH/0.3.2//JavaEWAH-0.3.2.jar -RoaringBitmap/0.9.0//RoaringBitmap-0.9.0.jar -ST4/4.0.4//ST4-4.0.4.jar -activation/1.1.1//activation-1.1.1.jar -aircompressor/0.10//aircompressor-0.10.jar -algebra_2.12/2.0.0-M2//algebra_2.12-2.0.0-M2.jar -antlr-runtime/3.4//antlr-runtime-3.4.jar -antlr/2.7.7//antlr-2.7.7.jar -antlr4-runtime/4.7.1//antlr4-runtime-4.7.1.jar -aopalliance-repackaged/2.6.1//aopalliance-repackaged-2.6.1.jar -aopalliance/1.0//aopalliance-1.0.jar -apache-log4j-extras/1.2.17//apache-log4j-extras-1.2.17.jar -apacheds-i18n/2.0.0-M15//apacheds-i18n-2.0.0-M15.jar -apacheds-kerberos-codec/2.0.0-M15//apacheds-kerberos-codec-2.0.0-M15.jar -api-asn1-api/1.0.0-M20//api-asn1-api-1.0.0-M20.jar -api-util/1.0.0-M20//api-util-1.0.0-M20.jar -arpack_combined_all/0.1//arpack_combined_all-0.1.jar -arrow-format/0.15.1//arrow-format-0.15.1.jar -arrow-memory/0.15.1//arrow-memory-0.15.1.jar -arrow-vector/0.15.1//arrow-vector-0.15.1.jar -audience-annotations/0.5.0//audience-annotations-0.5.0.jar -automaton/1.11-8//automaton-1.11-8.jar -avro-ipc/1.8.2//avro-ipc-1.8.2.jar -avro-mapred/1.8.2/hadoop2/avro-mapred-1.8.2-hadoop2.jar -avro/1.8.2//avro-1.8.2.jar -bonecp/0.8.0.RELEASE//bonecp-0.8.0.RELEASE.jar -breeze-macros_2.12/1.0//breeze-macros_2.12-1.0.jar -breeze_2.12/1.0//breeze_2.12-1.0.jar -cats-kernel_2.12/2.0.0-M4//cats-kernel_2.12-2.0.0-M4.jar -chill-java/0.9.5//chill-java-0.9.5.jar -chill_2.12/0.9.5//chill_2.12-0.9.5.jar -commons-beanutils/1.9.4//commons-beanutils-1.9.4.jar -commons-cli/1.2//commons-cli-1.2.jar -commons-codec/1.10//commons-codec-1.10.jar -commons-collections/3.2.2//commons-collections-3.2.2.jar -commons-compiler/3.0.16//commons-compiler-3.0.16.jar -commons-compress/1.8.1//commons-compress-1.8.1.jar -commons-configuration/1.6//commons-configuration-1.6.jar -commons-crypto/1.0.0//commons-crypto-1.0.0.jar -commons-dbcp/1.4//commons-dbcp-1.4.jar -commons-digester/1.8//commons-digester-1.8.jar -commons-httpclient/3.1//commons-httpclient-3.1.jar -commons-io/2.4//commons-io-2.4.jar -commons-lang/2.6//commons-lang-2.6.jar -commons-lang3/3.10//commons-lang3-3.10.jar -commons-logging/1.1.3//commons-logging-1.1.3.jar -commons-math3/3.4.1//commons-math3-3.4.1.jar -commons-net/3.1//commons-net-3.1.jar -commons-pool/1.5.4//commons-pool-1.5.4.jar -commons-text/1.6//commons-text-1.6.jar -compress-lzf/1.0.3//compress-lzf-1.0.3.jar -core/1.1.2//core-1.1.2.jar -curator-client/2.7.1//curator-client-2.7.1.jar -curator-framework/2.7.1//curator-framework-2.7.1.jar -curator-recipes/2.7.1//curator-recipes-2.7.1.jar -datanucleus-api-jdo/3.2.6//datanucleus-api-jdo-3.2.6.jar -datanucleus-core/3.2.10//datanucleus-core-3.2.10.jar -datanucleus-rdbms/3.2.9//datanucleus-rdbms-3.2.9.jar -derby/10.12.1.1//derby-10.12.1.1.jar -flatbuffers-java/1.9.0//flatbuffers-java-1.9.0.jar -generex/1.0.2//generex-1.0.2.jar -gson/2.2.4//gson-2.2.4.jar -guava/14.0.1//guava-14.0.1.jar -guice-servlet/3.0//guice-servlet-3.0.jar -guice/3.0//guice-3.0.jar -hadoop-annotations/2.7.4//hadoop-annotations-2.7.4.jar -hadoop-auth/2.7.4//hadoop-auth-2.7.4.jar -hadoop-client/2.7.4//hadoop-client-2.7.4.jar -hadoop-common/2.7.4//hadoop-common-2.7.4.jar -hadoop-hdfs/2.7.4//hadoop-hdfs-2.7.4.jar -hadoop-mapreduce-client-app/2.7.4//hadoop-mapreduce-client-app-2.7.4.jar -hadoop-mapreduce-client-common/2.7.4//hadoop-mapreduce-client-common-2.7.4.jar -hadoop-mapreduce-client-core/2.7.4//hadoop-mapreduce-client-core-2.7.4.jar -hadoop-mapreduce-client-jobclient/2.7.4//hadoop-mapreduce-client-jobclient-2.7.4.jar -hadoop-mapreduce-client-shuffle/2.7.4//hadoop-mapreduce-client-shuffle-2.7.4.jar -hadoop-yarn-api/2.7.4//hadoop-yarn-api-2.7.4.jar -hadoop-yarn-client/2.7.4//hadoop-yarn-client-2.7.4.jar -hadoop-yarn-common/2.7.4//hadoop-yarn-common-2.7.4.jar -hadoop-yarn-server-common/2.7.4//hadoop-yarn-server-common-2.7.4.jar -hadoop-yarn-server-web-proxy/2.7.4//hadoop-yarn-server-web-proxy-2.7.4.jar -hk2-api/2.6.1//hk2-api-2.6.1.jar -hk2-locator/2.6.1//hk2-locator-2.6.1.jar -hk2-utils/2.6.1//hk2-utils-2.6.1.jar -htrace-core/3.1.0-incubating//htrace-core-3.1.0-incubating.jar -httpclient/4.5.6//httpclient-4.5.6.jar -httpcore/4.4.12//httpcore-4.4.12.jar -istack-commons-runtime/3.0.8//istack-commons-runtime-3.0.8.jar -ivy/2.4.0//ivy-2.4.0.jar -jackson-annotations/2.10.0//jackson-annotations-2.10.0.jar -jackson-core-asl/1.9.13//jackson-core-asl-1.9.13.jar -jackson-core/2.10.0//jackson-core-2.10.0.jar -jackson-databind/2.10.0//jackson-databind-2.10.0.jar -jackson-dataformat-yaml/2.10.0//jackson-dataformat-yaml-2.10.0.jar -jackson-datatype-jsr310/2.10.3//jackson-datatype-jsr310-2.10.3.jar -jackson-jaxrs/1.9.13//jackson-jaxrs-1.9.13.jar -jackson-mapper-asl/1.9.13//jackson-mapper-asl-1.9.13.jar -jackson-module-jaxb-annotations/2.10.0//jackson-module-jaxb-annotations-2.10.0.jar -jackson-module-paranamer/2.10.0//jackson-module-paranamer-2.10.0.jar -jackson-module-scala_2.12/2.10.0//jackson-module-scala_2.12-2.10.0.jar -jackson-xc/1.9.13//jackson-xc-1.9.13.jar -jakarta.activation-api/1.2.1//jakarta.activation-api-1.2.1.jar -jakarta.annotation-api/1.3.5//jakarta.annotation-api-1.3.5.jar -jakarta.inject/2.6.1//jakarta.inject-2.6.1.jar -jakarta.validation-api/2.0.2//jakarta.validation-api-2.0.2.jar -jakarta.ws.rs-api/2.1.6//jakarta.ws.rs-api-2.1.6.jar -jakarta.xml.bind-api/2.3.2//jakarta.xml.bind-api-2.3.2.jar -janino/3.0.16//janino-3.0.16.jar -javassist/3.25.0-GA//javassist-3.25.0-GA.jar -javax.inject/1//javax.inject-1.jar -javax.servlet-api/3.1.0//javax.servlet-api-3.1.0.jar -javolution/5.5.1//javolution-5.5.1.jar -jaxb-api/2.2.2//jaxb-api-2.2.2.jar -jaxb-runtime/2.3.2//jaxb-runtime-2.3.2.jar -jcl-over-slf4j/1.7.30//jcl-over-slf4j-1.7.30.jar -jdo-api/3.0.1//jdo-api-3.0.1.jar -jersey-client/2.30//jersey-client-2.30.jar -jersey-common/2.30//jersey-common-2.30.jar -jersey-container-servlet-core/2.30//jersey-container-servlet-core-2.30.jar -jersey-container-servlet/2.30//jersey-container-servlet-2.30.jar -jersey-hk2/2.30//jersey-hk2-2.30.jar -jersey-media-jaxb/2.30//jersey-media-jaxb-2.30.jar -jersey-server/2.30//jersey-server-2.30.jar -jetty-sslengine/6.1.26//jetty-sslengine-6.1.26.jar -jetty-util/6.1.26//jetty-util-6.1.26.jar -jetty/6.1.26//jetty-6.1.26.jar -jline/2.14.6//jline-2.14.6.jar -joda-time/2.10.5//joda-time-2.10.5.jar -jodd-core/3.5.2//jodd-core-3.5.2.jar -jpam/1.1//jpam-1.1.jar -json4s-ast_2.12/3.7.0-M5//json4s-ast_2.12-3.7.0-M5.jar -json4s-core_2.12/3.7.0-M5//json4s-core_2.12-3.7.0-M5.jar -json4s-jackson_2.12/3.7.0-M5//json4s-jackson_2.12-3.7.0-M5.jar -json4s-scalap_2.12/3.7.0-M5//json4s-scalap_2.12-3.7.0-M5.jar -jsp-api/2.1//jsp-api-2.1.jar -jsr305/3.0.0//jsr305-3.0.0.jar -jta/1.1//jta-1.1.jar -jul-to-slf4j/1.7.30//jul-to-slf4j-1.7.30.jar -kryo-shaded/4.0.2//kryo-shaded-4.0.2.jar -kubernetes-client/4.9.2//kubernetes-client-4.9.2.jar -kubernetes-model-common/4.9.2//kubernetes-model-common-4.9.2.jar -kubernetes-model/4.9.2//kubernetes-model-4.9.2.jar -leveldbjni-all/1.8//leveldbjni-all-1.8.jar -libfb303/0.9.3//libfb303-0.9.3.jar -libthrift/0.12.0//libthrift-0.12.0.jar -log4j/1.2.17//log4j-1.2.17.jar -logging-interceptor/3.12.6//logging-interceptor-3.12.6.jar -lz4-java/1.7.1//lz4-java-1.7.1.jar -machinist_2.12/0.6.8//machinist_2.12-0.6.8.jar -macro-compat_2.12/1.1.1//macro-compat_2.12-1.1.1.jar -mesos/1.4.0/shaded-protobuf/mesos-1.4.0-shaded-protobuf.jar -metrics-core/4.1.1//metrics-core-4.1.1.jar -metrics-graphite/4.1.1//metrics-graphite-4.1.1.jar -metrics-jmx/4.1.1//metrics-jmx-4.1.1.jar -metrics-json/4.1.1//metrics-json-4.1.1.jar -metrics-jvm/4.1.1//metrics-jvm-4.1.1.jar -minlog/1.3.0//minlog-1.3.0.jar -netty-all/4.1.51.Final//netty-all-4.1.51.Final.jar -objenesis/2.6//objenesis-2.6.jar -okhttp/3.12.6//okhttp-3.12.6.jar -okio/1.14.0//okio-1.14.0.jar -opencsv/2.3//opencsv-2.3.jar -orc-core/1.5.10/nohive/orc-core-1.5.10-nohive.jar -orc-mapreduce/1.5.10/nohive/orc-mapreduce-1.5.10-nohive.jar -orc-shims/1.5.10//orc-shims-1.5.10.jar -oro/2.0.8//oro-2.0.8.jar -osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar -paranamer/2.8//paranamer-2.8.jar -parquet-column/1.10.1//parquet-column-1.10.1.jar -parquet-common/1.10.1//parquet-common-1.10.1.jar -parquet-encoding/1.10.1//parquet-encoding-1.10.1.jar -parquet-format/2.4.0//parquet-format-2.4.0.jar -parquet-hadoop-bundle/1.6.0//parquet-hadoop-bundle-1.6.0.jar -parquet-hadoop/1.10.1//parquet-hadoop-1.10.1.jar -parquet-jackson/1.10.1//parquet-jackson-1.10.1.jar -protobuf-java/2.5.0//protobuf-java-2.5.0.jar -py4j/0.10.9//py4j-0.10.9.jar -pyrolite/4.30//pyrolite-4.30.jar -scala-collection-compat_2.12/2.1.1//scala-collection-compat_2.12-2.1.1.jar -scala-compiler/2.12.10//scala-compiler-2.12.10.jar -scala-java8-compat_2.12/0.9.1//scala-java8-compat_2.12-0.9.1.jar -scala-library/2.12.10//scala-library-2.12.10.jar -scala-parser-combinators_2.12/1.1.2//scala-parser-combinators_2.12-1.1.2.jar -scala-reflect/2.12.10//scala-reflect-2.12.10.jar -scala-xml_2.12/1.2.0//scala-xml_2.12-1.2.0.jar -shapeless_2.12/2.3.3//shapeless_2.12-2.3.3.jar -shims/0.9.0//shims-0.9.0.jar -slf4j-api/1.7.30//slf4j-api-1.7.30.jar -slf4j-log4j12/1.7.30//slf4j-log4j12-1.7.30.jar -snakeyaml/1.24//snakeyaml-1.24.jar -snappy-java/1.1.7.5//snappy-java-1.1.7.5.jar -snappy/0.2//snappy-0.2.jar -spire-macros_2.12/0.17.0-M1//spire-macros_2.12-0.17.0-M1.jar -spire-platform_2.12/0.17.0-M1//spire-platform_2.12-0.17.0-M1.jar -spire-util_2.12/0.17.0-M1//spire-util_2.12-0.17.0-M1.jar -spire_2.12/0.17.0-M1//spire_2.12-0.17.0-M1.jar -stax-api/1.0-2//stax-api-1.0-2.jar -stax-api/1.0.1//stax-api-1.0.1.jar -stream/2.9.6//stream-2.9.6.jar -stringtemplate/3.2.1//stringtemplate-3.2.1.jar -super-csv/2.2.0//super-csv-2.2.0.jar -threeten-extra/1.5.0//threeten-extra-1.5.0.jar -univocity-parsers/2.9.0//univocity-parsers-2.9.0.jar -xbean-asm7-shaded/4.15//xbean-asm7-shaded-4.15.jar -xercesImpl/2.12.0//xercesImpl-2.12.0.jar -xml-apis/1.4.01//xml-apis-1.4.01.jar -xmlenc/0.52//xmlenc-0.52.jar -xz/1.5//xz-1.5.jar -zjsonpatch/0.3.0//zjsonpatch-0.3.0.jar -zookeeper/3.4.14//zookeeper-3.4.14.jar -zstd-jni/1.4.5-4//zstd-jni-1.4.5-4.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 00bc3c0b63551..b89e7367d351c 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -15,9 +15,10 @@ apacheds-kerberos-codec/2.0.0-M15//apacheds-kerberos-codec-2.0.0-M15.jar api-asn1-api/1.0.0-M20//api-asn1-api-1.0.0-M20.jar api-util/1.0.0-M20//api-util-1.0.0-M20.jar arpack_combined_all/0.1//arpack_combined_all-0.1.jar -arrow-format/0.15.1//arrow-format-0.15.1.jar -arrow-memory/0.15.1//arrow-memory-0.15.1.jar -arrow-vector/0.15.1//arrow-vector-0.15.1.jar +arrow-format/1.0.1//arrow-format-1.0.1.jar +arrow-memory-core/1.0.1//arrow-memory-core-1.0.1.jar +arrow-memory-netty/1.0.1//arrow-memory-netty-1.0.1.jar +arrow-vector/1.0.1//arrow-vector-1.0.1.jar audience-annotations/0.5.0//audience-annotations-0.5.0.jar automaton/1.11-8//automaton-1.11-8.jar avro-ipc/1.8.2//avro-ipc-1.8.2.jar @@ -91,7 +92,7 @@ hive-shims-0.23/2.3.7//hive-shims-0.23-2.3.7.jar hive-shims-common/2.3.7//hive-shims-common-2.3.7.jar hive-shims-scheduler/2.3.7//hive-shims-scheduler-2.3.7.jar hive-shims/2.3.7//hive-shims-2.3.7.jar -hive-storage-api/2.7.1//hive-storage-api-2.7.1.jar +hive-storage-api/2.7.2//hive-storage-api-2.7.2.jar hive-vector-code-gen/2.3.7//hive-vector-code-gen-2.3.7.jar hk2-api/2.6.1//hk2-api-2.6.1.jar hk2-locator/2.6.1//hk2-locator-2.6.1.jar @@ -153,14 +154,31 @@ jsr305/3.0.0//jsr305-3.0.0.jar jta/1.1//jta-1.1.jar jul-to-slf4j/1.7.30//jul-to-slf4j-1.7.30.jar kryo-shaded/4.0.2//kryo-shaded-4.0.2.jar -kubernetes-client/4.9.2//kubernetes-client-4.9.2.jar -kubernetes-model-common/4.9.2//kubernetes-model-common-4.9.2.jar -kubernetes-model/4.9.2//kubernetes-model-4.9.2.jar +kubernetes-client/4.10.3//kubernetes-client-4.10.3.jar +kubernetes-model-admissionregistration/4.10.3//kubernetes-model-admissionregistration-4.10.3.jar +kubernetes-model-apiextensions/4.10.3//kubernetes-model-apiextensions-4.10.3.jar +kubernetes-model-apps/4.10.3//kubernetes-model-apps-4.10.3.jar +kubernetes-model-autoscaling/4.10.3//kubernetes-model-autoscaling-4.10.3.jar +kubernetes-model-batch/4.10.3//kubernetes-model-batch-4.10.3.jar +kubernetes-model-certificates/4.10.3//kubernetes-model-certificates-4.10.3.jar +kubernetes-model-common/4.10.3//kubernetes-model-common-4.10.3.jar +kubernetes-model-coordination/4.10.3//kubernetes-model-coordination-4.10.3.jar +kubernetes-model-core/4.10.3//kubernetes-model-core-4.10.3.jar +kubernetes-model-discovery/4.10.3//kubernetes-model-discovery-4.10.3.jar +kubernetes-model-events/4.10.3//kubernetes-model-events-4.10.3.jar +kubernetes-model-extensions/4.10.3//kubernetes-model-extensions-4.10.3.jar +kubernetes-model-metrics/4.10.3//kubernetes-model-metrics-4.10.3.jar +kubernetes-model-networking/4.10.3//kubernetes-model-networking-4.10.3.jar +kubernetes-model-policy/4.10.3//kubernetes-model-policy-4.10.3.jar +kubernetes-model-rbac/4.10.3//kubernetes-model-rbac-4.10.3.jar +kubernetes-model-scheduling/4.10.3//kubernetes-model-scheduling-4.10.3.jar +kubernetes-model-settings/4.10.3//kubernetes-model-settings-4.10.3.jar +kubernetes-model-storageclass/4.10.3//kubernetes-model-storageclass-4.10.3.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar libfb303/0.9.3//libfb303-0.9.3.jar libthrift/0.12.0//libthrift-0.12.0.jar log4j/1.2.17//log4j-1.2.17.jar -logging-interceptor/3.12.6//logging-interceptor-3.12.6.jar +logging-interceptor/3.12.12//logging-interceptor-3.12.12.jar lz4-java/1.7.1//lz4-java-1.7.1.jar machinist_2.12/0.6.8//machinist_2.12-0.6.8.jar macro-compat_2.12/1.1.1//macro-compat_2.12-1.1.1.jar @@ -173,12 +191,13 @@ metrics-jvm/4.1.1//metrics-jvm-4.1.1.jar minlog/1.3.0//minlog-1.3.0.jar netty-all/4.1.51.Final//netty-all-4.1.51.Final.jar objenesis/2.6//objenesis-2.6.jar -okhttp/3.12.6//okhttp-3.12.6.jar +okhttp/3.12.12//okhttp-3.12.12.jar okio/1.14.0//okio-1.14.0.jar opencsv/2.3//opencsv-2.3.jar -orc-core/1.5.10//orc-core-1.5.10.jar -orc-mapreduce/1.5.10//orc-mapreduce-1.5.10.jar -orc-shims/1.5.10//orc-shims-1.5.10.jar +openshift-model/4.10.3//openshift-model-4.10.3.jar +orc-core/1.5.12//orc-core-1.5.12.jar +orc-mapreduce/1.5.12//orc-mapreduce-1.5.12.jar +orc-shims/1.5.12//orc-shims-1.5.12.jar oro/2.0.8//oro-2.0.8.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar @@ -223,4 +242,4 @@ xmlenc/0.52//xmlenc-0.52.jar xz/1.5//xz-1.5.jar zjsonpatch/0.3.0//zjsonpatch-0.3.0.jar zookeeper/3.4.14//zookeeper-3.4.14.jar -zstd-jni/1.4.5-4//zstd-jni-1.4.5-4.jar +zstd-jni/1.4.5-6//zstd-jni-1.4.5-6.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index c4e86003a09c9..b67e662f9a29a 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -12,9 +12,10 @@ antlr4-runtime/4.7.1//antlr4-runtime-4.7.1.jar aopalliance-repackaged/2.6.1//aopalliance-repackaged-2.6.1.jar aopalliance/1.0//aopalliance-1.0.jar arpack_combined_all/0.1//arpack_combined_all-0.1.jar -arrow-format/0.15.1//arrow-format-0.15.1.jar -arrow-memory/0.15.1//arrow-memory-0.15.1.jar -arrow-vector/0.15.1//arrow-vector-0.15.1.jar +arrow-format/1.0.1//arrow-format-1.0.1.jar +arrow-memory-core/1.0.1//arrow-memory-core-1.0.1.jar +arrow-memory-netty/1.0.1//arrow-memory-netty-1.0.1.jar +arrow-vector/1.0.1//arrow-vector-1.0.1.jar audience-annotations/0.5.0//audience-annotations-0.5.0.jar automaton/1.11-8//automaton-1.11-8.jar avro-ipc/1.8.2//avro-ipc-1.8.2.jar @@ -90,7 +91,7 @@ hive-shims-0.23/2.3.7//hive-shims-0.23-2.3.7.jar hive-shims-common/2.3.7//hive-shims-common-2.3.7.jar hive-shims-scheduler/2.3.7//hive-shims-scheduler-2.3.7.jar hive-shims/2.3.7//hive-shims-2.3.7.jar -hive-storage-api/2.7.1//hive-storage-api-2.7.1.jar +hive-storage-api/2.7.2//hive-storage-api-2.7.2.jar hive-vector-code-gen/2.3.7//hive-vector-code-gen-2.3.7.jar hk2-api/2.6.1//hk2-api-2.6.1.jar hk2-locator/2.6.1//hk2-locator-2.6.1.jar @@ -165,14 +166,31 @@ kerby-pkix/1.0.1//kerby-pkix-1.0.1.jar kerby-util/1.0.1//kerby-util-1.0.1.jar kerby-xdr/1.0.1//kerby-xdr-1.0.1.jar kryo-shaded/4.0.2//kryo-shaded-4.0.2.jar -kubernetes-client/4.9.2//kubernetes-client-4.9.2.jar -kubernetes-model-common/4.9.2//kubernetes-model-common-4.9.2.jar -kubernetes-model/4.9.2//kubernetes-model-4.9.2.jar +kubernetes-client/4.10.3//kubernetes-client-4.10.3.jar +kubernetes-model-admissionregistration/4.10.3//kubernetes-model-admissionregistration-4.10.3.jar +kubernetes-model-apiextensions/4.10.3//kubernetes-model-apiextensions-4.10.3.jar +kubernetes-model-apps/4.10.3//kubernetes-model-apps-4.10.3.jar +kubernetes-model-autoscaling/4.10.3//kubernetes-model-autoscaling-4.10.3.jar +kubernetes-model-batch/4.10.3//kubernetes-model-batch-4.10.3.jar +kubernetes-model-certificates/4.10.3//kubernetes-model-certificates-4.10.3.jar +kubernetes-model-common/4.10.3//kubernetes-model-common-4.10.3.jar +kubernetes-model-coordination/4.10.3//kubernetes-model-coordination-4.10.3.jar +kubernetes-model-core/4.10.3//kubernetes-model-core-4.10.3.jar +kubernetes-model-discovery/4.10.3//kubernetes-model-discovery-4.10.3.jar +kubernetes-model-events/4.10.3//kubernetes-model-events-4.10.3.jar +kubernetes-model-extensions/4.10.3//kubernetes-model-extensions-4.10.3.jar +kubernetes-model-metrics/4.10.3//kubernetes-model-metrics-4.10.3.jar +kubernetes-model-networking/4.10.3//kubernetes-model-networking-4.10.3.jar +kubernetes-model-policy/4.10.3//kubernetes-model-policy-4.10.3.jar +kubernetes-model-rbac/4.10.3//kubernetes-model-rbac-4.10.3.jar +kubernetes-model-scheduling/4.10.3//kubernetes-model-scheduling-4.10.3.jar +kubernetes-model-settings/4.10.3//kubernetes-model-settings-4.10.3.jar +kubernetes-model-storageclass/4.10.3//kubernetes-model-storageclass-4.10.3.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar libfb303/0.9.3//libfb303-0.9.3.jar libthrift/0.12.0//libthrift-0.12.0.jar log4j/1.2.17//log4j-1.2.17.jar -logging-interceptor/3.12.6//logging-interceptor-3.12.6.jar +logging-interceptor/3.12.12//logging-interceptor-3.12.12.jar lz4-java/1.7.1//lz4-java-1.7.1.jar machinist_2.12/0.6.8//machinist_2.12-0.6.8.jar macro-compat_2.12/1.1.1//macro-compat_2.12-1.1.1.jar @@ -187,12 +205,13 @@ netty-all/4.1.51.Final//netty-all-4.1.51.Final.jar nimbus-jose-jwt/4.41.1//nimbus-jose-jwt-4.41.1.jar objenesis/2.6//objenesis-2.6.jar okhttp/2.7.5//okhttp-2.7.5.jar -okhttp/3.12.6//okhttp-3.12.6.jar +okhttp/3.12.12//okhttp-3.12.12.jar okio/1.14.0//okio-1.14.0.jar opencsv/2.3//opencsv-2.3.jar -orc-core/1.5.10//orc-core-1.5.10.jar -orc-mapreduce/1.5.10//orc-mapreduce-1.5.10.jar -orc-shims/1.5.10//orc-shims-1.5.10.jar +openshift-model/4.10.3//openshift-model-4.10.3.jar +orc-core/1.5.12//orc-core-1.5.12.jar +orc-mapreduce/1.5.12//orc-mapreduce-1.5.12.jar +orc-shims/1.5.12//orc-shims-1.5.12.jar oro/2.0.8//oro-2.0.8.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar @@ -237,4 +256,4 @@ xbean-asm7-shaded/4.15//xbean-asm7-shaded-4.15.jar xz/1.5//xz-1.5.jar zjsonpatch/0.3.0//zjsonpatch-0.3.0.jar zookeeper/3.4.14//zookeeper-3.4.14.jar -zstd-jni/1.4.5-4//zstd-jni-1.4.5-4.jar +zstd-jni/1.4.5-6//zstd-jni-1.4.5-6.jar diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 4ff5b327e3325..610fb1fd27027 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -175,8 +175,6 @@ def main(): if "test-hadoop3.2" in ghprb_pull_title: os.environ["AMPLAB_JENKINS_BUILD_PROFILE"] = "hadoop3.2" # Switch the Hive profile based on the PR title: - if "test-hive1.2" in ghprb_pull_title: - os.environ["AMPLAB_JENKINS_BUILD_HIVE_PROFILE"] = "hive1.2" if "test-hive2.3" in ghprb_pull_title: os.environ["AMPLAB_JENKINS_BUILD_HIVE_PROFILE"] = "hive2.3" diff --git a/dev/run-tests.py b/dev/run-tests.py index 3e118dcbc160d..48191e9bb024d 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -325,7 +325,6 @@ def get_hive_profiles(hive_version): """ sbt_maven_hive_profiles = { - "hive1.2": ["-Phive-1.2"], "hive2.3": ["-Phive-2.3"], } diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 3c438e309c22d..868e4a5d23ed7 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -386,6 +386,7 @@ def __hash__(self): "pyspark.tests.test_conf", "pyspark.tests.test_context", "pyspark.tests.test_daemon", + "pyspark.tests.test_install_spark", "pyspark.tests.test_join", "pyspark.tests.test_profiler", "pyspark.tests.test_rdd", diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index 129b073d75254..e9e9227d239e1 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -32,7 +32,6 @@ export LC_ALL=C HADOOP_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkubernetes -Pyarn -Phive" MVN="build/mvn" HADOOP_HIVE_PROFILES=( - hadoop-2.7-hive-1.2 hadoop-2.7-hive-2.3 hadoop-3.2-hive-2.3 ) @@ -71,12 +70,9 @@ for HADOOP_HIVE_PROFILE in "${HADOOP_HIVE_PROFILES[@]}"; do if [[ $HADOOP_HIVE_PROFILE == **hadoop-3.2-hive-2.3** ]]; then HADOOP_PROFILE=hadoop-3.2 HIVE_PROFILE=hive-2.3 - elif [[ $HADOOP_HIVE_PROFILE == **hadoop-2.7-hive-2.3** ]]; then - HADOOP_PROFILE=hadoop-2.7 - HIVE_PROFILE=hive-2.3 else HADOOP_PROFILE=hadoop-2.7 - HIVE_PROFILE=hive-1.2 + HIVE_PROFILE=hive-2.3 fi echo "Performing Maven install for $HADOOP_HIVE_PROFILE" $MVN $HADOOP_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE jar:jar jar:test-jar install:install clean -q diff --git a/dev/tox.ini b/dev/tox.ini index c14e6b9446cca..7edf7d597fb58 100644 --- a/dev/tox.ini +++ b/dev/tox.ini @@ -20,5 +20,5 @@ exclude=python/pyspark/cloudpickle/*.py,shared.py,python/docs/source/conf.py,wor [flake8] select = E901,E999,F821,F822,F823,F401,F405 -exclude = python/pyspark/cloudpickle/*.py,shared.py,python/docs/source/conf.py,work/*/*.py,python/.eggs/*,dist/*,.git/* +exclude = python/pyspark/cloudpickle/*.py,shared.py*,python/docs/source/conf.py,work/*/*.py,python/.eggs/*,dist/*,.git/*,python/out,python/pyspark/sql/pandas/functions.pyi,python/pyspark/sql/column.pyi,python/pyspark/worker.pyi,python/pyspark/java_gateway.pyi max-line-length = 100 diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 09f7018262a0b..5f6cd7c6b7f20 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -110,7 +110,7 @@
  • `SaveMode.ErrorIfExists`: throw an exception at runtime.
  • * *

    - * When writing to data source v1, the default option is `ErrorIfExists`. When writing to data - * source v2, the default option is `Append`. + * The default option is `ErrorIfExists`. * * @since 1.4.0 */ @@ -286,13 +285,12 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ def save(path: String): Unit = { if (!df.sparkSession.sessionState.conf.legacyPathOptionBehavior && - extraOptions.contains("path") && path.nonEmpty) { + extraOptions.contains("path")) { throw new AnalysisException("There is a 'path' option set and save() is called with a path " + "parameter. Either remove the path option, or call save() without the parameter. " + s"To ignore this check, set '${SQLConf.LEGACY_PATH_OPTION_BEHAVIOR.key}' to 'true'.") } - this.extraOptions = this.extraOptions + ("path" -> path) - save() + saveInternal(Some(path)) } /** @@ -300,7 +298,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * * @since 1.4.0 */ - def save(): Unit = { + def save(): Unit = saveInternal(None) + + private def saveInternal(path: Option[String]): Unit = { if (source.toLowerCase(Locale.ROOT) == DDLUtils.HIVE_PROVIDER) { throw new AnalysisException("Hive data source can only be used with tables, you can not " + "write files of Hive data source directly.") @@ -313,8 +313,16 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val provider = maybeV2Provider.get val sessionOptions = DataSourceV2Utils.extractSessionConfigs( provider, df.sparkSession.sessionState.conf) - val options = sessionOptions.filterKeys(!extraOptions.contains(_)) ++ extraOptions.toMap - val dsOptions = new CaseInsensitiveStringMap(options.toMap.asJava) + + val optionsWithPath = if (path.isEmpty) { + extraOptions + } else { + extraOptions + ("path" -> path.get) + } + + val finalOptions = sessionOptions.filterKeys(!optionsWithPath.contains(_)).toMap ++ + optionsWithPath.originalMap + val dsOptions = new CaseInsensitiveStringMap(finalOptions.asJava) def getTable: Table = { // For file source, it's expensive to infer schema/partition at each write. Here we pass @@ -350,7 +358,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { // Streaming also uses the data source V2 API. So it may be that the data source // implements v2, but has no v2 implementation for batch writes. In that case, we // fall back to saving as though it's a V1 source. - return saveToV1Source() + return saveToV1Source(path) } } @@ -358,14 +366,14 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { checkPartitioningMatchesV2Table(table) if (mode == SaveMode.Append) { runCommand(df.sparkSession, "save") { - AppendData.byName(relation, df.logicalPlan, extraOptions.toMap) + AppendData.byName(relation, df.logicalPlan, finalOptions) } } else { // Truncate the table. TableCapabilityCheck will throw a nice exception if this // isn't supported runCommand(df.sparkSession, "save") { OverwriteByExpression.byName( - relation, df.logicalPlan, Literal(true), extraOptions.toMap) + relation, df.logicalPlan, Literal(true), finalOptions) } } @@ -385,7 +393,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { partitioningAsV2, df.queryExecution.analyzed, Map(TableCatalog.PROP_PROVIDER -> source) ++ location, - extraOptions.toMap, + finalOptions, ignoreIfExists = createMode == SaveMode.Ignore) } case _: TableProvider => @@ -397,30 +405,36 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { // Streaming also uses the data source V2 API. So it may be that the data source // implements v2, but has no v2 implementation for batch writes. In that case, we // fallback to saving as though it's a V1 source. - saveToV1Source() + saveToV1Source(path) } } } } else { - saveToV1Source() + saveToV1Source(path) } } - private def saveToV1Source(): Unit = { + private def saveToV1Source(path: Option[String]): Unit = { partitioningColumns.foreach { columns => extraOptions = extraOptions + ( DataSourceUtils.PARTITIONING_COLUMNS_KEY -> DataSourceUtils.encodePartitioningColumns(columns)) } + val optionsWithPath = if (path.isEmpty) { + extraOptions + } else { + extraOptions + ("path" -> path.get) + } + // Code path for data source v1. runCommand(df.sparkSession, "save") { DataSource( sparkSession = df.sparkSession, className = source, partitionColumns = partitioningColumns.getOrElse(Nil), - options = extraOptions.toMap).planForWriting(mode, df.logicalPlan) + options = optionsWithPath.originalMap).planForWriting(mode, df.logicalPlan) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 7b0bae6a82053..87b9aea80c823 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2038,10 +2038,10 @@ class Dataset[T] private[sql]( * The difference between this function and [[union]] is that this function * resolves columns by name (not by position). * - * When the parameter `allowMissingColumns` is true, this function allows different set - * of column names between two Datasets. Missing columns at each side, will be filled with - * null values. The missing columns at left Dataset will be added at the end in the schema - * of the union result: + * When the parameter `allowMissingColumns` is `true`, the set of column names + * in this and other `Dataset` can differ; missing columns will be filled with null. + * Further, the missing columns of this `Dataset` will be added at the end + * in the schema of the union result: * * {{{ * val df1 = Seq((1, 2, 3)).toDF("col0", "col1", "col2") @@ -3131,6 +3131,10 @@ class Dataset[T] private[sql]( * Returns a new Dataset that contains only the unique rows from this Dataset. * This is an alias for `dropDuplicates`. * + * Note that for a streaming [[Dataset]], this method returns distinct rows only once + * regardless of the output mode, which the behavior may not be same with `DISTINCT` in SQL + * against streaming [[Dataset]]. + * * @note Equality checking is performed directly on the encoded representation of the data * and thus is not affected by a custom `equals` function defined on `T`. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index e5d53f5fd4c65..5704414df2d0d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -81,7 +81,8 @@ class SparkSession private( @transient val sparkContext: SparkContext, @transient private val existingSharedState: Option[SharedState], @transient private val parentSessionState: Option[SessionState], - @transient private[sql] val extensions: SparkSessionExtensions) + @transient private[sql] val extensions: SparkSessionExtensions, + @transient private val initialSessionOptions: Map[String, String]) extends Serializable with Closeable with Logging { self => // The call site where this SparkSession was constructed. @@ -97,7 +98,7 @@ class SparkSession private( this(sc, None, None, SparkSession.applyExtensions( sc.getConf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS).getOrElse(Seq.empty), - new SparkSessionExtensions)) + new SparkSessionExtensions), Map.empty) } sparkContext.assertNotStopped() @@ -133,12 +134,6 @@ class SparkSession private( existingSharedState.getOrElse(new SharedState(sparkContext, initialSessionOptions)) } - /** - * Initial options for session. This options are applied once when sessionState is created. - */ - @transient - private[sql] val initialSessionOptions = new scala.collection.mutable.HashMap[String, String] - /** * State isolated across sessions, including SQL configurations, temporary tables, registered * functions, and everything else that accepts a [[org.apache.spark.sql.internal.SQLConf]]. @@ -156,8 +151,8 @@ class SparkSession private( .getOrElse { val state = SparkSession.instantiateSessionState( SparkSession.sessionStateClassName(sparkContext.conf), - self) - initialSessionOptions.foreach { case (k, v) => state.conf.setConfString(k, v) } + self, + initialSessionOptions) state } } @@ -244,7 +239,12 @@ class SparkSession private( * @since 2.0.0 */ def newSession(): SparkSession = { - new SparkSession(sparkContext, Some(sharedState), parentSessionState = None, extensions) + new SparkSession( + sparkContext, + Some(sharedState), + parentSessionState = None, + extensions, + initialSessionOptions) } /** @@ -260,7 +260,12 @@ class SparkSession private( * implementation is Hive, this will initialize the metastore, which may take some time. */ private[sql] def cloneSession(): SparkSession = { - val result = new SparkSession(sparkContext, Some(sharedState), Some(sessionState), extensions) + val result = new SparkSession( + sparkContext, + Some(sharedState), + Some(sessionState), + extensions, + Map.empty) result.sessionState // force copy of SessionState result } @@ -939,8 +944,7 @@ object SparkSession extends Logging { sparkContext.getConf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS).getOrElse(Seq.empty), extensions) - session = new SparkSession(sparkContext, None, None, extensions) - options.foreach { case (k, v) => session.initialSessionOptions.put(k, v) } + session = new SparkSession(sparkContext, None, None, extensions, options.toMap) setDefaultSession(session) setActiveSession(session) registerContextListener(sparkContext) @@ -1104,12 +1108,16 @@ object SparkSession extends Logging { */ private def instantiateSessionState( className: String, - sparkSession: SparkSession): SessionState = { + sparkSession: SparkSession, + options: Map[String, String]): SessionState = { try { - // invoke `new [Hive]SessionStateBuilder(SparkSession, Option[SessionState])` + // invoke new [Hive]SessionStateBuilder( + // SparkSession, + // Option[SessionState], + // Map[String, String]) val clazz = Utils.classForName(className) val ctor = clazz.getConstructors.head - ctor.newInstance(sparkSession, None).asInstanceOf[BaseSessionStateBuilder].build() + ctor.newInstance(sparkSession, None, options).asInstanceOf[BaseSessionStateBuilder].build() } catch { case NonFatal(e) => throw new IllegalArgumentException(s"Error while instantiating '$className':", e) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 8923d5c86e19a..24382e07a2966 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, CatalogV2Util, LookupCatalog, SupportsNamespaces, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} +import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBuilder, StructField, StructType} @@ -256,16 +256,20 @@ class ResolveSessionCatalog( case RenameTableStatement(TempViewOrV1Table(oldName), newName, isView) => AlterTableRenameCommand(oldName.asTableIdentifier, newName.asTableIdentifier, isView) - case DescribeRelation(ResolvedTable(_, ident, _: V1Table), partitionSpec, isExtended) => + case DescribeRelation(r @ ResolvedTable(_, ident, _: V1Table), partitionSpec, isExtended) + if isSessionCatalog(r.catalog) => DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended) // Use v1 command to describe (temp) view, as v2 catalog doesn't support view yet. case DescribeRelation(ResolvedView(ident), partitionSpec, isExtended) => DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended) - case DescribeColumnStatement(tbl, colNameParts, isExtended) => - val name = parseTempViewOrV1Table(tbl, "Describing columns") - DescribeColumnCommand(name.asTableIdentifier, colNameParts, isExtended) + case DescribeColumn(r @ ResolvedTable(_, _, _: V1Table), colNameParts, isExtended) + if isSessionCatalog(r.catalog) => + DescribeColumnCommand(r.identifier.asTableIdentifier, colNameParts, isExtended) + + case DescribeColumn(ResolvedView(ident), colNameParts, isExtended) => + DescribeColumnCommand(ident.asTableIdentifier, colNameParts, isExtended) // For CREATE TABLE [AS SELECT], we should use the v1 command if the catalog is resolved to the // session catalog and the table provider is not v2. @@ -318,9 +322,11 @@ class ResolveSessionCatalog( ignoreIfExists = c.ifNotExists) } - // v1 REFRESH TABLE supports temp view. - case RefreshTableStatement(TempViewOrV1Table(name)) => - RefreshTable(name.asTableIdentifier) + case RefreshTable(r @ ResolvedTable(_, _, _: V1Table)) if isSessionCatalog(r.catalog) => + RefreshTableCommand(r.identifier.asTableIdentifier) + + case RefreshTable(r: ResolvedView) => + RefreshTableCommand(r.identifier.asTableIdentifier) // For REPLACE TABLE [AS SELECT], we should fail if the catalog is resolved to the // session catalog and the table provider is not v2. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index ce5909a09442c..45d28ddb42fc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -55,7 +55,7 @@ trait DataSourceScanExec extends LeafExecNode { // Metadata that describes more details of this scan. protected def metadata: Map[String, String] - protected val maxMetadataValueLength = 100 + protected val maxMetadataValueLength = sqlContext.sessionState.conf.maxMetadataStringLength override def simpleString(maxFields: Int): String = { val metadataEntries = metadata.toSeq.sorted.map { @@ -156,7 +156,9 @@ case class RowDataSourceScanExec( * @param optionalBucketSet Bucket ids for bucket pruning. * @param optionalNumCoalescedBuckets Number of coalesced buckets. * @param dataFilters Filters on non-partition columns. - * @param tableIdentifier identifier for the table in the metastore. + * @param tableIdentifier Identifier for the table in the metastore. + * @param disableBucketedScan Disable bucketed scan based on physical query plan, see rule + * [[DisableUnnecessaryBucketedScan]] for details. */ case class FileSourceScanExec( @transient relation: HadoopFsRelation, @@ -166,7 +168,8 @@ case class FileSourceScanExec( optionalBucketSet: Option[BitSet], optionalNumCoalescedBuckets: Option[Int], dataFilters: Seq[Expression], - tableIdentifier: Option[TableIdentifier]) + tableIdentifier: Option[TableIdentifier], + disableBucketedScan: Boolean = false) extends DataSourceScanExec { // Note that some vals referring the file-based relation are lazy intentionally @@ -257,7 +260,8 @@ case class FileSourceScanExec( // exposed for testing lazy val bucketedScan: Boolean = { - if (relation.sparkSession.sessionState.conf.bucketingEnabled && relation.bucketSpec.isDefined) { + if (relation.sparkSession.sessionState.conf.bucketingEnabled && relation.bucketSpec.isDefined + && !disableBucketedScan) { val spec = relation.bucketSpec.get val bucketColumns = spec.bucketColumnNames.flatMap(n => toAttribute(n)) bucketColumns.size == spec.bucketColumnNames.size @@ -348,20 +352,23 @@ case class FileSourceScanExec( "DataFilters" -> seqToString(dataFilters), "Location" -> locationDesc) - val withSelectedBucketsCount = relation.bucketSpec.map { spec => - val numSelectedBuckets = optionalBucketSet.map { b => - b.cardinality() + // TODO(SPARK-32986): Add bucketed scan info in explain output of FileSourceScanExec + if (bucketedScan) { + relation.bucketSpec.map { spec => + val numSelectedBuckets = optionalBucketSet.map { b => + b.cardinality() + } getOrElse { + spec.numBuckets + } + metadata + ("SelectedBucketsCount" -> + (s"$numSelectedBuckets out of ${spec.numBuckets}" + + optionalNumCoalescedBuckets.map { b => s" (Coalesced to $b)"}.getOrElse(""))) } getOrElse { - spec.numBuckets + metadata } - metadata + ("SelectedBucketsCount" -> - (s"$numSelectedBuckets out of ${spec.numBuckets}" + - optionalNumCoalescedBuckets.map { b => s" (Coalesced to $b)"}.getOrElse(""))) - } getOrElse { + } else { metadata } - - withSelectedBucketsCount } override def verboseStringWithOperatorId(): String = { @@ -539,6 +546,7 @@ case class FileSourceScanExec( .getOrElse(sys.error(s"Invalid bucket file ${f.filePath}")) } + // TODO(SPARK-32985): Decouple bucket filter pruning and bucketed table scan val prunedFilesGroupedToBuckets = if (optionalBucketSet.isDefined) { val bucketSet = optionalBucketSet.get filesGroupedToBuckets.filter { @@ -624,6 +632,7 @@ case class FileSourceScanExec( optionalBucketSet, optionalNumCoalescedBuckets, QueryPlan.normalizePredicates(dataFilters, output), - None) + None, + disableBucketedScan) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index 4d388e40fb8bd..dcec0b019da28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -106,7 +106,7 @@ object HiveResult { case (n, _: NumericType) => n.toString case (s: String, StringType) => if (nested) "\"" + s + "\"" else s case (interval: CalendarInterval, CalendarIntervalType) => interval.toString - case (seq: Seq[_], ArrayType(typ, _)) => + case (seq: scala.collection.Seq[_], ArrayType(typ, _)) => seq.map(v => (v, typ)).map(e => toHiveString(e, true, formatters)).mkString("[", ",", "]") case (m: Map[_, _], MapType(kType, vType, _)) => m.map { case (key, value) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index dca2c5b16e8d5..a056500fa361a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.rules.{PlanChangeLogger, Rule} import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.adaptive.{AdaptiveExecutionContext, InsertAdaptiveSparkPlan} -import org.apache.spark.sql.execution.bucketing.CoalesceBucketsInJoin +import org.apache.spark.sql.execution.bucketing.{CoalesceBucketsInJoin, DisableUnnecessaryBucketedScan} import org.apache.spark.sql.execution.dynamicpruning.PlanDynamicPruningFilters import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} @@ -344,6 +344,7 @@ object QueryExecution { PlanSubqueries(sparkSession), RemoveRedundantProjects(sparkSession.sessionState.conf), EnsureRequirements(sparkSession.sessionState.conf), + DisableUnnecessaryBucketedScan(sparkSession.sessionState.conf), ApplyColumnarRulesAndInsertTransitions(sparkSession.sessionState.conf, sparkSession.sessionState.columnarRules), CollapseCodegenStages(sparkSession.sessionState.conf), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/RemoveRedundantProjects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/RemoveRedundantProjects.scala index ecb4ad0f6e8dd..2bcf86edbea37 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/RemoveRedundantProjects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/RemoveRedundantProjects.scala @@ -62,7 +62,9 @@ case class RemoveRedundantProjects(conf: SQLConf) extends Rule[SparkPlan] { val keepOrdering = a.aggregateExpressions .exists(ae => ae.mode.equals(Final) || ae.mode.equals(PartialMerge)) a.mapChildren(removeProject(_, keepOrdering)) - case g: GenerateExec => g.mapChildren(removeProject(_, false)) + // GenerateExec requires column ordering since it binds input rows directly with its + // requiredChildOutput without using child's output schema. + case g: GenerateExec => g.mapChildren(removeProject(_, true)) // JoinExec ordering requirement will inherit from its parent. If there is no ProjectExec in // its ancestors, JoinExec should require output columns to be ordered. case o => o.mapChildren(removeProject(_, requireOrdering)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 11934c934e316..0a5f4c3ed4bcb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -262,7 +262,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { operationNotAllowed("CREATE TEMPORARY TABLE IF NOT EXISTS", ctx) } - val (_, _, _, options, _, _) = visitCreateTableClauses(ctx.createTableClauses()) + val (_, _, _, options, location, _) = visitCreateTableClauses(ctx.createTableClauses()) val provider = Option(ctx.tableProvider).map(_.multipartIdentifier.getText).getOrElse( throw new ParseException("CREATE TEMPORARY TABLE without a provider is not allowed.", ctx)) val schema = Option(ctx.colTypeList()).map(createSchema) @@ -271,7 +271,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { "CREATE TEMPORARY VIEW ... USING ... instead") val table = tableIdentifier(ident, "CREATE TEMPORARY VIEW", ctx) - CreateTempViewUsing(table, schema, replace = false, global = false, provider, options) + val optionsWithLocation = location.map(l => options + ("path" -> l)).getOrElse(options) + CreateTempViewUsing(table, schema, replace = false, global = false, provider, + optionsWithLocation) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SubqueryBroadcastExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SubqueryBroadcastExec.scala index ddf0b72dd7a96..70ba13550afbf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SubqueryBroadcastExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SubqueryBroadcastExec.scala @@ -47,8 +47,15 @@ case class SubqueryBroadcastExec( // `SubqueryBroadcastExec` is only used with `InSubqueryExec`. No one would reference this output, // so the exprId doesn't matter here. But it's important to correctly report the output length, so // that `InSubqueryExec` can know it's the single-column execution mode, not multi-column. - override def output: Seq[Attribute] = Seq( - AttributeReference("key", buildKeys(index).dataType, buildKeys(index).nullable)()) + override def output: Seq[Attribute] = { + val key = buildKeys(index) + val name = key match { + case n: NamedExpression => n.name + case Cast(n: NamedExpression, _, _) => n.name + case _ => "key" + } + Seq(AttributeReference(name, key.dataType, key.nullable)()) + } override lazy val metrics = Map( "dataSize" -> SQLMetrics.createMetric(sparkContext, "data size (bytes)"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala index c82b264a600ef..0170f8b2f71c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.adaptive -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LogicalPlanIntegrity, PlanHelper} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils @@ -54,4 +54,10 @@ class AQEOptimizer(conf: SQLConf) extends RuleExecutor[LogicalPlan] { } } } + + override protected def isPlanIntegral(plan: LogicalPlan): Boolean = { + !Utils.isTesting || (plan.resolved && + plan.find(PlanHelper.specialExpressionsInUnsupportedOperator(_).nonEmpty).isEmpty && + LogicalPlanIntegrity.checkIfExprIdsAreGloballyUnique(plan)) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 014358b663bbb..5e75e26e6d074 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -32,10 +32,12 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} -import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor} +import org.apache.spark.sql.catalyst.rules.{PlanChangeLogger, Rule} import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec._ +import org.apache.spark.sql.execution.command.DataWritingCommandExec +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.ui.{SparkListenerSQLAdaptiveExecutionUpdate, SparkListenerSQLAdaptiveSQLMetricUpdates, SQLPlanMetric} import org.apache.spark.sql.internal.SQLConf @@ -75,6 +77,8 @@ case class AdaptiveSparkPlanExec( case _ => logDebug(_) } + @transient private val planChangeLogger = new PlanChangeLogger[SparkPlan]() + // The logical plan optimizer for re-optimizing the current logical plan. @transient private val optimizer = new AQEOptimizer(conf) @@ -100,6 +104,16 @@ case class AdaptiveSparkPlanExec( OptimizeLocalShuffleReader(conf) ) + private def finalStageOptimizerRules: Seq[Rule[SparkPlan]] = + context.qe.sparkPlan match { + case _: DataWritingCommandExec | _: V2TableWriteExec => + // SPARK-32932: Local shuffle reader could break partitioning that works best + // for the following writing command + queryStageOptimizerRules.filterNot(_.isInstanceOf[OptimizeLocalShuffleReader]) + case _ => + queryStageOptimizerRules + } + // A list of physical optimizer rules to be applied right after a new stage is created. The input // plan to these rules has exchange as its root node. @transient private val postStageCreationRules = Seq( @@ -109,7 +123,8 @@ case class AdaptiveSparkPlanExec( @transient private val costEvaluator = SimpleCostEvaluator - @transient private val initialPlan = applyPhysicalRules(inputPlan, queryStagePreparationRules) + @transient private val initialPlan = applyPhysicalRules( + inputPlan, queryStagePreparationRules, Some((planChangeLogger, "AQE Preparations"))) @volatile private var currentPhysicalPlan = initialPlan @@ -231,7 +246,9 @@ case class AdaptiveSparkPlanExec( // Run the final plan when there's no more unfinished stages. currentPhysicalPlan = applyPhysicalRules( - result.newPlan, queryStageOptimizerRules ++ postStageCreationRules) + result.newPlan, + finalStageOptimizerRules ++ postStageCreationRules, + Some((planChangeLogger, "AQE Final Query Stage Optimization"))) isFinalPlan = true executionId.foreach(onUpdatePlan(_, Seq(currentPhysicalPlan))) currentPhysicalPlan @@ -295,26 +312,40 @@ case class AdaptiveSparkPlanExec( maxFields, printNodeId, indent) - generateTreeStringWithHeader( - if (isFinalPlan) "Final Plan" else "Current Plan", - currentPhysicalPlan, - depth, - lastChildren, - append, - verbose, - maxFields, - printNodeId) - generateTreeStringWithHeader( - "Initial Plan", - initialPlan, - depth, - lastChildren, - append, - verbose, - maxFields, - printNodeId) + if (currentPhysicalPlan.fastEquals(initialPlan)) { + currentPhysicalPlan.generateTreeString( + depth + 1, + lastChildren :+ true, + append, + verbose, + prefix = "", + addSuffix = false, + maxFields, + printNodeId, + indent) + } else { + generateTreeStringWithHeader( + if (isFinalPlan) "Final Plan" else "Current Plan", + currentPhysicalPlan, + depth, + lastChildren, + append, + verbose, + maxFields, + printNodeId) + generateTreeStringWithHeader( + "Initial Plan", + initialPlan, + depth, + lastChildren, + append, + verbose, + maxFields, + printNodeId) + } } + private def generateTreeStringWithHeader( header: String, plan: SparkPlan, @@ -413,11 +444,14 @@ case class AdaptiveSparkPlanExec( } private def newQueryStage(e: Exchange): QueryStageExec = { - val optimizedPlan = applyPhysicalRules(e.child, queryStageOptimizerRules) + val optimizedPlan = applyPhysicalRules( + e.child, queryStageOptimizerRules, Some((planChangeLogger, "AQE Query Stage Optimization"))) val queryStage = e match { case s: ShuffleExchangeLike => val newShuffle = applyPhysicalRules( - s.withNewChildren(Seq(optimizedPlan)), postStageCreationRules) + s.withNewChildren(Seq(optimizedPlan)), + postStageCreationRules, + Some((planChangeLogger, "AQE Post Stage Creation"))) if (!newShuffle.isInstanceOf[ShuffleExchangeLike]) { throw new IllegalStateException( "Custom columnar rules cannot transform shuffle node to something else.") @@ -425,7 +459,9 @@ case class AdaptiveSparkPlanExec( ShuffleQueryStageExec(currentStageId, newShuffle) case b: BroadcastExchangeLike => val newBroadcast = applyPhysicalRules( - b.withNewChildren(Seq(optimizedPlan)), postStageCreationRules) + b.withNewChildren(Seq(optimizedPlan)), + postStageCreationRules, + Some((planChangeLogger, "AQE Post Stage Creation"))) if (!newBroadcast.isInstanceOf[BroadcastExchangeLike]) { throw new IllegalStateException( "Custom columnar rules cannot transform broadcast node to something else.") @@ -534,7 +570,10 @@ case class AdaptiveSparkPlanExec( logicalPlan.invalidateStatsCache() val optimized = optimizer.execute(logicalPlan) val sparkPlan = context.session.sessionState.planner.plan(ReturnAnswer(optimized)).next() - val newPlan = applyPhysicalRules(sparkPlan, preprocessingRules ++ queryStagePreparationRules) + val newPlan = applyPhysicalRules( + sparkPlan, + preprocessingRules ++ queryStagePreparationRules, + Some((planChangeLogger, "AQE Replanning"))) (newPlan, optimized) } @@ -630,8 +669,22 @@ object AdaptiveSparkPlanExec { /** * Apply a list of physical operator rules on a [[SparkPlan]]. */ - def applyPhysicalRules(plan: SparkPlan, rules: Seq[Rule[SparkPlan]]): SparkPlan = { - rules.foldLeft(plan) { case (sp, rule) => rule.apply(sp) } + def applyPhysicalRules( + plan: SparkPlan, + rules: Seq[Rule[SparkPlan]], + loggerAndBatchName: Option[(PlanChangeLogger[SparkPlan], String)] = None): SparkPlan = { + if (loggerAndBatchName.isEmpty) { + rules.foldLeft(plan) { case (sp, rule) => rule.apply(sp) } + } else { + val (logger, batchName) = loggerAndBatchName.get + val newPlan = rules.foldLeft(plan) { case (sp, rule) => + val result = rule.apply(sp) + logger.logRule(rule.ruleName, sp, result) + result + } + logger.logBatch(batchName, plan, newPlan) + newPlan + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index dcb465707a0ed..52d0450afb181 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -263,7 +263,7 @@ case class HashAggregateExec( } else { val inputVars = aggBufferUpdatingExprs.map { aggExprsForOneFunc => val inputVarsForOneFunc = aggExprsForOneFunc.map( - CodeGenerator.getLocalInputVariableValues(ctx, _, subExprs)).reduce(_ ++ _).toSeq + CodeGenerator.getLocalInputVariableValues(ctx, _, subExprs)._1).reduce(_ ++ _).toSeq val paramLength = CodeGenerator.calculateParamLengthFromExprValues(inputVarsForOneFunc) // Checks if a parameter length for the `aggExprsForOneFunc` does not go over the JVM limit diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index c240a182d32bb..7334ea1e27284 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -66,10 +66,23 @@ case class ProjectExec(projectList: Seq[NamedExpression], child: SparkPlan) override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { val exprs = bindReferences[Expression](projectList, child.output) - val resultVars = exprs.map(_.genCode(ctx)) + val (subExprsCode, resultVars, localValInputs) = if (conf.subexpressionEliminationEnabled) { + // subexpression elimination + val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(exprs) + val genVars = ctx.withSubExprEliminationExprs(subExprs.states) { + exprs.map(_.genCode(ctx)) + } + (subExprs.codes.mkString("\n"), genVars, subExprs.exprCodesNeedEvaluate) + } else { + ("", exprs.map(_.genCode(ctx)), Seq.empty) + } + // Evaluation of non-deterministic expressions can't be deferred. val nonDeterministicAttrs = projectList.filterNot(_.deterministic).map(_.toAttribute) s""" + |// common sub-expressions + |${evaluateVariables(localValInputs)} + |$subExprsCode |${evaluateRequiredVariables(output, resultVars, AttributeSet(nonDeterministicAttrs))} |${consume(ctx, resultVars)} """.stripMargin @@ -371,6 +384,7 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) val step: Long = range.step val numSlices: Int = range.numSlices.getOrElse(sparkContext.defaultParallelism) val numElements: BigInt = range.numElements + val isEmptyRange: Boolean = start == end || (start < end ^ 0 < step) override val output: Seq[Attribute] = range.output @@ -396,7 +410,7 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) } override def inputRDDs(): Seq[RDD[InternalRow]] = { - val rdd = if (start == end || (start < end ^ 0 < step)) { + val rdd = if (isEmptyRange) { new EmptyRDD[InternalRow](sqlContext.sparkContext) } else { sqlContext.sparkContext.parallelize(0 until numSlices, numSlices).map(i => InternalRow(i)) @@ -562,58 +576,64 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") - sqlContext - .sparkContext - .parallelize(0 until numSlices, numSlices) - .mapPartitionsWithIndex { (i, _) => - val partitionStart = (i * numElements) / numSlices * step + start - val partitionEnd = (((i + 1) * numElements) / numSlices) * step + start - def getSafeMargin(bi: BigInt): Long = - if (bi.isValidLong) { - bi.toLong - } else if (bi > 0) { - Long.MaxValue - } else { - Long.MinValue - } - val safePartitionStart = getSafeMargin(partitionStart) - val safePartitionEnd = getSafeMargin(partitionEnd) - val rowSize = UnsafeRow.calculateBitSetWidthInBytes(1) + LongType.defaultSize - val unsafeRow = UnsafeRow.createFromByteArray(rowSize, 1) - val taskContext = TaskContext.get() - - val iter = new Iterator[InternalRow] { - private[this] var number: Long = safePartitionStart - private[this] var overflow: Boolean = false - private[this] val inputMetrics = taskContext.taskMetrics().inputMetrics - - override def hasNext = - if (!overflow) { - if (step > 0) { - number < safePartitionEnd - } else { - number > safePartitionEnd - } - } else false - - override def next() = { - val ret = number - number += step - if (number < ret ^ step < 0) { - // we have Long.MaxValue + Long.MaxValue < Long.MaxValue - // and Long.MinValue + Long.MinValue > Long.MinValue, so iff the step causes a step - // back, we are pretty sure that we have an overflow. - overflow = true + if (isEmptyRange) { + new EmptyRDD[InternalRow](sqlContext.sparkContext) + } else { + sqlContext + .sparkContext + .parallelize(0 until numSlices, numSlices) + .mapPartitionsWithIndex { (i, _) => + val partitionStart = (i * numElements) / numSlices * step + start + val partitionEnd = (((i + 1) * numElements) / numSlices) * step + start + + def getSafeMargin(bi: BigInt): Long = + if (bi.isValidLong) { + bi.toLong + } else if (bi > 0) { + Long.MaxValue + } else { + Long.MinValue } - numOutputRows += 1 - inputMetrics.incRecordsRead(1) - unsafeRow.setLong(0, ret) - unsafeRow + val safePartitionStart = getSafeMargin(partitionStart) + val safePartitionEnd = getSafeMargin(partitionEnd) + val rowSize = UnsafeRow.calculateBitSetWidthInBytes(1) + LongType.defaultSize + val unsafeRow = UnsafeRow.createFromByteArray(rowSize, 1) + val taskContext = TaskContext.get() + + val iter = new Iterator[InternalRow] { + private[this] var number: Long = safePartitionStart + private[this] var overflow: Boolean = false + private[this] val inputMetrics = taskContext.taskMetrics().inputMetrics + + override def hasNext = + if (!overflow) { + if (step > 0) { + number < safePartitionEnd + } else { + number > safePartitionEnd + } + } else false + + override def next() = { + val ret = number + number += step + if (number < ret ^ step < 0) { + // we have Long.MaxValue + Long.MaxValue < Long.MaxValue + // and Long.MinValue + Long.MinValue > Long.MinValue, so iff the step causes a step + // back, we are pretty sure that we have an overflow. + overflow = true + } + + numOutputRows += 1 + inputMetrics.incRecordsRead(1) + unsafeRow.setLong(0, ret) + unsafeRow + } } + new InterruptibleIterator(taskContext, iter) } - new InterruptibleIterator(taskContext, iter) - } + } } override def simpleString(maxFields: Int): String = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/DisableUnnecessaryBucketedScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/DisableUnnecessaryBucketedScan.scala new file mode 100644 index 0000000000000..9b4f898df00b6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/DisableUnnecessaryBucketedScan.scala @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.bucketing + +import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, HashClusteredDistribution} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{FileSourceScanExec, FilterExec, ProjectExec, SortExec, SparkPlan} +import org.apache.spark.sql.execution.aggregate.BaseAggregateExec +import org.apache.spark.sql.execution.exchange.Exchange +import org.apache.spark.sql.internal.SQLConf + +/** + * Disable unnecessary bucketed table scan based on actual physical query plan. + * NOTE: this rule is designed to be applied right after [[EnsureRequirements]], + * where all [[ShuffleExchangeExec]] and [[SortExec]] have been added to plan properly. + * + * When BUCKETING_ENABLED and AUTO_BUCKETED_SCAN_ENABLED are set to true, go through + * query plan to check where bucketed table scan is unnecessary, and disable bucketed table + * scan if: + * + * 1. The sub-plan from root to bucketed table scan, does not contain + * [[hasInterestingPartition]] operator. + * + * 2. The sub-plan from the nearest downstream [[hasInterestingPartition]] operator + * to the bucketed table scan, contains only [[isAllowedUnaryExecNode]] operators + * and at least one [[Exchange]]. + * + * Examples: + * 1. no [[hasInterestingPartition]] operator: + * Project + * | + * Filter + * | + * Scan(t1: i, j) + * (bucketed on column j, DISABLE bucketed scan) + * + * 2. join: + * SortMergeJoin(t1.i = t2.j) + * / \ + * Sort(i) Sort(j) + * / \ + * Shuffle(i) Scan(t2: i, j) + * / (bucketed on column j, enable bucketed scan) + * Scan(t1: i, j) + * (bucketed on column j, DISABLE bucketed scan) + * + * 3. aggregate: + * HashAggregate(i, ..., Final) + * | + * Shuffle(i) + * | + * HashAggregate(i, ..., Partial) + * | + * Filter + * | + * Scan(t1: i, j) + * (bucketed on column j, DISABLE bucketed scan) + * + * The idea of [[hasInterestingPartition]] is inspired from "interesting order" in + * the paper "Access Path Selection in a Relational Database Management System" + * (https://dl.acm.org/doi/10.1145/582095.582099). + */ +case class DisableUnnecessaryBucketedScan(conf: SQLConf) extends Rule[SparkPlan] { + + /** + * Disable bucketed table scan with pre-order traversal of plan. + * + * @param withInterestingPartition The traversed plan has operator with interesting partition. + * @param withExchange The traversed plan has [[Exchange]] operator. + * @param withAllowedNode The traversed plan has only [[isAllowedUnaryExecNode]] operators. + */ + private def disableBucketWithInterestingPartition( + plan: SparkPlan, + withInterestingPartition: Boolean, + withExchange: Boolean, + withAllowedNode: Boolean): SparkPlan = { + plan match { + case p if hasInterestingPartition(p) => + // Operator with interesting partition, propagates `withInterestingPartition` as true + // to its children, and resets `withExchange` and `withAllowedNode`. + p.mapChildren(disableBucketWithInterestingPartition(_, true, false, true)) + case exchange: Exchange => + // Exchange operator propagates `withExchange` as true to its child. + exchange.mapChildren(disableBucketWithInterestingPartition( + _, withInterestingPartition, true, withAllowedNode)) + case scan: FileSourceScanExec => + if (isBucketedScanWithoutFilter(scan)) { + if (!withInterestingPartition || (withExchange && withAllowedNode)) { + scan.copy(disableBucketedScan = true) + } else { + scan + } + } else { + scan + } + case o => + o.mapChildren(disableBucketWithInterestingPartition( + _, + withInterestingPartition, + withExchange, + withAllowedNode && isAllowedUnaryExecNode(o))) + } + } + + private def hasInterestingPartition(plan: SparkPlan): Boolean = { + plan.requiredChildDistribution.exists { + case _: ClusteredDistribution | _: HashClusteredDistribution => true + case _ => false + } + } + + /** + * Check if the operator is allowed single-child operator. + * We may revisit this method later as we probably can + * remove this restriction to allow arbitrary operator between + * bucketed table scan and operator with interesting partition. + */ + private def isAllowedUnaryExecNode(plan: SparkPlan): Boolean = { + plan match { + case _: SortExec | _: ProjectExec | _: FilterExec => true + case partialAgg: BaseAggregateExec => + partialAgg.requiredChildDistributionExpressions.isEmpty + case _ => false + } + } + + private def isBucketedScanWithoutFilter(scan: FileSourceScanExec): Boolean = { + // Do not disable bucketed table scan if it has filter pruning, + // because bucketed table scan is still useful here to save CPU/IO cost with + // only reading selected bucket files. + scan.bucketedScan && scan.optionalBucketSet.isEmpty + } + + def apply(plan: SparkPlan): SparkPlan = { + lazy val hasBucketedScanWithoutFilter = plan.find { + case scan: FileSourceScanExec => isBucketedScanWithoutFilter(scan) + case _ => false + }.isDefined + + if (!conf.bucketingEnabled || !conf.autoBucketedScanEnabled || !hasBucketedScanWithoutFilter) { + plan + } else { + disableBucketWithInterestingPartition(plan, false, false, true) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala index 3cc59af9b7ce3..cb7efd3f7716b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala @@ -23,7 +23,6 @@ import java.nio.ByteOrder import scala.collection.mutable import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.vectorized.WritableColumnVector import org.apache.spark.sql.types._ @@ -182,8 +181,7 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { private var _uncompressedSize = 0 private var _compressedSize = 0 - // Using `MutableRow` to store the last value to avoid boxing/unboxing cost. - private val lastValue = new SpecificInternalRow(Seq(columnType.dataType)) + private var lastValue: T#InternalType = _ private var lastRun = 0 override def uncompressedSize: Int = _uncompressedSize @@ -195,16 +193,16 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { val actualSize = columnType.actualSize(row, ordinal) _uncompressedSize += actualSize - if (lastValue.isNullAt(0)) { - columnType.copyField(row, ordinal, lastValue, 0) + if (lastValue == null) { + lastValue = columnType.clone(value) lastRun = 1 _compressedSize += actualSize + 4 } else { - if (columnType.getField(lastValue, 0) == value) { + if (lastValue == value) { lastRun += 1 } else { _compressedSize += actualSize + 4 - columnType.copyField(row, ordinal, lastValue, 0) + lastValue = columnType.clone(value) lastRun = 1 } } @@ -214,30 +212,27 @@ private[columnar] case object RunLengthEncoding extends CompressionScheme { to.putInt(RunLengthEncoding.typeId) if (from.hasRemaining) { - val currentValue = new SpecificInternalRow(Seq(columnType.dataType)) var currentRun = 1 - val value = new SpecificInternalRow(Seq(columnType.dataType)) - - columnType.extract(from, currentValue, 0) + var currentValue = columnType.extract(from) while (from.hasRemaining) { - columnType.extract(from, value, 0) + val value = columnType.extract(from) - if (value.get(0, columnType.dataType) == currentValue.get(0, columnType.dataType)) { + if (value == currentValue) { currentRun += 1 } else { // Writes current run - columnType.append(currentValue, 0, to) + columnType.append(currentValue, to) to.putInt(currentRun) // Resets current run - columnType.copyField(value, 0, currentValue, 0) + currentValue = value currentRun = 1 } } // Writes the last run - columnType.append(currentValue, 0, to) + columnType.append(currentValue, to) to.putInt(currentRun) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala index c047be774d99a..8bf7504716f79 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala @@ -163,7 +163,7 @@ object CommandUtils extends Logging { .getConfString("hive.exec.stagingdir", ".hive-staging") val filter = new PathFilterIgnoreNonData(stagingDir) val sizes = InMemoryFileIndex.bulkListLeafFiles(paths.flatten, - sparkSession.sessionState.newHadoopConf(), filter, sparkSession, areRootPaths = true).map { + sparkSession.sessionState.newHadoopConf(), filter, sparkSession, isRootLevel = true).map { case (_, files) => files.map(_.getLen).sum } // the size is 0 where paths(i) is not defined and sizes(i) where it is defined diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 18fd2a5ac2330..70f20cd8b7c06 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -78,7 +78,7 @@ case class ExecutedCommandExec(cmd: RunnableCommand) extends LeafExecNode { override def executeCollect(): Array[InternalRow] = sideEffectResult.toArray - override def executeToIterator: Iterator[InternalRow] = sideEffectResult.toIterator + override def executeToIterator(): Iterator[InternalRow] = sideEffectResult.toIterator override def executeTake(limit: Int): Array[InternalRow] = sideEffectResult.take(limit).toArray @@ -119,7 +119,7 @@ case class DataWritingCommandExec(cmd: DataWritingCommand, child: SparkPlan) override def executeCollect(): Array[InternalRow] = sideEffectResult.toArray - override def executeToIterator: Iterator[InternalRow] = sideEffectResult.toIterator + override def executeToIterator(): Iterator[InternalRow] = sideEffectResult.toIterator override def executeTake(limit: Int): Array[InternalRow] = sideEffectResult.take(limit).toArray diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala index fae8de4780102..d76b4b8894783 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala @@ -88,7 +88,9 @@ case class CreateFunctionCommand( } else { // For a permanent, we will store the metadata into underlying external catalog. // This function will be loaded into the FunctionRegistry when a query uses it. - // We do not load it into FunctionRegistry right now. + // We do not load it into FunctionRegistry right now, to avoid loading the resource and + // UDF class immediately, as the Spark application to create the function may not have + // access to the resource and/or UDF class. catalog.createFunction(func, ignoreIfExists) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index f94c9712a31cc..206f952fed0ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.plans.DescribeTableSchema +import org.apache.spark.sql.catalyst.plans.DescribeCommandSchema import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdentifier, CaseInsensitiveMap} import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils} @@ -597,7 +597,7 @@ case class TruncateTableCommand( } abstract class DescribeCommandBase extends RunnableCommand { - override val output = DescribeTableSchema.describeTableAttributes() + override val output = DescribeCommandSchema.describeTableAttributes() protected def describeSchema( schema: StructType, @@ -760,14 +760,7 @@ case class DescribeColumnCommand( isExtended: Boolean) extends RunnableCommand { - override val output: Seq[Attribute] = { - Seq( - AttributeReference("info_name", StringType, nullable = false, - new MetadataBuilder().putString("comment", "name of the column info").build())(), - AttributeReference("info_value", StringType, nullable = false, - new MetadataBuilder().putString("comment", "value of the column info").build())() - ) - } + override val output: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes() override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog @@ -1381,3 +1374,22 @@ case class ShowCreateTableAsSerdeCommand(table: TableIdentifier) } } } + +/** + * A command to refresh all cached entries associated with the table. + * + * The syntax of using this command in SQL is: + * {{{ + * REFRESH TABLE [db_name.]table_name + * }}} + */ +case class RefreshTableCommand(tableIdent: TableIdentifier) + extends RunnableCommand { + + override def run(sparkSession: SparkSession): Seq[Row] = { + // Refresh the given table's metadata. If this table is cached as an InMemoryRelation, + // drop the original cached version and make the new version cached lazily. + sparkSession.catalog.refreshTable(tableIdent.quotedString) + Seq.empty[Row] + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index cc2a4a6b3ca96..94f34a9b39b28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -174,7 +174,7 @@ case class CreateViewCommand( def verify(child: LogicalPlan) { child.collect { // Disallow creating permanent views based on temporary views. - case UnresolvedRelation(nameParts, _) if catalog.isTempView(nameParts) => + case UnresolvedRelation(nameParts, _, _) if catalog.isTempView(nameParts) => throw new AnalysisException(s"Not allowed to create a permanent view $name by " + s"referencing a temporary view ${nameParts.quoted}. " + "Please create a temp view instead by CREATE TEMP VIEW") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 3ccff6d89babd..86e85719272e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources import java.util.Locale +import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.hadoop.fs.Path @@ -36,12 +37,17 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.ScanOperation import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoStatement, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2 +import org.apache.spark.sql.connector.catalog.SupportsRead +import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.execution.streaming.StreamingRelation import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.unsafe.types.UTF8String /** @@ -237,11 +243,12 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast * data source. */ class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] { - private def readDataSourceTable(table: CatalogTable): LogicalPlan = { + private def readDataSourceTable( + table: CatalogTable, extraOptions: CaseInsensitiveStringMap): LogicalPlan = { val qualifiedTableName = QualifiedTableName(table.database, table.identifier.table) val catalog = sparkSession.sessionState.catalog + val dsOptions = DataSourceUtils.generateDatasourceOptions(extraOptions, table) catalog.getCachedPlan(qualifiedTableName, () => { - val pathOption = table.storage.locationUri.map("path" -> CatalogUtils.URIToString(_)) val dataSource = DataSource( sparkSession, @@ -251,25 +258,54 @@ class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] partitionColumns = table.partitionColumnNames, bucketSpec = table.bucketSpec, className = table.provider.get, - options = table.storage.properties ++ pathOption, + options = dsOptions, catalogTable = Some(table)) LogicalRelation(dataSource.resolveRelation(checkFilesExist = false), table) }) } + private def getStreamingRelation( + table: CatalogTable, + extraOptions: CaseInsensitiveStringMap): StreamingRelation = { + val dsOptions = DataSourceUtils.generateDatasourceOptions(extraOptions, table) + val dataSource = DataSource( + sparkSession, + className = table.provider.get, + userSpecifiedSchema = Some(table.schema), + options = dsOptions) + StreamingRelation(dataSource) + } + + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case i @ InsertIntoStatement(UnresolvedCatalogRelation(tableMeta), _, _, _, _) + case i @ InsertIntoStatement(UnresolvedCatalogRelation(tableMeta, options, false), _, _, _, _) if DDLUtils.isDatasourceTable(tableMeta) => - i.copy(table = readDataSourceTable(tableMeta)) + i.copy(table = readDataSourceTable(tableMeta, options)) - case i @ InsertIntoStatement(UnresolvedCatalogRelation(tableMeta), _, _, _, _) => + case i @ InsertIntoStatement(UnresolvedCatalogRelation(tableMeta, _, false), _, _, _, _) => i.copy(table = DDLUtils.readHiveTable(tableMeta)) - case UnresolvedCatalogRelation(tableMeta) if DDLUtils.isDatasourceTable(tableMeta) => - readDataSourceTable(tableMeta) + case UnresolvedCatalogRelation(tableMeta, options, false) + if DDLUtils.isDatasourceTable(tableMeta) => + readDataSourceTable(tableMeta, options) - case UnresolvedCatalogRelation(tableMeta) => + case UnresolvedCatalogRelation(tableMeta, _, false) => DDLUtils.readHiveTable(tableMeta) + + case UnresolvedCatalogRelation(tableMeta, extraOptions, true) => + getStreamingRelation(tableMeta, extraOptions) + + case s @ StreamingRelationV2( + _, _, table, extraOptions, _, _, _, Some(UnresolvedCatalogRelation(tableMeta, _, true))) => + import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ + val v1Relation = getStreamingRelation(tableMeta, extraOptions) + if (table.isInstanceOf[SupportsRead] + && table.supportsAny(MICRO_BATCH_READ, CONTINUOUS_READ)) { + s.copy(v1Relation = Some(v1Relation)) + } else { + // Fallback to V1 relation + v1Relation + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala index abb74d8d09ec6..b4308a872bb39 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.datasources import java.util.Locale +import scala.collection.JavaConverters._ + import org.apache.hadoop.fs.Path import org.json4s.NoTypeHints import org.json4s.jackson.Serialization @@ -26,11 +28,13 @@ import org.json4s.jackson.Serialization import org.apache.spark.SparkUpgradeException import org.apache.spark.sql.{SPARK_LEGACY_DATETIME, SPARK_VERSION_METADATA_KEY} import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogUtils} import org.apache.spark.sql.catalyst.util.RebaseDateTime import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.Utils @@ -190,4 +194,34 @@ object DataSourceUtils { case LegacyBehaviorPolicy.LEGACY => RebaseDateTime.rebaseGregorianToJulianMicros case LegacyBehaviorPolicy.CORRECTED => identity[Long] } + + def generateDatasourceOptions( + extraOptions: CaseInsensitiveStringMap, table: CatalogTable): Map[String, String] = { + val pathOption = table.storage.locationUri.map("path" -> CatalogUtils.URIToString(_)) + val options = table.storage.properties ++ pathOption + if (!SQLConf.get.getConf(SQLConf.LEGACY_EXTRA_OPTIONS_BEHAVIOR)) { + // Check the same key with different values + table.storage.properties.foreach { case (k, v) => + if (extraOptions.containsKey(k) && extraOptions.get(k) != v) { + throw new AnalysisException( + s"Fail to resolve data source for the table ${table.identifier} since the table " + + s"serde property has the duplicated key $k with extra options specified for this " + + "scan operation. To fix this, you can rollback to the legacy behavior of ignoring " + + "the extra options by setting the config " + + s"${SQLConf.LEGACY_EXTRA_OPTIONS_BEHAVIOR.key} to `false`, or address the " + + s"conflicts of the same config.") + } + } + // To keep the original key from table properties, here we filter all case insensitive + // duplicate keys out from extra options. + val lowerCasedDuplicatedKeys = + table.storage.properties.keySet.map(_.toLowerCase(Locale.ROOT)) + .intersect(extraOptions.keySet.asScala) + extraOptions.asCaseSensitiveMap().asScala.filterNot { + case (k, _) => lowerCasedDuplicatedKeys.contains(k.toLowerCase(Locale.ROOT)) + }.toMap ++ options + } else { + options + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala index a488ed16a835a..130894e9bc025 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala @@ -17,23 +17,18 @@ package org.apache.spark.sql.execution.datasources -import java.io.FileNotFoundException - import scala.collection.mutable import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ -import org.apache.hadoop.fs.viewfs.ViewFileSystem -import org.apache.hadoop.hdfs.DistributedFileSystem import org.apache.hadoop.mapred.{FileInputFormat, JobConf} -import org.apache.spark.SparkContext import org.apache.spark.internal.Logging import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.streaming.FileStreamSink import org.apache.spark.sql.types.StructType -import org.apache.spark.util.SerializableConfiguration +import org.apache.spark.util.HadoopFSUtils /** @@ -133,7 +128,7 @@ class InMemoryFileIndex( } val filter = FileInputFormat.getInputPathFilter(new JobConf(hadoopConf, this.getClass)) val discovered = InMemoryFileIndex.bulkListLeafFiles( - pathsToFetch.toSeq, hadoopConf, filter, sparkSession, areRootPaths = true) + pathsToFetch.toSeq, hadoopConf, filter, sparkSession, isRootLevel = true) discovered.foreach { case (path, leafFiles) => HiveCatalogMetrics.incrementFilesDiscovered(leafFiles.size) fileStatusCache.putLeafFiles(path, leafFiles.toArray) @@ -147,286 +142,24 @@ class InMemoryFileIndex( object InMemoryFileIndex extends Logging { - /** A serializable variant of HDFS's BlockLocation. */ - private case class SerializableBlockLocation( - names: Array[String], - hosts: Array[String], - offset: Long, - length: Long) - - /** A serializable variant of HDFS's FileStatus. */ - private case class SerializableFileStatus( - path: String, - length: Long, - isDir: Boolean, - blockReplication: Short, - blockSize: Long, - modificationTime: Long, - accessTime: Long, - blockLocations: Array[SerializableBlockLocation]) - - /** - * Lists a collection of paths recursively. Picks the listing strategy adaptively depending - * on the number of paths to list. - * - * This may only be called on the driver. - * - * @return for each input path, the set of discovered files for the path - */ private[sql] def bulkListLeafFiles( paths: Seq[Path], hadoopConf: Configuration, filter: PathFilter, sparkSession: SparkSession, - areRootPaths: Boolean): Seq[(Path, Seq[FileStatus])] = { - - val ignoreMissingFiles = sparkSession.sessionState.conf.ignoreMissingFiles - val ignoreLocality = sparkSession.sessionState.conf.ignoreDataLocality - - // Short-circuits parallel listing when serial listing is likely to be faster. - if (paths.size <= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { - return paths.map { path => - val leafFiles = listLeafFiles( - path, - hadoopConf, - filter, - Some(sparkSession), - ignoreMissingFiles = ignoreMissingFiles, - ignoreLocality = ignoreLocality, - isRootPath = areRootPaths) - (path, leafFiles) - } - } - - logInfo(s"Listing leaf files and directories in parallel under ${paths.length} paths." + - s" The first several paths are: ${paths.take(10).mkString(", ")}.") - HiveCatalogMetrics.incrementParallelListingJobCount(1) - - val sparkContext = sparkSession.sparkContext - val serializableConfiguration = new SerializableConfiguration(hadoopConf) - val serializedPaths = paths.map(_.toString) - val parallelPartitionDiscoveryParallelism = - sparkSession.sessionState.conf.parallelPartitionDiscoveryParallelism - - // Set the number of parallelism to prevent following file listing from generating many tasks - // in case of large #defaultParallelism. - val numParallelism = Math.min(paths.size, parallelPartitionDiscoveryParallelism) - - val previousJobDescription = sparkContext.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION) - val statusMap = try { - val description = paths.size match { - case 0 => - s"Listing leaf files and directories 0 paths" - case 1 => - s"Listing leaf files and directories for 1 path:
    ${paths(0)}" - case s => - s"Listing leaf files and directories for $s paths:
    ${paths(0)}, ..." - } - sparkContext.setJobDescription(description) - sparkContext - .parallelize(serializedPaths, numParallelism) - .mapPartitions { pathStrings => - val hadoopConf = serializableConfiguration.value - pathStrings.map(new Path(_)).toSeq.map { path => - val leafFiles = listLeafFiles( - path, - hadoopConf, - filter, - None, - ignoreMissingFiles = ignoreMissingFiles, - ignoreLocality = ignoreLocality, - isRootPath = areRootPaths) - (path, leafFiles) - }.iterator - }.map { case (path, statuses) => - val serializableStatuses = statuses.map { status => - // Turn FileStatus into SerializableFileStatus so we can send it back to the driver - val blockLocations = status match { - case f: LocatedFileStatus => - f.getBlockLocations.map { loc => - SerializableBlockLocation( - loc.getNames, - loc.getHosts, - loc.getOffset, - loc.getLength) - } - - case _ => - Array.empty[SerializableBlockLocation] - } - - SerializableFileStatus( - status.getPath.toString, - status.getLen, - status.isDirectory, - status.getReplication, - status.getBlockSize, - status.getModificationTime, - status.getAccessTime, - blockLocations) - } - (path.toString, serializableStatuses) - }.collect() - } finally { - sparkContext.setJobDescription(previousJobDescription) - } - - // turn SerializableFileStatus back to Status - statusMap.map { case (path, serializableStatuses) => - val statuses = serializableStatuses.map { f => - val blockLocations = f.blockLocations.map { loc => - new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) - } - new LocatedFileStatus( - new FileStatus( - f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, - new Path(f.path)), - blockLocations) - } - (new Path(path), statuses) - } - } - - /** - * Lists a single filesystem path recursively. If a SparkSession object is specified, this - * function may launch Spark jobs to parallelize listing. - * - * If sessionOpt is None, this may be called on executors. - * - * @return all children of path that match the specified filter. - */ - private def listLeafFiles( - path: Path, - hadoopConf: Configuration, - filter: PathFilter, - sessionOpt: Option[SparkSession], - ignoreMissingFiles: Boolean, - ignoreLocality: Boolean, - isRootPath: Boolean): Seq[FileStatus] = { - logTrace(s"Listing $path") - val fs = path.getFileSystem(hadoopConf) - - // Note that statuses only include FileStatus for the files and dirs directly under path, - // and does not include anything else recursively. - val statuses: Array[FileStatus] = try { - fs match { - // DistributedFileSystem overrides listLocatedStatus to make 1 single call to namenode - // to retrieve the file status with the file block location. The reason to still fallback - // to listStatus is because the default implementation would potentially throw a - // FileNotFoundException which is better handled by doing the lookups manually below. - case (_: DistributedFileSystem | _: ViewFileSystem) if !ignoreLocality => - val remoteIter = fs.listLocatedStatus(path) - new Iterator[LocatedFileStatus]() { - def next(): LocatedFileStatus = remoteIter.next - def hasNext(): Boolean = remoteIter.hasNext - }.toArray - case _ => fs.listStatus(path) - } - } catch { - // If we are listing a root path (e.g. a top level directory of a table), we need to - // ignore FileNotFoundExceptions during this root level of the listing because - // - // (a) certain code paths might construct an InMemoryFileIndex with root paths that - // might not exist (i.e. not all callers are guaranteed to have checked - // path existence prior to constructing InMemoryFileIndex) and, - // (b) we need to ignore deleted root paths during REFRESH TABLE, otherwise we break - // existing behavior and break the ability drop SessionCatalog tables when tables' - // root directories have been deleted (which breaks a number of Spark's own tests). - // - // If we are NOT listing a root path then a FileNotFoundException here means that the - // directory was present in a previous level of file listing but is absent in this - // listing, likely indicating a race condition (e.g. concurrent table overwrite or S3 - // list inconsistency). - // - // The trade-off in supporting existing behaviors / use-cases is that we won't be - // able to detect race conditions involving root paths being deleted during - // InMemoryFileIndex construction. However, it's still a net improvement to detect and - // fail-fast on the non-root cases. For more info see the SPARK-27676 review discussion. - case _: FileNotFoundException if isRootPath || ignoreMissingFiles => - logWarning(s"The directory $path was not found. Was it deleted very recently?") - Array.empty[FileStatus] - } - - val filteredStatuses = statuses.filterNot(status => shouldFilterOut(status.getPath.getName)) - - val allLeafStatuses = { - val (dirs, topLevelFiles) = filteredStatuses.partition(_.isDirectory) - val nestedFiles: Seq[FileStatus] = sessionOpt match { - case Some(session) => - bulkListLeafFiles( - dirs.map(_.getPath), - hadoopConf, - filter, - session, - areRootPaths = false - ).flatMap(_._2) - case _ => - dirs.flatMap { dir => - listLeafFiles( - dir.getPath, - hadoopConf, - filter, - sessionOpt, - ignoreMissingFiles = ignoreMissingFiles, - ignoreLocality = ignoreLocality, - isRootPath = false) - } - } - val allFiles = topLevelFiles ++ nestedFiles - if (filter != null) allFiles.filter(f => filter.accept(f.getPath)) else allFiles - } - - val missingFiles = mutable.ArrayBuffer.empty[String] - val filteredLeafStatuses = allLeafStatuses.filterNot( - status => shouldFilterOut(status.getPath.getName)) - val resolvedLeafStatuses = filteredLeafStatuses.flatMap { - case f: LocatedFileStatus => - Some(f) - - // NOTE: - // - // - Although S3/S3A/S3N file system can be quite slow for remote file metadata - // operations, calling `getFileBlockLocations` does no harm here since these file system - // implementations don't actually issue RPC for this method. - // - // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not - // be a big deal since we always use to `bulkListLeafFiles` when the number of - // paths exceeds threshold. - case f if !ignoreLocality => - // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), - // which is very slow on some file system (RawLocalFileSystem, which is launch a - // subprocess and parse the stdout). - try { - val locations = fs.getFileBlockLocations(f, 0, f.getLen).map { loc => - // Store BlockLocation objects to consume less memory - if (loc.getClass == classOf[BlockLocation]) { - loc - } else { - new BlockLocation(loc.getNames, loc.getHosts, loc.getOffset, loc.getLength) - } - } - val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, - f.getModificationTime, 0, null, null, null, null, f.getPath, locations) - if (f.isSymlink) { - lfs.setSymlink(f.getSymlink) - } - Some(lfs) - } catch { - case _: FileNotFoundException if ignoreMissingFiles => - missingFiles += f.getPath.toString - None - } - - case f => Some(f) - } - - if (missingFiles.nonEmpty) { - logWarning( - s"the following files were missing during file scan:\n ${missingFiles.mkString("\n ")}") - } - - resolvedLeafStatuses - } + isRootLevel: Boolean): Seq[(Path, Seq[FileStatus])] = { + HadoopFSUtils.parallelListLeafFiles( + sc = sparkSession.sparkContext, + paths = paths, + hadoopConf = hadoopConf, + filter = filter, + isRootLevel = isRootLevel, + ignoreMissingFiles = sparkSession.sessionState.conf.ignoreMissingFiles, + ignoreLocality = sparkSession.sessionState.conf.ignoreDataLocality, + parallelismThreshold = sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold, + parallelismMax = sparkSession.sessionState.conf.parallelPartitionDiscoveryParallelism, + filterFun = Some(shouldFilterOut)) + } /** Checks if we should filter out this path name. */ def shouldFilterOut(pathName: String): Boolean = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index 4022640224424..e455fae4675f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -103,17 +103,6 @@ case class CreateTempViewUsing( } } -case class RefreshTable(tableIdent: TableIdentifier) - extends RunnableCommand { - - override def run(sparkSession: SparkSession): Seq[Row] = { - // Refresh the given table's metadata. If this table is cached as an InMemoryRelation, - // drop the original cached version and make the new version cached lazily. - sparkSession.catalog.refreshTable(tableIdent.quotedString) - Seq.empty[Row] - } -} - case class RefreshResource(path: String) extends RunnableCommand { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala index 9e0438c0016bd..e6fff8dbdbd7c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap * Options for the JDBC data source. */ class JDBCOptions( - @transient val parameters: CaseInsensitiveMap[String]) + val parameters: CaseInsensitiveMap[String]) extends Serializable with Logging { import JDBCOptions._ @@ -209,7 +209,7 @@ class JDBCOptions( } class JdbcOptionsInWrite( - @transient override val parameters: CaseInsensitiveMap[String]) + override val parameters: CaseInsensitiveMap[String]) extends JDBCOptions(parameters) { import JDBCOptions._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 5831c35c7e301..202f2e03b68d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -63,7 +63,7 @@ object JdbcUtils extends Logging { throw new IllegalStateException( s"Did not find registered driver with class $driverClass") } - val connection = ConnectionProvider.create(driver, options).getConnection() + val connection = ConnectionProvider.create(driver, options.parameters) require(connection != null, s"The driver could not open a JDBC connection. Check the URL: ${options.url}") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/BasicConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/BasicConnectionProvider.scala index 16b244cc617ce..1c0513f982a1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/BasicConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/BasicConnectionProvider.scala @@ -18,18 +18,32 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection import java.sql.{Connection, Driver} +import java.util.Properties -import scala.collection.JavaConverters._ - +import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions +import org.apache.spark.sql.jdbc.JdbcConnectionProvider + +private[jdbc] class BasicConnectionProvider extends JdbcConnectionProvider with Logging { + /** + * Additional properties for data connection (Data source property takes precedence). + */ + def getAdditionalProperties(options: JDBCOptions): Properties = new Properties() + + override val name: String = "basic" + + override def canHandle(driver: Driver, options: Map[String, String]): Boolean = { + val jdbcOptions = new JDBCOptions(options) + jdbcOptions.keytab == null || jdbcOptions.principal == null + } -private[jdbc] class BasicConnectionProvider(driver: Driver, options: JDBCOptions) - extends ConnectionProvider { - def getConnection(): Connection = { - val properties = getAdditionalProperties() - options.asConnectionProperties.entrySet().asScala.foreach { e => - properties.put(e.getKey(), e.getValue()) + override def getConnection(driver: Driver, options: Map[String, String]): Connection = { + val jdbcOptions = new JDBCOptions(options) + val properties = getAdditionalProperties(jdbcOptions) + options.foreach { case(k, v) => + properties.put(k, v) } - driver.connect(options.url, properties) + logDebug(s"JDBC connection initiated with URL: ${jdbcOptions.url} and properties: $properties") + driver.connect(jdbcOptions.url, properties) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala index ce45be442ccc3..e81add4df960a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala @@ -18,60 +18,49 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection import java.sql.{Connection, Driver} -import java.util.Properties +import java.util.ServiceLoader -import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions - -/** - * Connection provider which opens connection toward various databases (database specific instance - * needed). If kerberos authentication required then it's the provider's responsibility to set all - * the parameters. - */ -private[jdbc] trait ConnectionProvider { - /** - * Additional properties for data connection (Data source property takes precedence). - */ - def getAdditionalProperties(): Properties = new Properties() +import scala.collection.mutable - /** - * Opens connection toward the database. - */ - def getConnection(): Connection -} +import org.apache.spark.internal.Logging +import org.apache.spark.security.SecurityConfigurationLock +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.jdbc.JdbcConnectionProvider +import org.apache.spark.util.Utils private[jdbc] object ConnectionProvider extends Logging { - def create(driver: Driver, options: JDBCOptions): ConnectionProvider = { - if (options.keytab == null || options.principal == null) { - logDebug("No authentication configuration found, using basic connection provider") - new BasicConnectionProvider(driver, options) - } else { - logDebug("Authentication configuration found, using database specific connection provider") - options.driverClass match { - case PostgresConnectionProvider.driverClass => - logDebug("Postgres connection provider found") - new PostgresConnectionProvider(driver, options) + private val providers = loadProviders() - case MariaDBConnectionProvider.driverClass => - logDebug("MariaDB connection provider found") - new MariaDBConnectionProvider(driver, options) + def loadProviders(): Seq[JdbcConnectionProvider] = { + val loader = ServiceLoader.load(classOf[JdbcConnectionProvider], + Utils.getContextOrSparkClassLoader) + val providers = mutable.ArrayBuffer[JdbcConnectionProvider]() - case DB2ConnectionProvider.driverClass => - logDebug("DB2 connection provider found") - new DB2ConnectionProvider(driver, options) - - case MSSQLConnectionProvider.driverClass => - logDebug("MS SQL connection provider found") - new MSSQLConnectionProvider(driver, options) + val iterator = loader.iterator + while (iterator.hasNext) { + try { + val provider = iterator.next + logDebug(s"Loaded built-in provider: $provider") + providers += provider + } catch { + case t: Throwable => + logError("Failed to load built-in provider.") + logInfo("Loading of the provider failed with the exception:", t) + } + } - case OracleConnectionProvider.driverClass => - logDebug("Oracle connection provider found") - new OracleConnectionProvider(driver, options) + val disabledProviders = Utils.stringToSeq(SQLConf.get.disabledJdbcConnectionProviders) + // toSeq seems duplicate but it's needed for Scala 2.13 + providers.filterNot(p => disabledProviders.contains(p.name)).toSeq + } - case _ => - throw new IllegalArgumentException(s"Driver ${options.driverClass} does not support " + - "Kerberos authentication") - } + def create(driver: Driver, options: Map[String, String]): Connection = { + val filteredProviders = providers.filter(_.canHandle(driver, options)) + require(filteredProviders.size == 1, + "JDBC connection initiated but not exactly one connection provider found which can handle " + + s"it. Found active providers: ${filteredProviders.mkString(", ")}") + SecurityConfigurationLock.synchronized { + filteredProviders.head.getConnection(driver, options) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala index 095821cf83890..775c3ae4a533a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala @@ -25,22 +25,27 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions -private[sql] class DB2ConnectionProvider(driver: Driver, options: JDBCOptions) - extends SecureConnectionProvider(driver, options) { - override val appEntry: String = "JaasClient" - - override def getConnection(): Connection = { - setAuthenticationConfigIfNeeded() - UserGroupInformation.loginUserFromKeytabAndReturnUGI(options.principal, options.keytab).doAs( - new PrivilegedExceptionAction[Connection]() { - override def run(): Connection = { - DB2ConnectionProvider.super.getConnection() +private[sql] class DB2ConnectionProvider extends SecureConnectionProvider { + override val driverClass = "com.ibm.db2.jcc.DB2Driver" + + override val name: String = "db2" + + override def appEntry(driver: Driver, options: JDBCOptions): String = "JaasClient" + + override def getConnection(driver: Driver, options: Map[String, String]): Connection = { + val jdbcOptions = new JDBCOptions(options) + setAuthenticationConfigIfNeeded(driver, jdbcOptions) + UserGroupInformation.loginUserFromKeytabAndReturnUGI(jdbcOptions.principal, jdbcOptions.keytab) + .doAs( + new PrivilegedExceptionAction[Connection]() { + override def run(): Connection = { + DB2ConnectionProvider.super.getConnection(driver, options) + } } - } - ) + ) } - override def getAdditionalProperties(): Properties = { + override def getAdditionalProperties(options: JDBCOptions): Properties = { val result = new Properties() // 11 is the integer value for kerberos result.put("securityMechanism", new String("11")) @@ -48,14 +53,10 @@ private[sql] class DB2ConnectionProvider(driver: Driver, options: JDBCOptions) result } - override def setAuthenticationConfigIfNeeded(): Unit = SecurityConfigurationLock.synchronized { - val (parent, configEntry) = getConfigWithAppEntry() + override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = { + val (parent, configEntry) = getConfigWithAppEntry(driver, options) if (configEntry == null || configEntry.isEmpty) { - setAuthenticationConfig(parent) + setAuthenticationConfig(parent, driver, options) } } } - -private[sql] object DB2ConnectionProvider { - val driverClass = "com.ibm.db2.jcc.DB2Driver" -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala index 2950aa9b4db94..e3d3e1a43d510 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala @@ -25,12 +25,13 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions -private[sql] class MSSQLConnectionProvider( - driver: Driver, - options: JDBCOptions, - parserMethod: String = "parseAndMergeProperties" - ) extends SecureConnectionProvider(driver, options) { - override val appEntry: String = { +private[sql] class MSSQLConnectionProvider extends SecureConnectionProvider { + override val driverClass = "com.microsoft.sqlserver.jdbc.SQLServerDriver" + val parserMethod: String = "parseAndMergeProperties" + + override val name: String = "mssql" + + override def appEntry(driver: Driver, options: JDBCOptions): String = { val configName = "jaasConfigurationName" val appEntryDefault = "SQLJDBCDriver" @@ -58,18 +59,20 @@ private[sql] class MSSQLConnectionProvider( } } - override def getConnection(): Connection = { - setAuthenticationConfigIfNeeded() - UserGroupInformation.loginUserFromKeytabAndReturnUGI(options.principal, options.keytab).doAs( - new PrivilegedExceptionAction[Connection]() { - override def run(): Connection = { - MSSQLConnectionProvider.super.getConnection() + override def getConnection(driver: Driver, options: Map[String, String]): Connection = { + val jdbcOptions = new JDBCOptions(options) + setAuthenticationConfigIfNeeded(driver, jdbcOptions) + UserGroupInformation.loginUserFromKeytabAndReturnUGI(jdbcOptions.principal, jdbcOptions.keytab) + .doAs( + new PrivilegedExceptionAction[Connection]() { + override def run(): Connection = { + MSSQLConnectionProvider.super.getConnection(driver, options) + } } - } - ) + ) } - override def getAdditionalProperties(): Properties = { + override def getAdditionalProperties(options: JDBCOptions): Properties = { val result = new Properties() // These props needed to reach internal kerberos authentication in the JDBC driver result.put("integratedSecurity", "true") @@ -77,8 +80,8 @@ private[sql] class MSSQLConnectionProvider( result } - override def setAuthenticationConfigIfNeeded(): Unit = SecurityConfigurationLock.synchronized { - val (parent, configEntry) = getConfigWithAppEntry() + override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = { + val (parent, configEntry) = getConfigWithAppEntry(driver, options) /** * Couple of things to mention here (v8.2.2 client): * 1. MS SQL supports JAAS application name configuration @@ -87,11 +90,7 @@ private[sql] class MSSQLConnectionProvider( val entryUsesKeytab = configEntry != null && configEntry.exists(_.getOptions().get("useKeyTab") == "true") if (configEntry == null || configEntry.isEmpty || !entryUsesKeytab) { - setAuthenticationConfig(parent) + setAuthenticationConfig(parent, driver, options) } } } - -private[sql] object MSSQLConnectionProvider { - val driverClass = "com.microsoft.sqlserver.jdbc.SQLServerDriver" -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala index 3c0286654a8ec..29a08d0b5f269 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala @@ -21,14 +21,16 @@ import java.sql.Driver import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions -private[jdbc] class MariaDBConnectionProvider(driver: Driver, options: JDBCOptions) - extends SecureConnectionProvider(driver, options) { - override val appEntry: String = { +private[jdbc] class MariaDBConnectionProvider extends SecureConnectionProvider { + override val driverClass = "org.mariadb.jdbc.Driver" + + override val name: String = "mariadb" + + override def appEntry(driver: Driver, options: JDBCOptions): String = "Krb5ConnectorContext" - } - override def setAuthenticationConfigIfNeeded(): Unit = SecurityConfigurationLock.synchronized { - val (parent, configEntry) = getConfigWithAppEntry() + override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = { + val (parent, configEntry) = getConfigWithAppEntry(driver, options) /** * Couple of things to mention here (v2.5.4 client): * 1. MariaDB doesn't support JAAS application name configuration @@ -37,11 +39,7 @@ private[jdbc] class MariaDBConnectionProvider(driver: Driver, options: JDBCOptio val entryUsesKeytab = configEntry != null && configEntry.exists(_.getOptions().get("useKeyTab") == "true") if (configEntry == null || configEntry.isEmpty || !entryUsesKeytab) { - setAuthenticationConfig(parent) + setAuthenticationConfig(parent, driver, options) } } } - -private[sql] object MariaDBConnectionProvider { - val driverClass = "org.mariadb.jdbc.Driver" -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala index c2b71b35b8128..0d43851bb255e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala @@ -25,22 +25,27 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions -private[sql] class OracleConnectionProvider(driver: Driver, options: JDBCOptions) - extends SecureConnectionProvider(driver, options) { - override val appEntry: String = "kprb5module" - - override def getConnection(): Connection = { - setAuthenticationConfigIfNeeded() - UserGroupInformation.loginUserFromKeytabAndReturnUGI(options.principal, options.keytab).doAs( - new PrivilegedExceptionAction[Connection]() { - override def run(): Connection = { - OracleConnectionProvider.super.getConnection() +private[sql] class OracleConnectionProvider extends SecureConnectionProvider { + override val driverClass = "oracle.jdbc.OracleDriver" + + override val name: String = "oracle" + + override def appEntry(driver: Driver, options: JDBCOptions): String = "kprb5module" + + override def getConnection(driver: Driver, options: Map[String, String]): Connection = { + val jdbcOptions = new JDBCOptions(options) + setAuthenticationConfigIfNeeded(driver, jdbcOptions) + UserGroupInformation.loginUserFromKeytabAndReturnUGI(jdbcOptions.principal, jdbcOptions.keytab) + .doAs( + new PrivilegedExceptionAction[Connection]() { + override def run(): Connection = { + OracleConnectionProvider.super.getConnection(driver, options) + } } - } - ) + ) } - override def getAdditionalProperties(): Properties = { + override def getAdditionalProperties(options: JDBCOptions): Properties = { val result = new Properties() // This prop is needed to turn on kerberos authentication in the JDBC driver. // The possible values can be found in AnoServices public interface @@ -49,14 +54,10 @@ private[sql] class OracleConnectionProvider(driver: Driver, options: JDBCOptions result } - override def setAuthenticationConfigIfNeeded(): Unit = SecurityConfigurationLock.synchronized { - val (parent, configEntry) = getConfigWithAppEntry() + override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = { + val (parent, configEntry) = getConfigWithAppEntry(driver, options) if (configEntry == null || configEntry.isEmpty) { - setAuthenticationConfig(parent) + setAuthenticationConfig(parent, driver, options) } } } - -private[sql] object OracleConnectionProvider { - val driverClass = "oracle.jdbc.OracleDriver" -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala index fa9232e00bd88..f26a11e34dc38 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala @@ -22,22 +22,21 @@ import java.util.Properties import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions -private[jdbc] class PostgresConnectionProvider(driver: Driver, options: JDBCOptions) - extends SecureConnectionProvider(driver, options) { - override val appEntry: String = { +private[jdbc] class PostgresConnectionProvider extends SecureConnectionProvider { + override val driverClass = "org.postgresql.Driver" + + override val name: String = "postgres" + + override def appEntry(driver: Driver, options: JDBCOptions): String = { val parseURL = driver.getClass.getMethod("parseURL", classOf[String], classOf[Properties]) val properties = parseURL.invoke(driver, options.url, null).asInstanceOf[Properties] properties.getProperty("jaasApplicationName", "pgjdbc") } - override def setAuthenticationConfigIfNeeded(): Unit = SecurityConfigurationLock.synchronized { - val (parent, configEntry) = getConfigWithAppEntry() + override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = { + val (parent, configEntry) = getConfigWithAppEntry(driver, options) if (configEntry == null || configEntry.isEmpty) { - setAuthenticationConfig(parent) + setAuthenticationConfig(parent, driver, options) } } } - -private[sql] object PostgresConnectionProvider { - val driverClass = "org.postgresql.Driver" -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala index 24eec63a7244f..80c795957dac8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala @@ -26,39 +26,49 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions import org.apache.spark.util.SecurityUtils -/** - * Some of the secure connection providers modify global JVM security configuration. - * In order to avoid race the modification must be synchronized with this. - */ -private[connection] object SecurityConfigurationLock +private[jdbc] abstract class SecureConnectionProvider extends BasicConnectionProvider with Logging { + /** + * Returns the driver canonical class name which the connection provider supports. + */ + protected val driverClass: String + + override def canHandle(driver: Driver, options: Map[String, String]): Boolean = { + val jdbcOptions = new JDBCOptions(options) + jdbcOptions.keytab != null && jdbcOptions.principal != null && + driverClass.equalsIgnoreCase(jdbcOptions.driverClass) + } -private[jdbc] abstract class SecureConnectionProvider(driver: Driver, options: JDBCOptions) - extends BasicConnectionProvider(driver, options) with Logging { - override def getConnection(): Connection = { - setAuthenticationConfigIfNeeded() - super.getConnection() + override def getConnection(driver: Driver, options: Map[String, String]): Connection = { + val jdbcOptions = new JDBCOptions(options) + setAuthenticationConfigIfNeeded(driver, jdbcOptions) + super.getConnection(driver: Driver, options: Map[String, String]) } /** * Returns JAAS application name. This is sometimes configurable on the JDBC driver level. */ - val appEntry: String + def appEntry(driver: Driver, options: JDBCOptions): String /** * Sets database specific authentication configuration when needed. If configuration already set * then later calls must be no op. When the global JVM security configuration changed then the * related code parts must be synchronized properly. */ - def setAuthenticationConfigIfNeeded(): Unit + def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit - protected def getConfigWithAppEntry(): (Configuration, Array[AppConfigurationEntry]) = { + protected def getConfigWithAppEntry( + driver: Driver, + options: JDBCOptions): (Configuration, Array[AppConfigurationEntry]) = { val parent = Configuration.getConfiguration - (parent, parent.getAppConfigurationEntry(appEntry)) + (parent, parent.getAppConfigurationEntry(appEntry(driver, options))) } - protected def setAuthenticationConfig(parent: Configuration) = { + protected def setAuthenticationConfig( + parent: Configuration, + driver: Driver, + options: JDBCOptions) = { val config = new SecureConnectionProvider.JDBCConfiguration( - parent, appEntry, options.keytab, options.principal) + parent, appEntry(driver, options), options.keytab, options.principal) logDebug("Adding database specific security configuration") Configuration.setConfiguration(config) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala index 8a6c4dce75f30..472df286eb04f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, LogicalWriteInfo, PhysicalWriteInfo, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} -import org.apache.spark.sql.internal.connector.{SimpleTableProvider, SupportsStreamingUpdate} +import org.apache.spark.sql.internal.connector.{SimpleTableProvider, SupportsStreamingUpdateAsAppend} import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -53,9 +53,8 @@ private[noop] object NoopTable extends Table with SupportsWrite { } private[noop] object NoopWriteBuilder extends WriteBuilder - with SupportsTruncate with SupportsStreamingUpdate { + with SupportsTruncate with SupportsStreamingUpdateAsAppend { override def truncate(): WriteBuilder = this - override def update(): WriteBuilder = this override def buildForBatch(): BatchWrite = NoopBatchWrite override def buildForStreaming(): StreamingWrite = NoopStreamingWrite } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index 69badb4f7d595..2671682e18f31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -45,7 +45,7 @@ import org.apache.spark.util.{SerializableConfiguration, Utils} private[sql] object OrcFileFormat { private def checkFieldName(name: String): Unit = { try { - TypeDescription.fromString(s"struct<$name:int>") + TypeDescription.fromString(s"struct<`$name`:int>") } catch { case _: IllegalArgumentException => throw new AnalysisException( @@ -185,7 +185,7 @@ class OrcFileFormat } else { // ORC predicate pushdown if (orcFilterPushDown) { - OrcUtils.readCatalystSchema(filePath, conf, ignoreCorruptFiles).map { fileSchema => + OrcUtils.readCatalystSchema(filePath, conf, ignoreCorruptFiles).foreach { fileSchema => OrcFilters.createFilter(fileSchema, filters).foreach { f => OrcInputFormat.setSearchArgument(conf, f, fileSchema.fieldNames) } diff --git a/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala similarity index 100% rename from sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala diff --git a/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala similarity index 100% rename from sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index 264cf8165e13b..623f4f7a54d00 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -81,10 +81,10 @@ object OrcUtils extends Logging { } } - def readSchema(sparkSession: SparkSession, files: Seq[FileStatus]) + def readSchema(sparkSession: SparkSession, files: Seq[FileStatus], options: Map[String, String]) : Option[StructType] = { val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles - val conf = sparkSession.sessionState.newHadoopConf() + val conf = sparkSession.sessionState.newHadoopConfWithOptions(options) files.toIterator.map(file => readSchema(file.getPath, conf, ignoreCorruptFiles)).collectFirst { case Some(schema) => logDebug(s"Reading schema from file $files, got Hive schema string: $schema") @@ -125,7 +125,7 @@ object OrcUtils extends Logging { SchemaMergeUtils.mergeSchemasInParallel( sparkSession, options, files, OrcUtils.readOrcSchemasInParallel) } else { - OrcUtils.readSchema(sparkSession, files) + OrcUtils.readSchema(sparkSession, files, options) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 60cacda9f5f1c..5fb1a4d249070 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -386,7 +386,8 @@ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] { private def preprocess( insert: InsertIntoStatement, tblName: String, - partColNames: Seq[String]): InsertIntoStatement = { + partColNames: Seq[String], + catalogTable: Option[CatalogTable]): InsertIntoStatement = { val normalizedPartSpec = PartitioningUtils.normalizePartitionSpec( insert.partitionSpec, partColNames, tblName, conf.resolver) @@ -402,6 +403,18 @@ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] { s"including ${staticPartCols.size} partition column(s) having constant value(s).") } + val partitionsTrackedByCatalog = catalogTable.isDefined && + catalogTable.get.partitionColumnNames.nonEmpty && + catalogTable.get.tracksPartitionsInCatalog + if (partitionsTrackedByCatalog && normalizedPartSpec.nonEmpty) { + // empty partition column value + if (normalizedPartSpec.filter(_._2.isDefined).exists(_._2.get.isEmpty)) { + val spec = normalizedPartSpec.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]") + throw new AnalysisException( + s"Partition spec is invalid. The spec ($spec) contains an empty partition column value") + } + } + val newQuery = TableOutputResolver.resolveOutputColumns( tblName, expectedColumns, insert.query, byName = false, conf) if (normalizedPartSpec.nonEmpty) { @@ -427,13 +440,14 @@ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] { table match { case relation: HiveTableRelation => val metadata = relation.tableMeta - preprocess(i, metadata.identifier.quotedString, metadata.partitionColumnNames) + preprocess(i, metadata.identifier.quotedString, metadata.partitionColumnNames, + Some(metadata)) case LogicalRelation(h: HadoopFsRelation, _, catalogTable, _) => val tblName = catalogTable.map(_.identifier.quotedString).getOrElse("unknown") - preprocess(i, tblName, h.partitionSchema.map(_.name)) + preprocess(i, tblName, h.partitionSchema.map(_.name), catalogTable) case LogicalRelation(_: InsertableRelation, _, catalogTable, _) => val tblName = catalogTable.map(_.identifier.quotedString).getOrElse("unknown") - preprocess(i, tblName, Nil) + preprocess(i, tblName, Nil, catalogTable) case _ => i } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index fe4f8bc83fcff..3841bd0a66987 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -127,8 +127,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat propsWithOwner, writeOptions, ifNotExists) :: Nil } - case RefreshTable(catalog, ident) => - RefreshTableExec(catalog, ident) :: Nil + case RefreshTable(r: ResolvedTable) => + RefreshTableExec(r.catalog, r.identifier) :: Nil case ReplaceTable(catalog, ident, schema, parts, props, orCreate) => val propsWithOwner = CatalogV2Util.withDefaultOwnership(props) @@ -225,6 +225,9 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat } DescribeTableExec(desc.output, r.table, isExtended) :: Nil + case DescribeColumn(_: ResolvedTable, _, _) => + throw new AnalysisException("Describing columns is not supported for v2 tables.") + case DropTable(catalog, ident, ifExists) => DropTableExec(catalog, ident, ifExists) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala index f090d7861b629..363dd154b5fbb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala @@ -95,7 +95,7 @@ trait FileScan extends Scan override def hashCode(): Int = getClass.hashCode() - val maxMetadataValueLength = 100 + val maxMetadataValueLength = sparkSession.sessionState.conf.maxMetadataStringLength override def description(): String = { val metadataStr = getMetaData().toSeq.sorted.map { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2CommandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2CommandExec.scala index 4be4a6b30edcd..7738f26dfd266 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2CommandExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2CommandExec.scala @@ -44,7 +44,7 @@ abstract class V2CommandExec extends SparkPlan { */ override def executeCollect(): Array[InternalRow] = result.toArray - override def executeToIterator: Iterator[InternalRow] = result.toIterator + override def executeToIterator(): Iterator[InternalRow] = result.toIterator override def executeTake(limit: Int): Array[InternalRow] = result.take(limit).toArray diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala index 55759497bd910..5e11ea66be4c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala @@ -30,7 +30,6 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap case class JDBCTable(ident: Identifier, schema: StructType, jdbcOptions: JDBCOptions) extends Table with SupportsRead with SupportsWrite { - assert(ident.namespace().length == 1) override def name(): String = ident.toString diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala index 0138014a8e21e..8edc2fe5585e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala @@ -61,7 +61,7 @@ class JDBCTableCatalog extends TableCatalog with Logging { .getTables(null, schemaPattern, "%", Array("TABLE")); new Iterator[Identifier] { def hasNext = rs.next() - def next = Identifier.of(namespace, rs.getString("TABLE_NAME")) + def next() = Identifier.of(namespace, rs.getString("TABLE_NAME")) }.toArray } } @@ -70,7 +70,9 @@ class JDBCTableCatalog extends TableCatalog with Logging { checkNamespace(ident.namespace()) val writeOptions = new JdbcOptionsInWrite( options.parameters + (JDBCOptions.JDBC_TABLE_NAME -> getTableName(ident))) - withConnection(JdbcUtils.tableExists(_, writeOptions)) + classifyException(s"Failed table existence check: $ident") { + withConnection(JdbcUtils.tableExists(_, writeOptions)) + } } override def dropTable(ident: Identifier): Boolean = { @@ -88,7 +90,9 @@ class JDBCTableCatalog extends TableCatalog with Logging { override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit = { checkNamespace(oldIdent.namespace()) withConnection { conn => - JdbcUtils.renameTable(conn, getTableName(oldIdent), getTableName(newIdent), options) + classifyException(s"Failed table renaming from $oldIdent to $newIdent") { + JdbcUtils.renameTable(conn, getTableName(oldIdent), getTableName(newIdent), options) + } } } @@ -123,7 +127,9 @@ class JDBCTableCatalog extends TableCatalog with Logging { options.parameters + (JDBCOptions.JDBC_TABLE_NAME -> getTableName(ident))) val caseSensitive = SQLConf.get.caseSensitiveAnalysis withConnection { conn => - JdbcUtils.createTable(conn, getTableName(ident), schema, caseSensitive, writeOptions) + classifyException(s"Failed table creation: $ident") { + JdbcUtils.createTable(conn, getTableName(ident), schema, caseSensitive, writeOptions) + } } JDBCTable(ident, schema, writeOptions) @@ -132,7 +138,9 @@ class JDBCTableCatalog extends TableCatalog with Logging { override def alterTable(ident: Identifier, changes: TableChange*): Table = { checkNamespace(ident.namespace()) withConnection { conn => - JdbcUtils.alterTable(conn, getTableName(ident), changes, options) + classifyException(s"Failed table altering: $ident") { + JdbcUtils.alterTable(conn, getTableName(ident), changes, options) + } loadTable(ident) } } @@ -156,4 +164,12 @@ class JDBCTableCatalog extends TableCatalog with Logging { private def getTableName(ident: Identifier): String = { (ident.namespace() :+ ident.name()).map(dialect.quoteIdentifier).mkString(".") } + + private def classifyException[T](message: String)(f: => T): T = { + try { + f + } catch { + case e: Throwable => throw dialect.classifyException(message, e) + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index 1f38128e98fa5..b0ddee0a6b336 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -69,7 +69,7 @@ case class OrcPartitionReaderFactory( private def pushDownPredicates(filePath: Path, conf: Configuration): Unit = { if (orcFilterPushDown) { - OrcUtils.readCatalystSchema(filePath, conf, ignoreCorruptFiles).map { fileSchema => + OrcUtils.readCatalystSchema(filePath, conf, ignoreCorruptFiles).foreach { fileSchema => OrcFilters.createFilter(fileSchema, filters).foreach { f => OrcInputFormat.setSearchArgument(conf, f, fileSchema.fieldNames) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala index 6d8d37022ea42..0c5fee20385e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala @@ -78,6 +78,7 @@ case class BroadcastExchangeExec( override lazy val metrics = Map( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "collectTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to collect"), "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build"), "broadcastTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to broadcast")) @@ -90,7 +91,8 @@ case class BroadcastExchangeExec( override def runtimeStatistics: Statistics = { val dataSize = metrics("dataSize").value - Statistics(dataSize) + val rowCount = metrics("numOutputRows").value + Statistics(dataSize, Some(rowCount)) } @transient @@ -114,6 +116,7 @@ case class BroadcastExchangeExec( val beforeCollect = System.nanoTime() // Use executeCollect/executeCollectIterator to avoid conversion to Scala types val (numRows, input) = child.executeCollectIterator() + longMetric("numOutputRows") += numRows if (numRows >= MAX_BROADCAST_TABLE_ROWS) { throw new SparkException( s"Cannot broadcast the table over $MAX_BROADCAST_TABLE_ROWS rows: $numRows rows") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index b176598ed8c2c..3641654b89b76 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -135,9 +135,14 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { leftKeys: IndexedSeq[Expression], rightKeys: IndexedSeq[Expression], expectedOrderOfKeys: Seq[Expression], - currentOrderOfKeys: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { + currentOrderOfKeys: Seq[Expression]): Option[(Seq[Expression], Seq[Expression])] = { if (expectedOrderOfKeys.size != currentOrderOfKeys.size) { - return (leftKeys, rightKeys) + return None + } + + // Check if the current order already satisfies the expected order. + if (expectedOrderOfKeys.zip(currentOrderOfKeys).forall(p => p._1.semanticEquals(p._2))) { + return Some(leftKeys, rightKeys) } // Build a lookup between an expression and the positions its holds in the current key seq. @@ -164,10 +169,10 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { rightKeysBuffer += rightKeys(index) case _ => // The expression cannot be found, or we have exhausted all indices for that expression. - return (leftKeys, rightKeys) + return None } } - (leftKeysBuffer.toSeq, rightKeysBuffer.toSeq) + Some(leftKeysBuffer.toSeq, rightKeysBuffer.toSeq) } private def reorderJoinKeys( @@ -176,19 +181,48 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { leftPartitioning: Partitioning, rightPartitioning: Partitioning): (Seq[Expression], Seq[Expression]) = { if (leftKeys.forall(_.deterministic) && rightKeys.forall(_.deterministic)) { - (leftPartitioning, rightPartitioning) match { - case (HashPartitioning(leftExpressions, _), _) => - reorder(leftKeys.toIndexedSeq, rightKeys.toIndexedSeq, leftExpressions, leftKeys) - case (_, HashPartitioning(rightExpressions, _)) => - reorder(leftKeys.toIndexedSeq, rightKeys.toIndexedSeq, rightExpressions, rightKeys) - case _ => - (leftKeys, rightKeys) - } + reorderJoinKeysRecursively( + leftKeys, + rightKeys, + Some(leftPartitioning), + Some(rightPartitioning)) + .getOrElse((leftKeys, rightKeys)) } else { (leftKeys, rightKeys) } } + /** + * Recursively reorders the join keys based on partitioning. It starts reordering the + * join keys to match HashPartitioning on either side, followed by PartitioningCollection. + */ + private def reorderJoinKeysRecursively( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + leftPartitioning: Option[Partitioning], + rightPartitioning: Option[Partitioning]): Option[(Seq[Expression], Seq[Expression])] = { + (leftPartitioning, rightPartitioning) match { + case (Some(HashPartitioning(leftExpressions, _)), _) => + reorder(leftKeys.toIndexedSeq, rightKeys.toIndexedSeq, leftExpressions, leftKeys) + .orElse(reorderJoinKeysRecursively( + leftKeys, rightKeys, None, rightPartitioning)) + case (_, Some(HashPartitioning(rightExpressions, _))) => + reorder(leftKeys.toIndexedSeq, rightKeys.toIndexedSeq, rightExpressions, rightKeys) + .orElse(reorderJoinKeysRecursively( + leftKeys, rightKeys, leftPartitioning, None)) + case (Some(PartitioningCollection(partitionings)), _) => + partitionings.foldLeft(Option.empty[(Seq[Expression], Seq[Expression])]) { (res, p) => + res.orElse(reorderJoinKeysRecursively(leftKeys, rightKeys, Some(p), rightPartitioning)) + }.orElse(reorderJoinKeysRecursively(leftKeys, rightKeys, None, rightPartitioning)) + case (_, Some(PartitioningCollection(partitionings))) => + partitionings.foldLeft(Option.empty[(Seq[Expression], Seq[Expression])]) { (res, p) => + res.orElse(reorderJoinKeysRecursively(leftKeys, rightKeys, leftPartitioning, Some(p))) + }.orElse(None) + case _ => + None + } + } + /** * When the physical operators are created for JOIN, the ordering of join keys is based on order * in which the join keys appear in the user query. That might not match with the output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index 097ea61f13832..6e59ad07d7168 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -288,7 +288,8 @@ case class SortMergeJoinExec( RowIterator.fromScala(rightIter), inMemoryThreshold, spillThreshold, - cleanupResources + cleanupResources, + condition.isEmpty ) private[this] val joinRow = new JoinedRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 03d86e42e4db7..42401fe069551 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -178,10 +178,16 @@ class FileStreamSource( if (batchFiles.nonEmpty) { metadataLogCurrentOffset += 1 - metadataLog.add(metadataLogCurrentOffset, batchFiles.map { case (p, timestamp) => + + val fileEntries = batchFiles.map { case (p, timestamp) => FileEntry(path = p, timestamp = timestamp, batchId = metadataLogCurrentOffset) - }.toArray) - logInfo(s"Log offset set to $metadataLogCurrentOffset with ${batchFiles.size} new files") + }.toArray + if (metadataLog.add(metadataLogCurrentOffset, fileEntries)) { + logInfo(s"Log offset set to $metadataLogCurrentOffset with ${batchFiles.size} new files") + } else { + throw new IllegalStateException("Concurrent update to the log. Multiple streaming jobs " + + s"detected for $metadataLogCurrentOffset") + } } FileStreamSourceOffset(metadataLogCurrentOffset) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index 468a8c975b478..c485d0f7d8b2d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relat import org.apache.spark.sql.execution.streaming.sources.WriteToMicroBatchDataSource import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{OutputMode, Trigger} -import org.apache.spark.util.Clock +import org.apache.spark.util.{Clock, Utils} class MicroBatchExecution( sparkSession: SparkSession, @@ -76,7 +76,7 @@ class MicroBatchExecution( // transformation is responsible for replacing attributes with their final values. val disabledSources = - sparkSession.sqlContext.conf.disabledV2StreamingMicroBatchReaders.split(",") + Utils.stringToSeq(sparkSession.sqlContext.conf.disabledV2StreamingMicroBatchReaders) import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ val _logicalPlan = analyzedPlan.transform { @@ -90,7 +90,7 @@ class MicroBatchExecution( StreamingExecutionRelation(source, output)(sparkSession) }) - case s @ StreamingRelationV2(src, srcName, table: SupportsRead, options, output, v1) => + case s @ StreamingRelationV2(src, srcName, table: SupportsRead, options, output, _, _, v1) => val dsStr = if (src.nonEmpty) s"[${src.get}]" else "" val v2Disabled = disabledSources.contains(src.getOrElse(None).getClass.getCanonicalName) if (!v2Disabled && table.supports(TableCapability.MICRO_BATCH_READ)) { @@ -598,7 +598,9 @@ class MicroBatchExecution( withProgressLocked { sinkCommitProgress = batchSinkProgress watermarkTracker.updateWatermark(lastExecution.executedPlan) - commitLog.add(currentBatchId, CommitMetadata(watermarkTracker.currentWatermark)) + assert(commitLog.add(currentBatchId, CommitMetadata(watermarkTracker.currentWatermark)), + "Concurrent update to the commit log. Multiple streaming jobs detected for " + + s"$currentBatchId") committedOffsets ++= availableOffsets } logDebug(s"Completed batch ${currentBatchId}") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 18fe38caa5e65..aba0463f56cd7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -44,7 +44,7 @@ import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.command.StreamingExplainCommand import org.apache.spark.sql.execution.datasources.v2.StreamWriterCommitProgress import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.connector.SupportsStreamingUpdate +import org.apache.spark.sql.internal.connector.SupportsStreamingUpdateAsAppend import org.apache.spark.sql.streaming._ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.{Clock, UninterruptibleThread, Utils} @@ -630,9 +630,9 @@ abstract class StreamExecution( writeBuilder.asInstanceOf[SupportsTruncate].truncate().buildForStreaming() case Update => - require(writeBuilder.isInstanceOf[SupportsStreamingUpdate], + require(writeBuilder.isInstanceOf[SupportsStreamingUpdateAsAppend], table.name + " does not support Update mode.") - writeBuilder.asInstanceOf[SupportsStreamingUpdate].update().buildForStreaming() + writeBuilder.asInstanceOf[SupportsStreamingUpdateAsAppend].buildForStreaming() } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala index 3d071df493cec..a52f5f4ac94ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala @@ -56,8 +56,8 @@ import org.apache.spark.util.{CompletionIterator, SerializableConfiguration} * - Apply the optional condition to filter the joined rows as the final output. * * If a timestamp column with event time watermark is present in the join keys or in the input - * data, then the it uses the watermark figure out which rows in the buffer will not join with - * and the new data, and therefore can be discarded. Depending on the provided query conditions, we + * data, then it uses the watermark to figure out which rows in the buffer will not join with + * the new data, and therefore can be discarded. Depending on the provided query conditions, we * can define thresholds on both state key (i.e. joining keys) and state value (i.e. input rows). * There are three kinds of queries possible regarding this as explained below. * Assume that watermark has been defined on both `leftTime` and `rightTime` columns used below. @@ -134,7 +134,7 @@ case class StreamingSymmetricHashJoinExec( stateWatermarkPredicates: JoinStateWatermarkPredicates, stateFormatVersion: Int, left: SparkPlan, - right: SparkPlan) extends SparkPlan with BinaryExecNode with StateStoreWriter { + right: SparkPlan) extends BinaryExecNode with StateStoreWriter { def this( leftKeys: Seq[Expression], @@ -157,14 +157,16 @@ case class StreamingSymmetricHashJoinExec( " the checkpoint and rerun the query. See SPARK-26154 for more details.") } + private lazy val errorMessageForJoinType = + s"${getClass.getSimpleName} should not take $joinType as the JoinType" + private def throwBadJoinTypeException(): Nothing = { - throw new IllegalArgumentException( - s"${getClass.getSimpleName} should not take $joinType as the JoinType") + throw new IllegalArgumentException(errorMessageForJoinType) } require( joinType == Inner || joinType == LeftOuter || joinType == RightOuter, - s"${getClass.getSimpleName} should not take $joinType as the JoinType") + errorMessageForJoinType) require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType)) private val storeConf = new StateStoreConf(sqlContext.conf) @@ -189,11 +191,9 @@ case class StreamingSymmetricHashJoinExec( override def outputPartitioning: Partitioning = joinType match { case _: InnerLike => PartitioningCollection(Seq(left.outputPartitioning, right.outputPartitioning)) - case LeftOuter => PartitioningCollection(Seq(left.outputPartitioning)) - case RightOuter => PartitioningCollection(Seq(right.outputPartitioning)) - case x => - throw new IllegalArgumentException( - s"${getClass.getSimpleName} should not take $x as the JoinType") + case LeftOuter => left.outputPartitioning + case RightOuter => right.outputPartitioning + case _ => throwBadJoinTypeException() } override def shouldRunAnotherBatch(newMetadata: OffsetSeqMetadata): Boolean = { @@ -246,13 +246,14 @@ case class StreamingSymmetricHashJoinExec( // Join one side input using the other side's buffered/state rows. Here is how it is done. // - // - `leftJoiner.joinWith(rightJoiner)` generates all rows from matching new left input with - // stored right input, and also stores all the left input + // - `leftSideJoiner.storeAndJoinWithOtherSide(rightSideJoiner)` generates all rows from + // matching new left input with stored right input, and also stores all the left input // - // - `rightJoiner.joinWith(leftJoiner)` generates all rows from matching new right input with - // stored left input, and also stores all the right input. It also generates all rows from - // matching new left input with new right input, since the new left input has become stored - // by that point. This tiny asymmetry is necessary to avoid duplication. + // - `rightSideJoiner.storeAndJoinWithOtherSide(leftSideJoiner)` generates all rows from + // matching new right input with stored left input, and also stores all the right input. + // It also generates all rows from matching new left input with new right input, since + // the new left input has become stored by that point. This tiny asymmetry is necessary + // to avoid duplication. val leftOutputIter = leftSideJoiner.storeAndJoinWithOtherSide(rightSideJoiner) { (input: InternalRow, matched: InternalRow) => joinedRow.withLeft(input).withRight(matched) } @@ -459,8 +460,9 @@ case class StreamingSymmetricHashJoinExec( */ def storeAndJoinWithOtherSide( otherSideJoiner: OneSideHashJoiner)( - generateJoinedRow: (InternalRow, InternalRow) => JoinedRow): - Iterator[InternalRow] = { + generateJoinedRow: (InternalRow, InternalRow) => JoinedRow) + : Iterator[InternalRow] = { + val watermarkAttribute = inputAttributes.find(_.metadata.contains(delayKey)) val nonLateRows = WatermarkSupport.watermarkExpression(watermarkAttribute, eventTimeWatermark) match { @@ -471,6 +473,14 @@ case class StreamingSymmetricHashJoinExec( inputIter } + val generateFilteredJoinedRow: InternalRow => Iterator[InternalRow] = joinSide match { + case LeftSide if joinType == LeftOuter => + (row: InternalRow) => Iterator(generateJoinedRow(row, nullRight)) + case RightSide if joinType == RightOuter => + (row: InternalRow) => Iterator(generateJoinedRow(row, nullLeft)) + case _ => (_: InternalRow) => Iterator.empty + } + nonLateRows.flatMap { row => val thisRow = row.asInstanceOf[UnsafeRow] // If this row fails the pre join filter, that means it can never satisfy the full join @@ -483,13 +493,7 @@ case class StreamingSymmetricHashJoinExec( .getJoinedRows(key, thatRow => generateJoinedRow(thisRow, thatRow), postJoinFilter) new AddingProcessedRowToStateCompletionIterator(key, thisRow, outputIter) } else { - joinSide match { - case LeftSide if joinType == LeftOuter => - Iterator(generateJoinedRow(thisRow, nullRight)) - case RightSide if joinType == RightOuter => - Iterator(generateJoinedRow(thisRow, nullLeft)) - case _ => Iterator() - } + generateFilteredJoinedRow(thisRow) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala index 1e64021c8105e..fb316598a128c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapabi import org.apache.spark.sql.connector.write.{LogicalWriteInfo, SupportsTruncate, WriteBuilder} import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.streaming.sources.ConsoleWrite -import org.apache.spark.sql.internal.connector.{SimpleTableProvider, SupportsStreamingUpdate} +import org.apache.spark.sql.internal.connector.{SimpleTableProvider, SupportsStreamingUpdateAsAppend} import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -73,12 +73,11 @@ object ConsoleTable extends Table with SupportsWrite { } override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { - new WriteBuilder with SupportsTruncate with SupportsStreamingUpdate { + new WriteBuilder with SupportsTruncate with SupportsStreamingUpdateAsAppend { private val inputSchema: StructType = info.schema() - // Do nothing for truncate/update. Console sink is special and it just prints all the records. + // Do nothing for truncate. Console sink is special and it just prints all the records. override def truncate(): WriteBuilder = this - override def update(): WriteBuilder = this override def buildForStreaming(): StreamingWrite = { assert(inputSchema != null) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index 12198f735c4c3..6eb28d4c66ded 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -65,7 +65,7 @@ class ContinuousExecution( var nextSourceId = 0 import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ val _logicalPlan = analyzedPlan.transform { - case s @ StreamingRelationV2(ds, sourceName, table: SupportsRead, options, output, _) => + case s @ StreamingRelationV2(ds, sourceName, table: SupportsRead, options, output, _, _, _) => val dsStr = if (ds.nonEmpty) s"[${ds.get}]" else "" if (!table.supports(TableCapability.CONTINUOUS_READ)) { throw new UnsupportedOperationException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index c6ba0da6ef04d..ee1cb127a3bc5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -83,6 +83,8 @@ abstract class MemoryStreamBase[A : Encoder](sqlContext: SQLContext) extends Spa new MemoryStreamTable(this), CaseInsensitiveStringMap.empty(), attributes, + None, + None, None) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala index 57a73c740310e..09abc51c697b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapabi import org.apache.spark.sql.connector.write.{DataWriter, LogicalWriteInfo, PhysicalWriteInfo, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.execution.python.PythonForeachWriter -import org.apache.spark.sql.internal.connector.SupportsStreamingUpdate +import org.apache.spark.sql.internal.connector.SupportsStreamingUpdateAsAppend import org.apache.spark.sql.types.StructType /** @@ -55,13 +55,12 @@ case class ForeachWriterTable[T]( } override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { - new WriteBuilder with SupportsTruncate with SupportsStreamingUpdate { + new WriteBuilder with SupportsTruncate with SupportsStreamingUpdateAsAppend { private var inputSchema: StructType = info.schema() - // Do nothing for truncate/update. Foreach sink is special and it just forwards all the + // Do nothing for truncate. Foreach sink is special and it just forwards all the // records to ForeachWriter. override def truncate(): WriteBuilder = this - override def update(): WriteBuilder = this override def buildForStreaming(): StreamingWrite = { new StreamingWrite { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala index 24ff9c2e8384d..a6ac6f2da8e41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapabi import org.apache.spark.sql.connector.write.{DataWriter, DataWriterFactory, LogicalWriteInfo, PhysicalWriteInfo, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.execution.streaming.Sink -import org.apache.spark.sql.internal.connector.SupportsStreamingUpdate +import org.apache.spark.sql.internal.connector.SupportsStreamingUpdateAsAppend import org.apache.spark.sql.types.StructType /** @@ -54,7 +54,7 @@ class MemorySink extends Table with SupportsWrite with Logging { } override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { - new WriteBuilder with SupportsTruncate with SupportsStreamingUpdate { + new WriteBuilder with SupportsTruncate with SupportsStreamingUpdateAsAppend { private var needTruncate: Boolean = false private val inputSchema: StructType = info.schema() @@ -63,9 +63,6 @@ class MemorySink extends Table with SupportsWrite with Logging { this } - // The in-memory sink treats update as append. - override def update(): WriteBuilder = this - override def buildForStreaming(): StreamingWrite = { new MemoryStreamingWrite(MemorySink.this, inputSchema, needTruncate) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala index 1a5b50dcc7901..2aa2a18b9eaf4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala @@ -171,7 +171,7 @@ class SymmetricHashJoinStateManager( return null } - override def close: Unit = {} + override def close(): Unit = {} } } @@ -280,7 +280,7 @@ class SymmetricHashJoinStateManager( return reusedRet.withNew(currentKey, currentValue.value, currentValue.matched) } - override def close: Unit = {} + override def close(): Unit = {} } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index b56c0792f1aa4..14cc76f0dbb78 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{AttributeSeq, CreateNamedStruct, Expression, ExprId, InSet, ListQuery, Literal, PlanExpression} +import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, Expression, ExprId, InSet, ListQuery, Literal, PlanExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf @@ -80,7 +80,8 @@ case class ScalarSubquery( @volatile private var updated: Boolean = false def updateResult(): Unit = { - val rows = plan.executeCollect() + // Only return the first two rows as an array to avoid Driver OOM. + val rows = plan.executeTake(2) if (rows.length > 1) { sys.error(s"more than one row returned by a subquery used as an expression:\n$plan") } @@ -114,10 +115,10 @@ case class InSubqueryExec( child: Expression, plan: BaseSubqueryExec, exprId: ExprId, - private var resultBroadcast: Broadcast[Set[Any]] = null) extends ExecSubqueryExpression { + private var resultBroadcast: Broadcast[Array[Any]] = null) extends ExecSubqueryExpression { - @transient private var result: Set[Any] = _ - @transient private lazy val inSet = InSet(child, result) + @transient private var result: Array[Any] = _ + @transient private lazy val inSet = InSet(child, result.toSet) override def dataType: DataType = BooleanType override def children: Seq[Expression] = child :: Nil @@ -133,14 +134,14 @@ case class InSubqueryExec( def updateResult(): Unit = { val rows = plan.executeCollect() result = if (plan.output.length > 1) { - rows.toSet + rows.asInstanceOf[Array[Any]] } else { - rows.map(_.get(0, child.dataType)).toSet + rows.map(_.get(0, child.dataType)) } resultBroadcast = plan.sqlContext.sparkContext.broadcast(result) } - def values(): Option[Set[Any]] = Option(resultBroadcast).map(_.value) + def values(): Option[Array[Any]] = Option(resultBroadcast).map(_.value) private def prepareResult(): Unit = { require(resultBroadcast != null, s"$this has not finished") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala index 175340d2dfaa7..963aec7ca36c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala @@ -341,7 +341,7 @@ class SQLAppStatusListener( val exec = getOrCreateExecution(executionId) exec.physicalPlanDescription = physicalPlanDescription - exec.metrics = sqlPlanMetrics + exec.metrics ++= sqlPlanMetrics update(exec) } @@ -349,7 +349,7 @@ class SQLAppStatusListener( val SparkListenerSQLAdaptiveSQLMetricUpdates(executionId, sqlPlanMetrics) = event val exec = getOrCreateExecution(executionId) - exec.metrics = exec.metrics ++ sqlPlanMetrics + exec.metrics ++= sqlPlanMetrics update(exec) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 6201492d04b0c..21e22d90f0f80 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -684,8 +684,9 @@ object functions { def min(columnName: String): Column = min(Column(columnName)) /** - * Aggregate function: returns and array of the approximate percentile values - * of numeric column col at the given percentages. + * Aggregate function: returns the approximate `percentile` of the numeric column `col` which + * is the smallest value in the ordered `col` values (sorted from least to greatest) such that + * no more than `percentage` of `col` values is less than the value or equal to that value. * * If percentage is an array, each value must be between 0.0 and 1.0. * If it is a single floating point value, it must be between 0.0 and 1.0. @@ -993,6 +994,35 @@ object functions { Lead(e.expr, Literal(offset), Literal(defaultValue)) } + /** + * Window function: returns the value that is the `offset`th row of the window frame + * (counting from 1), and `null` if the size of window frame is less than `offset` rows. + * + * It will return the `offset`th non-null value it sees when ignoreNulls is set to true. + * If all values are null, then null is returned. + * + * This is equivalent to the nth_value function in SQL. + * + * @group window_funcs + * @since 3.1.0 + */ + def nth_value(e: Column, offset: Int, ignoreNulls: Boolean): Column = withExpr { + NthValue(e.expr, Literal(offset), ignoreNulls) + } + + /** + * Window function: returns the value that is the `offset`th row of the window frame + * (counting from 1), and `null` if the size of window frame is less than `offset` rows. + * + * This is equivalent to the nth_value function in SQL. + * + * @group window_funcs + * @since 3.1.0 + */ + def nth_value(e: Column, offset: Int): Column = withExpr { + NthValue(e.expr, Literal(offset), false) + } + /** * Window function: returns the ntile group id (from 1 to `n` inclusive) in an ordered window * partition. For example, if `n` is 4, the first quarter of the rows will get value 1, the second @@ -1397,6 +1427,22 @@ object functions { */ def acos(columnName: String): Column = acos(Column(columnName)) + /** + * @return inverse hyperbolic cosine of `e` + * + * @group math_funcs + * @since 3.1.0 + */ + def acosh(e: Column): Column = withExpr { Acosh(e.expr) } + + /** + * @return inverse hyperbolic cosine of `columnName` + * + * @group math_funcs + * @since 3.1.0 + */ + def acosh(columnName: String): Column = acosh(Column(columnName)) + /** * @return inverse sine of `e` in radians, as if computed by `java.lang.Math.asin` * @@ -1414,7 +1460,23 @@ object functions { def asin(columnName: String): Column = asin(Column(columnName)) /** - * @return inverse tangent of `e`, as if computed by `java.lang.Math.atan` + * @return inverse hyperbolic sine of `e` + * + * @group math_funcs + * @since 3.1.0 + */ + def asinh(e: Column): Column = withExpr { Asinh(e.expr) } + + /** + * @return inverse hyperbolic sine of `columnName` + * + * @group math_funcs + * @since 3.1.0 + */ + def asinh(columnName: String): Column = asinh(Column(columnName)) + + /** + * @return inverse tangent of `e` as if computed by `java.lang.Math.atan` * * @group math_funcs * @since 1.4.0 @@ -1542,6 +1604,22 @@ object functions { */ def atan2(yValue: Double, xName: String): Column = atan2(yValue, Column(xName)) + /** + * @return inverse hyperbolic tangent of `e` + * + * @group math_funcs + * @since 3.1.0 + */ + def atanh(e: Column): Column = withExpr { Atanh(e.expr) } + + /** + * @return inverse hyperbolic tangent of `columnName` + * + * @group math_funcs + * @since 3.1.0 + */ + def atanh(columnName: String): Column = atanh(Column(columnName)) + /** * An expression that returns the string representation of the binary value of the given long * column. For example, bin("12") returns "1100". @@ -2288,6 +2366,36 @@ object functions { new XxHash64(cols.map(_.expr)) } + /** + * Returns null if the condition is true, and throws an exception otherwise. + * + * @group misc_funcs + * @since 3.1.0 + */ + def assert_true(c: Column): Column = withExpr { + new AssertTrue(c.expr) + } + + /** + * Returns null if the condition is true; throws an exception with the error message otherwise. + * + * @group misc_funcs + * @since 3.1.0 + */ + def assert_true(c: Column, e: Column): Column = withExpr { + new AssertTrue(c.expr, e.expr) + } + + /** + * Throws an exception with the provided error message. + * + * @group misc_funcs + * @since 3.1.0 + */ + def raise_error(c: Column): Column = withExpr { + RaiseError(c.expr) + } + ////////////////////////////////////////////////////////////////////////////////////////////// // String functions ////////////////////////////////////////////////////////////////////////////////////////////// @@ -2730,7 +2838,8 @@ object functions { } /** - * Returns the current date as a date column. + * Returns the current date at the start of query evaluation as a date column. + * All calls of current_date within the same query return the same value. * * @group datetime_funcs * @since 1.5.0 @@ -2738,7 +2847,8 @@ object functions { def current_date(): Column = withExpr { CurrentDate() } /** - * Returns the current timestamp as a timestamp column. + * Returns the current timestamp at the start of query evaluation as a timestamp column. + * All calls of current_timestamp within the same query return the same value. * * @group datetime_funcs * @since 1.5.0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 83a7a557305e9..4ca1ac863addc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -57,7 +57,8 @@ import org.apache.spark.sql.util.ExecutionListenerManager @Unstable abstract class BaseSessionStateBuilder( val session: SparkSession, - val parentState: Option[SessionState] = None) { + val parentState: Option[SessionState], + val options: Map[String, String]) { type NewBuilder = (SparkSession, Option[SessionState]) => BaseSessionStateBuilder /** @@ -97,6 +98,9 @@ abstract class BaseSessionStateBuilder( }.getOrElse { val conf = new SQLConf mergeSparkConf(conf, session.sparkContext.conf) + options.foreach { + case (k, v) => conf.setConfString(k, v) + } conf } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index cd425b04ef311..0f9a89741c192 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -136,9 +136,10 @@ private[sql] object SessionState { @Unstable class SessionStateBuilder( session: SparkSession, - parentState: Option[SessionState] = None) - extends BaseSessionStateBuilder(session, parentState) { - override protected def newBuilder: NewBuilder = new SessionStateBuilder(_, _) + parentState: Option[SessionState], + options: Map[String, String]) + extends BaseSessionStateBuilder(session, parentState, options) { + override protected def newBuilder: NewBuilder = new SessionStateBuilder(_, _, Map.empty) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala index 430ca9edab799..0b394db5c8932 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala @@ -58,4 +58,25 @@ private object DB2Dialect extends JdbcDialect { override def renameTable(oldTable: String, newTable: String): String = { s"RENAME TABLE $oldTable TO $newTable" } + + // scalastyle:off line.size.limit + // See https://www.ibm.com/support/knowledgecenter/en/SSEPGG_11.5.0/com.ibm.db2.luw.sql.ref.doc/doc/r0000888.html + // scalastyle:on line.size.limit + override def getUpdateColumnTypeQuery( + tableName: String, + columnName: String, + newDataType: String): String = + s"ALTER TABLE $tableName ALTER COLUMN ${quoteIdentifier(columnName)}" + + s" SET DATA TYPE $newDataType" + + // scalastyle:off line.size.limit + // See https://www.ibm.com/support/knowledgecenter/en/SSEPGG_11.5.0/com.ibm.db2.luw.sql.ref.doc/doc/r0000888.html + // scalastyle:on line.size.limit + override def getUpdateColumnNullabilityQuery( + tableName: String, + columnName: String, + isNullable: Boolean): String = { + val nullable = if (isNullable) "DROP NOT NULL" else "SET NOT NULL" + s"ALTER TABLE $tableName ALTER COLUMN ${quoteIdentifier(columnName)} $nullable" + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala new file mode 100644 index 0000000000000..9c727957ffab8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.sql.SQLException +import java.util.Locale + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} + +private object H2Dialect extends JdbcDialect { + override def canHandle(url: String): Boolean = + url.toLowerCase(Locale.ROOT).startsWith("jdbc:h2") + + override def classifyException(message: String, e: Throwable): AnalysisException = { + if (e.isInstanceOf[SQLException]) { + // Error codes are from https://www.h2database.com/javadoc/org/h2/api/ErrorCode.html + e.asInstanceOf[SQLException].getErrorCode match { + // TABLE_OR_VIEW_ALREADY_EXISTS_1 + case 42101 => + throw new TableAlreadyExistsException(message, cause = Some(e)) + // TABLE_OR_VIEW_NOT_FOUND_1 + case 42102 => + throw new NoSuchTableException(message, cause = Some(e)) + // SCHEMA_NOT_FOUND_1 + case 90079 => + throw new NoSuchNamespaceException(message, cause = Some(e)) + case _ => + } + } + super.classifyException(message, e) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcConnectionProvider.scala new file mode 100644 index 0000000000000..1e8abca197025 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcConnectionProvider.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.sql.{Connection, Driver} + +import org.apache.spark.annotation.{DeveloperApi, Unstable} + +/** + * ::DeveloperApi:: + * Connection provider which opens connection toward various databases (database specific instance + * needed). If any authentication required then it's the provider's responsibility to set all + * the parameters. + * Important to mention connection providers within a JVM used from multiple threads so adding + * internal state is not advised. If any state added then it must be synchronized properly. + * + * @since 3.1.0 + */ +@DeveloperApi +@Unstable +abstract class JdbcConnectionProvider { + /** + * Name of the service to provide JDBC connections. This name should be unique. Spark will + * internally use this name to differentiate JDBC connection providers. + */ + val name: String + + /** + * Checks if this connection provider instance can handle the connection initiated by the driver. + * There must be exactly one active connection provider which can handle the connection for a + * specific driver. If this requirement doesn't met then `IllegalArgumentException` + * will be thrown by the provider framework. + * + * @param driver Java driver which initiates the connection + * @param options Driver options which initiates the connection + * @return True if the connection provider can handle the driver with the given options. + */ + def canHandle(driver: Driver, options: Map[String, String]): Boolean + + /** + * Opens connection toward the database. Since global JVM security configuration change may needed + * this API is called synchronized by `SecurityConfigurationLock` to avoid race. + * + * @param driver Java driver which initiates the connection + * @param options Driver options which initiates the connection + * @return a `Connection` object that represents a connection to the URL + */ + def getConnection(driver: Driver, options: Map[String, String]): Connection +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index cea5a20917532..e0703195051dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -24,6 +24,7 @@ import scala.collection.mutable.ArrayBuilder import org.apache.commons.lang3.StringUtils import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.connector.catalog.TableChange import org.apache.spark.sql.connector.catalog.TableChange._ import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils @@ -199,7 +200,6 @@ abstract class JdbcDialect extends Serializable { /** * Alter an existing table. - * TODO (SPARK-32523): Override this method in the dialects that have different syntax. * * @param tableName The name of the table to be altered. * @param changes Changes to apply to the table. @@ -212,28 +212,61 @@ abstract class JdbcDialect extends Serializable { case add: AddColumn if add.fieldNames.length == 1 => val dataType = JdbcUtils.getJdbcType(add.dataType(), this).databaseTypeDefinition val name = add.fieldNames - updateClause += s"ALTER TABLE $tableName ADD COLUMN ${name(0)} $dataType" + updateClause += getAddColumnQuery(tableName, name(0), dataType) case rename: RenameColumn if rename.fieldNames.length == 1 => val name = rename.fieldNames - updateClause += s"ALTER TABLE $tableName RENAME COLUMN ${name(0)} TO ${rename.newName}" + updateClause += getRenameColumnQuery(tableName, name(0), rename.newName) case delete: DeleteColumn if delete.fieldNames.length == 1 => val name = delete.fieldNames - updateClause += s"ALTER TABLE $tableName DROP COLUMN ${name(0)}" + updateClause += getDeleteColumnQuery(tableName, name(0)) case updateColumnType: UpdateColumnType if updateColumnType.fieldNames.length == 1 => val name = updateColumnType.fieldNames val dataType = JdbcUtils.getJdbcType(updateColumnType.newDataType(), this) .databaseTypeDefinition - updateClause += s"ALTER TABLE $tableName ALTER COLUMN ${name(0)} $dataType" + updateClause += getUpdateColumnTypeQuery(tableName, name(0), dataType) case updateNull: UpdateColumnNullability if updateNull.fieldNames.length == 1 => val name = updateNull.fieldNames - val nullable = if (updateNull.nullable()) "NULL" else "NOT NULL" - updateClause += s"ALTER TABLE $tableName ALTER COLUMN ${name(0)} SET $nullable" + updateClause += getUpdateColumnNullabilityQuery(tableName, name(0), updateNull.nullable()) case _ => throw new SQLFeatureNotSupportedException(s"Unsupported TableChange $change") } } updateClause.result() } + + def getAddColumnQuery(tableName: String, columnName: String, dataType: String): String = + s"ALTER TABLE $tableName ADD COLUMN ${quoteIdentifier(columnName)} $dataType" + + def getRenameColumnQuery(tableName: String, columnName: String, newName: String): String = + s"ALTER TABLE $tableName RENAME COLUMN ${quoteIdentifier(columnName)} TO" + + s" ${quoteIdentifier(newName)}" + + def getDeleteColumnQuery(tableName: String, columnName: String): String = + s"ALTER TABLE $tableName DROP COLUMN ${quoteIdentifier(columnName)}" + + def getUpdateColumnTypeQuery( + tableName: String, + columnName: String, + newDataType: String): String = + s"ALTER TABLE $tableName ALTER COLUMN ${quoteIdentifier(columnName)} $newDataType" + + def getUpdateColumnNullabilityQuery( + tableName: String, + columnName: String, + isNullable: Boolean): String = { + val nullable = if (isNullable) "NULL" else "NOT NULL" + s"ALTER TABLE $tableName ALTER COLUMN ${quoteIdentifier(columnName)} SET $nullable" + } + + /** + * Gets a dialect exception, classifies it and wraps it by `AnalysisException`. + * @param message The error message to be placed to the returned exception. + * @param e The dialect specific exception. + * @return `AnalysisException` or its sub-class. + */ + def classifyException(message: String, e: Throwable): AnalysisException = { + new AnalysisException(message, cause = Some(e)) + } } /** @@ -278,6 +311,7 @@ object JdbcDialects { registerDialect(DerbyDialect) registerDialect(OracleDialect) registerDialect(TeradataDialect) + registerDialect(H2Dialect) /** * Fetch the JdbcDialect class corresponding to a given database url. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala index 4c0623729e00d..491b6e29ecf2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala @@ -64,6 +64,12 @@ private case object OracleDialect extends JdbcDialect { => Some(TimestampType) // Value for Timestamp with Time Zone in Oracle case BINARY_FLOAT => Some(FloatType) // Value for OracleTypes.BINARY_FLOAT case BINARY_DOUBLE => Some(DoubleType) // Value for OracleTypes.BINARY_DOUBLE + // scalastyle:off line.size.limit + // According to the documentation for Oracle Database 19c: + // "Values of the ROWID pseudocolumn are strings representing the address of each row." + // https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/Data-Types.html#GUID-AEF1FE4C-2DE5-4BE7-BB53-83AD8F1E34EF + // scalastyle:on line.size.limit + case Types.ROWID => Some(StringType) case _ => None } } @@ -112,4 +118,26 @@ private case object OracleDialect extends JdbcDialect { case _ => s"TRUNCATE TABLE $table" } } + + // see https://docs.oracle.com/cd/B28359_01/server.111/b28286/statements_3001.htm#SQLRF01001 + override def getAddColumnQuery( + tableName: String, + columnName: String, + dataType: String): String = + s"ALTER TABLE $tableName ADD ${quoteIdentifier(columnName)} $dataType" + + // see https://docs.oracle.com/cd/B28359_01/server.111/b28286/statements_3001.htm#SQLRF01001 + override def getUpdateColumnTypeQuery( + tableName: String, + columnName: String, + newDataType: String): String = + s"ALTER TABLE $tableName MODIFY ${quoteIdentifier(columnName)} $newDataType" + + override def getUpdateColumnNullabilityQuery( + tableName: String, + columnName: String, + isNullable: Boolean): String = { + val nullable = if (isNullable) "NULL" else "NOT NULL" + s"ALTER TABLE $tableName MODIFY ${quoteIdentifier(columnName)} $nullable" + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index 4064df0b04d12..9bc4acd49a980 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -24,7 +24,9 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.Evolving import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2 +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.catalog.{SupportsRead, TableProvider} import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.command.DDLUtils @@ -187,12 +189,20 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * * @since 2.0.0 */ - def load(): DataFrame = { + def load(): DataFrame = loadInternal(None) + + private def loadInternal(path: Option[String]): DataFrame = { if (source.toLowerCase(Locale.ROOT) == DDLUtils.HIVE_PROVIDER) { throw new AnalysisException("Hive data source can only be used with tables, you can not " + "read files of Hive data source directly.") } + val optionsWithPath = if (path.isEmpty) { + extraOptions + } else { + extraOptions + ("path" -> path.get) + } + val ds = DataSource.lookupDataSource(source, sparkSession.sqlContext.conf). getConstructor().newInstance() // We need to generate the V1 data source so we can pass it to the V2 relation as a shim. @@ -202,7 +212,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo sparkSession, userSpecifiedSchema = userSpecifiedSchema, className = source, - options = extraOptions.toMap) + options = optionsWithPath.originalMap) val v1Relation = ds match { case _: StreamSourceProvider => Some(StreamingRelation(v1DataSource)) case _ => None @@ -212,8 +222,9 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo case provider: TableProvider if !provider.isInstanceOf[FileDataSourceV2] => val sessionOptions = DataSourceV2Utils.extractSessionConfigs( source = provider, conf = sparkSession.sessionState.conf) - val options = sessionOptions ++ extraOptions - val dsOptions = new CaseInsensitiveStringMap(options.asJava) + val finalOptions = sessionOptions.filterKeys(!optionsWithPath.contains(_)).toMap ++ + optionsWithPath.originalMap + val dsOptions = new CaseInsensitiveStringMap(finalOptions.asJava) val table = DataSourceV2Utils.getTableFromProvider(provider, dsOptions, userSpecifiedSchema) import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ table match { @@ -221,7 +232,8 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo Dataset.ofRows( sparkSession, StreamingRelationV2( - Some(provider), source, table, dsOptions, table.schema.toAttributes, v1Relation)) + Some(provider), source, table, dsOptions, + table.schema.toAttributes, None, None, v1Relation)) // fallback to v1 // TODO (SPARK-27483): we should move this fallback logic to an analyzer rule. @@ -241,12 +253,12 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo */ def load(path: String): DataFrame = { if (!sparkSession.sessionState.conf.legacyPathOptionBehavior && - extraOptions.contains("path") && path.nonEmpty) { + extraOptions.contains("path")) { throw new AnalysisException("There is a 'path' option set and load() is called with a path" + "parameter. Either remove the path option, or call load() without the parameter. " + s"To ignore this check, set '${SQLConf.LEGACY_PATH_OPTION_BEHAVIOR.key}' to 'true'.") } - option("path", path).load() + loadInternal(Some(path)) } /** @@ -465,6 +477,23 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo format("parquet").load(path) } + /** + * Define a Streaming DataFrame on a Table. The DataSource corresponding to the table should + * support streaming mode. + * @param tableName The name of the table + * @since 3.1.0 + */ + def table(tableName: String): DataFrame = { + require(tableName != null, "The table name can't be null") + val identifier = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(tableName) + Dataset.ofRows( + sparkSession, + UnresolvedRelation( + identifier, + new CaseInsensitiveStringMap(extraOptions.toMap.asJava), + isStreaming = true)) + } + /** * Loads text files and returns a `DataFrame` whose schema starts with a string column named * "value", and followed by partitioned columns if there are any. @@ -535,5 +564,5 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo private var userSpecifiedSchema: Option[StructType] = None - private var extraOptions = new scala.collection.mutable.HashMap[String, String] + private var extraOptions = CaseInsensitiveMap[String](Map.empty) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index 45250c50a970e..2867bf581df81 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -26,7 +26,8 @@ import org.apache.spark.annotation.Evolving import org.apache.spark.api.java.function.VoidFunction2 import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes -import org.apache.spark.sql.connector.catalog.{SupportsWrite, TableProvider} +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableProvider} import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource @@ -35,6 +36,7 @@ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.Utils /** * Interface used to write a streaming `Dataset` to external storage systems (e.g. file systems, @@ -44,6 +46,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap */ @Evolving final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { + import DataStreamWriter._ private val df = ds.toDF() @@ -268,12 +271,12 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { */ def start(path: String): StreamingQuery = { if (!df.sparkSession.sessionState.conf.legacyPathOptionBehavior && - extraOptions.contains("path") && path.nonEmpty) { + extraOptions.contains("path")) { throw new AnalysisException("There is a 'path' option set and start() is called with a " + "path parameter. Either remove the path option, or call start() without the parameter. " + s"To ignore this check, set '${SQLConf.LEGACY_PATH_OPTION_BEHAVIOR.key}' to 'true'.") } - option("path", path).start() + startInternal(Some(path)) } /** @@ -291,103 +294,137 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * @since 2.0.0 */ @throws[TimeoutException] - def start(): StreamingQuery = { + def start(): StreamingQuery = startInternal(None) + + /** + * Starts the execution of the streaming query, which will continually output results to the given + * table as new data arrives. The returned [[StreamingQuery]] object can be used to interact with + * the stream. + * + * @since 3.1.0 + */ + @throws[TimeoutException] + def saveAsTable(tableName: String): StreamingQuery = { + this.source = SOURCE_NAME_TABLE + this.tableName = tableName + startInternal(None) + } + + private def startInternal(path: Option[String]): StreamingQuery = { if (source.toLowerCase(Locale.ROOT) == DDLUtils.HIVE_PROVIDER) { throw new AnalysisException("Hive data source can only be used with tables, you can not " + "write files of Hive data source directly.") } - if (source == "memory") { - assertNotPartitioned("memory") + if (source == SOURCE_NAME_TABLE) { + assertNotPartitioned(SOURCE_NAME_TABLE) + + import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + val originalMultipartIdentifier = df.sparkSession.sessionState.sqlParser + .parseMultipartIdentifier(tableName) + val CatalogAndIdentifier(catalog, identifier) = originalMultipartIdentifier + + // Currently we don't create a logical streaming writer node in logical plan, so cannot rely + // on analyzer to resolve it. Directly lookup only for temp view to provide clearer message. + // TODO (SPARK-27484): we should add the writing node before the plan is analyzed. + if (df.sparkSession.sessionState.catalog.isTempView(originalMultipartIdentifier)) { + throw new AnalysisException(s"Temporary view $tableName doesn't support streaming write") + } + + val tableInstance = catalog.asTableCatalog.loadTable(identifier) + + import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ + val sink = tableInstance match { + case t: SupportsWrite if t.supports(STREAMING_WRITE) => t + case t => throw new AnalysisException(s"Table $tableName doesn't support streaming " + + s"write - $t") + } + + startQuery(sink, extraOptions) + } else if (source == SOURCE_NAME_MEMORY) { + assertNotPartitioned(SOURCE_NAME_MEMORY) if (extraOptions.get("queryName").isEmpty) { throw new AnalysisException("queryName must be specified for memory sink") } val sink = new MemorySink() val resultDf = Dataset.ofRows(df.sparkSession, new MemoryPlan(sink, df.schema.toAttributes)) - val chkpointLoc = extraOptions.get("checkpointLocation") val recoverFromChkpoint = outputMode == OutputMode.Complete() - val query = df.sparkSession.sessionState.streamingQueryManager.startQuery( - extraOptions.get("queryName"), - chkpointLoc, - df, - extraOptions.toMap, - sink, - outputMode, - useTempCheckpointLocation = true, - recoverFromCheckpointLocation = recoverFromChkpoint, - trigger = trigger) + val query = startQuery(sink, extraOptions, recoverFromCheckpoint = recoverFromChkpoint) resultDf.createOrReplaceTempView(query.name) query - } else if (source == "foreach") { - assertNotPartitioned("foreach") + } else if (source == SOURCE_NAME_FOREACH) { + assertNotPartitioned(SOURCE_NAME_FOREACH) val sink = ForeachWriterTable[T](foreachWriter, ds.exprEnc) - df.sparkSession.sessionState.streamingQueryManager.startQuery( - extraOptions.get("queryName"), - extraOptions.get("checkpointLocation"), - df, - extraOptions.toMap, - sink, - outputMode, - useTempCheckpointLocation = true, - trigger = trigger) - } else if (source == "foreachBatch") { - assertNotPartitioned("foreachBatch") + startQuery(sink, extraOptions) + } else if (source == SOURCE_NAME_FOREACH_BATCH) { + assertNotPartitioned(SOURCE_NAME_FOREACH_BATCH) if (trigger.isInstanceOf[ContinuousTrigger]) { - throw new AnalysisException("'foreachBatch' is not supported with continuous trigger") + throw new AnalysisException(s"'$source' is not supported with continuous trigger") } val sink = new ForeachBatchSink[T](foreachBatchWriter, ds.exprEnc) - df.sparkSession.sessionState.streamingQueryManager.startQuery( - extraOptions.get("queryName"), - extraOptions.get("checkpointLocation"), - df, - extraOptions.toMap, - sink, - outputMode, - useTempCheckpointLocation = true, - trigger = trigger) + startQuery(sink, extraOptions) } else { val cls = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf) - val disabledSources = df.sparkSession.sqlContext.conf.disabledV2StreamingWriters.split(",") + val disabledSources = + Utils.stringToSeq(df.sparkSession.sqlContext.conf.disabledV2StreamingWriters) val useV1Source = disabledSources.contains(cls.getCanonicalName) || // file source v2 does not support streaming yet. classOf[FileDataSourceV2].isAssignableFrom(cls) + val optionsWithPath = if (path.isEmpty) { + extraOptions + } else { + extraOptions + ("path" -> path.get) + } + val sink = if (classOf[TableProvider].isAssignableFrom(cls) && !useV1Source) { val provider = cls.getConstructor().newInstance().asInstanceOf[TableProvider] val sessionOptions = DataSourceV2Utils.extractSessionConfigs( source = provider, conf = df.sparkSession.sessionState.conf) - val options = sessionOptions ++ extraOptions - val dsOptions = new CaseInsensitiveStringMap(options.asJava) + val finalOptions = sessionOptions.filterKeys(!optionsWithPath.contains(_)).toMap ++ + optionsWithPath.originalMap + val dsOptions = new CaseInsensitiveStringMap(finalOptions.asJava) val table = DataSourceV2Utils.getTableFromProvider( provider, dsOptions, userSpecifiedSchema = None) import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ table match { case table: SupportsWrite if table.supports(STREAMING_WRITE) => table - case _ => createV1Sink() + case _ => createV1Sink(optionsWithPath) } } else { - createV1Sink() + createV1Sink(optionsWithPath) } - df.sparkSession.sessionState.streamingQueryManager.startQuery( - extraOptions.get("queryName"), - extraOptions.get("checkpointLocation"), - df, - extraOptions.toMap, - sink, - outputMode, - useTempCheckpointLocation = source == "console" || source == "noop", - recoverFromCheckpointLocation = true, - trigger = trigger) + startQuery(sink, optionsWithPath) } } - private def createV1Sink(): Sink = { + private def startQuery( + sink: Table, + newOptions: CaseInsensitiveMap[String], + recoverFromCheckpoint: Boolean = true): StreamingQuery = { + val useTempCheckpointLocation = SOURCES_ALLOW_ONE_TIME_QUERY.contains(source) + + df.sparkSession.sessionState.streamingQueryManager.startQuery( + newOptions.get("queryName"), + newOptions.get("checkpointLocation"), + df, + newOptions.originalMap, + sink, + outputMode, + useTempCheckpointLocation = useTempCheckpointLocation, + recoverFromCheckpointLocation = recoverFromCheckpoint, + trigger = trigger) + } + + private def createV1Sink(optionsWithPath: CaseInsensitiveMap[String]): Sink = { val ds = DataSource( df.sparkSession, className = source, - options = extraOptions.toMap, + options = optionsWithPath.originalMap, partitionColumns = normalizedParCols.getOrElse(Nil)) ds.createSink(outputMode) } @@ -399,7 +436,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * @since 2.0.0 */ def foreach(writer: ForeachWriter[T]): DataStreamWriter[T] = { - this.source = "foreach" + this.source = SOURCE_NAME_FOREACH this.foreachWriter = if (writer != null) { ds.sparkSession.sparkContext.clean(writer) } else { @@ -423,7 +460,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { */ @Evolving def foreachBatch(function: (Dataset[T], Long) => Unit): DataStreamWriter[T] = { - this.source = "foreachBatch" + this.source = SOURCE_NAME_FOREACH_BATCH if (function == null) throw new IllegalArgumentException("foreachBatch function cannot be null") this.foreachBatchWriter = function this @@ -475,11 +512,13 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { private var source: String = df.sparkSession.sessionState.conf.defaultDataSourceName + private var tableName: String = null + private var outputMode: OutputMode = OutputMode.Append private var trigger: Trigger = Trigger.ProcessingTime(0L) - private var extraOptions = new scala.collection.mutable.HashMap[String, String] + private var extraOptions = CaseInsensitiveMap[String](Map.empty) private var foreachWriter: ForeachWriter[T] = null @@ -487,3 +526,16 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { private var partitioningColumns: Option[Seq[String]] = None } + +object DataStreamWriter { + val SOURCE_NAME_MEMORY = "memory" + val SOURCE_NAME_FOREACH = "foreach" + val SOURCE_NAME_FOREACH_BATCH = "foreachBatch" + val SOURCE_NAME_CONSOLE = "console" + val SOURCE_NAME_TABLE = "table" + val SOURCE_NAME_NOOP = "noop" + + // these writer sources are also used for one-time query, hence allow temp checkpoint location + val SOURCES_ALLOW_ONE_TIME_QUERY = Seq(SOURCE_NAME_MEMORY, SOURCE_NAME_FOREACH, + SOURCE_NAME_FOREACH_BATCH, SOURCE_NAME_CONSOLE, SOURCE_NAME_NOOP) +} diff --git a/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.jdbc.JdbcConnectionProvider b/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.jdbc.JdbcConnectionProvider new file mode 100644 index 0000000000000..afb48e1a3511f --- /dev/null +++ b/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.jdbc.JdbcConnectionProvider @@ -0,0 +1 @@ +org.apache.spark.sql.execution.datasources.jdbc.connection.IntentionallyFaultyConnectionProvider \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index 855ba3f00a4e6..1675fb1cc7c62 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -1,8 +1,8 @@ ## Summary - - Number of queries: 339 - - Number of expressions that missing example: 34 - - Expressions missing examples: and,bigint,binary,boolean,date,decimal,double,float,int,smallint,string,timestamp,tinyint,struct,cume_dist,dense_rank,input_file_block_length,input_file_block_start,input_file_name,lag,lead,monotonically_increasing_id,ntile,!,not,or,percent_rank,rank,row_number,spark_partition_id,version,window,positive,count_min_sketch + - Number of queries: 341 + - Number of expressions that missing example: 13 + - Expressions missing examples: bigint,binary,boolean,date,decimal,double,float,int,smallint,string,timestamp,tinyint,window ## Schema of Built-in Functions | Class name | Function name or alias | Query example | Output schema | | ---------- | ---------------------- | ------------- | ------------- | @@ -11,7 +11,7 @@ | org.apache.spark.sql.catalyst.expressions.Acosh | acosh | SELECT acosh(1) | struct | | org.apache.spark.sql.catalyst.expressions.Add | + | SELECT 1 + 2 | struct<(1 + 2):int> | | org.apache.spark.sql.catalyst.expressions.AddMonths | add_months | SELECT add_months('2016-08-31', 1) | struct | -| org.apache.spark.sql.catalyst.expressions.And | and | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.And | and | SELECT true and true | struct<(true AND true):boolean> | | org.apache.spark.sql.catalyst.expressions.ArrayAggregate | aggregate | SELECT aggregate(array(1, 2, 3), 0, (acc, x) -> acc + x) | struct | | org.apache.spark.sql.catalyst.expressions.ArrayContains | array_contains | SELECT array_contains(array(1, 2, 3), 2) | struct | | org.apache.spark.sql.catalyst.expressions.ArrayDistinct | array_distinct | SELECT array_distinct(array(1, 2, 3, null, 3)) | struct> | @@ -34,7 +34,7 @@ | org.apache.spark.sql.catalyst.expressions.Ascii | ascii | SELECT ascii('222') | struct | | org.apache.spark.sql.catalyst.expressions.Asin | asin | SELECT asin(0) | struct | | org.apache.spark.sql.catalyst.expressions.Asinh | asinh | SELECT asinh(0) | struct | -| org.apache.spark.sql.catalyst.expressions.AssertTrue | assert_true | SELECT assert_true(0 < 1) | struct | +| org.apache.spark.sql.catalyst.expressions.AssertTrue | assert_true | SELECT assert_true(0 < 1) | struct | | org.apache.spark.sql.catalyst.expressions.Atan | atan | SELECT atan(0) | struct | | org.apache.spark.sql.catalyst.expressions.Atan2 | atan2 | SELECT atan2(0, 0) | struct | | org.apache.spark.sql.catalyst.expressions.Atanh | atanh | SELECT atanh(0) | struct | @@ -79,10 +79,10 @@ | org.apache.spark.sql.catalyst.expressions.CreateArray | array | SELECT array(1, 2, 3) | struct> | | org.apache.spark.sql.catalyst.expressions.CreateMap | map | SELECT map(1.0, '2', 3.0, '4') | struct> | | org.apache.spark.sql.catalyst.expressions.CreateNamedStruct | named_struct | SELECT named_struct("a", 1, "b", 2, "c", 3) | struct> | -| org.apache.spark.sql.catalyst.expressions.CreateNamedStruct | struct | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.CreateNamedStruct | struct | SELECT struct(1, 2, 3) | struct> | | org.apache.spark.sql.catalyst.expressions.CsvToStructs | from_csv | SELECT from_csv('1, 0.8', 'a INT, b DOUBLE') | struct> | | org.apache.spark.sql.catalyst.expressions.Cube | cube | SELECT name, age, count(*) FROM VALUES (2, 'Alice'), (5, 'Bob') people(age, name) GROUP BY cube(name, age) | struct | -| org.apache.spark.sql.catalyst.expressions.CumeDist | cume_dist | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.CumeDist | cume_dist | SELECT a, b, cume_dist() OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b) | struct | | org.apache.spark.sql.catalyst.expressions.CurrentCatalog | current_catalog | SELECT current_catalog() | struct | | org.apache.spark.sql.catalyst.expressions.CurrentDatabase | current_database | SELECT current_database() | struct | | org.apache.spark.sql.catalyst.expressions.CurrentDate | current_date | SELECT current_date() | struct | @@ -97,7 +97,7 @@ | org.apache.spark.sql.catalyst.expressions.DayOfWeek | dayofweek | SELECT dayofweek('2009-07-30') | struct | | org.apache.spark.sql.catalyst.expressions.DayOfYear | dayofyear | SELECT dayofyear('2016-04-09') | struct | | org.apache.spark.sql.catalyst.expressions.Decode | decode | SELECT decode(encode('abc', 'utf-8'), 'utf-8') | struct | -| org.apache.spark.sql.catalyst.expressions.DenseRank | dense_rank | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.DenseRank | dense_rank | SELECT a, b, dense_rank(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b) | struct | | org.apache.spark.sql.catalyst.expressions.Divide | / | SELECT 3 / 2 | struct<(CAST(3 AS DOUBLE) / CAST(2 AS DOUBLE)):double> | | org.apache.spark.sql.catalyst.expressions.ElementAt | element_at | SELECT element_at(array(1, 2, 3), 2) | struct | | org.apache.spark.sql.catalyst.expressions.Elt | elt | SELECT elt(1, 'scala', 'java') | struct | @@ -135,9 +135,9 @@ | org.apache.spark.sql.catalyst.expressions.InitCap | initcap | SELECT initcap('sPark sql') | struct | | org.apache.spark.sql.catalyst.expressions.Inline | inline | SELECT inline(array(struct(1, 'a'), struct(2, 'b'))) | struct | | org.apache.spark.sql.catalyst.expressions.Inline | inline_outer | SELECT inline_outer(array(struct(1, 'a'), struct(2, 'b'))) | struct | -| org.apache.spark.sql.catalyst.expressions.InputFileBlockLength | input_file_block_length | N/A | N/A | -| org.apache.spark.sql.catalyst.expressions.InputFileBlockStart | input_file_block_start | N/A | N/A | -| org.apache.spark.sql.catalyst.expressions.InputFileName | input_file_name | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.InputFileBlockLength | input_file_block_length | SELECT input_file_block_length() | struct | +| org.apache.spark.sql.catalyst.expressions.InputFileBlockStart | input_file_block_start | SELECT input_file_block_start() | struct | +| org.apache.spark.sql.catalyst.expressions.InputFileName | input_file_name | SELECT input_file_name() | struct | | org.apache.spark.sql.catalyst.expressions.IntegralDivide | div | SELECT 3 div 2 | struct<(CAST(3 AS BIGINT) div CAST(2 AS BIGINT)):bigint> | | org.apache.spark.sql.catalyst.expressions.IsNaN | isnan | SELECT isnan(cast('NaN' as double)) | struct | | org.apache.spark.sql.catalyst.expressions.IsNotNull | isnotnull | SELECT isnotnull(1) | struct<(1 IS NOT NULL):boolean> | @@ -145,9 +145,9 @@ | org.apache.spark.sql.catalyst.expressions.JsonObjectKeys | json_object_keys | SELECT json_object_keys('{}') | struct> | | org.apache.spark.sql.catalyst.expressions.JsonToStructs | from_json | SELECT from_json('{"a":1, "b":0.8}', 'a INT, b DOUBLE') | struct> | | org.apache.spark.sql.catalyst.expressions.JsonTuple | json_tuple | SELECT json_tuple('{"a":1, "b":2}', 'a', 'b') | struct | -| org.apache.spark.sql.catalyst.expressions.Lag | lag | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.Lag | lag | SELECT a, b, lag(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b) | struct | | org.apache.spark.sql.catalyst.expressions.LastDay | last_day | SELECT last_day('2009-01-12') | struct | -| org.apache.spark.sql.catalyst.expressions.Lead | lead | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.Lead | lead | SELECT a, b, lead(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b) | struct | | org.apache.spark.sql.catalyst.expressions.Least | least | SELECT least(10, 9, 2, 4, 3) | struct | | org.apache.spark.sql.catalyst.expressions.Left | left | SELECT left('Spark SQL', 3) | struct | | org.apache.spark.sql.catalyst.expressions.Length | char_length | SELECT char_length('Spark SQL ') | struct | @@ -180,27 +180,28 @@ | org.apache.spark.sql.catalyst.expressions.MicrosToTimestamp | timestamp_micros | SELECT timestamp_micros(1230219000123123) | struct | | org.apache.spark.sql.catalyst.expressions.MillisToTimestamp | timestamp_millis | SELECT timestamp_millis(1230219000123) | struct | | org.apache.spark.sql.catalyst.expressions.Minute | minute | SELECT minute('2009-07-30 12:58:59') | struct | -| org.apache.spark.sql.catalyst.expressions.MonotonicallyIncreasingID | monotonically_increasing_id | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.MonotonicallyIncreasingID | monotonically_increasing_id | SELECT monotonically_increasing_id() | struct | | org.apache.spark.sql.catalyst.expressions.Month | month | SELECT month('2016-07-30') | struct | | org.apache.spark.sql.catalyst.expressions.MonthsBetween | months_between | SELECT months_between('1997-02-28 10:30:00', '1996-10-30') | struct | | org.apache.spark.sql.catalyst.expressions.Multiply | * | SELECT 2 * 3 | struct<(2 * 3):int> | | org.apache.spark.sql.catalyst.expressions.Murmur3Hash | hash | SELECT hash('Spark', array(123), 2) | struct | -| org.apache.spark.sql.catalyst.expressions.NTile | ntile | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.NTile | ntile | SELECT a, b, ntile(2) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b) | struct | | org.apache.spark.sql.catalyst.expressions.NaNvl | nanvl | SELECT nanvl(cast('NaN' as double), 123) | struct | | org.apache.spark.sql.catalyst.expressions.NextDay | next_day | SELECT next_day('2015-01-14', 'TU') | struct | -| org.apache.spark.sql.catalyst.expressions.Not | ! | N/A | N/A | -| org.apache.spark.sql.catalyst.expressions.Not | not | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.Not | ! | SELECT ! true | struct<(NOT true):boolean> | +| org.apache.spark.sql.catalyst.expressions.Not | not | SELECT not true | struct<(NOT true):boolean> | | org.apache.spark.sql.catalyst.expressions.Now | now | SELECT now() | struct | +| org.apache.spark.sql.catalyst.expressions.NthValue | nth_value | SELECT a, b, nth_value(b, 2) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b) | struct | | org.apache.spark.sql.catalyst.expressions.NullIf | nullif | SELECT nullif(2, 2) | struct | | org.apache.spark.sql.catalyst.expressions.Nvl | nvl | SELECT nvl(NULL, array('2')) | struct> | | org.apache.spark.sql.catalyst.expressions.Nvl2 | nvl2 | SELECT nvl2(NULL, 2, 1) | struct | | org.apache.spark.sql.catalyst.expressions.OctetLength | octet_length | SELECT octet_length('Spark SQL') | struct | -| org.apache.spark.sql.catalyst.expressions.Or | or | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.Or | or | SELECT true or false | struct<(true OR false):boolean> | | org.apache.spark.sql.catalyst.expressions.Overlay | overlay | SELECT overlay('Spark SQL' PLACING '_' FROM 6) | struct | | org.apache.spark.sql.catalyst.expressions.ParseToDate | to_date | SELECT to_date('2009-07-30 04:17:52') | struct | | org.apache.spark.sql.catalyst.expressions.ParseToTimestamp | to_timestamp | SELECT to_timestamp('2016-12-31 00:12:00') | struct | | org.apache.spark.sql.catalyst.expressions.ParseUrl | parse_url | SELECT parse_url('http://spark.apache.org/path?query=1', 'HOST') | struct | -| org.apache.spark.sql.catalyst.expressions.PercentRank | percent_rank | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.PercentRank | percent_rank | SELECT a, b, percent_rank(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b) | struct | | org.apache.spark.sql.catalyst.expressions.Pi | pi | SELECT pi() | struct | | org.apache.spark.sql.catalyst.expressions.Pmod | pmod | SELECT pmod(10, 3) | struct | | org.apache.spark.sql.catalyst.expressions.PosExplode | posexplode | SELECT posexplode(array(10,20)) | struct | @@ -209,10 +210,11 @@ | org.apache.spark.sql.catalyst.expressions.Pow | power | SELECT power(2, 3) | struct | | org.apache.spark.sql.catalyst.expressions.Quarter | quarter | SELECT quarter('2016-08-31') | struct | | org.apache.spark.sql.catalyst.expressions.RLike | rlike | SELECT '%SystemDrive%\Users\John' rlike '%SystemDrive%\\Users.*' | struct<%SystemDrive%UsersJohn RLIKE %SystemDrive%\Users.*:boolean> | +| org.apache.spark.sql.catalyst.expressions.RaiseError | raise_error | SELECT raise_error('custom error message') | struct | | org.apache.spark.sql.catalyst.expressions.Rand | rand | SELECT rand() | struct | | org.apache.spark.sql.catalyst.expressions.Rand | random | SELECT random() | struct | | org.apache.spark.sql.catalyst.expressions.Randn | randn | SELECT randn() | struct | -| org.apache.spark.sql.catalyst.expressions.Rank | rank | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.Rank | rank | SELECT a, b, rank(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b) | struct | | org.apache.spark.sql.catalyst.expressions.RegExpExtract | regexp_extract | SELECT regexp_extract('100-200', '(\\d+)-(\\d+)', 1) | struct | | org.apache.spark.sql.catalyst.expressions.RegExpExtractAll | regexp_extract_all | SELECT regexp_extract_all('100-200, 300-400', '(\\d+)-(\\d+)', 1) | struct> | | org.apache.spark.sql.catalyst.expressions.RegExpReplace | regexp_replace | SELECT regexp_replace('100-200', '(\\d+)', 'num') | struct | @@ -223,7 +225,7 @@ | org.apache.spark.sql.catalyst.expressions.Rint | rint | SELECT rint(12.3456) | struct | | org.apache.spark.sql.catalyst.expressions.Rollup | rollup | SELECT name, age, count(*) FROM VALUES (2, 'Alice'), (5, 'Bob') people(age, name) GROUP BY rollup(name, age) | struct | | org.apache.spark.sql.catalyst.expressions.Round | round | SELECT round(2.5, 0) | struct | -| org.apache.spark.sql.catalyst.expressions.RowNumber | row_number | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.RowNumber | row_number | SELECT a, b, row_number() OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b) | struct | | org.apache.spark.sql.catalyst.expressions.SchemaOfCsv | schema_of_csv | SELECT schema_of_csv('1,abc') | struct | | org.apache.spark.sql.catalyst.expressions.SchemaOfJson | schema_of_json | SELECT schema_of_json('[{"col":0}]') | struct | | org.apache.spark.sql.catalyst.expressions.Second | second | SELECT second('2009-07-30 12:58:59') | struct | @@ -246,8 +248,8 @@ | org.apache.spark.sql.catalyst.expressions.Slice | slice | SELECT slice(array(1, 2, 3, 4), 2, 2) | struct> | | org.apache.spark.sql.catalyst.expressions.SortArray | sort_array | SELECT sort_array(array('b', 'd', null, 'c', 'a'), true) | struct> | | org.apache.spark.sql.catalyst.expressions.SoundEx | soundex | SELECT soundex('Miller') | struct | -| org.apache.spark.sql.catalyst.expressions.SparkPartitionID | spark_partition_id | N/A | N/A | -| org.apache.spark.sql.catalyst.expressions.SparkVersion | version | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.SparkPartitionID | spark_partition_id | SELECT spark_partition_id() | struct | +| org.apache.spark.sql.catalyst.expressions.SparkVersion | version | SELECT version() | struct | | org.apache.spark.sql.catalyst.expressions.Sqrt | sqrt | SELECT sqrt(4) | struct | | org.apache.spark.sql.catalyst.expressions.Stack | stack | SELECT stack(2, 1, 2, 3) | struct | | org.apache.spark.sql.catalyst.expressions.StringInstr | instr | SELECT instr('SparkSQL', 'SQL') | struct | @@ -284,7 +286,7 @@ | org.apache.spark.sql.catalyst.expressions.TypeOf | typeof | SELECT typeof(1) | struct | | org.apache.spark.sql.catalyst.expressions.UnBase64 | unbase64 | SELECT unbase64('U3BhcmsgU1FM') | struct | | org.apache.spark.sql.catalyst.expressions.UnaryMinus | negative | SELECT negative(1) | struct | -| org.apache.spark.sql.catalyst.expressions.UnaryPositive | positive | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.UnaryPositive | positive | SELECT positive(1) | struct<(+ 1):int> | | org.apache.spark.sql.catalyst.expressions.Unhex | unhex | SELECT decode(unhex('537061726B2053514C'), 'UTF-8') | struct | | org.apache.spark.sql.catalyst.expressions.UnixTimestamp | unix_timestamp | SELECT unix_timestamp() | struct | | org.apache.spark.sql.catalyst.expressions.Upper | ucase | SELECT ucase('SparkSql') | struct | @@ -296,8 +298,8 @@ | org.apache.spark.sql.catalyst.expressions.XxHash64 | xxhash64 | SELECT xxhash64('Spark', array(123), 2) | struct | | org.apache.spark.sql.catalyst.expressions.Year | year | SELECT year('2016-07-30') | struct | | org.apache.spark.sql.catalyst.expressions.ZipWith | zip_with | SELECT zip_with(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, x)) | struct>> | -| org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile | approx_percentile | SELECT approx_percentile(10.0, array(0.5, 0.4, 0.1), 100) | struct> | -| org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile | percentile_approx | SELECT percentile_approx(10.0, array(0.5, 0.4, 0.1), 100) | struct> | +| org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile | approx_percentile | SELECT approx_percentile(col, array(0.5, 0.4, 0.1), 100) FROM VALUES (0), (1), (2), (10) AS tab(col) | struct> | +| org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile | percentile_approx | SELECT percentile_approx(col, array(0.5, 0.4, 0.1), 100) FROM VALUES (0), (1), (2), (10) AS tab(col) | struct> | | org.apache.spark.sql.catalyst.expressions.aggregate.Average | avg | SELECT avg(col) FROM VALUES (1), (2), (3) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Average | mean | SELECT mean(col) FROM VALUES (1), (2), (3) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.BitAndAgg | bit_and | SELECT bit_and(col) FROM VALUES (3), (5) AS tab(col) | struct | @@ -313,7 +315,7 @@ | org.apache.spark.sql.catalyst.expressions.aggregate.Corr | corr | SELECT corr(c1, c2) FROM VALUES (3, 2), (3, 3), (6, 4) as tab(c1, c2) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Count | count | SELECT count(*) FROM VALUES (NULL), (5), (5), (20) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.CountIf | count_if | SELECT count_if(col % 2 = 0) FROM VALUES (NULL), (0), (1), (2), (3) AS tab(col) | struct | -| org.apache.spark.sql.catalyst.expressions.aggregate.CountMinSketchAgg | count_min_sketch | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.aggregate.CountMinSketchAgg | count_min_sketch | SELECT hex(count_min_sketch(col, 0.5d, 0.5d, 1)) FROM VALUES (1), (2), (1) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.CovPopulation | covar_pop | SELECT covar_pop(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.CovSample | covar_samp | SELECT covar_samp(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.First | first | SELECT first(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | diff --git a/sql/core/src/test/resources/sql-tests/inputs/count.sql b/sql/core/src/test/resources/sql-tests/inputs/count.sql index 9f9ee4a873d4f..203f04c589373 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/count.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/count.sql @@ -25,3 +25,13 @@ SELECT count(a, b), count(b, a), count(testData.*) FROM testData; SELECT count(DISTINCT a, b), count(DISTINCT b, a), count(DISTINCT *), count(DISTINCT testData.*) FROM testData; + +-- distinct count with multiple literals +SELECT count(DISTINCT 3,2); +SELECT count(DISTINCT 2), count(DISTINCT 2,3); +SELECT count(DISTINCT 2), count(DISTINCT 3,2); +SELECT count(DISTINCT a), count(DISTINCT 2,3) FROM testData; +SELECT count(DISTINCT a), count(DISTINCT 3,2) FROM testData; +SELECT count(DISTINCT a), count(DISTINCT 2), count(DISTINCT 2,3) FROM testData; +SELECT count(DISTINCT a), count(DISTINCT 2), count(DISTINCT 3,2) FROM testData; +SELECT count(distinct 0.8), percentile_approx(distinct a, 0.8) FROM testData; diff --git a/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql b/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql index f4afa2b77a9d7..7aef901da4fb5 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql @@ -1,3 +1,4 @@ --IMPORT explain.sql --SET spark.sql.adaptive.enabled=true +--SET spark.sql.maxMetadataStringLength = 500 diff --git a/sql/core/src/test/resources/sql-tests/inputs/explain.sql b/sql/core/src/test/resources/sql-tests/inputs/explain.sql index 80bf258704c70..fdff1b4eef941 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/explain.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/explain.sql @@ -1,5 +1,6 @@ --SET spark.sql.codegen.wholeStage = true --SET spark.sql.adaptive.enabled = false +--SET spark.sql.maxMetadataStringLength = 500 -- Test tables CREATE table explain_temp1 (key int, val int) USING PARQUET; @@ -9,6 +10,11 @@ CREATE table explain_temp4 (key int, val string) USING PARQUET; SET spark.sql.codegen.wholeStage = true; +-- distinct func +EXPLAIN EXTENDED + SELECT sum(distinct val) + FROM explain_temp1; + -- single table EXPLAIN FORMATTED SELECT key, max(val) diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql index 24d303621faea..e4193d845f2e2 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql @@ -43,6 +43,14 @@ SELECT SUM(salary), COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE hiredat SELECT COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData; SELECT COUNT(DISTINCT id) FILTER (WHERE true) FROM emp; SELECT COUNT(DISTINCT id) FILTER (WHERE false) FROM emp; +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id = 40) FROM emp; +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id = 40) FROM emp; +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id > 0) FROM emp; +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id > 0) FROM emp; +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id = 40) FROM emp; +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id = 40) FROM emp; +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id > 0) FROM emp; +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id > 0) FROM emp; -- Aggregate with filter and non-empty GroupBy expressions. SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData GROUP BY a; diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index fedf03d774e42..81e2204358bc9 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -166,3 +166,6 @@ SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L; SELECT count(*) FROM test_agg WHERE count(*) > 1L; SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L; SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1; + +-- Aggregate with multiple distinct decimal columns +SELECT AVG(DISTINCT decimal_col), SUM(DISTINCT decimal_col) FROM VALUES (CAST(1 AS DECIMAL(9, 0))) t(decimal_col); diff --git a/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql index cfa06aea82b04..0b7bf517f916b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql @@ -92,3 +92,6 @@ select transform_values(ys, (k, v) -> k + v) as v from nested; -- use non reversed keywords: all is non reversed only if !ansi select transform(ys, all -> all * all) as v from values (array(32, 97)) as t(ys); select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t(ys); + +-- SPARK-32819: Aggregate on nested string arrays +select aggregate(split('abcdefgh',''), array(array('')), (acc, x) -> array(array(x))); \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/interval.sql index e925c4508f630..c3e4748e76e3c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/interval.sql @@ -206,3 +206,7 @@ select interval '1.2'; select interval '- 2'; select interval '1 day -'; select interval '1 day 1'; + +select interval '1 day 2' day; +select interval 'interval 1' day; +select interval '-\t 1' day; diff --git a/sql/core/src/test/resources/sql-tests/inputs/misc-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/misc-functions.sql index 95f71925e9294..907ff33000d8e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/misc-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/misc-functions.sql @@ -8,3 +8,15 @@ select typeof(cast(1.0 as float)), typeof(1.0D), typeof(1.2); select typeof(date '1986-05-23'), typeof(timestamp '1986-05-23'), typeof(interval '23 days'); select typeof(x'ABCD'), typeof('SPARK'); select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', 'spark')); + +-- Spark-32793: Rewrite AssertTrue with RaiseError +SELECT assert_true(true), assert_true(boolean(1)); +SELECT assert_true(false); +SELECT assert_true(boolean(0)); +SELECT assert_true(null); +SELECT assert_true(boolean(null)); +SELECT assert_true(false, 'custom error message'); + +CREATE TEMPORARY VIEW tbl_misc AS SELECT * FROM (VALUES (1), (8), (2)) AS T(v); +SELECT raise_error('error message'); +SELECT if(v > 5, raise_error('too big: ' || v), v + 1) FROM tbl_misc; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql index 6e95aca7aff62..d12bee6e47223 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql @@ -95,7 +95,7 @@ SELECT last(ten) OVER (PARTITION BY four), ten, four FROM (SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s ORDER BY four, ten; --- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- [SPARK-30707] Lead/Lag window function throws AnalysisException without ORDER BY clause -- SELECT nth_value(ten, four + 1) OVER (PARTITION BY four), ten, four -- FROM (SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s; @@ -301,7 +301,7 @@ FROM tenk1 WHERE unique1 < 10; -- unique1, four -- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); --- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- [SPARK-30707] Lead/Lag window function throws AnalysisException without ORDER BY clause -- SELECT first_value(unique1) over w, -- nth_value(unique1, 2) over w AS nth_2, -- last_value(unique1) over w, unique1, four diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql index ba1acc9f56b4a..50c0bc3410312 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql @@ -105,7 +105,7 @@ FROM tenk1 WHERE unique1 < 10; -- select sum(salary) over (order by enroll_date range between '1 year' preceding and '1 year' following -- exclude ties), salary, enroll_date from empsalary; --- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- [SPARK-28310] ANSI SQL grammar support: first_value/last_value(expression, [RESPECT NULLS | IGNORE NULLS]) -- select first_value(salary) over(order by salary range between 1000 preceding and 1000 following), -- lead(salary) over(order by salary range between 1000 preceding and 1000 following), -- nth_value(salary, 1) over(order by salary range between 1000 preceding and 1000 following), @@ -116,7 +116,7 @@ FROM tenk1 WHERE unique1 < 10; -- lag(salary) over(order by salary range between 1000 preceding and 1000 following), -- salary from empsalary; --- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- [SPARK-28310] ANSI SQL grammar support: first_value/last_value(expression, [RESPECT NULLS | IGNORE NULLS]) -- select first_value(salary) over(order by salary range between 1000 following and 3000 following -- exclude current row), -- lead(salary) over(order by salary range between 1000 following and 3000 following exclude ties), diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part3.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part3.sql index f4b8454da0d82..6f33a07631f7a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part3.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part3.sql @@ -399,8 +399,7 @@ SELECT range(1, 100) OVER () FROM empsalary; SELECT ntile(0) OVER (ORDER BY ten), ten, four FROM tenk1; --- [SPARK-27951] ANSI SQL: NTH_VALUE function --- SELECT nth_value(four, 0) OVER (ORDER BY ten), ten, four FROM tenk1; +SELECT nth_value(four, 0) OVER (ORDER BY ten), ten, four FROM tenk1; -- filter diff --git a/sql/core/src/test/resources/sql-tests/inputs/window.sql b/sql/core/src/test/resources/sql-tests/inputs/window.sql index 72d812d6a4e49..5de6db210ce36 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/window.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/window.sql @@ -16,6 +16,26 @@ CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) AS testData(val, val_long, val_double, val_date, val_timestamp, cate); +CREATE OR REPLACE TEMPORARY VIEW basic_pays AS SELECT * FROM VALUES +('Diane Murphy','Accounting',8435), +('Mary Patterson','Accounting',9998), +('Jeff Firrelli','Accounting',8992), +('William Patterson','Accounting',8870), +('Gerard Bondur','Accounting',11472), +('Anthony Bow','Accounting',6627), +('Leslie Jennings','IT',8113), +('Leslie Thompson','IT',5186), +('Julie Firrelli','Sales',9181), +('Steve Patterson','Sales',9441), +('Foon Yue Tseng','Sales',6660), +('George Vanauf','Sales',10563), +('Loui Bondur','SCM',10449), +('Gerard Hernandez','SCM',6949), +('Pamela Castillo','SCM',11303), +('Larry Bott','SCM',11798), +('Barry Jones','SCM',10586) +AS basic_pays(employee_name, department, salary); + -- RowsBetween SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val ROWS CURRENT ROW) FROM testData ORDER BY cate, val; @@ -124,4 +144,76 @@ WINDOW w AS (PARTITION BY cate ORDER BY val); -- with filter predicate SELECT val, cate, count(val) FILTER (WHERE val > 1) OVER(PARTITION BY cate) -FROM testData ORDER BY cate, val; \ No newline at end of file +FROM testData ORDER BY cate, val; + +-- nth_value() over () +SELECT + employee_name, + salary, + nth_value(employee_name, 2) OVER (ORDER BY salary DESC) second_highest_salary +FROM + basic_pays +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + nth_value(employee_name, 2) OVER ( + ORDER BY salary DESC + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) second_highest_salary +FROM + basic_pays +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + nth_value(employee_name, 2) OVER ( + ORDER BY salary + RANGE BETWEEN 2000 PRECEDING AND 1000 FOLLOWING) second_highest_salary +FROM + basic_pays +ORDER BY salary; + +SELECT + employee_name, + salary, + nth_value(employee_name, 2) OVER ( + ORDER BY salary DESC + ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING) second_highest_salary +FROM + basic_pays +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + nth_value(employee_name, 2) OVER ( + ORDER BY salary DESC + RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) second_highest_salary +FROM + basic_pays +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + nth_value(employee_name, 2) OVER ( + ORDER BY salary DESC + RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) second_highest_salary +FROM + basic_pays +ORDER BY salary DESC; + +SELECT + employee_name, + department, + salary, + NTH_VALUE(employee_name, 2) OVER ( + PARTITION BY department + ORDER BY salary DESC + RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING + ) second_highest_salary +FROM + basic_pays +ORDER BY department; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out index 7bef1bad4507e..6d26fae73b11c 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 29 +-- Number of queries: 30 -- !query @@ -282,3 +282,11 @@ no viable alternative at input 'all'(line 1, pos 22) == SQL == select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t(ys) ----------------------^^^ + + +-- !query +select aggregate(split('abcdefgh',''), array(array('')), (acc, x) -> array(array(x))) +-- !query schema +struct>> +-- !query output +[[""]] diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 33d918bbeb94d..d6cf9433a06b7 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 107 +-- Number of queries: 110 -- !query @@ -1122,3 +1122,39 @@ Cannot parse the INTERVAL value: 1 day 1(line 1, pos 7) == SQL == select interval '1 day 1' -------^^^ + + +-- !query +select interval '1 day 2' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only use numbers in the interval value part for multiple unit value pairs interval form, but got invalid value: 1 day 2(line 1, pos 16) + +== SQL == +select interval '1 day 2' day +----------------^^^ + + +-- !query +select interval 'interval 1' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only use numbers in the interval value part for multiple unit value pairs interval form, but got invalid value: interval 1(line 1, pos 16) + +== SQL == +select interval 'interval 1' day +----------------^^^ + + +-- !query +select interval '-\t 1' day +-- !query schema +struct +-- !query output +-1 days diff --git a/sql/core/src/test/resources/sql-tests/results/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/cast.sql.out index d4872ca03199b..42d12b80be989 100644 --- a/sql/core/src/test/resources/sql-tests/results/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cast.sql.out @@ -269,6 +269,8 @@ Class: org.apache.spark.sql.catalyst.expressions.Cast Extended Usage: No example/argument for boolean. + Since: 2.0.1 + Function: boolean Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`. diff --git a/sql/core/src/test/resources/sql-tests/results/count.sql.out b/sql/core/src/test/resources/sql-tests/results/count.sql.out index 68a5114bb5859..c0cdd0d697538 100644 --- a/sql/core/src/test/resources/sql-tests/results/count.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/count.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 5 +-- Number of queries: 13 -- !query @@ -53,3 +53,67 @@ FROM testData struct -- !query output 3 3 3 3 + + +-- !query +SELECT count(DISTINCT 3,2) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT count(DISTINCT 2), count(DISTINCT 2,3) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT count(DISTINCT 2), count(DISTINCT 3,2) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT count(DISTINCT a), count(DISTINCT 2,3) FROM testData +-- !query schema +struct +-- !query output +2 1 + + +-- !query +SELECT count(DISTINCT a), count(DISTINCT 3,2) FROM testData +-- !query schema +struct +-- !query output +2 1 + + +-- !query +SELECT count(DISTINCT a), count(DISTINCT 2), count(DISTINCT 2,3) FROM testData +-- !query schema +struct +-- !query output +2 1 1 + + +-- !query +SELECT count(DISTINCT a), count(DISTINCT 2), count(DISTINCT 3,2) FROM testData +-- !query schema +struct +-- !query output +2 1 1 + + +-- !query +SELECT count(distinct 0.8), percentile_approx(distinct a, 0.8) FROM testData +-- !query schema +struct +-- !query output +1 2 \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out index 5a59ffa03880f..567e0eabe1805 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out @@ -42,6 +42,39 @@ struct spark.sql.codegen.wholeStage true +-- !query +EXPLAIN EXTENDED + SELECT sum(distinct val) + FROM explain_temp1 +-- !query schema +struct +-- !query output +== Parsed Logical Plan == +'Project [unresolvedalias('sum(distinct 'val), None)] ++- 'UnresolvedRelation [explain_temp1], [], false + +== Analyzed Logical Plan == +sum(DISTINCT val): bigint +Aggregate [sum(distinct cast(val#x as bigint)) AS sum(DISTINCT val)#xL] ++- SubqueryAlias spark_catalog.default.explain_temp1 + +- Relation[key#x,val#x] parquet + +== Optimized Logical Plan == +Aggregate [sum(distinct cast(val#x as bigint)) AS sum(DISTINCT val)#xL] ++- Project [val#x] + +- Relation[key#x,val#x] parquet + +== Physical Plan == +AdaptiveSparkPlan isFinalPlan=false ++- HashAggregate(keys=[], functions=[sum(distinct cast(val#x as bigint)#xL)], output=[sum(DISTINCT val)#xL]) + +- Exchange SinglePartition, true, [id=#x] + +- HashAggregate(keys=[], functions=[partial_sum(distinct cast(val#x as bigint)#xL)], output=[sum#xL]) + +- HashAggregate(keys=[cast(val#x as bigint)#xL], functions=[], output=[cast(val#x as bigint)#xL]) + +- Exchange hashpartitioning(cast(val#x as bigint)#xL, 4), true, [id=#x] + +- HashAggregate(keys=[cast(val#x as bigint) AS cast(val#x as bigint)#xL], functions=[], output=[cast(val#x as bigint)#xL]) + +- FileScan parquet default.explain_temp1[val#x] Batched: true, DataFilters: [], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp1], PartitionFilters: [], PushedFilters: [], ReadSchema: struct + + -- !query EXPLAIN FORMATTED SELECT key, max(val) @@ -54,16 +87,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (8) -+- == Current Plan == - Sort (7) - +- Exchange (6) - +- HashAggregate (5) - +- Exchange (4) - +- HashAggregate (3) - +- Filter (2) - +- Scan parquet default.explain_temp1 (1) -+- == Initial Plan == - Sort (7) ++- Sort (7) +- Exchange (6) +- HashAggregate (5) +- Exchange (4) @@ -126,16 +150,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (8) -+- == Current Plan == - Project (7) - +- Filter (6) - +- HashAggregate (5) - +- Exchange (4) - +- HashAggregate (3) - +- Filter (2) - +- Scan parquet default.explain_temp1 (1) -+- == Initial Plan == - Project (7) ++- Project (7) +- Filter (6) +- HashAggregate (5) +- Exchange (4) @@ -196,17 +211,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (9) -+- == Current Plan == - HashAggregate (8) - +- Exchange (7) - +- HashAggregate (6) - +- Union (5) - :- Filter (2) - : +- Scan parquet default.explain_temp1 (1) - +- Filter (4) - +- Scan parquet default.explain_temp1 (3) -+- == Initial Plan == - HashAggregate (8) ++- HashAggregate (8) +- Exchange (7) +- HashAggregate (6) +- Union (5) @@ -274,15 +279,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (7) -+- == Current Plan == - BroadcastHashJoin Inner BuildRight (6) - :- Filter (2) - : +- Scan parquet default.explain_temp1 (1) - +- BroadcastExchange (5) - +- Filter (4) - +- Scan parquet default.explain_temp2 (3) -+- == Initial Plan == - BroadcastHashJoin Inner BuildRight (6) ++- BroadcastHashJoin Inner BuildRight (6) :- Filter (2) : +- Scan parquet default.explain_temp1 (1) +- BroadcastExchange (5) @@ -337,14 +334,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (6) -+- == Current Plan == - BroadcastHashJoin LeftOuter BuildRight (5) - :- Scan parquet default.explain_temp1 (1) - +- BroadcastExchange (4) - +- Filter (3) - +- Scan parquet default.explain_temp2 (2) -+- == Initial Plan == - BroadcastHashJoin LeftOuter BuildRight (5) ++- BroadcastHashJoin LeftOuter BuildRight (5) :- Scan parquet default.explain_temp1 (1) +- BroadcastExchange (4) +- Filter (3) @@ -398,11 +388,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (3) -+- == Current Plan == - Filter (2) - +- Scan parquet default.explain_temp1 (1) -+- == Initial Plan == - Filter (2) ++- Filter (2) +- Scan parquet default.explain_temp1 (1) @@ -438,11 +424,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (3) -+- == Current Plan == - Filter (2) - +- Scan parquet default.explain_temp1 (1) -+- == Initial Plan == - Filter (2) ++- Filter (2) +- Scan parquet default.explain_temp1 (1) @@ -470,11 +452,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (3) -+- == Current Plan == - Project (2) - +- Scan parquet default.explain_temp1 (1) -+- == Initial Plan == - Project (2) ++- Project (2) +- Scan parquet default.explain_temp1 (1) @@ -506,15 +484,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (7) -+- == Current Plan == - BroadcastHashJoin Inner BuildRight (6) - :- Filter (2) - : +- Scan parquet default.explain_temp1 (1) - +- BroadcastExchange (5) - +- Filter (4) - +- Scan parquet default.explain_temp1 (3) -+- == Initial Plan == - BroadcastHashJoin Inner BuildRight (6) ++- BroadcastHashJoin Inner BuildRight (6) :- Filter (2) : +- Scan parquet default.explain_temp1 (1) +- BroadcastExchange (5) @@ -572,21 +542,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (13) -+- == Current Plan == - BroadcastHashJoin Inner BuildRight (12) - :- HashAggregate (5) - : +- Exchange (4) - : +- HashAggregate (3) - : +- Filter (2) - : +- Scan parquet default.explain_temp1 (1) - +- BroadcastExchange (11) - +- HashAggregate (10) - +- Exchange (9) - +- HashAggregate (8) - +- Filter (7) - +- Scan parquet default.explain_temp1 (6) -+- == Initial Plan == - BroadcastHashJoin Inner BuildRight (12) ++- BroadcastHashJoin Inner BuildRight (12) :- HashAggregate (5) : +- Exchange (4) : +- HashAggregate (3) @@ -693,7 +649,7 @@ Output: [] Arguments: `default`.`explain_view`, SELECT key, val FROM explain_temp1, false, false, PersistedView (3) UnresolvedRelation -Arguments: [explain_temp1], [] +Arguments: [explain_temp1], [], false (4) Project Arguments: ['key, 'val] @@ -710,13 +666,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (5) -+- == Current Plan == - HashAggregate (4) - +- Exchange (3) - +- HashAggregate (2) - +- Scan parquet default.explain_temp1 (1) -+- == Initial Plan == - HashAggregate (4) ++- HashAggregate (4) +- Exchange (3) +- HashAggregate (2) +- Scan parquet default.explain_temp1 (1) @@ -761,13 +711,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (5) -+- == Current Plan == - ObjectHashAggregate (4) - +- Exchange (3) - +- ObjectHashAggregate (2) - +- Scan parquet default.explain_temp4 (1) -+- == Initial Plan == - ObjectHashAggregate (4) ++- ObjectHashAggregate (4) +- Exchange (3) +- ObjectHashAggregate (2) +- Scan parquet default.explain_temp4 (1) @@ -812,15 +756,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (7) -+- == Current Plan == - SortAggregate (6) - +- Sort (5) - +- Exchange (4) - +- SortAggregate (3) - +- Sort (2) - +- Scan parquet default.explain_temp4 (1) -+- == Initial Plan == - SortAggregate (6) ++- SortAggregate (6) +- Sort (5) +- Exchange (4) +- SortAggregate (3) diff --git a/sql/core/src/test/resources/sql-tests/results/explain.sql.out b/sql/core/src/test/resources/sql-tests/results/explain.sql.out index f28c408407c3f..fcd69549f2c6e 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain.sql.out @@ -42,6 +42,39 @@ struct spark.sql.codegen.wholeStage true +-- !query +EXPLAIN EXTENDED + SELECT sum(distinct val) + FROM explain_temp1 +-- !query schema +struct +-- !query output +== Parsed Logical Plan == +'Project [unresolvedalias('sum(distinct 'val), None)] ++- 'UnresolvedRelation [explain_temp1], [], false + +== Analyzed Logical Plan == +sum(DISTINCT val): bigint +Aggregate [sum(distinct cast(val#x as bigint)) AS sum(DISTINCT val)#xL] ++- SubqueryAlias spark_catalog.default.explain_temp1 + +- Relation[key#x,val#x] parquet + +== Optimized Logical Plan == +Aggregate [sum(distinct cast(val#x as bigint)) AS sum(DISTINCT val)#xL] ++- Project [val#x] + +- Relation[key#x,val#x] parquet + +== Physical Plan == +*HashAggregate(keys=[], functions=[sum(distinct cast(val#x as bigint)#xL)], output=[sum(DISTINCT val)#xL]) ++- Exchange SinglePartition, true, [id=#x] + +- *HashAggregate(keys=[], functions=[partial_sum(distinct cast(val#x as bigint)#xL)], output=[sum#xL]) + +- *HashAggregate(keys=[cast(val#x as bigint)#xL], functions=[], output=[cast(val#x as bigint)#xL]) + +- Exchange hashpartitioning(cast(val#x as bigint)#xL, 4), true, [id=#x] + +- *HashAggregate(keys=[cast(val#x as bigint) AS cast(val#x as bigint)#xL], functions=[], output=[cast(val#x as bigint)#xL]) + +- *ColumnarToRow + +- FileScan parquet default.explain_temp1[val#x] Batched: true, DataFilters: [], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp1], PartitionFilters: [], PushedFilters: [], ReadSchema: struct + + -- !query EXPLAIN FORMATTED SELECT key, max(val) @@ -827,7 +860,7 @@ Output: [] Arguments: `default`.`explain_view`, SELECT key, val FROM explain_temp1, false, false, PersistedView (3) UnresolvedRelation -Arguments: [explain_temp1], [] +Arguments: [explain_temp1], [], false (4) Project Arguments: ['key, 'val] diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out index c349d9d84c226..89a4da116a6b3 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 68 +-- Number of queries: 76 -- !query @@ -150,6 +150,70 @@ struct 0 +-- !query +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id = 40) FROM emp +-- !query schema +struct +-- !query output +1 0 + + +-- !query +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id = 40) FROM emp +-- !query schema +struct +-- !query output +1 0 + + +-- !query +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id > 0) FROM emp +-- !query schema +struct 0)):bigint> +-- !query output +1 1 + + +-- !query +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id > 0) FROM emp +-- !query schema +struct 0)):bigint> +-- !query output +1 1 + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id = 40) FROM emp +-- !query schema +struct +-- !query output +8 0 + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id = 40) FROM emp +-- !query schema +struct +-- !query output +8 0 + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id > 0) FROM emp +-- !query schema +struct 0)):bigint> +-- !query output +8 1 + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id > 0) FROM emp +-- !query schema +struct 0)):bigint> +-- !query output +8 1 + + -- !query SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData GROUP BY a -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 50eb2a9f22f69..5d9553f804059 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 56 +-- Number of queries: 57 -- !query @@ -573,3 +573,11 @@ org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. Expression in where clause: [(((test_agg.`k` = 1) OR (test_agg.`k` = 2)) OR (((count(1) + 1L) > 1L) OR (max(test_agg.`k`) > 1)))] Invalid expressions: [count(1), max(test_agg.`k`)]; + + +-- !query +SELECT AVG(DISTINCT decimal_col), SUM(DISTINCT decimal_col) FROM VALUES (CAST(1 AS DECIMAL(9, 0))) t(decimal_col) +-- !query schema +struct +-- !query output +1.0000 1 diff --git a/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out index d35d0d5d944bb..7b31b5690998c 100644 --- a/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 29 +-- Number of queries: 30 -- !query @@ -270,3 +270,11 @@ select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t( struct> -- !query output [32,98] + + +-- !query +select aggregate(split('abcdefgh',''), array(array('')), (acc, x) -> array(array(x))) +-- !query schema +struct>> +-- !query output +[[""]] diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 898be09a40318..6b149fd6bb961 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 107 +-- Number of queries: 110 -- !query @@ -1110,3 +1110,39 @@ Cannot parse the INTERVAL value: 1 day 1(line 1, pos 7) == SQL == select interval '1 day 1' -------^^^ + + +-- !query +select interval '1 day 2' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only use numbers in the interval value part for multiple unit value pairs interval form, but got invalid value: 1 day 2(line 1, pos 16) + +== SQL == +select interval '1 day 2' day +----------------^^^ + + +-- !query +select interval 'interval 1' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only use numbers in the interval value part for multiple unit value pairs interval form, but got invalid value: interval 1(line 1, pos 16) + +== SQL == +select interval 'interval 1' day +----------------^^^ + + +-- !query +select interval '-\t 1' day +-- !query schema +struct +-- !query output +-1 days diff --git a/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out index bd8ffb82ee129..bf45ec3d10215 100644 --- a/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 7 +-- Number of queries: 16 -- !query @@ -56,3 +56,82 @@ select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', struct -- !query output array map struct + + +-- !query +SELECT assert_true(true), assert_true(boolean(1)) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +SELECT assert_true(false) +-- !query schema +struct<> +-- !query output +java.lang.RuntimeException +'false' is not true! + + +-- !query +SELECT assert_true(boolean(0)) +-- !query schema +struct<> +-- !query output +java.lang.RuntimeException +'cast(0 as boolean)' is not true! + + +-- !query +SELECT assert_true(null) +-- !query schema +struct<> +-- !query output +java.lang.RuntimeException +'null' is not true! + + +-- !query +SELECT assert_true(boolean(null)) +-- !query schema +struct<> +-- !query output +java.lang.RuntimeException +'cast(null as boolean)' is not true! + + +-- !query +SELECT assert_true(false, 'custom error message') +-- !query schema +struct<> +-- !query output +java.lang.RuntimeException +custom error message + + +-- !query +CREATE TEMPORARY VIEW tbl_misc AS SELECT * FROM (VALUES (1), (8), (2)) AS T(v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT raise_error('error message') +-- !query schema +struct<> +-- !query output +java.lang.RuntimeException +error message + + +-- !query +SELECT if(v > 5, raise_error('too big: ' || v), v + 1) FROM tbl_misc +-- !query schema +struct<> +-- !query output +java.lang.RuntimeException +too big: 8 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out index f7bba96738eab..212365f92946c 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out @@ -143,7 +143,7 @@ SELECT var_pop(1.0), var_samp(2.0) -- !query schema struct -- !query output -0.0 NaN +0.0 NULL -- !query @@ -151,7 +151,7 @@ SELECT stddev_pop(CAST(3.0 AS Decimal(38,0))), stddev_samp(CAST(4.0 AS Decimal(3 -- !query schema struct -- !query output -0.0 NaN +0.0 NULL -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out index 08eba6797b01d..553432e503d5c 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 29 +-- Number of queries: 30 -- !query @@ -385,6 +385,15 @@ org.apache.spark.sql.AnalysisException cannot resolve 'ntile(0)' due to data type mismatch: Buckets expression must be positive, but got: 0; line 1 pos 7 +-- !query +SELECT nth_value(four, 0) OVER (ORDER BY ten), ten, four FROM tenk1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'nth_value(spark_catalog.default.tenk1.`four`, 0)' due to data type mismatch: The 'offset' argument of nth_value must be greater than zero but it is 0.; line 1 pos 7 + + -- !query DROP TABLE empsalary -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out index 4dd4712345a89..f7439d873b4eb 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out @@ -195,7 +195,7 @@ struct -- !query output -NaN +NULL -- !query @@ -2558,7 +2558,7 @@ SELECT var_samp('1') FROM t -- !query schema struct -- !query output -NaN +NULL -- !query @@ -2566,7 +2566,7 @@ SELECT skewness('1') FROM t -- !query schema struct -- !query output -NaN +NULL -- !query @@ -2574,4 +2574,4 @@ SELECT kurtosis('1') FROM t -- !query schema struct -- !query output -NaN +NULL diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out index 76637bf578e6f..a428a7a9c923b 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out @@ -143,7 +143,7 @@ SELECT udf(var_pop(1.0)), var_samp(udf(2.0)) -- !query schema struct -- !query output -0.0 NaN +0.0 NULL -- !query @@ -151,7 +151,7 @@ SELECT stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))), stddev_samp(CAST(udf(4.0) AS -- !query schema struct -- !query output -0.0 NaN +0.0 NULL -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out index a84070535b658..928b9ebb12364 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out @@ -289,13 +289,13 @@ ORDER BY cate, udf(val) struct,collect_set:array,skewness:double,kurtosis:double> -- !query output NULL NULL NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.5 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL -3 NULL 3 3 3 1 3 3.0 NaN NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NaN 1 0.0 NaN NaN 0.0 [3] [3] NaN NaN -NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NaN NaN +3 NULL 3 3 3 1 3 3.0 NULL NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NULL 1 0.0 NULL NULL 0.0 [3] [3] NULL NULL +NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL 1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 1 2 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 2 3 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 2 a 2 1 1 3 4 1.3333333333333333 0.5773502691896258 NULL 1 NULL 2 2 2 4 3 1.0 1.0 2 4 0.22222222222222224 0.33333333333333337 2 4.772185885555555E8 1.0 0.5773502691896258 0.4714045207910317 [1,1,2] [1,2] 1.1539890888012805 -0.6672217220327235 -1 b 1 1 1 1 1 1.0 NaN 1 1 1 1 1 1 1 1 0.3333333333333333 0.0 1 1 0.0 NaN 1 NULL NULL NaN 0.0 [1] [1] NaN NaN -2 b 2 1 1 2 3 1.5 0.7071067811865476 1 1 1 2 2 2 2 2 0.6666666666666666 0.5 1 2 0.25 0.5 2 0.0 NaN 0.7071067811865476 0.5 [1,2] [1,2] 0.0 -2.0000000000000013 +1 b 1 1 1 1 1 1.0 NULL 1 1 1 1 1 1 1 1 0.3333333333333333 0.0 1 1 0.0 NULL 1 NULL NULL NULL 0.0 [1] [1] NULL NULL +2 b 2 1 1 2 3 1.5 0.7071067811865476 1 1 1 2 2 2 2 2 0.6666666666666666 0.5 1 2 0.25 0.5 2 0.0 NULL 0.7071067811865476 0.5 [1,2] [1,2] 0.0 -2.0000000000000013 3 b 3 1 1 3 6 2.0 1.0 1 1 1 3 3 3 3 3 1.0 1.0 2 3 0.6666666666666666 1.0 3 5.3687091175E8 1.0 1.0 0.816496580927726 [1,2,3] [1,2,3] 0.7057890433107311 -1.4999999999999984 diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index ede044a44fdaa..028dd7a12d25d 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 24 +-- Number of queries: 32 -- !query @@ -19,6 +19,30 @@ struct<> -- !query output +-- !query +CREATE OR REPLACE TEMPORARY VIEW basic_pays AS SELECT * FROM VALUES +('Diane Murphy','Accounting',8435), +('Mary Patterson','Accounting',9998), +('Jeff Firrelli','Accounting',8992), +('William Patterson','Accounting',8870), +('Gerard Bondur','Accounting',11472), +('Anthony Bow','Accounting',6627), +('Leslie Jennings','IT',8113), +('Leslie Thompson','IT',5186), +('Julie Firrelli','Sales',9181), +('Steve Patterson','Sales',9441), +('Foon Yue Tseng','Sales',6660), +('George Vanauf','Sales',10563), +('Loui Bondur','SCM',10449), +('Gerard Hernandez','SCM',6949), +('Pamela Castillo','SCM',11303), +('Larry Bott','SCM',11798), +('Barry Jones','SCM',10586) +AS basic_pays(employee_name, department, salary) +-- !query schema +struct<> +-- !query output + -- !query SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val ROWS CURRENT ROW) FROM testData @@ -289,13 +313,13 @@ ORDER BY cate, val struct,collect_set:array,skewness:double,kurtosis:double> -- !query output NULL NULL NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.5 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL -3 NULL 3 3 3 1 3 3.0 NaN NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NaN 1 0.0 NaN NaN 0.0 [3] [3] NaN NaN -NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NaN NaN +3 NULL 3 3 3 1 3 3.0 NULL NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NULL 1 0.0 NULL NULL 0.0 [3] [3] NULL NULL +NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL 1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 1 2 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 2 3 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 2 a 2 1 1 3 4 1.3333333333333333 0.5773502691896258 NULL 1 NULL 2 2 2 4 3 1.0 1.0 2 4 0.22222222222222224 0.33333333333333337 2 4.772185885555555E8 1.0 0.5773502691896258 0.4714045207910317 [1,1,2] [1,2] 1.1539890888012805 -0.6672217220327235 -1 b 1 1 1 1 1 1.0 NaN 1 1 1 1 1 1 1 1 0.3333333333333333 0.0 1 1 0.0 NaN 1 NULL NULL NaN 0.0 [1] [1] NaN NaN -2 b 2 1 1 2 3 1.5 0.7071067811865476 1 1 1 2 2 2 2 2 0.6666666666666666 0.5 1 2 0.25 0.5 2 0.0 NaN 0.7071067811865476 0.5 [1,2] [1,2] 0.0 -2.0000000000000013 +1 b 1 1 1 1 1 1.0 NULL 1 1 1 1 1 1 1 1 0.3333333333333333 0.0 1 1 0.0 NULL 1 NULL NULL NULL 0.0 [1] [1] NULL NULL +2 b 2 1 1 2 3 1.5 0.7071067811865476 1 1 1 2 2 2 2 2 0.6666666666666666 0.5 1 2 0.25 0.5 2 0.0 NULL 0.7071067811865476 0.5 [1,2] [1,2] 0.0 -2.0000000000000013 3 b 3 1 1 3 6 2.0 1.0 1 1 1 3 3 3 3 3 1.0 1.0 2 3 0.6666666666666666 1.0 3 5.3687091175E8 1.0 1.0 0.816496580927726 [1,2,3] [1,2,3] 0.7057890433107311 -1.4999999999999984 @@ -391,3 +415,228 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException window aggregate function with filter predicate is not supported yet.; + + +-- !query +SELECT + employee_name, + salary, + nth_value(employee_name, 2) OVER (ORDER BY salary DESC) second_highest_salary +FROM + basic_pays +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 NULL +Gerard Bondur 11472 Gerard Bondur +Pamela Castillo 11303 Gerard Bondur +Barry Jones 10586 Gerard Bondur +George Vanauf 10563 Gerard Bondur +Loui Bondur 10449 Gerard Bondur +Mary Patterson 9998 Gerard Bondur +Steve Patterson 9441 Gerard Bondur +Julie Firrelli 9181 Gerard Bondur +Jeff Firrelli 8992 Gerard Bondur +William Patterson 8870 Gerard Bondur +Diane Murphy 8435 Gerard Bondur +Leslie Jennings 8113 Gerard Bondur +Gerard Hernandez 6949 Gerard Bondur +Foon Yue Tseng 6660 Gerard Bondur +Anthony Bow 6627 Gerard Bondur +Leslie Thompson 5186 Gerard Bondur + + +-- !query +SELECT + employee_name, + salary, + nth_value(employee_name, 2) OVER ( + ORDER BY salary DESC + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) second_highest_salary +FROM + basic_pays +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 NULL +Gerard Bondur 11472 Gerard Bondur +Pamela Castillo 11303 Gerard Bondur +Barry Jones 10586 Gerard Bondur +George Vanauf 10563 Gerard Bondur +Loui Bondur 10449 Gerard Bondur +Mary Patterson 9998 Gerard Bondur +Steve Patterson 9441 Gerard Bondur +Julie Firrelli 9181 Gerard Bondur +Jeff Firrelli 8992 Gerard Bondur +William Patterson 8870 Gerard Bondur +Diane Murphy 8435 Gerard Bondur +Leslie Jennings 8113 Gerard Bondur +Gerard Hernandez 6949 Gerard Bondur +Foon Yue Tseng 6660 Gerard Bondur +Anthony Bow 6627 Gerard Bondur +Leslie Thompson 5186 Gerard Bondur + + +-- !query +SELECT + employee_name, + salary, + nth_value(employee_name, 2) OVER ( + ORDER BY salary + RANGE BETWEEN 2000 PRECEDING AND 1000 FOLLOWING) second_highest_salary +FROM + basic_pays +ORDER BY salary +-- !query schema +struct +-- !query output +Leslie Thompson 5186 NULL +Anthony Bow 6627 Anthony Bow +Foon Yue Tseng 6660 Anthony Bow +Gerard Hernandez 6949 Anthony Bow +Leslie Jennings 8113 Foon Yue Tseng +Diane Murphy 8435 Foon Yue Tseng +William Patterson 8870 Leslie Jennings +Jeff Firrelli 8992 Diane Murphy +Julie Firrelli 9181 Diane Murphy +Steve Patterson 9441 Diane Murphy +Mary Patterson 9998 Diane Murphy +Loui Bondur 10449 Jeff Firrelli +George Vanauf 10563 Jeff Firrelli +Barry Jones 10586 Jeff Firrelli +Pamela Castillo 11303 Mary Patterson +Gerard Bondur 11472 Loui Bondur +Larry Bott 11798 Loui Bondur + + +-- !query +SELECT + employee_name, + salary, + nth_value(employee_name, 2) OVER ( + ORDER BY salary DESC + ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING) second_highest_salary +FROM + basic_pays +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Gerard Bondur +Gerard Bondur 11472 Gerard Bondur +Pamela Castillo 11303 Gerard Bondur +Barry Jones 10586 Pamela Castillo +George Vanauf 10563 Barry Jones +Loui Bondur 10449 George Vanauf +Mary Patterson 9998 Loui Bondur +Steve Patterson 9441 Mary Patterson +Julie Firrelli 9181 Steve Patterson +Jeff Firrelli 8992 Julie Firrelli +William Patterson 8870 Jeff Firrelli +Diane Murphy 8435 William Patterson +Leslie Jennings 8113 Diane Murphy +Gerard Hernandez 6949 Leslie Jennings +Foon Yue Tseng 6660 Gerard Hernandez +Anthony Bow 6627 Foon Yue Tseng +Leslie Thompson 5186 Anthony Bow + + +-- !query +SELECT + employee_name, + salary, + nth_value(employee_name, 2) OVER ( + ORDER BY salary DESC + RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) second_highest_salary +FROM + basic_pays +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Gerard Bondur +Gerard Bondur 11472 Pamela Castillo +Pamela Castillo 11303 Barry Jones +Barry Jones 10586 George Vanauf +George Vanauf 10563 Loui Bondur +Loui Bondur 10449 Mary Patterson +Mary Patterson 9998 Steve Patterson +Steve Patterson 9441 Julie Firrelli +Julie Firrelli 9181 Jeff Firrelli +Jeff Firrelli 8992 William Patterson +William Patterson 8870 Diane Murphy +Diane Murphy 8435 Leslie Jennings +Leslie Jennings 8113 Gerard Hernandez +Gerard Hernandez 6949 Foon Yue Tseng +Foon Yue Tseng 6660 Anthony Bow +Anthony Bow 6627 Leslie Thompson +Leslie Thompson 5186 NULL + + +-- !query +SELECT + employee_name, + salary, + nth_value(employee_name, 2) OVER ( + ORDER BY salary DESC + RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) second_highest_salary +FROM + basic_pays +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Gerard Bondur +Gerard Bondur 11472 Gerard Bondur +Pamela Castillo 11303 Gerard Bondur +Barry Jones 10586 Gerard Bondur +George Vanauf 10563 Gerard Bondur +Loui Bondur 10449 Gerard Bondur +Mary Patterson 9998 Gerard Bondur +Steve Patterson 9441 Gerard Bondur +Julie Firrelli 9181 Gerard Bondur +Jeff Firrelli 8992 Gerard Bondur +William Patterson 8870 Gerard Bondur +Diane Murphy 8435 Gerard Bondur +Leslie Jennings 8113 Gerard Bondur +Gerard Hernandez 6949 Gerard Bondur +Foon Yue Tseng 6660 Gerard Bondur +Anthony Bow 6627 Gerard Bondur +Leslie Thompson 5186 Gerard Bondur + + +-- !query +SELECT + employee_name, + department, + salary, + NTH_VALUE(employee_name, 2) OVER ( + PARTITION BY department + ORDER BY salary DESC + RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING + ) second_highest_salary +FROM + basic_pays +ORDER BY department +-- !query schema +struct +-- !query output +Gerard Bondur Accounting 11472 Mary Patterson +Mary Patterson Accounting 9998 Mary Patterson +Jeff Firrelli Accounting 8992 Mary Patterson +William Patterson Accounting 8870 Mary Patterson +Diane Murphy Accounting 8435 Mary Patterson +Anthony Bow Accounting 6627 Mary Patterson +Leslie Jennings IT 8113 Leslie Thompson +Leslie Thompson IT 5186 Leslie Thompson +Larry Bott SCM 11798 Pamela Castillo +Pamela Castillo SCM 11303 Pamela Castillo +Barry Jones SCM 10586 Pamela Castillo +Loui Bondur SCM 10449 Pamela Castillo +Gerard Hernandez SCM 6949 Pamela Castillo +George Vanauf Sales 10563 Steve Patterson +Steve Patterson Sales 9441 Steve Patterson +Julie Firrelli Sales 9181 Steve Patterson +Foon Yue Tseng Sales 6660 Steve Patterson \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/percentile_approx-input.csv.bz2 b/sql/core/src/test/resources/test-data/percentile_approx-input.csv.bz2 new file mode 100644 index 0000000000000..f85e2896b3a89 Binary files /dev/null and b/sql/core/src/test/resources/test-data/percentile_approx-input.csv.bz2 differ diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt index fa01042350149..b3b11b60ded0b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt @@ -11,15 +11,15 @@ TakeOrderedAndProject (77) : : :- * Project (17) : : : +- * BroadcastHashJoin Inner BuildRight (16) : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : :- BroadcastExchange (5) - : : : : : +- * Project (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.date_dim (1) - : : : : +- * Filter (8) - : : : : +- * ColumnarToRow (7) - : : : : +- Scan parquet default.store_sales (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.store_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * Project (7) + : : : : +- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet default.date_dim (4) : : : +- BroadcastExchange (15) : : : +- * Project (14) : : : +- * Filter (13) @@ -43,11 +43,11 @@ TakeOrderedAndProject (77) : : :- * Project (45) : : : +- * BroadcastHashJoin Inner BuildRight (44) : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildLeft (37) - : : : : :- ReusedExchange (33) - : : : : +- * Filter (36) - : : : : +- * ColumnarToRow (35) - : : : : +- Scan parquet default.store_sales (34) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Filter (35) + : : : : : +- * ColumnarToRow (34) + : : : : : +- Scan parquet default.store_sales (33) + : : : : +- ReusedExchange (36) : : : +- BroadcastExchange (43) : : : +- * Project (42) : : : +- * Filter (41) @@ -65,11 +65,11 @@ TakeOrderedAndProject (77) : :- * Project (63) : : +- * BroadcastHashJoin Inner BuildRight (62) : : :- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildLeft (59) - : : : :- ReusedExchange (55) - : : : +- * Filter (58) - : : : +- * ColumnarToRow (57) - : : : +- Scan parquet default.store_sales (56) + : : : +- * BroadcastHashJoin Inner BuildRight (59) + : : : :- * Filter (57) + : : : : +- * ColumnarToRow (56) + : : : : +- Scan parquet default.store_sales (55) + : : : +- ReusedExchange (58) : : +- ReusedExchange (61) : +- ReusedExchange (64) +- BroadcastExchange (70) @@ -78,50 +78,50 @@ TakeOrderedAndProject (77) +- Scan parquet default.item (67) -(1) Scan parquet default.date_dim -Output [2]: [d_date_sk#1, d_year#2] +(1) Scan parquet default.store_sales +Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), GreaterThanOrEqual(d_date_sk,2451545), LessThanOrEqual(d_date_sk,2451910), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), GreaterThanOrEqual(ss_sold_date_sk,2451545), LessThanOrEqual(ss_sold_date_sk,2451910), IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#1, d_year#2] +(2) ColumnarToRow [codegen id : 5] +Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] -(3) Filter [codegen id : 1] -Input [2]: [d_date_sk#1, d_year#2] -Condition : ((((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND (d_date_sk#1 >= 2451545)) AND (d_date_sk#1 <= 2451910)) AND isnotnull(d_date_sk#1)) +(3) Filter [codegen id : 5] +Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Condition : (((((isnotnull(ss_sold_date_sk#1) AND (ss_sold_date_sk#1 >= 2451545)) AND (ss_sold_date_sk#1 <= 2451910)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_store_sk#4)) AND isnotnull(ss_item_sk#2)) -(4) Project [codegen id : 1] -Output [1]: [d_date_sk#1] -Input [2]: [d_date_sk#1, d_year#2] +(4) Scan parquet default.date_dim +Output [2]: [d_date_sk#9, d_year#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), GreaterThanOrEqual(d_date_sk,2451545), LessThanOrEqual(d_date_sk,2451910), IsNotNull(d_date_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [1]: [d_date_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#3] +(5) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#9, d_year#10] -(6) Scan parquet default.store_sales -Output [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), GreaterThanOrEqual(ss_sold_date_sk,2451545), LessThanOrEqual(ss_sold_date_sk,2451910), IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [2]: [d_date_sk#9, d_year#10] +Condition : ((((isnotnull(d_year#10) AND (d_year#10 = 2000)) AND (d_date_sk#9 >= 2451545)) AND (d_date_sk#9 <= 2451910)) AND isnotnull(d_date_sk#9)) -(7) ColumnarToRow -Input [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +(7) Project [codegen id : 1] +Output [1]: [d_date_sk#9] +Input [2]: [d_date_sk#9, d_year#10] -(8) Filter -Input [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Condition : (((((isnotnull(ss_sold_date_sk#4) AND (ss_sold_date_sk#4 >= 2451545)) AND (ss_sold_date_sk#4 <= 2451910)) AND isnotnull(ss_cdemo_sk#6)) AND isnotnull(ss_store_sk#7)) AND isnotnull(ss_item_sk#5)) +(8) BroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#4] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#9] Join condition: None (10) Project [codegen id : 5] -Output [7]: [ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [9]: [d_date_sk#1, ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +Output [7]: [ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, d_date_sk#9] (11) Scan parquet default.customer_demographics Output [4]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_education_status#15] @@ -146,13 +146,13 @@ Input [1]: [cd_demo_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#6] +Left keys [1]: [ss_cdemo_sk#3] Right keys [1]: [cd_demo_sk#12] Join condition: None (17) Project [codegen id : 5] -Output [6]: [ss_item_sk#5, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [8]: [ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, cd_demo_sk#12] +Output [6]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [8]: [ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, cd_demo_sk#12] (18) Scan parquet default.store Output [2]: [s_store_sk#17, s_state#18] @@ -173,13 +173,13 @@ Input [2]: [s_store_sk#17, s_state#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] (22) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#7] +Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#17] Join condition: None (23) Project [codegen id : 5] -Output [6]: [ss_item_sk#5, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, s_state#18] -Input [8]: [ss_item_sk#5, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, s_store_sk#17, s_state#18] +Output [6]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, s_state#18] +Input [8]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, s_store_sk#17, s_state#18] (24) Scan parquet default.item Output [2]: [i_item_sk#20, i_item_id#21] @@ -200,13 +200,13 @@ Input [2]: [i_item_sk#20, i_item_id#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] (28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#5] +Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#20] Join condition: None (29) Project [codegen id : 5] -Output [6]: [i_item_id#21, s_state#18, ss_quantity#8 AS agg1#23, ss_list_price#9 AS agg2#24, ss_coupon_amt#11 AS agg3#25, ss_sales_price#10 AS agg4#26] -Input [8]: [ss_item_sk#5, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, s_state#18, i_item_sk#20, i_item_id#21] +Output [6]: [i_item_id#21, s_state#18, ss_quantity#5 AS agg1#23, ss_list_price#6 AS agg2#24, ss_coupon_amt#8 AS agg3#25, ss_sales_price#7 AS agg4#26] +Input [8]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, s_state#18, i_item_sk#20, i_item_id#21] (30) HashAggregate [codegen id : 5] Input [6]: [i_item_id#21, s_state#18, agg1#23, agg2#24, agg3#25, agg4#26] @@ -226,31 +226,31 @@ Functions [4]: [avg(cast(agg1#23 as bigint)), avg(UnscaledValue(agg2#24)), avg(U Aggregate Attributes [4]: [avg(cast(agg1#23 as bigint))#44, avg(UnscaledValue(agg2#24))#45, avg(UnscaledValue(agg3#25))#46, avg(UnscaledValue(agg4#26))#47] Results [7]: [i_item_id#21, s_state#18, 0 AS g_state#48, avg(cast(agg1#23 as bigint))#44 AS agg1#49, cast((avg(UnscaledValue(agg2#24))#45 / 100.0) as decimal(11,6)) AS agg2#50, cast((avg(UnscaledValue(agg3#25))#46 / 100.0) as decimal(11,6)) AS agg3#51, cast((avg(UnscaledValue(agg4#26))#47 / 100.0) as decimal(11,6)) AS agg4#52] -(33) ReusedExchange [Reuses operator id: 5] -Output [1]: [d_date_sk#1] - -(34) Scan parquet default.store_sales -Output [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +(33) Scan parquet default.store_sales +Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk), GreaterThanOrEqual(ss_sold_date_sk,2451545), LessThanOrEqual(ss_sold_date_sk,2451910), IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(35) ColumnarToRow -Input [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +(34) ColumnarToRow [codegen id : 11] +Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] + +(35) Filter [codegen id : 11] +Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Condition : (((((isnotnull(ss_sold_date_sk#1) AND (ss_sold_date_sk#1 >= 2451545)) AND (ss_sold_date_sk#1 <= 2451910)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_store_sk#4)) AND isnotnull(ss_item_sk#2)) -(36) Filter -Input [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Condition : (((((isnotnull(ss_sold_date_sk#4) AND (ss_sold_date_sk#4 >= 2451545)) AND (ss_sold_date_sk#4 <= 2451910)) AND isnotnull(ss_cdemo_sk#6)) AND isnotnull(ss_store_sk#7)) AND isnotnull(ss_item_sk#5)) +(36) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#9] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#4] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#9] Join condition: None (38) Project [codegen id : 11] -Output [7]: [ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [9]: [d_date_sk#1, ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +Output [7]: [ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, d_date_sk#9] (39) Scan parquet default.store Output [2]: [s_store_sk#17, s_state#18] @@ -275,37 +275,37 @@ Input [1]: [s_store_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] (44) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#7] +Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#17] Join condition: None (45) Project [codegen id : 11] -Output [6]: [ss_item_sk#5, ss_cdemo_sk#6, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [8]: [ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, s_store_sk#17] +Output [6]: [ss_item_sk#2, ss_cdemo_sk#3, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [8]: [ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, s_store_sk#17] (46) ReusedExchange [Reuses operator id: 15] Output [1]: [cd_demo_sk#12] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#6] +Left keys [1]: [ss_cdemo_sk#3] Right keys [1]: [cd_demo_sk#12] Join condition: None (48) Project [codegen id : 11] -Output [5]: [ss_item_sk#5, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [7]: [ss_item_sk#5, ss_cdemo_sk#6, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, cd_demo_sk#12] +Output [5]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [7]: [ss_item_sk#2, ss_cdemo_sk#3, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, cd_demo_sk#12] (49) ReusedExchange [Reuses operator id: 27] Output [2]: [i_item_sk#20, i_item_id#21] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#5] +Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#20] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#21, ss_quantity#8 AS agg1#23, ss_list_price#9 AS agg2#24, ss_coupon_amt#11 AS agg3#25, ss_sales_price#10 AS agg4#26] -Input [7]: [ss_item_sk#5, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, i_item_sk#20, i_item_id#21] +Output [5]: [i_item_id#21, ss_quantity#5 AS agg1#23, ss_list_price#6 AS agg2#24, ss_coupon_amt#8 AS agg3#25, ss_sales_price#7 AS agg4#26] +Input [7]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_sk#20, i_item_id#21] (52) HashAggregate [codegen id : 11] Input [5]: [i_item_id#21, agg1#23, agg2#24, agg3#25, agg4#26] @@ -325,55 +325,55 @@ Functions [4]: [avg(cast(agg1#23 as bigint)), avg(UnscaledValue(agg2#24)), avg(U Aggregate Attributes [4]: [avg(cast(agg1#23 as bigint))#71, avg(UnscaledValue(agg2#24))#72, avg(UnscaledValue(agg3#25))#73, avg(UnscaledValue(agg4#26))#74] Results [7]: [i_item_id#21, null AS s_state#75, 1 AS g_state#76, avg(cast(agg1#23 as bigint))#71 AS agg1#77, cast((avg(UnscaledValue(agg2#24))#72 / 100.0) as decimal(11,6)) AS agg2#78, cast((avg(UnscaledValue(agg3#25))#73 / 100.0) as decimal(11,6)) AS agg3#79, cast((avg(UnscaledValue(agg4#26))#74 / 100.0) as decimal(11,6)) AS agg4#80] -(55) ReusedExchange [Reuses operator id: 5] -Output [1]: [d_date_sk#1] - -(56) Scan parquet default.store_sales -Output [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +(55) Scan parquet default.store_sales +Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk), GreaterThanOrEqual(ss_sold_date_sk,2451545), LessThanOrEqual(ss_sold_date_sk,2451910), IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(57) ColumnarToRow -Input [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +(56) ColumnarToRow [codegen id : 17] +Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] + +(57) Filter [codegen id : 17] +Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Condition : (((((isnotnull(ss_sold_date_sk#1) AND (ss_sold_date_sk#1 >= 2451545)) AND (ss_sold_date_sk#1 <= 2451910)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_store_sk#4)) AND isnotnull(ss_item_sk#2)) -(58) Filter -Input [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Condition : (((((isnotnull(ss_sold_date_sk#4) AND (ss_sold_date_sk#4 >= 2451545)) AND (ss_sold_date_sk#4 <= 2451910)) AND isnotnull(ss_cdemo_sk#6)) AND isnotnull(ss_store_sk#7)) AND isnotnull(ss_item_sk#5)) +(58) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#9] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#4] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#9] Join condition: None (60) Project [codegen id : 17] -Output [7]: [ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [9]: [d_date_sk#1, ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +Output [7]: [ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, d_date_sk#9] (61) ReusedExchange [Reuses operator id: 43] Output [1]: [s_store_sk#17] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#7] +Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#17] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#5, ss_cdemo_sk#6, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [8]: [ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, s_store_sk#17] +Output [6]: [ss_item_sk#2, ss_cdemo_sk#3, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [8]: [ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, s_store_sk#17] (64) ReusedExchange [Reuses operator id: 15] Output [1]: [cd_demo_sk#12] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#6] +Left keys [1]: [ss_cdemo_sk#3] Right keys [1]: [cd_demo_sk#12] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#5, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [7]: [ss_item_sk#5, ss_cdemo_sk#6, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, cd_demo_sk#12] +Output [5]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [7]: [ss_item_sk#2, ss_cdemo_sk#3, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, cd_demo_sk#12] (67) Scan parquet default.item Output [1]: [i_item_sk#20] @@ -394,13 +394,13 @@ Input [1]: [i_item_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#5] +Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#20] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#8 AS agg1#23, ss_list_price#9 AS agg2#24, ss_coupon_amt#11 AS agg3#25, ss_sales_price#10 AS agg4#26] -Input [6]: [ss_item_sk#5, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, i_item_sk#20] +Output [4]: [ss_quantity#5 AS agg1#23, ss_list_price#6 AS agg2#24, ss_coupon_amt#8 AS agg3#25, ss_sales_price#7 AS agg4#26] +Input [6]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_sk#20] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#23, agg2#24, agg3#25, agg4#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/simplified.txt index fc7202e739bcc..d14061de1d1f4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/simplified.txt @@ -13,7 +13,11 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] Project [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_cdemo_sk,ss_store_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -22,10 +26,6 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] - Filter [ss_sold_date_sk,ss_cdemo_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -61,13 +61,13 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Project [ss_item_sk,ss_cdemo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_cdemo_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #7 WholeStageCodegen (8) @@ -93,13 +93,13 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Project [ss_item_sk,ss_cdemo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_cdemo_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [s_store_sk] #7 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt index 6071139e809cf..220d661fd45e9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt @@ -10,15 +10,15 @@ TakeOrderedAndProject (34) : :- * Project (17) : : +- * BroadcastHashJoin Inner BuildRight (16) : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : :- BroadcastExchange (5) - : : : : +- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.date_dim (1) - : : : +- * Filter (8) - : : : +- * ColumnarToRow (7) - : : : +- Scan parquet default.store_sales (6) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) : : +- BroadcastExchange (15) : : +- * Project (14) : : +- * Filter (13) @@ -35,50 +35,50 @@ TakeOrderedAndProject (34) +- Scan parquet default.item (25) -(1) Scan parquet default.date_dim -Output [2]: [d_date_sk#1, d_year#2] +(1) Scan parquet default.store_sales +Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), GreaterThanOrEqual(d_date_sk,2450815), LessThanOrEqual(d_date_sk,2451179), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), GreaterThanOrEqual(ss_sold_date_sk,2450815), LessThanOrEqual(ss_sold_date_sk,2451179), IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#1, d_year#2] +(2) ColumnarToRow [codegen id : 5] +Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] -(3) Filter [codegen id : 1] -Input [2]: [d_date_sk#1, d_year#2] -Condition : ((((isnotnull(d_year#2) AND (d_year#2 = 1998)) AND (d_date_sk#1 >= 2450815)) AND (d_date_sk#1 <= 2451179)) AND isnotnull(d_date_sk#1)) +(3) Filter [codegen id : 5] +Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Condition : (((((isnotnull(ss_sold_date_sk#1) AND (ss_sold_date_sk#1 >= 2450815)) AND (ss_sold_date_sk#1 <= 2451179)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_promo_sk#4)) -(4) Project [codegen id : 1] -Output [1]: [d_date_sk#1] -Input [2]: [d_date_sk#1, d_year#2] +(4) Scan parquet default.date_dim +Output [2]: [d_date_sk#9, d_year#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), GreaterThanOrEqual(d_date_sk,2450815), LessThanOrEqual(d_date_sk,2451179), IsNotNull(d_date_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [1]: [d_date_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#3] +(5) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#9, d_year#10] -(6) Scan parquet default.store_sales -Output [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_promo_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), GreaterThanOrEqual(ss_sold_date_sk,2450815), LessThanOrEqual(ss_sold_date_sk,2451179), IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [2]: [d_date_sk#9, d_year#10] +Condition : ((((isnotnull(d_year#10) AND (d_year#10 = 1998)) AND (d_date_sk#9 >= 2450815)) AND (d_date_sk#9 <= 2451179)) AND isnotnull(d_date_sk#9)) -(7) ColumnarToRow -Input [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_promo_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +(7) Project [codegen id : 1] +Output [1]: [d_date_sk#9] +Input [2]: [d_date_sk#9, d_year#10] -(8) Filter -Input [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_promo_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Condition : (((((isnotnull(ss_sold_date_sk#4) AND (ss_sold_date_sk#4 >= 2450815)) AND (ss_sold_date_sk#4 <= 2451179)) AND isnotnull(ss_cdemo_sk#6)) AND isnotnull(ss_item_sk#5)) AND isnotnull(ss_promo_sk#7)) +(8) BroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#4] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#9] Join condition: None (10) Project [codegen id : 5] -Output [7]: [ss_item_sk#5, ss_cdemo_sk#6, ss_promo_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [9]: [d_date_sk#1, ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_promo_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +Output [7]: [ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, d_date_sk#9] (11) Scan parquet default.promotion Output [3]: [p_promo_sk#12, p_channel_email#13, p_channel_event#14] @@ -103,13 +103,13 @@ Input [1]: [p_promo_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_promo_sk#7] +Left keys [1]: [ss_promo_sk#4] Right keys [1]: [p_promo_sk#12] Join condition: None (17) Project [codegen id : 5] -Output [6]: [ss_item_sk#5, ss_cdemo_sk#6, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [8]: [ss_item_sk#5, ss_cdemo_sk#6, ss_promo_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, p_promo_sk#12] +Output [6]: [ss_item_sk#2, ss_cdemo_sk#3, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [8]: [ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, p_promo_sk#12] (18) Scan parquet default.customer_demographics Output [4]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19] @@ -134,13 +134,13 @@ Input [1]: [cd_demo_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#6] +Left keys [1]: [ss_cdemo_sk#3] Right keys [1]: [cd_demo_sk#16] Join condition: None (24) Project [codegen id : 5] -Output [5]: [ss_item_sk#5, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [7]: [ss_item_sk#5, ss_cdemo_sk#6, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, cd_demo_sk#16] +Output [5]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [7]: [ss_item_sk#2, ss_cdemo_sk#3, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, cd_demo_sk#16] (25) Scan parquet default.item Output [2]: [i_item_sk#21, i_item_id#22] @@ -161,18 +161,18 @@ Input [2]: [i_item_sk#21, i_item_id#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] (29) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#5] +Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#21] Join condition: None (30) Project [codegen id : 5] -Output [5]: [ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, i_item_id#22] -Input [7]: [ss_item_sk#5, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, i_item_sk#21, i_item_id#22] +Output [5]: [ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_id#22] +Input [7]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_sk#21, i_item_id#22] (31) HashAggregate [codegen id : 5] -Input [5]: [ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, i_item_id#22] +Input [5]: [ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_id#22] Keys [1]: [i_item_id#22] -Functions [4]: [partial_avg(cast(ss_quantity#8 as bigint)), partial_avg(UnscaledValue(ss_list_price#9)), partial_avg(UnscaledValue(ss_coupon_amt#11)), partial_avg(UnscaledValue(ss_sales_price#10))] +Functions [4]: [partial_avg(cast(ss_quantity#5 as bigint)), partial_avg(UnscaledValue(ss_list_price#6)), partial_avg(UnscaledValue(ss_coupon_amt#8)), partial_avg(UnscaledValue(ss_sales_price#7))] Aggregate Attributes [8]: [sum#24, count#25, sum#26, count#27, sum#28, count#29, sum#30, count#31] Results [9]: [i_item_id#22, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] @@ -183,9 +183,9 @@ Arguments: hashpartitioning(i_item_id#22, 5), true, [id=#40] (33) HashAggregate [codegen id : 6] Input [9]: [i_item_id#22, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] Keys [1]: [i_item_id#22] -Functions [4]: [avg(cast(ss_quantity#8 as bigint)), avg(UnscaledValue(ss_list_price#9)), avg(UnscaledValue(ss_coupon_amt#11)), avg(UnscaledValue(ss_sales_price#10))] -Aggregate Attributes [4]: [avg(cast(ss_quantity#8 as bigint))#41, avg(UnscaledValue(ss_list_price#9))#42, avg(UnscaledValue(ss_coupon_amt#11))#43, avg(UnscaledValue(ss_sales_price#10))#44] -Results [5]: [i_item_id#22, avg(cast(ss_quantity#8 as bigint))#41 AS agg1#45, cast((avg(UnscaledValue(ss_list_price#9))#42 / 100.0) as decimal(11,6)) AS agg2#46, cast((avg(UnscaledValue(ss_coupon_amt#11))#43 / 100.0) as decimal(11,6)) AS agg3#47, cast((avg(UnscaledValue(ss_sales_price#10))#44 / 100.0) as decimal(11,6)) AS agg4#48] +Functions [4]: [avg(cast(ss_quantity#5 as bigint)), avg(UnscaledValue(ss_list_price#6)), avg(UnscaledValue(ss_coupon_amt#8)), avg(UnscaledValue(ss_sales_price#7))] +Aggregate Attributes [4]: [avg(cast(ss_quantity#5 as bigint))#41, avg(UnscaledValue(ss_list_price#6))#42, avg(UnscaledValue(ss_coupon_amt#8))#43, avg(UnscaledValue(ss_sales_price#7))#44] +Results [5]: [i_item_id#22, avg(cast(ss_quantity#5 as bigint))#41 AS agg1#45, cast((avg(UnscaledValue(ss_list_price#6))#42 / 100.0) as decimal(11,6)) AS agg2#46, cast((avg(UnscaledValue(ss_coupon_amt#8))#43 / 100.0) as decimal(11,6)) AS agg3#47, cast((avg(UnscaledValue(ss_sales_price#7))#44 / 100.0) as decimal(11,6)) AS agg4#48] (34) TakeOrderedAndProject Input [5]: [i_item_id#22, agg1#45, agg2#46, agg3#47, agg4#48] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/simplified.txt index 4576b8cef59ee..61cc7daa76456 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/simplified.txt @@ -12,7 +12,11 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] Project [ss_item_sk,ss_cdemo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] BroadcastHashJoin [ss_promo_sk,p_promo_sk] Project [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_cdemo_sk,ss_item_sk,ss_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -21,10 +25,6 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] - Filter [ss_sold_date_sk,ss_cdemo_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt index 586abbd8f3fef..8ee427262b332 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt @@ -11,14 +11,14 @@ : : :- * Project (15) : : : +- * BroadcastHashJoin Inner BuildRight (14) : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildLeft (8) - : : : : :- BroadcastExchange (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.customer_demographics (1) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet default.store_sales (5) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.store_sales (1) + : : : : +- BroadcastExchange (7) + : : : : +- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet default.customer_demographics (4) : : : +- BroadcastExchange (13) : : : +- * Filter (12) : : : +- * ColumnarToRow (11) @@ -39,46 +39,46 @@ +- Scan parquet default.customer_address (29) -(1) Scan parquet default.customer_demographics -Output [3]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3] +(1) Scan parquet default.store_sales +Output [10]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [3]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3] +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_sold_date_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] +ReadSchema: struct -(3) Filter [codegen id : 1] -Input [3]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3] -Condition : (isnotnull(cd_demo_sk#1) AND ((((cd_marital_status#2 = M) AND (cd_education_status#3 = Advanced Degree)) OR ((cd_marital_status#2 = S) AND (cd_education_status#3 = College))) OR ((cd_marital_status#2 = W) AND (cd_education_status#3 = 2 yr Degree)))) +(2) ColumnarToRow [codegen id : 6] +Input [10]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] -(4) BroadcastExchange -Input [3]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#4] +(3) Filter [codegen id : 6] +Input [10]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] +Condition : ((((((isnotnull(ss_store_sk#5) AND isnotnull(ss_addr_sk#4)) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_cdemo_sk#2)) AND isnotnull(ss_hdemo_sk#3)) AND ((((ss_net_profit#10 >= 100.00) AND (ss_net_profit#10 <= 200.00)) OR ((ss_net_profit#10 >= 150.00) AND (ss_net_profit#10 <= 300.00))) OR ((ss_net_profit#10 >= 50.00) AND (ss_net_profit#10 <= 250.00)))) AND ((((ss_sales_price#7 >= 100.00) AND (ss_sales_price#7 <= 150.00)) OR ((ss_sales_price#7 >= 50.00) AND (ss_sales_price#7 <= 100.00))) OR ((ss_sales_price#7 >= 150.00) AND (ss_sales_price#7 <= 200.00)))) -(5) Scan parquet default.store_sales -Output [10]: [ss_sold_date_sk#5, ss_cdemo_sk#6, ss_hdemo_sk#7, ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_sales_price#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14] +(4) Scan parquet default.customer_demographics +Output [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_sold_date_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] +ReadSchema: struct + +(5) ColumnarToRow [codegen id : 1] +Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] -(6) ColumnarToRow -Input [10]: [ss_sold_date_sk#5, ss_cdemo_sk#6, ss_hdemo_sk#7, ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_sales_price#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14] +(6) Filter [codegen id : 1] +Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] +Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree)) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College))) OR ((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree)))) -(7) Filter -Input [10]: [ss_sold_date_sk#5, ss_cdemo_sk#6, ss_hdemo_sk#7, ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_sales_price#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14] -Condition : ((((((isnotnull(ss_store_sk#9) AND isnotnull(ss_addr_sk#8)) AND isnotnull(ss_sold_date_sk#5)) AND isnotnull(ss_cdemo_sk#6)) AND isnotnull(ss_hdemo_sk#7)) AND ((((ss_net_profit#14 >= 100.00) AND (ss_net_profit#14 <= 200.00)) OR ((ss_net_profit#14 >= 150.00) AND (ss_net_profit#14 <= 300.00))) OR ((ss_net_profit#14 >= 50.00) AND (ss_net_profit#14 <= 250.00)))) AND ((((ss_sales_price#11 >= 100.00) AND (ss_sales_price#11 <= 150.00)) OR ((ss_sales_price#11 >= 50.00) AND (ss_sales_price#11 <= 100.00))) OR ((ss_sales_price#11 >= 150.00) AND (ss_sales_price#11 <= 200.00)))) +(7) BroadcastExchange +Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] (8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cd_demo_sk#1] -Right keys [1]: [ss_cdemo_sk#6] -Join condition: ((((((cd_marital_status#2 = M) AND (cd_education_status#3 = Advanced Degree)) AND (ss_sales_price#11 >= 100.00)) AND (ss_sales_price#11 <= 150.00)) OR ((((cd_marital_status#2 = S) AND (cd_education_status#3 = College)) AND (ss_sales_price#11 >= 50.00)) AND (ss_sales_price#11 <= 100.00))) OR ((((cd_marital_status#2 = W) AND (cd_education_status#3 = 2 yr Degree)) AND (ss_sales_price#11 >= 150.00)) AND (ss_sales_price#11 <= 200.00))) +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] +Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree)) AND (ss_sales_price#7 >= 100.00)) AND (ss_sales_price#7 <= 150.00)) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College)) AND (ss_sales_price#7 >= 50.00)) AND (ss_sales_price#7 <= 100.00))) OR ((((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree)) AND (ss_sales_price#7 >= 150.00)) AND (ss_sales_price#7 <= 200.00))) (9) Project [codegen id : 6] -Output [11]: [cd_marital_status#2, cd_education_status#3, ss_sold_date_sk#5, ss_hdemo_sk#7, ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_sales_price#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14] -Input [13]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3, ss_sold_date_sk#5, ss_cdemo_sk#6, ss_hdemo_sk#7, ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_sales_price#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14] +Output [11]: [ss_sold_date_sk#1, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, cd_marital_status#12, cd_education_status#13] +Input [13]: [ss_sold_date_sk#1, ss_cdemo_sk#2, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] (10) Scan parquet default.household_demographics Output [2]: [hd_demo_sk#15, hd_dep_count#16] @@ -99,13 +99,13 @@ Input [2]: [hd_demo_sk#15, hd_dep_count#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_hdemo_sk#7] +Left keys [1]: [ss_hdemo_sk#3] Right keys [1]: [hd_demo_sk#15] -Join condition: (((((((cd_marital_status#2 = M) AND (cd_education_status#3 = Advanced Degree)) AND (ss_sales_price#11 >= 100.00)) AND (ss_sales_price#11 <= 150.00)) AND (hd_dep_count#16 = 3)) OR (((((cd_marital_status#2 = S) AND (cd_education_status#3 = College)) AND (ss_sales_price#11 >= 50.00)) AND (ss_sales_price#11 <= 100.00)) AND (hd_dep_count#16 = 1))) OR (((((cd_marital_status#2 = W) AND (cd_education_status#3 = 2 yr Degree)) AND (ss_sales_price#11 >= 150.00)) AND (ss_sales_price#11 <= 200.00)) AND (hd_dep_count#16 = 1))) +Join condition: (((((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree)) AND (ss_sales_price#7 >= 100.00)) AND (ss_sales_price#7 <= 150.00)) AND (hd_dep_count#16 = 3)) OR (((((cd_marital_status#12 = S) AND (cd_education_status#13 = College)) AND (ss_sales_price#7 >= 50.00)) AND (ss_sales_price#7 <= 100.00)) AND (hd_dep_count#16 = 1))) OR (((((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree)) AND (ss_sales_price#7 >= 150.00)) AND (ss_sales_price#7 <= 200.00)) AND (hd_dep_count#16 = 1))) (15) Project [codegen id : 6] -Output [7]: [ss_sold_date_sk#5, ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14] -Input [13]: [cd_marital_status#2, cd_education_status#3, ss_sold_date_sk#5, ss_hdemo_sk#7, ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_sales_price#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14, hd_demo_sk#15, hd_dep_count#16] +Output [7]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] +Input [13]: [ss_sold_date_sk#1, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, cd_marital_status#12, cd_education_status#13, hd_demo_sk#15, hd_dep_count#16] (16) Scan parquet default.date_dim Output [2]: [d_date_sk#18, d_year#19] @@ -130,13 +130,13 @@ Input [1]: [d_date_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#5] +Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#18] Join condition: None (22) Project [codegen id : 6] -Output [6]: [ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14] -Input [8]: [ss_sold_date_sk#5, ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14, d_date_sk#18] +Output [6]: [ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] +Input [8]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, d_date_sk#18] (23) Scan parquet default.store Output [1]: [s_store_sk#21] @@ -157,13 +157,13 @@ Input [1]: [s_store_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#9] +Left keys [1]: [ss_store_sk#5] Right keys [1]: [s_store_sk#21] Join condition: None (28) Project [codegen id : 6] -Output [5]: [ss_addr_sk#8, ss_quantity#10, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14] -Input [7]: [ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14, s_store_sk#21] +Output [5]: [ss_addr_sk#4, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] +Input [7]: [ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, s_store_sk#21] (29) Scan parquet default.customer_address Output [3]: [ca_address_sk#23, ca_state#24, ca_country#25] @@ -188,18 +188,18 @@ Input [2]: [ca_address_sk#23, ca_state#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#8] +Left keys [1]: [ss_addr_sk#4] Right keys [1]: [ca_address_sk#23] -Join condition: ((((ca_state#24 IN (TX,OH) AND (ss_net_profit#14 >= 100.00)) AND (ss_net_profit#14 <= 200.00)) OR ((ca_state#24 IN (OR,NM,KY) AND (ss_net_profit#14 >= 150.00)) AND (ss_net_profit#14 <= 300.00))) OR ((ca_state#24 IN (VA,TX,MS) AND (ss_net_profit#14 >= 50.00)) AND (ss_net_profit#14 <= 250.00))) +Join condition: ((((ca_state#24 IN (TX,OH) AND (ss_net_profit#10 >= 100.00)) AND (ss_net_profit#10 <= 200.00)) OR ((ca_state#24 IN (OR,NM,KY) AND (ss_net_profit#10 >= 150.00)) AND (ss_net_profit#10 <= 300.00))) OR ((ca_state#24 IN (VA,TX,MS) AND (ss_net_profit#10 >= 50.00)) AND (ss_net_profit#10 <= 250.00))) (35) Project [codegen id : 6] -Output [3]: [ss_quantity#10, ss_ext_sales_price#12, ss_ext_wholesale_cost#13] -Input [7]: [ss_addr_sk#8, ss_quantity#10, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14, ca_address_sk#23, ca_state#24] +Output [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] +Input [7]: [ss_addr_sk#4, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, ca_address_sk#23, ca_state#24] (36) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#10, ss_ext_sales_price#12, ss_ext_wholesale_cost#13] +Input [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] Keys: [] -Functions [4]: [partial_avg(cast(ss_quantity#10 as bigint)), partial_avg(UnscaledValue(ss_ext_sales_price#12)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#13)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#13))] +Functions [4]: [partial_avg(cast(ss_quantity#6 as bigint)), partial_avg(UnscaledValue(ss_ext_sales_price#8)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#9)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#9))] Aggregate Attributes [7]: [sum#27, count#28, sum#29, count#30, sum#31, count#32, sum#33] Results [7]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40] @@ -210,7 +210,7 @@ Arguments: SinglePartition, true, [id=#41] (38) HashAggregate [codegen id : 7] Input [7]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40] Keys: [] -Functions [4]: [avg(cast(ss_quantity#10 as bigint)), avg(UnscaledValue(ss_ext_sales_price#12)), avg(UnscaledValue(ss_ext_wholesale_cost#13)), sum(UnscaledValue(ss_ext_wholesale_cost#13))] -Aggregate Attributes [4]: [avg(cast(ss_quantity#10 as bigint))#42, avg(UnscaledValue(ss_ext_sales_price#12))#43, avg(UnscaledValue(ss_ext_wholesale_cost#13))#44, sum(UnscaledValue(ss_ext_wholesale_cost#13))#45] -Results [4]: [avg(cast(ss_quantity#10 as bigint))#42 AS avg(ss_quantity)#46, cast((avg(UnscaledValue(ss_ext_sales_price#12))#43 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#47, cast((avg(UnscaledValue(ss_ext_wholesale_cost#13))#44 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#48, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#13))#45,17,2) AS sum(ss_ext_wholesale_cost)#49] +Functions [4]: [avg(cast(ss_quantity#6 as bigint)), avg(UnscaledValue(ss_ext_sales_price#8)), avg(UnscaledValue(ss_ext_wholesale_cost#9)), sum(UnscaledValue(ss_ext_wholesale_cost#9))] +Aggregate Attributes [4]: [avg(cast(ss_quantity#6 as bigint))#42, avg(UnscaledValue(ss_ext_sales_price#8))#43, avg(UnscaledValue(ss_ext_wholesale_cost#9))#44, sum(UnscaledValue(ss_ext_wholesale_cost#9))#45] +Results [4]: [avg(cast(ss_quantity#6 as bigint))#42 AS avg(ss_quantity)#46, cast((avg(UnscaledValue(ss_ext_sales_price#8))#43 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#47, cast((avg(UnscaledValue(ss_ext_wholesale_cost#9))#44 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#48, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#9))#45,17,2) AS sum(ss_ext_wholesale_cost)#49] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt index e410b27e9cf3b..b457788dbd0b2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt @@ -12,8 +12,12 @@ WholeStageCodegen (7) BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [ss_sold_date_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] - Project [cd_marital_status,cd_education_status,ss_sold_date_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk,cd_marital_status,cd_education_status,ss_sales_price] + Project [ss_sold_date_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,cd_marital_status,cd_education_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + Filter [ss_store_sk,ss_addr_sk,ss_sold_date_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -21,10 +25,6 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - Filter [ss_store_sk,ss_addr_sk,ss_sold_date_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt index c3e9f9418cef5..b346701fa3148 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt @@ -517,15 +517,15 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, cou Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40, count(1)#41] -Results [7]: [store AS channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sales#43, count(1)#41 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sales#42, count(1)#41 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] (86) Filter [codegen id : 39] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45 as decimal(32,6)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 as decimal(32,6)) > cast(Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (87) Project [codegen id : 39] -Output [6]: [sales#43, number_sales#44, channel#42, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45] +Output [6]: [sales#42, number_sales#43, store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] (88) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] @@ -601,15 +601,15 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, cou Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] -Results [7]: [catalog AS channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#61, count(1)#59 AS number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] (105) Filter [codegen id : 78] -Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (106) Project [codegen id : 78] -Output [6]: [sales#61, number_sales#62, channel#60, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63] +Output [6]: [sales#60, number_sales#61, catalog AS channel#63, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] (107) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] @@ -685,26 +685,26 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, cou Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74, count(1)#75] -Results [7]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sales#77, count(1)#75 AS number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sales#76, count(1)#75 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] (124) Filter [codegen id : 117] -Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (125) Project [codegen id : 117] -Output [6]: [sales#77, number_sales#78, channel#76, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79] +Output [6]: [sales#76, number_sales#77, web AS channel#79, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] (126) Union (127) Expand [codegen id : 118] -Input [6]: [sales#43, number_sales#44, channel#42, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: [List(sales#43, number_sales#44, channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#43, number_sales#44, channel#42, i_brand_id#7, i_class_id#8, null, 1), List(sales#43, number_sales#44, channel#42, i_brand_id#7, null, null, 3), List(sales#43, number_sales#44, channel#42, null, null, null, 7), List(sales#43, number_sales#44, null, null, null, null, 15)], [sales#43, number_sales#44, channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] +Input [6]: [sales#42, number_sales#43, channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: [List(sales#42, number_sales#43, channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#42, number_sales#43, channel#47, i_brand_id#7, i_class_id#8, null, 1), List(sales#42, number_sales#43, channel#47, i_brand_id#7, null, null, 3), List(sales#42, number_sales#43, channel#47, null, null, null, 7), List(sales#42, number_sales#43, null, null, null, null, 15)], [sales#42, number_sales#43, channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] (128) HashAggregate [codegen id : 118] -Input [7]: [sales#43, number_sales#44, channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] +Input [7]: [sales#42, number_sales#43, channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] Keys [5]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] -Functions [2]: [partial_sum(sales#43), partial_sum(number_sales#44)] +Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#85, isEmpty#86, sum#87] Results [8]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84, sum#88, isEmpty#89, sum#90] @@ -715,9 +715,9 @@ Arguments: hashpartitioning(channel#80, i_brand_id#81, i_class_id#82, i_category (130) HashAggregate [codegen id : 119] Input [8]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84, sum#88, isEmpty#89, sum#90] Keys [5]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] -Functions [2]: [sum(sales#43), sum(number_sales#44)] -Aggregate Attributes [2]: [sum(sales#43)#92, sum(number_sales#44)#93] -Results [6]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, sum(sales#43)#92 AS sum(sales)#94, sum(number_sales#44)#93 AS sum(number_sales)#95] +Functions [2]: [sum(sales#42), sum(number_sales#43)] +Aggregate Attributes [2]: [sum(sales#42)#92, sum(number_sales#43)#93] +Results [6]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, sum(sales#42)#92 AS sum(sales)#94, sum(number_sales#43)#93 AS sum(number_sales)#95] (131) TakeOrderedAndProject Input [6]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, sum(sales)#94, sum(number_sales)#95] @@ -725,7 +725,7 @@ Arguments: 100, [channel#80 ASC NULLS FIRST, i_brand_id#81 ASC NULLS FIRST, i_cl ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#46, [id=#47] +Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#45, [id=#46] * HashAggregate (157) +- Exchange (156) +- * HashAggregate (155) @@ -871,8 +871,8 @@ Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as de Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#98 as decimal(12,2)))), DecimalType(18,2), true))#108] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#98 as decimal(12,2)))), DecimalType(18,2), true))#108 AS average_sales#109] -Subquery:2 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:2 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:3 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:3 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt index c6dbfcaa3fe43..5b93392d023db 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter Union WholeStageCodegen (39) - Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #1 WholeStageCodegen (8) @@ -53,7 +53,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [d_date_sk] #17 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 WholeStageCodegen (38) @@ -189,10 +189,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [ss_item_sk] #4 WholeStageCodegen (78) - Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #18 WholeStageCodegen (77) @@ -221,10 +221,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 WholeStageCodegen (117) - Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #20 WholeStageCodegen (116) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index c1b77321f16e6..3f0cc9e7acb1e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -461,15 +461,15 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, cou Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37, count(1)#38] -Results [7]: [store AS channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sales#40, count(1)#38 AS number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sales#39, count(1)#38 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] (76) Filter [codegen id : 26] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 as decimal(32,6)) > cast(Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41 as decimal(32,6)) > cast(Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [sales#40, number_sales#41, channel#39, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42] +Output [6]: [sales#39, number_sales#40, store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] (78) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] @@ -533,15 +533,15 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, cou Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55] -Results [7]: [catalog AS channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#57, count(1)#55 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] (92) Filter [codegen id : 52] -Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [sales#57, number_sales#58, channel#56, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59] +Output [6]: [sales#56, number_sales#57, catalog AS channel#59, i_brand_id#6, i_class_id#7, i_category_id#8] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] (94) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] @@ -605,26 +605,26 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, cou Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69, count(1)#70] -Results [7]: [web AS channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sales#72, count(1)#70 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sales#71, count(1)#70 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] (108) Filter [codegen id : 78] -Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [sales#72, number_sales#73, channel#71, i_brand_id#6, i_class_id#7, i_category_id#8] -Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74] +Output [6]: [sales#71, number_sales#72, web AS channel#74, i_brand_id#6, i_class_id#7, i_category_id#8] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] (110) Union (111) Expand [codegen id : 79] -Input [6]: [sales#40, number_sales#41, channel#39, i_brand_id#6, i_class_id#7, i_category_id#8] -Arguments: [List(sales#40, number_sales#41, channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, 0), List(sales#40, number_sales#41, channel#39, i_brand_id#6, i_class_id#7, null, 1), List(sales#40, number_sales#41, channel#39, i_brand_id#6, null, null, 3), List(sales#40, number_sales#41, channel#39, null, null, null, 7), List(sales#40, number_sales#41, null, null, null, null, 15)], [sales#40, number_sales#41, channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] +Input [6]: [sales#39, number_sales#40, channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] +Arguments: [List(sales#39, number_sales#40, channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 0), List(sales#39, number_sales#40, channel#44, i_brand_id#6, i_class_id#7, null, 1), List(sales#39, number_sales#40, channel#44, i_brand_id#6, null, null, 3), List(sales#39, number_sales#40, channel#44, null, null, null, 7), List(sales#39, number_sales#40, null, null, null, null, 15)], [sales#39, number_sales#40, channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] (112) HashAggregate [codegen id : 79] -Input [7]: [sales#40, number_sales#41, channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] +Input [7]: [sales#39, number_sales#40, channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] Keys [5]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] -Functions [2]: [partial_sum(sales#40), partial_sum(number_sales#41)] +Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#80, isEmpty#81, sum#82] Results [8]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79, sum#83, isEmpty#84, sum#85] @@ -635,9 +635,9 @@ Arguments: hashpartitioning(channel#75, i_brand_id#76, i_class_id#77, i_category (114) HashAggregate [codegen id : 80] Input [8]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79, sum#83, isEmpty#84, sum#85] Keys [5]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] -Functions [2]: [sum(sales#40), sum(number_sales#41)] -Aggregate Attributes [2]: [sum(sales#40)#87, sum(number_sales#41)#88] -Results [6]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, sum(sales#40)#87 AS sum(sales)#89, sum(number_sales#41)#88 AS sum(number_sales)#90] +Functions [2]: [sum(sales#39), sum(number_sales#40)] +Aggregate Attributes [2]: [sum(sales#39)#87, sum(number_sales#40)#88] +Results [6]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, sum(sales#39)#87 AS sum(sales)#89, sum(number_sales#40)#88 AS sum(number_sales)#90] (115) TakeOrderedAndProject Input [6]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, sum(sales)#89, sum(number_sales)#90] @@ -645,7 +645,7 @@ Arguments: 100, [channel#75 ASC NULLS FIRST, i_brand_id#76 ASC NULLS FIRST, i_cl ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#43, [id=#44] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#42, [id=#43] * HashAggregate (141) +- Exchange (140) +- * HashAggregate (139) @@ -791,8 +791,8 @@ Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as de Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#103] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#103 AS average_sales#104] -Subquery:2 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] +Subquery:2 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:3 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] +Subquery:3 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index 604bd792f5ffd..dfa8c1bcc1579 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -9,7 +9,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter Union WholeStageCodegen (26) - Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #1 WholeStageCodegen (8) @@ -53,7 +53,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [d_date_sk] #14 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 WholeStageCodegen (25) @@ -165,10 +165,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (52) - Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #15 WholeStageCodegen (51) @@ -189,10 +189,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [d_date_sk] #12 WholeStageCodegen (78) - Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #16 WholeStageCodegen (77) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt index f71ceaaf91f47..2d2b56e32bdb8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt @@ -496,15 +496,15 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, cou Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42, count(1)#43] -Results [7]: [store AS channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sales#45, count(1)#43 AS number_sales#46, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sales#44, count(1)#43 AS number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] (86) Filter [codegen id : 78] -Input [7]: [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47 as decimal(32,6)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46 as decimal(32,6)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) (87) Project [codegen id : 78] -Output [6]: [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46] -Input [7]: [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47] +Output [6]: [store AS channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] (88) ReusedExchange [Reuses operator id: 4] Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] @@ -584,18 +584,18 @@ Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] -Results [7]: [store AS channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#67, count(1)#65 AS number_sales#68, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69] +Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] (106) Filter [codegen id : 77] -Input [7]: [channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#48, [id=#49] as decimal(32,6)))) +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) (107) Project [codegen id : 77] -Output [6]: [channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68] -Input [7]: [channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69] +Output [6]: [store AS channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] (108) BroadcastExchange -Input [6]: [channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68] +Input [6]: [channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] (109) BroadcastHashJoin [codegen id : 78] @@ -604,12 +604,12 @@ Right keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Join condition: None (110) TakeOrderedAndProject -Input [12]: [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46, channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68] -Arguments: 100, [channel#44 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46, channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68] +Input [12]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] +Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#47, [id=#48] * HashAggregate (136) +- Exchange (135) +- * HashAggregate (134) @@ -780,7 +780,7 @@ Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_d Output [1]: [d_week_seq#29] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#48, [id=#49] +Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] Subquery:4 Hosting operator id = 95 Hosting Expression = Subquery scalar-subquery#50, [id=#51] * Project (144) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt index 37186560cb3b8..d6b8ba4395d2e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] +TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] WholeStageCodegen (78) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #2 WholeStageCodegen (8) @@ -45,7 +45,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_ Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [d_date_sk] #16 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 WholeStageCodegen (38) @@ -190,10 +190,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_ InputAdapter BroadcastExchange #17 WholeStageCodegen (77) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #2 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #18 WholeStageCodegen (76) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index b68ce0e9f2264..1f31ded51f1ef 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -446,15 +446,15 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, cou Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] -Results [7]: [store AS channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#42, count(1)#40 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] (76) Filter [codegen id : 52] -Input [7]: [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 as decimal(32,6)) > cast(Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (77) Project [codegen id : 52] -Output [6]: [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43] -Input [7]: [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] +Output [6]: [store AS channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] (78) Scan parquet default.store_sales Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] @@ -537,18 +537,18 @@ Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] -Results [7]: [store AS channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#64, count(1)#62 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66] +Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] (96) Filter [codegen id : 51] -Input [7]: [channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) +Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (97) Project [codegen id : 51] -Output [6]: [channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65] -Input [7]: [channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66] +Output [6]: [store AS channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] +Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] (98) BroadcastExchange -Input [6]: [channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65] +Input [6]: [channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#67] (99) BroadcastHashJoin [codegen id : 52] @@ -557,12 +557,12 @@ Right keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Join condition: None (100) TakeOrderedAndProject -Input [12]: [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43, channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65] -Arguments: 100, [channel#41 ASC NULLS FIRST, i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43, channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65] +Input [12]: [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] +Arguments: 100, [i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] * HashAggregate (126) +- Exchange (125) +- * HashAggregate (124) @@ -733,7 +733,7 @@ Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_d Output [1]: [d_week_seq#28] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -Subquery:3 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:3 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] Subquery:4 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#51, [id=#52] * Project (134) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index 6e6950d4cb33a..7bbf83e3de707 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] +TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] WholeStageCodegen (52) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #2 WholeStageCodegen (8) @@ -45,7 +45,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_ Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [d_date_sk] #13 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 WholeStageCodegen (25) @@ -166,10 +166,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_ InputAdapter BroadcastExchange #14 WholeStageCodegen (51) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #2 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #15 WholeStageCodegen (50) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index e24b656e843aa..a17356ae04a03 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -39,15 +39,15 @@ TakeOrderedAndProject (57) : +- * Sort (39) : +- Exchange (38) : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildLeft (36) - : :- BroadcastExchange (32) - : : +- * Project (31) - : : +- * Filter (30) - : : +- * ColumnarToRow (29) - : : +- Scan parquet default.date_dim (28) - : +- * Filter (35) - : +- * ColumnarToRow (34) - : +- Scan parquet default.store_returns (33) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Filter (30) + : : +- * ColumnarToRow (29) + : : +- Scan parquet default.store_returns (28) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.date_dim (31) +- * Sort (51) +- Exchange (50) +- * Project (49) @@ -177,75 +177,75 @@ Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 Input [7]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16] Arguments: [cast(ss_customer_sk#3 as bigint) ASC NULLS FIRST, cast(ss_item_sk#2 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#5 as bigint) ASC NULLS FIRST], false, 0 -(28) Scan parquet default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#20] +(28) Scan parquet default.store_returns +Output [5]: [sr_returned_date_sk#19, sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] +ReadSchema: struct -(29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#19, d_quarter_name#20] +(29) ColumnarToRow [codegen id : 10] +Input [5]: [sr_returned_date_sk#19, sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] -(30) Filter [codegen id : 9] -Input [2]: [d_date_sk#19, d_quarter_name#20] -Condition : (d_quarter_name#20 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#19)) +(30) Filter [codegen id : 10] +Input [5]: [sr_returned_date_sk#19, sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] +Condition : (((isnotnull(sr_customer_sk#21) AND isnotnull(sr_item_sk#20)) AND isnotnull(sr_ticket_number#22)) AND isnotnull(sr_returned_date_sk#19)) -(31) Project [codegen id : 9] -Output [1]: [d_date_sk#19] -Input [2]: [d_date_sk#19, d_quarter_name#20] +(31) Scan parquet default.date_dim +Output [2]: [d_date_sk#24, d_quarter_name#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] +ReadSchema: struct -(32) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +(32) ColumnarToRow [codegen id : 9] +Input [2]: [d_date_sk#24, d_quarter_name#25] -(33) Scan parquet default.store_returns -Output [5]: [sr_returned_date_sk#22, sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] -ReadSchema: struct +(33) Filter [codegen id : 9] +Input [2]: [d_date_sk#24, d_quarter_name#25] +Condition : (d_quarter_name#25 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) -(34) ColumnarToRow -Input [5]: [sr_returned_date_sk#22, sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] +(34) Project [codegen id : 9] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_quarter_name#25] -(35) Filter -Input [5]: [sr_returned_date_sk#22, sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] -Condition : (((isnotnull(sr_customer_sk#24) AND isnotnull(sr_item_sk#23)) AND isnotnull(sr_ticket_number#25)) AND isnotnull(sr_returned_date_sk#22)) +(35) BroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cast(d_date_sk#19 as bigint)] -Right keys [1]: [sr_returned_date_sk#22] +Left keys [1]: [sr_returned_date_sk#19] +Right keys [1]: [cast(d_date_sk#24 as bigint)] Join condition: None (37) Project [codegen id : 10] -Output [4]: [sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] -Input [6]: [d_date_sk#19, sr_returned_date_sk#22, sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] +Output [4]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] +Input [6]: [sr_returned_date_sk#19, sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, d_date_sk#24] (38) Exchange -Input [4]: [sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] -Arguments: hashpartitioning(sr_customer_sk#24, sr_item_sk#23, sr_ticket_number#25, 5), true, [id=#27] +Input [4]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] +Arguments: hashpartitioning(sr_customer_sk#21, sr_item_sk#20, sr_ticket_number#22, 5), true, [id=#27] (39) Sort [codegen id : 11] -Input [4]: [sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] -Arguments: [sr_customer_sk#24 ASC NULLS FIRST, sr_item_sk#23 ASC NULLS FIRST, sr_ticket_number#25 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] +Arguments: [sr_customer_sk#21 ASC NULLS FIRST, sr_item_sk#20 ASC NULLS FIRST, sr_ticket_number#22 ASC NULLS FIRST], false, 0 (40) SortMergeJoin [codegen id : 12] Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] -Right keys [3]: [sr_customer_sk#24, sr_item_sk#23, sr_ticket_number#25] +Right keys [3]: [sr_customer_sk#21, sr_item_sk#20, sr_ticket_number#22] Join condition: None (41) Project [codegen id : 12] -Output [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#23, sr_customer_sk#24, sr_return_quantity#26] -Input [11]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] +Output [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23] +Input [11]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] (42) Exchange -Input [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#23, sr_customer_sk#24, sr_return_quantity#26] -Arguments: hashpartitioning(sr_customer_sk#24, sr_item_sk#23, 5), true, [id=#28] +Input [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23] +Arguments: hashpartitioning(sr_customer_sk#21, sr_item_sk#20, 5), true, [id=#28] (43) Sort [codegen id : 13] -Input [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#23, sr_customer_sk#24, sr_return_quantity#26] -Arguments: [sr_customer_sk#24 ASC NULLS FIRST, sr_item_sk#23 ASC NULLS FIRST], false, 0 +Input [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23] +Arguments: [sr_customer_sk#21 ASC NULLS FIRST, sr_item_sk#20 ASC NULLS FIRST], false, 0 (44) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] @@ -261,7 +261,7 @@ Input [4]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quanti Input [4]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] Condition : ((isnotnull(cs_bill_customer_sk#30) AND isnotnull(cs_item_sk#31)) AND isnotnull(cs_sold_date_sk#29)) -(47) ReusedExchange [Reuses operator id: 32] +(47) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#33] (48) BroadcastHashJoin [codegen id : 15] @@ -282,18 +282,18 @@ Input [3]: [cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] Arguments: [cast(cs_bill_customer_sk#30 as bigint) ASC NULLS FIRST, cast(cs_item_sk#31 as bigint) ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 17] -Left keys [2]: [sr_customer_sk#24, sr_item_sk#23] +Left keys [2]: [sr_customer_sk#21, sr_item_sk#20] Right keys [2]: [cast(cs_bill_customer_sk#30 as bigint), cast(cs_item_sk#31 as bigint)] Join condition: None (53) Project [codegen id : 17] -Output [6]: [ss_quantity#6, sr_return_quantity#26, cs_quantity#32, s_state#11, i_item_id#15, i_item_desc#16] -Input [10]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#23, sr_customer_sk#24, sr_return_quantity#26, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] +Output [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#32, s_state#11, i_item_id#15, i_item_desc#16] +Input [10]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] (54) HashAggregate [codegen id : 17] -Input [6]: [ss_quantity#6, sr_return_quantity#26, cs_quantity#32, s_state#11, i_item_id#15, i_item_desc#16] +Input [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#32, s_state#11, i_item_id#15, i_item_desc#16] Keys [3]: [i_item_id#15, i_item_desc#16, s_state#11] -Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double)), partial_count(sr_return_quantity#26), partial_avg(cast(sr_return_quantity#26 as bigint)), partial_stddev_samp(cast(sr_return_quantity#26 as double)), partial_count(cs_quantity#32), partial_avg(cast(cs_quantity#32 as bigint)), partial_stddev_samp(cast(cs_quantity#32 as double))] +Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double)), partial_count(sr_return_quantity#23), partial_avg(cast(sr_return_quantity#23 as bigint)), partial_stddev_samp(cast(sr_return_quantity#23 as double)), partial_count(cs_quantity#32), partial_avg(cast(cs_quantity#32 as bigint)), partial_stddev_samp(cast(cs_quantity#32 as double))] Aggregate Attributes [18]: [count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46, count#47, sum#48, count#49, n#50, avg#51, m2#52] Results [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] @@ -304,9 +304,9 @@ Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_state#11, 5), true, (56) HashAggregate [codegen id : 18] Input [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] Keys [3]: [i_item_id#15, i_item_desc#16, s_state#11] -Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double)), count(sr_return_quantity#26), avg(cast(sr_return_quantity#26 as bigint)), stddev_samp(cast(sr_return_quantity#26 as double)), count(cs_quantity#32), avg(cast(cs_quantity#32 as bigint)), stddev_samp(cast(cs_quantity#32 as double))] -Aggregate Attributes [9]: [count(ss_quantity#6)#72, avg(cast(ss_quantity#6 as bigint))#73, stddev_samp(cast(ss_quantity#6 as double))#74, count(sr_return_quantity#26)#75, avg(cast(sr_return_quantity#26 as bigint))#76, stddev_samp(cast(sr_return_quantity#26 as double))#77, count(cs_quantity#32)#78, avg(cast(cs_quantity#32 as bigint))#79, stddev_samp(cast(cs_quantity#32 as double))#80] -Results [15]: [i_item_id#15, i_item_desc#16, s_state#11, count(ss_quantity#6)#72 AS store_sales_quantitycount#81, avg(cast(ss_quantity#6 as bigint))#73 AS store_sales_quantityave#82, stddev_samp(cast(ss_quantity#6 as double))#74 AS store_sales_quantitystdev#83, (stddev_samp(cast(ss_quantity#6 as double))#74 / avg(cast(ss_quantity#6 as bigint))#73) AS store_sales_quantitycov#84, count(sr_return_quantity#26)#75 AS as_store_returns_quantitycount#85, avg(cast(sr_return_quantity#26 as bigint))#76 AS as_store_returns_quantityave#86, stddev_samp(cast(sr_return_quantity#26 as double))#77 AS as_store_returns_quantitystdev#87, (stddev_samp(cast(sr_return_quantity#26 as double))#77 / avg(cast(sr_return_quantity#26 as bigint))#76) AS store_returns_quantitycov#88, count(cs_quantity#32)#78 AS catalog_sales_quantitycount#89, avg(cast(cs_quantity#32 as bigint))#79 AS catalog_sales_quantityave#90, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitystdev#91, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitycov#92] +Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double)), count(sr_return_quantity#23), avg(cast(sr_return_quantity#23 as bigint)), stddev_samp(cast(sr_return_quantity#23 as double)), count(cs_quantity#32), avg(cast(cs_quantity#32 as bigint)), stddev_samp(cast(cs_quantity#32 as double))] +Aggregate Attributes [9]: [count(ss_quantity#6)#72, avg(cast(ss_quantity#6 as bigint))#73, stddev_samp(cast(ss_quantity#6 as double))#74, count(sr_return_quantity#23)#75, avg(cast(sr_return_quantity#23 as bigint))#76, stddev_samp(cast(sr_return_quantity#23 as double))#77, count(cs_quantity#32)#78, avg(cast(cs_quantity#32 as bigint))#79, stddev_samp(cast(cs_quantity#32 as double))#80] +Results [15]: [i_item_id#15, i_item_desc#16, s_state#11, count(ss_quantity#6)#72 AS store_sales_quantitycount#81, avg(cast(ss_quantity#6 as bigint))#73 AS store_sales_quantityave#82, stddev_samp(cast(ss_quantity#6 as double))#74 AS store_sales_quantitystdev#83, (stddev_samp(cast(ss_quantity#6 as double))#74 / avg(cast(ss_quantity#6 as bigint))#73) AS store_sales_quantitycov#84, count(sr_return_quantity#23)#75 AS as_store_returns_quantitycount#85, avg(cast(sr_return_quantity#23 as bigint))#76 AS as_store_returns_quantityave#86, stddev_samp(cast(sr_return_quantity#23 as double))#77 AS as_store_returns_quantitystdev#87, (stddev_samp(cast(sr_return_quantity#23 as double))#77 / avg(cast(sr_return_quantity#23 as bigint))#76) AS store_returns_quantitycov#88, count(cs_quantity#32)#78 AS catalog_sales_quantitycount#89, avg(cast(cs_quantity#32 as bigint))#79 AS catalog_sales_quantityave#90, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitystdev#91, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitycov#92] (57) TakeOrderedAndProject Input [15]: [i_item_id#15, i_item_desc#16, s_state#11, store_sales_quantitycount#81, store_sales_quantityave#82, store_sales_quantitystdev#83, store_sales_quantitycov#84, as_store_returns_quantitycount#85, as_store_returns_quantityave#86, as_store_returns_quantitystdev#87, store_returns_quantitycov#88, catalog_sales_quantitycount#89, catalog_sales_quantityave#90, catalog_sales_quantitystdev#91, catalog_sales_quantitycov#92] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt index 216adf3588eca..bfb59441f483b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt @@ -69,7 +69,11 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 WholeStageCodegen (10) Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] InputAdapter BroadcastExchange #9 WholeStageCodegen (9) @@ -78,10 +82,6 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_quarter_name] - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] InputAdapter WholeStageCodegen (16) Sort [cs_bill_customer_sk,cs_item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt index 0fbe0ccef6d13..88b5168f6049c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt @@ -12,15 +12,15 @@ TakeOrderedAndProject (45) : :- * Project (17) : : +- * BroadcastHashJoin Inner BuildRight (16) : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : :- BroadcastExchange (5) - : : : : +- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.item (1) - : : : +- * Filter (8) - : : : +- * ColumnarToRow (7) - : : : +- Scan parquet default.store_sales (6) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.item (4) : : +- BroadcastExchange (15) : : +- * Project (14) : : +- * Filter (13) @@ -38,58 +38,58 @@ TakeOrderedAndProject (45) : +- Exchange (29) : +- * Filter (28) : +- * ColumnarToRow (27) - : +- Scan parquet default.customer_address (26) + : +- Scan parquet default.customer (26) +- * Sort (35) +- Exchange (34) +- * Filter (33) +- * ColumnarToRow (32) - +- Scan parquet default.customer (31) + +- Scan parquet default.customer_address (31) -(1) Scan parquet default.item -Output [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, i_manager_id#6] +(1) Scan parquet default.store_sales +Output [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, i_manager_id#6] +(2) ColumnarToRow [codegen id : 4] +Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] -(3) Filter [codegen id : 1] -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, i_manager_id#6] -Condition : ((isnotnull(i_manager_id#6) AND (i_manager_id#6 = 8)) AND isnotnull(i_item_sk#1)) +(3) Filter [codegen id : 4] +Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] +Condition : (((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_store_sk#4)) -(4) Project [codegen id : 1] -Output [5]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5] -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, i_manager_id#6] +(4) Scan parquet default.item +Output [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [5]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] +(5) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] -(6) Scan parquet default.store_sales -Output [5]: [ss_sold_date_sk#8, ss_item_sk#9, ss_customer_sk#10, ss_store_sk#11, ss_ext_sales_price#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] +Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 8)) AND isnotnull(i_item_sk#6)) -(7) ColumnarToRow -Input [5]: [ss_sold_date_sk#8, ss_item_sk#9, ss_customer_sk#10, ss_store_sk#11, ss_ext_sales_price#12] +(7) Project [codegen id : 1] +Output [5]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] -(8) Filter -Input [5]: [ss_sold_date_sk#8, ss_item_sk#9, ss_customer_sk#10, ss_store_sk#11, ss_ext_sales_price#12] -Condition : (((isnotnull(ss_sold_date_sk#8) AND isnotnull(ss_item_sk#9)) AND isnotnull(ss_customer_sk#10)) AND isnotnull(ss_store_sk#11)) +(8) BroadcastExchange +Input [5]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#9] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#6] Join condition: None (10) Project [codegen id : 4] -Output [8]: [i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_sold_date_sk#8, ss_customer_sk#10, ss_store_sk#11, ss_ext_sales_price#12] -Input [10]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_sold_date_sk#8, ss_item_sk#9, ss_customer_sk#10, ss_store_sk#11, ss_ext_sales_price#12] +Output [8]: [ss_sold_date_sk#1, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Input [10]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] (11) Scan parquet default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] @@ -114,13 +114,13 @@ Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#8] +Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#13] Join condition: None (17) Project [codegen id : 4] -Output [7]: [i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_customer_sk#10, ss_store_sk#11, ss_ext_sales_price#12] -Input [9]: [i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_sold_date_sk#8, ss_customer_sk#10, ss_store_sk#11, ss_ext_sales_price#12, d_date_sk#13] +Output [7]: [ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Input [9]: [ss_sold_date_sk#1, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, d_date_sk#13] (18) Scan parquet default.store Output [2]: [s_store_sk#17, s_zip#18] @@ -141,111 +141,111 @@ Input [2]: [s_store_sk#17, s_zip#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] (22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#11] +Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#17] Join condition: None (23) Project [codegen id : 4] -Output [7]: [i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_customer_sk#10, ss_ext_sales_price#12, s_zip#18] -Input [9]: [i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_customer_sk#10, ss_store_sk#11, ss_ext_sales_price#12, s_store_sk#17, s_zip#18] +Output [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] +Input [9]: [ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_store_sk#17, s_zip#18] (24) Exchange -Input [7]: [i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_customer_sk#10, ss_ext_sales_price#12, s_zip#18] -Arguments: hashpartitioning(ss_customer_sk#10, 5), true, [id=#20] +Input [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] +Arguments: hashpartitioning(ss_customer_sk#3, 5), true, [id=#20] (25) Sort [codegen id : 5] -Input [7]: [i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_customer_sk#10, ss_ext_sales_price#12, s_zip#18] -Arguments: [ss_customer_sk#10 ASC NULLS FIRST], false, 0 +Input [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] +Arguments: [ss_customer_sk#3 ASC NULLS FIRST], false, 0 -(26) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_zip#22] +(26) Scan parquet default.customer +Output [2]: [c_customer_sk#21, c_current_addr_sk#22] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_zip)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct (27) ColumnarToRow [codegen id : 6] -Input [2]: [ca_address_sk#21, ca_zip#22] +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] (28) Filter [codegen id : 6] -Input [2]: [ca_address_sk#21, ca_zip#22] -Condition : (isnotnull(ca_address_sk#21) AND isnotnull(ca_zip#22)) +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] +Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#22)) (29) Exchange -Input [2]: [ca_address_sk#21, ca_zip#22] -Arguments: hashpartitioning(ca_address_sk#21, 5), true, [id=#23] +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] +Arguments: hashpartitioning(c_current_addr_sk#22, 5), true, [id=#23] (30) Sort [codegen id : 7] -Input [2]: [ca_address_sk#21, ca_zip#22] -Arguments: [ca_address_sk#21 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] +Arguments: [c_current_addr_sk#22 ASC NULLS FIRST], false, 0 -(31) Scan parquet default.customer -Output [2]: [c_customer_sk#24, c_current_addr_sk#25] +(31) Scan parquet default.customer_address +Output [2]: [ca_address_sk#24, ca_zip#25] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_zip)] +ReadSchema: struct (32) ColumnarToRow [codegen id : 8] -Input [2]: [c_customer_sk#24, c_current_addr_sk#25] +Input [2]: [ca_address_sk#24, ca_zip#25] (33) Filter [codegen id : 8] -Input [2]: [c_customer_sk#24, c_current_addr_sk#25] -Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_current_addr_sk#25)) +Input [2]: [ca_address_sk#24, ca_zip#25] +Condition : (isnotnull(ca_address_sk#24) AND isnotnull(ca_zip#25)) (34) Exchange -Input [2]: [c_customer_sk#24, c_current_addr_sk#25] -Arguments: hashpartitioning(c_current_addr_sk#25, 5), true, [id=#26] +Input [2]: [ca_address_sk#24, ca_zip#25] +Arguments: hashpartitioning(ca_address_sk#24, 5), true, [id=#26] (35) Sort [codegen id : 9] -Input [2]: [c_customer_sk#24, c_current_addr_sk#25] -Arguments: [c_current_addr_sk#25 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#24, ca_zip#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 (36) SortMergeJoin [codegen id : 10] -Left keys [1]: [ca_address_sk#21] -Right keys [1]: [c_current_addr_sk#25] +Left keys [1]: [c_current_addr_sk#22] +Right keys [1]: [ca_address_sk#24] Join condition: None (37) Project [codegen id : 10] -Output [2]: [ca_zip#22, c_customer_sk#24] -Input [4]: [ca_address_sk#21, ca_zip#22, c_customer_sk#24, c_current_addr_sk#25] +Output [2]: [c_customer_sk#21, ca_zip#25] +Input [4]: [c_customer_sk#21, c_current_addr_sk#22, ca_address_sk#24, ca_zip#25] (38) Exchange -Input [2]: [ca_zip#22, c_customer_sk#24] -Arguments: hashpartitioning(c_customer_sk#24, 5), true, [id=#27] +Input [2]: [c_customer_sk#21, ca_zip#25] +Arguments: hashpartitioning(c_customer_sk#21, 5), true, [id=#27] (39) Sort [codegen id : 11] -Input [2]: [ca_zip#22, c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#21, ca_zip#25] +Arguments: [c_customer_sk#21 ASC NULLS FIRST], false, 0 (40) SortMergeJoin [codegen id : 12] -Left keys [1]: [ss_customer_sk#10] -Right keys [1]: [c_customer_sk#24] -Join condition: NOT (substr(ca_zip#22, 1, 5) = substr(s_zip#18, 1, 5)) +Left keys [1]: [ss_customer_sk#3] +Right keys [1]: [c_customer_sk#21] +Join condition: NOT (substr(ca_zip#25, 1, 5) = substr(s_zip#18, 1, 5)) (41) Project [codegen id : 12] -Output [5]: [ss_ext_sales_price#12, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5] -Input [9]: [i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_customer_sk#10, ss_ext_sales_price#12, s_zip#18, ca_zip#22, c_customer_sk#24] +Output [5]: [ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Input [9]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18, c_customer_sk#21, ca_zip#25] (42) HashAggregate [codegen id : 12] -Input [5]: [ss_ext_sales_price#12, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5] -Keys [4]: [i_brand#3, i_brand_id#2, i_manufact_id#4, i_manufact#5] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#12))] +Input [5]: [ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Keys [4]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#28] -Results [5]: [i_brand#3, i_brand_id#2, i_manufact_id#4, i_manufact#5, sum#29] +Results [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] (43) Exchange -Input [5]: [i_brand#3, i_brand_id#2, i_manufact_id#4, i_manufact#5, sum#29] -Arguments: hashpartitioning(i_brand#3, i_brand_id#2, i_manufact_id#4, i_manufact#5, 5), true, [id=#30] +Input [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] +Arguments: hashpartitioning(i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, 5), true, [id=#30] (44) HashAggregate [codegen id : 13] -Input [5]: [i_brand#3, i_brand_id#2, i_manufact_id#4, i_manufact#5, sum#29] -Keys [4]: [i_brand#3, i_brand_id#2, i_manufact_id#4, i_manufact#5] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#12))#31] -Results [5]: [i_brand_id#2 AS brand_id#32, i_brand#3 AS brand#33, i_manufact_id#4, i_manufact#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#12))#31,17,2) AS ext_price#34] +Input [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] +Keys [4]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#31] +Results [5]: [i_brand_id#7 AS brand_id#32, i_brand#8 AS brand#33, i_manufact_id#9, i_manufact#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#31,17,2) AS ext_price#34] (45) TakeOrderedAndProject -Input [5]: [brand_id#32, brand#33, i_manufact_id#4, i_manufact#5, ext_price#34] -Arguments: 100, [ext_price#34 DESC NULLS LAST, brand#33 ASC NULLS FIRST, brand_id#32 ASC NULLS FIRST, i_manufact_id#4 ASC NULLS FIRST, i_manufact#5 ASC NULLS FIRST], [brand_id#32, brand#33, i_manufact_id#4, i_manufact#5, ext_price#34] +Input [5]: [brand_id#32, brand#33, i_manufact_id#9, i_manufact#10, ext_price#34] +Arguments: 100, [ext_price#34 DESC NULLS LAST, brand#33 ASC NULLS FIRST, brand_id#32 ASC NULLS FIRST, i_manufact_id#9 ASC NULLS FIRST, i_manufact#10 ASC NULLS FIRST], [brand_id#32, brand#33, i_manufact_id#9, i_manufact#10, ext_price#34] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt index c8737d8a70782..05fa3f82e27df 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt @@ -13,12 +13,16 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] InputAdapter Exchange [ss_customer_sk] #2 WholeStageCodegen (4) - Project [i_brand_id,i_brand,i_manufact_id,i_manufact,ss_customer_sk,ss_ext_sales_price,s_zip] + Project [ss_customer_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,s_zip] BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand_id,i_brand,i_manufact_id,i_manufact,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand_id,i_brand,i_manufact_id,i_manufact,ss_sold_date_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] + Project [ss_sold_date_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -27,10 +31,6 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -52,25 +52,25 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] InputAdapter Exchange [c_customer_sk] #6 WholeStageCodegen (10) - Project [ca_zip,c_customer_sk] - SortMergeJoin [ca_address_sk,c_current_addr_sk] + Project [c_customer_sk,ca_zip] + SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (7) - Sort [ca_address_sk] + Sort [c_current_addr_sk] InputAdapter - Exchange [ca_address_sk] #7 + Exchange [c_current_addr_sk] #7 WholeStageCodegen (6) - Filter [ca_address_sk,ca_zip] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_zip] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter WholeStageCodegen (9) - Sort [c_current_addr_sk] + Sort [ca_address_sk] InputAdapter - Exchange [c_current_addr_sk] #8 + Exchange [ca_address_sk] #8 WholeStageCodegen (8) - Filter [c_customer_sk,c_current_addr_sk] + Filter [ca_address_sk,ca_zip] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] + Scan parquet default.customer_address [ca_address_sk,ca_zip] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt index 6e0a5ced1992a..ffcf6bd4f6d47 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt @@ -296,15 +296,15 @@ Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquer : : : :- * Sort (60) : : : : +- Exchange (59) : : : : +- * Project (58) - : : : : +- * BroadcastHashJoin Inner BuildLeft (57) - : : : : :- BroadcastExchange (53) - : : : : : +- * Project (52) - : : : : : +- * Filter (51) - : : : : : +- * ColumnarToRow (50) - : : : : : +- Scan parquet default.store (49) - : : : : +- * Filter (56) - : : : : +- * ColumnarToRow (55) - : : : : +- Scan parquet default.store_sales (54) + : : : : +- * BroadcastHashJoin Inner BuildRight (57) + : : : : :- * Filter (51) + : : : : : +- * ColumnarToRow (50) + : : : : : +- Scan parquet default.store_sales (49) + : : : : +- BroadcastExchange (56) + : : : : +- * Project (55) + : : : : +- * Filter (54) + : : : : +- * ColumnarToRow (53) + : : : : +- Scan parquet default.store (52) : : : +- * Sort (65) : : : +- Exchange (64) : : : +- * Filter (63) @@ -327,57 +327,57 @@ Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquer +- Scan parquet default.store_returns (88) -(49) Scan parquet default.store +(49) Scan parquet default.store_sales +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] +ReadSchema: struct + +(50) ColumnarToRow [codegen id : 2] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] + +(51) Filter [codegen id : 2] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) + +(52) Scan parquet default.store Output [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(50) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] -(51) Filter [codegen id : 1] +(54) Filter [codegen id : 1] Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] Condition : (((isnotnull(s_market_id#21) AND (s_market_id#21 = 8)) AND isnotnull(s_store_sk#19)) AND isnotnull(s_zip#23)) -(52) Project [codegen id : 1] +(55) Project [codegen id : 1] Output [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] -(53) BroadcastExchange +(56) BroadcastExchange Input [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] -(54) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(55) ColumnarToRow -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(56) Filter -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - (57) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [s_store_sk#19] -Right keys [1]: [ss_store_sk#3] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#19] Join condition: None (58) Project [codegen id : 2] -Output [7]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5] -Input [9]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] +Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] (59) Exchange -Input [7]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] Arguments: hashpartitioning(ss_item_sk#1, 5), true, [id=#49] (60) Sort [codegen id : 3] -Input [7]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (61) Scan parquet default.item @@ -408,15 +408,15 @@ Right keys [1]: [i_item_sk#6] Join condition: None (67) Project [codegen id : 6] -Output [12]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Input [13]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] (68) Exchange -Input [12]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#51] (69) Sort [codegen id : 7] -Input [12]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (70) Scan parquet default.customer @@ -447,15 +447,15 @@ Right keys [1]: [c_customer_sk#14] Join condition: None (76) Project [codegen id : 10] -Output [14]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [16]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [16]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (77) Exchange -Input [14]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), true, [id=#53] (78) Sort [codegen id : 11] -Input [14]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] Arguments: [c_birth_country#17 ASC NULLS FIRST, s_zip#23 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer_address @@ -486,15 +486,15 @@ Right keys [2]: [upper(ca_country#27), ca_zip#26] Join condition: None (85) Project [codegen id : 14] -Output [13]: [s_store_name#20, s_state#22, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [17]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17, ca_state#25, ca_zip#26, ca_country#27] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] +Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17, ca_state#25, ca_zip#26, ca_country#27] (86) Exchange -Input [13]: [s_store_name#20, s_state#22, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), true, [id=#55] (87) Sort [codegen id : 15] -Input [13]: [s_store_name#20, s_state#22, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (88) Scan parquet default.store_returns @@ -526,7 +526,7 @@ Join condition: None (94) Project [codegen id : 18] Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [s_store_name#20, s_state#22, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25, sr_item_sk#30, sr_ticket_number#31] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25, sr_item_sk#30, sr_ticket_number#31] (95) HashAggregate [codegen id : 18] Input [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt index f51d1972b630f..10f874f8f5543 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt @@ -21,7 +21,7 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_ticket_number,ss_item_sk] #12 WholeStageCodegen (14) - Project [s_store_name,s_state,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] SortMergeJoin [c_birth_country,s_zip,ca_country,ca_zip] InputAdapter WholeStageCodegen (11) @@ -29,7 +29,7 @@ WholeStageCodegen (14) InputAdapter Exchange [c_birth_country,s_zip] #13 WholeStageCodegen (10) - Project [s_store_name,s_state,s_zip,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (7) @@ -37,7 +37,7 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_customer_sk] #14 WholeStageCodegen (6) - Project [s_store_name,s_state,s_zip,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id] + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (3) @@ -45,8 +45,12 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_item_sk] #15 WholeStageCodegen (2) - Project [s_store_name,s_state,s_zip,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid] - BroadcastHashJoin [s_store_sk,ss_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter BroadcastExchange #16 WholeStageCodegen (1) @@ -55,10 +59,6 @@ WholeStageCodegen (14) ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter WholeStageCodegen (5) Sort [i_item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt index cbac3787cab6c..73f36e3a9ca23 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt @@ -296,15 +296,15 @@ Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquer : : : :- * Sort (60) : : : : +- Exchange (59) : : : : +- * Project (58) - : : : : +- * BroadcastHashJoin Inner BuildLeft (57) - : : : : :- BroadcastExchange (53) - : : : : : +- * Project (52) - : : : : : +- * Filter (51) - : : : : : +- * ColumnarToRow (50) - : : : : : +- Scan parquet default.store (49) - : : : : +- * Filter (56) - : : : : +- * ColumnarToRow (55) - : : : : +- Scan parquet default.store_sales (54) + : : : : +- * BroadcastHashJoin Inner BuildRight (57) + : : : : :- * Filter (51) + : : : : : +- * ColumnarToRow (50) + : : : : : +- Scan parquet default.store_sales (49) + : : : : +- BroadcastExchange (56) + : : : : +- * Project (55) + : : : : +- * Filter (54) + : : : : +- * ColumnarToRow (53) + : : : : +- Scan parquet default.store (52) : : : +- * Sort (65) : : : +- Exchange (64) : : : +- * Filter (63) @@ -327,57 +327,57 @@ Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquer +- Scan parquet default.store_returns (88) -(49) Scan parquet default.store +(49) Scan parquet default.store_sales +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] +ReadSchema: struct + +(50) ColumnarToRow [codegen id : 2] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] + +(51) Filter [codegen id : 2] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) + +(52) Scan parquet default.store Output [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(50) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] -(51) Filter [codegen id : 1] +(54) Filter [codegen id : 1] Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] Condition : (((isnotnull(s_market_id#21) AND (s_market_id#21 = 8)) AND isnotnull(s_store_sk#19)) AND isnotnull(s_zip#23)) -(52) Project [codegen id : 1] +(55) Project [codegen id : 1] Output [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] -(53) BroadcastExchange +(56) BroadcastExchange Input [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] -(54) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(55) ColumnarToRow -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(56) Filter -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - (57) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [s_store_sk#19] -Right keys [1]: [ss_store_sk#3] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#19] Join condition: None (58) Project [codegen id : 2] -Output [7]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5] -Input [9]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] +Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] (59) Exchange -Input [7]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] Arguments: hashpartitioning(ss_item_sk#1, 5), true, [id=#49] (60) Sort [codegen id : 3] -Input [7]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (61) Scan parquet default.item @@ -408,15 +408,15 @@ Right keys [1]: [i_item_sk#6] Join condition: None (67) Project [codegen id : 6] -Output [12]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Input [13]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] (68) Exchange -Input [12]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#51] (69) Sort [codegen id : 7] -Input [12]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (70) Scan parquet default.customer @@ -447,15 +447,15 @@ Right keys [1]: [c_customer_sk#14] Join condition: None (76) Project [codegen id : 10] -Output [14]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [16]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [16]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (77) Exchange -Input [14]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), true, [id=#53] (78) Sort [codegen id : 11] -Input [14]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] Arguments: [c_birth_country#17 ASC NULLS FIRST, s_zip#23 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer_address @@ -486,15 +486,15 @@ Right keys [2]: [upper(ca_country#27), ca_zip#26] Join condition: None (85) Project [codegen id : 14] -Output [13]: [s_store_name#20, s_state#22, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [17]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17, ca_state#25, ca_zip#26, ca_country#27] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] +Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17, ca_state#25, ca_zip#26, ca_country#27] (86) Exchange -Input [13]: [s_store_name#20, s_state#22, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), true, [id=#55] (87) Sort [codegen id : 15] -Input [13]: [s_store_name#20, s_state#22, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (88) Scan parquet default.store_returns @@ -526,7 +526,7 @@ Join condition: None (94) Project [codegen id : 18] Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [s_store_name#20, s_state#22, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25, sr_item_sk#30, sr_ticket_number#31] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25, sr_item_sk#30, sr_ticket_number#31] (95) HashAggregate [codegen id : 18] Input [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt index f51d1972b630f..10f874f8f5543 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt @@ -21,7 +21,7 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_ticket_number,ss_item_sk] #12 WholeStageCodegen (14) - Project [s_store_name,s_state,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] SortMergeJoin [c_birth_country,s_zip,ca_country,ca_zip] InputAdapter WholeStageCodegen (11) @@ -29,7 +29,7 @@ WholeStageCodegen (14) InputAdapter Exchange [c_birth_country,s_zip] #13 WholeStageCodegen (10) - Project [s_store_name,s_state,s_zip,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (7) @@ -37,7 +37,7 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_customer_sk] #14 WholeStageCodegen (6) - Project [s_store_name,s_state,s_zip,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id] + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (3) @@ -45,8 +45,12 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_item_sk] #15 WholeStageCodegen (2) - Project [s_store_name,s_state,s_zip,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid] - BroadcastHashJoin [s_store_sk,ss_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter BroadcastExchange #16 WholeStageCodegen (1) @@ -55,10 +59,6 @@ WholeStageCodegen (14) ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter WholeStageCodegen (5) Sort [i_item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt index 87a72d3bbe777..c6dc3db869003 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt @@ -39,15 +39,15 @@ TakeOrderedAndProject (57) : +- * Sort (39) : +- Exchange (38) : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildLeft (36) - : :- BroadcastExchange (32) - : : +- * Project (31) - : : +- * Filter (30) - : : +- * ColumnarToRow (29) - : : +- Scan parquet default.date_dim (28) - : +- * Filter (35) - : +- * ColumnarToRow (34) - : +- Scan parquet default.store_returns (33) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Filter (30) + : : +- * ColumnarToRow (29) + : : +- Scan parquet default.store_returns (28) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.date_dim (31) +- * Sort (51) +- Exchange (50) +- * Project (49) @@ -177,75 +177,75 @@ Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] Arguments: [cast(ss_customer_sk#3 as bigint) ASC NULLS FIRST, cast(ss_item_sk#2 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#5 as bigint) ASC NULLS FIRST], false, 0 -(28) Scan parquet default.date_dim -Output [3]: [d_date_sk#21, d_year#22, d_moy#23] +(28) Scan parquet default.store_returns +Output [5]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] +ReadSchema: struct -(29) ColumnarToRow [codegen id : 9] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] +(29) ColumnarToRow [codegen id : 10] +Input [5]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] -(30) Filter [codegen id : 9] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] -Condition : (((((isnotnull(d_moy#23) AND isnotnull(d_year#22)) AND (d_moy#23 >= 4)) AND (d_moy#23 <= 10)) AND (d_year#22 = 2001)) AND isnotnull(d_date_sk#21)) +(30) Filter [codegen id : 10] +Input [5]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] +Condition : (((isnotnull(sr_customer_sk#23) AND isnotnull(sr_item_sk#22)) AND isnotnull(sr_ticket_number#24)) AND isnotnull(sr_returned_date_sk#21)) -(31) Project [codegen id : 9] -Output [1]: [d_date_sk#21] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] +(31) Scan parquet default.date_dim +Output [3]: [d_date_sk#26, d_year#27, d_moy#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct -(32) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +(32) ColumnarToRow [codegen id : 9] +Input [3]: [d_date_sk#26, d_year#27, d_moy#28] -(33) Scan parquet default.store_returns -Output [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] -ReadSchema: struct +(33) Filter [codegen id : 9] +Input [3]: [d_date_sk#26, d_year#27, d_moy#28] +Condition : (((((isnotnull(d_moy#28) AND isnotnull(d_year#27)) AND (d_moy#28 >= 4)) AND (d_moy#28 <= 10)) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) -(34) ColumnarToRow -Input [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] +(34) Project [codegen id : 9] +Output [1]: [d_date_sk#26] +Input [3]: [d_date_sk#26, d_year#27, d_moy#28] -(35) Filter -Input [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] -Condition : (((isnotnull(sr_customer_sk#27) AND isnotnull(sr_item_sk#26)) AND isnotnull(sr_ticket_number#28)) AND isnotnull(sr_returned_date_sk#25)) +(35) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cast(d_date_sk#21 as bigint)] -Right keys [1]: [sr_returned_date_sk#25] +Left keys [1]: [sr_returned_date_sk#21] +Right keys [1]: [cast(d_date_sk#26 as bigint)] Join condition: None (37) Project [codegen id : 10] -Output [4]: [sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] -Input [6]: [d_date_sk#21, sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] +Output [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] +Input [6]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, d_date_sk#26] (38) Exchange -Input [4]: [sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] -Arguments: hashpartitioning(sr_customer_sk#27, sr_item_sk#26, sr_ticket_number#28, 5), true, [id=#30] +Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 5), true, [id=#30] (39) Sort [codegen id : 11] -Input [4]: [sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] -Arguments: [sr_customer_sk#27 ASC NULLS FIRST, sr_item_sk#26 ASC NULLS FIRST, sr_ticket_number#28 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] +Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST, sr_ticket_number#24 ASC NULLS FIRST], false, 0 (40) SortMergeJoin [codegen id : 12] Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] -Right keys [3]: [sr_customer_sk#27, sr_item_sk#26, sr_ticket_number#28] +Right keys [3]: [sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24] Join condition: None (41) Project [codegen id : 12] -Output [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#26, sr_customer_sk#27, sr_net_loss#29] -Input [12]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] +Output [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] +Input [12]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] (42) Exchange -Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#26, sr_customer_sk#27, sr_net_loss#29] -Arguments: hashpartitioning(sr_customer_sk#27, sr_item_sk#26, 5), true, [id=#31] +Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, 5), true, [id=#31] (43) Sort [codegen id : 13] -Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#26, sr_customer_sk#27, sr_net_loss#29] -Arguments: [sr_customer_sk#27 ASC NULLS FIRST, sr_item_sk#26 ASC NULLS FIRST], false, 0 +Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] +Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST], false, 0 (44) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] @@ -261,7 +261,7 @@ Input [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_pr Input [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] Condition : ((isnotnull(cs_bill_customer_sk#33) AND isnotnull(cs_item_sk#34)) AND isnotnull(cs_sold_date_sk#32)) -(47) ReusedExchange [Reuses operator id: 32] +(47) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#36] (48) BroadcastHashJoin [codegen id : 15] @@ -282,18 +282,18 @@ Input [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] Arguments: [cast(cs_bill_customer_sk#33 as bigint) ASC NULLS FIRST, cast(cs_item_sk#34 as bigint) ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 17] -Left keys [2]: [sr_customer_sk#27, sr_item_sk#26] +Left keys [2]: [sr_customer_sk#23, sr_item_sk#22] Right keys [2]: [cast(cs_bill_customer_sk#33 as bigint), cast(cs_item_sk#34 as bigint)] Join condition: None (53) Project [codegen id : 17] -Output [7]: [ss_net_profit#6, sr_net_loss#29, cs_net_profit#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] -Input [11]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#26, sr_customer_sk#27, sr_net_loss#29, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +Output [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [11]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] (54) HashAggregate [codegen id : 17] -Input [7]: [ss_net_profit#6, sr_net_loss#29, cs_net_profit#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] Keys [4]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#6)), partial_sum(UnscaledValue(sr_net_loss#29)), partial_sum(UnscaledValue(cs_net_profit#35))] +Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#6)), partial_sum(UnscaledValue(sr_net_loss#25)), partial_sum(UnscaledValue(cs_net_profit#35))] Aggregate Attributes [3]: [sum#38, sum#39, sum#40] Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#41, sum#42, sum#43] @@ -304,9 +304,9 @@ Arguments: hashpartitioning(i_item_id#17, i_item_desc#18, s_store_id#12, s_store (56) HashAggregate [codegen id : 18] Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#41, sum#42, sum#43] Keys [4]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13] -Functions [3]: [sum(UnscaledValue(ss_net_profit#6)), sum(UnscaledValue(sr_net_loss#29)), sum(UnscaledValue(cs_net_profit#35))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#6))#45, sum(UnscaledValue(sr_net_loss#29))#46, sum(UnscaledValue(cs_net_profit#35))#47] -Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#6))#45,17,2) AS store_sales_profit#48, MakeDecimal(sum(UnscaledValue(sr_net_loss#29))#46,17,2) AS store_returns_loss#49, MakeDecimal(sum(UnscaledValue(cs_net_profit#35))#47,17,2) AS catalog_sales_profit#50] +Functions [3]: [sum(UnscaledValue(ss_net_profit#6)), sum(UnscaledValue(sr_net_loss#25)), sum(UnscaledValue(cs_net_profit#35))] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#6))#45, sum(UnscaledValue(sr_net_loss#25))#46, sum(UnscaledValue(cs_net_profit#35))#47] +Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#6))#45,17,2) AS store_sales_profit#48, MakeDecimal(sum(UnscaledValue(sr_net_loss#25))#46,17,2) AS store_returns_loss#49, MakeDecimal(sum(UnscaledValue(cs_net_profit#35))#47,17,2) AS catalog_sales_profit#50] (57) TakeOrderedAndProject Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, store_sales_profit#48, store_returns_loss#49, catalog_sales_profit#50] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt index 8e61cf9c519fd..ad9fa718ff2bd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt @@ -69,7 +69,11 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 WholeStageCodegen (10) Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] InputAdapter BroadcastExchange #9 WholeStageCodegen (9) @@ -78,10 +82,6 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] InputAdapter WholeStageCodegen (16) Sort [cs_bill_customer_sk,cs_item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt index 35e24698c517e..a949b93f3bcb0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt @@ -39,15 +39,15 @@ TakeOrderedAndProject (61) : +- * Sort (39) : +- Exchange (38) : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildLeft (36) - : :- BroadcastExchange (32) - : : +- * Project (31) - : : +- * Filter (30) - : : +- * ColumnarToRow (29) - : : +- Scan parquet default.date_dim (28) - : +- * Filter (35) - : +- * ColumnarToRow (34) - : +- Scan parquet default.store_returns (33) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Filter (30) + : : +- * ColumnarToRow (29) + : : +- Scan parquet default.store_returns (28) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.date_dim (31) +- * Sort (55) +- Exchange (54) +- * Project (53) @@ -181,75 +181,75 @@ Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] Arguments: [cast(ss_customer_sk#3 as bigint) ASC NULLS FIRST, cast(ss_item_sk#2 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#5 as bigint) ASC NULLS FIRST], false, 0 -(28) Scan parquet default.date_dim -Output [3]: [d_date_sk#21, d_year#22, d_moy#23] +(28) Scan parquet default.store_returns +Output [5]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_return_quantity#25] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] +ReadSchema: struct -(29) ColumnarToRow [codegen id : 9] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] +(29) ColumnarToRow [codegen id : 10] +Input [5]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_return_quantity#25] -(30) Filter [codegen id : 9] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] -Condition : (((((isnotnull(d_moy#23) AND isnotnull(d_year#22)) AND (d_moy#23 >= 9)) AND (d_moy#23 <= 12)) AND (d_year#22 = 1999)) AND isnotnull(d_date_sk#21)) +(30) Filter [codegen id : 10] +Input [5]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_return_quantity#25] +Condition : (((isnotnull(sr_customer_sk#23) AND isnotnull(sr_item_sk#22)) AND isnotnull(sr_ticket_number#24)) AND isnotnull(sr_returned_date_sk#21)) -(31) Project [codegen id : 9] -Output [1]: [d_date_sk#21] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] +(31) Scan parquet default.date_dim +Output [3]: [d_date_sk#26, d_year#27, d_moy#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct -(32) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +(32) ColumnarToRow [codegen id : 9] +Input [3]: [d_date_sk#26, d_year#27, d_moy#28] -(33) Scan parquet default.store_returns -Output [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] -ReadSchema: struct +(33) Filter [codegen id : 9] +Input [3]: [d_date_sk#26, d_year#27, d_moy#28] +Condition : (((((isnotnull(d_moy#28) AND isnotnull(d_year#27)) AND (d_moy#28 >= 9)) AND (d_moy#28 <= 12)) AND (d_year#27 = 1999)) AND isnotnull(d_date_sk#26)) -(34) ColumnarToRow -Input [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] +(34) Project [codegen id : 9] +Output [1]: [d_date_sk#26] +Input [3]: [d_date_sk#26, d_year#27, d_moy#28] -(35) Filter -Input [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] -Condition : (((isnotnull(sr_customer_sk#27) AND isnotnull(sr_item_sk#26)) AND isnotnull(sr_ticket_number#28)) AND isnotnull(sr_returned_date_sk#25)) +(35) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cast(d_date_sk#21 as bigint)] -Right keys [1]: [sr_returned_date_sk#25] +Left keys [1]: [sr_returned_date_sk#21] +Right keys [1]: [cast(d_date_sk#26 as bigint)] Join condition: None (37) Project [codegen id : 10] -Output [4]: [sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] -Input [6]: [d_date_sk#21, sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] +Output [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_return_quantity#25] +Input [6]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_return_quantity#25, d_date_sk#26] (38) Exchange -Input [4]: [sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] -Arguments: hashpartitioning(sr_customer_sk#27, sr_item_sk#26, sr_ticket_number#28, 5), true, [id=#30] +Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_return_quantity#25] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 5), true, [id=#30] (39) Sort [codegen id : 11] -Input [4]: [sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] -Arguments: [sr_customer_sk#27 ASC NULLS FIRST, sr_item_sk#26 ASC NULLS FIRST, sr_ticket_number#28 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_return_quantity#25] +Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST, sr_ticket_number#24 ASC NULLS FIRST], false, 0 (40) SortMergeJoin [codegen id : 12] Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] -Right keys [3]: [sr_customer_sk#27, sr_item_sk#26, sr_ticket_number#28] +Right keys [3]: [sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24] Join condition: None (41) Project [codegen id : 12] -Output [8]: [ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#26, sr_customer_sk#27, sr_return_quantity#29] -Input [12]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] +Output [8]: [ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_return_quantity#25] +Input [12]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_return_quantity#25] (42) Exchange -Input [8]: [ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#26, sr_customer_sk#27, sr_return_quantity#29] -Arguments: hashpartitioning(sr_customer_sk#27, sr_item_sk#26, 5), true, [id=#31] +Input [8]: [ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_return_quantity#25] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, 5), true, [id=#31] (43) Sort [codegen id : 13] -Input [8]: [ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#26, sr_customer_sk#27, sr_return_quantity#29] -Arguments: [sr_customer_sk#27 ASC NULLS FIRST, sr_item_sk#26 ASC NULLS FIRST], false, 0 +Input [8]: [ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_return_quantity#25] +Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST], false, 0 (44) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_quantity#35] @@ -305,18 +305,18 @@ Input [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_quantity#35] Arguments: [cast(cs_bill_customer_sk#33 as bigint) ASC NULLS FIRST, cast(cs_item_sk#34 as bigint) ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 17] -Left keys [2]: [sr_customer_sk#27, sr_item_sk#26] +Left keys [2]: [sr_customer_sk#23, sr_item_sk#22] Right keys [2]: [cast(cs_bill_customer_sk#33 as bigint), cast(cs_item_sk#34 as bigint)] Join condition: None (57) Project [codegen id : 17] -Output [7]: [ss_quantity#6, sr_return_quantity#29, cs_quantity#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] -Input [11]: [ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#26, sr_customer_sk#27, sr_return_quantity#29, cs_bill_customer_sk#33, cs_item_sk#34, cs_quantity#35] +Output [7]: [ss_quantity#6, sr_return_quantity#25, cs_quantity#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [11]: [ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_return_quantity#25, cs_bill_customer_sk#33, cs_item_sk#34, cs_quantity#35] (58) HashAggregate [codegen id : 17] -Input [7]: [ss_quantity#6, sr_return_quantity#29, cs_quantity#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [7]: [ss_quantity#6, sr_return_quantity#25, cs_quantity#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] Keys [4]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13] -Functions [3]: [partial_sum(cast(ss_quantity#6 as bigint)), partial_sum(cast(sr_return_quantity#29 as bigint)), partial_sum(cast(cs_quantity#35 as bigint))] +Functions [3]: [partial_sum(cast(ss_quantity#6 as bigint)), partial_sum(cast(sr_return_quantity#25 as bigint)), partial_sum(cast(cs_quantity#35 as bigint))] Aggregate Attributes [3]: [sum#40, sum#41, sum#42] Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#43, sum#44, sum#45] @@ -327,9 +327,9 @@ Arguments: hashpartitioning(i_item_id#17, i_item_desc#18, s_store_id#12, s_store (60) HashAggregate [codegen id : 18] Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#43, sum#44, sum#45] Keys [4]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13] -Functions [3]: [sum(cast(ss_quantity#6 as bigint)), sum(cast(sr_return_quantity#29 as bigint)), sum(cast(cs_quantity#35 as bigint))] -Aggregate Attributes [3]: [sum(cast(ss_quantity#6 as bigint))#47, sum(cast(sr_return_quantity#29 as bigint))#48, sum(cast(cs_quantity#35 as bigint))#49] -Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum(cast(ss_quantity#6 as bigint))#47 AS store_sales_quantity#50, sum(cast(sr_return_quantity#29 as bigint))#48 AS store_returns_quantity#51, sum(cast(cs_quantity#35 as bigint))#49 AS catalog_sales_quantity#52] +Functions [3]: [sum(cast(ss_quantity#6 as bigint)), sum(cast(sr_return_quantity#25 as bigint)), sum(cast(cs_quantity#35 as bigint))] +Aggregate Attributes [3]: [sum(cast(ss_quantity#6 as bigint))#47, sum(cast(sr_return_quantity#25 as bigint))#48, sum(cast(cs_quantity#35 as bigint))#49] +Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum(cast(ss_quantity#6 as bigint))#47 AS store_sales_quantity#50, sum(cast(sr_return_quantity#25 as bigint))#48 AS store_returns_quantity#51, sum(cast(cs_quantity#35 as bigint))#49 AS catalog_sales_quantity#52] (61) TakeOrderedAndProject Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, store_sales_quantity#50, store_returns_quantity#51, catalog_sales_quantity#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt index f10b8e245c50e..ea91af9e8f755 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt @@ -69,7 +69,11 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 WholeStageCodegen (10) Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] InputAdapter BroadcastExchange #9 WholeStageCodegen (9) @@ -78,10 +82,6 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] InputAdapter WholeStageCodegen (16) Sort [cs_bill_customer_sk,cs_item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt index d3b013660ba28..9f123c4044cc8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt @@ -138,7 +138,7 @@ Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_addr_sk#2)) Output [3]: [d_date_sk#4, d_year#5, d_qoy#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -146,7 +146,7 @@ Input [3]: [d_date_sk#4, d_year#5, d_qoy#6] (6) Filter [codegen id : 1] Input [3]: [d_date_sk#4, d_year#5, d_qoy#6] -Condition : ((((isnotnull(d_qoy#6) AND isnotnull(d_year#5)) AND (d_qoy#6 = 3)) AND (d_year#5 = 2000)) AND isnotnull(d_date_sk#4)) +Condition : ((((isnotnull(d_qoy#6) AND isnotnull(d_year#5)) AND (d_qoy#6 = 2)) AND (d_year#5 = 2000)) AND isnotnull(d_date_sk#4)) (7) BroadcastExchange Input [3]: [d_date_sk#4, d_year#5, d_qoy#6] @@ -236,7 +236,7 @@ Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_addr_sk#2)) Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 7] @@ -244,7 +244,7 @@ Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] (27) Filter [codegen id : 7] Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] -Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 2)) AND (d_year#18 = 2000)) AND isnotnull(d_date_sk#17)) +Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 3)) AND (d_year#18 = 2000)) AND isnotnull(d_date_sk#17)) (28) BroadcastExchange Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] @@ -311,7 +311,7 @@ Right keys [1]: [ca_county#23] Join condition: None (42) Project [codegen id : 42] -Output [3]: [store_sales#16, ca_county#23, store_sales#28] +Output [3]: [ca_county#10, store_sales#16, store_sales#28] Input [4]: [ca_county#10, store_sales#16, ca_county#23, store_sales#28] (43) Scan parquet default.store_sales @@ -402,13 +402,13 @@ Input [3]: [ca_county#36, d_year#31, store_sales#41] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#42] (62) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ca_county#23] +Left keys [1]: [ca_county#10] Right keys [1]: [ca_county#36] Join condition: None (63) Project [codegen id : 42] Output [5]: [store_sales#16, store_sales#28, ca_county#36, d_year#31, store_sales#41] -Input [6]: [store_sales#16, ca_county#23, store_sales#28, ca_county#36, d_year#31, store_sales#41] +Input [6]: [ca_county#10, store_sales#16, store_sales#28, ca_county#36, d_year#31, store_sales#41] (64) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#43, ws_bill_addr_sk#44, ws_ext_sales_price#45] @@ -424,7 +424,7 @@ Input [3]: [ws_sold_date_sk#43, ws_bill_addr_sk#44, ws_ext_sales_price#45] Input [3]: [ws_sold_date_sk#43, ws_bill_addr_sk#44, ws_ext_sales_price#45] Condition : (isnotnull(ws_sold_date_sk#43) AND isnotnull(ws_bill_addr_sk#44)) -(67) ReusedExchange [Reuses operator id: 28] +(67) ReusedExchange [Reuses operator id: 49] Output [3]: [d_date_sk#46, d_year#47, d_qoy#48] (68) BroadcastHashJoin [codegen id : 22] @@ -492,7 +492,7 @@ Input [3]: [ws_sold_date_sk#43, ws_bill_addr_sk#44, ws_ext_sales_price#45] Input [3]: [ws_sold_date_sk#43, ws_bill_addr_sk#44, ws_ext_sales_price#45] Condition : (isnotnull(ws_sold_date_sk#43) AND isnotnull(ws_bill_addr_sk#44)) -(82) ReusedExchange [Reuses operator id: 49] +(82) ReusedExchange [Reuses operator id: 7] Output [3]: [d_date_sk#57, d_year#58, d_qoy#59] (83) BroadcastHashJoin [codegen id : 28] @@ -556,7 +556,7 @@ Right keys [1]: [ca_county#62] Join condition: None (96) Project [codegen id : 41] -Output [3]: [web_sales#56, ca_county#62, web_sales#67] +Output [3]: [ca_county#51, web_sales#56, web_sales#67] Input [4]: [ca_county#51, web_sales#56, ca_county#62, web_sales#67] (97) Scan parquet default.web_sales @@ -573,7 +573,7 @@ Input [3]: [ws_sold_date_sk#43, ws_bill_addr_sk#44, ws_ext_sales_price#45] Input [3]: [ws_sold_date_sk#43, ws_bill_addr_sk#44, ws_ext_sales_price#45] Condition : (isnotnull(ws_sold_date_sk#43) AND isnotnull(ws_bill_addr_sk#44)) -(100) ReusedExchange [Reuses operator id: 7] +(100) ReusedExchange [Reuses operator id: 28] Output [3]: [d_date_sk#69, d_year#70, d_qoy#71] (101) BroadcastHashJoin [codegen id : 35] @@ -632,26 +632,26 @@ Input [2]: [ca_county#74, web_sales#79] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#80] (113) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#62] +Left keys [1]: [ca_county#51] Right keys [1]: [ca_county#74] Join condition: None (114) Project [codegen id : 41] -Output [4]: [web_sales#56, ca_county#62, web_sales#67, web_sales#79] -Input [5]: [web_sales#56, ca_county#62, web_sales#67, ca_county#74, web_sales#79] +Output [4]: [ca_county#51, web_sales#56, web_sales#67, web_sales#79] +Input [5]: [ca_county#51, web_sales#56, web_sales#67, ca_county#74, web_sales#79] (115) BroadcastExchange -Input [4]: [web_sales#56, ca_county#62, web_sales#67, web_sales#79] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#81] +Input [4]: [ca_county#51, web_sales#56, web_sales#67, web_sales#79] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#81] (116) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ca_county#36] -Right keys [1]: [ca_county#62] -Join condition: ((CASE WHEN (web_sales#67 > 0.00) THEN CheckOverflow((promote_precision(web_sales#56) / promote_precision(web_sales#67)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#41 > 0.00) THEN CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#41)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#56 > 0.00) THEN CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#56)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#28 > 0.00) THEN CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#28)), DecimalType(37,20), true) ELSE null END)) +Right keys [1]: [ca_county#51] +Join condition: ((CASE WHEN (web_sales#56 > 0.00) THEN CheckOverflow((promote_precision(web_sales#67) / promote_precision(web_sales#56)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#41 > 0.00) THEN CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#41)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#67 > 0.00) THEN CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#67)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#16 > 0.00) THEN CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#16)), DecimalType(37,20), true) ELSE null END)) (117) Project [codegen id : 42] -Output [6]: [ca_county#36, d_year#31, CheckOverflow((promote_precision(web_sales#56) / promote_precision(web_sales#67)), DecimalType(37,20), true) AS web_q1_q2_increase#82, CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#41)), DecimalType(37,20), true) AS store_q1_q2_increase#83, CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#56)), DecimalType(37,20), true) AS web_q2_q3_increase#84, CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#28)), DecimalType(37,20), true) AS store_q2_q3_increase#85] -Input [9]: [store_sales#16, store_sales#28, ca_county#36, d_year#31, store_sales#41, web_sales#56, ca_county#62, web_sales#67, web_sales#79] +Output [6]: [ca_county#36, d_year#31, CheckOverflow((promote_precision(web_sales#67) / promote_precision(web_sales#56)), DecimalType(37,20), true) AS web_q1_q2_increase#82, CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#41)), DecimalType(37,20), true) AS store_q1_q2_increase#83, CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#67)), DecimalType(37,20), true) AS web_q2_q3_increase#84, CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q2_q3_increase#85] +Input [9]: [store_sales#16, store_sales#28, ca_county#36, d_year#31, store_sales#41, ca_county#51, web_sales#56, web_sales#67, web_sales#79] (118) Exchange Input [6]: [ca_county#36, d_year#31, web_q1_q2_increase#82, store_q1_q2_increase#83, web_q2_q3_increase#84, store_q2_q3_increase#85] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt index 9ec06b597cb64..c7b69500ed8a6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt @@ -7,7 +7,7 @@ WholeStageCodegen (43) BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] Project [store_sales,store_sales,ca_county,d_year,store_sales] BroadcastHashJoin [ca_county,ca_county] - Project [store_sales,ca_county,store_sales] + Project [ca_county,store_sales,store_sales] BroadcastHashJoin [ca_county,ca_county] HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter @@ -116,9 +116,9 @@ WholeStageCodegen (43) InputAdapter BroadcastExchange #14 WholeStageCodegen (41) - Project [web_sales,ca_county,web_sales,web_sales] + Project [ca_county,web_sales,web_sales,web_sales] BroadcastHashJoin [ca_county,ca_county] - Project [web_sales,ca_county,web_sales] + Project [ca_county,web_sales,web_sales] BroadcastHashJoin [ca_county,ca_county] HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter @@ -140,7 +140,7 @@ WholeStageCodegen (43) InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #9 + ReusedExchange [d_date_sk,d_year,d_qoy] #13 InputAdapter WholeStageCodegen (25) Sort [ca_address_sk] @@ -169,7 +169,7 @@ WholeStageCodegen (43) InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #13 + ReusedExchange [d_date_sk,d_year,d_qoy] #4 InputAdapter WholeStageCodegen (31) Sort [ca_address_sk] @@ -198,7 +198,7 @@ WholeStageCodegen (43) InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #4 + ReusedExchange [d_date_sk,d_year,d_qoy] #9 InputAdapter WholeStageCodegen (38) Sort [ca_address_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt index c5eb50e25d82c..13d73e61e1443 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt @@ -73,19 +73,19 @@ Input [2]: [i_manufact#2, count#9] Keys [1]: [i_manufact#2] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#11] -Results [3]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13, true AS alwaysTrue#14] +Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13] (12) Filter [codegen id : 2] -Input [3]: [item_cnt#12, i_manufact#2#13, alwaysTrue#14] -Condition : (if (isnull(alwaysTrue#14)) 0 else item_cnt#12 > 0) +Input [2]: [item_cnt#12, i_manufact#2#13] +Condition : (item_cnt#12 > 0) (13) Project [codegen id : 2] Output [1]: [i_manufact#2#13] -Input [3]: [item_cnt#12, i_manufact#2#13, alwaysTrue#14] +Input [2]: [item_cnt#12, i_manufact#2#13] (14) BroadcastExchange Input [1]: [i_manufact#2#13] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#15] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14] (15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_manufact#2] @@ -105,7 +105,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#16] +Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt index 350aa9a3c572b..2d14d75ca9362 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject [i_product_name] BroadcastExchange #2 WholeStageCodegen (2) Project [i_manufact] - Filter [alwaysTrue,item_cnt] - HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,alwaysTrue,count] + Filter [item_cnt] + HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,count] InputAdapter Exchange [i_manufact] #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index c5eb50e25d82c..13d73e61e1443 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -73,19 +73,19 @@ Input [2]: [i_manufact#2, count#9] Keys [1]: [i_manufact#2] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#11] -Results [3]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13, true AS alwaysTrue#14] +Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13] (12) Filter [codegen id : 2] -Input [3]: [item_cnt#12, i_manufact#2#13, alwaysTrue#14] -Condition : (if (isnull(alwaysTrue#14)) 0 else item_cnt#12 > 0) +Input [2]: [item_cnt#12, i_manufact#2#13] +Condition : (item_cnt#12 > 0) (13) Project [codegen id : 2] Output [1]: [i_manufact#2#13] -Input [3]: [item_cnt#12, i_manufact#2#13, alwaysTrue#14] +Input [2]: [item_cnt#12, i_manufact#2#13] (14) BroadcastExchange Input [1]: [i_manufact#2#13] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#15] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14] (15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_manufact#2] @@ -105,7 +105,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#16] +Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index 350aa9a3c572b..2d14d75ca9362 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject [i_product_name] BroadcastExchange #2 WholeStageCodegen (2) Project [i_manufact] - Filter [alwaysTrue,item_cnt] - HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,alwaysTrue,count] + Filter [item_cnt] + HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,count] InputAdapter Exchange [i_manufact] #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt index 0232d56ab7481..54e117e6cac10 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt @@ -34,12 +34,12 @@ TakeOrderedAndProject (46) : : +- Exchange (22) : : +- * Filter (21) : : +- * ColumnarToRow (20) - : : +- Scan parquet default.customer_address (19) + : : +- Scan parquet default.customer (19) : +- * Sort (28) : +- Exchange (27) : +- * Filter (26) : +- * ColumnarToRow (25) - : +- Scan parquet default.customer (24) + : +- Scan parquet default.customer_address (24) +- BroadcastExchange (39) +- * Project (38) +- * Filter (37) @@ -127,75 +127,75 @@ Arguments: hashpartitioning(ws_bill_customer_sk#4, 5), true, [id=#13] Input [3]: [ws_bill_customer_sk#4, ws_sales_price#5, i_item_id#11] Arguments: [ws_bill_customer_sk#4 ASC NULLS FIRST], false, 0 -(19) Scan parquet default.customer_address -Output [3]: [ca_address_sk#14, ca_city#15, ca_zip#16] +(19) Scan parquet default.customer +Output [2]: [c_customer_sk#14, c_current_addr_sk#15] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct (20) ColumnarToRow [codegen id : 5] -Input [3]: [ca_address_sk#14, ca_city#15, ca_zip#16] +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] (21) Filter [codegen id : 5] -Input [3]: [ca_address_sk#14, ca_city#15, ca_zip#16] -Condition : isnotnull(ca_address_sk#14) +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_current_addr_sk#15)) (22) Exchange -Input [3]: [ca_address_sk#14, ca_city#15, ca_zip#16] -Arguments: hashpartitioning(ca_address_sk#14, 5), true, [id=#17] +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +Arguments: hashpartitioning(c_current_addr_sk#15, 5), true, [id=#16] (23) Sort [codegen id : 6] -Input [3]: [ca_address_sk#14, ca_city#15, ca_zip#16] -Arguments: [ca_address_sk#14 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +Arguments: [c_current_addr_sk#15 ASC NULLS FIRST], false, 0 -(24) Scan parquet default.customer -Output [2]: [c_customer_sk#18, c_current_addr_sk#19] +(24) Scan parquet default.customer_address +Output [3]: [ca_address_sk#17, ca_city#18, ca_zip#19] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk)] +ReadSchema: struct (25) ColumnarToRow [codegen id : 7] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] +Input [3]: [ca_address_sk#17, ca_city#18, ca_zip#19] (26) Filter [codegen id : 7] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Condition : (isnotnull(c_customer_sk#18) AND isnotnull(c_current_addr_sk#19)) +Input [3]: [ca_address_sk#17, ca_city#18, ca_zip#19] +Condition : isnotnull(ca_address_sk#17) (27) Exchange -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: hashpartitioning(c_current_addr_sk#19, 5), true, [id=#20] +Input [3]: [ca_address_sk#17, ca_city#18, ca_zip#19] +Arguments: hashpartitioning(ca_address_sk#17, 5), true, [id=#20] (28) Sort [codegen id : 8] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: [c_current_addr_sk#19 ASC NULLS FIRST], false, 0 +Input [3]: [ca_address_sk#17, ca_city#18, ca_zip#19] +Arguments: [ca_address_sk#17 ASC NULLS FIRST], false, 0 (29) SortMergeJoin [codegen id : 9] -Left keys [1]: [ca_address_sk#14] -Right keys [1]: [c_current_addr_sk#19] +Left keys [1]: [c_current_addr_sk#15] +Right keys [1]: [ca_address_sk#17] Join condition: None (30) Project [codegen id : 9] -Output [3]: [ca_city#15, ca_zip#16, c_customer_sk#18] -Input [5]: [ca_address_sk#14, ca_city#15, ca_zip#16, c_customer_sk#18, c_current_addr_sk#19] +Output [3]: [c_customer_sk#14, ca_city#18, ca_zip#19] +Input [5]: [c_customer_sk#14, c_current_addr_sk#15, ca_address_sk#17, ca_city#18, ca_zip#19] (31) Exchange -Input [3]: [ca_city#15, ca_zip#16, c_customer_sk#18] -Arguments: hashpartitioning(c_customer_sk#18, 5), true, [id=#21] +Input [3]: [c_customer_sk#14, ca_city#18, ca_zip#19] +Arguments: hashpartitioning(c_customer_sk#14, 5), true, [id=#21] (32) Sort [codegen id : 10] -Input [3]: [ca_city#15, ca_zip#16, c_customer_sk#18] -Arguments: [c_customer_sk#18 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#14, ca_city#18, ca_zip#19] +Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 (33) SortMergeJoin [codegen id : 12] Left keys [1]: [ws_bill_customer_sk#4] -Right keys [1]: [c_customer_sk#18] +Right keys [1]: [c_customer_sk#14] Join condition: None (34) Project [codegen id : 12] -Output [4]: [ws_sales_price#5, ca_city#15, ca_zip#16, i_item_id#11] -Input [6]: [ws_bill_customer_sk#4, ws_sales_price#5, i_item_id#11, ca_city#15, ca_zip#16, c_customer_sk#18] +Output [4]: [ws_sales_price#5, ca_city#18, ca_zip#19, i_item_id#11] +Input [6]: [ws_bill_customer_sk#4, ws_sales_price#5, i_item_id#11, c_customer_sk#14, ca_city#18, ca_zip#19] (35) Scan parquet default.item Output [2]: [i_item_sk#10, i_item_id#11] @@ -225,32 +225,32 @@ Right keys [1]: [i_item_id#11#22] Join condition: None (41) Filter [codegen id : 12] -Input [5]: [ws_sales_price#5, ca_city#15, ca_zip#16, i_item_id#11, exists#1] -Condition : (substr(ca_zip#16, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) +Input [5]: [ws_sales_price#5, ca_city#18, ca_zip#19, i_item_id#11, exists#1] +Condition : (substr(ca_zip#19, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) (42) Project [codegen id : 12] -Output [3]: [ws_sales_price#5, ca_city#15, ca_zip#16] -Input [5]: [ws_sales_price#5, ca_city#15, ca_zip#16, i_item_id#11, exists#1] +Output [3]: [ws_sales_price#5, ca_city#18, ca_zip#19] +Input [5]: [ws_sales_price#5, ca_city#18, ca_zip#19, i_item_id#11, exists#1] (43) HashAggregate [codegen id : 12] -Input [3]: [ws_sales_price#5, ca_city#15, ca_zip#16] -Keys [2]: [ca_zip#16, ca_city#15] +Input [3]: [ws_sales_price#5, ca_city#18, ca_zip#19] +Keys [2]: [ca_zip#19, ca_city#18] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#5))] Aggregate Attributes [1]: [sum#24] -Results [3]: [ca_zip#16, ca_city#15, sum#25] +Results [3]: [ca_zip#19, ca_city#18, sum#25] (44) Exchange -Input [3]: [ca_zip#16, ca_city#15, sum#25] -Arguments: hashpartitioning(ca_zip#16, ca_city#15, 5), true, [id=#26] +Input [3]: [ca_zip#19, ca_city#18, sum#25] +Arguments: hashpartitioning(ca_zip#19, ca_city#18, 5), true, [id=#26] (45) HashAggregate [codegen id : 13] -Input [3]: [ca_zip#16, ca_city#15, sum#25] -Keys [2]: [ca_zip#16, ca_city#15] +Input [3]: [ca_zip#19, ca_city#18, sum#25] +Keys [2]: [ca_zip#19, ca_city#18] Functions [1]: [sum(UnscaledValue(ws_sales_price#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#5))#27] -Results [3]: [ca_zip#16, ca_city#15, MakeDecimal(sum(UnscaledValue(ws_sales_price#5))#27,17,2) AS sum(ws_sales_price)#28] +Results [3]: [ca_zip#19, ca_city#18, MakeDecimal(sum(UnscaledValue(ws_sales_price#5))#27,17,2) AS sum(ws_sales_price)#28] (46) TakeOrderedAndProject -Input [3]: [ca_zip#16, ca_city#15, sum(ws_sales_price)#28] -Arguments: 100, [ca_zip#16 ASC NULLS FIRST, ca_city#15 ASC NULLS FIRST], [ca_zip#16, ca_city#15, sum(ws_sales_price)#28] +Input [3]: [ca_zip#19, ca_city#18, sum(ws_sales_price)#28] +Arguments: 100, [ca_zip#19 ASC NULLS FIRST, ca_city#18 ASC NULLS FIRST], [ca_zip#19, ca_city#18, sum(ws_sales_price)#28] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/simplified.txt index 1eab468e67bc0..0e9662bb6aca5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/simplified.txt @@ -45,28 +45,28 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] InputAdapter Exchange [c_customer_sk] #5 WholeStageCodegen (9) - Project [ca_city,ca_zip,c_customer_sk] - SortMergeJoin [ca_address_sk,c_current_addr_sk] + Project [c_customer_sk,ca_city,ca_zip] + SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (6) - Sort [ca_address_sk] + Sort [c_current_addr_sk] InputAdapter - Exchange [ca_address_sk] #6 + Exchange [c_current_addr_sk] #6 WholeStageCodegen (5) - Filter [ca_address_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_city,ca_zip] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter WholeStageCodegen (8) - Sort [c_current_addr_sk] + Sort [ca_address_sk] InputAdapter - Exchange [c_current_addr_sk] #7 + Exchange [ca_address_sk] #7 WholeStageCodegen (7) - Filter [c_customer_sk,c_current_addr_sk] + Filter [ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] + Scan parquet default.customer_address [ca_address_sk,ca_city,ca_zip] InputAdapter BroadcastExchange #8 WholeStageCodegen (11) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt index df1197d7c925e..741ee50f800ec 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt @@ -25,15 +25,15 @@ TakeOrderedAndProject (35) +- * Sort (29) +- Exchange (28) +- * Project (27) - +- * BroadcastHashJoin Inner BuildLeft (26) - :- BroadcastExchange (22) - : +- * Project (21) - : +- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet default.date_dim (18) - +- * Filter (25) - +- * ColumnarToRow (24) - +- Scan parquet default.store_returns (23) + +- * BroadcastHashJoin Inner BuildRight (26) + :- * Filter (20) + : +- * ColumnarToRow (19) + : +- Scan parquet default.store_returns (18) + +- BroadcastExchange (25) + +- * Project (24) + +- * Filter (23) + +- * ColumnarToRow (22) + +- Scan parquet default.date_dim (21) (1) Scan parquet default.store_sales @@ -112,72 +112,72 @@ Arguments: hashpartitioning(cast(ss_ticket_number#5 as bigint), cast(ss_item_sk# Input [14]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18] Arguments: [cast(ss_ticket_number#5 as bigint) ASC NULLS FIRST, cast(ss_item_sk#2 as bigint) ASC NULLS FIRST, cast(ss_customer_sk#3 as bigint) ASC NULLS FIRST], false, 0 -(18) Scan parquet default.date_dim -Output [3]: [d_date_sk#21, d_year#22, d_moy#23] +(18) Scan parquet default.store_returns +Output [4]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk), IsNotNull(sr_returned_date_sk)] +ReadSchema: struct -(19) ColumnarToRow [codegen id : 5] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] +(19) ColumnarToRow [codegen id : 6] +Input [4]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24] -(20) Filter [codegen id : 5] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] -Condition : ((((isnotnull(d_year#22) AND isnotnull(d_moy#23)) AND (d_year#22 = 2001)) AND (d_moy#23 = 8)) AND isnotnull(d_date_sk#21)) +(20) Filter [codegen id : 6] +Input [4]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24] +Condition : (((isnotnull(sr_ticket_number#24) AND isnotnull(sr_item_sk#22)) AND isnotnull(sr_customer_sk#23)) AND isnotnull(sr_returned_date_sk#21)) -(21) Project [codegen id : 5] -Output [1]: [d_date_sk#21] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] +(21) Scan parquet default.date_dim +Output [3]: [d_date_sk#25, d_year#26, d_moy#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] +ReadSchema: struct -(22) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +(22) ColumnarToRow [codegen id : 5] +Input [3]: [d_date_sk#25, d_year#26, d_moy#27] -(23) Scan parquet default.store_returns -Output [4]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk), IsNotNull(sr_returned_date_sk)] -ReadSchema: struct +(23) Filter [codegen id : 5] +Input [3]: [d_date_sk#25, d_year#26, d_moy#27] +Condition : ((((isnotnull(d_year#26) AND isnotnull(d_moy#27)) AND (d_year#26 = 2001)) AND (d_moy#27 = 8)) AND isnotnull(d_date_sk#25)) -(24) ColumnarToRow -Input [4]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] +(24) Project [codegen id : 5] +Output [1]: [d_date_sk#25] +Input [3]: [d_date_sk#25, d_year#26, d_moy#27] -(25) Filter -Input [4]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] -Condition : (((isnotnull(sr_ticket_number#28) AND isnotnull(sr_item_sk#26)) AND isnotnull(sr_customer_sk#27)) AND isnotnull(sr_returned_date_sk#25)) +(25) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cast(d_date_sk#21 as bigint)] -Right keys [1]: [sr_returned_date_sk#25] +Left keys [1]: [sr_returned_date_sk#21] +Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None (27) Project [codegen id : 6] -Output [4]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] -Input [5]: [d_date_sk#21, sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] +Output [4]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24] +Input [5]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, d_date_sk#25] (28) Exchange -Input [4]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] -Arguments: hashpartitioning(sr_ticket_number#28, sr_item_sk#26, sr_customer_sk#27, 5), true, [id=#29] +Input [4]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24] +Arguments: hashpartitioning(sr_ticket_number#24, sr_item_sk#22, sr_customer_sk#23, 5), true, [id=#29] (29) Sort [codegen id : 7] -Input [4]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] -Arguments: [sr_ticket_number#28 ASC NULLS FIRST, sr_item_sk#26 ASC NULLS FIRST, sr_customer_sk#27 ASC NULLS FIRST], false, 0 +Input [4]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24] +Arguments: [sr_ticket_number#24 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST, sr_customer_sk#23 ASC NULLS FIRST], false, 0 (30) SortMergeJoin [codegen id : 8] Left keys [3]: [cast(ss_ticket_number#5 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_customer_sk#3 as bigint)] -Right keys [3]: [sr_ticket_number#28, sr_item_sk#26, sr_customer_sk#27] +Right keys [3]: [sr_ticket_number#24, sr_item_sk#22, sr_customer_sk#23] Join condition: None (31) Project [codegen id : 8] -Output [12]: [ss_sold_date_sk#1, sr_returned_date_sk#25, s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18] -Input [18]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18, sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] +Output [12]: [ss_sold_date_sk#1, sr_returned_date_sk#21, s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18] +Input [18]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18, sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24] (32) HashAggregate [codegen id : 8] -Input [12]: [ss_sold_date_sk#1, sr_returned_date_sk#25, s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18] +Input [12]: [ss_sold_date_sk#1, sr_returned_date_sk#21, s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18] Keys [10]: [s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18] -Functions [5]: [partial_sum(cast(CASE WHEN ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))] +Functions [5]: [partial_sum(cast(CASE WHEN ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))] Aggregate Attributes [5]: [sum#30, sum#31, sum#32, sum#33, sum#34] Results [15]: [s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18, sum#35, sum#36, sum#37, sum#38, sum#39] @@ -188,9 +188,9 @@ Arguments: hashpartitioning(s_store_name#9, s_company_id#10, s_street_number#11, (34) HashAggregate [codegen id : 9] Input [15]: [s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18, sum#35, sum#36, sum#37, sum#38, sum#39] Keys [10]: [s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18] -Functions [5]: [sum(cast(CASE WHEN ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))] -Aggregate Attributes [5]: [sum(cast(CASE WHEN ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint))#41, sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint))#42, sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint))#43, sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint))#44, sum(cast(CASE WHEN ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))#45] -Results [15]: [s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18, sum(cast(CASE WHEN ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint))#41 AS 30 days #46, sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint))#42 AS 31 - 60 days #47, sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint))#43 AS 61 - 90 days #48, sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint))#44 AS 91 - 120 days #49, sum(cast(CASE WHEN ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))#45 AS >120 days #50] +Functions [5]: [sum(cast(CASE WHEN ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))] +Aggregate Attributes [5]: [sum(cast(CASE WHEN ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint))#41, sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint))#42, sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint))#43, sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint))#44, sum(cast(CASE WHEN ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))#45] +Results [15]: [s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18, sum(cast(CASE WHEN ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint))#41 AS 30 days #46, sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint))#42 AS 31 - 60 days #47, sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint))#43 AS 61 - 90 days #48, sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint))#44 AS 91 - 120 days #49, sum(cast(CASE WHEN ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))#45 AS >120 days #50] (35) TakeOrderedAndProject Input [15]: [s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18, 30 days #46, 31 - 60 days #47, 61 - 90 days #48, 91 - 120 days #49, >120 days #50] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/simplified.txt index 5d6f38e882a5c..be11a69176810 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/simplified.txt @@ -42,7 +42,11 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s Exchange [sr_ticket_number,sr_item_sk,sr_customer_sk] #5 WholeStageCodegen (6) Project [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_ticket_number,sr_item_sk,sr_customer_sk,sr_returned_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) @@ -51,7 +55,3 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - Filter [sr_ticket_number,sr_item_sk,sr_customer_sk,sr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt index 511e1b46cd7a7..675cff99ad729 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt @@ -11,30 +11,30 @@ TakeOrderedAndProject (50) : +- Exchange (27) : +- * Project (26) : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) : : :- * Filter (3) : : : +- * ColumnarToRow (2) : : : +- Scan parquet default.store_sales (1) - : : +- BroadcastExchange (8) - : : +- * Project (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (17) + : : +- * Project (16) + : : +- * Filter (15) + : : +- * BroadcastHashJoin LeftOuter BuildRight (14) + : : :- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.item (4) + : : +- BroadcastExchange (13) + : : +- * HashAggregate (12) + : : +- Exchange (11) + : : +- * HashAggregate (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet default.item (7) : +- BroadcastExchange (24) : +- * Project (23) : +- * Filter (22) - : +- * BroadcastHashJoin LeftOuter BuildRight (21) - : :- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.item (11) - : +- BroadcastExchange (20) - : +- * HashAggregate (19) - : +- Exchange (18) - : +- * HashAggregate (17) - : +- * Filter (16) - : +- * ColumnarToRow (15) - : +- Scan parquet default.item (14) + : +- * ColumnarToRow (21) + : +- Scan parquet default.date_dim (20) +- * Sort (42) +- Exchange (41) +- * Project (40) @@ -65,112 +65,112 @@ Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3] Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3] Condition : ((isnotnull(ss_customer_sk#3) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_item_sk#2)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_month_seq#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_month_seq#5] - -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_month_seq#5] -Condition : ((isnotnull(d_month_seq#5) AND (d_month_seq#5 = Subquery scalar-subquery#6, [id=#7])) AND isnotnull(d_date_sk#4)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_month_seq#5] - -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None - -(10) Project [codegen id : 5] -Output [2]: [ss_item_sk#2, ss_customer_sk#3] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, d_date_sk#4] - -(11) Scan parquet default.item -Output [3]: [i_item_sk#9, i_current_price#10, i_category#11] +(4) Scan parquet default.item +Output [3]: [i_item_sk#4, i_current_price#5, i_category#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#9, i_current_price#10, i_category#11] +(5) ColumnarToRow [codegen id : 3] +Input [3]: [i_item_sk#4, i_current_price#5, i_category#6] -(13) Filter [codegen id : 4] -Input [3]: [i_item_sk#9, i_current_price#10, i_category#11] -Condition : (isnotnull(i_current_price#10) AND isnotnull(i_item_sk#9)) +(6) Filter [codegen id : 3] +Input [3]: [i_item_sk#4, i_current_price#5, i_category#6] +Condition : (isnotnull(i_current_price#5) AND isnotnull(i_item_sk#4)) -(14) Scan parquet default.item -Output [2]: [i_current_price#10, i_category#11] +(7) Scan parquet default.item +Output [2]: [i_current_price#5, i_category#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 2] -Input [2]: [i_current_price#10, i_category#11] - -(16) Filter [codegen id : 2] -Input [2]: [i_current_price#10, i_category#11] -Condition : isnotnull(i_category#11) - -(17) HashAggregate [codegen id : 2] -Input [2]: [i_current_price#10, i_category#11] -Keys [1]: [i_category#11] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#10))] -Aggregate Attributes [2]: [sum#12, count#13] -Results [3]: [i_category#11, sum#14, count#15] - -(18) Exchange -Input [3]: [i_category#11, sum#14, count#15] -Arguments: hashpartitioning(i_category#11, 5), true, [id=#16] - -(19) HashAggregate [codegen id : 3] -Input [3]: [i_category#11, sum#14, count#15] -Keys [1]: [i_category#11] -Functions [1]: [avg(UnscaledValue(i_current_price#10))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#10))#17] -Results [2]: [cast((avg(UnscaledValue(i_current_price#10))#17 / 100.0) as decimal(11,6)) AS avg(i_current_price)#18, i_category#11 AS i_category#11#19] - -(20) BroadcastExchange -Input [2]: [avg(i_current_price)#18, i_category#11#19] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#20] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_category#11] -Right keys [1]: [i_category#11#19] +(8) ColumnarToRow [codegen id : 1] +Input [2]: [i_current_price#5, i_category#6] + +(9) Filter [codegen id : 1] +Input [2]: [i_current_price#5, i_category#6] +Condition : isnotnull(i_category#6) + +(10) HashAggregate [codegen id : 1] +Input [2]: [i_current_price#5, i_category#6] +Keys [1]: [i_category#6] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#5))] +Aggregate Attributes [2]: [sum#7, count#8] +Results [3]: [i_category#6, sum#9, count#10] + +(11) Exchange +Input [3]: [i_category#6, sum#9, count#10] +Arguments: hashpartitioning(i_category#6, 5), true, [id=#11] + +(12) HashAggregate [codegen id : 2] +Input [3]: [i_category#6, sum#9, count#10] +Keys [1]: [i_category#6] +Functions [1]: [avg(UnscaledValue(i_current_price#5))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#5))#12] +Results [2]: [cast((avg(UnscaledValue(i_current_price#5))#12 / 100.0) as decimal(11,6)) AS avg(i_current_price)#13, i_category#6 AS i_category#6#14] + +(13) BroadcastExchange +Input [2]: [avg(i_current_price)#13, i_category#6#14] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#15] + +(14) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [i_category#6] +Right keys [1]: [i_category#6#14] +Join condition: None + +(15) Filter [codegen id : 3] +Input [5]: [i_item_sk#4, i_current_price#5, i_category#6, avg(i_current_price)#13, i_category#6#14] +Condition : (cast(i_current_price#5 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#13)), DecimalType(14,7), true)) + +(16) Project [codegen id : 3] +Output [1]: [i_item_sk#4] +Input [5]: [i_item_sk#4, i_current_price#5, i_category#6, avg(i_current_price)#13, i_category#6#14] + +(17) BroadcastExchange +Input [1]: [i_item_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] + +(18) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#4] Join condition: None +(19) Project [codegen id : 5] +Output [2]: [ss_sold_date_sk#1, ss_customer_sk#3] +Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, i_item_sk#4] + +(20) Scan parquet default.date_dim +Output [2]: [d_date_sk#17, d_month_seq#18] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] +ReadSchema: struct + +(21) ColumnarToRow [codegen id : 4] +Input [2]: [d_date_sk#17, d_month_seq#18] + (22) Filter [codegen id : 4] -Input [5]: [i_item_sk#9, i_current_price#10, i_category#11, avg(i_current_price)#18, i_category#11#19] -Condition : (cast(i_current_price#10 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#18)), DecimalType(14,7), true)) +Input [2]: [d_date_sk#17, d_month_seq#18] +Condition : ((isnotnull(d_month_seq#18) AND (d_month_seq#18 = Subquery scalar-subquery#19, [id=#20])) AND isnotnull(d_date_sk#17)) (23) Project [codegen id : 4] -Output [1]: [i_item_sk#9] -Input [5]: [i_item_sk#9, i_current_price#10, i_category#11, avg(i_current_price)#18, i_category#11#19] +Output [1]: [d_date_sk#17] +Input [2]: [d_date_sk#17, d_month_seq#18] (24) BroadcastExchange -Input [1]: [i_item_sk#9] +Input [1]: [d_date_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#9] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#17] Join condition: None (26) Project [codegen id : 5] Output [1]: [ss_customer_sk#3] -Input [3]: [ss_item_sk#2, ss_customer_sk#3, i_item_sk#9] +Input [3]: [ss_sold_date_sk#1, ss_customer_sk#3, d_date_sk#17] (27) Exchange Input [1]: [ss_customer_sk#3] @@ -282,7 +282,7 @@ Arguments: 100, [cnt#35 ASC NULLS FIRST], [state#34, cnt#35] ===== Subqueries ===== -Subquery:1 Hosting operator id = 6 Hosting Expression = Subquery scalar-subquery#6, [id=#7] +Subquery:1 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#19, [id=#20] * HashAggregate (57) +- Exchange (56) +- * HashAggregate (55) @@ -293,39 +293,39 @@ Subquery:1 Hosting operator id = 6 Hosting Expression = Subquery scalar-subquery (51) Scan parquet default.date_dim -Output [3]: [d_month_seq#5, d_year#37, d_moy#38] +Output [3]: [d_month_seq#18, d_year#37, d_moy#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (52) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#5, d_year#37, d_moy#38] +Input [3]: [d_month_seq#18, d_year#37, d_moy#38] (53) Filter [codegen id : 1] -Input [3]: [d_month_seq#5, d_year#37, d_moy#38] +Input [3]: [d_month_seq#18, d_year#37, d_moy#38] Condition : (((isnotnull(d_year#37) AND isnotnull(d_moy#38)) AND (d_year#37 = 2000)) AND (d_moy#38 = 1)) (54) Project [codegen id : 1] -Output [1]: [d_month_seq#5] -Input [3]: [d_month_seq#5, d_year#37, d_moy#38] +Output [1]: [d_month_seq#18] +Input [3]: [d_month_seq#18, d_year#37, d_moy#38] (55) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#5] -Keys [1]: [d_month_seq#5] +Input [1]: [d_month_seq#18] +Keys [1]: [d_month_seq#18] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#5] +Results [1]: [d_month_seq#18] (56) Exchange -Input [1]: [d_month_seq#5] -Arguments: hashpartitioning(d_month_seq#5, 5), true, [id=#39] +Input [1]: [d_month_seq#18] +Arguments: hashpartitioning(d_month_seq#18, 5), true, [id=#39] (57) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#5] -Keys [1]: [d_month_seq#5] +Input [1]: [d_month_seq#18] +Keys [1]: [d_month_seq#18] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#5] +Results [1]: [d_month_seq#18] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt index dcebba331afb3..73d42163240f0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt @@ -16,55 +16,55 @@ TakeOrderedAndProject [cnt,state] Exchange [ss_customer_sk] #2 WholeStageCodegen (5) Project [ss_customer_sk] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_customer_sk] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_customer_sk,ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - Subquery #1 - WholeStageCodegen (2) - HashAggregate [d_month_seq] + WholeStageCodegen (3) + Project [i_item_sk] + Filter [i_current_price,avg(i_current_price)] + BroadcastHashJoin [i_category,i_category] + Filter [i_current_price,i_item_sk] + ColumnarToRow InputAdapter - Exchange [d_month_seq] #4 - WholeStageCodegen (1) - HashAggregate [d_month_seq] - Project [d_month_seq] - Filter [d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_month_seq,d_year,d_moy] - ColumnarToRow + Scan parquet default.item [i_item_sk,i_current_price,i_category] InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] + BroadcastExchange #4 + WholeStageCodegen (2) + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] + InputAdapter + Exchange [i_category] #5 + WholeStageCodegen (1) + HashAggregate [i_category,i_current_price] [sum,count,sum,count] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_current_price,i_category] InputAdapter - BroadcastExchange #5 + BroadcastExchange #6 WholeStageCodegen (4) - Project [i_item_sk] - Filter [i_current_price,avg(i_current_price)] - BroadcastHashJoin [i_category,i_category] - Filter [i_current_price,i_item_sk] - ColumnarToRow + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + Subquery #1 + WholeStageCodegen (2) + HashAggregate [d_month_seq] InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_category] + Exchange [d_month_seq] #7 + WholeStageCodegen (1) + HashAggregate [d_month_seq] + Project [d_month_seq] + Filter [d_year,d_moy] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] - InputAdapter - Exchange [i_category] #7 - WholeStageCodegen (2) - HashAggregate [i_category,i_current_price] [sum,count,sum,count] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_current_price,i_category] + Scan parquet default.date_dim [d_date_sk,d_month_seq] InputAdapter WholeStageCodegen (12) Sort [c_customer_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt index e616934bbd073..58a60763b2b57 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (75) -+- * Project (74) - +- BroadcastNestedLoopJoin Inner BuildRight (73) +TakeOrderedAndProject (69) ++- * Project (68) + +- BroadcastNestedLoopJoin Inner BuildRight (67) :- * HashAggregate (47) : +- Exchange (46) : +- * HashAggregate (45) @@ -49,31 +49,25 @@ TakeOrderedAndProject (75) : +- * Filter (37) : +- * ColumnarToRow (36) : +- Scan parquet default.customer_address (35) - +- BroadcastExchange (72) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) + +- BroadcastExchange (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * HashAggregate (63) + +- * Project (62) + +- * BroadcastHashJoin Inner BuildRight (61) :- * Project (59) : +- * BroadcastHashJoin Inner BuildRight (58) : :- * Project (56) : : +- * BroadcastHashJoin Inner BuildRight (55) : : :- * Project (53) - : : : +- * BroadcastHashJoin Inner BuildLeft (52) - : : : :- ReusedExchange (48) - : : : +- * Filter (51) - : : : +- * ColumnarToRow (50) - : : : +- Scan parquet default.store_sales (49) + : : : +- * BroadcastHashJoin Inner BuildRight (52) + : : : :- * Filter (50) + : : : : +- * ColumnarToRow (49) + : : : : +- Scan parquet default.store_sales (48) + : : : +- ReusedExchange (51) : : +- ReusedExchange (54) : +- ReusedExchange (57) - +- BroadcastExchange (66) - +- * Project (65) - +- * BroadcastHashJoin Inner BuildLeft (64) - :- ReusedExchange (60) - +- * Filter (63) - +- * ColumnarToRow (62) - +- Scan parquet default.customer (61) + +- ReusedExchange (60) (1) Scan parquet default.store_sales @@ -290,31 +284,31 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#6))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#6))#31] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#31,17,2) AS promotions#32] -(48) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#7] - -(49) Scan parquet default.store_sales +(48) Scan parquet default.store_sales Output [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(50) ColumnarToRow +(49) ColumnarToRow [codegen id : 14] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6] -(51) Filter +(50) Filter [codegen id : 14] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6] Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_item_sk#2)) +(51) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#7] + (52) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [d_date_sk#7] -Right keys [1]: [ss_sold_date_sk#1] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#7] Join condition: None (53) Project [codegen id : 14] Output [4]: [ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6] -Input [6]: [d_date_sk#7, ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6] +Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6, d_date_sk#7] (54) ReusedExchange [Reuses operator id: 15] Output [1]: [i_item_sk#11] @@ -340,75 +334,48 @@ Join condition: None Output [2]: [ss_customer_sk#3, ss_ext_sales_price#6] Input [4]: [ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6, s_store_sk#19] -(60) ReusedExchange [Reuses operator id: 39] -Output [1]: [ca_address_sk#24] - -(61) Scan parquet default.customer -Output [2]: [c_customer_sk#22, c_current_addr_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(62) ColumnarToRow -Input [2]: [c_customer_sk#22, c_current_addr_sk#23] - -(63) Filter -Input [2]: [c_customer_sk#22, c_current_addr_sk#23] -Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_current_addr_sk#23)) - -(64) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ca_address_sk#24] -Right keys [1]: [c_current_addr_sk#23] -Join condition: None - -(65) Project [codegen id : 13] +(60) ReusedExchange [Reuses operator id: 42] Output [1]: [c_customer_sk#22] -Input [3]: [ca_address_sk#24, c_customer_sk#22, c_current_addr_sk#23] - -(66) BroadcastExchange -Input [1]: [c_customer_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] -(67) BroadcastHashJoin [codegen id : 14] +(61) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_customer_sk#3] Right keys [1]: [c_customer_sk#22] Join condition: None -(68) Project [codegen id : 14] +(62) Project [codegen id : 14] Output [1]: [ss_ext_sales_price#6] Input [3]: [ss_customer_sk#3, ss_ext_sales_price#6, c_customer_sk#22] -(69) HashAggregate [codegen id : 14] +(63) HashAggregate [codegen id : 14] Input [1]: [ss_ext_sales_price#6] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum#34] -Results [1]: [sum#35] +Aggregate Attributes [1]: [sum#33] +Results [1]: [sum#34] -(70) Exchange -Input [1]: [sum#35] -Arguments: SinglePartition, true, [id=#36] +(64) Exchange +Input [1]: [sum#34] +Arguments: SinglePartition, true, [id=#35] -(71) HashAggregate [codegen id : 15] -Input [1]: [sum#35] +(65) HashAggregate [codegen id : 15] +Input [1]: [sum#34] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#6))#37] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#37,17,2) AS total#38] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#6))#36] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#36,17,2) AS total#37] -(72) BroadcastExchange -Input [1]: [total#38] -Arguments: IdentityBroadcastMode, [id=#39] +(66) BroadcastExchange +Input [1]: [total#37] +Arguments: IdentityBroadcastMode, [id=#38] -(73) BroadcastNestedLoopJoin +(67) BroadcastNestedLoopJoin Join condition: None -(74) Project [codegen id : 16] -Output [3]: [promotions#32, total#38, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#32 as decimal(15,4))) / promote_precision(cast(total#38 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS (CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))#40] -Input [2]: [promotions#32, total#38] +(68) Project [codegen id : 16] +Output [3]: [promotions#32, total#37, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#32 as decimal(15,4))) / promote_precision(cast(total#37 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS (CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))#39] +Input [2]: [promotions#32, total#37] -(75) TakeOrderedAndProject -Input [3]: [promotions#32, total#38, (CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))#40] -Arguments: 100, [promotions#32 ASC NULLS FIRST, total#38 ASC NULLS FIRST], [promotions#32, total#38, (CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))#40] +(69) TakeOrderedAndProject +Input [3]: [promotions#32, total#37, (CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))#39] +Arguments: 100, [promotions#32 ASC NULLS FIRST, total#37 ASC NULLS FIRST], [promotions#32, total#37, (CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/simplified.txt index 039ccb1aa18cf..87f2b3ae03746 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/simplified.txt @@ -86,25 +86,16 @@ TakeOrderedAndProject [promotions,total,(CAST((CAST(CAST(promotions AS DECIMAL(1 Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_store_sk,ss_sold_date_sk,ss_customer_sk,ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk] #3 InputAdapter ReusedExchange [s_store_sk] #5 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (13) - Project [c_customer_sk] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - InputAdapter - ReusedExchange [ca_address_sk] #7 - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] + ReusedExchange [c_customer_sk] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt index e9a2b7a375b01..b74dfb49c9f03 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt @@ -10,15 +10,15 @@ TakeOrderedAndProject (32) : :- * Project (16) : : +- * BroadcastHashJoin Inner BuildRight (15) : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : :- BroadcastExchange (5) - : : : : +- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.date_dim (1) - : : : +- * Filter (8) - : : : +- * ColumnarToRow (7) - : : : +- Scan parquet default.web_sales (6) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.web_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) : : +- BroadcastExchange (14) : : +- * Filter (13) : : +- * ColumnarToRow (12) @@ -33,50 +33,50 @@ TakeOrderedAndProject (32) +- Scan parquet default.warehouse (23) -(1) Scan parquet default.date_dim -Output [2]: [d_date_sk#1, d_month_seq#2] +(1) Scan parquet default.web_sales +Output [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, ws_warehouse_sk#5] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_mode_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_ship_date_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#1, d_month_seq#2] +(2) ColumnarToRow [codegen id : 5] +Input [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, ws_warehouse_sk#5] -(3) Filter [codegen id : 1] -Input [2]: [d_date_sk#1, d_month_seq#2] -Condition : (((isnotnull(d_month_seq#2) AND (d_month_seq#2 >= 1200)) AND (d_month_seq#2 <= 1211)) AND isnotnull(d_date_sk#1)) +(3) Filter [codegen id : 5] +Input [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, ws_warehouse_sk#5] +Condition : (((isnotnull(ws_warehouse_sk#5) AND isnotnull(ws_ship_mode_sk#4)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_ship_date_sk#2)) -(4) Project [codegen id : 1] -Output [1]: [d_date_sk#1] -Input [2]: [d_date_sk#1, d_month_seq#2] +(4) Scan parquet default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [1]: [d_date_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#3] +(5) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_month_seq#7] -(6) Scan parquet default.web_sales -Output [5]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_web_site_sk#6, ws_ship_mode_sk#7, ws_warehouse_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_mode_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_ship_date_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) -(7) ColumnarToRow -Input [5]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_web_site_sk#6, ws_ship_mode_sk#7, ws_warehouse_sk#8] +(7) Project [codegen id : 1] +Output [1]: [d_date_sk#6] +Input [2]: [d_date_sk#6, d_month_seq#7] -(8) Filter -Input [5]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_web_site_sk#6, ws_ship_mode_sk#7, ws_warehouse_sk#8] -Condition : (((isnotnull(ws_warehouse_sk#8) AND isnotnull(ws_ship_mode_sk#7)) AND isnotnull(ws_web_site_sk#6)) AND isnotnull(ws_ship_date_sk#5)) +(8) BroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] (9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ws_ship_date_sk#5] +Left keys [1]: [ws_ship_date_sk#2] +Right keys [1]: [d_date_sk#6] Join condition: None (10) Project [codegen id : 5] -Output [5]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_web_site_sk#6, ws_ship_mode_sk#7, ws_warehouse_sk#8] -Input [6]: [d_date_sk#1, ws_sold_date_sk#4, ws_ship_date_sk#5, ws_web_site_sk#6, ws_ship_mode_sk#7, ws_warehouse_sk#8] +Output [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, ws_warehouse_sk#5] +Input [6]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, ws_warehouse_sk#5, d_date_sk#6] (11) Scan parquet default.web_site Output [2]: [web_site_sk#9, web_name#10] @@ -97,13 +97,13 @@ Input [2]: [web_site_sk#9, web_name#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] (15) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_web_site_sk#6] +Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#9] Join condition: None (16) Project [codegen id : 5] -Output [5]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_ship_mode_sk#7, ws_warehouse_sk#8, web_name#10] -Input [7]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_web_site_sk#6, ws_ship_mode_sk#7, ws_warehouse_sk#8, web_site_sk#9, web_name#10] +Output [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_ship_mode_sk#4, ws_warehouse_sk#5, web_name#10] +Input [7]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, ws_warehouse_sk#5, web_site_sk#9, web_name#10] (17) Scan parquet default.ship_mode Output [2]: [sm_ship_mode_sk#12, sm_type#13] @@ -124,13 +124,13 @@ Input [2]: [sm_ship_mode_sk#12, sm_type#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] (21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_ship_mode_sk#7] +Left keys [1]: [ws_ship_mode_sk#4] Right keys [1]: [sm_ship_mode_sk#12] Join condition: None (22) Project [codegen id : 5] -Output [5]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_warehouse_sk#8, web_name#10, sm_type#13] -Input [7]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_ship_mode_sk#7, ws_warehouse_sk#8, web_name#10, sm_ship_mode_sk#12, sm_type#13] +Output [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_warehouse_sk#5, web_name#10, sm_type#13] +Input [7]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_ship_mode_sk#4, ws_warehouse_sk#5, web_name#10, sm_ship_mode_sk#12, sm_type#13] (23) Scan parquet default.warehouse Output [2]: [w_warehouse_sk#15, w_warehouse_name#16] @@ -151,18 +151,18 @@ Input [2]: [w_warehouse_sk#15, w_warehouse_name#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (27) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_warehouse_sk#8] +Left keys [1]: [ws_warehouse_sk#5] Right keys [1]: [w_warehouse_sk#15] Join condition: None (28) Project [codegen id : 5] -Output [5]: [ws_sold_date_sk#4, ws_ship_date_sk#5, w_warehouse_name#16, sm_type#13, web_name#10] -Input [7]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_warehouse_sk#8, web_name#10, sm_type#13, w_warehouse_sk#15, w_warehouse_name#16] +Output [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, w_warehouse_name#16, sm_type#13, web_name#10] +Input [7]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_warehouse_sk#5, web_name#10, sm_type#13, w_warehouse_sk#15, w_warehouse_name#16] (29) HashAggregate [codegen id : 5] -Input [5]: [ws_sold_date_sk#4, ws_ship_date_sk#5, w_warehouse_name#16, sm_type#13, web_name#10] +Input [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, w_warehouse_name#16, sm_type#13, web_name#10] Keys [3]: [substr(w_warehouse_name#16, 1, 20) AS substr(w_warehouse_name#16, 1, 20)#18, sm_type#13, web_name#10] -Functions [5]: [partial_sum(cast(CASE WHEN ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 30) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 30) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 60)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 60) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 90)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 90) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 120)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN ((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 120) THEN 1 ELSE 0 END as bigint))] +Functions [5]: [partial_sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 30) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 60) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 90) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))] Aggregate Attributes [5]: [sum#19, sum#20, sum#21, sum#22, sum#23] Results [8]: [substr(w_warehouse_name#16, 1, 20)#18, sm_type#13, web_name#10, sum#24, sum#25, sum#26, sum#27, sum#28] @@ -173,9 +173,9 @@ Arguments: hashpartitioning(substr(w_warehouse_name#16, 1, 20)#18, sm_type#13, w (31) HashAggregate [codegen id : 6] Input [8]: [substr(w_warehouse_name#16, 1, 20)#18, sm_type#13, web_name#10, sum#24, sum#25, sum#26, sum#27, sum#28] Keys [3]: [substr(w_warehouse_name#16, 1, 20)#18, sm_type#13, web_name#10] -Functions [5]: [sum(cast(CASE WHEN ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 30) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 30) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 60)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 60) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 90)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 90) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 120)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN ((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 120) THEN 1 ELSE 0 END as bigint))] -Aggregate Attributes [5]: [sum(cast(CASE WHEN ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 30) THEN 1 ELSE 0 END as bigint))#30, sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 30) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 60)) THEN 1 ELSE 0 END as bigint))#31, sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 60) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 90)) THEN 1 ELSE 0 END as bigint))#32, sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 90) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 120)) THEN 1 ELSE 0 END as bigint))#33, sum(cast(CASE WHEN ((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 120) THEN 1 ELSE 0 END as bigint))#34] -Results [8]: [substr(w_warehouse_name#16, 1, 20)#18 AS substr(w_warehouse_name, 1, 20)#35, sm_type#13, web_name#10, sum(cast(CASE WHEN ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 30) THEN 1 ELSE 0 END as bigint))#30 AS 30 days #36, sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 30) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 60)) THEN 1 ELSE 0 END as bigint))#31 AS 31 - 60 days #37, sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 60) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 90)) THEN 1 ELSE 0 END as bigint))#32 AS 61 - 90 days #38, sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 90) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 120)) THEN 1 ELSE 0 END as bigint))#33 AS 91 - 120 days #39, sum(cast(CASE WHEN ((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 120) THEN 1 ELSE 0 END as bigint))#34 AS >120 days #40] +Functions [5]: [sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 30) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 60) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 90) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))] +Aggregate Attributes [5]: [sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint))#30, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 30) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint))#31, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 60) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint))#32, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 90) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint))#33, sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))#34] +Results [8]: [substr(w_warehouse_name#16, 1, 20)#18 AS substr(w_warehouse_name, 1, 20)#35, sm_type#13, web_name#10, sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint))#30 AS 30 days #36, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 30) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint))#31 AS 31 - 60 days #37, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 60) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint))#32 AS 61 - 90 days #38, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 90) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint))#33 AS 91 - 120 days #39, sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))#34 AS >120 days #40] (32) TakeOrderedAndProject Input [8]: [substr(w_warehouse_name, 1, 20)#35, sm_type#13, web_name#10, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/simplified.txt index 59cfc4b7b249a..9b16b44792ca4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/simplified.txt @@ -12,7 +12,11 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days Project [ws_sold_date_sk,ws_ship_date_sk,ws_ship_mode_sk,ws_warehouse_sk,web_name] BroadcastHashJoin [ws_web_site_sk,web_site_sk] Project [ws_sold_date_sk,ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk] - BroadcastHashJoin [d_date_sk,ws_ship_date_sk] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + Filter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -21,10 +25,6 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] - Filter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt index 4b863587b08d9..5db04537d6371 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt @@ -14,15 +14,15 @@ TakeOrderedAndProject (55) : : :- * Project (17) : : : +- * BroadcastHashJoin Inner BuildRight (16) : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : :- BroadcastExchange (5) - : : : : : +- * Project (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.ship_mode (1) - : : : : +- * Filter (8) - : : : : +- * ColumnarToRow (7) - : : : : +- Scan parquet default.web_sales (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.web_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * Project (7) + : : : : +- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet default.ship_mode (4) : : : +- BroadcastExchange (15) : : : +- * Project (14) : : : +- * Filter (13) @@ -46,60 +46,60 @@ TakeOrderedAndProject (55) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) - : : : +- * BroadcastHashJoin Inner BuildLeft (37) - : : : :- ReusedExchange (33) - : : : +- * Filter (36) - : : : +- * ColumnarToRow (35) - : : : +- Scan parquet default.catalog_sales (34) + : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : :- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.catalog_sales (33) + : : : +- ReusedExchange (36) : : +- ReusedExchange (39) : +- ReusedExchange (42) +- ReusedExchange (45) -(1) Scan parquet default.ship_mode -Output [2]: [sm_ship_mode_sk#1, sm_carrier#2] +(1) Scan parquet default.web_sales +Output [7]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7] Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [In(sm_carrier, [DHL,BARIAN]), IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_date_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [2]: [sm_ship_mode_sk#1, sm_carrier#2] +(2) ColumnarToRow [codegen id : 5] +Input [7]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7] -(3) Filter [codegen id : 1] -Input [2]: [sm_ship_mode_sk#1, sm_carrier#2] -Condition : (sm_carrier#2 IN (DHL,BARIAN) AND isnotnull(sm_ship_mode_sk#1)) +(3) Filter [codegen id : 5] +Input [7]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7] +Condition : (((isnotnull(ws_warehouse_sk#4) AND isnotnull(ws_sold_date_sk#1)) AND isnotnull(ws_sold_time_sk#2)) AND isnotnull(ws_ship_mode_sk#3)) -(4) Project [codegen id : 1] -Output [1]: [sm_ship_mode_sk#1] -Input [2]: [sm_ship_mode_sk#1, sm_carrier#2] +(4) Scan parquet default.ship_mode +Output [2]: [sm_ship_mode_sk#8, sm_carrier#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/ship_mode] +PushedFilters: [In(sm_carrier, [DHL,BARIAN]), IsNotNull(sm_ship_mode_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [1]: [sm_ship_mode_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#3] +(5) ColumnarToRow [codegen id : 1] +Input [2]: [sm_ship_mode_sk#8, sm_carrier#9] -(6) Scan parquet default.web_sales -Output [7]: [ws_sold_date_sk#4, ws_sold_time_sk#5, ws_ship_mode_sk#6, ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_date_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [2]: [sm_ship_mode_sk#8, sm_carrier#9] +Condition : (sm_carrier#9 IN (DHL,BARIAN) AND isnotnull(sm_ship_mode_sk#8)) -(7) ColumnarToRow -Input [7]: [ws_sold_date_sk#4, ws_sold_time_sk#5, ws_ship_mode_sk#6, ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10] +(7) Project [codegen id : 1] +Output [1]: [sm_ship_mode_sk#8] +Input [2]: [sm_ship_mode_sk#8, sm_carrier#9] -(8) Filter -Input [7]: [ws_sold_date_sk#4, ws_sold_time_sk#5, ws_ship_mode_sk#6, ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10] -Condition : (((isnotnull(ws_warehouse_sk#7) AND isnotnull(ws_sold_date_sk#4)) AND isnotnull(ws_sold_time_sk#5)) AND isnotnull(ws_ship_mode_sk#6)) +(8) BroadcastExchange +Input [1]: [sm_ship_mode_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] (9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [sm_ship_mode_sk#1] -Right keys [1]: [ws_ship_mode_sk#6] +Left keys [1]: [ws_ship_mode_sk#3] +Right keys [1]: [sm_ship_mode_sk#8] Join condition: None (10) Project [codegen id : 5] -Output [6]: [ws_sold_date_sk#4, ws_sold_time_sk#5, ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10] -Input [8]: [sm_ship_mode_sk#1, ws_sold_date_sk#4, ws_sold_time_sk#5, ws_ship_mode_sk#6, ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10] +Output [6]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7] +Input [8]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, sm_ship_mode_sk#8] (11) Scan parquet default.time_dim Output [2]: [t_time_sk#11, t_time#12] @@ -124,13 +124,13 @@ Input [1]: [t_time_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] (16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_time_sk#5] +Left keys [1]: [ws_sold_time_sk#2] Right keys [1]: [t_time_sk#11] Join condition: None (17) Project [codegen id : 5] -Output [5]: [ws_sold_date_sk#4, ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10] -Input [7]: [ws_sold_date_sk#4, ws_sold_time_sk#5, ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10, t_time_sk#11] +Output [5]: [ws_sold_date_sk#1, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7] +Input [7]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, t_time_sk#11] (18) Scan parquet default.date_dim Output [3]: [d_date_sk#14, d_year#15, d_moy#16] @@ -151,13 +151,13 @@ Input [3]: [d_date_sk#14, d_year#15, d_moy#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (22) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_date_sk#4] +Left keys [1]: [ws_sold_date_sk#1] Right keys [1]: [d_date_sk#14] Join condition: None (23) Project [codegen id : 5] -Output [6]: [ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10, d_year#15, d_moy#16] -Input [8]: [ws_sold_date_sk#4, ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10, d_date_sk#14, d_year#15, d_moy#16] +Output [6]: [ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, d_year#15, d_moy#16] +Input [8]: [ws_sold_date_sk#1, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, d_date_sk#14, d_year#15, d_moy#16] (24) Scan parquet default.warehouse Output [7]: [w_warehouse_sk#18, w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24] @@ -178,18 +178,18 @@ Input [7]: [w_warehouse_sk#18, w_warehouse_name#19, w_warehouse_sq_ft#20, w_city Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] (28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_warehouse_sk#7] +Left keys [1]: [ws_warehouse_sk#4] Right keys [1]: [w_warehouse_sk#18] Join condition: None (29) Project [codegen id : 5] -Output [11]: [ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10, w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, d_year#15, d_moy#16] -Input [13]: [ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10, d_year#15, d_moy#16, w_warehouse_sk#18, w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24] +Output [11]: [ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, d_year#15, d_moy#16] +Input [13]: [ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, d_year#15, d_moy#16, w_warehouse_sk#18, w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24] (30) HashAggregate [codegen id : 5] -Input [11]: [ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10, w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, d_year#15, d_moy#16] +Input [11]: [ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, d_year#15, d_moy#16] Keys [7]: [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, d_year#15] -Functions [24]: [partial_sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Functions [24]: [partial_sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] Aggregate Attributes [48]: [sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71, sum#72, isEmpty#73] Results [55]: [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, d_year#15, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121] @@ -200,35 +200,35 @@ Arguments: hashpartitioning(w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21 (32) HashAggregate [codegen id : 6] Input [55]: [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, d_year#15, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121] Keys [7]: [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, d_year#15] -Functions [24]: [sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#123, sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#124, sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#125, sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#126, sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#127, sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#128, sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#129, sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#130, sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#131, sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#132, sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#133, sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#134, sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#135, sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#136, sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#137, sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#138, sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#139, sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#140, sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#141, sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#142, sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#143, sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#144, sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#145, sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#146] -Results [32]: [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, DHL,BARIAN AS ship_carriers#147, d_year#15 AS year#148, sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#123 AS jan_sales#149, sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#124 AS feb_sales#150, sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#125 AS mar_sales#151, sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#126 AS apr_sales#152, sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#127 AS may_sales#153, sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#128 AS jun_sales#154, sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#129 AS jul_sales#155, sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#130 AS aug_sales#156, sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#131 AS sep_sales#157, sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#132 AS oct_sales#158, sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#133 AS nov_sales#159, sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#134 AS dec_sales#160, sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#135 AS jan_net#161, sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#136 AS feb_net#162, sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#137 AS mar_net#163, sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#138 AS apr_net#164, sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#139 AS may_net#165, sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#140 AS jun_net#166, sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#141 AS jul_net#167, sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#142 AS aug_net#168, sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#143 AS sep_net#169, sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#144 AS oct_net#170, sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#145 AS nov_net#171, sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#146 AS dec_net#172] +Functions [24]: [sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#123, sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#124, sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#125, sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#126, sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#127, sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#128, sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#129, sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#130, sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#131, sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#132, sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#133, sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#134, sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#135, sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#136, sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#137, sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#138, sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#139, sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#140, sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#141, sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#142, sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#143, sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#144, sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#145, sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#146] +Results [32]: [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, DHL,BARIAN AS ship_carriers#147, d_year#15 AS year#148, sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#123 AS jan_sales#149, sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#124 AS feb_sales#150, sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#125 AS mar_sales#151, sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#126 AS apr_sales#152, sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#127 AS may_sales#153, sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#128 AS jun_sales#154, sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#129 AS jul_sales#155, sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#130 AS aug_sales#156, sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#131 AS sep_sales#157, sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#132 AS oct_sales#158, sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#133 AS nov_sales#159, sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#134 AS dec_sales#160, sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#135 AS jan_net#161, sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#136 AS feb_net#162, sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#137 AS mar_net#163, sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#138 AS apr_net#164, sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#139 AS may_net#165, sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#140 AS jun_net#166, sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#141 AS jul_net#167, sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#142 AS aug_net#168, sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#143 AS sep_net#169, sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#144 AS oct_net#170, sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#145 AS nov_net#171, sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#146 AS dec_net#172] -(33) ReusedExchange [Reuses operator id: 5] -Output [1]: [sm_ship_mode_sk#1] - -(34) Scan parquet default.catalog_sales +(33) Scan parquet default.catalog_sales Output [7]: [cs_sold_date_sk#173, cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] ReadSchema: struct -(35) ColumnarToRow +(34) ColumnarToRow [codegen id : 11] Input [7]: [cs_sold_date_sk#173, cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179] -(36) Filter +(35) Filter [codegen id : 11] Input [7]: [cs_sold_date_sk#173, cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179] Condition : (((isnotnull(cs_warehouse_sk#176) AND isnotnull(cs_sold_date_sk#173)) AND isnotnull(cs_sold_time_sk#174)) AND isnotnull(cs_ship_mode_sk#175)) +(36) ReusedExchange [Reuses operator id: 8] +Output [1]: [sm_ship_mode_sk#8] + (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [sm_ship_mode_sk#1] -Right keys [1]: [cs_ship_mode_sk#175] +Left keys [1]: [cs_ship_mode_sk#175] +Right keys [1]: [sm_ship_mode_sk#8] Join condition: None (38) Project [codegen id : 11] Output [6]: [cs_sold_date_sk#173, cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179] -Input [8]: [sm_ship_mode_sk#1, cs_sold_date_sk#173, cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179] +Input [8]: [cs_sold_date_sk#173, cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, sm_ship_mode_sk#8] (39) ReusedExchange [Reuses operator id: 15] Output [1]: [t_time_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt index 465d269a847c3..ddfb04d8df5e3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt @@ -20,7 +20,11 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat Project [ws_sold_date_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid] BroadcastHashJoin [ws_sold_time_sk,t_time_sk] Project [ws_sold_date_sk,ws_sold_time_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid] - BroadcastHashJoin [sm_ship_mode_sk,ws_ship_mode_sk] + BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + Filter [ws_warehouse_sk,ws_sold_date_sk,ws_sold_time_sk,ws_ship_mode_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -29,10 +33,6 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat ColumnarToRow InputAdapter Scan parquet default.ship_mode [sm_ship_mode_sk,sm_carrier] - Filter [ws_warehouse_sk,ws_sold_date_sk,ws_sold_time_sk,ws_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -68,13 +68,13 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat Project [cs_sold_date_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax] BroadcastHashJoin [cs_sold_time_sk,t_time_sk] Project [cs_sold_date_sk,cs_sold_time_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax] - BroadcastHashJoin [sm_ship_mode_sk,cs_ship_mode_sk] - InputAdapter - ReusedExchange [sm_ship_mode_sk] #3 + BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] Filter [cs_warehouse_sk,cs_sold_date_sk,cs_sold_time_sk,cs_ship_mode_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_sold_date_sk,cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax] + InputAdapter + ReusedExchange [sm_ship_mode_sk] #3 InputAdapter ReusedExchange [t_time_sk] #4 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt index a100b6659f162..3f8106c96379a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt @@ -24,15 +24,15 @@ TakeOrderedAndProject (79) : : : : : :- * Project (17) : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) : : : : : : :- * Project (10) - : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : : :- BroadcastExchange (5) - : : : : : : : : +- * Project (4) - : : : : : : : : +- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet default.household_demographics (1) - : : : : : : : +- * Filter (8) - : : : : : : : +- * ColumnarToRow (7) - : : : : : : : +- Scan parquet default.catalog_sales (6) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : : :- * Filter (3) + : : : : : : : : +- * ColumnarToRow (2) + : : : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : : : +- BroadcastExchange (8) + : : : : : : : +- * Project (7) + : : : : : : : +- * Filter (6) + : : : : : : : +- * ColumnarToRow (5) + : : : : : : : +- Scan parquet default.household_demographics (4) : : : : : : +- BroadcastExchange (15) : : : : : : +- * Project (14) : : : : : : +- * Filter (13) @@ -49,26 +49,26 @@ TakeOrderedAndProject (79) : : : : +- Scan parquet default.item (26) : : : +- BroadcastExchange (43) : : : +- * Project (42) - : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : :- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.date_dim (33) - : : : +- BroadcastExchange (40) - : : : +- * Project (39) - : : : +- * Filter (38) - : : : +- * ColumnarToRow (37) - : : : +- Scan parquet default.date_dim (36) + : : : +- * BroadcastHashJoin Inner BuildLeft (41) + : : : :- BroadcastExchange (37) + : : : : +- * Project (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.date_dim (33) + : : : +- * Filter (40) + : : : +- * ColumnarToRow (39) + : : : +- Scan parquet default.date_dim (38) : : +- * Sort (58) : : +- Exchange (57) : : +- * Project (56) - : : +- * BroadcastHashJoin Inner BuildLeft (55) - : : :- BroadcastExchange (51) - : : : +- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.warehouse (48) - : : +- * Filter (54) - : : +- * ColumnarToRow (53) - : : +- Scan parquet default.inventory (52) + : : +- * BroadcastHashJoin Inner BuildRight (55) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.inventory (48) + : : +- BroadcastExchange (54) + : : +- * Filter (53) + : : +- * ColumnarToRow (52) + : : +- Scan parquet default.warehouse (51) : +- BroadcastExchange (64) : +- * Filter (63) : +- * ColumnarToRow (62) @@ -80,50 +80,50 @@ TakeOrderedAndProject (79) +- Scan parquet default.catalog_returns (69) -(1) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#1, hd_buy_potential#2] +(1) Scan parquet default.catalog_sales +Output [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000), IsNotNull(hd_demo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_ship_date_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [2]: [hd_demo_sk#1, hd_buy_potential#2] +(2) ColumnarToRow [codegen id : 4] +Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] -(3) Filter [codegen id : 1] -Input [2]: [hd_demo_sk#1, hd_buy_potential#2] -Condition : ((isnotnull(hd_buy_potential#2) AND (hd_buy_potential#2 = >10000)) AND isnotnull(hd_demo_sk#1)) +(3) Filter [codegen id : 4] +Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] +Condition : (((((isnotnull(cs_quantity#8) AND isnotnull(cs_item_sk#5)) AND isnotnull(cs_bill_cdemo_sk#3)) AND isnotnull(cs_bill_hdemo_sk#4)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_ship_date_sk#2)) -(4) Project [codegen id : 1] -Output [1]: [hd_demo_sk#1] -Input [2]: [hd_demo_sk#1, hd_buy_potential#2] +(4) Scan parquet default.household_demographics +Output [2]: [hd_demo_sk#9, hd_buy_potential#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000), IsNotNull(hd_demo_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [1]: [hd_demo_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#3] +(5) ColumnarToRow [codegen id : 1] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -(6) Scan parquet default.catalog_sales -Output [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_bill_hdemo_sk#7, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = >10000)) AND isnotnull(hd_demo_sk#9)) -(7) ColumnarToRow -Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_bill_hdemo_sk#7, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] +(7) Project [codegen id : 1] +Output [1]: [hd_demo_sk#9] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -(8) Filter -Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_bill_hdemo_sk#7, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] -Condition : (((((isnotnull(cs_quantity#11) AND isnotnull(cs_item_sk#8)) AND isnotnull(cs_bill_cdemo_sk#6)) AND isnotnull(cs_bill_hdemo_sk#7)) AND isnotnull(cs_sold_date_sk#4)) AND isnotnull(cs_ship_date_sk#5)) +(8) BroadcastExchange +Input [1]: [hd_demo_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [hd_demo_sk#1] -Right keys [1]: [cs_bill_hdemo_sk#7] +Left keys [1]: [cs_bill_hdemo_sk#4] +Right keys [1]: [hd_demo_sk#9] Join condition: None (10) Project [codegen id : 4] -Output [7]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] -Input [9]: [hd_demo_sk#1, cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_bill_hdemo_sk#7, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] +Output [7]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] +Input [9]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, hd_demo_sk#9] (11) Scan parquet default.customer_demographics Output [2]: [cd_demo_sk#12, cd_marital_status#13] @@ -148,13 +148,13 @@ Input [1]: [cd_demo_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_bill_cdemo_sk#6] +Left keys [1]: [cs_bill_cdemo_sk#3] Right keys [1]: [cd_demo_sk#12] Join condition: None (17) Project [codegen id : 4] -Output [6]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] -Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, cd_demo_sk#12] +Output [6]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] +Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, cd_demo_sk#12] (18) Scan parquet default.date_dim Output [2]: [d_date_sk#15, d_date#16] @@ -175,21 +175,21 @@ Input [2]: [d_date_sk#15, d_date#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_ship_date_sk#5] +Left keys [1]: [cs_ship_date_sk#2] Right keys [1]: [d_date_sk#15] Join condition: None (23) Project [codegen id : 4] -Output [6]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16] -Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date_sk#15, d_date#16] +Output [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] +Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date_sk#15, d_date#16] (24) Exchange -Input [6]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16] -Arguments: hashpartitioning(cs_item_sk#8, 5), true, [id=#18] +Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] +Arguments: hashpartitioning(cs_item_sk#5, 5), true, [id=#18] (25) Sort [codegen id : 5] -Input [6]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16] -Arguments: [cs_item_sk#8 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] +Arguments: [cs_item_sk#5 ASC NULLS FIRST], false, 0 (26) Scan parquet default.item Output [2]: [i_item_sk#19, i_item_desc#20] @@ -214,137 +214,137 @@ Input [2]: [i_item_sk#19, i_item_desc#20] Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 (31) SortMergeJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#8] +Left keys [1]: [cs_item_sk#5] Right keys [1]: [i_item_sk#19] Join condition: None (32) Project [codegen id : 10] -Output [7]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16, i_item_desc#20] -Input [8]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16, i_item_sk#19, i_item_desc#20] +Output [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] +Input [8]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_sk#19, i_item_desc#20] (33) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_week_seq#23] +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] +ReadSchema: struct -(34) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#22, d_week_seq#23] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -(35) Filter [codegen id : 9] -Input [2]: [d_date_sk#22, d_week_seq#23] -Condition : (isnotnull(d_week_seq#23) AND isnotnull(d_date_sk#22)) +(35) Filter [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(36) Scan parquet default.date_dim -Output [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct +(36) Project [codegen id : 8] +Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -(37) ColumnarToRow [codegen id : 8] -Input [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#27] +(37) BroadcastExchange +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#26] -(38) Filter [codegen id : 8] -Input [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#27] -Condition : ((((isnotnull(d_year#27) AND (d_year#27 = 1999)) AND isnotnull(d_date_sk#24)) AND isnotnull(d_week_seq#26)) AND isnotnull(d_date#25)) +(38) Scan parquet default.date_dim +Output [2]: [d_date_sk#27, d_week_seq#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct -(39) Project [codegen id : 8] -Output [3]: [d_date_sk#24, d_date#25, d_week_seq#26] -Input [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#27] +(39) ColumnarToRow +Input [2]: [d_date_sk#27, d_week_seq#28] -(40) BroadcastExchange -Input [3]: [d_date_sk#24, d_date#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#28] +(40) Filter +Input [2]: [d_date_sk#27, d_week_seq#28] +Condition : (isnotnull(d_week_seq#28) AND isnotnull(d_date_sk#27)) (41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#23] -Right keys [1]: [d_week_seq#26] +Left keys [1]: [d_week_seq#24] +Right keys [1]: [d_week_seq#28] Join condition: None (42) Project [codegen id : 9] -Output [4]: [d_date_sk#22, d_date_sk#24, d_date#25, d_week_seq#26] -Input [5]: [d_date_sk#22, d_week_seq#23, d_date_sk#24, d_date#25, d_week_seq#26] +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +Input [5]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27, d_week_seq#28] (43) BroadcastExchange -Input [4]: [d_date_sk#22, d_date_sk#24, d_date#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#29] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#4] -Right keys [1]: [d_date_sk#24] -Join condition: (d_date#16 > d_date#25 + 5 days) +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#22] +Join condition: (d_date#16 > d_date#23 + 5 days) (45) Project [codegen id : 10] -Output [7]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, i_item_desc#20, d_date_sk#22, d_week_seq#26] -Input [11]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16, i_item_desc#20, d_date_sk#22, d_date_sk#24, d_date#25, d_week_seq#26] +Output [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Input [11]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] (46) Exchange -Input [7]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, i_item_desc#20, d_date_sk#22, d_week_seq#26] -Arguments: hashpartitioning(cs_item_sk#8, d_date_sk#22, 5), true, [id=#30] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: hashpartitioning(cs_item_sk#5, d_date_sk#27, 5), true, [id=#30] (47) Sort [codegen id : 11] -Input [7]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, i_item_desc#20, d_date_sk#22, d_week_seq#26] -Arguments: [cs_item_sk#8 ASC NULLS FIRST, d_date_sk#22 ASC NULLS FIRST], false, 0 +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: [cs_item_sk#5 ASC NULLS FIRST, d_date_sk#27 ASC NULLS FIRST], false, 0 -(48) Scan parquet default.warehouse -Output [2]: [w_warehouse_sk#31, w_warehouse_name#32] +(48) Scan parquet default.inventory +Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(49) ColumnarToRow [codegen id : 12] -Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] +Location [not included in comparison]/{warehouse_dir}/inventory] +PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] +ReadSchema: struct -(50) Filter [codegen id : 12] -Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] -Condition : isnotnull(w_warehouse_sk#31) +(49) ColumnarToRow [codegen id : 13] +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -(51) BroadcastExchange -Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +(50) Filter [codegen id : 13] +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] +Condition : (((isnotnull(inv_quantity_on_hand#34) AND isnotnull(inv_item_sk#32)) AND isnotnull(inv_warehouse_sk#33)) AND isnotnull(inv_date_sk#31)) -(52) Scan parquet default.inventory -Output [4]: [inv_date_sk#34, inv_item_sk#35, inv_warehouse_sk#36, inv_quantity_on_hand#37] +(51) Scan parquet default.warehouse +Output [2]: [w_warehouse_sk#35, w_warehouse_name#36] Batched: true -Location [not included in comparison]/{warehouse_dir}/inventory] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/warehouse] +PushedFilters: [IsNotNull(w_warehouse_sk)] +ReadSchema: struct + +(52) ColumnarToRow [codegen id : 12] +Input [2]: [w_warehouse_sk#35, w_warehouse_name#36] -(53) ColumnarToRow -Input [4]: [inv_date_sk#34, inv_item_sk#35, inv_warehouse_sk#36, inv_quantity_on_hand#37] +(53) Filter [codegen id : 12] +Input [2]: [w_warehouse_sk#35, w_warehouse_name#36] +Condition : isnotnull(w_warehouse_sk#35) -(54) Filter -Input [4]: [inv_date_sk#34, inv_item_sk#35, inv_warehouse_sk#36, inv_quantity_on_hand#37] -Condition : (((isnotnull(inv_quantity_on_hand#37) AND isnotnull(inv_item_sk#35)) AND isnotnull(inv_warehouse_sk#36)) AND isnotnull(inv_date_sk#34)) +(54) BroadcastExchange +Input [2]: [w_warehouse_sk#35, w_warehouse_name#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] (55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [w_warehouse_sk#31] -Right keys [1]: [inv_warehouse_sk#36] +Left keys [1]: [inv_warehouse_sk#33] +Right keys [1]: [w_warehouse_sk#35] Join condition: None (56) Project [codegen id : 13] -Output [4]: [w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_quantity_on_hand#37] -Input [6]: [w_warehouse_sk#31, w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_warehouse_sk#36, inv_quantity_on_hand#37] +Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Input [6]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] (57) Exchange -Input [4]: [w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_quantity_on_hand#37] -Arguments: hashpartitioning(inv_item_sk#35, inv_date_sk#34, 5), true, [id=#38] +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: hashpartitioning(inv_item_sk#32, inv_date_sk#31, 5), true, [id=#38] (58) Sort [codegen id : 14] -Input [4]: [w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_quantity_on_hand#37] -Arguments: [inv_item_sk#35 ASC NULLS FIRST, inv_date_sk#34 ASC NULLS FIRST], false, 0 +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: [inv_item_sk#32 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 16] -Left keys [2]: [cs_item_sk#8, d_date_sk#22] -Right keys [2]: [inv_item_sk#35, inv_date_sk#34] -Join condition: (inv_quantity_on_hand#37 < cs_quantity#11) +Left keys [2]: [cs_item_sk#5, d_date_sk#27] +Right keys [2]: [inv_item_sk#32, inv_date_sk#31] +Join condition: (inv_quantity_on_hand#34 < cs_quantity#8) (60) Project [codegen id : 16] -Output [6]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Input [11]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, i_item_desc#20, d_date_sk#22, d_week_seq#26, w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_quantity_on_hand#37] +Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [11]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27, inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] (61) Scan parquet default.promotion Output [1]: [p_promo_sk#39] @@ -365,21 +365,21 @@ Input [1]: [p_promo_sk#39] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] (65) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [cs_promo_sk#9] +Left keys [1]: [cs_promo_sk#6] Right keys [1]: [p_promo_sk#39] Join condition: None (66) Project [codegen id : 16] -Output [5]: [cs_item_sk#8, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Input [7]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26, p_promo_sk#39] +Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, p_promo_sk#39] (67) Exchange -Input [5]: [cs_item_sk#8, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Arguments: hashpartitioning(cs_item_sk#8, cs_order_number#10, 5), true, [id=#41] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), true, [id=#41] (68) Sort [codegen id : 17] -Input [5]: [cs_item_sk#8, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Arguments: [cs_item_sk#8 ASC NULLS FIRST, cs_order_number#10 ASC NULLS FIRST], false, 0 +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_order_number#7 ASC NULLS FIRST], false, 0 (69) Scan parquet default.catalog_returns Output [2]: [cr_item_sk#42, cr_order_number#43] @@ -404,33 +404,33 @@ Input [2]: [cr_item_sk#42, cr_order_number#43] Arguments: [cr_item_sk#42 ASC NULLS FIRST, cr_order_number#43 ASC NULLS FIRST], false, 0 (74) SortMergeJoin -Left keys [2]: [cs_item_sk#8, cs_order_number#10] +Left keys [2]: [cs_item_sk#5, cs_order_number#7] Right keys [2]: [cr_item_sk#42, cr_order_number#43] Join condition: None (75) Project [codegen id : 20] -Output [3]: [w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Input [7]: [cs_item_sk#8, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26, cr_item_sk#42, cr_order_number#43] +Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, cr_item_sk#42, cr_order_number#43] (76) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Keys [3]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26] +Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#45] -Results [4]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26, count#46] +Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] (77) Exchange -Input [4]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26, count#46] -Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#32, d_week_seq#26, 5), true, [id=#47] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#24, 5), true, [id=#47] (78) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26, count#46] -Keys [3]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#48] -Results [6]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] +Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] (79) TakeOrderedAndProject -Input [6]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26, no_promo#49, promo#50, total_cnt#51] -Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#32 ASC NULLS FIRST, d_week_seq#26 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#32, d_week_seq#26, no_promo#49, promo#50, total_cnt#51] +Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] +Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt index 39dba3af02359..918508787c4b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom InputAdapter Exchange [cs_item_sk,d_date_sk] #3 WholeStageCodegen (10) - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_date_sk,d_week_seq] + Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] SortMergeJoin [cs_item_sk,i_item_sk] @@ -38,7 +38,11 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [hd_demo_sk,cs_bill_hdemo_sk] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) @@ -47,10 +51,6 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) @@ -79,12 +79,8 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom InputAdapter BroadcastExchange #9 WholeStageCodegen (9) - Project [d_date_sk,d_date_sk,d_date,d_week_seq] + Project [d_date_sk,d_date,d_week_seq,d_date_sk] BroadcastHashJoin [d_week_seq,d_week_seq] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #10 WholeStageCodegen (8) @@ -93,14 +89,22 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter WholeStageCodegen (14) Sort [inv_item_sk,inv_date_sk] InputAdapter Exchange [inv_item_sk,inv_date_sk] #11 WholeStageCodegen (13) - Project [w_warehouse_name,inv_date_sk,inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [w_warehouse_sk,inv_warehouse_sk] + Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #12 WholeStageCodegen (12) @@ -108,10 +112,6 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] - Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #13 WholeStageCodegen (15) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt index 057d786afbcdd..9ac081b356c94 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (108) : : : +- * Project (23) : : : +- * Filter (22) : : : +- * ColumnarToRow (21) - : : : +- Scan parquet default.date_dim (20) + : : : +- Scan parquet default.promotion (20) : : +- BroadcastExchange (31) : : +- * Project (30) : : +- * Filter (29) : : +- * ColumnarToRow (28) - : : +- Scan parquet default.promotion (27) + : : +- Scan parquet default.date_dim (27) : +- BroadcastExchange (37) : +- * Filter (36) : +- * ColumnarToRow (35) @@ -193,67 +193,67 @@ Join condition: None Output [7]: [ss_sold_date_sk#1, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12] Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, i_item_sk#14] -(20) Scan parquet default.date_dim -Output [2]: [d_date_sk#17, d_date#18] +(20) Scan parquet default.promotion +Output [2]: [p_promo_sk#17, p_channel_tv#18] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/promotion] +PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] +ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#17, d_date#18] +Input [2]: [p_promo_sk#17, p_channel_tv#18] (22) Filter [codegen id : 6] -Input [2]: [d_date_sk#17, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 11192)) AND (d_date#18 <= 11222)) AND isnotnull(d_date_sk#17)) +Input [2]: [p_promo_sk#17, p_channel_tv#18] +Condition : ((isnotnull(p_channel_tv#18) AND (p_channel_tv#18 = N)) AND isnotnull(p_promo_sk#17)) (23) Project [codegen id : 6] -Output [1]: [d_date_sk#17] -Input [2]: [d_date_sk#17, d_date#18] +Output [1]: [p_promo_sk#17] +Input [2]: [p_promo_sk#17, p_channel_tv#18] (24) BroadcastExchange -Input [1]: [d_date_sk#17] +Input [1]: [p_promo_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (25) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ss_promo_sk#4] +Right keys [1]: [p_promo_sk#17] Join condition: None (26) Project [codegen id : 9] -Output [6]: [ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12] -Input [8]: [ss_sold_date_sk#1, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#17] +Output [6]: [ss_sold_date_sk#1, ss_store_sk#3, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12] +Input [8]: [ss_sold_date_sk#1, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, p_promo_sk#17] -(27) Scan parquet default.promotion -Output [2]: [p_promo_sk#20, p_channel_tv#21] +(27) Scan parquet default.date_dim +Output [2]: [d_date_sk#20, d_date#21] Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] +ReadSchema: struct (28) ColumnarToRow [codegen id : 7] -Input [2]: [p_promo_sk#20, p_channel_tv#21] +Input [2]: [d_date_sk#20, d_date#21] (29) Filter [codegen id : 7] -Input [2]: [p_promo_sk#20, p_channel_tv#21] -Condition : ((isnotnull(p_channel_tv#21) AND (p_channel_tv#21 = N)) AND isnotnull(p_promo_sk#20)) +Input [2]: [d_date_sk#20, d_date#21] +Condition : (((isnotnull(d_date#21) AND (d_date#21 >= 11192)) AND (d_date#21 <= 11222)) AND isnotnull(d_date_sk#20)) (30) Project [codegen id : 7] -Output [1]: [p_promo_sk#20] -Input [2]: [p_promo_sk#20, p_channel_tv#21] +Output [1]: [d_date_sk#20] +Input [2]: [d_date_sk#20, d_date#21] (31) BroadcastExchange -Input [1]: [p_promo_sk#20] +Input [1]: [d_date_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_promo_sk#4] -Right keys [1]: [p_promo_sk#20] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#20] Join condition: None (33) Project [codegen id : 9] Output [5]: [ss_store_sk#3, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12] -Input [7]: [ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, p_promo_sk#20] +Input [7]: [ss_sold_date_sk#1, ss_store_sk#3, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#20] (34) Scan parquet default.store Output [2]: [s_store_sk#23, s_store_id#24] @@ -366,28 +366,28 @@ Output [7]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_s Input [9]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_item_sk#47, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, i_item_sk#14] (58) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#17] +Output [1]: [p_promo_sk#17] (59) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_sold_date_sk#45] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [cs_promo_sk#48] +Right keys [1]: [p_promo_sk#17] Join condition: None (60) Project [codegen id : 19] -Output [6]: [cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56] -Input [8]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, d_date_sk#17] +Output [6]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56] +Input [8]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, p_promo_sk#17] (61) ReusedExchange [Reuses operator id: 31] -Output [1]: [p_promo_sk#20] +Output [1]: [d_date_sk#20] (62) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_promo_sk#48] -Right keys [1]: [p_promo_sk#20] +Left keys [1]: [cs_sold_date_sk#45] +Right keys [1]: [d_date_sk#20] Join condition: None (63) Project [codegen id : 19] Output [5]: [cs_catalog_page_sk#46, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56] -Input [7]: [cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, p_promo_sk#20] +Input [7]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, d_date_sk#20] (64) Scan parquet default.catalog_page Output [2]: [cp_catalog_page_sk#58, cp_catalog_page_id#59] @@ -500,28 +500,28 @@ Output [7]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales Input [9]: [ws_sold_date_sk#80, ws_item_sk#81, ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, i_item_sk#14] (88) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#17] +Output [1]: [p_promo_sk#17] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#80] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ws_promo_sk#83] +Right keys [1]: [p_promo_sk#17] Join condition: None (90) Project [codegen id : 29] -Output [6]: [ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91] -Input [8]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, d_date_sk#17] +Output [6]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91] +Input [8]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, p_promo_sk#17] (91) ReusedExchange [Reuses operator id: 31] -Output [1]: [p_promo_sk#20] +Output [1]: [d_date_sk#20] (92) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#83] -Right keys [1]: [p_promo_sk#20] +Left keys [1]: [ws_sold_date_sk#80] +Right keys [1]: [d_date_sk#20] Join condition: None (93) Project [codegen id : 29] Output [5]: [ws_web_site_sk#82, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91] -Input [7]: [ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, p_promo_sk#20] +Input [7]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, d_date_sk#20] (94) Scan parquet default.web_site Output [2]: [web_site_sk#93, web_site_id#94] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/simplified.txt index 7b73e4307dcf0..ec00b49e71989 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/simplified.txt @@ -17,9 +17,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] Project [ss_sold_date_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] @@ -54,19 +54,19 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) - Project [d_date_sk] - Filter [d_date,d_date_sk] + Project [p_promo_sk] + Filter [p_channel_tv,p_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Scan parquet default.promotion [p_promo_sk,p_channel_tv] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) - Project [p_promo_sk] - Filter [p_channel_tv,p_promo_sk] + Project [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.promotion [p_promo_sk,p_channel_tv] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) @@ -83,9 +83,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] Project [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_catalog_page_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_sold_date_sk,cs_catalog_page_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] BroadcastHashJoin [cs_item_sk,i_item_sk] Project [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] @@ -112,9 +112,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter ReusedExchange [i_item_sk] #5 InputAdapter - ReusedExchange [d_date_sk] #6 + ReusedExchange [p_promo_sk] #6 InputAdapter - ReusedExchange [p_promo_sk] #7 + ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #12 WholeStageCodegen (18) @@ -131,9 +131,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] BroadcastHashJoin [ws_web_site_sk,web_site_sk] Project [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_web_site_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_promo_sk,p_promo_sk] Project [ws_sold_date_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_sold_date_sk,ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] @@ -160,9 +160,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter ReusedExchange [i_item_sk] #5 InputAdapter - ReusedExchange [d_date_sk] #6 + ReusedExchange [p_promo_sk] #6 InputAdapter - ReusedExchange [p_promo_sk] #7 + ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #16 WholeStageCodegen (28) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/explain.txt index ae0b996ec28be..83ec6391d7736 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/explain.txt @@ -20,15 +20,15 @@ TakeOrderedAndProject (36) : : : +- Scan parquet default.customer_address (4) : : +- BroadcastExchange (21) : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildLeft (19) - : : :- BroadcastExchange (15) - : : : +- * Project (14) - : : : +- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.income_band (11) - : : +- * Filter (18) - : : +- * ColumnarToRow (17) - : : +- Scan parquet default.household_demographics (16) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * Filter (13) + : : : +- * ColumnarToRow (12) + : : : +- Scan parquet default.household_demographics (11) + : : +- BroadcastExchange (18) + : : +- * Project (17) + : : +- * Filter (16) + : : +- * ColumnarToRow (15) + : : +- Scan parquet default.income_band (14) : +- * Filter (27) : +- * ColumnarToRow (26) : +- Scan parquet default.customer_demographics (25) @@ -82,63 +82,63 @@ Join condition: None Output [5]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6] Input [7]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6, ca_address_sk#7] -(11) Scan parquet default.income_band -Output [3]: [ib_income_band_sk#10, ib_lower_bound#11, ib_upper_bound#12] +(11) Scan parquet default.household_demographics +Output [2]: [hd_demo_sk#10, hd_income_band_sk#11] Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] +ReadSchema: struct -(12) ColumnarToRow [codegen id : 2] -Input [3]: [ib_income_band_sk#10, ib_lower_bound#11, ib_upper_bound#12] +(12) ColumnarToRow [codegen id : 3] +Input [2]: [hd_demo_sk#10, hd_income_band_sk#11] -(13) Filter [codegen id : 2] -Input [3]: [ib_income_band_sk#10, ib_lower_bound#11, ib_upper_bound#12] -Condition : ((((isnotnull(ib_lower_bound#11) AND isnotnull(ib_upper_bound#12)) AND (ib_lower_bound#11 >= 38128)) AND (ib_upper_bound#12 <= 88128)) AND isnotnull(ib_income_band_sk#10)) +(13) Filter [codegen id : 3] +Input [2]: [hd_demo_sk#10, hd_income_band_sk#11] +Condition : (isnotnull(hd_demo_sk#10) AND isnotnull(hd_income_band_sk#11)) -(14) Project [codegen id : 2] -Output [1]: [ib_income_band_sk#10] -Input [3]: [ib_income_band_sk#10, ib_lower_bound#11, ib_upper_bound#12] +(14) Scan parquet default.income_band +Output [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/income_band] +PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] +ReadSchema: struct -(15) BroadcastExchange -Input [1]: [ib_income_band_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] +(15) ColumnarToRow [codegen id : 2] +Input [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] -(16) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#14, hd_income_band_sk#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct +(16) Filter [codegen id : 2] +Input [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] +Condition : ((((isnotnull(ib_lower_bound#13) AND isnotnull(ib_upper_bound#14)) AND (ib_lower_bound#13 >= 38128)) AND (ib_upper_bound#14 <= 88128)) AND isnotnull(ib_income_band_sk#12)) -(17) ColumnarToRow -Input [2]: [hd_demo_sk#14, hd_income_band_sk#15] +(17) Project [codegen id : 2] +Output [1]: [ib_income_band_sk#12] +Input [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] -(18) Filter -Input [2]: [hd_demo_sk#14, hd_income_band_sk#15] -Condition : (isnotnull(hd_demo_sk#14) AND isnotnull(hd_income_band_sk#15)) +(18) BroadcastExchange +Input [1]: [ib_income_band_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ib_income_band_sk#10] -Right keys [1]: [hd_income_band_sk#15] +Left keys [1]: [hd_income_band_sk#11] +Right keys [1]: [ib_income_band_sk#12] Join condition: None (20) Project [codegen id : 3] -Output [1]: [hd_demo_sk#14] -Input [3]: [ib_income_band_sk#10, hd_demo_sk#14, hd_income_band_sk#15] +Output [1]: [hd_demo_sk#10] +Input [3]: [hd_demo_sk#10, hd_income_band_sk#11, ib_income_band_sk#12] (21) BroadcastExchange -Input [1]: [hd_demo_sk#14] +Input [1]: [hd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (22) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#14] +Right keys [1]: [hd_demo_sk#10] Join condition: None (23) Project [codegen id : 4] Output [4]: [c_customer_id#1, c_current_cdemo_sk#2, c_first_name#5, c_last_name#6] -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, hd_demo_sk#14] +Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, hd_demo_sk#10] (24) BroadcastExchange Input [4]: [c_customer_id#1, c_current_cdemo_sk#2, c_first_name#5, c_last_name#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/simplified.txt index 1fbc57ee7e47a..16087526bc130 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/simplified.txt @@ -30,7 +30,11 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] BroadcastExchange #4 WholeStageCodegen (3) Project [hd_demo_sk] - BroadcastHashJoin [ib_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Filter [hd_demo_sk,hd_income_band_sk] + ColumnarToRow + InputAdapter + Scan parquet default.household_demographics [hd_demo_sk,hd_income_band_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -39,10 +43,6 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] ColumnarToRow InputAdapter Scan parquet default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - Filter [hd_demo_sk,hd_income_band_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_income_band_sk] Filter [cd_demo_sk] ColumnarToRow InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt index ee550f1af4947..7c3f00d33f24e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt @@ -12,30 +12,30 @@ TakeOrderedAndProject (57) : :- * Project (31) : : +- * BroadcastHashJoin Inner BuildRight (30) : : :- * Project (25) - : : : +- * BroadcastHashJoin Inner BuildLeft (24) - : : : :- BroadcastExchange (5) - : : : : +- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.date_dim (1) - : : : +- * Project (23) - : : : +- * SortMergeJoin Inner (22) - : : : :- * Sort (16) - : : : : +- Exchange (15) - : : : : +- * Project (14) - : : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : : :- * Filter (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- Scan parquet default.web_sales (6) - : : : : +- BroadcastExchange (12) - : : : : +- * Filter (11) - : : : : +- * ColumnarToRow (10) - : : : : +- Scan parquet default.web_page (9) - : : : +- * Sort (21) - : : : +- Exchange (20) - : : : +- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.web_returns (17) + : : : +- * BroadcastHashJoin Inner BuildRight (24) + : : : :- * Project (18) + : : : : +- * SortMergeJoin Inner (17) + : : : : :- * Sort (11) + : : : : : +- Exchange (10) + : : : : : +- * Project (9) + : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : :- * Filter (3) + : : : : : : +- * ColumnarToRow (2) + : : : : : : +- Scan parquet default.web_sales (1) + : : : : : +- BroadcastExchange (7) + : : : : : +- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet default.web_page (4) + : : : : +- * Sort (16) + : : : : +- Exchange (15) + : : : : +- * Filter (14) + : : : : +- * ColumnarToRow (13) + : : : : +- Scan parquet default.web_returns (12) + : : : +- BroadcastExchange (23) + : : : +- * Project (22) + : : : +- * Filter (21) + : : : +- * ColumnarToRow (20) + : : : +- Scan parquet default.date_dim (19) : : +- BroadcastExchange (29) : : +- * Filter (28) : : +- * ColumnarToRow (27) @@ -48,126 +48,126 @@ TakeOrderedAndProject (57) +- * Sort (51) +- Exchange (50) +- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Filter (43) - : +- * ColumnarToRow (42) - : +- Scan parquet default.customer_demographics (41) - +- BroadcastExchange (47) - +- * Filter (46) - +- * ColumnarToRow (45) - +- Scan parquet default.customer_demographics (44) - - -(1) Scan parquet default.date_dim -Output [2]: [d_date_sk#1, d_year#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#1, d_year#2] - -(3) Filter [codegen id : 1] -Input [2]: [d_date_sk#1, d_year#2] -Condition : ((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(4) Project [codegen id : 1] -Output [1]: [d_date_sk#1] -Input [2]: [d_date_sk#1, d_year#2] - -(5) BroadcastExchange -Input [1]: [d_date_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#3] - -(6) Scan parquet default.web_sales -Output [7]: [ws_sold_date_sk#4, ws_item_sk#5, ws_web_page_sk#6, ws_order_number#7, ws_quantity#8, ws_sales_price#9, ws_net_profit#10] + +- * BroadcastHashJoin Inner BuildLeft (48) + :- BroadcastExchange (44) + : +- * Filter (43) + : +- * ColumnarToRow (42) + : +- Scan parquet default.customer_demographics (41) + +- * Filter (47) + +- * ColumnarToRow (46) + +- Scan parquet default.customer_demographics (45) + + +(1) Scan parquet default.web_sales +Output [7]: [ws_sold_date_sk#1, ws_item_sk#2, ws_web_page_sk#3, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_page_sk), IsNotNull(ws_sold_date_sk), Or(Or(And(GreaterThanOrEqual(ws_sales_price,100.00),LessThanOrEqual(ws_sales_price,150.00)),And(GreaterThanOrEqual(ws_sales_price,50.00),LessThanOrEqual(ws_sales_price,100.00))),And(GreaterThanOrEqual(ws_sales_price,150.00),LessThanOrEqual(ws_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ws_net_profit,100.00),LessThanOrEqual(ws_net_profit,200.00)),And(GreaterThanOrEqual(ws_net_profit,150.00),LessThanOrEqual(ws_net_profit,300.00))),And(GreaterThanOrEqual(ws_net_profit,50.00),LessThanOrEqual(ws_net_profit,250.00)))] ReadSchema: struct -(7) ColumnarToRow [codegen id : 3] -Input [7]: [ws_sold_date_sk#4, ws_item_sk#5, ws_web_page_sk#6, ws_order_number#7, ws_quantity#8, ws_sales_price#9, ws_net_profit#10] +(2) ColumnarToRow [codegen id : 2] +Input [7]: [ws_sold_date_sk#1, ws_item_sk#2, ws_web_page_sk#3, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7] -(8) Filter [codegen id : 3] -Input [7]: [ws_sold_date_sk#4, ws_item_sk#5, ws_web_page_sk#6, ws_order_number#7, ws_quantity#8, ws_sales_price#9, ws_net_profit#10] -Condition : (((((isnotnull(ws_item_sk#5) AND isnotnull(ws_order_number#7)) AND isnotnull(ws_web_page_sk#6)) AND isnotnull(ws_sold_date_sk#4)) AND ((((ws_sales_price#9 >= 100.00) AND (ws_sales_price#9 <= 150.00)) OR ((ws_sales_price#9 >= 50.00) AND (ws_sales_price#9 <= 100.00))) OR ((ws_sales_price#9 >= 150.00) AND (ws_sales_price#9 <= 200.00)))) AND ((((ws_net_profit#10 >= 100.00) AND (ws_net_profit#10 <= 200.00)) OR ((ws_net_profit#10 >= 150.00) AND (ws_net_profit#10 <= 300.00))) OR ((ws_net_profit#10 >= 50.00) AND (ws_net_profit#10 <= 250.00)))) +(3) Filter [codegen id : 2] +Input [7]: [ws_sold_date_sk#1, ws_item_sk#2, ws_web_page_sk#3, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7] +Condition : (((((isnotnull(ws_item_sk#2) AND isnotnull(ws_order_number#4)) AND isnotnull(ws_web_page_sk#3)) AND isnotnull(ws_sold_date_sk#1)) AND ((((ws_sales_price#6 >= 100.00) AND (ws_sales_price#6 <= 150.00)) OR ((ws_sales_price#6 >= 50.00) AND (ws_sales_price#6 <= 100.00))) OR ((ws_sales_price#6 >= 150.00) AND (ws_sales_price#6 <= 200.00)))) AND ((((ws_net_profit#7 >= 100.00) AND (ws_net_profit#7 <= 200.00)) OR ((ws_net_profit#7 >= 150.00) AND (ws_net_profit#7 <= 300.00))) OR ((ws_net_profit#7 >= 50.00) AND (ws_net_profit#7 <= 250.00)))) -(9) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#11] +(4) Scan parquet default.web_page +Output [1]: [wp_web_page_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 2] -Input [1]: [wp_web_page_sk#11] +(5) ColumnarToRow [codegen id : 1] +Input [1]: [wp_web_page_sk#8] -(11) Filter [codegen id : 2] -Input [1]: [wp_web_page_sk#11] -Condition : isnotnull(wp_web_page_sk#11) +(6) Filter [codegen id : 1] +Input [1]: [wp_web_page_sk#8] +Condition : isnotnull(wp_web_page_sk#8) -(12) BroadcastExchange -Input [1]: [wp_web_page_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] +(7) BroadcastExchange +Input [1]: [wp_web_page_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#9] -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_web_page_sk#6] -Right keys [1]: [wp_web_page_sk#11] +(8) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ws_web_page_sk#3] +Right keys [1]: [wp_web_page_sk#8] Join condition: None -(14) Project [codegen id : 3] -Output [6]: [ws_sold_date_sk#4, ws_item_sk#5, ws_order_number#7, ws_quantity#8, ws_sales_price#9, ws_net_profit#10] -Input [8]: [ws_sold_date_sk#4, ws_item_sk#5, ws_web_page_sk#6, ws_order_number#7, ws_quantity#8, ws_sales_price#9, ws_net_profit#10, wp_web_page_sk#11] +(9) Project [codegen id : 2] +Output [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7] +Input [8]: [ws_sold_date_sk#1, ws_item_sk#2, ws_web_page_sk#3, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wp_web_page_sk#8] -(15) Exchange -Input [6]: [ws_sold_date_sk#4, ws_item_sk#5, ws_order_number#7, ws_quantity#8, ws_sales_price#9, ws_net_profit#10] -Arguments: hashpartitioning(cast(ws_item_sk#5 as bigint), cast(ws_order_number#7 as bigint), 5), true, [id=#13] +(10) Exchange +Input [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7] +Arguments: hashpartitioning(cast(ws_item_sk#2 as bigint), cast(ws_order_number#4 as bigint), 5), true, [id=#10] -(16) Sort [codegen id : 4] -Input [6]: [ws_sold_date_sk#4, ws_item_sk#5, ws_order_number#7, ws_quantity#8, ws_sales_price#9, ws_net_profit#10] -Arguments: [cast(ws_item_sk#5 as bigint) ASC NULLS FIRST, cast(ws_order_number#7 as bigint) ASC NULLS FIRST], false, 0 +(11) Sort [codegen id : 3] +Input [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7] +Arguments: [cast(ws_item_sk#2 as bigint) ASC NULLS FIRST, cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 -(17) Scan parquet default.web_returns -Output [8]: [wr_item_sk#14, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_order_number#19, wr_fee#20, wr_refunded_cash#21] +(12) Scan parquet default.web_returns +Output [8]: [wr_item_sk#11, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_order_number#16, wr_fee#17, wr_refunded_cash#18] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number), IsNotNull(wr_refunded_cdemo_sk), IsNotNull(wr_returning_cdemo_sk), IsNotNull(wr_refunded_addr_sk), IsNotNull(wr_reason_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 5] -Input [8]: [wr_item_sk#14, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_order_number#19, wr_fee#20, wr_refunded_cash#21] +(13) ColumnarToRow [codegen id : 4] +Input [8]: [wr_item_sk#11, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_order_number#16, wr_fee#17, wr_refunded_cash#18] -(19) Filter [codegen id : 5] -Input [8]: [wr_item_sk#14, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_order_number#19, wr_fee#20, wr_refunded_cash#21] -Condition : (((((isnotnull(wr_item_sk#14) AND isnotnull(wr_order_number#19)) AND isnotnull(wr_refunded_cdemo_sk#15)) AND isnotnull(wr_returning_cdemo_sk#17)) AND isnotnull(wr_refunded_addr_sk#16)) AND isnotnull(wr_reason_sk#18)) +(14) Filter [codegen id : 4] +Input [8]: [wr_item_sk#11, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_order_number#16, wr_fee#17, wr_refunded_cash#18] +Condition : (((((isnotnull(wr_item_sk#11) AND isnotnull(wr_order_number#16)) AND isnotnull(wr_refunded_cdemo_sk#12)) AND isnotnull(wr_returning_cdemo_sk#14)) AND isnotnull(wr_refunded_addr_sk#13)) AND isnotnull(wr_reason_sk#15)) -(20) Exchange -Input [8]: [wr_item_sk#14, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_order_number#19, wr_fee#20, wr_refunded_cash#21] -Arguments: hashpartitioning(wr_item_sk#14, wr_order_number#19, 5), true, [id=#22] +(15) Exchange +Input [8]: [wr_item_sk#11, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_order_number#16, wr_fee#17, wr_refunded_cash#18] +Arguments: hashpartitioning(wr_item_sk#11, wr_order_number#16, 5), true, [id=#19] -(21) Sort [codegen id : 6] -Input [8]: [wr_item_sk#14, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_order_number#19, wr_fee#20, wr_refunded_cash#21] -Arguments: [wr_item_sk#14 ASC NULLS FIRST, wr_order_number#19 ASC NULLS FIRST], false, 0 +(16) Sort [codegen id : 5] +Input [8]: [wr_item_sk#11, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_order_number#16, wr_fee#17, wr_refunded_cash#18] +Arguments: [wr_item_sk#11 ASC NULLS FIRST, wr_order_number#16 ASC NULLS FIRST], false, 0 -(22) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#5 as bigint), cast(ws_order_number#7 as bigint)] -Right keys [2]: [wr_item_sk#14, wr_order_number#19] +(17) SortMergeJoin [codegen id : 9] +Left keys [2]: [cast(ws_item_sk#2 as bigint), cast(ws_order_number#4 as bigint)] +Right keys [2]: [wr_item_sk#11, wr_order_number#16] Join condition: None -(23) Project -Output [10]: [ws_sold_date_sk#4, ws_quantity#8, ws_sales_price#9, ws_net_profit#10, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_fee#20, wr_refunded_cash#21] -Input [14]: [ws_sold_date_sk#4, ws_item_sk#5, ws_order_number#7, ws_quantity#8, ws_sales_price#9, ws_net_profit#10, wr_item_sk#14, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_order_number#19, wr_fee#20, wr_refunded_cash#21] +(18) Project [codegen id : 9] +Output [10]: [ws_sold_date_sk#1, ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_fee#17, wr_refunded_cash#18] +Input [14]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_item_sk#11, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_order_number#16, wr_fee#17, wr_refunded_cash#18] + +(19) Scan parquet default.date_dim +Output [2]: [d_date_sk#20, d_year#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct + +(20) ColumnarToRow [codegen id : 6] +Input [2]: [d_date_sk#20, d_year#21] + +(21) Filter [codegen id : 6] +Input [2]: [d_date_sk#20, d_year#21] +Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2000)) AND isnotnull(d_date_sk#20)) + +(22) Project [codegen id : 6] +Output [1]: [d_date_sk#20] +Input [2]: [d_date_sk#20, d_year#21] + +(23) BroadcastExchange +Input [1]: [d_date_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (24) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ws_sold_date_sk#4] +Left keys [1]: [ws_sold_date_sk#1] +Right keys [1]: [d_date_sk#20] Join condition: None (25) Project [codegen id : 9] -Output [9]: [ws_quantity#8, ws_sales_price#9, ws_net_profit#10, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_fee#20, wr_refunded_cash#21] -Input [11]: [d_date_sk#1, ws_sold_date_sk#4, ws_quantity#8, ws_sales_price#9, ws_net_profit#10, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_fee#20, wr_refunded_cash#21] +Output [9]: [ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_fee#17, wr_refunded_cash#18] +Input [11]: [ws_sold_date_sk#1, ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_fee#17, wr_refunded_cash#18, d_date_sk#20] (26) Scan parquet default.reason Output [2]: [r_reason_sk#23, r_reason_desc#24] @@ -188,13 +188,13 @@ Input [2]: [r_reason_sk#23, r_reason_desc#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] (30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [wr_reason_sk#18] +Left keys [1]: [wr_reason_sk#15] Right keys [1]: [cast(r_reason_sk#23 as bigint)] Join condition: None (31) Project [codegen id : 9] -Output [9]: [ws_quantity#8, ws_sales_price#9, ws_net_profit#10, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_fee#20, wr_refunded_cash#21, r_reason_desc#24] -Input [11]: [ws_quantity#8, ws_sales_price#9, ws_net_profit#10, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_fee#20, wr_refunded_cash#21, r_reason_sk#23, r_reason_desc#24] +Output [9]: [ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_fee#17, wr_refunded_cash#18, r_reason_desc#24] +Input [11]: [ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_fee#17, wr_refunded_cash#18, r_reason_sk#23, r_reason_desc#24] (32) Scan parquet default.customer_address Output [3]: [ca_address_sk#26, ca_state#27, ca_country#28] @@ -219,84 +219,84 @@ Input [2]: [ca_address_sk#26, ca_state#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [wr_refunded_addr_sk#16] +Left keys [1]: [wr_refunded_addr_sk#13] Right keys [1]: [cast(ca_address_sk#26 as bigint)] -Join condition: ((((ca_state#27 IN (IN,OH,NJ) AND (ws_net_profit#10 >= 100.00)) AND (ws_net_profit#10 <= 200.00)) OR ((ca_state#27 IN (WI,CT,KY) AND (ws_net_profit#10 >= 150.00)) AND (ws_net_profit#10 <= 300.00))) OR ((ca_state#27 IN (LA,IA,AR) AND (ws_net_profit#10 >= 50.00)) AND (ws_net_profit#10 <= 250.00))) +Join condition: ((((ca_state#27 IN (IN,OH,NJ) AND (ws_net_profit#7 >= 100.00)) AND (ws_net_profit#7 <= 200.00)) OR ((ca_state#27 IN (WI,CT,KY) AND (ws_net_profit#7 >= 150.00)) AND (ws_net_profit#7 <= 300.00))) OR ((ca_state#27 IN (LA,IA,AR) AND (ws_net_profit#7 >= 50.00)) AND (ws_net_profit#7 <= 250.00))) (38) Project [codegen id : 9] -Output [7]: [ws_quantity#8, ws_sales_price#9, wr_refunded_cdemo_sk#15, wr_returning_cdemo_sk#17, wr_fee#20, wr_refunded_cash#21, r_reason_desc#24] -Input [11]: [ws_quantity#8, ws_sales_price#9, ws_net_profit#10, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_fee#20, wr_refunded_cash#21, r_reason_desc#24, ca_address_sk#26, ca_state#27] +Output [7]: [ws_quantity#5, ws_sales_price#6, wr_refunded_cdemo_sk#12, wr_returning_cdemo_sk#14, wr_fee#17, wr_refunded_cash#18, r_reason_desc#24] +Input [11]: [ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_fee#17, wr_refunded_cash#18, r_reason_desc#24, ca_address_sk#26, ca_state#27] (39) Exchange -Input [7]: [ws_quantity#8, ws_sales_price#9, wr_refunded_cdemo_sk#15, wr_returning_cdemo_sk#17, wr_fee#20, wr_refunded_cash#21, r_reason_desc#24] -Arguments: hashpartitioning(wr_refunded_cdemo_sk#15, wr_returning_cdemo_sk#17, 5), true, [id=#30] +Input [7]: [ws_quantity#5, ws_sales_price#6, wr_refunded_cdemo_sk#12, wr_returning_cdemo_sk#14, wr_fee#17, wr_refunded_cash#18, r_reason_desc#24] +Arguments: hashpartitioning(wr_refunded_cdemo_sk#12, wr_returning_cdemo_sk#14, 5), true, [id=#30] (40) Sort [codegen id : 10] -Input [7]: [ws_quantity#8, ws_sales_price#9, wr_refunded_cdemo_sk#15, wr_returning_cdemo_sk#17, wr_fee#20, wr_refunded_cash#21, r_reason_desc#24] -Arguments: [wr_refunded_cdemo_sk#15 ASC NULLS FIRST, wr_returning_cdemo_sk#17 ASC NULLS FIRST], false, 0 +Input [7]: [ws_quantity#5, ws_sales_price#6, wr_refunded_cdemo_sk#12, wr_returning_cdemo_sk#14, wr_fee#17, wr_refunded_cash#18, r_reason_desc#24] +Arguments: [wr_refunded_cdemo_sk#12 ASC NULLS FIRST, wr_returning_cdemo_sk#14 ASC NULLS FIRST], false, 0 (41) Scan parquet default.customer_demographics Output [3]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status)] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] ReadSchema: struct -(42) ColumnarToRow [codegen id : 12] +(42) ColumnarToRow [codegen id : 11] +Input [3]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33] + +(43) Filter [codegen id : 11] Input [3]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33] +Condition : (((isnotnull(cd_demo_sk#31) AND isnotnull(cd_marital_status#32)) AND isnotnull(cd_education_status#33)) AND ((((cd_marital_status#32 = M) AND (cd_education_status#33 = Advanced Degree)) OR ((cd_marital_status#32 = S) AND (cd_education_status#33 = College))) OR ((cd_marital_status#32 = W) AND (cd_education_status#33 = 2 yr Degree)))) -(43) Filter [codegen id : 12] +(44) BroadcastExchange Input [3]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33] -Condition : ((isnotnull(cd_demo_sk#31) AND isnotnull(cd_marital_status#32)) AND isnotnull(cd_education_status#33)) +Arguments: HashedRelationBroadcastMode(List(input[1, string, false], input[2, string, false]),false), [id=#34] -(44) Scan parquet default.customer_demographics -Output [3]: [cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] +(45) Scan parquet default.customer_demographics +Output [3]: [cd_demo_sk#35, cd_marital_status#36, cd_education_status#37] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 11] -Input [3]: [cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] - -(46) Filter [codegen id : 11] -Input [3]: [cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] -Condition : (((isnotnull(cd_demo_sk#34) AND isnotnull(cd_marital_status#35)) AND isnotnull(cd_education_status#36)) AND ((((cd_marital_status#35 = M) AND (cd_education_status#36 = Advanced Degree)) OR ((cd_marital_status#35 = S) AND (cd_education_status#36 = College))) OR ((cd_marital_status#35 = W) AND (cd_education_status#36 = 2 yr Degree)))) +(46) ColumnarToRow +Input [3]: [cd_demo_sk#35, cd_marital_status#36, cd_education_status#37] -(47) BroadcastExchange -Input [3]: [cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] -Arguments: HashedRelationBroadcastMode(List(input[1, string, false], input[2, string, false]),false), [id=#37] +(47) Filter +Input [3]: [cd_demo_sk#35, cd_marital_status#36, cd_education_status#37] +Condition : ((isnotnull(cd_demo_sk#35) AND isnotnull(cd_marital_status#36)) AND isnotnull(cd_education_status#37)) (48) BroadcastHashJoin [codegen id : 12] Left keys [2]: [cd_marital_status#32, cd_education_status#33] -Right keys [2]: [cd_marital_status#35, cd_education_status#36] +Right keys [2]: [cd_marital_status#36, cd_education_status#37] Join condition: None (49) Project [codegen id : 12] -Output [4]: [cd_demo_sk#31, cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] -Input [6]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33, cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] +Output [4]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33, cd_demo_sk#35] +Input [6]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33, cd_demo_sk#35, cd_marital_status#36, cd_education_status#37] (50) Exchange -Input [4]: [cd_demo_sk#31, cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] -Arguments: hashpartitioning(cast(cd_demo_sk#34 as bigint), cast(cd_demo_sk#31 as bigint), 5), true, [id=#38] +Input [4]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33, cd_demo_sk#35] +Arguments: hashpartitioning(cast(cd_demo_sk#31 as bigint), cast(cd_demo_sk#35 as bigint), 5), true, [id=#38] (51) Sort [codegen id : 13] -Input [4]: [cd_demo_sk#31, cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] -Arguments: [cast(cd_demo_sk#34 as bigint) ASC NULLS FIRST, cast(cd_demo_sk#31 as bigint) ASC NULLS FIRST], false, 0 +Input [4]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33, cd_demo_sk#35] +Arguments: [cast(cd_demo_sk#31 as bigint) ASC NULLS FIRST, cast(cd_demo_sk#35 as bigint) ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 14] -Left keys [2]: [wr_refunded_cdemo_sk#15, wr_returning_cdemo_sk#17] -Right keys [2]: [cast(cd_demo_sk#34 as bigint), cast(cd_demo_sk#31 as bigint)] -Join condition: ((((((cd_marital_status#35 = M) AND (cd_education_status#36 = Advanced Degree)) AND (ws_sales_price#9 >= 100.00)) AND (ws_sales_price#9 <= 150.00)) OR ((((cd_marital_status#35 = S) AND (cd_education_status#36 = College)) AND (ws_sales_price#9 >= 50.00)) AND (ws_sales_price#9 <= 100.00))) OR ((((cd_marital_status#35 = W) AND (cd_education_status#36 = 2 yr Degree)) AND (ws_sales_price#9 >= 150.00)) AND (ws_sales_price#9 <= 200.00))) +Left keys [2]: [wr_refunded_cdemo_sk#12, wr_returning_cdemo_sk#14] +Right keys [2]: [cast(cd_demo_sk#31 as bigint), cast(cd_demo_sk#35 as bigint)] +Join condition: ((((((cd_marital_status#32 = M) AND (cd_education_status#33 = Advanced Degree)) AND (ws_sales_price#6 >= 100.00)) AND (ws_sales_price#6 <= 150.00)) OR ((((cd_marital_status#32 = S) AND (cd_education_status#33 = College)) AND (ws_sales_price#6 >= 50.00)) AND (ws_sales_price#6 <= 100.00))) OR ((((cd_marital_status#32 = W) AND (cd_education_status#33 = 2 yr Degree)) AND (ws_sales_price#6 >= 150.00)) AND (ws_sales_price#6 <= 200.00))) (53) Project [codegen id : 14] -Output [4]: [ws_quantity#8, wr_fee#20, wr_refunded_cash#21, r_reason_desc#24] -Input [11]: [ws_quantity#8, ws_sales_price#9, wr_refunded_cdemo_sk#15, wr_returning_cdemo_sk#17, wr_fee#20, wr_refunded_cash#21, r_reason_desc#24, cd_demo_sk#31, cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] +Output [4]: [ws_quantity#5, wr_fee#17, wr_refunded_cash#18, r_reason_desc#24] +Input [11]: [ws_quantity#5, ws_sales_price#6, wr_refunded_cdemo_sk#12, wr_returning_cdemo_sk#14, wr_fee#17, wr_refunded_cash#18, r_reason_desc#24, cd_demo_sk#31, cd_marital_status#32, cd_education_status#33, cd_demo_sk#35] (54) HashAggregate [codegen id : 14] -Input [4]: [ws_quantity#8, wr_fee#20, wr_refunded_cash#21, r_reason_desc#24] +Input [4]: [ws_quantity#5, wr_fee#17, wr_refunded_cash#18, r_reason_desc#24] Keys [1]: [r_reason_desc#24] -Functions [3]: [partial_avg(cast(ws_quantity#8 as bigint)), partial_avg(UnscaledValue(wr_refunded_cash#21)), partial_avg(UnscaledValue(wr_fee#20))] +Functions [3]: [partial_avg(cast(ws_quantity#5 as bigint)), partial_avg(UnscaledValue(wr_refunded_cash#18)), partial_avg(UnscaledValue(wr_fee#17))] Aggregate Attributes [6]: [sum#39, count#40, sum#41, count#42, sum#43, count#44] Results [7]: [r_reason_desc#24, sum#45, count#46, sum#47, count#48, sum#49, count#50] @@ -307,9 +307,9 @@ Arguments: hashpartitioning(r_reason_desc#24, 5), true, [id=#51] (56) HashAggregate [codegen id : 15] Input [7]: [r_reason_desc#24, sum#45, count#46, sum#47, count#48, sum#49, count#50] Keys [1]: [r_reason_desc#24] -Functions [3]: [avg(cast(ws_quantity#8 as bigint)), avg(UnscaledValue(wr_refunded_cash#21)), avg(UnscaledValue(wr_fee#20))] -Aggregate Attributes [3]: [avg(cast(ws_quantity#8 as bigint))#52, avg(UnscaledValue(wr_refunded_cash#21))#53, avg(UnscaledValue(wr_fee#20))#54] -Results [5]: [substr(r_reason_desc#24, 1, 20) AS substr(r_reason_desc, 1, 20)#55, avg(cast(ws_quantity#8 as bigint))#52 AS avg(ws_quantity)#56, cast((avg(UnscaledValue(wr_refunded_cash#21))#53 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#57, cast((avg(UnscaledValue(wr_fee#20))#54 / 100.0) as decimal(11,6)) AS avg(wr_fee)#58, avg(cast(ws_quantity#8 as bigint))#52 AS aggOrder#59] +Functions [3]: [avg(cast(ws_quantity#5 as bigint)), avg(UnscaledValue(wr_refunded_cash#18)), avg(UnscaledValue(wr_fee#17))] +Aggregate Attributes [3]: [avg(cast(ws_quantity#5 as bigint))#52, avg(UnscaledValue(wr_refunded_cash#18))#53, avg(UnscaledValue(wr_fee#17))#54] +Results [5]: [substr(r_reason_desc#24, 1, 20) AS substr(r_reason_desc, 1, 20)#55, avg(cast(ws_quantity#5 as bigint))#52 AS avg(ws_quantity)#56, cast((avg(UnscaledValue(wr_refunded_cash#18))#53 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#57, cast((avg(UnscaledValue(wr_fee#17))#54 / 100.0) as decimal(11,6)) AS avg(wr_fee)#58, avg(cast(ws_quantity#5 as bigint))#52 AS aggOrder#59] (57) TakeOrderedAndProject Input [5]: [substr(r_reason_desc, 1, 20)#55, avg(ws_quantity)#56, avg(wr_refunded_cash)#57, avg(wr_fee)#58, aggOrder#59] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/simplified.txt index e7aee17172e60..3fa7d84f55966 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/simplified.txt @@ -18,23 +18,15 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),aggOrder,avg(wr_refunded_cas Project [ws_quantity,ws_sales_price,ws_net_profit,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_fee,wr_refunded_cash,r_reason_desc] BroadcastHashJoin [wr_reason_sk,r_reason_sk] Project [ws_quantity,ws_sales_price,ws_net_profit,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_sold_date_sk,ws_quantity,ws_sales_price,ws_net_profit,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] InputAdapter - WholeStageCodegen (4) + WholeStageCodegen (3) Sort [ws_item_sk,ws_order_number] InputAdapter - Exchange [ws_item_sk,ws_order_number] #4 - WholeStageCodegen (3) + Exchange [ws_item_sk,ws_order_number] #3 + WholeStageCodegen (2) Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit] BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] Filter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sold_date_sk,ws_sales_price,ws_net_profit] @@ -42,22 +34,30 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),aggOrder,avg(wr_refunded_cas InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (1) Filter [wp_web_page_sk] ColumnarToRow InputAdapter Scan parquet default.web_page [wp_web_page_sk] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (5) Sort [wr_item_sk,wr_order_number] InputAdapter - Exchange [wr_item_sk,wr_order_number] #6 - WholeStageCodegen (5) + Exchange [wr_item_sk,wr_order_number] #5 + WholeStageCodegen (4) Filter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) @@ -79,12 +79,8 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),aggOrder,avg(wr_refunded_cas InputAdapter Exchange [cd_demo_sk,cd_demo_sk] #9 WholeStageCodegen (12) - Project [cd_demo_sk,cd_demo_sk,cd_marital_status,cd_education_status] + Project [cd_demo_sk,cd_marital_status,cd_education_status,cd_demo_sk] BroadcastHashJoin [cd_marital_status,cd_education_status,cd_marital_status,cd_education_status] - Filter [cd_demo_sk,cd_marital_status,cd_education_status] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #10 WholeStageCodegen (11) @@ -92,3 +88,7 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),aggOrder,avg(wr_refunded_cas ColumnarToRow InputAdapter Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + Filter [cd_demo_sk,cd_marital_status,cd_education_status] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt index 69b02557c4750..4e85516b594f7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt @@ -13,14 +13,14 @@ : : :- * Project (16) : : : +- * BroadcastHashJoin Inner BuildRight (15) : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildLeft (8) - : : : : :- BroadcastExchange (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.customer_demographics (1) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet default.customer (5) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.customer (1) + : : : : +- BroadcastExchange (7) + : : : : +- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet default.customer_demographics (4) : : : +- BroadcastExchange (14) : : : +- * Project (13) : : : +- * Filter (12) @@ -33,61 +33,61 @@ : : +- Scan parquet default.customer_address (17) : +- BroadcastExchange (34) : +- * Project (33) - : +- * BroadcastHashJoin Inner BuildLeft (32) - : :- BroadcastExchange (28) - : : +- * Project (27) - : : +- * Filter (26) - : : +- * ColumnarToRow (25) - : : +- Scan parquet default.date_dim (24) - : +- * Filter (31) - : +- * ColumnarToRow (30) - : +- Scan parquet default.catalog_returns (29) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Filter (26) + : : +- * ColumnarToRow (25) + : : +- Scan parquet default.catalog_returns (24) + : +- BroadcastExchange (31) + : +- * Project (30) + : +- * Filter (29) + : +- * ColumnarToRow (28) + : +- Scan parquet default.date_dim (27) +- BroadcastExchange (40) +- * Filter (39) +- * ColumnarToRow (38) +- Scan parquet default.call_center (37) -(1) Scan parquet default.customer_demographics -Output [3]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3] +(1) Scan parquet default.customer +Output [4]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [3]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3] +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] +ReadSchema: struct -(3) Filter [codegen id : 1] -Input [3]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3] -Condition : ((((cd_marital_status#2 = M) AND (cd_education_status#3 = Unknown)) OR ((cd_marital_status#2 = W) AND (cd_education_status#3 = Advanced Degree))) AND isnotnull(cd_demo_sk#1)) +(2) ColumnarToRow [codegen id : 7] +Input [4]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4] -(4) BroadcastExchange -Input [3]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#4] +(3) Filter [codegen id : 7] +Input [4]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4] +Condition : (((isnotnull(c_customer_sk#1) AND isnotnull(c_current_addr_sk#4)) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) -(5) Scan parquet default.customer -Output [4]: [c_customer_sk#5, c_current_cdemo_sk#6, c_current_hdemo_sk#7, c_current_addr_sk#8] +(4) Scan parquet default.customer_demographics +Output [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] +ReadSchema: struct + +(5) ColumnarToRow [codegen id : 1] +Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] -(6) ColumnarToRow -Input [4]: [c_customer_sk#5, c_current_cdemo_sk#6, c_current_hdemo_sk#7, c_current_addr_sk#8] +(6) Filter [codegen id : 1] +Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] +Condition : ((((cd_marital_status#6 = M) AND (cd_education_status#7 = Unknown)) OR ((cd_marital_status#6 = W) AND (cd_education_status#7 = Advanced Degree))) AND isnotnull(cd_demo_sk#5)) -(7) Filter -Input [4]: [c_customer_sk#5, c_current_cdemo_sk#6, c_current_hdemo_sk#7, c_current_addr_sk#8] -Condition : (((isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#8)) AND isnotnull(c_current_cdemo_sk#6)) AND isnotnull(c_current_hdemo_sk#7)) +(7) BroadcastExchange +Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#8] (8) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cd_demo_sk#1] -Right keys [1]: [c_current_cdemo_sk#6] +Left keys [1]: [c_current_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#5] Join condition: None (9) Project [codegen id : 7] -Output [5]: [cd_marital_status#2, cd_education_status#3, c_customer_sk#5, c_current_hdemo_sk#7, c_current_addr_sk#8] -Input [7]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3, c_customer_sk#5, c_current_cdemo_sk#6, c_current_hdemo_sk#7, c_current_addr_sk#8] +Output [5]: [c_customer_sk#1, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7] +Input [7]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] (10) Scan parquet default.household_demographics Output [2]: [hd_demo_sk#9, hd_buy_potential#10] @@ -112,13 +112,13 @@ Input [1]: [hd_demo_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (15) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_hdemo_sk#7] +Left keys [1]: [c_current_hdemo_sk#3] Right keys [1]: [hd_demo_sk#9] Join condition: None (16) Project [codegen id : 7] -Output [4]: [cd_marital_status#2, cd_education_status#3, c_customer_sk#5, c_current_addr_sk#8] -Input [6]: [cd_marital_status#2, cd_education_status#3, c_customer_sk#5, c_current_hdemo_sk#7, c_current_addr_sk#8, hd_demo_sk#9] +Output [4]: [c_customer_sk#1, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7] +Input [6]: [c_customer_sk#1, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7, hd_demo_sk#9] (17) Scan parquet default.customer_address Output [2]: [ca_address_sk#12, ca_gmt_offset#13] @@ -143,71 +143,71 @@ Input [1]: [ca_address_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#8] +Left keys [1]: [c_current_addr_sk#4] Right keys [1]: [ca_address_sk#12] Join condition: None (23) Project [codegen id : 7] -Output [3]: [cd_marital_status#2, cd_education_status#3, c_customer_sk#5] -Input [5]: [cd_marital_status#2, cd_education_status#3, c_customer_sk#5, c_current_addr_sk#8, ca_address_sk#12] +Output [3]: [c_customer_sk#1, cd_marital_status#6, cd_education_status#7] +Input [5]: [c_customer_sk#1, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7, ca_address_sk#12] -(24) Scan parquet default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_moy#17] +(24) Scan parquet default.catalog_returns +Output [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_customer_sk)] +ReadSchema: struct -(25) ColumnarToRow [codegen id : 4] -Input [3]: [d_date_sk#15, d_year#16, d_moy#17] +(25) ColumnarToRow [codegen id : 5] +Input [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] -(26) Filter [codegen id : 4] -Input [3]: [d_date_sk#15, d_year#16, d_moy#17] -Condition : ((((isnotnull(d_year#16) AND isnotnull(d_moy#17)) AND (d_year#16 = 1998)) AND (d_moy#17 = 11)) AND isnotnull(d_date_sk#15)) +(26) Filter [codegen id : 5] +Input [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +Condition : ((isnotnull(cr_call_center_sk#17) AND isnotnull(cr_returned_date_sk#15)) AND isnotnull(cr_returning_customer_sk#16)) -(27) Project [codegen id : 4] -Output [1]: [d_date_sk#15] -Input [3]: [d_date_sk#15, d_year#16, d_moy#17] +(27) Scan parquet default.date_dim +Output [3]: [d_date_sk#19, d_year#20, d_moy#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] +ReadSchema: struct -(28) BroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +(28) ColumnarToRow [codegen id : 4] +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -(29) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#19, cr_returning_customer_sk#20, cr_call_center_sk#21, cr_net_loss#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct +(29) Filter [codegen id : 4] +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Condition : ((((isnotnull(d_year#20) AND isnotnull(d_moy#21)) AND (d_year#20 = 1998)) AND (d_moy#21 = 11)) AND isnotnull(d_date_sk#19)) -(30) ColumnarToRow -Input [4]: [cr_returned_date_sk#19, cr_returning_customer_sk#20, cr_call_center_sk#21, cr_net_loss#22] +(30) Project [codegen id : 4] +Output [1]: [d_date_sk#19] +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -(31) Filter -Input [4]: [cr_returned_date_sk#19, cr_returning_customer_sk#20, cr_call_center_sk#21, cr_net_loss#22] -Condition : ((isnotnull(cr_call_center_sk#21) AND isnotnull(cr_returned_date_sk#19)) AND isnotnull(cr_returning_customer_sk#20)) +(31) BroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (32) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [d_date_sk#15] -Right keys [1]: [cr_returned_date_sk#19] +Left keys [1]: [cr_returned_date_sk#15] +Right keys [1]: [d_date_sk#19] Join condition: None (33) Project [codegen id : 5] -Output [3]: [cr_returning_customer_sk#20, cr_call_center_sk#21, cr_net_loss#22] -Input [5]: [d_date_sk#15, cr_returned_date_sk#19, cr_returning_customer_sk#20, cr_call_center_sk#21, cr_net_loss#22] +Output [3]: [cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +Input [5]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18, d_date_sk#19] (34) BroadcastExchange -Input [3]: [cr_returning_customer_sk#20, cr_call_center_sk#21, cr_net_loss#22] +Input [3]: [cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (35) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_customer_sk#5] -Right keys [1]: [cr_returning_customer_sk#20] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [cr_returning_customer_sk#16] Join condition: None (36) Project [codegen id : 7] -Output [4]: [cd_marital_status#2, cd_education_status#3, cr_call_center_sk#21, cr_net_loss#22] -Input [6]: [cd_marital_status#2, cd_education_status#3, c_customer_sk#5, cr_returning_customer_sk#20, cr_call_center_sk#21, cr_net_loss#22] +Output [4]: [cd_marital_status#6, cd_education_status#7, cr_call_center_sk#17, cr_net_loss#18] +Input [6]: [c_customer_sk#1, cd_marital_status#6, cd_education_status#7, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] (37) Scan parquet default.call_center Output [4]: [cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] @@ -228,31 +228,31 @@ Input [4]: [cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#2 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] (41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_call_center_sk#21] +Left keys [1]: [cr_call_center_sk#17] Right keys [1]: [cc_call_center_sk#24] Join condition: None (42) Project [codegen id : 7] -Output [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#22, cd_marital_status#2, cd_education_status#3] -Input [8]: [cd_marital_status#2, cd_education_status#3, cr_call_center_sk#21, cr_net_loss#22, cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] +Output [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#18, cd_marital_status#6, cd_education_status#7] +Input [8]: [cd_marital_status#6, cd_education_status#7, cr_call_center_sk#17, cr_net_loss#18, cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] (43) HashAggregate [codegen id : 7] -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#22, cd_marital_status#2, cd_education_status#3] -Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#2, cd_education_status#3] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#22))] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#18, cd_marital_status#6, cd_education_status#7] +Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7] +Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#18))] Aggregate Attributes [1]: [sum#29] -Results [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#2, cd_education_status#3, sum#30] +Results [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] (44) Exchange -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#2, cd_education_status#3, sum#30] -Arguments: hashpartitioning(cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#2, cd_education_status#3, 5), true, [id=#31] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] +Arguments: hashpartitioning(cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, 5), true, [id=#31] (45) HashAggregate [codegen id : 8] -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#2, cd_education_status#3, sum#30] -Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#2, cd_education_status#3] -Functions [1]: [sum(UnscaledValue(cr_net_loss#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#22))#32] -Results [4]: [cc_call_center_id#25 AS Call_Center#33, cc_name#26 AS Call_Center_Name#34, cc_manager#27 AS Manager#35, MakeDecimal(sum(UnscaledValue(cr_net_loss#22))#32,17,2) AS Returns_Loss#36] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] +Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7] +Functions [1]: [sum(UnscaledValue(cr_net_loss#18))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#18))#32] +Results [4]: [cc_call_center_id#25 AS Call_Center#33, cc_name#26 AS Call_Center_Name#34, cc_manager#27 AS Manager#35, MakeDecimal(sum(UnscaledValue(cr_net_loss#18))#32,17,2) AS Returns_Loss#36] (46) Exchange Input [4]: [Call_Center#33, Call_Center_Name#34, Manager#35, Returns_Loss#36] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt index f64791821893d..87beb3b565cc1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt @@ -12,12 +12,16 @@ WholeStageCodegen (9) BroadcastHashJoin [cr_call_center_sk,cc_call_center_sk] Project [cd_marital_status,cd_education_status,cr_call_center_sk,cr_net_loss] BroadcastHashJoin [c_customer_sk,cr_returning_customer_sk] - Project [cd_marital_status,cd_education_status,c_customer_sk] + Project [c_customer_sk,cd_marital_status,cd_education_status] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cd_marital_status,cd_education_status,c_customer_sk,c_current_addr_sk] + Project [c_customer_sk,c_current_addr_sk,cd_marital_status,cd_education_status] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [cd_marital_status,cd_education_status,c_customer_sk,c_current_hdemo_sk,c_current_addr_sk] - BroadcastHashJoin [cd_demo_sk,c_current_cdemo_sk] + Project [c_customer_sk,c_current_hdemo_sk,c_current_addr_sk,cd_marital_status,cd_education_status] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -25,10 +29,6 @@ WholeStageCodegen (9) ColumnarToRow InputAdapter Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -49,7 +49,11 @@ WholeStageCodegen (9) BroadcastExchange #6 WholeStageCodegen (5) Project [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] - BroadcastHashJoin [d_date_sk,cr_returned_date_sk] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Filter [cr_call_center_sk,cr_returned_date_sk,cr_returning_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] InputAdapter BroadcastExchange #7 WholeStageCodegen (4) @@ -58,10 +62,6 @@ WholeStageCodegen (9) ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - Filter [cr_call_center_sk,cr_returned_date_sk,cr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt index c547e7af5d790..34eba382992c3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt @@ -10,15 +10,15 @@ TakeOrderedAndProject (32) : :- * Project (16) : : +- * BroadcastHashJoin Inner BuildRight (15) : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : :- BroadcastExchange (5) - : : : : +- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.date_dim (1) - : : : +- * Filter (8) - : : : +- * ColumnarToRow (7) - : : : +- Scan parquet default.catalog_sales (6) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.catalog_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) : : +- BroadcastExchange (14) : : +- * Filter (13) : : +- * ColumnarToRow (12) @@ -33,50 +33,50 @@ TakeOrderedAndProject (32) +- Scan parquet default.warehouse (23) -(1) Scan parquet default.date_dim -Output [2]: [d_date_sk#1, d_month_seq#2] +(1) Scan parquet default.catalog_sales +Output [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, cs_warehouse_sk#5] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_mode_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_ship_date_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#1, d_month_seq#2] +(2) ColumnarToRow [codegen id : 5] +Input [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, cs_warehouse_sk#5] -(3) Filter [codegen id : 1] -Input [2]: [d_date_sk#1, d_month_seq#2] -Condition : (((isnotnull(d_month_seq#2) AND (d_month_seq#2 >= 1200)) AND (d_month_seq#2 <= 1211)) AND isnotnull(d_date_sk#1)) +(3) Filter [codegen id : 5] +Input [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, cs_warehouse_sk#5] +Condition : (((isnotnull(cs_warehouse_sk#5) AND isnotnull(cs_ship_mode_sk#4)) AND isnotnull(cs_call_center_sk#3)) AND isnotnull(cs_ship_date_sk#2)) -(4) Project [codegen id : 1] -Output [1]: [d_date_sk#1] -Input [2]: [d_date_sk#1, d_month_seq#2] +(4) Scan parquet default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [1]: [d_date_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#3] +(5) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_month_seq#7] -(6) Scan parquet default.catalog_sales -Output [5]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_call_center_sk#6, cs_ship_mode_sk#7, cs_warehouse_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_mode_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) -(7) ColumnarToRow -Input [5]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_call_center_sk#6, cs_ship_mode_sk#7, cs_warehouse_sk#8] +(7) Project [codegen id : 1] +Output [1]: [d_date_sk#6] +Input [2]: [d_date_sk#6, d_month_seq#7] -(8) Filter -Input [5]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_call_center_sk#6, cs_ship_mode_sk#7, cs_warehouse_sk#8] -Condition : (((isnotnull(cs_warehouse_sk#8) AND isnotnull(cs_ship_mode_sk#7)) AND isnotnull(cs_call_center_sk#6)) AND isnotnull(cs_ship_date_sk#5)) +(8) BroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] (9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [cs_ship_date_sk#5] +Left keys [1]: [cs_ship_date_sk#2] +Right keys [1]: [d_date_sk#6] Join condition: None (10) Project [codegen id : 5] -Output [5]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_call_center_sk#6, cs_ship_mode_sk#7, cs_warehouse_sk#8] -Input [6]: [d_date_sk#1, cs_sold_date_sk#4, cs_ship_date_sk#5, cs_call_center_sk#6, cs_ship_mode_sk#7, cs_warehouse_sk#8] +Output [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, cs_warehouse_sk#5] +Input [6]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, cs_warehouse_sk#5, d_date_sk#6] (11) Scan parquet default.ship_mode Output [2]: [sm_ship_mode_sk#9, sm_type#10] @@ -97,13 +97,13 @@ Input [2]: [sm_ship_mode_sk#9, sm_type#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] (15) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_ship_mode_sk#7] +Left keys [1]: [cs_ship_mode_sk#4] Right keys [1]: [sm_ship_mode_sk#9] Join condition: None (16) Project [codegen id : 5] -Output [5]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_call_center_sk#6, cs_warehouse_sk#8, sm_type#10] -Input [7]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_call_center_sk#6, cs_ship_mode_sk#7, cs_warehouse_sk#8, sm_ship_mode_sk#9, sm_type#10] +Output [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_warehouse_sk#5, sm_type#10] +Input [7]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, cs_warehouse_sk#5, sm_ship_mode_sk#9, sm_type#10] (17) Scan parquet default.call_center Output [2]: [cc_call_center_sk#12, cc_name#13] @@ -124,13 +124,13 @@ Input [2]: [cc_call_center_sk#12, cc_name#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] (21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_call_center_sk#6] +Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#12] Join condition: None (22) Project [codegen id : 5] -Output [5]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_warehouse_sk#8, sm_type#10, cc_name#13] -Input [7]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_call_center_sk#6, cs_warehouse_sk#8, sm_type#10, cc_call_center_sk#12, cc_name#13] +Output [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_warehouse_sk#5, sm_type#10, cc_name#13] +Input [7]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_warehouse_sk#5, sm_type#10, cc_call_center_sk#12, cc_name#13] (23) Scan parquet default.warehouse Output [2]: [w_warehouse_sk#15, w_warehouse_name#16] @@ -151,18 +151,18 @@ Input [2]: [w_warehouse_sk#15, w_warehouse_name#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (27) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_warehouse_sk#8] +Left keys [1]: [cs_warehouse_sk#5] Right keys [1]: [w_warehouse_sk#15] Join condition: None (28) Project [codegen id : 5] -Output [5]: [cs_sold_date_sk#4, cs_ship_date_sk#5, w_warehouse_name#16, sm_type#10, cc_name#13] -Input [7]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_warehouse_sk#8, sm_type#10, cc_name#13, w_warehouse_sk#15, w_warehouse_name#16] +Output [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, w_warehouse_name#16, sm_type#10, cc_name#13] +Input [7]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_warehouse_sk#5, sm_type#10, cc_name#13, w_warehouse_sk#15, w_warehouse_name#16] (29) HashAggregate [codegen id : 5] -Input [5]: [cs_sold_date_sk#4, cs_ship_date_sk#5, w_warehouse_name#16, sm_type#10, cc_name#13] +Input [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, w_warehouse_name#16, sm_type#10, cc_name#13] Keys [3]: [substr(w_warehouse_name#16, 1, 20) AS substr(w_warehouse_name#16, 1, 20)#18, sm_type#10, cc_name#13] -Functions [5]: [partial_sum(cast(CASE WHEN ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 30) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 30) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 60)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 60) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 90)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 90) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 120)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN ((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 120) THEN 1 ELSE 0 END as bigint))] +Functions [5]: [partial_sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 30) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 60) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 90) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))] Aggregate Attributes [5]: [sum#19, sum#20, sum#21, sum#22, sum#23] Results [8]: [substr(w_warehouse_name#16, 1, 20)#18, sm_type#10, cc_name#13, sum#24, sum#25, sum#26, sum#27, sum#28] @@ -173,9 +173,9 @@ Arguments: hashpartitioning(substr(w_warehouse_name#16, 1, 20)#18, sm_type#10, c (31) HashAggregate [codegen id : 6] Input [8]: [substr(w_warehouse_name#16, 1, 20)#18, sm_type#10, cc_name#13, sum#24, sum#25, sum#26, sum#27, sum#28] Keys [3]: [substr(w_warehouse_name#16, 1, 20)#18, sm_type#10, cc_name#13] -Functions [5]: [sum(cast(CASE WHEN ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 30) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 30) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 60)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 60) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 90)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 90) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 120)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN ((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 120) THEN 1 ELSE 0 END as bigint))] -Aggregate Attributes [5]: [sum(cast(CASE WHEN ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 30) THEN 1 ELSE 0 END as bigint))#30, sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 30) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 60)) THEN 1 ELSE 0 END as bigint))#31, sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 60) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 90)) THEN 1 ELSE 0 END as bigint))#32, sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 90) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 120)) THEN 1 ELSE 0 END as bigint))#33, sum(cast(CASE WHEN ((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 120) THEN 1 ELSE 0 END as bigint))#34] -Results [8]: [substr(w_warehouse_name#16, 1, 20)#18 AS substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, sum(cast(CASE WHEN ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 30) THEN 1 ELSE 0 END as bigint))#30 AS 30 days #36, sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 30) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 60)) THEN 1 ELSE 0 END as bigint))#31 AS 31 - 60 days #37, sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 60) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 90)) THEN 1 ELSE 0 END as bigint))#32 AS 61 - 90 days #38, sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 90) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 120)) THEN 1 ELSE 0 END as bigint))#33 AS 91 - 120 days #39, sum(cast(CASE WHEN ((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 120) THEN 1 ELSE 0 END as bigint))#34 AS >120 days #40] +Functions [5]: [sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 30) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 60) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 90) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))] +Aggregate Attributes [5]: [sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint))#30, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 30) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint))#31, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 60) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint))#32, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 90) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint))#33, sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))#34] +Results [8]: [substr(w_warehouse_name#16, 1, 20)#18 AS substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint))#30 AS 30 days #36, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 30) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint))#31 AS 31 - 60 days #37, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 60) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint))#32 AS 61 - 90 days #38, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 90) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint))#33 AS 91 - 120 days #39, sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))#34 AS >120 days #40] (32) TakeOrderedAndProject Input [8]: [substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/simplified.txt index de3b1913ae25c..b25b16136992c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/simplified.txt @@ -12,7 +12,11 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , Project [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,cs_warehouse_sk,sm_type] BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] Project [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk] - BroadcastHashJoin [d_date_sk,cs_ship_date_sk] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Filter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -21,10 +25,6 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] - Filter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt index 25a1ca79cc500..dad6098ce4685 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt @@ -496,15 +496,15 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, cou Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42, count(1)#43] -Results [7]: [store AS channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sales#45, count(1)#43 AS number_sales#46, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sales#44, count(1)#43 AS number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] (86) Filter [codegen id : 78] -Input [7]: [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47 as decimal(32,6)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46 as decimal(32,6)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) (87) Project [codegen id : 78] -Output [6]: [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46] -Input [7]: [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47] +Output [6]: [store AS channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] (88) ReusedExchange [Reuses operator id: 4] Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] @@ -584,18 +584,18 @@ Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] -Results [7]: [store AS channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#67, count(1)#65 AS number_sales#68, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69] +Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] (106) Filter [codegen id : 77] -Input [7]: [channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#48, [id=#49] as decimal(32,6)))) +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) (107) Project [codegen id : 77] -Output [6]: [channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68] -Input [7]: [channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69] +Output [6]: [store AS channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] (108) BroadcastExchange -Input [6]: [channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68] +Input [6]: [channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] (109) BroadcastHashJoin [codegen id : 78] @@ -604,12 +604,12 @@ Right keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Join condition: None (110) TakeOrderedAndProject -Input [12]: [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46, channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68] -Arguments: 100, [channel#44 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46, channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68] +Input [12]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] +Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#47, [id=#48] * HashAggregate (136) +- Exchange (135) +- * HashAggregate (134) @@ -780,7 +780,7 @@ Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_d Output [1]: [d_week_seq#29] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#48, [id=#49] +Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] Subquery:4 Hosting operator id = 95 Hosting Expression = Subquery scalar-subquery#50, [id=#51] * Project (144) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt index 37186560cb3b8..d6b8ba4395d2e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] +TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] WholeStageCodegen (78) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #2 WholeStageCodegen (8) @@ -45,7 +45,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_ Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [d_date_sk] #16 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 WholeStageCodegen (38) @@ -190,10 +190,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_ InputAdapter BroadcastExchange #17 WholeStageCodegen (77) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #2 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #18 WholeStageCodegen (76) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index ea0e8319f3fe0..1af2e69d57338 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -446,15 +446,15 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, cou Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] -Results [7]: [store AS channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#42, count(1)#40 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] (76) Filter [codegen id : 52] -Input [7]: [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 as decimal(32,6)) > cast(Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (77) Project [codegen id : 52] -Output [6]: [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43] -Input [7]: [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] +Output [6]: [store AS channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] (78) Scan parquet default.store_sales Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] @@ -537,18 +537,18 @@ Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] -Results [7]: [store AS channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#64, count(1)#62 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66] +Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] (96) Filter [codegen id : 51] -Input [7]: [channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) +Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (97) Project [codegen id : 51] -Output [6]: [channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65] -Input [7]: [channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66] +Output [6]: [store AS channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] +Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] (98) BroadcastExchange -Input [6]: [channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65] +Input [6]: [channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#67] (99) BroadcastHashJoin [codegen id : 52] @@ -557,12 +557,12 @@ Right keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Join condition: None (100) TakeOrderedAndProject -Input [12]: [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43, channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65] -Arguments: 100, [channel#41 ASC NULLS FIRST, i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43, channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65] +Input [12]: [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] +Arguments: 100, [i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] * HashAggregate (126) +- Exchange (125) +- * HashAggregate (124) @@ -733,7 +733,7 @@ Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_d Output [1]: [d_week_seq#28] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -Subquery:3 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:3 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] Subquery:4 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#51, [id=#52] * Project (134) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index 6e6950d4cb33a..7bbf83e3de707 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] +TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] WholeStageCodegen (52) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #2 WholeStageCodegen (8) @@ -45,7 +45,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_ Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [d_date_sk] #13 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 WholeStageCodegen (25) @@ -166,10 +166,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_ InputAdapter BroadcastExchange #14 WholeStageCodegen (51) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #2 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #15 WholeStageCodegen (50) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt index 8c697ff080952..38292528b42fc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt @@ -608,15 +608,15 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, cou Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40, count(1)#41] -Results [7]: [store AS channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sales#43, count(1)#41 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sales#42, count(1)#41 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] (86) Filter [codegen id : 39] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45 as decimal(32,6)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 as decimal(32,6)) > cast(Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (87) Project [codegen id : 39] -Output [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45] +Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] (88) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] @@ -692,15 +692,15 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, cou Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] -Results [7]: [catalog AS channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#61, count(1)#59 AS number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] (105) Filter [codegen id : 78] -Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (106) Project [codegen id : 78] -Output [6]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62] -Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63] +Output [6]: [catalog AS channel#63, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] (107) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] @@ -776,35 +776,35 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, cou Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74, count(1)#75] -Results [7]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sales#77, count(1)#75 AS number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sales#76, count(1)#75 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] (124) Filter [codegen id : 117] -Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (125) Project [codegen id : 117] -Output [6]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78] -Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79] +Output [6]: [web AS channel#79, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] (126) Union (127) HashAggregate [codegen id : 118] -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] -Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#43), partial_sum(number_sales#44)] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] +Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#80, isEmpty#81, sum#82] -Results [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] +Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] (128) Exchange -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] -Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#86] +Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] +Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#86] (129) HashAggregate [codegen id : 119] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] -Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#43), sum(number_sales#44)] -Aggregate Attributes [2]: [sum(sales#43)#87, sum(number_sales#44)#88] -Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#43)#87 AS sum_sales#89, sum(number_sales#44)#88 AS number_sales#90] +Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] +Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#42), sum(number_sales#43)] +Aggregate Attributes [2]: [sum(sales#42)#87, sum(number_sales#43)#88] +Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#42)#87 AS sum_sales#89, sum(number_sales#43)#88 AS number_sales#90] (130) ReusedExchange [Reuses operator id: 84] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#91, isEmpty#92, count#93] @@ -814,15 +814,15 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#91, isEmpty#92, cou Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94, count(1)#95] -Results [7]: [store AS channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94 AS sales#43, count(1)#95 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94 AS sales#42, count(1)#95 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] (132) Filter [codegen id : 158] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (133) Project [codegen id : 158] -Output [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] +Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] (134) ReusedExchange [Reuses operator id: 103] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#97, isEmpty#98, count#99] @@ -832,435 +832,435 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#97, isEmpty#98, cou Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100, count(1)#101] -Results [7]: [catalog AS channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100 AS sales#61, count(1)#101 AS number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100 AS sales#60, count(1)#101 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] (136) Filter [codegen id : 197] -Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (137) Project [codegen id : 197] -Output [6]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62] -Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] +Output [6]: [catalog AS channel#103, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] (138) ReusedExchange [Reuses operator id: 122] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#103, isEmpty#104, count#105] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#104, isEmpty#105, count#106] (139) HashAggregate [codegen id : 236] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#103, isEmpty#104, count#105] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#104, isEmpty#105, count#106] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#106, count(1)#107] -Results [7]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#106 AS sales#77, count(1)#107 AS number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#106 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#108] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#76, count(1)#108 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109] (140) Filter [codegen id : 236] -Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#108] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#108) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#108 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (141) Project [codegen id : 236] -Output [6]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78] -Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#108] +Output [6]: [web AS channel#110, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109] (142) Union (143) HashAggregate [codegen id : 237] -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] -Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#43), partial_sum(number_sales#44)] -Aggregate Attributes [3]: [sum#109, isEmpty#110, sum#111] -Results [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#112, isEmpty#113, sum#114] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] +Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] +Aggregate Attributes [3]: [sum#111, isEmpty#112, sum#113] +Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116] (144) Exchange -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#112, isEmpty#113, sum#114] -Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#115] +Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116] +Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#117] (145) HashAggregate [codegen id : 238] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#112, isEmpty#113, sum#114] -Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#43), sum(number_sales#44)] -Aggregate Attributes [2]: [sum(sales#43)#116, sum(number_sales#44)#117] -Results [5]: [channel#42, i_brand_id#7, i_class_id#8, sum(sales#43)#116 AS sum_sales#89, sum(number_sales#44)#117 AS number_sales#90] +Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116] +Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#42), sum(number_sales#43)] +Aggregate Attributes [2]: [sum(sales#42)#118, sum(number_sales#43)#119] +Results [5]: [channel#47, i_brand_id#7, i_class_id#8, sum(sales#42)#118 AS sum_sales#89, sum(number_sales#43)#119 AS number_sales#90] (146) HashAggregate [codegen id : 238] -Input [5]: [channel#42, i_brand_id#7, i_class_id#8, sum_sales#89, number_sales#90] -Keys [3]: [channel#42, i_brand_id#7, i_class_id#8] +Input [5]: [channel#47, i_brand_id#7, i_class_id#8, sum_sales#89, number_sales#90] +Keys [3]: [channel#47, i_brand_id#7, i_class_id#8] Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] -Aggregate Attributes [3]: [sum#118, isEmpty#119, sum#120] -Results [6]: [channel#42, i_brand_id#7, i_class_id#8, sum#121, isEmpty#122, sum#123] +Aggregate Attributes [3]: [sum#120, isEmpty#121, sum#122] +Results [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125] (147) Exchange -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, sum#121, isEmpty#122, sum#123] -Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, 5), true, [id=#124] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125] +Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, 5), true, [id=#126] (148) HashAggregate [codegen id : 239] -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, sum#121, isEmpty#122, sum#123] -Keys [3]: [channel#42, i_brand_id#7, i_class_id#8] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125] +Keys [3]: [channel#47, i_brand_id#7, i_class_id#8] Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] -Aggregate Attributes [2]: [sum(sum_sales#89)#125, sum(number_sales#90)#126] -Results [6]: [channel#42, i_brand_id#7, i_class_id#8, null AS i_category_id#127, sum(sum_sales#89)#125 AS sum(sum_sales)#128, sum(number_sales#90)#126 AS sum(number_sales)#129] +Aggregate Attributes [2]: [sum(sum_sales#89)#127, sum(number_sales#90)#128] +Results [6]: [channel#47, i_brand_id#7, i_class_id#8, null AS i_category_id#129, sum(sum_sales#89)#127 AS sum(sum_sales)#130, sum(number_sales#90)#128 AS sum(number_sales)#131] (149) Union (150) HashAggregate [codegen id : 240] -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Keys [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] (151) Exchange -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#130] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#132] (152) HashAggregate [codegen id : 241] -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Keys [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] (153) ReusedExchange [Reuses operator id: 84] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#131, isEmpty#132, count#133] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#133, isEmpty#134, count#135] (154) HashAggregate [codegen id : 280] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#131, isEmpty#132, count#133] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#133, isEmpty#134, count#135] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#134, count(1)#135] -Results [7]: [store AS channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#134 AS sales#43, count(1)#135 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#134 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136, count(1)#137] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136 AS sales#42, count(1)#137 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138] (155) Filter [codegen id : 280] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (156) Project [codegen id : 280] -Output [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136] +Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138] (157) ReusedExchange [Reuses operator id: 103] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#137, isEmpty#138, count#139] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#139, isEmpty#140, count#141] (158) HashAggregate [codegen id : 319] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#137, isEmpty#138, count#139] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#139, isEmpty#140, count#141] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#140, count(1)#141] -Results [7]: [catalog AS channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#140 AS sales#61, count(1)#141 AS number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#140 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142, count(1)#143] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142 AS sales#60, count(1)#143 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144] (159) Filter [codegen id : 319] -Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (160) Project [codegen id : 319] -Output [6]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62] -Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142] +Output [6]: [catalog AS channel#145, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144] (161) ReusedExchange [Reuses operator id: 122] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#143, isEmpty#144, count#145] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#146, isEmpty#147, count#148] (162) HashAggregate [codegen id : 358] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#143, isEmpty#144, count#145] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#146, isEmpty#147, count#148] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#146, count(1)#147] -Results [7]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#146 AS sales#77, count(1)#147 AS number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#146 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#148] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#149, count(1)#150] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#149 AS sales#76, count(1)#150 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151] (163) Filter [codegen id : 358] -Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#148] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#148) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#148 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (164) Project [codegen id : 358] -Output [6]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78] -Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#148] +Output [6]: [web AS channel#152, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151] (165) Union (166) HashAggregate [codegen id : 359] -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] -Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#43), partial_sum(number_sales#44)] -Aggregate Attributes [3]: [sum#149, isEmpty#150, sum#151] -Results [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] +Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] +Aggregate Attributes [3]: [sum#153, isEmpty#154, sum#155] +Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#156, isEmpty#157, sum#158] (167) Exchange -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] -Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#155] +Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#156, isEmpty#157, sum#158] +Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#159] (168) HashAggregate [codegen id : 360] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] -Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#43), sum(number_sales#44)] -Aggregate Attributes [2]: [sum(sales#43)#156, sum(number_sales#44)#157] -Results [4]: [channel#42, i_brand_id#7, sum(sales#43)#156 AS sum_sales#89, sum(number_sales#44)#157 AS number_sales#90] +Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#156, isEmpty#157, sum#158] +Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#42), sum(number_sales#43)] +Aggregate Attributes [2]: [sum(sales#42)#160, sum(number_sales#43)#161] +Results [4]: [channel#47, i_brand_id#7, sum(sales#42)#160 AS sum_sales#89, sum(number_sales#43)#161 AS number_sales#90] (169) HashAggregate [codegen id : 360] -Input [4]: [channel#42, i_brand_id#7, sum_sales#89, number_sales#90] -Keys [2]: [channel#42, i_brand_id#7] +Input [4]: [channel#47, i_brand_id#7, sum_sales#89, number_sales#90] +Keys [2]: [channel#47, i_brand_id#7] Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] -Aggregate Attributes [3]: [sum#158, isEmpty#159, sum#160] -Results [5]: [channel#42, i_brand_id#7, sum#161, isEmpty#162, sum#163] +Aggregate Attributes [3]: [sum#162, isEmpty#163, sum#164] +Results [5]: [channel#47, i_brand_id#7, sum#165, isEmpty#166, sum#167] (170) Exchange -Input [5]: [channel#42, i_brand_id#7, sum#161, isEmpty#162, sum#163] -Arguments: hashpartitioning(channel#42, i_brand_id#7, 5), true, [id=#164] +Input [5]: [channel#47, i_brand_id#7, sum#165, isEmpty#166, sum#167] +Arguments: hashpartitioning(channel#47, i_brand_id#7, 5), true, [id=#168] (171) HashAggregate [codegen id : 361] -Input [5]: [channel#42, i_brand_id#7, sum#161, isEmpty#162, sum#163] -Keys [2]: [channel#42, i_brand_id#7] +Input [5]: [channel#47, i_brand_id#7, sum#165, isEmpty#166, sum#167] +Keys [2]: [channel#47, i_brand_id#7] Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] -Aggregate Attributes [2]: [sum(sum_sales#89)#165, sum(number_sales#90)#166] -Results [6]: [channel#42, i_brand_id#7, null AS i_class_id#167, null AS i_category_id#168, sum(sum_sales#89)#165 AS sum(sum_sales)#169, sum(number_sales#90)#166 AS sum(number_sales)#170] +Aggregate Attributes [2]: [sum(sum_sales#89)#169, sum(number_sales#90)#170] +Results [6]: [channel#47, i_brand_id#7, null AS i_class_id#171, null AS i_category_id#172, sum(sum_sales#89)#169 AS sum(sum_sales)#173, sum(number_sales#90)#170 AS sum(number_sales)#174] (172) Union (173) HashAggregate [codegen id : 362] -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Keys [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] (174) Exchange -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#171] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#175] (175) HashAggregate [codegen id : 363] -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Keys [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] (176) ReusedExchange [Reuses operator id: 84] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#172, isEmpty#173, count#174] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#176, isEmpty#177, count#178] (177) HashAggregate [codegen id : 402] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#172, isEmpty#173, count#174] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#176, isEmpty#177, count#178] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#175, count(1)#176] -Results [7]: [store AS channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#175 AS sales#43, count(1)#176 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#175 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#177] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179, count(1)#180] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179 AS sales#42, count(1)#180 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181] (178) Filter [codegen id : 402] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#177] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#177) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#177 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (179) Project [codegen id : 402] -Output [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#177] +Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181] (180) ReusedExchange [Reuses operator id: 103] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#178, isEmpty#179, count#180] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#182, isEmpty#183, count#184] (181) HashAggregate [codegen id : 441] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#178, isEmpty#179, count#180] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#182, isEmpty#183, count#184] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#181, count(1)#182] -Results [7]: [catalog AS channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sales#61, count(1)#182 AS number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#183] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185, count(1)#186] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185 AS sales#60, count(1)#186 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187] (182) Filter [codegen id : 441] -Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#183] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#183) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#183 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (183) Project [codegen id : 441] -Output [6]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62] -Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#183] +Output [6]: [catalog AS channel#188, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187] (184) ReusedExchange [Reuses operator id: 122] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#184, isEmpty#185, count#186] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#189, isEmpty#190, count#191] (185) HashAggregate [codegen id : 480] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#184, isEmpty#185, count#186] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#189, isEmpty#190, count#191] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#187, count(1)#188] -Results [7]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sales#77, count(1)#188 AS number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#189] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192, count(1)#193] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sales#76, count(1)#193 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194] (186) Filter [codegen id : 480] -Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#189] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#189) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#189 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (187) Project [codegen id : 480] -Output [6]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78] -Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#189] +Output [6]: [web AS channel#195, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194] (188) Union (189) HashAggregate [codegen id : 481] -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] -Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#43), partial_sum(number_sales#44)] -Aggregate Attributes [3]: [sum#190, isEmpty#191, sum#192] -Results [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#193, isEmpty#194, sum#195] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] +Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] +Aggregate Attributes [3]: [sum#196, isEmpty#197, sum#198] +Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#199, isEmpty#200, sum#201] (190) Exchange -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#193, isEmpty#194, sum#195] -Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#196] +Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#199, isEmpty#200, sum#201] +Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#202] (191) HashAggregate [codegen id : 482] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#193, isEmpty#194, sum#195] -Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#43), sum(number_sales#44)] -Aggregate Attributes [2]: [sum(sales#43)#197, sum(number_sales#44)#198] -Results [3]: [channel#42, sum(sales#43)#197 AS sum_sales#89, sum(number_sales#44)#198 AS number_sales#90] +Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#199, isEmpty#200, sum#201] +Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#42), sum(number_sales#43)] +Aggregate Attributes [2]: [sum(sales#42)#203, sum(number_sales#43)#204] +Results [3]: [channel#47, sum(sales#42)#203 AS sum_sales#89, sum(number_sales#43)#204 AS number_sales#90] (192) HashAggregate [codegen id : 482] -Input [3]: [channel#42, sum_sales#89, number_sales#90] -Keys [1]: [channel#42] +Input [3]: [channel#47, sum_sales#89, number_sales#90] +Keys [1]: [channel#47] Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] -Aggregate Attributes [3]: [sum#199, isEmpty#200, sum#201] -Results [4]: [channel#42, sum#202, isEmpty#203, sum#204] +Aggregate Attributes [3]: [sum#205, isEmpty#206, sum#207] +Results [4]: [channel#47, sum#208, isEmpty#209, sum#210] (193) Exchange -Input [4]: [channel#42, sum#202, isEmpty#203, sum#204] -Arguments: hashpartitioning(channel#42, 5), true, [id=#205] +Input [4]: [channel#47, sum#208, isEmpty#209, sum#210] +Arguments: hashpartitioning(channel#47, 5), true, [id=#211] (194) HashAggregate [codegen id : 483] -Input [4]: [channel#42, sum#202, isEmpty#203, sum#204] -Keys [1]: [channel#42] +Input [4]: [channel#47, sum#208, isEmpty#209, sum#210] +Keys [1]: [channel#47] Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] -Aggregate Attributes [2]: [sum(sum_sales#89)#206, sum(number_sales#90)#207] -Results [6]: [channel#42, null AS i_brand_id#208, null AS i_class_id#209, null AS i_category_id#210, sum(sum_sales#89)#206 AS sum(sum_sales)#211, sum(number_sales#90)#207 AS sum(number_sales)#212] +Aggregate Attributes [2]: [sum(sum_sales#89)#212, sum(number_sales#90)#213] +Results [6]: [channel#47, null AS i_brand_id#214, null AS i_class_id#215, null AS i_category_id#216, sum(sum_sales#89)#212 AS sum(sum_sales)#217, sum(number_sales#90)#213 AS sum(number_sales)#218] (195) Union (196) HashAggregate [codegen id : 484] -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Keys [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] (197) Exchange -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#213] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#219] (198) HashAggregate [codegen id : 485] -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Keys [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] (199) ReusedExchange [Reuses operator id: 84] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#214, isEmpty#215, count#216] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] (200) HashAggregate [codegen id : 524] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#214, isEmpty#215, count#216] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#217, count(1)#218] -Results [7]: [store AS channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#217 AS sales#43, count(1)#218 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#217 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#219] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#223, count(1)#224] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sales#42, count(1)#224 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225] (201) Filter [codegen id : 524] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#219] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#219) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#219 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (202) Project [codegen id : 524] -Output [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#219] +Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225] (203) ReusedExchange [Reuses operator id: 103] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#226, isEmpty#227, count#228] (204) HashAggregate [codegen id : 563] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#226, isEmpty#227, count#228] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#223, count(1)#224] -Results [7]: [catalog AS channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sales#61, count(1)#224 AS number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#225] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#229, count(1)#230] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#229 AS sales#60, count(1)#230 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#229 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231] (205) Filter [codegen id : 563] -Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#225] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#225) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#225 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (206) Project [codegen id : 563] -Output [6]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62] -Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#225] +Output [6]: [catalog AS channel#232, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231] (207) ReusedExchange [Reuses operator id: 122] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#226, isEmpty#227, count#228] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#233, isEmpty#234, count#235] (208) HashAggregate [codegen id : 602] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#226, isEmpty#227, count#228] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#233, isEmpty#234, count#235] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#229, count(1)#230] -Results [7]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#229 AS sales#77, count(1)#230 AS number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#229 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#231] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#236, count(1)#237] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#236 AS sales#76, count(1)#237 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#236 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238] (209) Filter [codegen id : 602] -Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#231] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#231) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#231 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) (210) Project [codegen id : 602] -Output [6]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78] -Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#231] +Output [6]: [web AS channel#239, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238] (211) Union (212) HashAggregate [codegen id : 603] -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] -Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#43), partial_sum(number_sales#44)] -Aggregate Attributes [3]: [sum#232, isEmpty#233, sum#234] -Results [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#235, isEmpty#236, sum#237] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] +Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] +Aggregate Attributes [3]: [sum#240, isEmpty#241, sum#242] +Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#243, isEmpty#244, sum#245] (213) Exchange -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#235, isEmpty#236, sum#237] -Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#238] +Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#243, isEmpty#244, sum#245] +Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#246] (214) HashAggregate [codegen id : 604] -Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#235, isEmpty#236, sum#237] -Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#43), sum(number_sales#44)] -Aggregate Attributes [2]: [sum(sales#43)#239, sum(number_sales#44)#240] -Results [2]: [sum(sales#43)#239 AS sum_sales#89, sum(number_sales#44)#240 AS number_sales#90] +Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#243, isEmpty#244, sum#245] +Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#42), sum(number_sales#43)] +Aggregate Attributes [2]: [sum(sales#42)#247, sum(number_sales#43)#248] +Results [2]: [sum(sales#42)#247 AS sum_sales#89, sum(number_sales#43)#248 AS number_sales#90] (215) HashAggregate [codegen id : 604] Input [2]: [sum_sales#89, number_sales#90] Keys: [] Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] -Aggregate Attributes [3]: [sum#241, isEmpty#242, sum#243] -Results [3]: [sum#244, isEmpty#245, sum#246] +Aggregate Attributes [3]: [sum#249, isEmpty#250, sum#251] +Results [3]: [sum#252, isEmpty#253, sum#254] (216) Exchange -Input [3]: [sum#244, isEmpty#245, sum#246] -Arguments: SinglePartition, true, [id=#247] +Input [3]: [sum#252, isEmpty#253, sum#254] +Arguments: SinglePartition, true, [id=#255] (217) HashAggregate [codegen id : 605] -Input [3]: [sum#244, isEmpty#245, sum#246] +Input [3]: [sum#252, isEmpty#253, sum#254] Keys: [] Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] -Aggregate Attributes [2]: [sum(sum_sales#89)#248, sum(number_sales#90)#249] -Results [6]: [null AS channel#250, null AS i_brand_id#251, null AS i_class_id#252, null AS i_category_id#253, sum(sum_sales#89)#248 AS sum(sum_sales)#254, sum(number_sales#90)#249 AS sum(number_sales)#255] +Aggregate Attributes [2]: [sum(sum_sales#89)#256, sum(number_sales#90)#257] +Results [6]: [null AS channel#258, null AS i_brand_id#259, null AS i_class_id#260, null AS i_category_id#261, sum(sum_sales#89)#256 AS sum(sum_sales)#262, sum(number_sales#90)#257 AS sum(number_sales)#263] (218) Union (219) HashAggregate [codegen id : 606] -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Keys [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] (220) Exchange -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#256] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#264] (221) HashAggregate [codegen id : 607] -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Keys [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] (222) TakeOrderedAndProject -Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Arguments: 100, [channel#42 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Arguments: 100, [channel#47 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#46, [id=#47] +Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#45, [id=#46] * HashAggregate (252) +- Exchange (251) +- * HashAggregate (250) @@ -1327,7 +1327,7 @@ Input [2]: [d_date_sk#10, d_year#11] (230) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#257] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#265] (231) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] @@ -1335,7 +1335,7 @@ Right keys [1]: [d_date_sk#10] Join condition: None (232) Project [codegen id : 2] -Output [2]: [ss_quantity#3 AS quantity#258, ss_list_price#4 AS list_price#259] +Output [2]: [ss_quantity#3 AS quantity#266, ss_list_price#4 AS list_price#267] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] (233) Scan parquet default.catalog_sales @@ -1372,7 +1372,7 @@ Input [2]: [d_date_sk#10, d_year#11] (240) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#260] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#268] (241) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#18] @@ -1380,7 +1380,7 @@ Right keys [1]: [d_date_sk#10] Join condition: None (242) Project [codegen id : 4] -Output [2]: [cs_quantity#48 AS quantity#261, cs_list_price#49 AS list_price#262] +Output [2]: [cs_quantity#48 AS quantity#269, cs_list_price#49 AS list_price#270] Input [4]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49, d_date_sk#10] (243) Scan parquet default.web_sales @@ -1406,55 +1406,55 @@ Right keys [1]: [d_date_sk#10] Join condition: None (248) Project [codegen id : 6] -Output [2]: [ws_quantity#64 AS quantity#263, ws_list_price#65 AS list_price#264] +Output [2]: [ws_quantity#64 AS quantity#271, ws_list_price#65 AS list_price#272] Input [4]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65, d_date_sk#10] (249) Union (250) HashAggregate [codegen id : 7] -Input [2]: [quantity#258, list_price#259] +Input [2]: [quantity#266, list_price#267] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#258 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#259 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#265, count#266] -Results [2]: [sum#267, count#268] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#266 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#267 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#273, count#274] +Results [2]: [sum#275, count#276] (251) Exchange -Input [2]: [sum#267, count#268] -Arguments: SinglePartition, true, [id=#269] +Input [2]: [sum#275, count#276] +Arguments: SinglePartition, true, [id=#277] (252) HashAggregate [codegen id : 8] -Input [2]: [sum#267, count#268] +Input [2]: [sum#275, count#276] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#258 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#259 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#258 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#259 as decimal(12,2)))), DecimalType(18,2), true))#270] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#258 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#259 as decimal(12,2)))), DecimalType(18,2), true))#270 AS average_sales#271] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#266 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#267 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#266 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#267 as decimal(12,2)))), DecimalType(18,2), true))#278] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#266 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#267 as decimal(12,2)))), DecimalType(18,2), true))#278 AS average_sales#279] -Subquery:2 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:2 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:3 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:3 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:4 Hosting operator id = 132 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:4 Hosting operator id = 132 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:5 Hosting operator id = 136 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:5 Hosting operator id = 136 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:6 Hosting operator id = 140 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:6 Hosting operator id = 140 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:7 Hosting operator id = 155 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:7 Hosting operator id = 155 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:8 Hosting operator id = 159 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:8 Hosting operator id = 159 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:9 Hosting operator id = 163 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:9 Hosting operator id = 163 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:10 Hosting operator id = 178 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:10 Hosting operator id = 178 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:11 Hosting operator id = 182 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:11 Hosting operator id = 182 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:12 Hosting operator id = 186 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:12 Hosting operator id = 186 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:13 Hosting operator id = 201 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:13 Hosting operator id = 201 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:14 Hosting operator id = 205 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:14 Hosting operator id = 205 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:15 Hosting operator id = 209 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:15 Hosting operator id = 209 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt index e4a9b46cf741d..30856e02f2b62 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Union WholeStageCodegen (39) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #1 WholeStageCodegen (8) @@ -90,7 +90,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [d_date_sk] #22 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #6 WholeStageCodegen (38) @@ -226,10 +226,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter ReusedExchange [ss_item_sk] #8 WholeStageCodegen (78) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #23 WholeStageCodegen (77) @@ -258,10 +258,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #18 WholeStageCodegen (117) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #25 WholeStageCodegen (116) @@ -303,24 +303,24 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Union WholeStageCodegen (158) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #6 WholeStageCodegen (197) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #23 WholeStageCodegen (236) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #25 WholeStageCodegen (361) @@ -337,24 +337,24 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Union WholeStageCodegen (280) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #6 WholeStageCodegen (319) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #23 WholeStageCodegen (358) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #25 WholeStageCodegen (483) @@ -371,24 +371,24 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Union WholeStageCodegen (402) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #6 WholeStageCodegen (441) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #23 WholeStageCodegen (480) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #25 WholeStageCodegen (605) @@ -405,23 +405,23 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Union WholeStageCodegen (524) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #6 WholeStageCodegen (563) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #23 WholeStageCodegen (602) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #25 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index c54ad0e36216d..238053a3428e3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -552,15 +552,15 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, cou Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37, count(1)#38] -Results [7]: [store AS channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sales#40, count(1)#38 AS number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sales#39, count(1)#38 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] (76) Filter [codegen id : 26] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 as decimal(32,6)) > cast(Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41 as decimal(32,6)) > cast(Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42] +Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] (78) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] @@ -624,15 +624,15 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, cou Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55] -Results [7]: [catalog AS channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#57, count(1)#55 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] (92) Filter [codegen id : 52] -Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58] -Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59] +Output [6]: [catalog AS channel#59, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] (94) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] @@ -696,35 +696,35 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, cou Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69, count(1)#70] -Results [7]: [web AS channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sales#72, count(1)#70 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sales#71, count(1)#70 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] (108) Filter [codegen id : 78] -Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73] -Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74] +Output [6]: [web AS channel#74, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] (110) Union (111) HashAggregate [codegen id : 79] -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41] -Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [partial_sum(sales#40), partial_sum(number_sales#41)] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] +Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] +Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#75, isEmpty#76, sum#77] -Results [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80] +Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80] (112) Exchange -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80] -Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#81] +Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80] +Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#81] (113) HashAggregate [codegen id : 80] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80] -Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [sum(sales#40), sum(number_sales#41)] -Aggregate Attributes [2]: [sum(sales#40)#82, sum(number_sales#41)#83] -Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum(sales#40)#82 AS sum_sales#84, sum(number_sales#41)#83 AS number_sales#85] +Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80] +Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] +Functions [2]: [sum(sales#39), sum(number_sales#40)] +Aggregate Attributes [2]: [sum(sales#39)#82, sum(number_sales#40)#83] +Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum(sales#39)#82 AS sum_sales#84, sum(number_sales#40)#83 AS number_sales#85] (114) ReusedExchange [Reuses operator id: 74] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#86, isEmpty#87, count#88] @@ -734,15 +734,15 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#86, isEmpty#87, cou Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89, count(1)#90] -Results [7]: [store AS channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89 AS sales#40, count(1)#90 AS number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89 AS sales#39, count(1)#90 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91] (116) Filter [codegen id : 106] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (117) Project [codegen id : 106] -Output [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91] +Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91] (118) ReusedExchange [Reuses operator id: 90] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#92, isEmpty#93, count#94] @@ -752,435 +752,435 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#92, isEmpty#93, cou Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95, count(1)#96] -Results [7]: [catalog AS channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95 AS sales#57, count(1)#96 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95 AS sales#56, count(1)#96 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97] (120) Filter [codegen id : 132] -Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (121) Project [codegen id : 132] -Output [6]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58] -Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97] +Output [6]: [catalog AS channel#98, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97] (122) ReusedExchange [Reuses operator id: 106] -Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#98, isEmpty#99, count#100] +Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#99, isEmpty#100, count#101] (123) HashAggregate [codegen id : 158] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#98, isEmpty#99, count#100] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#99, isEmpty#100, count#101] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#101, count(1)#102] -Results [7]: [web AS channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#101 AS sales#72, count(1)#102 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#101 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#103] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#102, count(1)#103] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sales#71, count(1)#103 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104] (124) Filter [codegen id : 158] -Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#103] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#103) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#103 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (125) Project [codegen id : 158] -Output [6]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73] -Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#103] +Output [6]: [web AS channel#105, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104] (126) Union (127) HashAggregate [codegen id : 159] -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41] -Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [partial_sum(sales#40), partial_sum(number_sales#41)] -Aggregate Attributes [3]: [sum#104, isEmpty#105, sum#106] -Results [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#107, isEmpty#108, sum#109] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] +Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] +Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] +Aggregate Attributes [3]: [sum#106, isEmpty#107, sum#108] +Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#109, isEmpty#110, sum#111] (128) Exchange -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#107, isEmpty#108, sum#109] -Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#110] +Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#109, isEmpty#110, sum#111] +Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#112] (129) HashAggregate [codegen id : 160] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#107, isEmpty#108, sum#109] -Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [sum(sales#40), sum(number_sales#41)] -Aggregate Attributes [2]: [sum(sales#40)#111, sum(number_sales#41)#112] -Results [5]: [channel#39, i_brand_id#6, i_class_id#7, sum(sales#40)#111 AS sum_sales#84, sum(number_sales#41)#112 AS number_sales#85] +Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#109, isEmpty#110, sum#111] +Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] +Functions [2]: [sum(sales#39), sum(number_sales#40)] +Aggregate Attributes [2]: [sum(sales#39)#113, sum(number_sales#40)#114] +Results [5]: [channel#44, i_brand_id#6, i_class_id#7, sum(sales#39)#113 AS sum_sales#84, sum(number_sales#40)#114 AS number_sales#85] (130) HashAggregate [codegen id : 160] -Input [5]: [channel#39, i_brand_id#6, i_class_id#7, sum_sales#84, number_sales#85] -Keys [3]: [channel#39, i_brand_id#6, i_class_id#7] +Input [5]: [channel#44, i_brand_id#6, i_class_id#7, sum_sales#84, number_sales#85] +Keys [3]: [channel#44, i_brand_id#6, i_class_id#7] Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)] -Aggregate Attributes [3]: [sum#113, isEmpty#114, sum#115] -Results [6]: [channel#39, i_brand_id#6, i_class_id#7, sum#116, isEmpty#117, sum#118] +Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] +Results [6]: [channel#44, i_brand_id#6, i_class_id#7, sum#118, isEmpty#119, sum#120] (131) Exchange -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, sum#116, isEmpty#117, sum#118] -Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, 5), true, [id=#119] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, sum#118, isEmpty#119, sum#120] +Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, 5), true, [id=#121] (132) HashAggregate [codegen id : 161] -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, sum#116, isEmpty#117, sum#118] -Keys [3]: [channel#39, i_brand_id#6, i_class_id#7] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, sum#118, isEmpty#119, sum#120] +Keys [3]: [channel#44, i_brand_id#6, i_class_id#7] Functions [2]: [sum(sum_sales#84), sum(number_sales#85)] -Aggregate Attributes [2]: [sum(sum_sales#84)#120, sum(number_sales#85)#121] -Results [6]: [channel#39, i_brand_id#6, i_class_id#7, null AS i_category_id#122, sum(sum_sales#84)#120 AS sum(sum_sales)#123, sum(number_sales#85)#121 AS sum(number_sales)#124] +Aggregate Attributes [2]: [sum(sum_sales#84)#122, sum(number_sales#85)#123] +Results [6]: [channel#44, i_brand_id#6, i_class_id#7, null AS i_category_id#124, sum(sum_sales#84)#122 AS sum(sum_sales)#125, sum(number_sales#85)#123 AS sum(number_sales)#126] (133) Union (134) HashAggregate [codegen id : 162] -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -Keys [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] (135) Exchange -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#125] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#127] (136) HashAggregate [codegen id : 163] -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -Keys [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] (137) ReusedExchange [Reuses operator id: 74] -Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#126, isEmpty#127, count#128] +Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#128, isEmpty#129, count#130] (138) HashAggregate [codegen id : 189] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#126, isEmpty#127, count#128] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#128, isEmpty#129, count#130] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#129, count(1)#130] -Results [7]: [store AS channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#129 AS sales#40, count(1)#130 AS number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#129 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131, count(1)#132] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131 AS sales#39, count(1)#132 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133] (139) Filter [codegen id : 189] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (140) Project [codegen id : 189] -Output [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131] +Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133] (141) ReusedExchange [Reuses operator id: 90] -Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#132, isEmpty#133, count#134] +Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#134, isEmpty#135, count#136] (142) HashAggregate [codegen id : 215] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#132, isEmpty#133, count#134] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#134, isEmpty#135, count#136] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#135, count(1)#136] -Results [7]: [catalog AS channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#135 AS sales#57, count(1)#136 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#135 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137, count(1)#138] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137 AS sales#56, count(1)#138 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139] (143) Filter [codegen id : 215] -Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (144) Project [codegen id : 215] -Output [6]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58] -Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137] +Output [6]: [catalog AS channel#140, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139] (145) ReusedExchange [Reuses operator id: 106] -Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#138, isEmpty#139, count#140] +Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#141, isEmpty#142, count#143] (146) HashAggregate [codegen id : 241] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#138, isEmpty#139, count#140] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#141, isEmpty#142, count#143] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#141, count(1)#142] -Results [7]: [web AS channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#141 AS sales#72, count(1)#142 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#141 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#143] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#144, count(1)#145] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#144 AS sales#71, count(1)#145 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#144 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146] (147) Filter [codegen id : 241] -Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#143] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#143) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#143 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (148) Project [codegen id : 241] -Output [6]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73] -Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#143] +Output [6]: [web AS channel#147, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146] (149) Union (150) HashAggregate [codegen id : 242] -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41] -Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [partial_sum(sales#40), partial_sum(number_sales#41)] -Aggregate Attributes [3]: [sum#144, isEmpty#145, sum#146] -Results [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#147, isEmpty#148, sum#149] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] +Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] +Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] +Aggregate Attributes [3]: [sum#148, isEmpty#149, sum#150] +Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#151, isEmpty#152, sum#153] (151) Exchange -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#147, isEmpty#148, sum#149] -Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#150] +Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#151, isEmpty#152, sum#153] +Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#154] (152) HashAggregate [codegen id : 243] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#147, isEmpty#148, sum#149] -Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [sum(sales#40), sum(number_sales#41)] -Aggregate Attributes [2]: [sum(sales#40)#151, sum(number_sales#41)#152] -Results [4]: [channel#39, i_brand_id#6, sum(sales#40)#151 AS sum_sales#84, sum(number_sales#41)#152 AS number_sales#85] +Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#151, isEmpty#152, sum#153] +Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] +Functions [2]: [sum(sales#39), sum(number_sales#40)] +Aggregate Attributes [2]: [sum(sales#39)#155, sum(number_sales#40)#156] +Results [4]: [channel#44, i_brand_id#6, sum(sales#39)#155 AS sum_sales#84, sum(number_sales#40)#156 AS number_sales#85] (153) HashAggregate [codegen id : 243] -Input [4]: [channel#39, i_brand_id#6, sum_sales#84, number_sales#85] -Keys [2]: [channel#39, i_brand_id#6] +Input [4]: [channel#44, i_brand_id#6, sum_sales#84, number_sales#85] +Keys [2]: [channel#44, i_brand_id#6] Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)] -Aggregate Attributes [3]: [sum#153, isEmpty#154, sum#155] -Results [5]: [channel#39, i_brand_id#6, sum#156, isEmpty#157, sum#158] +Aggregate Attributes [3]: [sum#157, isEmpty#158, sum#159] +Results [5]: [channel#44, i_brand_id#6, sum#160, isEmpty#161, sum#162] (154) Exchange -Input [5]: [channel#39, i_brand_id#6, sum#156, isEmpty#157, sum#158] -Arguments: hashpartitioning(channel#39, i_brand_id#6, 5), true, [id=#159] +Input [5]: [channel#44, i_brand_id#6, sum#160, isEmpty#161, sum#162] +Arguments: hashpartitioning(channel#44, i_brand_id#6, 5), true, [id=#163] (155) HashAggregate [codegen id : 244] -Input [5]: [channel#39, i_brand_id#6, sum#156, isEmpty#157, sum#158] -Keys [2]: [channel#39, i_brand_id#6] +Input [5]: [channel#44, i_brand_id#6, sum#160, isEmpty#161, sum#162] +Keys [2]: [channel#44, i_brand_id#6] Functions [2]: [sum(sum_sales#84), sum(number_sales#85)] -Aggregate Attributes [2]: [sum(sum_sales#84)#160, sum(number_sales#85)#161] -Results [6]: [channel#39, i_brand_id#6, null AS i_class_id#162, null AS i_category_id#163, sum(sum_sales#84)#160 AS sum(sum_sales)#164, sum(number_sales#85)#161 AS sum(number_sales)#165] +Aggregate Attributes [2]: [sum(sum_sales#84)#164, sum(number_sales#85)#165] +Results [6]: [channel#44, i_brand_id#6, null AS i_class_id#166, null AS i_category_id#167, sum(sum_sales#84)#164 AS sum(sum_sales)#168, sum(number_sales#85)#165 AS sum(number_sales)#169] (156) Union (157) HashAggregate [codegen id : 245] -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -Keys [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] (158) Exchange -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#166] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#170] (159) HashAggregate [codegen id : 246] -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -Keys [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] (160) ReusedExchange [Reuses operator id: 74] -Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#167, isEmpty#168, count#169] +Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#171, isEmpty#172, count#173] (161) HashAggregate [codegen id : 272] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#167, isEmpty#168, count#169] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#171, isEmpty#172, count#173] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#170, count(1)#171] -Results [7]: [store AS channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#170 AS sales#40, count(1)#171 AS number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#170 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#172] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174, count(1)#175] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sales#39, count(1)#175 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176] (162) Filter [codegen id : 272] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#172] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#172) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#172 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (163) Project [codegen id : 272] -Output [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#172] +Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176] (164) ReusedExchange [Reuses operator id: 90] -Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#173, isEmpty#174, count#175] +Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#177, isEmpty#178, count#179] (165) HashAggregate [codegen id : 298] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#173, isEmpty#174, count#175] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#177, isEmpty#178, count#179] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#176, count(1)#177] -Results [7]: [catalog AS channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#176 AS sales#57, count(1)#177 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#176 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#178] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180, count(1)#181] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sales#56, count(1)#181 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182] (166) Filter [codegen id : 298] -Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#178] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#178) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#178 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (167) Project [codegen id : 298] -Output [6]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58] -Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#178] +Output [6]: [catalog AS channel#183, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182] (168) ReusedExchange [Reuses operator id: 106] -Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#179, isEmpty#180, count#181] +Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#184, isEmpty#185, count#186] (169) HashAggregate [codegen id : 324] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#179, isEmpty#180, count#181] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#184, isEmpty#185, count#186] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#182, count(1)#183] -Results [7]: [web AS channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#182 AS sales#72, count(1)#183 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#182 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#184] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187, count(1)#188] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sales#71, count(1)#188 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189] (170) Filter [codegen id : 324] -Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#184] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#184) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#184 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (171) Project [codegen id : 324] -Output [6]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73] -Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#184] +Output [6]: [web AS channel#190, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189] (172) Union (173) HashAggregate [codegen id : 325] -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41] -Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [partial_sum(sales#40), partial_sum(number_sales#41)] -Aggregate Attributes [3]: [sum#185, isEmpty#186, sum#187] -Results [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#188, isEmpty#189, sum#190] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] +Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] +Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] +Aggregate Attributes [3]: [sum#191, isEmpty#192, sum#193] +Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#194, isEmpty#195, sum#196] (174) Exchange -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#188, isEmpty#189, sum#190] -Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#191] +Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#194, isEmpty#195, sum#196] +Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#197] (175) HashAggregate [codegen id : 326] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#188, isEmpty#189, sum#190] -Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [sum(sales#40), sum(number_sales#41)] -Aggregate Attributes [2]: [sum(sales#40)#192, sum(number_sales#41)#193] -Results [3]: [channel#39, sum(sales#40)#192 AS sum_sales#84, sum(number_sales#41)#193 AS number_sales#85] +Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#194, isEmpty#195, sum#196] +Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] +Functions [2]: [sum(sales#39), sum(number_sales#40)] +Aggregate Attributes [2]: [sum(sales#39)#198, sum(number_sales#40)#199] +Results [3]: [channel#44, sum(sales#39)#198 AS sum_sales#84, sum(number_sales#40)#199 AS number_sales#85] (176) HashAggregate [codegen id : 326] -Input [3]: [channel#39, sum_sales#84, number_sales#85] -Keys [1]: [channel#39] +Input [3]: [channel#44, sum_sales#84, number_sales#85] +Keys [1]: [channel#44] Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)] -Aggregate Attributes [3]: [sum#194, isEmpty#195, sum#196] -Results [4]: [channel#39, sum#197, isEmpty#198, sum#199] +Aggregate Attributes [3]: [sum#200, isEmpty#201, sum#202] +Results [4]: [channel#44, sum#203, isEmpty#204, sum#205] (177) Exchange -Input [4]: [channel#39, sum#197, isEmpty#198, sum#199] -Arguments: hashpartitioning(channel#39, 5), true, [id=#200] +Input [4]: [channel#44, sum#203, isEmpty#204, sum#205] +Arguments: hashpartitioning(channel#44, 5), true, [id=#206] (178) HashAggregate [codegen id : 327] -Input [4]: [channel#39, sum#197, isEmpty#198, sum#199] -Keys [1]: [channel#39] +Input [4]: [channel#44, sum#203, isEmpty#204, sum#205] +Keys [1]: [channel#44] Functions [2]: [sum(sum_sales#84), sum(number_sales#85)] -Aggregate Attributes [2]: [sum(sum_sales#84)#201, sum(number_sales#85)#202] -Results [6]: [channel#39, null AS i_brand_id#203, null AS i_class_id#204, null AS i_category_id#205, sum(sum_sales#84)#201 AS sum(sum_sales)#206, sum(number_sales#85)#202 AS sum(number_sales)#207] +Aggregate Attributes [2]: [sum(sum_sales#84)#207, sum(number_sales#85)#208] +Results [6]: [channel#44, null AS i_brand_id#209, null AS i_class_id#210, null AS i_category_id#211, sum(sum_sales#84)#207 AS sum(sum_sales)#212, sum(number_sales#85)#208 AS sum(number_sales)#213] (179) Union (180) HashAggregate [codegen id : 328] -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -Keys [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] (181) Exchange -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#208] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#214] (182) HashAggregate [codegen id : 329] -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -Keys [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] (183) ReusedExchange [Reuses operator id: 74] -Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#209, isEmpty#210, count#211] +Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#215, isEmpty#216, count#217] (184) HashAggregate [codegen id : 355] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#209, isEmpty#210, count#211] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#215, isEmpty#216, count#217] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#212, count(1)#213] -Results [7]: [store AS channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#212 AS sales#40, count(1)#213 AS number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#212 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#214] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#218, count(1)#219] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#218 AS sales#39, count(1)#219 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#218 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220] (185) Filter [codegen id : 355] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#214] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#214) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#214 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (186) Project [codegen id : 355] -Output [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#214] +Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220] (187) ReusedExchange [Reuses operator id: 90] -Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#215, isEmpty#216, count#217] +Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#221, isEmpty#222, count#223] (188) HashAggregate [codegen id : 381] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#215, isEmpty#216, count#217] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#221, isEmpty#222, count#223] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#218, count(1)#219] -Results [7]: [catalog AS channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#218 AS sales#57, count(1)#219 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#218 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#220] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#224, count(1)#225] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sales#56, count(1)#225 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226] (189) Filter [codegen id : 381] -Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#220] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#220) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#220 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (190) Project [codegen id : 381] -Output [6]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58] -Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#220] +Output [6]: [catalog AS channel#227, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226] (191) ReusedExchange [Reuses operator id: 106] -Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#221, isEmpty#222, count#223] +Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#228, isEmpty#229, count#230] (192) HashAggregate [codegen id : 407] -Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#221, isEmpty#222, count#223] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#228, isEmpty#229, count#230] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#224, count(1)#225] -Results [7]: [web AS channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sales#72, count(1)#225 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#226] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#231, count(1)#232] +Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#231 AS sales#71, count(1)#232 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#231 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233] (193) Filter [codegen id : 407] -Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#226] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#226) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#226 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6)))) +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) (194) Project [codegen id : 407] -Output [6]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73] -Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#226] +Output [6]: [web AS channel#234, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] +Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233] (195) Union (196) HashAggregate [codegen id : 408] -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41] -Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [partial_sum(sales#40), partial_sum(number_sales#41)] -Aggregate Attributes [3]: [sum#227, isEmpty#228, sum#229] -Results [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#230, isEmpty#231, sum#232] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] +Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] +Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] +Aggregate Attributes [3]: [sum#235, isEmpty#236, sum#237] +Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#238, isEmpty#239, sum#240] (197) Exchange -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#230, isEmpty#231, sum#232] -Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#233] +Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#238, isEmpty#239, sum#240] +Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#241] (198) HashAggregate [codegen id : 409] -Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#230, isEmpty#231, sum#232] -Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8] -Functions [2]: [sum(sales#40), sum(number_sales#41)] -Aggregate Attributes [2]: [sum(sales#40)#234, sum(number_sales#41)#235] -Results [2]: [sum(sales#40)#234 AS sum_sales#84, sum(number_sales#41)#235 AS number_sales#85] +Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#238, isEmpty#239, sum#240] +Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] +Functions [2]: [sum(sales#39), sum(number_sales#40)] +Aggregate Attributes [2]: [sum(sales#39)#242, sum(number_sales#40)#243] +Results [2]: [sum(sales#39)#242 AS sum_sales#84, sum(number_sales#40)#243 AS number_sales#85] (199) HashAggregate [codegen id : 409] Input [2]: [sum_sales#84, number_sales#85] Keys: [] Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)] -Aggregate Attributes [3]: [sum#236, isEmpty#237, sum#238] -Results [3]: [sum#239, isEmpty#240, sum#241] +Aggregate Attributes [3]: [sum#244, isEmpty#245, sum#246] +Results [3]: [sum#247, isEmpty#248, sum#249] (200) Exchange -Input [3]: [sum#239, isEmpty#240, sum#241] -Arguments: SinglePartition, true, [id=#242] +Input [3]: [sum#247, isEmpty#248, sum#249] +Arguments: SinglePartition, true, [id=#250] (201) HashAggregate [codegen id : 410] -Input [3]: [sum#239, isEmpty#240, sum#241] +Input [3]: [sum#247, isEmpty#248, sum#249] Keys: [] Functions [2]: [sum(sum_sales#84), sum(number_sales#85)] -Aggregate Attributes [2]: [sum(sum_sales#84)#243, sum(number_sales#85)#244] -Results [6]: [null AS channel#245, null AS i_brand_id#246, null AS i_class_id#247, null AS i_category_id#248, sum(sum_sales#84)#243 AS sum(sum_sales)#249, sum(number_sales#85)#244 AS sum(number_sales)#250] +Aggregate Attributes [2]: [sum(sum_sales#84)#251, sum(number_sales#85)#252] +Results [6]: [null AS channel#253, null AS i_brand_id#254, null AS i_class_id#255, null AS i_category_id#256, sum(sum_sales#84)#251 AS sum(sum_sales)#257, sum(number_sales#85)#252 AS sum(number_sales)#258] (202) Union (203) HashAggregate [codegen id : 411] -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -Keys [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] (204) Exchange -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#251] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#259] (205) HashAggregate [codegen id : 412] -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -Keys [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] (206) TakeOrderedAndProject -Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -Arguments: 100, [channel#39 ASC NULLS FIRST, i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] +Arguments: 100, [channel#44 ASC NULLS FIRST, i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#43, [id=#44] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#42, [id=#43] * HashAggregate (236) +- Exchange (235) +- * HashAggregate (234) @@ -1247,7 +1247,7 @@ Input [2]: [d_date_sk#10, d_year#11] (214) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#252] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#260] (215) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] @@ -1255,7 +1255,7 @@ Right keys [1]: [d_date_sk#10] Join condition: None (216) Project [codegen id : 2] -Output [2]: [ss_quantity#3 AS quantity#253, ss_list_price#4 AS list_price#254] +Output [2]: [ss_quantity#3 AS quantity#261, ss_list_price#4 AS list_price#262] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] (217) Scan parquet default.catalog_sales @@ -1292,7 +1292,7 @@ Input [2]: [d_date_sk#10, d_year#11] (224) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#255] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#263] (225) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#16] @@ -1300,7 +1300,7 @@ Right keys [1]: [d_date_sk#10] Join condition: None (226) Project [codegen id : 4] -Output [2]: [cs_quantity#45 AS quantity#256, cs_list_price#46 AS list_price#257] +Output [2]: [cs_quantity#45 AS quantity#264, cs_list_price#46 AS list_price#265] Input [4]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, d_date_sk#10] (227) Scan parquet default.web_sales @@ -1326,55 +1326,55 @@ Right keys [1]: [d_date_sk#10] Join condition: None (232) Project [codegen id : 6] -Output [2]: [ws_quantity#60 AS quantity#258, ws_list_price#61 AS list_price#259] +Output [2]: [ws_quantity#60 AS quantity#266, ws_list_price#61 AS list_price#267] Input [4]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, d_date_sk#10] (233) Union (234) HashAggregate [codegen id : 7] -Input [2]: [quantity#253, list_price#254] +Input [2]: [quantity#261, list_price#262] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#253 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#254 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#260, count#261] -Results [2]: [sum#262, count#263] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#268, count#269] +Results [2]: [sum#270, count#271] (235) Exchange -Input [2]: [sum#262, count#263] -Arguments: SinglePartition, true, [id=#264] +Input [2]: [sum#270, count#271] +Arguments: SinglePartition, true, [id=#272] (236) HashAggregate [codegen id : 8] -Input [2]: [sum#262, count#263] +Input [2]: [sum#270, count#271] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#253 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#254 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#253 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#254 as decimal(12,2)))), DecimalType(18,2), true))#265] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#253 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#254 as decimal(12,2)))), DecimalType(18,2), true))#265 AS average_sales#266] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#273] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#273 AS average_sales#274] -Subquery:2 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] +Subquery:2 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:3 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] +Subquery:3 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:4 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] +Subquery:4 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:5 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] +Subquery:5 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:6 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] +Subquery:6 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:7 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] +Subquery:7 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:8 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] +Subquery:8 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:9 Hosting operator id = 147 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] +Subquery:9 Hosting operator id = 147 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:10 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] +Subquery:10 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:11 Hosting operator id = 166 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] +Subquery:11 Hosting operator id = 166 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:12 Hosting operator id = 170 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] +Subquery:12 Hosting operator id = 170 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:13 Hosting operator id = 185 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] +Subquery:13 Hosting operator id = 185 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:14 Hosting operator id = 189 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] +Subquery:14 Hosting operator id = 189 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:15 Hosting operator id = 193 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] +Subquery:15 Hosting operator id = 193 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index fc86da1801926..e96f1d6fed14f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Union WholeStageCodegen (26) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #1 WholeStageCodegen (8) @@ -90,7 +90,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [d_date_sk] #19 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #6 WholeStageCodegen (25) @@ -202,10 +202,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (52) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #20 WholeStageCodegen (51) @@ -226,10 +226,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter ReusedExchange [d_date_sk] #16 WholeStageCodegen (78) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 WholeStageCodegen (77) @@ -263,24 +263,24 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Union WholeStageCodegen (106) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #6 WholeStageCodegen (132) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #20 WholeStageCodegen (158) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #21 WholeStageCodegen (244) @@ -297,24 +297,24 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Union WholeStageCodegen (189) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #6 WholeStageCodegen (215) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #20 WholeStageCodegen (241) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #21 WholeStageCodegen (327) @@ -331,24 +331,24 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Union WholeStageCodegen (272) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #6 WholeStageCodegen (298) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #20 WholeStageCodegen (324) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #21 WholeStageCodegen (410) @@ -365,23 +365,23 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Union WholeStageCodegen (355) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #6 WholeStageCodegen (381) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #20 WholeStageCodegen (407) - Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #21 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt index ab246a3449557..1b9e8f37e9418 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt @@ -11,30 +11,30 @@ TakeOrderedAndProject (50) : +- Exchange (27) : +- * Project (26) : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) : : :- * Filter (3) : : : +- * ColumnarToRow (2) : : : +- Scan parquet default.store_sales (1) - : : +- BroadcastExchange (8) - : : +- * Project (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (17) + : : +- * Project (16) + : : +- * Filter (15) + : : +- * BroadcastHashJoin LeftOuter BuildRight (14) + : : :- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.item (4) + : : +- BroadcastExchange (13) + : : +- * HashAggregate (12) + : : +- Exchange (11) + : : +- * HashAggregate (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet default.item (7) : +- BroadcastExchange (24) : +- * Project (23) : +- * Filter (22) - : +- * BroadcastHashJoin LeftOuter BuildRight (21) - : :- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.item (11) - : +- BroadcastExchange (20) - : +- * HashAggregate (19) - : +- Exchange (18) - : +- * HashAggregate (17) - : +- * Filter (16) - : +- * ColumnarToRow (15) - : +- Scan parquet default.item (14) + : +- * ColumnarToRow (21) + : +- Scan parquet default.date_dim (20) +- * Sort (42) +- Exchange (41) +- * Project (40) @@ -65,112 +65,112 @@ Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3] Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3] Condition : ((isnotnull(ss_customer_sk#3) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_item_sk#2)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_month_seq#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_month_seq#5] - -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_month_seq#5] -Condition : ((isnotnull(d_month_seq#5) AND (d_month_seq#5 = Subquery scalar-subquery#6, [id=#7])) AND isnotnull(d_date_sk#4)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_month_seq#5] - -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None - -(10) Project [codegen id : 5] -Output [2]: [ss_item_sk#2, ss_customer_sk#3] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, d_date_sk#4] - -(11) Scan parquet default.item -Output [3]: [i_item_sk#9, i_current_price#10, i_category#11] +(4) Scan parquet default.item +Output [3]: [i_item_sk#4, i_current_price#5, i_category#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#9, i_current_price#10, i_category#11] +(5) ColumnarToRow [codegen id : 3] +Input [3]: [i_item_sk#4, i_current_price#5, i_category#6] -(13) Filter [codegen id : 4] -Input [3]: [i_item_sk#9, i_current_price#10, i_category#11] -Condition : (isnotnull(i_current_price#10) AND isnotnull(i_item_sk#9)) +(6) Filter [codegen id : 3] +Input [3]: [i_item_sk#4, i_current_price#5, i_category#6] +Condition : (isnotnull(i_current_price#5) AND isnotnull(i_item_sk#4)) -(14) Scan parquet default.item -Output [2]: [i_current_price#10, i_category#11] +(7) Scan parquet default.item +Output [2]: [i_current_price#5, i_category#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 2] -Input [2]: [i_current_price#10, i_category#11] - -(16) Filter [codegen id : 2] -Input [2]: [i_current_price#10, i_category#11] -Condition : isnotnull(i_category#11) - -(17) HashAggregate [codegen id : 2] -Input [2]: [i_current_price#10, i_category#11] -Keys [1]: [i_category#11] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#10))] -Aggregate Attributes [2]: [sum#12, count#13] -Results [3]: [i_category#11, sum#14, count#15] - -(18) Exchange -Input [3]: [i_category#11, sum#14, count#15] -Arguments: hashpartitioning(i_category#11, 5), true, [id=#16] - -(19) HashAggregate [codegen id : 3] -Input [3]: [i_category#11, sum#14, count#15] -Keys [1]: [i_category#11] -Functions [1]: [avg(UnscaledValue(i_current_price#10))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#10))#17] -Results [2]: [cast((avg(UnscaledValue(i_current_price#10))#17 / 100.0) as decimal(11,6)) AS avg(i_current_price)#18, i_category#11 AS i_category#11#19] - -(20) BroadcastExchange -Input [2]: [avg(i_current_price)#18, i_category#11#19] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#20] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_category#11] -Right keys [1]: [i_category#11#19] +(8) ColumnarToRow [codegen id : 1] +Input [2]: [i_current_price#5, i_category#6] + +(9) Filter [codegen id : 1] +Input [2]: [i_current_price#5, i_category#6] +Condition : isnotnull(i_category#6) + +(10) HashAggregate [codegen id : 1] +Input [2]: [i_current_price#5, i_category#6] +Keys [1]: [i_category#6] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#5))] +Aggregate Attributes [2]: [sum#7, count#8] +Results [3]: [i_category#6, sum#9, count#10] + +(11) Exchange +Input [3]: [i_category#6, sum#9, count#10] +Arguments: hashpartitioning(i_category#6, 5), true, [id=#11] + +(12) HashAggregate [codegen id : 2] +Input [3]: [i_category#6, sum#9, count#10] +Keys [1]: [i_category#6] +Functions [1]: [avg(UnscaledValue(i_current_price#5))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#5))#12] +Results [2]: [cast((avg(UnscaledValue(i_current_price#5))#12 / 100.0) as decimal(11,6)) AS avg(i_current_price)#13, i_category#6 AS i_category#6#14] + +(13) BroadcastExchange +Input [2]: [avg(i_current_price)#13, i_category#6#14] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#15] + +(14) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [i_category#6] +Right keys [1]: [i_category#6#14] +Join condition: None + +(15) Filter [codegen id : 3] +Input [5]: [i_item_sk#4, i_current_price#5, i_category#6, avg(i_current_price)#13, i_category#6#14] +Condition : (cast(i_current_price#5 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#13)), DecimalType(14,7), true)) + +(16) Project [codegen id : 3] +Output [1]: [i_item_sk#4] +Input [5]: [i_item_sk#4, i_current_price#5, i_category#6, avg(i_current_price)#13, i_category#6#14] + +(17) BroadcastExchange +Input [1]: [i_item_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] + +(18) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#4] Join condition: None +(19) Project [codegen id : 5] +Output [2]: [ss_sold_date_sk#1, ss_customer_sk#3] +Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, i_item_sk#4] + +(20) Scan parquet default.date_dim +Output [2]: [d_date_sk#17, d_month_seq#18] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] +ReadSchema: struct + +(21) ColumnarToRow [codegen id : 4] +Input [2]: [d_date_sk#17, d_month_seq#18] + (22) Filter [codegen id : 4] -Input [5]: [i_item_sk#9, i_current_price#10, i_category#11, avg(i_current_price)#18, i_category#11#19] -Condition : (cast(i_current_price#10 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#18)), DecimalType(14,7), true)) +Input [2]: [d_date_sk#17, d_month_seq#18] +Condition : ((isnotnull(d_month_seq#18) AND (d_month_seq#18 = Subquery scalar-subquery#19, [id=#20])) AND isnotnull(d_date_sk#17)) (23) Project [codegen id : 4] -Output [1]: [i_item_sk#9] -Input [5]: [i_item_sk#9, i_current_price#10, i_category#11, avg(i_current_price)#18, i_category#11#19] +Output [1]: [d_date_sk#17] +Input [2]: [d_date_sk#17, d_month_seq#18] (24) BroadcastExchange -Input [1]: [i_item_sk#9] +Input [1]: [d_date_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#9] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#17] Join condition: None (26) Project [codegen id : 5] Output [1]: [ss_customer_sk#3] -Input [3]: [ss_item_sk#2, ss_customer_sk#3, i_item_sk#9] +Input [3]: [ss_sold_date_sk#1, ss_customer_sk#3, d_date_sk#17] (27) Exchange Input [1]: [ss_customer_sk#3] @@ -282,7 +282,7 @@ Arguments: 100, [cnt#35 ASC NULLS FIRST, ca_state#24 ASC NULLS FIRST], [state#34 ===== Subqueries ===== -Subquery:1 Hosting operator id = 6 Hosting Expression = Subquery scalar-subquery#6, [id=#7] +Subquery:1 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#19, [id=#20] * HashAggregate (57) +- Exchange (56) +- * HashAggregate (55) @@ -293,39 +293,39 @@ Subquery:1 Hosting operator id = 6 Hosting Expression = Subquery scalar-subquery (51) Scan parquet default.date_dim -Output [3]: [d_month_seq#5, d_year#37, d_moy#38] +Output [3]: [d_month_seq#18, d_year#37, d_moy#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (52) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#5, d_year#37, d_moy#38] +Input [3]: [d_month_seq#18, d_year#37, d_moy#38] (53) Filter [codegen id : 1] -Input [3]: [d_month_seq#5, d_year#37, d_moy#38] +Input [3]: [d_month_seq#18, d_year#37, d_moy#38] Condition : (((isnotnull(d_year#37) AND isnotnull(d_moy#38)) AND (d_year#37 = 2000)) AND (d_moy#38 = 1)) (54) Project [codegen id : 1] -Output [1]: [d_month_seq#5] -Input [3]: [d_month_seq#5, d_year#37, d_moy#38] +Output [1]: [d_month_seq#18] +Input [3]: [d_month_seq#18, d_year#37, d_moy#38] (55) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#5] -Keys [1]: [d_month_seq#5] +Input [1]: [d_month_seq#18] +Keys [1]: [d_month_seq#18] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#5] +Results [1]: [d_month_seq#18] (56) Exchange -Input [1]: [d_month_seq#5] -Arguments: hashpartitioning(d_month_seq#5, 5), true, [id=#39] +Input [1]: [d_month_seq#18] +Arguments: hashpartitioning(d_month_seq#18, 5), true, [id=#39] (57) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#5] -Keys [1]: [d_month_seq#5] +Input [1]: [d_month_seq#18] +Keys [1]: [d_month_seq#18] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#5] +Results [1]: [d_month_seq#18] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt index 2700741b82c04..3cbd44fc5a7d9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt @@ -16,55 +16,55 @@ TakeOrderedAndProject [cnt,ca_state,state] Exchange [ss_customer_sk] #2 WholeStageCodegen (5) Project [ss_customer_sk] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_customer_sk] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_customer_sk,ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - Subquery #1 - WholeStageCodegen (2) - HashAggregate [d_month_seq] + WholeStageCodegen (3) + Project [i_item_sk] + Filter [i_current_price,avg(i_current_price)] + BroadcastHashJoin [i_category,i_category] + Filter [i_current_price,i_item_sk] + ColumnarToRow InputAdapter - Exchange [d_month_seq] #4 - WholeStageCodegen (1) - HashAggregate [d_month_seq] - Project [d_month_seq] - Filter [d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_month_seq,d_year,d_moy] - ColumnarToRow + Scan parquet default.item [i_item_sk,i_current_price,i_category] InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] + BroadcastExchange #4 + WholeStageCodegen (2) + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] + InputAdapter + Exchange [i_category] #5 + WholeStageCodegen (1) + HashAggregate [i_category,i_current_price] [sum,count,sum,count] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_current_price,i_category] InputAdapter - BroadcastExchange #5 + BroadcastExchange #6 WholeStageCodegen (4) - Project [i_item_sk] - Filter [i_current_price,avg(i_current_price)] - BroadcastHashJoin [i_category,i_category] - Filter [i_current_price,i_item_sk] - ColumnarToRow + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + Subquery #1 + WholeStageCodegen (2) + HashAggregate [d_month_seq] InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_category] + Exchange [d_month_seq] #7 + WholeStageCodegen (1) + HashAggregate [d_month_seq] + Project [d_month_seq] + Filter [d_year,d_moy] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] - InputAdapter - Exchange [i_category] #7 - WholeStageCodegen (2) - HashAggregate [i_category,i_current_price] [sum,count,sum,count] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_current_price,i_category] + Scan parquet default.date_dim [d_date_sk,d_month_seq] InputAdapter WholeStageCodegen (12) Sort [c_customer_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt index c2627bd7e4cc9..a7f328537b7ac 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt @@ -24,15 +24,15 @@ TakeOrderedAndProject (79) : : : : : :- * Project (17) : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) : : : : : : :- * Project (10) - : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : : :- BroadcastExchange (5) - : : : : : : : : +- * Project (4) - : : : : : : : : +- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet default.household_demographics (1) - : : : : : : : +- * Filter (8) - : : : : : : : +- * ColumnarToRow (7) - : : : : : : : +- Scan parquet default.catalog_sales (6) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : : :- * Filter (3) + : : : : : : : : +- * ColumnarToRow (2) + : : : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : : : +- BroadcastExchange (8) + : : : : : : : +- * Project (7) + : : : : : : : +- * Filter (6) + : : : : : : : +- * ColumnarToRow (5) + : : : : : : : +- Scan parquet default.household_demographics (4) : : : : : : +- BroadcastExchange (15) : : : : : : +- * Project (14) : : : : : : +- * Filter (13) @@ -49,26 +49,26 @@ TakeOrderedAndProject (79) : : : : +- Scan parquet default.item (26) : : : +- BroadcastExchange (43) : : : +- * Project (42) - : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : :- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.date_dim (33) - : : : +- BroadcastExchange (40) - : : : +- * Project (39) - : : : +- * Filter (38) - : : : +- * ColumnarToRow (37) - : : : +- Scan parquet default.date_dim (36) + : : : +- * BroadcastHashJoin Inner BuildLeft (41) + : : : :- BroadcastExchange (37) + : : : : +- * Project (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.date_dim (33) + : : : +- * Filter (40) + : : : +- * ColumnarToRow (39) + : : : +- Scan parquet default.date_dim (38) : : +- * Sort (58) : : +- Exchange (57) : : +- * Project (56) - : : +- * BroadcastHashJoin Inner BuildLeft (55) - : : :- BroadcastExchange (51) - : : : +- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.warehouse (48) - : : +- * Filter (54) - : : +- * ColumnarToRow (53) - : : +- Scan parquet default.inventory (52) + : : +- * BroadcastHashJoin Inner BuildRight (55) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.inventory (48) + : : +- BroadcastExchange (54) + : : +- * Filter (53) + : : +- * ColumnarToRow (52) + : : +- Scan parquet default.warehouse (51) : +- BroadcastExchange (64) : +- * Filter (63) : +- * ColumnarToRow (62) @@ -80,50 +80,50 @@ TakeOrderedAndProject (79) +- Scan parquet default.catalog_returns (69) -(1) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#1, hd_buy_potential#2] +(1) Scan parquet default.catalog_sales +Output [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000), IsNotNull(hd_demo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_ship_date_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [2]: [hd_demo_sk#1, hd_buy_potential#2] +(2) ColumnarToRow [codegen id : 4] +Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] -(3) Filter [codegen id : 1] -Input [2]: [hd_demo_sk#1, hd_buy_potential#2] -Condition : ((isnotnull(hd_buy_potential#2) AND (hd_buy_potential#2 = 1001-5000)) AND isnotnull(hd_demo_sk#1)) +(3) Filter [codegen id : 4] +Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] +Condition : (((((isnotnull(cs_quantity#8) AND isnotnull(cs_item_sk#5)) AND isnotnull(cs_bill_cdemo_sk#3)) AND isnotnull(cs_bill_hdemo_sk#4)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_ship_date_sk#2)) -(4) Project [codegen id : 1] -Output [1]: [hd_demo_sk#1] -Input [2]: [hd_demo_sk#1, hd_buy_potential#2] +(4) Scan parquet default.household_demographics +Output [2]: [hd_demo_sk#9, hd_buy_potential#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000), IsNotNull(hd_demo_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [1]: [hd_demo_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#3] +(5) ColumnarToRow [codegen id : 1] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -(6) Scan parquet default.catalog_sales -Output [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_bill_hdemo_sk#7, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = 1001-5000)) AND isnotnull(hd_demo_sk#9)) -(7) ColumnarToRow -Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_bill_hdemo_sk#7, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] +(7) Project [codegen id : 1] +Output [1]: [hd_demo_sk#9] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -(8) Filter -Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_bill_hdemo_sk#7, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] -Condition : (((((isnotnull(cs_quantity#11) AND isnotnull(cs_item_sk#8)) AND isnotnull(cs_bill_cdemo_sk#6)) AND isnotnull(cs_bill_hdemo_sk#7)) AND isnotnull(cs_sold_date_sk#4)) AND isnotnull(cs_ship_date_sk#5)) +(8) BroadcastExchange +Input [1]: [hd_demo_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [hd_demo_sk#1] -Right keys [1]: [cs_bill_hdemo_sk#7] +Left keys [1]: [cs_bill_hdemo_sk#4] +Right keys [1]: [hd_demo_sk#9] Join condition: None (10) Project [codegen id : 4] -Output [7]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] -Input [9]: [hd_demo_sk#1, cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_bill_hdemo_sk#7, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] +Output [7]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] +Input [9]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, hd_demo_sk#9] (11) Scan parquet default.customer_demographics Output [2]: [cd_demo_sk#12, cd_marital_status#13] @@ -148,13 +148,13 @@ Input [1]: [cd_demo_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_bill_cdemo_sk#6] +Left keys [1]: [cs_bill_cdemo_sk#3] Right keys [1]: [cd_demo_sk#12] Join condition: None (17) Project [codegen id : 4] -Output [6]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] -Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, cd_demo_sk#12] +Output [6]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] +Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, cd_demo_sk#12] (18) Scan parquet default.date_dim Output [2]: [d_date_sk#15, d_date#16] @@ -175,21 +175,21 @@ Input [2]: [d_date_sk#15, d_date#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_ship_date_sk#5] +Left keys [1]: [cs_ship_date_sk#2] Right keys [1]: [d_date_sk#15] Join condition: None (23) Project [codegen id : 4] -Output [6]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16] -Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date_sk#15, d_date#16] +Output [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] +Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date_sk#15, d_date#16] (24) Exchange -Input [6]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16] -Arguments: hashpartitioning(cs_item_sk#8, 5), true, [id=#18] +Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] +Arguments: hashpartitioning(cs_item_sk#5, 5), true, [id=#18] (25) Sort [codegen id : 5] -Input [6]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16] -Arguments: [cs_item_sk#8 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] +Arguments: [cs_item_sk#5 ASC NULLS FIRST], false, 0 (26) Scan parquet default.item Output [2]: [i_item_sk#19, i_item_desc#20] @@ -214,137 +214,137 @@ Input [2]: [i_item_sk#19, i_item_desc#20] Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 (31) SortMergeJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#8] +Left keys [1]: [cs_item_sk#5] Right keys [1]: [i_item_sk#19] Join condition: None (32) Project [codegen id : 10] -Output [7]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16, i_item_desc#20] -Input [8]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16, i_item_sk#19, i_item_desc#20] +Output [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] +Input [8]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_sk#19, i_item_desc#20] (33) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_week_seq#23] +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] +ReadSchema: struct -(34) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#22, d_week_seq#23] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -(35) Filter [codegen id : 9] -Input [2]: [d_date_sk#22, d_week_seq#23] -Condition : (isnotnull(d_week_seq#23) AND isnotnull(d_date_sk#22)) +(35) Filter [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(36) Scan parquet default.date_dim -Output [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct +(36) Project [codegen id : 8] +Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -(37) ColumnarToRow [codegen id : 8] -Input [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#27] +(37) BroadcastExchange +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#26] -(38) Filter [codegen id : 8] -Input [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#27] -Condition : ((((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#24)) AND isnotnull(d_week_seq#26)) AND isnotnull(d_date#25)) +(38) Scan parquet default.date_dim +Output [2]: [d_date_sk#27, d_week_seq#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct -(39) Project [codegen id : 8] -Output [3]: [d_date_sk#24, d_date#25, d_week_seq#26] -Input [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#27] +(39) ColumnarToRow +Input [2]: [d_date_sk#27, d_week_seq#28] -(40) BroadcastExchange -Input [3]: [d_date_sk#24, d_date#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#28] +(40) Filter +Input [2]: [d_date_sk#27, d_week_seq#28] +Condition : (isnotnull(d_week_seq#28) AND isnotnull(d_date_sk#27)) (41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#23] -Right keys [1]: [d_week_seq#26] +Left keys [1]: [d_week_seq#24] +Right keys [1]: [d_week_seq#28] Join condition: None (42) Project [codegen id : 9] -Output [4]: [d_date_sk#22, d_date_sk#24, d_date#25, d_week_seq#26] -Input [5]: [d_date_sk#22, d_week_seq#23, d_date_sk#24, d_date#25, d_week_seq#26] +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +Input [5]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27, d_week_seq#28] (43) BroadcastExchange -Input [4]: [d_date_sk#22, d_date_sk#24, d_date#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#29] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#4] -Right keys [1]: [d_date_sk#24] -Join condition: (d_date#16 > d_date#25 + 5 days) +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#22] +Join condition: (d_date#16 > d_date#23 + 5 days) (45) Project [codegen id : 10] -Output [7]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, i_item_desc#20, d_date_sk#22, d_week_seq#26] -Input [11]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16, i_item_desc#20, d_date_sk#22, d_date_sk#24, d_date#25, d_week_seq#26] +Output [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Input [11]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] (46) Exchange -Input [7]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, i_item_desc#20, d_date_sk#22, d_week_seq#26] -Arguments: hashpartitioning(cs_item_sk#8, d_date_sk#22, 5), true, [id=#30] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: hashpartitioning(cs_item_sk#5, d_date_sk#27, 5), true, [id=#30] (47) Sort [codegen id : 11] -Input [7]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, i_item_desc#20, d_date_sk#22, d_week_seq#26] -Arguments: [cs_item_sk#8 ASC NULLS FIRST, d_date_sk#22 ASC NULLS FIRST], false, 0 +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: [cs_item_sk#5 ASC NULLS FIRST, d_date_sk#27 ASC NULLS FIRST], false, 0 -(48) Scan parquet default.warehouse -Output [2]: [w_warehouse_sk#31, w_warehouse_name#32] +(48) Scan parquet default.inventory +Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(49) ColumnarToRow [codegen id : 12] -Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] +Location [not included in comparison]/{warehouse_dir}/inventory] +PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] +ReadSchema: struct -(50) Filter [codegen id : 12] -Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] -Condition : isnotnull(w_warehouse_sk#31) +(49) ColumnarToRow [codegen id : 13] +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -(51) BroadcastExchange -Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +(50) Filter [codegen id : 13] +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] +Condition : (((isnotnull(inv_quantity_on_hand#34) AND isnotnull(inv_item_sk#32)) AND isnotnull(inv_warehouse_sk#33)) AND isnotnull(inv_date_sk#31)) -(52) Scan parquet default.inventory -Output [4]: [inv_date_sk#34, inv_item_sk#35, inv_warehouse_sk#36, inv_quantity_on_hand#37] +(51) Scan parquet default.warehouse +Output [2]: [w_warehouse_sk#35, w_warehouse_name#36] Batched: true -Location [not included in comparison]/{warehouse_dir}/inventory] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/warehouse] +PushedFilters: [IsNotNull(w_warehouse_sk)] +ReadSchema: struct + +(52) ColumnarToRow [codegen id : 12] +Input [2]: [w_warehouse_sk#35, w_warehouse_name#36] -(53) ColumnarToRow -Input [4]: [inv_date_sk#34, inv_item_sk#35, inv_warehouse_sk#36, inv_quantity_on_hand#37] +(53) Filter [codegen id : 12] +Input [2]: [w_warehouse_sk#35, w_warehouse_name#36] +Condition : isnotnull(w_warehouse_sk#35) -(54) Filter -Input [4]: [inv_date_sk#34, inv_item_sk#35, inv_warehouse_sk#36, inv_quantity_on_hand#37] -Condition : (((isnotnull(inv_quantity_on_hand#37) AND isnotnull(inv_item_sk#35)) AND isnotnull(inv_warehouse_sk#36)) AND isnotnull(inv_date_sk#34)) +(54) BroadcastExchange +Input [2]: [w_warehouse_sk#35, w_warehouse_name#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] (55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [w_warehouse_sk#31] -Right keys [1]: [inv_warehouse_sk#36] +Left keys [1]: [inv_warehouse_sk#33] +Right keys [1]: [w_warehouse_sk#35] Join condition: None (56) Project [codegen id : 13] -Output [4]: [w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_quantity_on_hand#37] -Input [6]: [w_warehouse_sk#31, w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_warehouse_sk#36, inv_quantity_on_hand#37] +Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Input [6]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] (57) Exchange -Input [4]: [w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_quantity_on_hand#37] -Arguments: hashpartitioning(inv_item_sk#35, inv_date_sk#34, 5), true, [id=#38] +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: hashpartitioning(inv_item_sk#32, inv_date_sk#31, 5), true, [id=#38] (58) Sort [codegen id : 14] -Input [4]: [w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_quantity_on_hand#37] -Arguments: [inv_item_sk#35 ASC NULLS FIRST, inv_date_sk#34 ASC NULLS FIRST], false, 0 +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: [inv_item_sk#32 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 16] -Left keys [2]: [cs_item_sk#8, d_date_sk#22] -Right keys [2]: [inv_item_sk#35, inv_date_sk#34] -Join condition: (inv_quantity_on_hand#37 < cs_quantity#11) +Left keys [2]: [cs_item_sk#5, d_date_sk#27] +Right keys [2]: [inv_item_sk#32, inv_date_sk#31] +Join condition: (inv_quantity_on_hand#34 < cs_quantity#8) (60) Project [codegen id : 16] -Output [6]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Input [11]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, i_item_desc#20, d_date_sk#22, d_week_seq#26, w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_quantity_on_hand#37] +Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [11]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27, inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] (61) Scan parquet default.promotion Output [1]: [p_promo_sk#39] @@ -365,21 +365,21 @@ Input [1]: [p_promo_sk#39] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] (65) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [cs_promo_sk#9] +Left keys [1]: [cs_promo_sk#6] Right keys [1]: [p_promo_sk#39] Join condition: None (66) Project [codegen id : 16] -Output [5]: [cs_item_sk#8, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Input [7]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26, p_promo_sk#39] +Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, p_promo_sk#39] (67) Exchange -Input [5]: [cs_item_sk#8, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Arguments: hashpartitioning(cs_item_sk#8, cs_order_number#10, 5), true, [id=#41] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), true, [id=#41] (68) Sort [codegen id : 17] -Input [5]: [cs_item_sk#8, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Arguments: [cs_item_sk#8 ASC NULLS FIRST, cs_order_number#10 ASC NULLS FIRST], false, 0 +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_order_number#7 ASC NULLS FIRST], false, 0 (69) Scan parquet default.catalog_returns Output [2]: [cr_item_sk#42, cr_order_number#43] @@ -404,33 +404,33 @@ Input [2]: [cr_item_sk#42, cr_order_number#43] Arguments: [cr_item_sk#42 ASC NULLS FIRST, cr_order_number#43 ASC NULLS FIRST], false, 0 (74) SortMergeJoin -Left keys [2]: [cs_item_sk#8, cs_order_number#10] +Left keys [2]: [cs_item_sk#5, cs_order_number#7] Right keys [2]: [cr_item_sk#42, cr_order_number#43] Join condition: None (75) Project [codegen id : 20] -Output [3]: [w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Input [7]: [cs_item_sk#8, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26, cr_item_sk#42, cr_order_number#43] +Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, cr_item_sk#42, cr_order_number#43] (76) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Keys [3]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26] +Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#45] -Results [4]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26, count#46] +Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] (77) Exchange -Input [4]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26, count#46] -Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#32, d_week_seq#26, 5), true, [id=#47] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#24, 5), true, [id=#47] (78) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26, count#46] -Keys [3]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#48] -Results [6]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] +Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] (79) TakeOrderedAndProject -Input [6]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26, no_promo#49, promo#50, total_cnt#51] -Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#32 ASC NULLS FIRST, d_week_seq#26 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#32, d_week_seq#26, no_promo#49, promo#50, total_cnt#51] +Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] +Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt index 39dba3af02359..918508787c4b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom InputAdapter Exchange [cs_item_sk,d_date_sk] #3 WholeStageCodegen (10) - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_date_sk,d_week_seq] + Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] SortMergeJoin [cs_item_sk,i_item_sk] @@ -38,7 +38,11 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [hd_demo_sk,cs_bill_hdemo_sk] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) @@ -47,10 +51,6 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) @@ -79,12 +79,8 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom InputAdapter BroadcastExchange #9 WholeStageCodegen (9) - Project [d_date_sk,d_date_sk,d_date,d_week_seq] + Project [d_date_sk,d_date,d_week_seq,d_date_sk] BroadcastHashJoin [d_week_seq,d_week_seq] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #10 WholeStageCodegen (8) @@ -93,14 +89,22 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter WholeStageCodegen (14) Sort [inv_item_sk,inv_date_sk] InputAdapter Exchange [inv_item_sk,inv_date_sk] #11 WholeStageCodegen (13) - Project [w_warehouse_name,inv_date_sk,inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [w_warehouse_sk,inv_warehouse_sk] + Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #12 WholeStageCodegen (12) @@ -108,10 +112,6 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] - Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #13 WholeStageCodegen (15) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt index e6210f4a26281..025e5a6f94741 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt @@ -44,12 +44,12 @@ TakeOrderedAndProject (125) : : : : : +- * Project (23) : : : : : +- * Filter (22) : : : : : +- * ColumnarToRow (21) - : : : : : +- Scan parquet default.date_dim (20) + : : : : : +- Scan parquet default.promotion (20) : : : : +- BroadcastExchange (31) : : : : +- * Project (30) : : : : +- * Filter (29) : : : : +- * ColumnarToRow (28) - : : : : +- Scan parquet default.promotion (27) + : : : : +- Scan parquet default.date_dim (27) : : : +- BroadcastExchange (37) : : : +- * Filter (36) : : : +- * ColumnarToRow (35) @@ -210,67 +210,67 @@ Join condition: None Output [7]: [ss_sold_date_sk#1, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12] Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, i_item_sk#14] -(20) Scan parquet default.date_dim -Output [2]: [d_date_sk#17, d_date#18] +(20) Scan parquet default.promotion +Output [2]: [p_promo_sk#17, p_channel_tv#18] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/promotion] +PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] +ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#17, d_date#18] +Input [2]: [p_promo_sk#17, p_channel_tv#18] (22) Filter [codegen id : 6] -Input [2]: [d_date_sk#17, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 10442)) AND (d_date#18 <= 10472)) AND isnotnull(d_date_sk#17)) +Input [2]: [p_promo_sk#17, p_channel_tv#18] +Condition : ((isnotnull(p_channel_tv#18) AND (p_channel_tv#18 = N)) AND isnotnull(p_promo_sk#17)) (23) Project [codegen id : 6] -Output [1]: [d_date_sk#17] -Input [2]: [d_date_sk#17, d_date#18] +Output [1]: [p_promo_sk#17] +Input [2]: [p_promo_sk#17, p_channel_tv#18] (24) BroadcastExchange -Input [1]: [d_date_sk#17] +Input [1]: [p_promo_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (25) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ss_promo_sk#4] +Right keys [1]: [p_promo_sk#17] Join condition: None (26) Project [codegen id : 9] -Output [6]: [ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12] -Input [8]: [ss_sold_date_sk#1, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#17] +Output [6]: [ss_sold_date_sk#1, ss_store_sk#3, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12] +Input [8]: [ss_sold_date_sk#1, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, p_promo_sk#17] -(27) Scan parquet default.promotion -Output [2]: [p_promo_sk#20, p_channel_tv#21] +(27) Scan parquet default.date_dim +Output [2]: [d_date_sk#20, d_date#21] Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] +ReadSchema: struct (28) ColumnarToRow [codegen id : 7] -Input [2]: [p_promo_sk#20, p_channel_tv#21] +Input [2]: [d_date_sk#20, d_date#21] (29) Filter [codegen id : 7] -Input [2]: [p_promo_sk#20, p_channel_tv#21] -Condition : ((isnotnull(p_channel_tv#21) AND (p_channel_tv#21 = N)) AND isnotnull(p_promo_sk#20)) +Input [2]: [d_date_sk#20, d_date#21] +Condition : (((isnotnull(d_date#21) AND (d_date#21 >= 10442)) AND (d_date#21 <= 10472)) AND isnotnull(d_date_sk#20)) (30) Project [codegen id : 7] -Output [1]: [p_promo_sk#20] -Input [2]: [p_promo_sk#20, p_channel_tv#21] +Output [1]: [d_date_sk#20] +Input [2]: [d_date_sk#20, d_date#21] (31) BroadcastExchange -Input [1]: [p_promo_sk#20] +Input [1]: [d_date_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_promo_sk#4] -Right keys [1]: [p_promo_sk#20] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#20] Join condition: None (33) Project [codegen id : 9] Output [5]: [ss_store_sk#3, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12] -Input [7]: [ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, p_promo_sk#20] +Input [7]: [ss_sold_date_sk#1, ss_store_sk#3, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#20] (34) Scan parquet default.store Output [2]: [s_store_sk#23, s_store_id#24] @@ -383,28 +383,28 @@ Output [7]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_s Input [9]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_item_sk#47, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, i_item_sk#14] (58) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#17] +Output [1]: [p_promo_sk#17] (59) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_sold_date_sk#45] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [cs_promo_sk#48] +Right keys [1]: [p_promo_sk#17] Join condition: None (60) Project [codegen id : 19] -Output [6]: [cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56] -Input [8]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, d_date_sk#17] +Output [6]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56] +Input [8]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, p_promo_sk#17] (61) ReusedExchange [Reuses operator id: 31] -Output [1]: [p_promo_sk#20] +Output [1]: [d_date_sk#20] (62) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_promo_sk#48] -Right keys [1]: [p_promo_sk#20] +Left keys [1]: [cs_sold_date_sk#45] +Right keys [1]: [d_date_sk#20] Join condition: None (63) Project [codegen id : 19] Output [5]: [cs_catalog_page_sk#46, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56] -Input [7]: [cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, p_promo_sk#20] +Input [7]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, d_date_sk#20] (64) Scan parquet default.catalog_page Output [2]: [cp_catalog_page_sk#58, cp_catalog_page_id#59] @@ -517,28 +517,28 @@ Output [7]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales Input [9]: [ws_sold_date_sk#80, ws_item_sk#81, ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, i_item_sk#14] (88) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#17] +Output [1]: [p_promo_sk#17] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#80] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ws_promo_sk#83] +Right keys [1]: [p_promo_sk#17] Join condition: None (90) Project [codegen id : 29] -Output [6]: [ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91] -Input [8]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, d_date_sk#17] +Output [6]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91] +Input [8]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, p_promo_sk#17] (91) ReusedExchange [Reuses operator id: 31] -Output [1]: [p_promo_sk#20] +Output [1]: [d_date_sk#20] (92) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#83] -Right keys [1]: [p_promo_sk#20] +Left keys [1]: [ws_sold_date_sk#80] +Right keys [1]: [d_date_sk#20] Join condition: None (93) Project [codegen id : 29] Output [5]: [ws_web_site_sk#82, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91] -Input [7]: [ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, p_promo_sk#20] +Input [7]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, d_date_sk#20] (94) Scan parquet default.web_site Output [2]: [web_site_sk#93, web_site_id#94] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/simplified.txt index 13781c8bd5993..ad59968740aaa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/simplified.txt @@ -32,9 +32,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] Project [ss_sold_date_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] @@ -69,19 +69,19 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) - Project [d_date_sk] - Filter [d_date,d_date_sk] + Project [p_promo_sk] + Filter [p_channel_tv,p_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Scan parquet default.promotion [p_promo_sk,p_channel_tv] InputAdapter BroadcastExchange #9 WholeStageCodegen (7) - Project [p_promo_sk] - Filter [p_channel_tv,p_promo_sk] + Project [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.promotion [p_promo_sk,p_channel_tv] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #10 WholeStageCodegen (8) @@ -98,9 +98,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] Project [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_catalog_page_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_sold_date_sk,cs_catalog_page_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] BroadcastHashJoin [cs_item_sk,i_item_sk] Project [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] @@ -127,9 +127,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter ReusedExchange [i_item_sk] #7 InputAdapter - ReusedExchange [d_date_sk] #8 + ReusedExchange [p_promo_sk] #8 InputAdapter - ReusedExchange [p_promo_sk] #9 + ReusedExchange [d_date_sk] #9 InputAdapter BroadcastExchange #14 WholeStageCodegen (18) @@ -146,9 +146,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] BroadcastHashJoin [ws_web_site_sk,web_site_sk] Project [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_web_site_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_promo_sk,p_promo_sk] Project [ws_sold_date_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_sold_date_sk,ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] @@ -175,9 +175,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter ReusedExchange [i_item_sk] #7 InputAdapter - ReusedExchange [d_date_sk] #8 + ReusedExchange [p_promo_sk] #8 InputAdapter - ReusedExchange [p_promo_sk] #9 + ReusedExchange [d_date_sk] #9 InputAdapter BroadcastExchange #18 WholeStageCodegen (28) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala index 2b4abed645910..4991e397eb11c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala @@ -150,7 +150,7 @@ class ApproximatePercentileQuerySuite extends QueryTest with SharedSparkSession (1 to 1000).toDF("col").createOrReplaceTempView(table) checkAnswer( spark.sql(s"SELECT percentile_approx(col, array(0.25 + 0.25D), 200 + 800) FROM $table"), - Row(Seq(499)) + Row(Seq(500)) ) } } @@ -296,4 +296,23 @@ class ApproximatePercentileQuerySuite extends QueryTest with SharedSparkSession buffer.quantileSummaries assert(buffer.isCompressed) } + + test("SPARK-32908: maximum target error in percentile_approx") { + withTempView(table) { + spark.read + .schema("col int") + .csv(testFile("test-data/percentile_approx-input.csv.bz2")) + .repartition(1) + .createOrReplaceTempView(table) + checkAnswer( + spark.sql( + s"""SELECT + | percentile_approx(col, 0.77, 1000), + | percentile_approx(col, 0.77, 10000), + | percentile_approx(col, 0.77, 100000), + | percentile_approx(col, 0.77, 1000000) + |FROM $table""".stripMargin), + Row(18, 17, 17, 17)) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 24419968c0472..937de92bcaba6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -24,6 +24,8 @@ import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat} import org.scalatest.matchers.should.Matchers._ +import org.apache.spark.SparkException +import org.apache.spark.sql.UpdateFieldsBenchmark._ import org.apache.spark.sql.catalyst.expressions.{InSet, Literal, NamedExpression} import org.apache.spark.sql.execution.ProjectExec import org.apache.spark.sql.functions._ @@ -922,11 +924,10 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { assert(inSet.sql === "('a' IN ('a', 'b'))") } - def checkAnswerAndSchema( + def checkAnswer( df: => DataFrame, expectedAnswer: Seq[Row], expectedSchema: StructType): Unit = { - checkAnswer(df, expectedAnswer) assert(df.schema == expectedSchema) } @@ -940,8 +941,8 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { sparkContext.parallelize(Row(Row(1, null, 3)) :: Nil), StructType(Seq(StructField("a", structType, nullable = false)))) - private lazy val nullStructLevel1: DataFrame = spark.createDataFrame( - sparkContext.parallelize(Row(null) :: Nil), + private lazy val nullableStructLevel1: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(null) :: Row(Row(1, null, 3)) :: Nil), StructType(Seq(StructField("a", structType, nullable = true)))) private lazy val structLevel2: DataFrame = spark.createDataFrame( @@ -951,12 +952,12 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { StructField("a", structType, nullable = false))), nullable = false)))) - private lazy val nullStructLevel2: DataFrame = spark.createDataFrame( - sparkContext.parallelize(Row(Row(null)) :: Nil), + private lazy val nullableStructLevel2: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(null) :: Row(Row(null)) :: Row(Row(Row(1, null, 3))) :: Nil), StructType(Seq( StructField("a", StructType(Seq( StructField("a", structType, nullable = true))), - nullable = false)))) + nullable = true)))) private lazy val structLevel3: DataFrame = spark.createDataFrame( sparkContext.parallelize(Row(Row(Row(Row(1, null, 3)))) :: Nil), @@ -1018,7 +1019,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } test("withField should add field with no name") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", $"a".withField("", lit(4))), Row(Row(1, null, 3, 4)) :: Nil, StructType(Seq( @@ -1031,7 +1032,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } test("withField should add field to struct") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("d", lit(4))), Row(Row(1, null, 3, 4)) :: Nil, StructType(Seq( @@ -1043,10 +1044,10 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { nullable = false)))) } - test("withField should add field to null struct") { - checkAnswerAndSchema( - nullStructLevel1.withColumn("a", $"a".withField("d", lit(4))), - Row(null) :: Nil, + test("withField should add field to nullable struct") { + checkAnswer( + nullableStructLevel1.withColumn("a", $"a".withField("d", lit(4))), + Row(null) :: Row(Row(1, null, 3, 4)) :: Nil, StructType(Seq( StructField("a", StructType(Seq( StructField("a", IntegerType, nullable = false), @@ -1056,10 +1057,10 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { nullable = true)))) } - test("withField should add field to nested null struct") { - checkAnswerAndSchema( - nullStructLevel2.withColumn("a", $"a".withField("a.d", lit(4))), - Row(Row(null)) :: Nil, + test("withField should add field to nested nullable struct") { + checkAnswer( + nullableStructLevel2.withColumn("a", $"a".withField("a.d", lit(4))), + Row(null) :: Row(Row(null)) :: Row(Row(Row(1, null, 3, 4))) :: Nil, StructType( Seq(StructField("a", StructType(Seq( StructField("a", StructType(Seq( @@ -1068,11 +1069,11 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { StructField("c", IntegerType, nullable = false), StructField("d", IntegerType, nullable = false))), nullable = true))), - nullable = false)))) + nullable = true)))) } test("withField should add null field to struct") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("d", lit(null).cast(IntegerType))), Row(Row(1, null, 3, null)) :: Nil, StructType(Seq( @@ -1085,7 +1086,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } test("withField should add multiple fields to struct") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("d", lit(4)).withField("e", lit(5))), Row(Row(1, null, 3, 4, 5)) :: Nil, StructType(Seq( @@ -1098,12 +1099,26 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { nullable = false)))) } + test("withField should add multiple fields to nullable struct") { + checkAnswer( + nullableStructLevel1.withColumn("a", 'a.withField("d", lit(4)).withField("e", lit(5))), + Row(null) :: Row(Row(1, null, 3, 4, 5)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = false), + StructField("e", IntegerType, nullable = false))), + nullable = true)))) + } + test("withField should add field to nested struct") { Seq( structLevel2.withColumn("a", 'a.withField("a.d", lit(4))), structLevel2.withColumn("a", 'a.withField("a", $"a.a".withField("d", lit(4)))) ).foreach { df => - checkAnswerAndSchema( + checkAnswer( df, Row(Row(Row(1, null, 3, 4))) :: Nil, StructType( @@ -1118,8 +1133,50 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } } + test("withField should add multiple fields to nested struct") { + Seq( + col("a").withField("a", $"a.a".withField("d", lit(4)).withField("e", lit(5))), + col("a").withField("a.d", lit(4)).withField("a.e", lit(5)) + ).foreach { column => + checkAnswer( + structLevel2.select(column.as("a")), + Row(Row(Row(1, null, 3, 4, 5))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = false), + StructField("e", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + } + } + + test("withField should add multiple fields to nested nullable struct") { + Seq( + col("a").withField("a", $"a.a".withField("d", lit(4)).withField("e", lit(5))), + col("a").withField("a.d", lit(4)).withField("a.e", lit(5)) + ).foreach { column => + checkAnswer( + nullableStructLevel2.select(column.as("a")), + Row(null) :: Row(Row(null)) :: Row(Row(Row(1, null, 3, 4, 5))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = false), + StructField("e", IntegerType, nullable = false))), + nullable = true))), + nullable = true)))) + } + } + test("withField should add field to deeply nested struct") { - checkAnswerAndSchema( + checkAnswer( structLevel3.withColumn("a", 'a.withField("a.a.d", lit(4))), Row(Row(Row(Row(1, null, 3, 4)))) :: Nil, StructType(Seq( @@ -1136,7 +1193,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } test("withField should replace field in struct") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("b", lit(2))), Row(Row(1, 2, 3)) :: Nil, StructType(Seq( @@ -1147,10 +1204,10 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { nullable = false)))) } - test("withField should replace field in null struct") { - checkAnswerAndSchema( - nullStructLevel1.withColumn("a", 'a.withField("b", lit("foo"))), - Row(null) :: Nil, + test("withField should replace field in nullable struct") { + checkAnswer( + nullableStructLevel1.withColumn("a", 'a.withField("b", lit("foo"))), + Row(null) :: Row(Row(1, "foo", 3)) :: Nil, StructType(Seq( StructField("a", StructType(Seq( StructField("a", IntegerType, nullable = false), @@ -1159,10 +1216,10 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { nullable = true)))) } - test("withField should replace field in nested null struct") { - checkAnswerAndSchema( - nullStructLevel2.withColumn("a", $"a".withField("a.b", lit("foo"))), - Row(Row(null)) :: Nil, + test("withField should replace field in nested nullable struct") { + checkAnswer( + nullableStructLevel2.withColumn("a", $"a".withField("a.b", lit("foo"))), + Row(null) :: Row(Row(null)) :: Row(Row(Row(1, "foo", 3))) :: Nil, StructType( Seq(StructField("a", StructType(Seq( StructField("a", StructType(Seq( @@ -1170,11 +1227,11 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { StructField("b", StringType, nullable = false), StructField("c", IntegerType, nullable = false))), nullable = true))), - nullable = false)))) + nullable = true)))) } test("withField should replace field with null value in struct") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("c", lit(null).cast(IntegerType))), Row(Row(1, null, null)) :: Nil, StructType(Seq( @@ -1186,7 +1243,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } test("withField should replace multiple fields in struct") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("a", lit(10)).withField("b", lit(20))), Row(Row(10, 20, 3)) :: Nil, StructType(Seq( @@ -1197,12 +1254,24 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { nullable = false)))) } + test("withField should replace multiple fields in nullable struct") { + checkAnswer( + nullableStructLevel1.withColumn("a", 'a.withField("a", lit(10)).withField("b", lit(20))), + Row(null) :: Row(Row(10, 20, 3)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = true)))) + } + test("withField should replace field in nested struct") { Seq( structLevel2.withColumn("a", $"a".withField("a.b", lit(2))), structLevel2.withColumn("a", 'a.withField("a", $"a.a".withField("b", lit(2)))) ).foreach { df => - checkAnswerAndSchema( + checkAnswer( df, Row(Row(Row(1, 2, 3))) :: Nil, StructType(Seq( @@ -1216,8 +1285,46 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } } + test("withField should replace multiple fields in nested struct") { + Seq( + col("a").withField("a", $"a.a".withField("a", lit(10)).withField("b", lit(20))), + col("a").withField("a.a", lit(10)).withField("a.b", lit(20)) + ).foreach { column => + checkAnswer( + structLevel2.select(column.as("a")), + Row(Row(Row(10, 20, 3))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + } + } + + test("withField should replace multiple fields in nested nullable struct") { + Seq( + col("a").withField("a", $"a.a".withField("a", lit(10)).withField("b", lit(20))), + col("a").withField("a.a", lit(10)).withField("a.b", lit(20)) + ).foreach { column => + checkAnswer( + nullableStructLevel2.select(column.as("a")), + Row(null) :: Row(Row(null)) :: Row(Row(Row(10, 20, 3))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = true))), + nullable = true)))) + } + } + test("withField should replace field in deeply nested struct") { - checkAnswerAndSchema( + checkAnswer( structLevel3.withColumn("a", $"a".withField("a.a.b", lit(2))), Row(Row(Row(Row(1, 2, 3)))) :: Nil, StructType(Seq( @@ -1242,7 +1349,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { StructField("b", IntegerType, nullable = false))), nullable = false)))) - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("b", lit(100))), Row(Row(1, 100, 100)) :: Nil, StructType(Seq( @@ -1254,7 +1361,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } test("withField should replace fields in struct in given order") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("b", lit(2)).withField("b", lit(20))), Row(Row(1, 20, 3)) :: Nil, StructType(Seq( @@ -1266,7 +1373,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } test("withField should add field and then replace same field in struct") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("d", lit(4)).withField("d", lit(5))), Row(Row(1, null, 3, 5)) :: Nil, StructType(Seq( @@ -1290,7 +1397,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { nullable = false))), nullable = false)))) - checkAnswerAndSchema( + checkAnswer( df.withColumn("a", 'a.withField("`a.b`.`e.f`", lit(2))), Row(Row(Row(1, 2, 3))) :: Nil, StructType(Seq( @@ -1317,7 +1424,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { test("withField should replace field in struct even if casing is different") { withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { - checkAnswerAndSchema( + checkAnswer( mixedCaseStructLevel1.withColumn("a", 'a.withField("A", lit(2))), Row(Row(2, 1)) :: Nil, StructType(Seq( @@ -1326,7 +1433,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { StructField("B", IntegerType, nullable = false))), nullable = false)))) - checkAnswerAndSchema( + checkAnswer( mixedCaseStructLevel1.withColumn("a", 'a.withField("b", lit(2))), Row(Row(1, 2)) :: Nil, StructType(Seq( @@ -1339,7 +1446,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { test("withField should add field to struct because casing is different") { withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - checkAnswerAndSchema( + checkAnswer( mixedCaseStructLevel1.withColumn("a", 'a.withField("A", lit(2))), Row(Row(1, 1, 2)) :: Nil, StructType(Seq( @@ -1349,7 +1456,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { StructField("A", IntegerType, nullable = false))), nullable = false)))) - checkAnswerAndSchema( + checkAnswer( mixedCaseStructLevel1.withColumn("a", 'a.withField("b", lit(2))), Row(Row(1, 1, 2)) :: Nil, StructType(Seq( @@ -1377,7 +1484,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { test("withField should replace nested field in struct even if casing is different") { withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { - checkAnswerAndSchema( + checkAnswer( mixedCaseStructLevel2.withColumn("a", 'a.withField("A.a", lit(2))), Row(Row(Row(2, 1), Row(1, 1))) :: Nil, StructType(Seq( @@ -1392,7 +1499,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { nullable = false))), nullable = false)))) - checkAnswerAndSchema( + checkAnswer( mixedCaseStructLevel2.withColumn("a", 'a.withField("b.a", lit(2))), Row(Row(Row(1, 1), Row(2, 1))) :: Nil, StructType(Seq( @@ -1451,30 +1558,41 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col") .select($"struct_col".withField("a.c", lit(3))) }.getMessage should include("Ambiguous reference to fields") + + checkAnswer( + sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") + .select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4))), + Row(Row(Row(1, 2, 3, 4)))) + + checkAnswer( + sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") + .select($"struct_col".withField("a", + $"struct_col.a".withField("c", lit(3)).withField("d", lit(4)))), + Row(Row(Row(1, 2, 3, 4)))) } test("SPARK-32641: extracting field from non-null struct column after withField should return " + "field value") { // extract newly added field - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", $"a".withField("d", lit(4)).getField("d")), Row(4) :: Nil, StructType(Seq(StructField("a", IntegerType, nullable = false)))) // extract newly replaced field - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", $"a".withField("a", lit(4)).getField("a")), Row(4) :: Nil, StructType(Seq(StructField("a", IntegerType, nullable = false)))) // add new field, extract another field from original struct - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", $"a".withField("d", lit(4)).getField("c")), Row(3):: Nil, StructType(Seq(StructField("a", IntegerType, nullable = false)))) // replace field, extract another field from original struct - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", $"a".withField("a", lit(4)).getField("c")), Row(3):: Nil, StructType(Seq(StructField("a", IntegerType, nullable = false)))) @@ -1482,26 +1600,30 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { test("SPARK-32641: extracting field from null struct column after withField should return " + "null if the original struct was null") { + val nullStructLevel1 = spark.createDataFrame( + sparkContext.parallelize(Row(null) :: Nil), + StructType(Seq(StructField("a", structType, nullable = true)))) + // extract newly added field - checkAnswerAndSchema( + checkAnswer( nullStructLevel1.withColumn("a", $"a".withField("d", lit(4)).getField("d")), Row(null) :: Nil, StructType(Seq(StructField("a", IntegerType, nullable = true)))) // extract newly replaced field - checkAnswerAndSchema( + checkAnswer( nullStructLevel1.withColumn("a", $"a".withField("a", lit(4)).getField("a")), Row(null):: Nil, StructType(Seq(StructField("a", IntegerType, nullable = true)))) // add new field, extract another field from original struct - checkAnswerAndSchema( + checkAnswer( nullStructLevel1.withColumn("a", $"a".withField("d", lit(4)).getField("c")), Row(null):: Nil, StructType(Seq(StructField("a", IntegerType, nullable = true)))) // replace field, extract another field from original struct - checkAnswerAndSchema( + checkAnswer( nullStructLevel1.withColumn("a", $"a".withField("a", lit(4)).getField("c")), Row(null):: Nil, StructType(Seq(StructField("a", IntegerType, nullable = true)))) @@ -1514,27 +1636,721 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { StructType(Seq(StructField("a", structType, nullable = true)))) // extract newly added field - checkAnswerAndSchema( + checkAnswer( df.withColumn("a", $"a".withField("d", lit(4)).getField("d")), Row(4) :: Row(null) :: Nil, StructType(Seq(StructField("a", IntegerType, nullable = true)))) // extract newly replaced field - checkAnswerAndSchema( + checkAnswer( df.withColumn("a", $"a".withField("a", lit(4)).getField("a")), Row(4) :: Row(null):: Nil, StructType(Seq(StructField("a", IntegerType, nullable = true)))) // add new field, extract another field from original struct - checkAnswerAndSchema( + checkAnswer( df.withColumn("a", $"a".withField("d", lit(4)).getField("c")), Row(3) :: Row(null):: Nil, StructType(Seq(StructField("a", IntegerType, nullable = true)))) // replace field, extract another field from original struct - checkAnswerAndSchema( + checkAnswer( df.withColumn("a", $"a".withField("a", lit(4)).getField("c")), Row(3) :: Row(null):: Nil, StructType(Seq(StructField("a", IntegerType, nullable = true)))) } + + + test("dropFields should throw an exception if called on a non-StructType column") { + intercept[AnalysisException] { + testData.withColumn("key", $"key".dropFields("a")) + }.getMessage should include("struct argument should be struct type, got: int") + } + + test("dropFields should throw an exception if fieldName argument is null") { + intercept[IllegalArgumentException] { + structLevel1.withColumn("a", $"a".dropFields(null)) + }.getMessage should include("fieldName cannot be null") + } + + test("dropFields should throw an exception if any intermediate structs don't exist") { + intercept[AnalysisException] { + structLevel2.withColumn("a", 'a.dropFields("x.b")) + }.getMessage should include("No such struct field x in a") + + intercept[AnalysisException] { + structLevel3.withColumn("a", 'a.dropFields("a.x.b")) + }.getMessage should include("No such struct field x in a") + } + + test("dropFields should throw an exception if intermediate field is not a struct") { + intercept[AnalysisException] { + structLevel1.withColumn("a", 'a.dropFields("b.a")) + }.getMessage should include("struct argument should be struct type, got: int") + } + + test("dropFields should throw an exception if intermediate field reference is ambiguous") { + intercept[AnalysisException] { + val structLevel2: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(Row(1, null, 3), 4)) :: Nil), + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", structType, nullable = false), + StructField("a", structType, nullable = false))), + nullable = false)))) + + structLevel2.withColumn("a", 'a.dropFields("a.b")) + }.getMessage should include("Ambiguous reference to fields") + } + + test("dropFields should drop field in struct") { + checkAnswer( + structLevel1.withColumn("a", 'a.dropFields("b")), + Row(Row(1, 3)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false)))) + } + + test("dropFields should drop field in nullable struct") { + checkAnswer( + nullableStructLevel1.withColumn("a", $"a".dropFields("b")), + Row(null) :: Row(Row(1, 3)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = true)))) + } + + test("dropFields should drop multiple fields in struct") { + Seq( + structLevel1.withColumn("a", $"a".dropFields("b", "c")), + structLevel1.withColumn("a", 'a.dropFields("b").dropFields("c")) + ).foreach { df => + checkAnswer( + df, + Row(Row(1)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false))), + nullable = false)))) + } + } + + test("dropFields should throw an exception if no fields will be left in struct") { + intercept[AnalysisException] { + structLevel1.withColumn("a", 'a.dropFields("a", "b", "c")) + }.getMessage should include("cannot drop all fields in struct") + } + + test("dropFields should drop field with no name in struct") { + val structType = StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("", IntegerType, nullable = false))) + + val structLevel1: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(1, 2)) :: Nil), + StructType(Seq(StructField("a", structType, nullable = false)))) + + checkAnswer( + structLevel1.withColumn("a", $"a".dropFields("")), + Row(Row(1)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false))), + nullable = false)))) + } + + test("dropFields should drop field in nested struct") { + checkAnswer( + structLevel2.withColumn("a", 'a.dropFields("a.b")), + Row(Row(Row(1, 3))) :: Nil, + StructType( + Seq(StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + } + + test("dropFields should drop multiple fields in nested struct") { + checkAnswer( + structLevel2.withColumn("a", 'a.dropFields("a.b", "a.c")), + Row(Row(Row(1))) :: Nil, + StructType( + Seq(StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + } + + test("dropFields should drop field in nested nullable struct") { + checkAnswer( + nullableStructLevel2.withColumn("a", $"a".dropFields("a.b")), + Row(null) :: Row(Row(null)) :: Row(Row(Row(1, 3))) :: Nil, + StructType( + Seq(StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = true))), + nullable = true)))) + } + + test("dropFields should drop multiple fields in nested nullable struct") { + checkAnswer( + nullableStructLevel2.withColumn("a", $"a".dropFields("a.b", "a.c")), + Row(null) :: Row(Row(null)) :: Row(Row(Row(1))) :: Nil, + StructType( + Seq(StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false))), + nullable = true))), + nullable = true)))) + } + + test("dropFields should drop field in deeply nested struct") { + checkAnswer( + structLevel3.withColumn("a", 'a.dropFields("a.a.b")), + Row(Row(Row(Row(1, 3)))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false))), + nullable = false))), + nullable = false)))) + } + + test("dropFields should drop all fields with given name in struct") { + val structLevel1 = spark.createDataFrame( + sparkContext.parallelize(Row(Row(1, 2, 3)) :: Nil), + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + structLevel1.withColumn("a", 'a.dropFields("b")), + Row(Row(1)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false))), + nullable = false)))) + } + + test("dropFields should drop field in struct even if casing is different") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + checkAnswer( + mixedCaseStructLevel1.withColumn("a", 'a.dropFields("A")), + Row(Row(1)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("B", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + mixedCaseStructLevel1.withColumn("a", 'a.dropFields("b")), + Row(Row(1)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false))), + nullable = false)))) + } + } + + test("dropFields should not drop field in struct because casing is different") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + checkAnswer( + mixedCaseStructLevel1.withColumn("a", 'a.dropFields("A")), + Row(Row(1, 1)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("B", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + mixedCaseStructLevel1.withColumn("a", 'a.dropFields("b")), + Row(Row(1, 1)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("B", IntegerType, nullable = false))), + nullable = false)))) + } + } + + test("dropFields should drop nested field in struct even if casing is different") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + checkAnswer( + mixedCaseStructLevel2.withColumn("a", 'a.dropFields("A.a")), + Row(Row(Row(1), Row(1, 1))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("A", StructType(Seq( + StructField("b", IntegerType, nullable = false))), + nullable = false), + StructField("B", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + + checkAnswer( + mixedCaseStructLevel2.withColumn("a", 'a.dropFields("b.a")), + Row(Row(Row(1, 1), Row(1))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false), + StructField("b", StructType(Seq( + StructField("b", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + } + } + + test("dropFields should throw an exception because casing is different") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + intercept[AnalysisException] { + mixedCaseStructLevel2.withColumn("a", 'a.dropFields("A.a")) + }.getMessage should include("No such struct field A in a, B") + + intercept[AnalysisException] { + mixedCaseStructLevel2.withColumn("a", 'a.dropFields("b.a")) + }.getMessage should include("No such struct field b in a, B") + } + } + + test("dropFields should drop only fields that exist") { + checkAnswer( + structLevel1.withColumn("a", 'a.dropFields("d")), + Row(Row(1, null, 3)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + structLevel1.withColumn("a", 'a.dropFields("b", "d")), + Row(Row(1, 3)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + structLevel2.withColumn("a", $"a".dropFields("a.b", "a.d")), + Row(Row(Row(1, 3))) :: Nil, + StructType( + Seq(StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + } + + test("dropFields should drop multiple fields at arbitrary levels of nesting in a single call") { + val df: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(Row(1, null, 3), 4)) :: Nil), + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", structType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + df.withColumn("a", $"a".dropFields("a.b", "b")), + Row(Row(Row(1, 3))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), nullable = false))), + nullable = false)))) + } + + test("dropFields user-facing examples") { + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + .select($"struct_col".dropFields("b")), + Row(Row(1))) + + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + .select($"struct_col".dropFields("c")), + Row(Row(1, 2))) + + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2, 'c', 3) struct_col") + .select($"struct_col".dropFields("b", "c")), + Row(Row(1))) + + intercept[AnalysisException] { + sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + .select($"struct_col".dropFields("a", "b")) + }.getMessage should include("cannot drop all fields in struct") + + checkAnswer( + sql("SELECT CAST(NULL AS struct) struct_col") + .select($"struct_col".dropFields("b")), + Row(null)) + + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col") + .select($"struct_col".dropFields("b")), + Row(Row(1))) + + checkAnswer( + sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") + .select($"struct_col".dropFields("a.b")), + Row(Row(Row(1)))) + + intercept[AnalysisException] { + sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col") + .select($"struct_col".dropFields("a.c")) + }.getMessage should include("Ambiguous reference to fields") + + checkAnswer( + sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2, 'c', 3)) struct_col") + .select($"struct_col".dropFields("a.b", "a.c")), + Row(Row(Row(1)))) + + checkAnswer( + sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2, 'c', 3)) struct_col") + .select($"struct_col".withField("a", $"struct_col.a".dropFields("b", "c"))), + Row(Row(Row(1)))) + } + + test("should correctly handle different dropField + withField + getField combinations") { + val structType = StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))) + + val structLevel1: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(1, 2)) :: Nil), + StructType(Seq(StructField("a", structType, nullable = false)))) + + val nullStructLevel1: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(null) :: Nil), + StructType(Seq(StructField("a", structType, nullable = true)))) + + val nullableStructLevel1: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(1, 2)) :: Row(null) :: Nil), + StructType(Seq(StructField("a", structType, nullable = true)))) + + def check( + fieldOps: Column => Column, + getFieldName: String, + expectedValue: Option[Int]): Unit = { + + def query(df: DataFrame): DataFrame = + df.select(fieldOps(col("a")).getField(getFieldName).as("res")) + + checkAnswer( + query(structLevel1), + Row(expectedValue.orNull) :: Nil, + StructType(Seq(StructField("res", IntegerType, nullable = expectedValue.isEmpty)))) + + checkAnswer( + query(nullStructLevel1), + Row(null) :: Nil, + StructType(Seq(StructField("res", IntegerType, nullable = true)))) + + checkAnswer( + query(nullableStructLevel1), + Row(expectedValue.orNull) :: Row(null) :: Nil, + StructType(Seq(StructField("res", IntegerType, nullable = true)))) + } + + // add attribute, extract an attribute from the original struct + check(_.withField("c", lit(3)), "a", Some(1)) + check(_.withField("c", lit(3)), "b", Some(2)) + + // add attribute, extract added attribute + check(_.withField("c", lit(3)), "c", Some(3)) + check(_.withField("c", col("a.a")), "c", Some(1)) + check(_.withField("c", col("a.b")), "c", Some(2)) + check(_.withField("c", lit(null).cast(IntegerType)), "c", None) + + // replace attribute, extract an attribute from the original struct + check(_.withField("b", lit(3)), "a", Some(1)) + check(_.withField("a", lit(3)), "b", Some(2)) + + // replace attribute, extract replaced attribute + check(_.withField("b", lit(3)), "b", Some(3)) + check(_.withField("b", lit(null).cast(IntegerType)), "b", None) + check(_.withField("a", lit(3)), "a", Some(3)) + check(_.withField("a", lit(null).cast(IntegerType)), "a", None) + + // drop attribute, extract an attribute from the original struct + check(_.dropFields("b"), "a", Some(1)) + check(_.dropFields("a"), "b", Some(2)) + + // drop attribute, add attribute, extract an attribute from the original struct + check(_.dropFields("b").withField("c", lit(3)), "a", Some(1)) + check(_.dropFields("a").withField("c", lit(3)), "b", Some(2)) + + // drop attribute, add another attribute, extract added attribute + check(_.dropFields("a").withField("c", lit(3)), "c", Some(3)) + check(_.dropFields("b").withField("c", lit(3)), "c", Some(3)) + + // add attribute, drop attribute, extract an attribute from the original struct + check(_.withField("c", lit(3)).dropFields("a"), "b", Some(2)) + check(_.withField("c", lit(3)).dropFields("b"), "a", Some(1)) + + // add attribute, drop another attribute, extract added attribute + check(_.withField("c", lit(3)).dropFields("a"), "c", Some(3)) + check(_.withField("c", lit(3)).dropFields("b"), "c", Some(3)) + + // replace attribute, drop same attribute, extract an attribute from the original struct + check(_.withField("b", lit(3)).dropFields("b"), "a", Some(1)) + check(_.withField("a", lit(3)).dropFields("a"), "b", Some(2)) + + // add attribute, drop same attribute, extract an attribute from the original struct + check(_.withField("c", lit(3)).dropFields("c"), "a", Some(1)) + check(_.withField("c", lit(3)).dropFields("c"), "b", Some(2)) + + // add attribute, drop another attribute, extract added attribute + check(_.withField("b", lit(3)).dropFields("a"), "b", Some(3)) + check(_.withField("a", lit(3)).dropFields("b"), "a", Some(3)) + check(_.withField("b", lit(null).cast(IntegerType)).dropFields("a"), "b", None) + check(_.withField("a", lit(null).cast(IntegerType)).dropFields("b"), "a", None) + + // drop attribute, add same attribute, extract added attribute + check(_.dropFields("b").withField("b", lit(3)), "b", Some(3)) + check(_.dropFields("a").withField("a", lit(3)), "a", Some(3)) + check(_.dropFields("b").withField("b", lit(null).cast(IntegerType)), "b", None) + check(_.dropFields("a").withField("a", lit(null).cast(IntegerType)), "a", None) + check(_.dropFields("c").withField("c", lit(3)), "c", Some(3)) + + // add attribute, drop same attribute, add same attribute again, extract added attribute + check(_.withField("c", lit(3)).dropFields("c").withField("c", lit(4)), "c", Some(4)) + } + + test("should move field up one level of nesting") { + // move a field up one level + checkAnswer( + nullableStructLevel2.select( + col("a").withField("c", col("a.a.c")).dropFields("a.c").as("res")), + Row(null) :: Row(Row(null, null)) :: Row(Row(Row(1, null), 3)) :: Nil, + StructType(Seq( + StructField("res", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true))), + nullable = true), + StructField("c", IntegerType, nullable = true))), + nullable = true)))) + + // move a field up one level and then extract it + checkAnswer( + nullableStructLevel2.select( + col("a").withField("c", col("a.a.c")).dropFields("a.c").getField("c").as("res")), + Row(null) :: Row(null) :: Row(3) :: Nil, + StructType(Seq(StructField("res", IntegerType, nullable = true)))) + } + + test("should be able to refer to newly added nested column") { + intercept[AnalysisException] { + structLevel1.select($"a".withField("d", lit(4)).withField("e", $"a.d" + 1).as("a")) + }.getMessage should include("No such struct field d in a, b, c") + + checkAnswer( + structLevel1 + .select($"a".withField("d", lit(4)).as("a")) + .select($"a".withField("e", $"a.d" + 1).as("a")), + Row(Row(1, null, 3, 4, 5)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = false), + StructField("e", IntegerType, nullable = false))), + nullable = false)))) + } + + test("should be able to drop newly added nested column") { + Seq( + structLevel1.select($"a".withField("d", lit(4)).dropFields("d").as("a")), + structLevel1 + .select($"a".withField("d", lit(4)).as("a")) + .select($"a".dropFields("d").as("a")) + ).foreach { query => + checkAnswer( + query, + Row(Row(1, null, 3)) :: Nil, + StructType(Seq( + StructField("a", structType, nullable = false)))) + } + } + + test("should still be able to refer to dropped column within the same select statement") { + // we can still access the nested column even after dropping it within the same select statement + checkAnswer( + structLevel1.select($"a".dropFields("c").withField("z", $"a.c").as("a")), + Row(Row(1, null, 3)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("z", IntegerType, nullable = false))), + nullable = false)))) + + // we can't access the nested column in subsequent select statement after dropping it in a + // previous select statement + intercept[AnalysisException]{ + structLevel1 + .select($"a".dropFields("c").as("a")) + .select($"a".withField("z", $"a.c")).as("a") + }.getMessage should include("No such struct field c in a, b;") + } + + test("nestedDf should generate nested DataFrames") { + checkAnswer( + emptyNestedDf(1, 1, nullable = false), + Seq.empty[Row], + StructType(Seq(StructField("nested0Col0", StructType(Seq( + StructField("nested1Col0", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + emptyNestedDf(1, 2, nullable = false), + Seq.empty[Row], + StructType(Seq(StructField("nested0Col0", StructType(Seq( + StructField("nested1Col0", IntegerType, nullable = false), + StructField("nested1Col1", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + emptyNestedDf(2, 1, nullable = false), + Seq.empty[Row], + StructType(Seq(StructField("nested0Col0", StructType(Seq( + StructField("nested1Col0", StructType(Seq( + StructField("nested2Col0", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + + checkAnswer( + emptyNestedDf(2, 2, nullable = false), + Seq.empty[Row], + StructType(Seq(StructField("nested0Col0", StructType(Seq( + StructField("nested1Col0", StructType(Seq( + StructField("nested2Col0", IntegerType, nullable = false), + StructField("nested2Col1", IntegerType, nullable = false))), + nullable = false), + StructField("nested1Col1", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + emptyNestedDf(2, 2, nullable = true), + Seq.empty[Row], + StructType(Seq(StructField("nested0Col0", StructType(Seq( + StructField("nested1Col0", StructType(Seq( + StructField("nested2Col0", IntegerType, nullable = false), + StructField("nested2Col1", IntegerType, nullable = false))), + nullable = true), + StructField("nested1Col1", IntegerType, nullable = false))), + nullable = true)))) + } + + Seq(Performant, NonPerformant).foreach { method => + Seq(false, true).foreach { nullable => + test(s"should add and drop 1 column at each depth of nesting using ${method.name} method, " + + s"nullable = $nullable") { + val maxDepth = 3 + + // dataframe with nested*Col0 to nested*Col2 at each depth + val inputDf = emptyNestedDf(maxDepth, 3, nullable) + + // add nested*Col3 and drop nested*Col2 + val modifiedColumn = method( + column = col(nestedColName(0, 0)), + numsToAdd = Seq(3), + numsToDrop = Seq(2), + maxDepth = maxDepth + ).as(nestedColName(0, 0)) + val resultDf = inputDf.select(modifiedColumn) + + // dataframe with nested*Col0, nested*Col1, nested*Col3 at each depth + val expectedDf = { + val colNums = Seq(0, 1, 3) + val nestedColumnDataType = nestedStructType(colNums, nullable, maxDepth) + + spark.createDataFrame( + spark.sparkContext.emptyRDD[Row], + StructType(Seq(StructField(nestedColName(0, 0), nestedColumnDataType, nullable)))) + } + + checkAnswer(resultDf, expectedDf.collect(), expectedDf.schema) + } + } + } + + test("assert_true") { + // assert_true(condition, errMsgCol) + val booleanDf = Seq((true), (false)).toDF("cond") + checkAnswer( + booleanDf.filter("cond = true").select(assert_true($"cond")), + Row(null) :: Nil + ) + val e1 = intercept[SparkException] { + booleanDf.select(assert_true($"cond", lit(null.asInstanceOf[String]))).collect() + } + assert(e1.getCause.isInstanceOf[RuntimeException]) + assert(e1.getCause.getMessage == null) + + val nullDf = Seq(("first row", None), ("second row", Some(true))).toDF("n", "cond") + checkAnswer( + nullDf.filter("cond = true").select(assert_true($"cond", $"cond")), + Row(null) :: Nil + ) + val e2 = intercept[SparkException] { + nullDf.select(assert_true($"cond", $"n")).collect() + } + assert(e2.getCause.isInstanceOf[RuntimeException]) + assert(e2.getCause.getMessage == "first row") + + // assert_true(condition) + val intDf = Seq((0, 1)).toDF("a", "b") + checkAnswer(intDf.select(assert_true($"a" < $"b")), Row(null) :: Nil) + val e3 = intercept[SparkException] { + intDf.select(assert_true($"a" > $"b")).collect() + } + assert(e3.getCause.isInstanceOf[RuntimeException]) + assert(e3.getCause.getMessage == "'('a > 'b)' is not true!") + } + + test("raise_error") { + val strDf = Seq(("hello")).toDF("a") + + val e1 = intercept[SparkException] { + strDf.select(raise_error(lit(null.asInstanceOf[String]))).collect() + } + assert(e1.getCause.isInstanceOf[RuntimeException]) + assert(e1.getCause.getMessage == null) + + val e2 = intercept[SparkException] { + strDf.select(raise_error($"a")).collect() + } + assert(e2.getCause.isInstanceOf[RuntimeException]) + assert(e2.getCause.getMessage == "hello") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index e954e2bf1c46d..d4e64aa03df0e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -456,25 +456,51 @@ class DataFrameAggregateSuite extends QueryTest } test("zero moments") { - val input = Seq((1, 2)).toDF("a", "b") - checkAnswer( - input.agg(stddev($"a"), stddev_samp($"a"), stddev_pop($"a"), variance($"a"), - var_samp($"a"), var_pop($"a"), skewness($"a"), kurtosis($"a")), - Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, - Double.NaN, Double.NaN)) + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "true") { + val input = Seq((1, 2)).toDF("a", "b") + checkAnswer( + input.agg(stddev($"a"), stddev_samp($"a"), stddev_pop($"a"), variance($"a"), + var_samp($"a"), var_pop($"a"), skewness($"a"), kurtosis($"a")), + Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, + Double.NaN, Double.NaN)) - checkAnswer( - input.agg( - expr("stddev(a)"), - expr("stddev_samp(a)"), - expr("stddev_pop(a)"), - expr("variance(a)"), - expr("var_samp(a)"), - expr("var_pop(a)"), - expr("skewness(a)"), - expr("kurtosis(a)")), - Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, - Double.NaN, Double.NaN)) + checkAnswer( + input.agg( + expr("stddev(a)"), + expr("stddev_samp(a)"), + expr("stddev_pop(a)"), + expr("variance(a)"), + expr("var_samp(a)"), + expr("var_pop(a)"), + expr("skewness(a)"), + expr("kurtosis(a)")), + Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, + Double.NaN, Double.NaN)) + } + } + + test("SPARK-13860: zero moments LEGACY_STATISTICAL_AGGREGATE off") { + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "false") { + val input = Seq((1, 2)).toDF("a", "b") + checkAnswer( + input.agg(stddev($"a"), stddev_samp($"a"), stddev_pop($"a"), variance($"a"), + var_samp($"a"), var_pop($"a"), skewness($"a"), kurtosis($"a")), + Row(null, null, 0.0, null, null, 0.0, + null, null)) + + checkAnswer( + input.agg( + expr("stddev(a)"), + expr("stddev_samp(a)"), + expr("stddev_pop(a)"), + expr("variance(a)"), + expr("var_samp(a)"), + expr("var_pop(a)"), + expr("skewness(a)"), + expr("kurtosis(a)")), + Row(null, null, 0.0, null, null, 0.0, + null, null)) + } } test("null moments") { @@ -1043,6 +1069,14 @@ class DataFrameAggregateSuite extends QueryTest checkAnswer(sql(queryTemplate("FIRST")), Row(1)) checkAnswer(sql(queryTemplate("LAST")), Row(3)) } + + test("SPARK-32906: struct field names should not change after normalizing floats") { + val df = Seq(Tuple1(Tuple2(-0.0d, Double.NaN)), Tuple1(Tuple2(0.0d, Double.NaN))).toDF("k") + val aggs = df.distinct().queryExecution.sparkPlan.collect { case a: HashAggregateExec => a } + assert(aggs.length == 2) + assert(aggs.head.output.map(_.dataType.simpleString).head === + aggs.last.output.map(_.dataType.simpleString).head) + } } case class B(c: Option[Double]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index b463a76a74026..14d03a30453ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -348,7 +348,7 @@ class DataFrameJoinSuite extends QueryTest } assert(broadcastExchanges.size == 1) val tables = broadcastExchanges.head.collect { - case FileSourceScanExec(_, _, _, _, _, _, _, Some(tableIdent)) => tableIdent + case FileSourceScanExec(_, _, _, _, _, _, _, Some(tableIdent), _) => tableIdent } assert(tables.size == 1) assert(tables.head === TableIdentifier(table1Name, Some(dbName))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index d95f09a4cc839..321f4966178d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -2555,6 +2555,18 @@ class DataFrameSuite extends QueryTest val df = Seq(0.0 -> -0.0).toDF("pos", "neg") checkAnswer(df.select($"pos" > $"neg"), Row(false)) } + + test("SPARK-32635: Replace references with foldables coming only from the node's children") { + val a = Seq("1").toDF("col1").withColumn("col2", lit("1")) + val b = Seq("2").toDF("col1").withColumn("col2", lit("2")) + val aub = a.union(b) + val c = aub.filter($"col1" === "2").cache() + val d = Seq("2").toDF("col4") + val r = d.join(aub, $"col2" === $"col4").select("col4") + val l = c.select("col2") + val df = l.join(r, $"col2" === $"col4", "LeftOuter") + checkAnswer(df, Row("2", "2")) + } } case class GroupByKey(a: Int, b: Int) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index bc6adfb857b02..616e333033aa9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -94,89 +94,187 @@ class DataFrameWindowFunctionsSuite extends QueryTest } test("corr, covar_pop, stddev_pop functions in specific window") { - val df = Seq( - ("a", "p1", 10.0, 20.0), - ("b", "p1", 20.0, 10.0), - ("c", "p2", 20.0, 20.0), - ("d", "p2", 20.0, 20.0), - ("e", "p3", 0.0, 0.0), - ("f", "p3", 6.0, 12.0), - ("g", "p3", 6.0, 12.0), - ("h", "p3", 8.0, 16.0), - ("i", "p4", 5.0, 5.0)).toDF("key", "partitionId", "value1", "value2") - checkAnswer( - df.select( - $"key", - corr("value1", "value2").over(Window.partitionBy("partitionId") - .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - covar_pop("value1", "value2") - .over(Window.partitionBy("partitionId") + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "true") { + val df = Seq( + ("a", "p1", 10.0, 20.0), + ("b", "p1", 20.0, 10.0), + ("c", "p2", 20.0, 20.0), + ("d", "p2", 20.0, 20.0), + ("e", "p3", 0.0, 0.0), + ("f", "p3", 6.0, 12.0), + ("g", "p3", 6.0, 12.0), + ("h", "p3", 8.0, 16.0), + ("i", "p4", 5.0, 5.0)).toDF("key", "partitionId", "value1", "value2") + checkAnswer( + df.select( + $"key", + corr("value1", "value2").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + covar_pop("value1", "value2") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_pop("value1") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_pop("value1") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_pop("value2") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_pop("value2") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing))), + + // As stddev_pop(expr) = sqrt(var_pop(expr)) + // the "stddev_pop" column can be calculated from the "var_pop" column. + // + // As corr(expr1, expr2) = covar_pop(expr1, expr2) / (stddev_pop(expr1) * stddev_pop(expr2)) + // the "corr" column can be calculated from the "covar_pop" and the two "stddev_pop" columns + Seq( + Row("a", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), + Row("b", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), + Row("c", null, 0.0, 0.0, 0.0, 0.0, 0.0), + Row("d", null, 0.0, 0.0, 0.0, 0.0, 0.0), + Row("e", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("f", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("g", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("h", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("i", Double.NaN, 0.0, 0.0, 0.0, 0.0, 0.0))) + } + } + + test("SPARK-13860: " + + "corr, covar_pop, stddev_pop functions in specific window " + + "LEGACY_STATISTICAL_AGGREGATE off") { + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "false") { + val df = Seq( + ("a", "p1", 10.0, 20.0), + ("b", "p1", 20.0, 10.0), + ("c", "p2", 20.0, 20.0), + ("d", "p2", 20.0, 20.0), + ("e", "p3", 0.0, 0.0), + ("f", "p3", 6.0, 12.0), + ("g", "p3", 6.0, 12.0), + ("h", "p3", 8.0, 16.0), + ("i", "p4", 5.0, 5.0)).toDF("key", "partitionId", "value1", "value2") + checkAnswer( + df.select( + $"key", + corr("value1", "value2").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + covar_pop("value1", "value2") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_pop("value1") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_pop("value1") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_pop("value2") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_pop("value2") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing))), + + // As stddev_pop(expr) = sqrt(var_pop(expr)) + // the "stddev_pop" column can be calculated from the "var_pop" column. + // + // As corr(expr1, expr2) = covar_pop(expr1, expr2) / (stddev_pop(expr1) * stddev_pop(expr2)) + // the "corr" column can be calculated from the "covar_pop" and the two "stddev_pop" columns + Seq( + Row("a", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), + Row("b", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), + Row("c", null, 0.0, 0.0, 0.0, 0.0, 0.0), + Row("d", null, 0.0, 0.0, 0.0, 0.0, 0.0), + Row("e", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("f", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("g", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("h", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("i", null, 0.0, 0.0, 0.0, 0.0, 0.0))) + } + } + + test("covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window") { + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "true") { + val df = Seq( + ("a", "p1", 10.0, 20.0), + ("b", "p1", 20.0, 10.0), + ("c", "p2", 20.0, 20.0), + ("d", "p2", 20.0, 20.0), + ("e", "p3", 0.0, 0.0), + ("f", "p3", 6.0, 12.0), + ("g", "p3", 6.0, 12.0), + ("h", "p3", 8.0, 16.0), + ("i", "p4", 5.0, 5.0)).toDF("key", "partitionId", "value1", "value2") + checkAnswer( + df.select( + $"key", + covar_samp("value1", "value2").over(Window.partitionBy("partitionId") .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - var_pop("value1") - .over(Window.partitionBy("partitionId") + var_samp("value1").over(Window.partitionBy("partitionId") .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - stddev_pop("value1") - .over(Window.partitionBy("partitionId") + variance("value1").over(Window.partitionBy("partitionId") .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - var_pop("value2") - .over(Window.partitionBy("partitionId") + stddev_samp("value1").over(Window.partitionBy("partitionId") .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - stddev_pop("value2") - .over(Window.partitionBy("partitionId") - .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing))), - - // As stddev_pop(expr) = sqrt(var_pop(expr)) - // the "stddev_pop" column can be calculated from the "var_pop" column. - // - // As corr(expr1, expr2) = covar_pop(expr1, expr2) / (stddev_pop(expr1) * stddev_pop(expr2)) - // the "corr" column can be calculated from the "covar_pop" and the two "stddev_pop" columns. - Seq( - Row("a", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), - Row("b", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), - Row("c", null, 0.0, 0.0, 0.0, 0.0, 0.0), - Row("d", null, 0.0, 0.0, 0.0, 0.0, 0.0), - Row("e", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), - Row("f", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), - Row("g", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), - Row("h", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), - Row("i", Double.NaN, 0.0, 0.0, 0.0, 0.0, 0.0))) + stddev("value1").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)) + ), + Seq( + Row("a", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), + Row("b", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), + Row("c", 0.0, 0.0, 0.0, 0.0, 0.0), + Row("d", 0.0, 0.0, 0.0, 0.0, 0.0), + Row("e", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("f", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("g", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("h", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("i", Double.NaN, Double.NaN, Double.NaN, Double.NaN, Double.NaN))) + } } - test("covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window") { - val df = Seq( - ("a", "p1", 10.0, 20.0), - ("b", "p1", 20.0, 10.0), - ("c", "p2", 20.0, 20.0), - ("d", "p2", 20.0, 20.0), - ("e", "p3", 0.0, 0.0), - ("f", "p3", 6.0, 12.0), - ("g", "p3", 6.0, 12.0), - ("h", "p3", 8.0, 16.0), - ("i", "p4", 5.0, 5.0)).toDF("key", "partitionId", "value1", "value2") - checkAnswer( - df.select( - $"key", - covar_samp("value1", "value2").over(Window.partitionBy("partitionId") - .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - var_samp("value1").over(Window.partitionBy("partitionId") - .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - variance("value1").over(Window.partitionBy("partitionId") - .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - stddev_samp("value1").over(Window.partitionBy("partitionId") - .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - stddev("value1").over(Window.partitionBy("partitionId") - .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)) - ), - Seq( - Row("a", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), - Row("b", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), - Row("c", 0.0, 0.0, 0.0, 0.0, 0.0), - Row("d", 0.0, 0.0, 0.0, 0.0, 0.0), - Row("e", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), - Row("f", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), - Row("g", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), - Row("h", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), - Row("i", Double.NaN, Double.NaN, Double.NaN, Double.NaN, Double.NaN))) + test("SPARK-13860: " + + "covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window " + + "LEGACY_STATISTICAL_AGGREGATE off") { + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "false") { + val df = Seq( + ("a", "p1", 10.0, 20.0), + ("b", "p1", 20.0, 10.0), + ("c", "p2", 20.0, 20.0), + ("d", "p2", 20.0, 20.0), + ("e", "p3", 0.0, 0.0), + ("f", "p3", 6.0, 12.0), + ("g", "p3", 6.0, 12.0), + ("h", "p3", 8.0, 16.0), + ("i", "p4", 5.0, 5.0)).toDF("key", "partitionId", "value1", "value2") + checkAnswer( + df.select( + $"key", + covar_samp("value1", "value2").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_samp("value1").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + variance("value1").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_samp("value1").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev("value1").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)) + ), + Seq( + Row("a", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), + Row("b", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), + Row("c", 0.0, 0.0, 0.0, 0.0, 0.0), + Row("d", 0.0, 0.0, 0.0, 0.0, 0.0), + Row("e", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("f", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("g", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("h", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("i", null, null, null, null, null))) + } } test("collect_list in ascending ordered window") { @@ -541,6 +639,66 @@ class DataFrameWindowFunctionsSuite extends QueryTest Row("b", 3, null, null, null, null, null, null))) } + test("nth_value with ignoreNulls") { + val nullStr: String = null + val df = Seq( + ("a", 0, nullStr), + ("a", 1, "x"), + ("a", 2, "y"), + ("a", 3, "z"), + ("a", 4, nullStr), + ("b", 1, nullStr), + ("b", 2, nullStr)). + toDF("key", "order", "value") + val window = Window.partitionBy($"key").orderBy($"order") + checkAnswer( + df.select( + $"key", + $"order", + nth_value($"value", 2).over(window), + nth_value($"value", 2, ignoreNulls = false).over(window), + nth_value($"value", 2, ignoreNulls = true).over(window)), + Seq( + Row("a", 0, null, null, null), + Row("a", 1, "x", "x", null), + Row("a", 2, "x", "x", "y"), + Row("a", 3, "x", "x", "y"), + Row("a", 4, "x", "x", "y"), + Row("b", 1, null, null, null), + Row("b", 2, null, null, null))) + } + + test("nth_value on descending ordered window") { + val nullStr: String = null + val df = Seq( + ("a", 0, nullStr), + ("a", 1, "x"), + ("a", 2, "y"), + ("a", 3, "z"), + ("a", 4, "v"), + ("b", 1, "k"), + ("b", 2, "l"), + ("b", 3, nullStr)). + toDF("key", "order", "value") + val window = Window.partitionBy($"key").orderBy($"order".desc) + checkAnswer( + df.select( + $"key", + $"order", + nth_value($"value", 2).over(window), + nth_value($"value", 2, ignoreNulls = false).over(window), + nth_value($"value", 2, ignoreNulls = true).over(window)), + Seq( + Row("a", 0, "z", "z", "z"), + Row("a", 1, "z", "z", "z"), + Row("a", 2, "z", "z", "z"), + Row("a", 3, "z", "z", "z"), + Row("a", 4, null, null, null), + Row("b", 1, "l", "l", "k"), + Row("b", 2, "l", "l", null), + Row("b", 3, null, null, null))) + } + test("SPARK-12989 ExtractWindowExpressions treats alias as regular attribute") { val src = Seq((0, 3, 5)).toDF("a", "b", "c") .withColumn("Data", struct("a", "b")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala index 508eefafd0754..ff5c6242987de 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala @@ -23,16 +23,15 @@ import scala.collection.JavaConverters._ import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic} +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NamedRelation, NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, V2WriteCommand} import org.apache.spark.sql.connector.{InMemoryTable, InMemoryTableCatalog} import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.connector.expressions.{BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, YearsTransform} import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} -import org.apache.spark.sql.types.TimestampType +import org.apache.spark.sql.types.{ArrayType, DataType, IntegerType, LongType, MapType, StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.QueryExecutionListener import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils @@ -101,6 +100,86 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo assert(v2.catalog.exists(_ == catalogPlugin)) } + case class FakeV2WriteCommand(table: NamedRelation, query: LogicalPlan) extends V2WriteCommand + + test("SPARK-33136 output resolved on complex types for V2 write commands") { + val tableCatalog = catalog("testcat") + + def assertTypeCompatibility(name: String, fromType: DataType, toType: DataType): Unit = { + val fromTableName = s"from_table_$name" + tableCatalog.createTable( + Identifier.of(Array(), fromTableName), + StructType(Array(StructField("col", fromType))), + Array.empty, + new java.util.HashMap[String, String]()) + + val toTable = tableCatalog.createTable( + Identifier.of(Array(), s"to_table_$name"), + StructType(Array(StructField("col", toType))), + Array.empty, + new java.util.HashMap[String, String]()) + + val df = spark.table(s"testcat.$fromTableName") + + val relation = DataSourceV2Relation.create(toTable, Some(tableCatalog), None) + val writeCommand = FakeV2WriteCommand(relation, df.queryExecution.analyzed) + + assert(writeCommand.outputResolved, s"Unable to write from type $fromType to type $toType.") + } + + // The major difference between `from` and `to` is that `from` is a complex type + // with non-nullable, whereas `to` is same data type with flipping nullable. + + // nested struct type + val fromStructType = StructType(Array( + StructField("s", StringType), + StructField("i_nonnull", IntegerType, nullable = false), + StructField("st", StructType(Array( + StructField("l", LongType), + StructField("s_nonnull", StringType, nullable = false)))))) + + val toStructType = StructType(Array( + StructField("s", StringType), + StructField("i_nonnull", IntegerType), + StructField("st", StructType(Array( + StructField("l", LongType), + StructField("s_nonnull", StringType)))))) + + assertTypeCompatibility("struct", fromStructType, toStructType) + + // array type + assertTypeCompatibility("array", ArrayType(LongType, containsNull = false), + ArrayType(LongType, containsNull = true)) + + // array type with struct type + val fromArrayWithStructType = ArrayType( + StructType(Array(StructField("s", StringType, nullable = false))), + containsNull = false) + + val toArrayWithStructType = ArrayType( + StructType(Array(StructField("s", StringType))), + containsNull = true) + + assertTypeCompatibility("array_struct", fromArrayWithStructType, toArrayWithStructType) + + // map type + assertTypeCompatibility("map", MapType(IntegerType, StringType, valueContainsNull = false), + MapType(IntegerType, StringType, valueContainsNull = true)) + + // map type with struct type + val fromMapWithStructType = MapType( + IntegerType, + StructType(Array(StructField("s", StringType, nullable = false))), + valueContainsNull = false) + + val toMapWithStructType = MapType( + IntegerType, + StructType(Array(StructField("s", StringType))), + valueContainsNull = true) + + assertTypeCompatibility("map_struct", fromMapWithStructType, toMapWithStructType) + } + test("Append: basic append") { spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 4923e8b556907..3c914ae043677 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.exceptions.TestFailedException import org.scalatest.prop.TableDrivenPropertyChecks._ import org.apache.spark.{SparkException, TaskContext} -import org.apache.spark.sql.catalyst.ScroogeLikeExample +import org.apache.spark.sql.catalyst.{FooClassWithEnum, FooEnum, ScroogeLikeExample} import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder} import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} import org.apache.spark.sql.catalyst.util.sideBySide @@ -1926,6 +1926,19 @@ class DatasetSuite extends QueryTest } } } + + test("SPARK-32585: Support scala enumeration in ScalaReflection") { + checkDataset( + Seq(FooClassWithEnum(1, FooEnum.E1), FooClassWithEnum(2, FooEnum.E2)).toDS(), + Seq(FooClassWithEnum(1, FooEnum.E1), FooClassWithEnum(2, FooEnum.E2)): _* + ) + + // test null + checkDataset( + Seq(FooClassWithEnum(1, null), FooClassWithEnum(2, FooEnum.E2)).toDS(), + Seq(FooClassWithEnum(1, null), FooClassWithEnum(2, FooEnum.E2)): _* + ) + } } object AssertExecutionId { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala index d18aa9c549eb1..f3db4d811dd86 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala @@ -32,12 +32,12 @@ import org.apache.spark.tags.ExtendedSQLTest * * To run the entire test suite: * {{{ - * build/sbt "sql/test-only *ExpressionsSchemaSuite" + * build/sbt "sql/testOnly *ExpressionsSchemaSuite" * }}} * * To re-generate golden files for entire suite, run: * {{{ - * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *ExpressionsSchemaSuite" + * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *ExpressionsSchemaSuite" * }}} * * For example: @@ -178,7 +178,15 @@ class ExpressionsSchemaSuite extends QueryTest with SharedSparkSession { s"$numberOfQueries record in result file. Try regenerating the result files.") val numberOfMissingExamples = lines(3).split(":")(1).trim.toInt - val expectedMissingExamples = lines(4).split(":")(1).trim.split(",") + val expectedMissingExamples = { + val missingExamples = lines(4).split(":")(1).trim + // Splitting on a empty string would return [""] + if (missingExamples.nonEmpty) { + missingExamples.split(",") + } else { + Array.empty[String] + } + } assert(numberOfMissingExamples == expectedMissingExamples.size, s"expected missing examples size: ${expectedMissingExamples.size} not same as " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index a3cd0c230d8af..77e07e5550f35 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -31,12 +31,14 @@ import org.apache.spark.SparkException import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.sql.TestingUDT.{IntervalUDT, NullData, NullUDT} import org.apache.spark.sql.catalyst.expressions.AttributeReference -import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.expressions.IntegralLiteralTestUtils.{negativeInt, positiveInt} import org.apache.spark.sql.catalyst.plans.logical.Filter +import org.apache.spark.sql.execution.SimpleMode import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.FilePartition -import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2ScanRelation, FileScan} -import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable +import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, FileScan} +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -231,6 +233,20 @@ class FileBasedDataSourceSuite extends QueryTest } } + Seq("json", "orc").foreach { format => + test(s"SPARK-32889: column name supports special characters using $format") { + Seq("$", " ", ",", ";", "{", "}", "(", ")", "\n", "\t", "=").foreach { name => + withTempDir { dir => + val dataDir = new File(dir, "file").getCanonicalPath + Seq(1).toDF(name).write.format(format).save(dataDir) + val schema = spark.read.format(format).load(dataDir).schema + assert(schema.size == 1) + assertResult(name)(schema.head.name) + } + } + } + } + // Text file format only supports string type test("SPARK-24691 error handling for unsupported types - text") { withTempDir { dir => @@ -881,6 +897,114 @@ class FileBasedDataSourceSuite extends QueryTest } } } + + test("test casts pushdown on orc/parquet for integral types") { + def checkPushedFilters( + format: String, + df: DataFrame, + filters: Array[sources.Filter], + noScan: Boolean = false): Unit = { + val scanExec = df.queryExecution.sparkPlan.find(_.isInstanceOf[BatchScanExec]) + if (noScan) { + assert(scanExec.isEmpty) + return + } + val scan = scanExec.get.asInstanceOf[BatchScanExec].scan + format match { + case "orc" => + assert(scan.isInstanceOf[OrcScan]) + assert(scan.asInstanceOf[OrcScan].pushedFilters === filters) + case "parquet" => + assert(scan.isInstanceOf[ParquetScan]) + assert(scan.asInstanceOf[ParquetScan].pushedFilters === filters) + case _ => + fail(s"unknown format $format") + } + } + + Seq("orc", "parquet").foreach { format => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { + withTempPath { dir => + spark.range(100).map(i => (i.toShort, i.toString)).toDF("id", "s") + .write + .format(format) + .save(dir.getCanonicalPath) + val df = spark.read.format(format).load(dir.getCanonicalPath) + + // cases when value == MAX + var v = Short.MaxValue + checkPushedFilters(format, df.where('id > v.toInt), Array(), noScan = true) + checkPushedFilters(format, df.where('id >= v.toInt), Array(sources.IsNotNull("id"), + sources.EqualTo("id", v))) + checkPushedFilters(format, df.where('id === v.toInt), Array(sources.IsNotNull("id"), + sources.EqualTo("id", v))) + checkPushedFilters(format, df.where('id <=> v.toInt), + Array(sources.EqualNullSafe("id", v))) + checkPushedFilters(format, df.where('id <= v.toInt), Array(sources.IsNotNull("id"))) + checkPushedFilters(format, df.where('id < v.toInt), Array(sources.IsNotNull("id"), + sources.Not(sources.EqualTo("id", v)))) + + // cases when value > MAX + var v1: Int = positiveInt + checkPushedFilters(format, df.where('id > v1), Array(), noScan = true) + checkPushedFilters(format, df.where('id >= v1), Array(), noScan = true) + checkPushedFilters(format, df.where('id === v1), Array(), noScan = true) + checkPushedFilters(format, df.where('id <=> v1), Array(), noScan = true) + checkPushedFilters(format, df.where('id <= v1), Array(sources.IsNotNull("id"))) + checkPushedFilters(format, df.where('id < v1), Array(sources.IsNotNull("id"))) + + // cases when value = MIN + v = Short.MinValue + checkPushedFilters(format, df.where(lit(v.toInt) < 'id), Array(sources.IsNotNull("id"), + sources.Not(sources.EqualTo("id", v)))) + checkPushedFilters(format, df.where(lit(v.toInt) <= 'id), Array(sources.IsNotNull("id"))) + checkPushedFilters(format, df.where(lit(v.toInt) === 'id), Array(sources.IsNotNull("id"), + sources.EqualTo("id", v))) + checkPushedFilters(format, df.where(lit(v.toInt) <=> 'id), + Array(sources.EqualNullSafe("id", v))) + checkPushedFilters(format, df.where(lit(v.toInt) >= 'id), Array(sources.IsNotNull("id"), + sources.EqualTo("id", v))) + checkPushedFilters(format, df.where(lit(v.toInt) > 'id), Array(), noScan = true) + + // cases when value < MIN + v1 = negativeInt + checkPushedFilters(format, df.where(lit(v1) < 'id), Array(sources.IsNotNull("id"))) + checkPushedFilters(format, df.where(lit(v1) <= 'id), Array(sources.IsNotNull("id"))) + checkPushedFilters(format, df.where(lit(v1) === 'id), Array(), noScan = true) + checkPushedFilters(format, df.where(lit(v1) >= 'id), Array(), noScan = true) + checkPushedFilters(format, df.where(lit(v1) > 'id), Array(), noScan = true) + + // cases when value is within range (MIN, MAX) + checkPushedFilters(format, df.where('id > 30), Array(sources.IsNotNull("id"), + sources.GreaterThan("id", 30))) + checkPushedFilters(format, df.where(lit(100) >= 'id), Array(sources.IsNotNull("id"), + sources.LessThanOrEqual("id", 100))) + } + } + } + } + + test("SPARK-32827: Set max metadata string length") { + withTempDir { dir => + val tableName = "t" + val path = s"${dir.getCanonicalPath}/$tableName" + withTable(tableName) { + sql(s"CREATE TABLE $tableName(c INT) USING PARQUET LOCATION '$path'") + withSQLConf(SQLConf.MAX_METADATA_STRING_LENGTH.key -> "5") { + val explain = spark.table(tableName).queryExecution.explainString(SimpleMode) + assert(!explain.contains(path)) + // metadata has abbreviated by ... + assert(explain.contains("...")) + } + + withSQLConf(SQLConf.MAX_METADATA_STRING_LENGTH.key -> "1000") { + val explain = spark.table(tableName).queryExecution.explainString(SimpleMode) + assert(explain.contains(path)) + assert(!explain.contains("...")) + } + } + } + } } object TestingUDT { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 942cf24a3a873..8755dccb801c2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -757,6 +757,14 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan ) } + // LEFT ANTI JOIN without bound condition does not spill + assertNotSpilled(sparkContext, "left anti join") { + checkAnswer( + sql("SELECT * FROM testData LEFT ANTI JOIN testData2 ON key = a WHERE key = 2"), + Nil + ) + } + val expected = new ListBuffer[Row]() expected.append( Row(1, "1", 1, 1), Row(1, "1", 1, 2), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index 03b48451c7495..5a1a3550d855b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -733,4 +733,18 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { | """.stripMargin) checkAnswer(toDF("yyyy-MM-dd'T'HH:mm:ss.SSSXXX"), toDF("yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]")) } + + test("SPARK-33134: return partial results only for root JSON objects") { + val st = new StructType() + .add("c1", LongType) + .add("c2", ArrayType(new StructType().add("c3", LongType).add("c4", StringType))) + val df1 = Seq("""{"c2": [19], "c1": 123456}""").toDF("c0") + checkAnswer(df1.select(from_json($"c0", st)), Row(Row(123456, null))) + val df2 = Seq("""{"data": {"c2": [19], "c1": 123456}}""").toDF("c0") + checkAnswer(df2.select(from_json($"c0", new StructType().add("data", st))), Row(Row(null))) + val df3 = Seq("""[{"c2": [19], "c1": 123456}]""").toDF("c0") + checkAnswer(df3.select(from_json($"c0", ArrayType(st))), Row(null)) + val df4 = Seq("""{"c2": [19]}""").toDF("c0") + checkAnswer(df4.select(from_json($"c0", MapType(StringType, st))), Row(null)) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala index bd86c2ec075b0..cd92976571230 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala @@ -125,6 +125,11 @@ class MathFunctionsSuite extends QueryTest with SharedSparkSession { testOneToOneMathFunction(sinh, math.sinh) } + test("asinh") { + testOneToOneMathFunction(asinh, + (x: Double) => math.log(x + math.sqrt(x * x + 1)) ) + } + test("cos") { testOneToOneMathFunction(cos, math.cos) } @@ -137,6 +142,11 @@ class MathFunctionsSuite extends QueryTest with SharedSparkSession { testOneToOneMathFunction(cosh, math.cosh) } + test("acosh") { + testOneToOneMathFunction(acosh, + (x: Double) => math.log(x + math.sqrt(x * x - 1)) ) + } + test("tan") { testOneToOneMathFunction(tan, math.tan) } @@ -149,6 +159,11 @@ class MathFunctionsSuite extends QueryTest with SharedSparkSession { testOneToOneMathFunction(tanh, math.tanh) } + test("atanh") { + testOneToOneMathFunction(atanh, + (x: Double) => (0.5 * (math.log1p(x) - math.log1p(-x))) ) + } + test("degrees") { testOneToOneMathFunction(degrees, math.toDegrees) checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala index f78fc269986b5..c2aee0ad4c9a1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala @@ -51,22 +51,22 @@ import org.apache.spark.tags.ExtendedSQLTest * * To run the entire test suite: * {{{ - * build/sbt "sql/test-only *PlanStability[WithStats]Suite" + * build/sbt "sql/testOnly *PlanStability[WithStats]Suite" * }}} * * To run a single test file upon change: * {{{ - * build/sbt "sql/test-only *PlanStability[WithStats]Suite -- -z (tpcds-v1.4/q49)" + * build/sbt "sql/testOnly *PlanStability[WithStats]Suite -- -z (tpcds-v1.4/q49)" * }}} * * To re-generate golden files for entire suite, run: * {{{ - * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *PlanStability[WithStats]Suite" + * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *PlanStability[WithStats]Suite" * }}} * * To re-generate golden file for a single test, run: * {{{ - * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *PlanStability[WithStats]Suite -- -z (tpcds-v1.4/q49)" + * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *PlanStability[WithStats]Suite -- -z (tpcds-v1.4/q49)" * }}} */ // scalastyle:on line.size.limit diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index b7cf0798a9d4b..36e55c0994f18 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -49,22 +49,22 @@ import org.apache.spark.util.Utils * * To run the entire test suite: * {{{ - * build/sbt "sql/test-only *SQLQueryTestSuite" + * build/sbt "sql/testOnly *SQLQueryTestSuite" * }}} * * To run a single test file upon change: * {{{ - * build/sbt "~sql/test-only *SQLQueryTestSuite -- -z inline-table.sql" + * build/sbt "~sql/testOnly *SQLQueryTestSuite -- -z inline-table.sql" * }}} * * To re-generate golden files for entire suite, run: * {{{ - * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *SQLQueryTestSuite" + * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite" * }}} * * To re-generate golden file for a single test, run: * {{{ - * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *SQLQueryTestSuite -- -z describe.sql" + * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite -- -z describe.sql" * }}} * * The format for input files is simple: @@ -502,7 +502,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper case _: DescribeCommandBase | _: DescribeColumnCommand | _: DescribeRelation - | _: DescribeColumnStatement => true + | _: DescribeColumn => true case PhysicalOperation(_, _, Sort(_, true, _)) => true case _ => plan.children.iterator.exists(isSorted) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index a21c461e84588..73b23496de515 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -1314,7 +1314,7 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark // need to execute the query before we can examine fs.inputRDDs() assert(stripAQEPlan(df.queryExecution.executedPlan) match { case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter( - fs @ FileSourceScanExec(_, _, _, partitionFilters, _, _, _, _)))) => + fs @ FileSourceScanExec(_, _, _, partitionFilters, _, _, _, _, _)))) => partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && fs.inputRDDs().forall( _.asInstanceOf[FileScanRDD].filePartitions.forall( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala index decd1d6d08d27..22e1b838f3f3f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala @@ -20,11 +20,13 @@ package org.apache.spark.sql import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.util.resourceToString import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.tags.ExtendedSQLTest /** * This test suite ensures all the TPC-DS queries can be successfully analyzed, optimized * and compiled without hitting the max iteration threshold. */ +@ExtendedSQLTest class TPCDSQuerySuite extends BenchmarkQueryTest with TPCDSBase { tpcdsQueries.foreach { name => @@ -64,10 +66,12 @@ class TPCDSQuerySuite extends BenchmarkQueryTest with TPCDSBase { } } +@ExtendedSQLTest class TPCDSQueryWithStatsSuite extends TPCDSQuerySuite { override def injectStats: Boolean = true } +@ExtendedSQLTest class TPCDSQueryANSISuite extends TPCDSQuerySuite { override protected def sparkConf: SparkConf = super.sparkConf.set(SQLConf.ANSI_ENABLED, true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UnwrapCastInComparisonEndToEndSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UnwrapCastInComparisonEndToEndSuite.scala new file mode 100644 index 0000000000000..e6f0426428bd4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/UnwrapCastInComparisonEndToEndSuite.scala @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.expressions.IntegralLiteralTestUtils.{negativeInt, positiveInt} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.Decimal + +class UnwrapCastInComparisonEndToEndSuite extends QueryTest with SharedSparkSession { + import testImplicits._ + + val t = "test_table" + + test("cases when literal is max") { + withTable(t) { + Seq[(Integer, java.lang.Short, java.lang.Float)]( + (1, 100.toShort, 3.14.toFloat), + (2, Short.MaxValue, Float.NaN), + (3, Short.MinValue, Float.PositiveInfinity), + (4, 0.toShort, Float.MaxValue), + (5, null, null)) + .toDF("c1", "c2", "c3").write.saveAsTable(t) + val df = spark.table(t) + + val lit = Short.MaxValue.toInt + checkAnswer(df.where(s"c2 > $lit").select("c1"), Seq.empty) + checkAnswer(df.where(s"c2 >= $lit").select("c1"), Row(2)) + checkAnswer(df.where(s"c2 == $lit").select("c1"), Row(2)) + checkAnswer(df.where(s"c2 <=> $lit").select("c1"), Row(2)) + checkAnswer(df.where(s"c2 != $lit").select("c1"), Row(1) :: Row(3) :: Row(4) :: Nil) + checkAnswer(df.where(s"c2 <= $lit").select("c1"), Row(1) :: Row(2) :: Row(3) :: Row(4) :: Nil) + checkAnswer(df.where(s"c2 < $lit").select("c1"), Row(1) :: Row(3) :: Row(4) :: Nil) + + checkAnswer(df.where(s"c3 > double('nan')").select("c1"), Seq.empty) + checkAnswer(df.where(s"c3 >= double('nan')").select("c1"), Row(2)) + checkAnswer(df.where(s"c3 == double('nan')").select("c1"), Row(2)) + checkAnswer(df.where(s"c3 <=> double('nan')").select("c1"), Row(2)) + checkAnswer(df.where(s"c3 != double('nan')").select("c1"), Row(1) :: Row(3) :: Row(4) :: Nil) + checkAnswer(df.where(s"c3 <= double('nan')").select("c1"), + Row(1) :: Row(2) :: Row(3) :: Row(4) :: Nil) + checkAnswer(df.where(s"c3 < double('nan')").select("c1"), Row(1) :: Row(3) :: Row(4) :: Nil) + } + } + + test("cases when literal is > max") { + withTable(t) { + Seq[(Integer, java.lang.Short)]( + (1, 100.toShort), + (2, Short.MaxValue), + (3, null)) + .toDF("c1", "c2").write.saveAsTable(t) + val df = spark.table(t) + val lit = positiveInt + checkAnswer(df.where(s"c2 > $lit").select("c1"), Seq.empty) + checkAnswer(df.where(s"c2 >= $lit").select("c1"), Seq.empty) + checkAnswer(df.where(s"c2 == $lit").select("c1"), Seq.empty) + checkAnswer(df.where(s"c2 <=> $lit").select("c1"), Seq.empty) + checkAnswer(df.where(s"c2 != $lit").select("c1"), Row(1) :: Row(2) :: Nil) + checkAnswer(df.where(s"c2 <= $lit").select("c1"), Row(1) :: Row(2) :: Nil) + checkAnswer(df.where(s"c2 < $lit").select("c1"), Row(1) :: Row(2) :: Nil) + + // No test for float case since NaN is greater than any other numeric value + } + } + + test("cases when literal is min") { + withTable(t) { + Seq[(Integer, java.lang.Short, java.lang.Float)]( + (1, 100.toShort, 3.14.toFloat), + (2, Short.MinValue, Float.NegativeInfinity), + (3, Short.MaxValue, Float.MinValue), + (4, null, null)) + .toDF("c1", "c2", "c3").write.saveAsTable(t) + val df = spark.table(t) + + val lit = Short.MinValue.toInt + checkAnswer(df.where(s"c2 > $lit").select("c1"), Row(1) :: Row(3) :: Nil) + checkAnswer(df.where(s"c2 >= $lit").select("c1"), Row(1) :: Row(2) :: Row(3) :: Nil) + checkAnswer(df.where(s"c2 == $lit").select("c1"), Row(2)) + checkAnswer(df.where(s"c2 <=> $lit").select("c1"), Row(2)) + checkAnswer(df.where(s"c2 != $lit").select("c1"), Row(1) :: Row(3) :: Nil) + checkAnswer(df.where(s"c2 <= $lit").select("c1"), Row(2)) + checkAnswer(df.where(s"c2 < $lit").select("c1"), Seq.empty) + + checkAnswer(df.where(s"c3 > double('-inf')").select("c1"), Row(1) :: Row(3) :: Nil) + checkAnswer(df.where(s"c3 >= double('-inf')").select("c1"), Row(1) :: Row(2) :: Row(3) :: Nil) + checkAnswer(df.where(s"c3 == double('-inf')").select("c1"), Row(2)) + checkAnswer(df.where(s"c3 <=> double('-inf')").select("c1"), Row(2)) + checkAnswer(df.where(s"c3 != double('-inf')").select("c1"), Row(1) :: Row(3) :: Nil) + checkAnswer(df.where(s"c3 <= double('-inf')").select("c1"), Row(2) :: Nil) + checkAnswer(df.where(s"c3 < double('-inf')").select("c1"), Seq.empty) + } + } + + test("cases when literal is < min") { + val t = "test_table" + withTable(t) { + Seq[(Integer, java.lang.Short)]( + (1, 100.toShort), + (2, Short.MinValue), + (3, null)) + .toDF("c1", "c2").write.saveAsTable(t) + val df = spark.table(t) + + val lit = negativeInt + checkAnswer(df.where(s"c2 > $lit").select("c1"), Row(1) :: Row(2) :: Nil) + checkAnswer(df.where(s"c2 >= $lit").select("c1"), Row(1) :: Row(2) :: Nil) + checkAnswer(df.where(s"c2 == $lit").select("c1"), Seq.empty) + checkAnswer(df.where(s"c2 <=> $lit").select("c1"), Seq.empty) + checkAnswer(df.where(s"c2 != $lit").select("c1"), Row(1) :: Row(2) :: Nil) + checkAnswer(df.where(s"c2 <= $lit").select("c1"), Seq.empty) + checkAnswer(df.where(s"c2 < $lit").select("c1"), Seq.empty) + } + } + + test("cases when literal is within range (min, max)") { + withTable(t) { + Seq((1, 300.toShort), (2, 500.toShort)).toDF("c1", "c2").write.saveAsTable(t) + val df = spark.table(t) + + checkAnswer(df.where("c2 < 200").select("c1"), Seq.empty) + checkAnswer(df.where("c2 < 400").select("c1"), Row(1) :: Nil) + checkAnswer(df.where("c2 < 600").select("c1"), Row(1) :: Row(2) :: Nil) + + checkAnswer(df.where("c2 <= 100").select("c1"), Seq.empty) + checkAnswer(df.where("c2 <= 300").select("c1"), Row(1) :: Nil) + checkAnswer(df.where("c2 <= 500").select("c1"), Row(1) :: Row(2) :: Nil) + + checkAnswer(df.where("c2 == 100").select("c1"), Seq.empty) + checkAnswer(df.where("c2 == 300").select("c1"), Row(1) :: Nil) + checkAnswer(df.where("c2 == 500").select("c1"), Row(2) :: Nil) + + checkAnswer(df.where("c2 <=> 100").select("c1"), Seq.empty) + checkAnswer(df.where("c2 <=> 300").select("c1"), Row(1) :: Nil) + checkAnswer(df.where("c2 <=> 500").select("c1"), Row(2) :: Nil) + checkAnswer(df.where("c2 <=> null").select("c1"), Seq.empty) + + checkAnswer(df.where("c2 >= 200").select("c1"), Row(1) :: Row(2) :: Nil) + checkAnswer(df.where("c2 >= 400").select("c1"), Row(2) :: Nil) + checkAnswer(df.where("c2 >= 600").select("c1"), Seq.empty) + + checkAnswer(df.where("c2 > 100").select("c1"), Row(1) :: Row(2) :: Nil) + checkAnswer(df.where("c2 > 300").select("c1"), Row(2) :: Nil) + checkAnswer(df.where("c2 > 500").select("c1"), Seq.empty) + } + } + + test("cases when literal is within range (min, max) and has rounding up or down") { + withTable(t) { + Seq((1, 100, 3.14.toFloat, decimal(200.12))) + .toDF("c1", "c2", "c3", "c4").write.saveAsTable(t) + val df = spark.table(t) + + checkAnswer(df.where("c2 > 99.6").select("c1"), Row(1)) + checkAnswer(df.where("c2 > 100.4").select("c1"), Seq.empty) + checkAnswer(df.where("c2 == 100.4").select("c1"), Seq.empty) + checkAnswer(df.where("c2 <=> 100.4").select("c1"), Seq.empty) + checkAnswer(df.where("c2 < 99.6").select("c1"), Seq.empty) + checkAnswer(df.where("c2 < 100.4").select("c1"), Row(1)) + + checkAnswer(df.where("c3 >= 3.14").select("c1"), Row(1)) + // float(3.14) is casted to double(3.140000104904175) + checkAnswer(df.where("c3 >= 3.14000010").select("c1"), Row(1)) + checkAnswer(df.where("c3 == 3.14").select("c1"), Seq.empty) + checkAnswer(df.where("c3 <=> 3.14").select("c1"), Seq.empty) + checkAnswer(df.where("c3 < 3.14000010").select("c1"), Seq.empty) + checkAnswer(df.where("c3 <= 3.14").select("c1"), Seq.empty) + + checkAnswer(df.where("c4 > cast(200.1199 as decimal(10, 4))").select("c1"), Row(1)) + checkAnswer(df.where("c4 >= cast(200.1201 as decimal(10, 4))").select("c1"), Seq.empty) + checkAnswer(df.where("c4 == cast(200.1156 as decimal(10, 4))").select("c1"), Seq.empty) + checkAnswer(df.where("c4 <=> cast(200.1201 as decimal(10, 4))").select("c1"), Seq.empty) + checkAnswer(df.where("c4 <= cast(200.1201 as decimal(10, 4))").select("c1"), Row(1)) + checkAnswer(df.where("c4 < cast(200.1159 as decimal(10, 4))").select("c1"), Seq.empty) + } + } + + private def decimal(v: BigDecimal): Decimal = Decimal(v, 5, 2) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala new file mode 100644 index 0000000000000..28af552fe586b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala @@ -0,0 +1,224 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark +import org.apache.spark.sql.functions.{col, lit} +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} + +/** + * Benchmark to measure Spark's performance analyzing and optimizing long UpdateFields chains. + * + * {{{ + * To run this benchmark: + * 1. without sbt: + * bin/spark-submit --class + * 2. with sbt: + * build/sbt "sql/test:runMain " + * 3. generate result: + * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " + * Results will be written to "benchmarks/UpdateFieldsBenchmark-results.txt". + * }}} + */ +object UpdateFieldsBenchmark extends SqlBasedBenchmark { + + def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum" + + def nestedStructType( + colNums: Seq[Int], + nullable: Boolean, + maxDepth: Int, + currDepth: Int = 1): StructType = { + + if (currDepth == maxDepth) { + val fields = colNums.map { colNum => + val name = nestedColName(currDepth, colNum) + StructField(name, IntegerType, nullable = false) + } + StructType(fields) + } else { + val fields = colNums.foldLeft(Seq.empty[StructField]) { + case (structFields, colNum) if colNum == 0 => + val nested = nestedStructType(colNums, nullable, maxDepth, currDepth + 1) + structFields :+ StructField(nestedColName(currDepth, colNum), nested, nullable) + case (structFields, colNum) => + val name = nestedColName(currDepth, colNum) + structFields :+ StructField(name, IntegerType, nullable = false) + } + StructType(fields) + } + } + + /** + * Utility function for generating an empty DataFrame with nested columns. + * + * @param maxDepth: The depth to which to create nested columns. + * @param numColsAtEachDepth: The number of columns to create at each depth. + * @param nullable: This value is used to set the nullability of any StructType columns. + */ + def emptyNestedDf(maxDepth: Int, numColsAtEachDepth: Int, nullable: Boolean): DataFrame = { + require(maxDepth > 0) + require(numColsAtEachDepth > 0) + + val nestedColumnDataType = nestedStructType(0 until numColsAtEachDepth, nullable, maxDepth) + spark.createDataFrame( + spark.sparkContext.emptyRDD[Row], + StructType(Seq(StructField(nestedColName(0, 0), nestedColumnDataType, nullable)))) + } + + trait ModifyNestedColumns { + val name: String + def apply(column: Column, numsToAdd: Seq[Int], numsToDrop: Seq[Int], maxDepth: Int): Column + } + + object Performant extends ModifyNestedColumns { + override val name: String = "performant" + + override def apply( + column: Column, + numsToAdd: Seq[Int], + numsToDrop: Seq[Int], + maxDepth: Int): Column = helper(column, numsToAdd, numsToDrop, maxDepth, 1) + + private def helper( + column: Column, + numsToAdd: Seq[Int], + numsToDrop: Seq[Int], + maxDepth: Int, + currDepth: Int): Column = { + + // drop columns at the current depth + val dropped = if (numsToDrop.nonEmpty) { + column.dropFields(numsToDrop.map(num => nestedColName(currDepth, num)): _*) + } else column + + // add columns at the current depth + val added = numsToAdd.foldLeft(dropped) { + (res, num) => res.withField(nestedColName(currDepth, num), lit(num)) + } + + if (currDepth == maxDepth) { + added + } else { + // add/drop columns at the next depth + val newValue = helper( + column = col((0 to currDepth).map(d => nestedColName(d, 0)).mkString(".")), + numsToAdd = numsToAdd, + numsToDrop = numsToDrop, + currDepth = currDepth + 1, + maxDepth = maxDepth) + added.withField(nestedColName(currDepth, 0), newValue) + } + } + } + + object NonPerformant extends ModifyNestedColumns { + override val name: String = "non-performant" + + override def apply( + column: Column, + numsToAdd: Seq[Int], + numsToDrop: Seq[Int], + maxDepth: Int): Column = { + + val dropped = if (numsToDrop.nonEmpty) { + val colsToDrop = (1 to maxDepth).flatMap { depth => + numsToDrop.map(num => s"${prefix(depth)}${nestedColName(depth, num)}") + } + column.dropFields(colsToDrop: _*) + } else column + + val added = { + val colsToAdd = (1 to maxDepth).flatMap { depth => + numsToAdd.map(num => (s"${prefix(depth)}${nestedColName(depth, num)}", lit(num))) + } + colsToAdd.foldLeft(dropped)((col, add) => col.withField(add._1, add._2)) + } + + added + } + + private def prefix(depth: Int): String = + if (depth == 1) "" + else (1 until depth).map(d => nestedColName(d, 0)).mkString("", ".", ".") + } + + private def updateFieldsBenchmark( + methods: Seq[ModifyNestedColumns], + maxDepth: Int, + initialNumberOfColumns: Int, + numsToAdd: Seq[Int] = Seq.empty, + numsToDrop: Seq[Int] = Seq.empty): Unit = { + + val name = s"Add ${numsToAdd.length} columns and drop ${numsToDrop.length} columns " + + s"at $maxDepth different depths of nesting" + + runBenchmark(name) { + val benchmark = new Benchmark( + name = name, + // The purpose of this benchmark is to ensure Spark is able to analyze and optimize long + // UpdateFields chains quickly so it runs over 0 rows of data. + valuesPerIteration = 0, + output = output) + + val nonNullableStructsDf = emptyNestedDf(maxDepth, initialNumberOfColumns, nullable = false) + val nullableStructsDf = emptyNestedDf(maxDepth, initialNumberOfColumns, nullable = true) + + methods.foreach { method => + val modifiedColumn = method( + column = col(nestedColName(0, 0)), + numsToAdd = numsToAdd, + numsToDrop = numsToDrop, + maxDepth = maxDepth + ).as(nestedColName(0, 0)) + + benchmark.addCase(s"To non-nullable StructTypes using ${method.name} method") { _ => + nonNullableStructsDf.select(modifiedColumn).queryExecution.optimizedPlan + } + + benchmark.addCase(s"To nullable StructTypes using ${method.name} method") { _ => + nullableStructsDf.select(modifiedColumn).queryExecution.optimizedPlan + } + } + + benchmark.run() + } + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + // This benchmark compares the performant and non-performant methods of writing the same query. + // We use small values for maxDepth, numsToAdd, and numsToDrop because the NonPerformant method + // scales extremely poorly with the number of nested columns being added/dropped. + updateFieldsBenchmark( + methods = Seq(Performant, NonPerformant), + maxDepth = 3, + initialNumberOfColumns = 5, + numsToAdd = 5 to 6, + numsToDrop = 3 to 4) + + // This benchmark is to show that the performant method of writing a query when we want to add + // and drop a large number of nested columns scales nicely. + updateFieldsBenchmark( + methods = Seq(Performant), + maxDepth = 100, + initialNumberOfColumns = 51, + numsToAdd = 51 to 100, + numsToDrop = 1 to 50) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 9acb00b7b6d0b..b6ab60a91955d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -21,8 +21,8 @@ import java.time.{LocalDateTime, ZoneOffset} import java.util.Arrays import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{Cast, ExpressionEvalHelper, GenericInternalRow, Literal} +import org.apache.spark.sql.catalyst.CatalystTypeConverters +import org.apache.spark.sql.catalyst.expressions.{Cast, ExpressionEvalHelper, Literal} import org.apache.spark.sql.execution.datasources.parquet.ParquetTest import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSparkSession @@ -33,77 +33,6 @@ private[sql] case class MyLabeledPoint(label: Double, features: TestUDT.MyDenseV def getFeatures: TestUDT.MyDenseVector = features } -// object and classes to test SPARK-19311 - -// Trait/Interface for base type -sealed trait IExampleBaseType extends Serializable { - def field: Int -} - -// Trait/Interface for derived type -sealed trait IExampleSubType extends IExampleBaseType - -// a base class -class ExampleBaseClass(override val field: Int) extends IExampleBaseType - -// a derived class -class ExampleSubClass(override val field: Int) - extends ExampleBaseClass(field) with IExampleSubType - -// UDT for base class -class ExampleBaseTypeUDT extends UserDefinedType[IExampleBaseType] { - - override def sqlType: StructType = { - StructType(Seq( - StructField("intfield", IntegerType, nullable = false))) - } - - override def serialize(obj: IExampleBaseType): InternalRow = { - val row = new GenericInternalRow(1) - row.setInt(0, obj.field) - row - } - - override def deserialize(datum: Any): IExampleBaseType = { - datum match { - case row: InternalRow => - require(row.numFields == 1, - "ExampleBaseTypeUDT requires row with length == 1") - val field = row.getInt(0) - new ExampleBaseClass(field) - } - } - - override def userClass: Class[IExampleBaseType] = classOf[IExampleBaseType] -} - -// UDT for derived class -private[spark] class ExampleSubTypeUDT extends UserDefinedType[IExampleSubType] { - - override def sqlType: StructType = { - StructType(Seq( - StructField("intfield", IntegerType, nullable = false))) - } - - override def serialize(obj: IExampleSubType): InternalRow = { - val row = new GenericInternalRow(1) - row.setInt(0, obj.field) - row - } - - override def deserialize(datum: Any): IExampleSubType = { - datum match { - case row: InternalRow => - require(row.numFields == 1, - "ExampleSubTypeUDT requires row with length == 1") - val field = row.getInt(0) - new ExampleSubClass(field) - } - } - - override def userClass: Class[IExampleSubType] = classOf[IExampleSubType] -} - private[sql] case class FooWithDate(date: LocalDateTime, s: String, i: Int) private[sql] class LocalDateTimeUDT extends UserDefinedType[LocalDateTime] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index ffc115e6b7600..e3618f1326941 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -168,7 +168,31 @@ class DataSourceV2SQLSuite Array("Provider", "foo", ""), Array(TableCatalog.PROP_OWNER.capitalize, defaultUser, ""), Array("Table Properties", "[bar=baz]", ""))) + } + test("Describe column is not supported for v2 catalog") { + withTable("testcat.tbl") { + spark.sql("CREATE TABLE testcat.tbl (id bigint) USING foo") + val ex = intercept[AnalysisException] { + spark.sql("DESCRIBE testcat.tbl id") + } + assert(ex.message.contains("Describing columns is not supported for v2 tables")) + } + } + + test("SPARK-33004: Describe column should resolve to a temporary view first") { + withTable("testcat.ns.t") { + withTempView("t") { + sql("CREATE TABLE testcat.ns.t (id bigint) USING foo") + sql("CREATE TEMPORARY VIEW t AS SELECT 2 as i") + sql("USE testcat.ns") + checkAnswer( + sql("DESCRIBE t i"), + Seq(Row("col_name", "i"), + Row("data_type", "int"), + Row("comment", "NULL"))) + } + } } test("CreateTable: use v2 plan and session catalog when provider is v2") { @@ -758,8 +782,9 @@ class DataSourceV2SQLSuite test("Relation: view text") { val t1 = "testcat.ns1.ns2.tbl" + val v1 = "view1" withTable(t1) { - withView("view1") { v1: String => + withView(v1) { sql(s"CREATE TABLE $t1 USING foo AS SELECT id, data FROM source") sql(s"CREATE VIEW $v1 AS SELECT * from $t1") checkAnswer(sql(s"TABLE $v1"), spark.table("source")) @@ -1728,6 +1753,23 @@ class DataSourceV2SQLSuite } } + test("SPARK-32990: REFRESH TABLE should resolve to a temporary view first") { + withTable("testcat.ns.t") { + withTempView("t") { + sql("CREATE TABLE testcat.ns.t (id bigint) USING foo") + sql("CREATE TEMPORARY VIEW t AS SELECT 2") + sql("USE testcat.ns") + + val testCatalog = catalog("testcat").asTableCatalog.asInstanceOf[InMemoryTableCatalog] + val identifier = Identifier.of(Array("ns"), "t") + + assert(!testCatalog.isTableInvalidated(identifier)) + sql("REFRESH TABLE t") + assert(!testCatalog.isTableInvalidated(identifier)) + } + } + } + test("REPLACE TABLE: v1 table") { val e = intercept[AnalysisException] { sql(s"CREATE OR REPLACE TABLE tbl (a int) USING ${classOf[SimpleScanSource].getName}") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index a9c521eb46499..ec1ac00d08bf8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -268,7 +268,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS } } // this input data will fail to read middle way. - val input = spark.range(10).select(failingUdf('id).as('i)).select('i, -'i as 'j) + val input = spark.range(15).select(failingUdf('id).as('i)).select('i, -'i as 'j) val e3 = intercept[SparkException] { input.write.format(cls.getName).option("path", path).mode("overwrite").save() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala index 1d016496df2de..2d75a35215866 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala @@ -46,6 +46,8 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { table, CaseInsensitiveStringMap.empty(), TableCapabilityCheckSuite.schema.toAttributes, + None, + None, v1Relation) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala index 637cf2fd16515..4e741ff35c29f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala @@ -22,8 +22,8 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException -import org.apache.spark.sql.connector.catalog.{DelegatingCatalogExtension, Identifier, Table} +import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.connector.catalog.{DelegatingCatalogExtension, Identifier, Table, V1Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType @@ -47,10 +47,13 @@ private[connector] trait TestV2SessionCatalogBase[T <: Table] extends Delegating tables.get(ident) } else { // Table was created through the built-in catalog - val t = super.loadTable(ident) - val table = newTable(t.name(), t.schema(), t.partitioning(), t.properties()) - tables.put(ident, table) - table + super.loadTable(ident) match { + case v1Table: V1Table if v1Table.v1Table.tableType == CatalogTableType.VIEW => v1Table + case t => + val table = newTable(t.name(), t.schema(), t.partitioning(), t.properties()) + tables.put(ident, table) + table + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala index 02f447bd14339..c07ea0f12f94e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala @@ -63,16 +63,6 @@ abstract class BaseScriptTransformationSuite extends SparkPlanTest with SQLTestU uncaughtExceptionHandler.cleanStatus() } - def isHive23OrSpark: Boolean - - // In Hive 1.2, the string representation of a decimal omits trailing zeroes. - // But in Hive 2.3, it is always padded to 18 digits with trailing zeroes if necessary. - val decimalToString: Column => Column = if (isHive23OrSpark) { - c => c.cast("string") - } else { - c => c.cast("decimal(1, 0)").cast("string") - } - def createScriptTransformationExec( input: Seq[Expression], script: String, @@ -142,7 +132,7 @@ abstract class BaseScriptTransformationSuite extends SparkPlanTest with SQLTestU 'a.cast("string"), 'b.cast("string"), 'c.cast("string"), - decimalToString('d), + 'd.cast("string"), 'e.cast("string")).collect()) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index d428b7ebc0e91..ca52e51c87ea7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -994,6 +994,13 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { } } } + + testWithWholeStageCodegenOnAndOff("Change the number of partitions to zero " + + "when a range is empty") { _ => + val range = spark.range(1, 1, 1, 1000) + val numPartitions = range.rdd.getNumPartitions + assert(numPartitions == 0) + } } // Used for unit-testing EnsureRequirements diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index 83c80b4f3eb08..585ce4e40471d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -219,7 +219,7 @@ class QueryExecutionSuite extends SharedSparkSession { spark.range(1).groupBy("id").count().queryExecution.executedPlan } } - Seq("=== Applying Rule org.apache.spark.sql.execution.CollapseCodegenStages ===", + Seq("=== Applying Rule org.apache.spark.sql.execution", "=== Result of Batch Preparations ===").foreach { expectedMsg => assert(testAppender.loggingEvents.exists(_.getRenderedMessage.contains(expectedMsg))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala index bc24436c5806a..930935f077665 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala @@ -18,17 +18,21 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils -class RemoveRedundantProjectsSuite extends QueryTest with SharedSparkSession with SQLTestUtils { +abstract class RemoveRedundantProjectsSuiteBase + extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { private def assertProjectExecCount(df: DataFrame, expected: Int): Unit = { withClue(df.queryExecution) { val plan = df.queryExecution.executedPlan - val actual = plan.collectWithSubqueries { case p: ProjectExec => p }.size + val actual = collectWithSubqueries(plan) { case p: ProjectExec => p }.size assert(actual == expected) } } @@ -115,9 +119,41 @@ class RemoveRedundantProjectsSuite extends QueryTest with SharedSparkSession wit assertProjectExec(query, 1, 2) } - test("generate") { - val query = "select a, key, explode(d) from testView where a > 10" - assertProjectExec(query, 0, 1) + test("generate should require column ordering") { + withTempView("testData") { + spark.range(0, 10, 1) + .selectExpr("id as key", "id * 2 as a", "id * 3 as b") + .createOrReplaceTempView("testData") + + val data = sql("select key, a, b, count(*) from testData group by key, a, b limit 2") + val df = data.selectExpr("a", "b", "key", "explode(array(key, a, b)) as d").filter("d > 0") + df.collect() + val plan = df.queryExecution.executedPlan + val numProjects = collectWithSubqueries(plan) { case p: ProjectExec => p }.length + + // Create a new plan that reverse the GenerateExec output and add a new ProjectExec between + // GenerateExec and its child. This is to test if the ProjectExec is removed, the output of + // the query will be incorrect. + val newPlan = stripAQEPlan(plan) transform { + case g @ GenerateExec(_, requiredChildOutput, _, _, child) => + g.copy(requiredChildOutput = requiredChildOutput.reverse, + child = ProjectExec(requiredChildOutput.reverse, child)) + } + + // Re-apply remove redundant project rule. + val rule = RemoveRedundantProjects(spark.sessionState.conf) + val newExecutedPlan = rule.apply(newPlan) + // The manually added ProjectExec node shouldn't be removed. + assert(collectWithSubqueries(newExecutedPlan) { + case p: ProjectExec => p + }.size == numProjects + 1) + + // Check the original plan's output and the new plan's output are the same. + val expectedRows = plan.executeCollect() + val actualRows = newExecutedPlan.executeCollect() + assert(expectedRows.length == actualRows.length) + expectedRows.zip(actualRows).foreach { case (expected, actual) => assert(expected == actual) } + } } test("subquery") { @@ -131,3 +167,9 @@ class RemoveRedundantProjectsSuite extends QueryTest with SharedSparkSession wit } } } + +class RemoveRedundantProjectsSuite extends RemoveRedundantProjectsSuiteBase + with DisableAdaptiveExecutionSuite + +class RemoveRedundantProjectsSuiteAE extends RemoveRedundantProjectsSuiteBase + with EnableAdaptiveExecutionSuite diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index af9088003f3b0..5e6808eeba0f6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, import org.apache.spark.sql.catalyst.expressions.{Ascending, AttributeReference, Concat, SortOrder} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.datasources.{CreateTable, RefreshResource} +import org.apache.spark.sql.execution.datasources.{CreateTable, CreateTempViewUsing, RefreshResource} import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, StaticSQLConf} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} @@ -160,6 +160,15 @@ class SparkSqlParserSuite extends AnalysisTest { intercept("REFRESH", "Resource paths cannot be empty in REFRESH statements") } + test("SPARK-33118 CREATE TMEPORARY TABLE with LOCATION") { + assertEqual("CREATE TEMPORARY TABLE t USING parquet OPTIONS (path '/data/tmp/testspark1')", + CreateTempViewUsing(TableIdentifier("t", None), None, false, false, "parquet", + Map("path" -> "/data/tmp/testspark1"))) + assertEqual("CREATE TEMPORARY TABLE t USING parquet LOCATION '/data/tmp/testspark1'", + CreateTempViewUsing(TableIdentifier("t", None), None, false, false, "parquet", + Map("path" -> "/data/tmp/testspark1"))) + } + private def createTableUsing( table: String, database: Option[String] = None, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 7e7248c312e11..38a323b1c057e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -26,15 +26,19 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent, SparkListe import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession, Strategy} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} -import org.apache.spark.sql.execution.{PartialReducerPartitionSpec, ReusedSubqueryExec, ShuffledRowRDD, SparkPlan} +import org.apache.spark.sql.execution.{PartialReducerPartitionSpec, QueryExecution, ReusedSubqueryExec, ShuffledRowRDD, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.command.DataWritingCommandExec +import org.apache.spark.sql.execution.datasources.noop.NoopDataSource +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.util.QueryExecutionListener import org.apache.spark.util.Utils class AdaptiveQueryExecSuite @@ -842,8 +846,8 @@ class AdaptiveQueryExecSuite withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { val df = sql("SELECT * FROM testData join testData2 ON key = a where value = '1'") val planBefore = df.queryExecution.executedPlan - assert(planBefore.toString.contains("== Current Plan ==")) - assert(planBefore.toString.contains("== Initial Plan ==")) + assert(!planBefore.toString.contains("== Current Plan ==")) + assert(!planBefore.toString.contains("== Initial Plan ==")) df.collect() val planAfter = df.queryExecution.executedPlan assert(planAfter.toString.contains("== Final Plan ==")) @@ -1238,4 +1242,69 @@ class AdaptiveQueryExecSuite } } } + + test("Logging plan changes for AQE") { + val testAppender = new LogAppender("plan changes") + withLogAppender(testAppender) { + withSQLConf( + SQLConf.PLAN_CHANGE_LOG_LEVEL.key -> "INFO", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + sql("SELECT * FROM testData JOIN testData2 ON key = a " + + "WHERE value = (SELECT max(a) FROM testData3)").collect() + } + Seq("=== Result of Batch AQE Preparations ===", + "=== Result of Batch AQE Post Stage Creation ===", + "=== Result of Batch AQE Replanning ===", + "=== Result of Batch AQE Query Stage Optimization ===", + "=== Result of Batch AQE Final Query Stage Optimization ===").foreach { expectedMsg => + assert(testAppender.loggingEvents.exists(_.getRenderedMessage.contains(expectedMsg))) + } + } + } + + test("SPARK-32932: Do not use local shuffle reader at final stage on write command") { + withSQLConf(SQLConf.PARTITION_OVERWRITE_MODE.key -> PartitionOverwriteMode.DYNAMIC.toString, + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val data = for ( + i <- 1L to 10L; + j <- 1L to 3L + ) yield (i, j) + + val df = data.toDF("i", "j").repartition($"j") + var noLocalReader: Boolean = false + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + qe.executedPlan match { + case plan@(_: DataWritingCommandExec | _: V2TableWriteExec) => + assert(plan.asInstanceOf[UnaryExecNode].child.isInstanceOf[AdaptiveSparkPlanExec]) + noLocalReader = collect(plan) { + case exec: CustomShuffleReaderExec if exec.isLocalReader => exec + }.isEmpty + case _ => // ignore other events + } + } + override def onFailure(funcName: String, qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + + withTable("t") { + df.write.partitionBy("j").saveAsTable("t") + sparkContext.listenerBus.waitUntilEmpty() + assert(noLocalReader) + noLocalReader = false + } + + // Test DataSource v2 + val format = classOf[NoopDataSource].getName + df.write.format(format).mode("overwrite").save() + sparkContext.listenerBus.waitUntilEmpty() + assert(noLocalReader) + noLocalReader = false + + spark.listenerManager.unregister(listener) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index b8ac5079b7745..348cf94dfc629 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -1863,6 +1863,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { "Returns the concatenation of col1, col2, ..., colN.") :: Nil ) // extended mode + // scalastyle:off whitespace.end.of.line checkAnswer( sql("DESCRIBE FUNCTION EXTENDED ^"), Row("Class: org.apache.spark.sql.catalyst.expressions.BitwiseXor") :: @@ -1871,11 +1872,14 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { | Examples: | > SELECT 3 ^ 5; | 6 - | """.stripMargin) :: + | + | Since: 1.4.0 + |""".stripMargin) :: Row("Function: ^") :: Row("Usage: expr1 ^ expr2 - Returns the result of " + "bitwise exclusive OR of `expr1` and `expr2`.") :: Nil ) + // scalastyle:on whitespace.end.of.line } test("create a data source table without schema") { @@ -3118,6 +3122,9 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val fs = tablePath.getFileSystem(hadoopConf) val trashCurrent = new Path(fs.getHomeDirectory, ".Trash/Current") val trashPath = Path.mergePaths(trashCurrent, tablePath) + assume( + fs.mkdirs(trashPath) && fs.delete(trashPath, false), + "Trash directory could not be created, skipping.") assert(!fs.exists(trashPath)) try { hadoopConf.set(trashIntervalKey, "5") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index a808546745817..dfd9ba03f5be0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -549,17 +549,22 @@ class FileSourceStrategySuite extends QueryTest with SharedSparkSession with Pre assert(table.rdd.partitions.length == 3) } - withSQLConf(SQLConf.FILES_MIN_PARTITION_NUM.key -> "16") { - val partitions = (1 to 100).map(i => s"file$i" -> 128 * 1024 * 1024) - val table = createTable(files = partitions) - // partition is limited by filesMaxPartitionBytes(128MB) - assert(table.rdd.partitions.length == 100) - } + withSQLConf( + SQLConf.FILES_MAX_PARTITION_BYTES.key -> "2MB", + SQLConf.FILES_OPEN_COST_IN_BYTES.key -> String.valueOf(4 * 1024 * 1024)) { + + withSQLConf(SQLConf.FILES_MIN_PARTITION_NUM.key -> "8") { + val partitions = (1 to 12).map(i => s"file$i" -> 2 * 1024 * 1024) + val table = createTable(files = partitions) + // partition is limited by filesMaxPartitionBytes(2MB) + assert(table.rdd.partitions.length == 12) + } - withSQLConf(SQLConf.FILES_MIN_PARTITION_NUM.key -> "32") { - val partitions = (1 to 800).map(i => s"file$i" -> 4 * 1024 * 1024) - val table = createTable(files = partitions) - assert(table.rdd.partitions.length == 50) + withSQLConf(SQLConf.FILES_MIN_PARTITION_NUM.key -> "16") { + val partitions = (1 to 12).map(i => s"file$i" -> 4 * 1024 * 1024) + val table = createTable(files = partitions) + assert(table.rdd.partitions.length == 24) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuite.scala index ff5fe4f620a1d..0e9498b2681e2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuite.scala @@ -19,27 +19,55 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection import javax.security.auth.login.Configuration -class ConnectionProviderSuite extends ConnectionProviderSuiteBase { +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +class ConnectionProviderSuite extends ConnectionProviderSuiteBase with SharedSparkSession { + test("All built-in providers must be loaded") { + IntentionallyFaultyConnectionProvider.constructed = false + val providers = ConnectionProvider.loadProviders() + assert(providers.exists(_.isInstanceOf[BasicConnectionProvider])) + assert(providers.exists(_.isInstanceOf[DB2ConnectionProvider])) + assert(providers.exists(_.isInstanceOf[MariaDBConnectionProvider])) + assert(providers.exists(_.isInstanceOf[MSSQLConnectionProvider])) + assert(providers.exists(_.isInstanceOf[PostgresConnectionProvider])) + assert(providers.exists(_.isInstanceOf[OracleConnectionProvider])) + assert(IntentionallyFaultyConnectionProvider.constructed) + assert(!providers.exists(_.isInstanceOf[IntentionallyFaultyConnectionProvider])) + assert(providers.size === 6) + } + + test("Disabled provider must not be loaded") { + withSQLConf(SQLConf.DISABLED_JDBC_CONN_PROVIDER_LIST.key -> "db2") { + val providers = ConnectionProvider.loadProviders() + assert(!providers.exists(_.isInstanceOf[DB2ConnectionProvider])) + assert(providers.size === 5) + } + } + test("Multiple security configs must be reachable") { Configuration.setConfiguration(null) - val postgresDriver = registerDriver(PostgresConnectionProvider.driverClass) - val postgresProvider = new PostgresConnectionProvider( - postgresDriver, options("jdbc:postgresql://localhost/postgres")) - val db2Driver = registerDriver(DB2ConnectionProvider.driverClass) - val db2Provider = new DB2ConnectionProvider(db2Driver, options("jdbc:db2://localhost/db2")) + val postgresProvider = new PostgresConnectionProvider() + val postgresDriver = registerDriver(postgresProvider.driverClass) + val postgresOptions = options("jdbc:postgresql://localhost/postgres") + val postgresAppEntry = postgresProvider.appEntry(postgresDriver, postgresOptions) + val db2Provider = new DB2ConnectionProvider() + val db2Driver = registerDriver(db2Provider.driverClass) + val db2Options = options("jdbc:db2://localhost/db2") + val db2AppEntry = db2Provider.appEntry(db2Driver, db2Options) // Make sure no authentication for the databases are set val oldConfig = Configuration.getConfiguration - assert(oldConfig.getAppConfigurationEntry(postgresProvider.appEntry) == null) - assert(oldConfig.getAppConfigurationEntry(db2Provider.appEntry) == null) + assert(oldConfig.getAppConfigurationEntry(postgresAppEntry) == null) + assert(oldConfig.getAppConfigurationEntry(db2AppEntry) == null) - postgresProvider.setAuthenticationConfigIfNeeded() - db2Provider.setAuthenticationConfigIfNeeded() + postgresProvider.setAuthenticationConfigIfNeeded(postgresDriver, postgresOptions) + db2Provider.setAuthenticationConfigIfNeeded(db2Driver, db2Options) // Make sure authentication for the databases are set val newConfig = Configuration.getConfiguration assert(oldConfig != newConfig) - assert(newConfig.getAppConfigurationEntry(postgresProvider.appEntry) != null) - assert(newConfig.getAppConfigurationEntry(db2Provider.appEntry) != null) + assert(newConfig.getAppConfigurationEntry(postgresAppEntry) != null) + assert(newConfig.getAppConfigurationEntry(db2AppEntry) != null) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala index d18a3088c4f2f..a299841b3c149 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala @@ -42,7 +42,7 @@ abstract class ConnectionProviderSuiteBase extends SparkFunSuite with BeforeAndA JDBCOptions.JDBC_PRINCIPAL -> "principal" )) - override def afterEach(): Unit = { + protected override def afterEach(): Unit = { try { Configuration.setConfiguration(null) } finally { @@ -50,20 +50,25 @@ abstract class ConnectionProviderSuiteBase extends SparkFunSuite with BeforeAndA } } - protected def testSecureConnectionProvider(provider: SecureConnectionProvider): Unit = { + protected def testSecureConnectionProvider( + provider: SecureConnectionProvider, + driver: Driver, + options: JDBCOptions): Unit = { + val providerAppEntry = provider.appEntry(driver, options) + // Make sure no authentication for the database is set - assert(Configuration.getConfiguration.getAppConfigurationEntry(provider.appEntry) == null) + assert(Configuration.getConfiguration.getAppConfigurationEntry(providerAppEntry) == null) // Make sure the first call sets authentication properly val savedConfig = Configuration.getConfiguration - provider.setAuthenticationConfigIfNeeded() + provider.setAuthenticationConfigIfNeeded(driver, options) val config = Configuration.getConfiguration assert(savedConfig != config) - val appEntry = config.getAppConfigurationEntry(provider.appEntry) + val appEntry = config.getAppConfigurationEntry(providerAppEntry) assert(appEntry != null) // Make sure a second call is not modifying the existing authentication - provider.setAuthenticationConfigIfNeeded() - assert(config.getAppConfigurationEntry(provider.appEntry) === appEntry) + provider.setAuthenticationConfigIfNeeded(driver, options) + assert(config.getAppConfigurationEntry(providerAppEntry) === appEntry) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProviderSuite.scala index d656f83e2ebb9..5885af82532d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProviderSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection class DB2ConnectionProviderSuite extends ConnectionProviderSuiteBase { test("setAuthenticationConfigIfNeeded must set authentication if not set") { - val driver = registerDriver(DB2ConnectionProvider.driverClass) - val provider = new DB2ConnectionProvider(driver, options("jdbc:db2://localhost/db2")) + val provider = new DB2ConnectionProvider() + val driver = registerDriver(provider.driverClass) - testSecureConnectionProvider(provider) + testSecureConnectionProvider(provider, driver, options("jdbc:db2://localhost/db2")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/IntentionallyFaultyConnectionProvider.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/IntentionallyFaultyConnectionProvider.scala new file mode 100644 index 0000000000000..329d79cae62e8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/IntentionallyFaultyConnectionProvider.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.jdbc.connection + +import java.sql.{Connection, Driver} + +import org.apache.spark.sql.jdbc.JdbcConnectionProvider + +private class IntentionallyFaultyConnectionProvider extends JdbcConnectionProvider { + IntentionallyFaultyConnectionProvider.constructed = true + throw new IllegalArgumentException("Intentional Exception") + override val name: String = "IntentionallyFaultyConnectionProvider" + override def canHandle(driver: Driver, options: Map[String, String]): Boolean = true + override def getConnection(driver: Driver, options: Map[String, String]): Connection = null +} + +private object IntentionallyFaultyConnectionProvider { + var constructed = false +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala index 249f1e36347ed..a5704e842e018 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala @@ -17,35 +17,35 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection +import java.sql.Driver + +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions + class MSSQLConnectionProviderSuite extends ConnectionProviderSuiteBase { test("setAuthenticationConfigIfNeeded default parser must set authentication if not set") { - val driver = registerDriver(MSSQLConnectionProvider.driverClass) - val defaultProvider = new MSSQLConnectionProvider( - driver, options("jdbc:sqlserver://localhost/mssql")) - val customProvider = new MSSQLConnectionProvider( - driver, options("jdbc:sqlserver://localhost/mssql;jaasConfigurationName=custommssql")) + val provider = new MSSQLConnectionProvider() + val driver = registerDriver(provider.driverClass) - testProviders(defaultProvider, customProvider) + testProviders(driver, provider, options("jdbc:sqlserver://localhost/mssql"), + options("jdbc:sqlserver://localhost/mssql;jaasConfigurationName=custommssql")) } test("setAuthenticationConfigIfNeeded custom parser must set authentication if not set") { - val parserMethod = "IntentionallyNotExistingMethod" - val driver = registerDriver(MSSQLConnectionProvider.driverClass) - val defaultProvider = new MSSQLConnectionProvider( - driver, options("jdbc:sqlserver://localhost/mssql"), parserMethod) - val customProvider = new MSSQLConnectionProvider( - driver, - options("jdbc:sqlserver://localhost/mssql;jaasConfigurationName=custommssql"), - parserMethod) - - testProviders(defaultProvider, customProvider) + val provider = new MSSQLConnectionProvider() { + override val parserMethod: String = "IntentionallyNotExistingMethod" + } + val driver = registerDriver(provider.driverClass) + + testProviders(driver, provider, options("jdbc:sqlserver://localhost/mssql"), + options("jdbc:sqlserver://localhost/mssql;jaasConfigurationName=custommssql")) } private def testProviders( - defaultProvider: SecureConnectionProvider, - customProvider: SecureConnectionProvider) = { - assert(defaultProvider.appEntry !== customProvider.appEntry) - testSecureConnectionProvider(defaultProvider) - testSecureConnectionProvider(customProvider) + driver: Driver, + provider: SecureConnectionProvider, + defaultOptions: JDBCOptions, + customOptions: JDBCOptions) = { + testSecureConnectionProvider(provider, driver, defaultOptions) + testSecureConnectionProvider(provider, driver, customOptions) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala index 70cad2097eb43..f450662fcbe74 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection class MariaDBConnectionProviderSuite extends ConnectionProviderSuiteBase { test("setAuthenticationConfigIfNeeded must set authentication if not set") { - val driver = registerDriver(MariaDBConnectionProvider.driverClass) - val provider = new MariaDBConnectionProvider(driver, options("jdbc:mysql://localhost/mysql")) + val provider = new MariaDBConnectionProvider() + val driver = registerDriver(provider.driverClass) - testSecureConnectionProvider(provider) + testSecureConnectionProvider(provider, driver, options("jdbc:mysql://localhost/mysql")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala index 13cde32ddbe4e..40e7f1191dccc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala @@ -19,10 +19,9 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection class OracleConnectionProviderSuite extends ConnectionProviderSuiteBase { test("setAuthenticationConfigIfNeeded must set authentication if not set") { - val driver = registerDriver(OracleConnectionProvider.driverClass) - val provider = new OracleConnectionProvider(driver, - options("jdbc:oracle:thin:@//localhost/xe")) + val provider = new OracleConnectionProvider() + val driver = registerDriver(provider.driverClass) - testSecureConnectionProvider(provider) + testSecureConnectionProvider(provider, driver, options("jdbc:oracle:thin:@//localhost/xe")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala index 8cef7652f9c54..ee43a7d9708c5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala @@ -19,14 +19,14 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection class PostgresConnectionProviderSuite extends ConnectionProviderSuiteBase { test("setAuthenticationConfigIfNeeded must set authentication if not set") { - val driver = registerDriver(PostgresConnectionProvider.driverClass) - val defaultProvider = new PostgresConnectionProvider( - driver, options("jdbc:postgresql://localhost/postgres")) - val customProvider = new PostgresConnectionProvider( - driver, options(s"jdbc:postgresql://localhost/postgres?jaasApplicationName=custompgjdbc")) + val provider = new PostgresConnectionProvider() + val defaultOptions = options("jdbc:postgresql://localhost/postgres") + val customOptions = + options(s"jdbc:postgresql://localhost/postgres?jaasApplicationName=custompgjdbc") + val driver = registerDriver(provider.driverClass) - assert(defaultProvider.appEntry !== customProvider.appEntry) - testSecureConnectionProvider(defaultProvider) - testSecureConnectionProvider(customProvider) + assert(provider.appEntry(driver, defaultOptions) !== provider.appEntry(driver, customOptions)) + testSecureConnectionProvider(provider, driver, defaultOptions) + testSecureConnectionProvider(provider, driver, customOptions) } } diff --git a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala similarity index 98% rename from sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index afc83d7c395f0..681ed91afaa12 100644 --- a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -39,10 +39,6 @@ import org.apache.spark.sql.types._ /** * A test suite that tests Apache ORC filter API based filter pushdown optimization. - * OrcFilterSuite and HiveOrcFilterSuite is logically duplicated to provide the same test coverage. - * The difference are the packages containing 'Predicate' and 'SearchArgument' classes. - * - OrcFilterSuite uses 'org.apache.orc.storage.ql.io.sarg' package. - * - HiveOrcFilterSuite uses 'org.apache.hadoop.hive.ql.io.sarg' package. */ class OrcFilterSuite extends OrcTest with SharedSparkSession { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index d2970ef1bb63d..ead2c2cf1b70f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -217,7 +217,6 @@ abstract class OrcQueryTest extends OrcTest { } } - // Hive supports zlib, snappy and none for Hive 1.2.1. test("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") { withTempPath { file => spark.range(0, 10).write diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index b70fd7476ed98..1242b8c693d64 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -32,7 +32,7 @@ import org.apache.orc.impl.RecordReaderImpl import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SPARK_VERSION_SHORT, SparkException} -import org.apache.spark.sql.{Row, SPARK_VERSION_METADATA_KEY} +import org.apache.spark.sql.{FakeFileSystemRequiringDSOption, Row, SPARK_VERSION_METADATA_KEY} import org.apache.spark.sql.execution.datasources.SchemaMergeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -120,8 +120,7 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { } } - protected def testSelectiveDictionaryEncoding(isSelective: Boolean, - isHive23: Boolean = false): Unit = { + protected def testSelectiveDictionaryEncoding(isSelective: Boolean, isHiveOrc: Boolean): Unit = { val tableName = "orcTable" withTempDir { dir => @@ -174,7 +173,7 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { // Hive 0.11 and RLE v2 is introduced in Hive 0.12 ORC with more improvements. // For more details, see https://orc.apache.org/specification/ assert(stripe.getColumns(1).getKind === DICTIONARY_V2) - if (isSelective || isHive23) { + if (isSelective || isHiveOrc) { assert(stripe.getColumns(2).getKind === DIRECT_V2) } else { assert(stripe.getColumns(2).getKind === DICTIONARY_V2) @@ -538,6 +537,21 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { } } } + + test("SPARK-33094: should propagate Hadoop config from DS options to underlying file system") { + withSQLConf( + "fs.file.impl" -> classOf[FakeFileSystemRequiringDSOption].getName, + "fs.file.impl.disable.cache" -> "true") { + Seq(false, true).foreach { mergeSchema => + withTempPath { dir => + val path = dir.getAbsolutePath + val conf = Map("ds_option" -> "value", "mergeSchema" -> mergeSchema.toString) + spark.range(1).write.options(conf).orc(path) + checkAnswer(spark.read.options(conf).orc(path), Row(0)) + } + } + } + } } class OrcSourceSuite extends OrcSuite with SharedSparkSession { @@ -581,7 +595,7 @@ class OrcSourceSuite extends OrcSuite with SharedSparkSession { } test("Enforce direct encoding column-wise selectively") { - testSelectiveDictionaryEncoding(isSelective = true) + testSelectiveDictionaryEncoding(isSelective = true, isHiveOrc = false) } test("SPARK-11412 read and merge orc schemas in parallel") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala index aec61acda5444..4243318ac1dd8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala @@ -46,7 +46,6 @@ import org.apache.spark.sql.internal.SQLConf.ORC_IMPLEMENTATION * -> OrcPartitionDiscoverySuite * -> HiveOrcPartitionDiscoverySuite * -> OrcFilterSuite - * -> HiveOrcFilterSuite */ abstract class OrcTest extends QueryTest with FileBasedDataSourceTest with BeforeAndAfterAll { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala index b308934ba03c0..209f5609e447f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala @@ -20,7 +20,9 @@ import java.sql.{Connection, DriverManager} import java.util.Properties import org.apache.spark.SparkConf -import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -63,6 +65,8 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { test("show tables") { checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "people"))) + // Check not existing namespace + checkAnswer(sql("SHOW TABLES IN h2.bad_test"), Seq()) } test("drop a table and test whether the table exists") { @@ -72,6 +76,15 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "to_drop"), Row("test", "people"))) sql("DROP TABLE h2.test.to_drop") checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "people"))) + Seq( + "h2.test.not_existing_table" -> "Table test.not_existing_table not found", + "h2.bad_test.not_existing_table" -> "Table bad_test.not_existing_table not found" + ).foreach { case (table, expectedMsg) => + val msg = intercept[NoSuchTableException] { + sql(s"DROP TABLE $table") + }.getMessage + assert(msg.contains(expectedMsg)) + } } test("rename a table") { @@ -87,6 +100,36 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { sql("SHOW TABLES IN h2.test"), Seq(Row("test", "dst_table"), Row("test", "people"))) } + // Rename not existing table or namespace + val exp1 = intercept[NoSuchTableException] { + sql(s"ALTER TABLE h2.test.not_existing_table RENAME TO test.dst_table") + } + assert(exp1.getMessage.contains( + "Failed table renaming from test.not_existing_table to test.dst_table")) + assert(exp1.cause.get.getMessage.contains("Table \"not_existing_table\" not found")) + val exp2 = intercept[NoSuchNamespaceException] { + sql(s"ALTER TABLE h2.bad_test.not_existing_table RENAME TO test.dst_table") + } + assert(exp2.getMessage.contains( + "Failed table renaming from bad_test.not_existing_table to test.dst_table")) + assert(exp2.cause.get.getMessage.contains("Schema \"bad_test\" not found")) + // Rename to an existing table + withTable("h2.test.dst_table") { + withConnection { conn => + conn.prepareStatement("""CREATE TABLE "test"."dst_table" (id INTEGER)""").executeUpdate() + } + withTable("h2.test.src_table") { + withConnection { conn => + conn.prepareStatement("""CREATE TABLE "test"."src_table" (id INTEGER)""").executeUpdate() + } + val exp = intercept[TableAlreadyExistsException] { + sql("ALTER TABLE h2.test.src_table RENAME TO test.dst_table") + } + assert(exp.getMessage.contains( + "Failed table renaming from test.src_table to test.dst_table")) + assert(exp.cause.get.getMessage.contains("Table \"dst_table\" already exists")) + } + } } test("load a table") { @@ -95,6 +138,12 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { .add("NAME", StringType) .add("ID", IntegerType) assert(t.schema === expectedSchema) + Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => + val msg = intercept[AnalysisException] { + spark.table(table).schema + }.getMessage + assert(msg.contains("Table or view not found")) + } } test("create a table") { @@ -105,6 +154,18 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { sql("SHOW TABLES IN h2.test"), Seq(Row("test", "people"), Row("test", "new_table"))) } + withTable("h2.test.new_table") { + sql("CREATE TABLE h2.test.new_table(i INT, j STRING) USING _") + val msg = intercept[AnalysisException] { + sql("CREATE TABLE h2.test.new_table(i INT, j STRING) USING _") + }.getMessage + assert(msg.contains("Table test.new_table already exists")) + } + val exp = intercept[NoSuchNamespaceException] { + sql("CREATE TABLE h2.bad_test.new_table(i INT, j STRING) USING _") + } + assert(exp.getMessage.contains("Failed table creation: bad_test.new_table")) + assert(exp.cause.get.getMessage.contains("Schema \"bad_test\" not found")) } test("alter table ... add column") { @@ -117,60 +178,144 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { .add("C1", IntegerType) .add("C2", StringType) assert(t.schema === expectedSchema) - sql("ALTER TABLE h2.test.alt_table ADD COLUMNS (C3 DOUBLE)") + sql("ALTER TABLE h2.test.alt_table ADD COLUMNS (c3 DOUBLE)") t = spark.table("h2.test.alt_table") - expectedSchema = expectedSchema.add("C3", DoubleType) + expectedSchema = expectedSchema.add("c3", DoubleType) assert(t.schema === expectedSchema) + // Add already existing column + val msg = intercept[AnalysisException] { + sql("ALTER TABLE h2.test.alt_table ADD COLUMNS (c3 DOUBLE)") + }.getMessage + assert(msg.contains("Cannot add column, because c3 already exists")) + } + // Add a column to not existing table and namespace + Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => + val msg = intercept[AnalysisException] { + sql(s"ALTER TABLE $table ADD COLUMNS (C4 STRING)") + }.getMessage + assert(msg.contains("Table not found")) } } test("alter table ... rename column") { withTable("h2.test.alt_table") { - sql("CREATE TABLE h2.test.alt_table (ID INTEGER) USING _") - sql("ALTER TABLE h2.test.alt_table RENAME COLUMN ID TO C") + sql("CREATE TABLE h2.test.alt_table (id INTEGER, C0 INTEGER) USING _") + sql("ALTER TABLE h2.test.alt_table RENAME COLUMN id TO C") val t = spark.table("h2.test.alt_table") - val expectedSchema = new StructType().add("C", IntegerType) + val expectedSchema = new StructType() + .add("C", IntegerType) + .add("C0", IntegerType) assert(t.schema === expectedSchema) + // Rename to already existing column + val msg = intercept[AnalysisException] { + sql("ALTER TABLE h2.test.alt_table RENAME COLUMN C TO C0") + }.getMessage + assert(msg.contains("Cannot rename column, because C0 already exists")) + } + // Rename a column in not existing table and namespace + Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => + val msg = intercept[AnalysisException] { + sql(s"ALTER TABLE $table RENAME COLUMN ID TO C") + }.getMessage + assert(msg.contains("Table not found")) } } test("alter table ... drop column") { withTable("h2.test.alt_table") { - sql("CREATE TABLE h2.test.alt_table (C1 INTEGER, C2 INTEGER) USING _") + sql("CREATE TABLE h2.test.alt_table (C1 INTEGER, C2 INTEGER, c3 INTEGER) USING _") sql("ALTER TABLE h2.test.alt_table DROP COLUMN C1") + sql("ALTER TABLE h2.test.alt_table DROP COLUMN c3") val t = spark.table("h2.test.alt_table") val expectedSchema = new StructType().add("C2", IntegerType) assert(t.schema === expectedSchema) + // Drop not existing column + val msg = intercept[AnalysisException] { + sql("ALTER TABLE h2.test.alt_table DROP COLUMN bad_column") + }.getMessage + assert(msg.contains("Cannot delete missing field bad_column in test.alt_table schema")) + } + // Drop a column to not existing table and namespace + Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => + val msg = intercept[AnalysisException] { + sql(s"ALTER TABLE $table DROP COLUMN C1") + }.getMessage + assert(msg.contains("Table not found")) } } test("alter table ... update column type") { withTable("h2.test.alt_table") { - sql("CREATE TABLE h2.test.alt_table (ID INTEGER) USING _") + sql("CREATE TABLE h2.test.alt_table (ID INTEGER, deptno INTEGER) USING _") sql("ALTER TABLE h2.test.alt_table ALTER COLUMN id TYPE DOUBLE") + sql("ALTER TABLE h2.test.alt_table ALTER COLUMN deptno TYPE DOUBLE") val t = spark.table("h2.test.alt_table") - val expectedSchema = new StructType().add("ID", DoubleType) + val expectedSchema = new StructType().add("ID", DoubleType).add("deptno", DoubleType) assert(t.schema === expectedSchema) + // Update not existing column + val msg1 = intercept[AnalysisException] { + sql("ALTER TABLE h2.test.alt_table ALTER COLUMN bad_column TYPE DOUBLE") + }.getMessage + assert(msg1.contains("Cannot update missing field bad_column in test.alt_table schema")) + // Update column to wrong type + val msg2 = intercept[ParseException] { + sql("ALTER TABLE h2.test.alt_table ALTER COLUMN id TYPE bad_type") + }.getMessage + assert(msg2.contains("DataType bad_type is not supported")) + } + // Update column type in not existing table and namespace + Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => + val msg = intercept[AnalysisException] { + sql(s"ALTER TABLE $table ALTER COLUMN id TYPE DOUBLE") + }.getMessage + assert(msg.contains("Table not found")) } } test("alter table ... update column nullability") { withTable("h2.test.alt_table") { - sql("CREATE TABLE h2.test.alt_table (ID INTEGER NOT NULL) USING _") + sql("CREATE TABLE h2.test.alt_table (ID INTEGER NOT NULL, deptno INTEGER NOT NULL) USING _") sql("ALTER TABLE h2.test.alt_table ALTER COLUMN ID DROP NOT NULL") + sql("ALTER TABLE h2.test.alt_table ALTER COLUMN deptno DROP NOT NULL") val t = spark.table("h2.test.alt_table") - val expectedSchema = new StructType().add("ID", IntegerType, nullable = true) + val expectedSchema = new StructType() + .add("ID", IntegerType, nullable = true).add("deptno", IntegerType, nullable = true) assert(t.schema === expectedSchema) + // Update nullability of not existing column + val msg = intercept[AnalysisException] { + sql("ALTER TABLE h2.test.alt_table ALTER COLUMN bad_column DROP NOT NULL") + }.getMessage + assert(msg.contains("Cannot update missing field bad_column in test.alt_table")) + } + // Update column nullability in not existing table and namespace + Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => + val msg = intercept[AnalysisException] { + sql(s"ALTER TABLE $table ALTER COLUMN ID DROP NOT NULL") + }.getMessage + assert(msg.contains("Table not found")) } } test("alter table ... update column comment not supported") { withTable("h2.test.alt_table") { sql("CREATE TABLE h2.test.alt_table (ID INTEGER) USING _") - val thrown = intercept[java.sql.SQLFeatureNotSupportedException] { + val exp = intercept[AnalysisException] { sql("ALTER TABLE h2.test.alt_table ALTER COLUMN ID COMMENT 'test'") } - assert(thrown.getMessage.contains("Unsupported TableChange")) + assert(exp.getMessage.contains("Failed table altering: test.alt_table")) + assert(exp.cause.get.getMessage.contains("Unsupported TableChange")) + // Update comment for not existing column + val msg = intercept[AnalysisException] { + sql("ALTER TABLE h2.test.alt_table ALTER COLUMN bad_column COMMENT 'test'") + }.getMessage + assert(msg.contains("Cannot update missing field bad_column in test.alt_table")) + } + // Update column comments in not existing table and namespace + Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => + val msg = intercept[AnalysisException] { + sql(s"ALTER TABLE $table ALTER COLUMN ID COMMENT 'test'") + }.getMessage + assert(msg.contains("Table not found")) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala new file mode 100644 index 0000000000000..38e68cd2512e7 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.exchange + +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.catalyst.plans.Inner +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection} +import org.apache.spark.sql.execution.{DummySparkPlan, SortExec} +import org.apache.spark.sql.execution.joins.SortMergeJoinExec +import org.apache.spark.sql.test.SharedSparkSession + +class EnsureRequirementsSuite extends SharedSparkSession { + private val exprA = Literal(1) + private val exprB = Literal(2) + private val exprC = Literal(3) + + test("reorder should handle PartitioningCollection") { + val plan1 = DummySparkPlan( + outputPartitioning = PartitioningCollection(Seq( + HashPartitioning(exprA :: exprB :: Nil, 5), + HashPartitioning(exprA :: Nil, 5)))) + val plan2 = DummySparkPlan() + + // Test PartitioningCollection on the left side of join. + val smjExec1 = SortMergeJoinExec( + exprB :: exprA :: Nil, exprA :: exprB :: Nil, Inner, None, plan1, plan2) + EnsureRequirements(spark.sessionState.conf).apply(smjExec1) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: PartitioningCollection, _, _), _), + SortExec(_, _, ShuffleExchangeExec(_: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB)) + assert(rightKeys === Seq(exprB, exprA)) + case other => fail(other.toString) + } + + // Test PartitioningCollection on the right side of join. + val smjExec2 = SortMergeJoinExec( + exprA :: exprB :: Nil, exprB :: exprA :: Nil, Inner, None, plan2, plan1) + EnsureRequirements(spark.sessionState.conf).apply(smjExec2) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, ShuffleExchangeExec(_: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, _: PartitioningCollection, _, _), _), _) => + assert(leftKeys === Seq(exprB, exprA)) + assert(rightKeys === Seq(exprA, exprB)) + case other => fail(other.toString) + } + + // Both sides are PartitioningCollection, but left side cannot be reorderd to match + // and it should fall back to the right side. + val smjExec3 = SortMergeJoinExec( + exprA :: exprC :: Nil, exprB :: exprA :: Nil, Inner, None, plan1, plan1) + EnsureRequirements(spark.sessionState.conf).apply(smjExec3) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, ShuffleExchangeExec(_: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, _: PartitioningCollection, _, _), _), _) => + assert(leftKeys === Seq(exprC, exprA)) + assert(rightKeys === Seq(exprA, exprB)) + case other => fail(other.toString) + } + } + + test("reorder should fallback to the other side partitioning") { + val plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprB :: exprC :: Nil, 5)) + val plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprB :: exprC :: Nil, 5)) + + // Test fallback to the right side, which has HashPartitioning. + val smjExec1 = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprB :: Nil, Inner, None, plan1, plan2) + EnsureRequirements(spark.sessionState.conf).apply(smjExec1) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, ShuffleExchangeExec(_: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprB, exprA)) + assert(rightKeys === Seq(exprB, exprC)) + case other => fail(other.toString) + } + + // Test fallback to the right side, which has PartitioningCollection. + val plan3 = DummySparkPlan( + outputPartitioning = PartitioningCollection(Seq(HashPartitioning(exprB :: exprC :: Nil, 5)))) + val smjExec2 = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprB :: Nil, Inner, None, plan1, plan3) + EnsureRequirements(spark.sessionState.conf).apply(smjExec2) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, ShuffleExchangeExec(_: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, _: PartitioningCollection, _, _), _), _) => + assert(leftKeys === Seq(exprB, exprA)) + assert(rightKeys === Seq(exprB, exprC)) + case other => fail(other.toString) + } + + // The right side has HashPartitioning, so it is matched first, but no reordering match is + // found, and it should fall back to the left side, which has a PartitioningCollection. + val smjExec3 = SortMergeJoinExec( + exprC :: exprB :: Nil, exprA :: exprB :: Nil, Inner, None, plan3, plan1) + EnsureRequirements(spark.sessionState.conf).apply(smjExec3) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: PartitioningCollection, _, _), _), + SortExec(_, _, ShuffleExchangeExec(_: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprB, exprC)) + assert(rightKeys === Seq(exprB, exprA)) + case other => fail(other.toString) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 4e10c27edb0e9..4872906dbfec3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.execution.{FilterExec, RangeExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.aggregate.HashAggregateExec -import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -736,4 +736,23 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils Map("dataSize" -> 3200, "shuffleRecordsWritten" -> 100)) testMetricsInSparkPlanOperator(exchanges(1), Map("dataSize" -> 0, "shuffleRecordsWritten" -> 0)) } + + test("Add numRows to metric of BroadcastExchangeExec") { + withSQLConf(SQLConf.AUTO_SIZE_UPDATE_ENABLED.key -> "true") { + withTable("t1", "t2") { + spark.range(2).write.saveAsTable("t1") + spark.range(2).write.saveAsTable("t2") + val df = sql("SELECT t1.* FROM t1 JOIN t2 ON t1.id = t2.id") + df.collect() + val plan = df.queryExecution.executedPlan + + val exchanges = plan.collect { + case s: BroadcastExchangeExec => s + } + + assert(exchanges.size === 1) + testMetricsInSparkPlanOperator(exchanges.head, Map("numOutputRows" -> 2)) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index f49a3a384b450..00f23718a0e9e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -680,7 +680,7 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils assert(sparkPlanInfo.nodeName === "WholeStageCodegen (2)") } - test("SPARK-32615: SQLMetrics validation after sparkPlanInfo updated in AQE") { + test("SPARK-32615,SPARK-33016: SQLMetrics validation after sparkPlanInfo updated in AQE") { val statusStore = createStatusStore() val listener = statusStore.listener.get @@ -755,7 +755,7 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils .allNodes.flatMap(_.metrics.map(_.accumulatorId)) // Assume that AQE update sparkPlanInfo with newPlan - // ExecutionMetrics will be replaced using newPlan's SQLMetrics + // ExecutionMetrics will be appended using newPlan's SQLMetrics listener.onOtherEvent(SparkListenerSQLAdaptiveExecutionUpdate( executionId, "test", @@ -770,8 +770,8 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(1, 0))) listener.onTaskStart(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0))) - // live metrics will be override, and ExecutionMetrics should be empty as the newPlan updated. - assert(statusStore.executionMetrics(executionId).isEmpty) + // historical metrics will be kept despite of the newPlan updated. + assert(statusStore.executionMetrics(executionId).size == 2) // update new metrics with Id 4 & 5, since 3 is timing metrics, // timing metrics has a complicated string presentation so we don't test it here. @@ -780,9 +780,9 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils (0L, 1, 0, createAccumulatorInfos(newMetricsValueMap)) ))) - assert(statusStore.executionMetrics(executionId).size == 2) + assert(statusStore.executionMetrics(executionId).size == 4) statusStore.executionMetrics(executionId).foreach { m => - assert(m._2 == "500") + assert(m._2 == "100" || m._2 == "500") } listener.onTaskEnd(SparkListenerTaskEnd( @@ -802,10 +802,10 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils JobSucceeded )) - // aggregateMetrics should ignore metrics from job 0 + // aggregateMetrics should contains all metrics from job 0 and job 1 val aggregateMetrics = listener.liveExecutionMetrics(executionId) if (aggregateMetrics.isDefined) { - oldAccumulatorIds.foreach(id => assert(!aggregateMetrics.get.contains(id))) + assert(aggregateMetrics.get.keySet.size == 4) } listener.onOtherEvent(SparkListenerSQLExecutionEnd( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala index 53f9757750735..9f62ff8301ebc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala @@ -105,11 +105,37 @@ class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { } } + test("SPARK-32870: Default expressions in FunctionRegistry should have their " + + "usage, examples and since filled") { + val ignoreSet = Set( + // Explicitly inherits NonSQLExpression, and has no ExpressionDescription + "org.apache.spark.sql.catalyst.expressions.TimeWindow", + // Cast aliases do not need examples + "org.apache.spark.sql.catalyst.expressions.Cast") + + spark.sessionState.functionRegistry.listFunction().foreach { funcId => + val info = spark.sessionState.catalog.lookupFunctionInfo(funcId) + if (!ignoreSet.contains(info.getClassName)) { + withClue(s"Function '${info.getName}', Expression class '${info.getClassName}'") { + assert(info.getUsage.nonEmpty) + assert(info.getExamples.startsWith("\n Examples:\n")) + assert(info.getExamples.endsWith("\n ")) + assert(info.getSince.matches("[0-9]+\\.[0-9]+\\.[0-9]+")) + + if (info.getArguments.nonEmpty) { + assert(info.getArguments.startsWith("\n Arguments:\n")) + assert(info.getArguments.endsWith("\n ")) + } + } + } + } + } + test("check outputs of expression examples") { def unindentAndTrim(s: String): String = { s.replaceAll("\n\\s+", "\n").trim } - val beginSqlStmtRe = " > ".r + val beginSqlStmtRe = "\n > ".r val endSqlStmtRe = ";\n".r def checkExampleSyntax(example: String): Unit = { val beginStmtNum = beginSqlStmtRe.findAllIn(example).length @@ -129,8 +155,17 @@ class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { "org.apache.spark.sql.catalyst.expressions.Randn", "org.apache.spark.sql.catalyst.expressions.Shuffle", "org.apache.spark.sql.catalyst.expressions.Uuid", + // Other nondeterministic expressions + "org.apache.spark.sql.catalyst.expressions.MonotonicallyIncreasingID", + "org.apache.spark.sql.catalyst.expressions.SparkPartitionID", + "org.apache.spark.sql.catalyst.expressions.InputFileName", + "org.apache.spark.sql.catalyst.expressions.InputFileBlockStart", + "org.apache.spark.sql.catalyst.expressions.InputFileBlockLength", // The example calls methods that return unstable results. - "org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection") + "org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection", + "org.apache.spark.sql.catalyst.expressions.SparkVersion", + // Throws an error + "org.apache.spark.sql.catalyst.expressions.RaiseError") val parFuncs = new ParVector(spark.sessionState.functionRegistry.listFunction().toVector) parFuncs.foreach { funcId => @@ -164,9 +199,16 @@ class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { val exprTypesToCheck = Seq(classOf[UnaryExpression], classOf[BinaryExpression], classOf[TernaryExpression], classOf[QuaternaryExpression], classOf[SeptenaryExpression]) - // Do not check these expressions, because these expressions extend NullIntolerant - // and override the eval method to avoid evaluating input1 if input2 is 0. - val ignoreSet = Set(classOf[IntegralDivide], classOf[Divide], classOf[Remainder], classOf[Pmod]) + // Do not check these expressions, because these expressions override the eval method + val ignoreSet = Set( + // Extend NullIntolerant and avoid evaluating input1 if input2 is 0 + classOf[IntegralDivide], + classOf[Divide], + classOf[Remainder], + classOf[Pmod], + // Throws an exception, even if input is null + classOf[RaiseError] + ) val candidateExprsToCheck = spark.sessionState.functionRegistry.listFunction() .map(spark.sessionState.catalog.lookupFunctionInfo).map(_.getClassName) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 7af55550a7736..f0b19071a969b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -770,9 +770,14 @@ class JDBCSuite extends QueryTest } test("Dialect unregister") { - JdbcDialects.registerDialect(testH2Dialect) - JdbcDialects.unregisterDialect(testH2Dialect) - assert(JdbcDialects.get(urlWithUserAndPass) == NoopDialect) + JdbcDialects.unregisterDialect(H2Dialect) + try { + JdbcDialects.registerDialect(testH2Dialect) + JdbcDialects.unregisterDialect(testH2Dialect) + assert(JdbcDialects.get(urlWithUserAndPass) == NoopDialect) + } finally { + JdbcDialects.registerDialect(H2Dialect) + } } test("Aggregated dialects") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index 3f621e04338a3..fb46c2ff4c0ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -194,24 +194,29 @@ class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter { } test("Truncate") { - JdbcDialects.registerDialect(testH2Dialect) - val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) - val df2 = spark.createDataFrame(sparkContext.parallelize(arr1x2), schema2) - val df3 = spark.createDataFrame(sparkContext.parallelize(arr2x3), schema3) - - df.write.jdbc(url1, "TEST.TRUNCATETEST", properties) - df2.write.mode(SaveMode.Overwrite).option("truncate", true) - .jdbc(url1, "TEST.TRUNCATETEST", properties) - assert(1 === spark.read.jdbc(url1, "TEST.TRUNCATETEST", properties).count()) - assert(2 === spark.read.jdbc(url1, "TEST.TRUNCATETEST", properties).collect()(0).length) + JdbcDialects.unregisterDialect(H2Dialect) + try { + JdbcDialects.registerDialect(testH2Dialect) + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + val df2 = spark.createDataFrame(sparkContext.parallelize(arr1x2), schema2) + val df3 = spark.createDataFrame(sparkContext.parallelize(arr2x3), schema3) - val m = intercept[AnalysisException] { - df3.write.mode(SaveMode.Overwrite).option("truncate", true) + df.write.jdbc(url1, "TEST.TRUNCATETEST", properties) + df2.write.mode(SaveMode.Overwrite).option("truncate", true) .jdbc(url1, "TEST.TRUNCATETEST", properties) - }.getMessage - assert(m.contains("Column \"seq\" not found")) - assert(0 === spark.read.jdbc(url1, "TEST.TRUNCATETEST", properties).count()) - JdbcDialects.unregisterDialect(testH2Dialect) + assert(1 === spark.read.jdbc(url1, "TEST.TRUNCATETEST", properties).count()) + assert(2 === spark.read.jdbc(url1, "TEST.TRUNCATETEST", properties).collect()(0).length) + + val m = intercept[AnalysisException] { + df3.write.mode(SaveMode.Overwrite).option("truncate", true) + .jdbc(url1, "TEST.TRUNCATETEST", properties) + }.getMessage + assert(m.contains("Column \"seq\" not found")) + assert(0 === spark.read.jdbc(url1, "TEST.TRUNCATETEST", properties).count()) + } finally { + JdbcDialects.unregisterDialect(testH2Dialect) + JdbcDialects.registerDialect(H2Dialect) + } } test("createTableOptions") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala new file mode 100644 index 0000000000000..1c258bc0dadb9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} + +class DisableUnnecessaryBucketedScanWithoutHiveSupportSuite + extends DisableUnnecessaryBucketedScanSuite + with SharedSparkSession { + + protected override def beforeAll(): Unit = { + super.beforeAll() + assert(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") + } +} + +abstract class DisableUnnecessaryBucketedScanSuite extends QueryTest with SQLTestUtils { + import testImplicits._ + + private lazy val df1 = + (0 until 50).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k").as("df1") + private lazy val df2 = + (0 until 50).map(i => (i % 7, i % 11, i.toString)).toDF("i", "j", "k").as("df2") + + private def checkDisableBucketedScan( + query: String, + expectedNumScanWithAutoScanEnabled: Int, + expectedNumScanWithAutoScanDisabled: Int): Unit = { + + def checkNumBucketedScan(query: String, expectedNumBucketedScan: Int): Unit = { + val plan = sql(query).queryExecution.executedPlan + val bucketedScan = plan.collect { case s: FileSourceScanExec if s.bucketedScan => s } + assert(bucketedScan.length == expectedNumBucketedScan) + } + + withSQLConf(SQLConf.AUTO_BUCKETED_SCAN_ENABLED.key -> "true") { + checkNumBucketedScan(query, expectedNumScanWithAutoScanEnabled) + val result = sql(query).collect() + + withSQLConf(SQLConf.AUTO_BUCKETED_SCAN_ENABLED.key -> "false") { + checkNumBucketedScan(query, expectedNumScanWithAutoScanDisabled) + checkAnswer(sql(query), result) + } + } + } + + test("SPARK-32859: disable unnecessary bucketed table scan - basic test") { + withTable("t1", "t2", "t3") { + df1.write.format("parquet").bucketBy(8, "i").saveAsTable("t1") + df2.write.format("parquet").bucketBy(8, "i").saveAsTable("t2") + df2.write.format("parquet").bucketBy(4, "i").saveAsTable("t3") + + Seq( + // Read bucketed table + ("SELECT * FROM t1", 0, 1), + ("SELECT i FROM t1", 0, 1), + ("SELECT j FROM t1", 0, 0), + // Filter on bucketed column + ("SELECT * FROM t1 WHERE i = 1", 1, 1), + // Filter on non-bucketed column + ("SELECT * FROM t1 WHERE j = 1", 0, 1), + // Join with same buckets + ("SELECT /*+ broadcast(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.i", 0, 2), + ("SELECT /*+ shuffle_hash(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.i", 2, 2), + ("SELECT /*+ merge(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.i", 2, 2), + // Join with different buckets + ("SELECT /*+ broadcast(t1)*/ * FROM t1 JOIN t3 ON t1.i = t3.i", 0, 2), + ("SELECT /*+ shuffle_hash(t1)*/ * FROM t1 JOIN t3 ON t1.i = t3.i", 1, 2), + ("SELECT /*+ merge(t1)*/ * FROM t1 JOIN t3 ON t1.i = t3.i", 1, 2), + // Join on non-bucketed column + ("SELECT /*+ broadcast(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.j", 0, 2), + ("SELECT /*+ shuffle_hash(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.j", 1, 2), + ("SELECT /*+ merge(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.j", 1, 2), + ("SELECT /*+ broadcast(t1)*/ * FROM t1 JOIN t2 ON t1.j = t2.j", 0, 2), + ("SELECT /*+ shuffle_hash(t1)*/ * FROM t1 JOIN t2 ON t1.j = t2.j", 0, 2), + ("SELECT /*+ merge(t1)*/ * FROM t1 JOIN t2 ON t1.j = t2.j", 0, 2), + // Aggregate on bucketed column + ("SELECT SUM(i) FROM t1 GROUP BY i", 1, 1), + // Aggregate on non-bucketed column + ("SELECT SUM(i) FROM t1 GROUP BY j", 0, 1), + ("SELECT j, SUM(i), COUNT(j) FROM t1 GROUP BY j", 0, 1) + ).foreach { case (query, numScanWithAutoScanEnabled, numScanWithAutoScanDisabled) => + checkDisableBucketedScan(query, numScanWithAutoScanEnabled, numScanWithAutoScanDisabled) + } + } + } + + test("SPARK-32859: disable unnecessary bucketed table scan - multiple joins test") { + withTable("t1", "t2", "t3") { + df1.write.format("parquet").bucketBy(8, "i").saveAsTable("t1") + df2.write.format("parquet").bucketBy(8, "i").saveAsTable("t2") + df2.write.format("parquet").bucketBy(4, "i").saveAsTable("t3") + + Seq( + // Multiple joins on bucketed columns + (""" + SELECT /*+ broadcast(t1, t3)*/ * FROM t1 JOIN t2 JOIN t3 + ON t1.i = t2.i AND t2.i = t3.i + """.stripMargin, 0, 3), + (""" + SELECT /*+ broadcast(t1) merge(t3)*/ * FROM t1 JOIN t2 JOIN t3 + ON t1.i = t2.i AND t2.i = t3.i + """.stripMargin, 2, 3), + (""" + SELECT /*+ merge(t1) broadcast(t3)*/ * FROM t1 JOIN t2 JOIN t3 + ON t1.i = t2.i AND t2.i = t3.i + """.stripMargin, 2, 3), + (""" + SELECT /*+ merge(t1, t3)*/ * FROM t1 JOIN t2 JOIN t3 + ON t1.i = t2.i AND t2.i = t3.i + """.stripMargin, 2, 3), + // Multiple joins on non-bucketed columns + (""" + SELECT /*+ broadcast(t1, t3)*/ * FROM t1 JOIN t2 JOIN t3 + ON t1.i = t2.j AND t2.j = t3.i + """.stripMargin, 0, 3), + (""" + SELECT /*+ merge(t1, t3)*/ * FROM t1 JOIN t2 JOIN t3 + ON t1.i = t2.j AND t2.j = t3.i + """.stripMargin, 1, 3), + (""" + SELECT /*+ merge(t1, t3)*/ * FROM t1 JOIN t2 JOIN t3 + ON t1.j = t2.j AND t2.j = t3.j + """.stripMargin, 0, 3) + ).foreach { case (query, numScanWithAutoScanEnabled, numScanWithAutoScanDisabled) => + checkDisableBucketedScan(query, numScanWithAutoScanEnabled, numScanWithAutoScanDisabled) + } + } + } + + test("SPARK-32859: disable unnecessary bucketed table scan - multiple bucketed columns test") { + withTable("t1", "t2", "t3") { + df1.write.format("parquet").bucketBy(8, "i", "j").saveAsTable("t1") + df2.write.format("parquet").bucketBy(8, "i", "j").saveAsTable("t2") + df2.write.format("parquet").bucketBy(4, "i", "j").saveAsTable("t3") + + Seq( + // Filter on bucketed columns + ("SELECT * FROM t1 WHERE i = 1", 0, 1), + ("SELECT * FROM t1 WHERE i = 1 AND j = 1", 0, 1), + // Join on bucketed columns + (""" + SELECT /*+ broadcast(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.i AND t1.j = t2.j + """.stripMargin, 0, 2), + (""" + SELECT /*+ merge(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.i AND t1.j = t2.j + """.stripMargin, 2, 2), + (""" + SELECT /*+ merge(t1)*/ * FROM t1 JOIN t3 ON t1.i = t3.i AND t1.j = t3.j + """.stripMargin, 1, 2), + ("SELECT /*+ merge(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.i", 0, 2), + // Aggregate on bucketed columns + ("SELECT i, j, COUNT(*) FROM t1 GROUP BY i, j", 1, 1), + ("SELECT i, COUNT(i) FROM t1 GROUP BY i", 0, 0), + ("SELECT i, COUNT(j) FROM t1 GROUP BY i", 0, 1) + ).foreach { case (query, numScanWithAutoScanEnabled, numScanWithAutoScanDisabled) => + checkDisableBucketedScan(query, numScanWithAutoScanEnabled, numScanWithAutoScanDisabled) + } + } + } + + test("SPARK-32859: disable unnecessary bucketed table scan - other operators test") { + withTable("t1", "t2", "t3") { + df1.write.format("parquet").bucketBy(8, "i").saveAsTable("t1") + df2.write.format("parquet").bucketBy(8, "i").saveAsTable("t2") + df1.write.format("parquet").saveAsTable("t3") + + Seq( + // Operator with interesting partition not in sub-plan + (""" + SELECT t1.i FROM t1 + UNION ALL + (SELECT t2.i FROM t2 GROUP BY t2.i) + """.stripMargin, 1, 2), + // Non-allowed operator in sub-plan + (""" + SELECT COUNT(*) + FROM (SELECT t1.i FROM t1 UNION ALL SELECT t2.i FROM t2) + GROUP BY i + """.stripMargin, 2, 2), + // Multiple [[Exchange]] in sub-plan + (""" + SELECT j, SUM(i), COUNT(*) FROM t1 GROUP BY j + DISTRIBUTE BY j + """.stripMargin, 0, 1), + (""" + SELECT j, COUNT(*) + FROM (SELECT i, j FROM t1 DISTRIBUTE BY i, j) + GROUP BY j + """.stripMargin, 0, 1), + // No bucketed table scan in plan + (""" + SELECT j, COUNT(*) + FROM (SELECT t1.j FROM t1 JOIN t3 ON t1.j = t3.j) + GROUP BY j + """.stripMargin, 0, 0) + ).foreach { case (query, numScanWithAutoScanEnabled, numScanWithAutoScanDisabled) => + checkDisableBucketedScan(query, numScanWithAutoScanEnabled, numScanWithAutoScanDisabled) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index abd33ab8a8f22..32c4fb60b8c54 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -866,6 +866,28 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { }.getMessage assert(message.contains("LOCAL is supported only with file: scheme")) } + + test("SPARK-32508 " + + "Disallow empty part col values in partition spec before static partition writing") { + withTable("insertTable") { + sql( + """ + |CREATE TABLE insertTable(i int, part1 string, part2 string) USING PARQUET + |PARTITIONED BY (part1, part2) + """.stripMargin) + val msg = "Partition spec is invalid" + assert(intercept[AnalysisException] { + sql("INSERT INTO TABLE insertTable PARTITION(part1=1, part2='') SELECT 1") + }.getMessage.contains(msg)) + assert(intercept[AnalysisException] { + sql("INSERT INTO TABLE insertTable PARTITION(part1='', part2) SELECT 1 ,'' AS part2") + }.getMessage.contains(msg)) + + sql("INSERT INTO TABLE insertTable PARTITION(part1='1', part2='2') SELECT 1") + sql("INSERT INTO TABLE insertTable PARTITION(part1='1', part2) SELECT 1 ,'2' AS part2") + sql("INSERT INTO TABLE insertTable PARTITION(part1='1', part2) SELECT 1 ,'' AS part2") + } + } } class FileExistingTestFileSystem extends RawLocalFileSystem { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 9f3ff1a6708e4..8797e5ad64149 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -36,7 +36,6 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.Range import org.apache.spark.sql.catalyst.streaming.{InternalOutputModes, StreamingRelationV2} -import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_MILLIS import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.{LocalLimitExec, SimpleMode, SparkPlan} import org.apache.spark.sql.execution.command.ExplainCommand @@ -47,7 +46,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.StreamSourceProvider import org.apache.spark.sql.streaming.util.{BlockOnStopSourceProvider, StreamManualClock} -import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.sql.types.{IntegerType, LongType, StructField, StructType} import org.apache.spark.util.Utils class StreamSuite extends StreamTest { @@ -1268,7 +1267,7 @@ class StreamSuite extends StreamTest { } abstract class FakeSource extends StreamSourceProvider { - private val fakeSchema = StructType(StructField("a", IntegerType) :: Nil) + private val fakeSchema = StructType(StructField("a", LongType) :: Nil) override def sourceSchema( spark: SQLContext, @@ -1290,7 +1289,7 @@ class FakeDefaultSource extends FakeSource { new Source { private var offset = -1L - override def schema: StructType = StructType(StructField("a", IntegerType) :: Nil) + override def schema: StructType = StructType(StructField("a", LongType) :: Nil) override def getOffset: Option[Offset] = { if (offset >= 10) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala index 8f34106d3d8f5..8d39704c61d4e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala @@ -43,11 +43,13 @@ object LastOptions { var mockStreamSourceProvider = mock(classOf[StreamSourceProvider]) var mockStreamSinkProvider = mock(classOf[StreamSinkProvider]) var parameters: Map[String, String] = null + var sinkParameters: Map[String, String] = null var schema: Option[StructType] = null var partitionColumns: Seq[String] = Nil def clear(): Unit = { parameters = null + sinkParameters = null schema = null partitionColumns = null reset(mockStreamSourceProvider) @@ -101,7 +103,7 @@ class DefaultSource extends StreamSourceProvider with StreamSinkProvider { parameters: Map[String, String], partitionColumns: Seq[String], outputMode: OutputMode): Sink = { - LastOptions.parameters = parameters + LastOptions.sinkParameters = parameters LastOptions.partitionColumns = partitionColumns LastOptions.mockStreamSinkProvider.createSink(spark, parameters, partitionColumns, outputMode) (_: Long, _: DataFrame) => {} @@ -172,16 +174,48 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { df.writeStream .format("org.apache.spark.sql.streaming.test") - .option("opt1", "1") - .options(Map("opt2" -> "2")) + .option("opt1", "5") + .options(Map("opt2" -> "4")) .options(map) .option("checkpointLocation", newMetadataDir) .start() .stop() - assert(LastOptions.parameters("opt1") == "1") - assert(LastOptions.parameters("opt2") == "2") - assert(LastOptions.parameters("opt3") == "3") + assert(LastOptions.sinkParameters("opt1") == "5") + assert(LastOptions.sinkParameters("opt2") == "4") + assert(LastOptions.sinkParameters("opt3") == "3") + assert(LastOptions.sinkParameters.contains("checkpointLocation")) + } + + test("SPARK-32832: later option should override earlier options for load()") { + spark.readStream + .format("org.apache.spark.sql.streaming.test") + .option("paTh", "1") + .option("PATH", "2") + .option("Path", "3") + .option("patH", "4") + .option("path", "5") + .load() + assert(LastOptions.parameters("path") == "5") + } + + test("SPARK-32832: later option should override earlier options for start()") { + val ds = spark.readStream + .format("org.apache.spark.sql.streaming.test") + .load() + assert(LastOptions.parameters.isEmpty) + + ds.writeStream + .format("org.apache.spark.sql.streaming.test") + .option("checkpointLocation", newMetadataDir) + .option("paTh", "1") + .option("PATH", "2") + .option("Path", "3") + .option("patH", "4") + .option("path", "5") + .start() + .stop() + assert(LastOptions.sinkParameters("path") == "5") } test("partitioning") { @@ -712,7 +746,9 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { } verifyLoadFails(spark.readStream.option("path", "tmp1").parquet("tmp2")) + verifyLoadFails(spark.readStream.option("path", "tmp1").parquet("")) verifyLoadFails(spark.readStream.option("path", "tmp1").format("parquet").load("tmp2")) + verifyLoadFails(spark.readStream.option("path", "tmp1").format("parquet").load("")) withClue("SPARK-32516: legacy behavior") { withSQLConf(SQLConf.LEGACY_PATH_OPTION_BEHAVIOR.key -> "true") { @@ -731,25 +767,51 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { .format("org.apache.spark.sql.streaming.test") .load("tmp1") - val e = intercept[AnalysisException] { + def verifyStartFails(f: => StreamingQuery): Unit = { + val e = intercept[AnalysisException](f) + assert(e.getMessage.contains( + "Either remove the path option, or call start() without the parameter")) + } + + verifyStartFails( df.writeStream .format("org.apache.spark.sql.streaming.test") .option("path", "tmp2") - .start("tmp3") - .stop() - } - assert(e.getMessage.contains( - "Either remove the path option, or call start() without the parameter")) + .start("tmp3")) + verifyStartFails( + df.writeStream + .format("org.apache.spark.sql.streaming.test") + .option("path", "tmp2") + .start("")) withClue("SPARK-32516: legacy behavior") { - withSQLConf(SQLConf.LEGACY_PATH_OPTION_BEHAVIOR.key -> "true") { - spark.readStream - .format("org.apache.spark.sql.streaming.test") - .option("path", "tmp4") - .load("tmp5") - // The legacy behavior overwrites the path option. - assert(LastOptions.parameters("path") == "tmp5") + withTempDir { checkpointPath => + withSQLConf(SQLConf.LEGACY_PATH_OPTION_BEHAVIOR.key -> "true", + SQLConf.CHECKPOINT_LOCATION.key -> checkpointPath.getAbsolutePath) { + df.writeStream + .format("org.apache.spark.sql.streaming.test") + .option("path", "tmp4") + .start("tmp5") + .stop() + // The legacy behavior overwrites the path option. + assert(LastOptions.sinkParameters("path") == "tmp5") + } } } } + + test("SPARK-32853: consecutive load/start calls should be allowed") { + val dfr = spark.readStream.format(classOf[DefaultSource].getName) + var df = dfr.load("1") + df = dfr.load("2") + withTempDir { checkpointPath => + val dfw = df.writeStream + .option("checkpointLocation", checkpointPath.getCanonicalPath) + .format(classOf[DefaultSource].getName) + var query = dfw.start("1") + query.stop() + query = dfw.start("2") + query.stop() + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala new file mode 100644 index 0000000000000..062b1060bc601 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala @@ -0,0 +1,376 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming.test + +import java.io.File +import java.util + +import scala.collection.JavaConverters._ + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.{AnalysisException, Row} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2 +import org.apache.spark.sql.connector.{FakeV2Provider, InMemoryTableCatalog, InMemoryTableSessionCatalog} +import org.apache.spark.sql.connector.catalog.{Identifier, SupportsRead, Table, TableCapability, V2TableWithV1Fallback} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.connector.read.ScanBuilder +import org.apache.spark.sql.execution.streaming.{MemoryStream, MemoryStreamScanBuilder} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.StreamTest +import org.apache.spark.sql.streaming.sources.FakeScanBuilder +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class DataStreamTableAPISuite extends StreamTest with BeforeAndAfter { + import testImplicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + before { + spark.conf.set("spark.sql.catalog.testcat", classOf[InMemoryTableCatalog].getName) + spark.conf.set("spark.sql.catalog.teststream", classOf[InMemoryStreamTableCatalog].getName) + } + + after { + spark.sessionState.catalogManager.reset() + spark.sessionState.conf.clear() + sqlContext.streams.active.foreach(_.stop()) + } + + test("read: table API with file source") { + Seq("parquet", "").foreach { source => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> source) { + withTempDir { tempDir => + val tblName = "my_table" + val dir = tempDir.getAbsolutePath + withTable(tblName) { + spark.range(3).write.format("parquet").option("path", dir).saveAsTable(tblName) + + testStream(spark.readStream.table(tblName))( + ProcessAllAvailable(), + CheckAnswer(Row(0), Row(1), Row(2)) + ) + } + } + } + } + } + + test("read: read non-exist table") { + intercept[AnalysisException] { + spark.readStream.table("non_exist_table") + }.message.contains("Table not found") + } + + test("read: stream table API with temp view") { + val tblName = "my_table" + val stream = MemoryStream[Int] + withTable(tblName) { + stream.toDF().createOrReplaceTempView(tblName) + + testStream(spark.readStream.table(tblName)) ( + AddData(stream, 1, 2, 3), + CheckLastBatch(1, 2, 3), + AddData(stream, 4, 5), + CheckLastBatch(4, 5) + ) + } + } + + test("read: stream table API with non-streaming temp view") { + val tblName = "my_table" + withTable(tblName) { + spark.range(3).createOrReplaceTempView(tblName) + intercept[AnalysisException] { + spark.readStream.table(tblName) + }.message.contains("is not a temp view of streaming logical plan") + } + } + + test("read: read table without streaming capability support") { + val tableIdentifer = "testcat.table_name" + + spark.sql(s"CREATE TABLE $tableIdentifer (id bigint, data string) USING foo") + + intercept[AnalysisException] { + spark.readStream.table(tableIdentifer) + }.message.contains("does not support either micro-batch or continuous scan") + } + + test("read: read table with custom catalog") { + val tblName = "teststream.table_name" + withTable(tblName) { + spark.sql(s"CREATE TABLE $tblName (data int) USING foo") + val stream = MemoryStream[Int] + val testCatalog = spark.sessionState.catalogManager.catalog("teststream").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + table.asInstanceOf[InMemoryStreamTable].setStream(stream) + + testStream(spark.readStream.table(tblName)) ( + AddData(stream, 1, 2, 3), + CheckLastBatch(1, 2, 3), + AddData(stream, 4, 5), + CheckLastBatch(4, 5) + ) + } + } + + test("read: read table with custom catalog & namespace") { + spark.sql("CREATE NAMESPACE teststream.ns") + + val tblName = "teststream.ns.table_name" + withTable(tblName) { + spark.sql(s"CREATE TABLE $tblName (data int) USING foo") + val stream = MemoryStream[Int] + val testCatalog = spark.sessionState.catalogManager.catalog("teststream").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns"), "table_name")) + table.asInstanceOf[InMemoryStreamTable].setStream(stream) + + testStream(spark.readStream.table(tblName)) ( + AddData(stream, 1, 2, 3), + CheckLastBatch(1, 2, 3), + AddData(stream, 4, 5), + CheckLastBatch(4, 5) + ) + } + } + + test("read: fallback to V1 relation") { + val tblName = DataStreamTableAPISuite.V1FallbackTestTableName + spark.conf.set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, + classOf[InMemoryStreamTableCatalog].getName) + val v2Source = classOf[FakeV2Provider].getName + withTempDir { tempDir => + withTable(tblName) { + spark.sql(s"CREATE TABLE $tblName (data int) USING $v2Source") + + // Check the StreamingRelationV2 has been replaced by StreamingRelation + val plan = spark.readStream.option("path", tempDir.getCanonicalPath).table(tblName) + .queryExecution.analyzed.collectFirst { + case d: StreamingRelationV2 => d + } + assert(plan.isEmpty) + } + } + } + + test("write: write to table with custom catalog & no namespace") { + val tableIdentifier = "testcat.table_name" + + spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo") + checkAnswer(spark.table(tableIdentifier), Seq.empty) + + runTestWithStreamAppend(tableIdentifier) + } + + test("write: write to table with custom catalog & namespace") { + spark.sql("CREATE NAMESPACE testcat.ns") + + val tableIdentifier = "testcat.ns.table_name" + + spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo") + checkAnswer(spark.table(tableIdentifier), Seq.empty) + + runTestWithStreamAppend(tableIdentifier) + } + + test("write: write to table with default session catalog") { + val v2Source = classOf[FakeV2Provider].getName + spark.conf.set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, + classOf[InMemoryTableSessionCatalog].getName) + + spark.sql("CREATE NAMESPACE ns") + + val tableIdentifier = "ns.table_name" + spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING $v2Source") + checkAnswer(spark.table(tableIdentifier), Seq.empty) + + runTestWithStreamAppend(tableIdentifier) + } + + test("write: write to non-exist table with custom catalog") { + val tableIdentifier = "testcat.nonexisttable" + spark.sql("CREATE NAMESPACE testcat.ns") + + withTempDir { checkpointDir => + val exc = intercept[NoSuchTableException] { + runStreamQueryAppendMode(tableIdentifier, checkpointDir, Seq.empty, Seq.empty) + } + assert(exc.getMessage.contains("nonexisttable")) + } + } + + test("write: write to file provider based table isn't allowed yet") { + val tableIdentifier = "table_name" + + spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING parquet") + checkAnswer(spark.table(tableIdentifier), Seq.empty) + + withTempDir { checkpointDir => + val exc = intercept[AnalysisException] { + runStreamQueryAppendMode(tableIdentifier, checkpointDir, Seq.empty, Seq.empty) + } + assert(exc.getMessage.contains("doesn't support streaming write")) + } + } + + test("write: write to temporary view isn't allowed yet") { + val tableIdentifier = "testcat.table_name" + val tempViewIdentifier = "temp_view" + + spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo") + checkAnswer(spark.table(tableIdentifier), Seq.empty) + + spark.table(tableIdentifier).createOrReplaceTempView(tempViewIdentifier) + + withTempDir { checkpointDir => + val exc = intercept[AnalysisException] { + runStreamQueryAppendMode(tempViewIdentifier, checkpointDir, Seq.empty, Seq.empty) + } + assert(exc.getMessage.contains("doesn't support streaming write")) + } + } + + test("write: write to view shouldn't be allowed") { + val tableIdentifier = "testcat.table_name" + val viewIdentifier = "table_view" + + spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo") + checkAnswer(spark.table(tableIdentifier), Seq.empty) + + spark.sql(s"CREATE VIEW $viewIdentifier AS SELECT id, data FROM $tableIdentifier") + + withTempDir { checkpointDir => + val exc = intercept[AnalysisException] { + runStreamQueryAppendMode(viewIdentifier, checkpointDir, Seq.empty, Seq.empty) + } + assert(exc.getMessage.contains("doesn't support streaming write")) + } + } + + private def runTestWithStreamAppend(tableIdentifier: String) = { + withTempDir { checkpointDir => + val input1 = Seq((1L, "a"), (2L, "b"), (3L, "c")) + verifyStreamAppend(tableIdentifier, checkpointDir, Seq.empty, input1, input1) + + val input2 = Seq((4L, "d"), (5L, "e"), (6L, "f")) + verifyStreamAppend(tableIdentifier, checkpointDir, Seq(input1), input2, input1 ++ input2) + } + } + + private def runStreamQueryAppendMode( + tableIdentifier: String, + checkpointDir: File, + prevInputs: Seq[Seq[(Long, String)]], + newInputs: Seq[(Long, String)]): Unit = { + val inputData = MemoryStream[(Long, String)] + val inputDF = inputData.toDF().toDF("id", "data") + + prevInputs.foreach { inputsPerBatch => + inputData.addData(inputsPerBatch: _*) + } + + val query = inputDF + .writeStream + .option("checkpointLocation", checkpointDir.getAbsolutePath) + .saveAsTable(tableIdentifier) + + inputData.addData(newInputs: _*) + + query.processAllAvailable() + query.stop() + } + + private def verifyStreamAppend( + tableIdentifier: String, + checkpointDir: File, + prevInputs: Seq[Seq[(Long, String)]], + newInputs: Seq[(Long, String)], + expectedOutputs: Seq[(Long, String)]): Unit = { + runStreamQueryAppendMode(tableIdentifier, checkpointDir, prevInputs, newInputs) + checkAnswer( + spark.table(tableIdentifier), + expectedOutputs.map { case (id, data) => Row(id, data) } + ) + } +} + +object DataStreamTableAPISuite { + val V1FallbackTestTableName = "fallbackV1Test" +} + +class InMemoryStreamTable(override val name: String) extends Table with SupportsRead { + var stream: MemoryStream[Int] = _ + + def setStream(inputData: MemoryStream[Int]): Unit = stream = inputData + + override def schema(): StructType = stream.fullSchema() + + override def capabilities(): util.Set[TableCapability] = { + Set(TableCapability.MICRO_BATCH_READ, TableCapability.CONTINUOUS_READ).asJava + } + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + new MemoryStreamScanBuilder(stream) + } +} + +class NonStreamV2Table(override val name: String) + extends Table with SupportsRead with V2TableWithV1Fallback { + override def schema(): StructType = StructType(Nil) + override def capabilities(): util.Set[TableCapability] = Set(TableCapability.BATCH_READ).asJava + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = new FakeScanBuilder + + override def v1Table: CatalogTable = { + CatalogTable( + identifier = + TableIdentifier(DataStreamTableAPISuite.V1FallbackTestTableName, Some("default")), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + owner = null, + schema = schema(), + provider = Some("parquet")) + } +} + + +class InMemoryStreamTableCatalog extends InMemoryTableCatalog { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override def createTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): Table = { + if (tables.containsKey(ident)) { + throw new TableAlreadyExistsException(ident) + } + + val table = if (ident.name() == DataStreamTableAPISuite.V1FallbackTestTableName) { + new NonStreamV2Table(s"$name.${ident.quoted}") + } else { + new InMemoryStreamTable(s"$name.${ident.quoted}") + } + tables.put(ident, table) + namespaces.putIfAbsent(ident.namespace.toList, Map()) + table + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index c84d361024309..4e61dba4955af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -40,7 +40,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression} import org.apache.spark.sql.execution.QueryExecution -import org.apache.spark.sql.execution.datasources.DataSourceUtils +import org.apache.spark.sql.execution.datasources.{DataSourceUtils, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.datasources.noop.NoopDataSource import org.apache.spark.sql.execution.datasources.parquet.SpecificParquetRecordReaderBase import org.apache.spark.sql.internal.SQLConf @@ -1135,17 +1135,21 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with } test("SPARK-32516: 'path' or 'paths' option cannot coexist with load()'s path parameters") { - def verifyLoadFails(f: () => DataFrame): Unit = { - val e = intercept[AnalysisException](f()) + def verifyLoadFails(f: => DataFrame): Unit = { + val e = intercept[AnalysisException](f) assert(e.getMessage.contains( "Either remove the path option if it's the same as the path parameter")) } val path = "/tmp" - verifyLoadFails(() => spark.read.option("path", path).parquet(path)) - verifyLoadFails(() => spark.read.option("path", path).format("parquet").load(path)) - verifyLoadFails(() => spark.read.option("paths", path).parquet(path)) - verifyLoadFails(() => spark.read.option("paths", path).format("parquet").load(path)) + verifyLoadFails(spark.read.option("path", path).parquet(path)) + verifyLoadFails(spark.read.option("path", path).parquet("")) + verifyLoadFails(spark.read.option("path", path).format("parquet").load(path)) + verifyLoadFails(spark.read.option("path", path).format("parquet").load("")) + verifyLoadFails(spark.read.option("paths", path).parquet(path)) + verifyLoadFails(spark.read.option("paths", path).parquet("")) + verifyLoadFails(spark.read.option("paths", path).format("parquet").load(path)) + verifyLoadFails(spark.read.option("paths", path).format("parquet").load("")) } test("SPARK-32516: legacy path option behavior in load()") { @@ -1182,6 +1186,37 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with val df = Seq(1).toDF val path = "tmp" verifyLoadFails(df.write.option("path", path).parquet(path)) + verifyLoadFails(df.write.option("path", path).parquet("")) verifyLoadFails(df.write.option("path", path).format("parquet").save(path)) + verifyLoadFails(df.write.option("path", path).format("parquet").save("")) + } + + test("SPARK-32853: consecutive load/save calls should be allowed") { + val dfr = spark.read.format(classOf[FakeSourceOne].getName) + dfr.load("1") + dfr.load("2") + val dfw = spark.range(10).write.format(classOf[DefaultSource].getName) + dfw.save("1") + dfw.save("2") + } + + test("SPARK-32844: DataFrameReader.table take the specified options for V1 relation") { + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { + withTable("t") { + sql("CREATE TABLE t(i int, d double) USING parquet OPTIONS ('p1'='v1', 'p2'='v2')") + + val msg = intercept[AnalysisException] { + spark.read.option("P1", "v3").table("t").count() + }.getMessage + assert(msg.contains("duplicated key")) + + val df = spark.read.option("P2", "v2").option("p3", "v3").table("t") + val options = df.queryExecution.analyzed.collectFirst { + case r: LogicalRelation => r.relation.asInstanceOf[HadoopFsRelation].options + }.get + assert(options("p2") == "v2") + assert(options("p3") == "v3") + } + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala index ee29b4b8fb32b..cfc92a780308d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala @@ -25,6 +25,7 @@ import org.scalatest.concurrent.Eventually import org.apache.spark.{DebugFilesystem, SparkConf} import org.apache.spark.internal.config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK import org.apache.spark.sql.{SparkSession, SQLContext} +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} @@ -67,6 +68,7 @@ trait SharedSparkSessionBase .set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) .set(UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true) .set(SQLConf.CODEGEN_FALLBACK.key, "false") + .set(SQLConf.CODEGEN_FACTORY_MODE.key, CodegenObjectFactoryMode.CODEGEN_ONLY.toString) // Disable ConvertToLocalRelation for better test coverage. Test cases built on // LocalRelation will exercise the optimization rules better by disabling it as // this rule may potentially block testing of other optimization rules such as diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala index 17603deacdcdd..ac06e1f41bfb3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -39,7 +39,7 @@ private[spark] class TestSparkSession(sc: SparkContext) extends SparkSession(sc) @transient override lazy val sessionState: SessionState = { - new TestSQLSessionStateBuilder(this, None).build() + new TestSQLSessionStateBuilder(this, None, Map.empty).build() } // Needed for Java tests @@ -66,8 +66,9 @@ private[sql] object TestSQLContext { private[sql] class TestSQLSessionStateBuilder( session: SparkSession, - state: Option[SessionState]) - extends SessionStateBuilder(session, state) with WithTestConf { + state: Option[SessionState], + options: Map[String, String]) + extends SessionStateBuilder(session, state, options) with WithTestConf { override def overrideConfs: Map[String, String] = TestSQLContext.overrideConfs - override def newBuilder: NewBuilder = new TestSQLSessionStateBuilder(_, _) + override def newBuilder: NewBuilder = new TestSQLSessionStateBuilder(_, _, Map.empty) } diff --git a/sql/core/v1.2/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java b/sql/core/v1.2/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java deleted file mode 100644 index 6601bcb9018f4..0000000000000 --- a/sql/core/v1.2/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java +++ /dev/null @@ -1,208 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.orc; - -import java.math.BigDecimal; - -import org.apache.orc.storage.ql.exec.vector.*; - -import org.apache.spark.sql.catalyst.util.DateTimeUtils; -import org.apache.spark.sql.catalyst.util.RebaseDateTime; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.DateType; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.sql.types.TimestampType; -import org.apache.spark.sql.vectorized.ColumnarArray; -import org.apache.spark.sql.vectorized.ColumnarMap; -import org.apache.spark.unsafe.types.UTF8String; - -/** - * A column vector class wrapping Hive's ColumnVector. Because Spark ColumnarBatch only accepts - * Spark's vectorized.ColumnVector, this column vector is used to adapt Hive ColumnVector with - * Spark ColumnarVector. - */ -public class OrcColumnVector extends org.apache.spark.sql.vectorized.ColumnVector { - private ColumnVector baseData; - private LongColumnVector longData; - private DoubleColumnVector doubleData; - private BytesColumnVector bytesData; - private DecimalColumnVector decimalData; - private TimestampColumnVector timestampData; - private final boolean isTimestamp; - private final boolean isDate; - - private int batchSize; - - OrcColumnVector(DataType type, ColumnVector vector) { - super(type); - - if (type instanceof TimestampType) { - isTimestamp = true; - } else { - isTimestamp = false; - } - - if (type instanceof DateType) { - isDate = true; - } else { - isDate = false; - } - - baseData = vector; - if (vector instanceof LongColumnVector) { - longData = (LongColumnVector) vector; - } else if (vector instanceof DoubleColumnVector) { - doubleData = (DoubleColumnVector) vector; - } else if (vector instanceof BytesColumnVector) { - bytesData = (BytesColumnVector) vector; - } else if (vector instanceof DecimalColumnVector) { - decimalData = (DecimalColumnVector) vector; - } else if (vector instanceof TimestampColumnVector) { - timestampData = (TimestampColumnVector) vector; - } else { - throw new UnsupportedOperationException(); - } - } - - public void setBatchSize(int batchSize) { - this.batchSize = batchSize; - } - - @Override - public void close() { - - } - - @Override - public boolean hasNull() { - return !baseData.noNulls; - } - - @Override - public int numNulls() { - if (baseData.isRepeating) { - if (baseData.isNull[0]) { - return batchSize; - } else { - return 0; - } - } else if (baseData.noNulls) { - return 0; - } else { - int count = 0; - for (int i = 0; i < batchSize; i++) { - if (baseData.isNull[i]) count++; - } - return count; - } - } - - /* A helper method to get the row index in a column. */ - private int getRowIndex(int rowId) { - return baseData.isRepeating ? 0 : rowId; - } - - @Override - public boolean isNullAt(int rowId) { - return baseData.isNull[getRowIndex(rowId)]; - } - - @Override - public boolean getBoolean(int rowId) { - return longData.vector[getRowIndex(rowId)] == 1; - } - - @Override - public byte getByte(int rowId) { - return (byte) longData.vector[getRowIndex(rowId)]; - } - - @Override - public short getShort(int rowId) { - return (short) longData.vector[getRowIndex(rowId)]; - } - - @Override - public int getInt(int rowId) { - int value = (int) longData.vector[getRowIndex(rowId)]; - if (isDate) { - return RebaseDateTime.rebaseJulianToGregorianDays(value); - } else { - return value; - } - } - - @Override - public long getLong(int rowId) { - int index = getRowIndex(rowId); - if (isTimestamp) { - return DateTimeUtils.fromJavaTimestamp(timestampData.asScratchTimestamp(index)); - } else { - return longData.vector[index]; - } - } - - @Override - public float getFloat(int rowId) { - return (float) doubleData.vector[getRowIndex(rowId)]; - } - - @Override - public double getDouble(int rowId) { - return doubleData.vector[getRowIndex(rowId)]; - } - - @Override - public Decimal getDecimal(int rowId, int precision, int scale) { - if (isNullAt(rowId)) return null; - BigDecimal data = decimalData.vector[getRowIndex(rowId)].getHiveDecimal().bigDecimalValue(); - return Decimal.apply(data, precision, scale); - } - - @Override - public UTF8String getUTF8String(int rowId) { - if (isNullAt(rowId)) return null; - int index = getRowIndex(rowId); - BytesColumnVector col = bytesData; - return UTF8String.fromBytes(col.vector[index], col.start[index], col.length[index]); - } - - @Override - public byte[] getBinary(int rowId) { - if (isNullAt(rowId)) return null; - int index = getRowIndex(rowId); - byte[] binary = new byte[bytesData.length[index]]; - System.arraycopy(bytesData.vector[index], bytesData.start[index], binary, 0, binary.length); - return binary; - } - - @Override - public ColumnarArray getArray(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public ColumnarMap getMap(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public org.apache.spark.sql.vectorized.ColumnVector getChild(int ordinal) { - throw new UnsupportedOperationException(); - } -} diff --git a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/DaysWritable.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/DaysWritable.scala deleted file mode 100644 index 1dccf0ca1faef..0000000000000 --- a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/DaysWritable.scala +++ /dev/null @@ -1,79 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.orc - -import java.io.{DataInput, DataOutput, IOException} -import java.sql.Date - -import org.apache.hadoop.io.WritableUtils -import org.apache.orc.storage.serde2.io.DateWritable - -import org.apache.spark.sql.catalyst.util.RebaseDateTime.{rebaseGregorianToJulianDays, rebaseJulianToGregorianDays} - -/** - * The class accepts/returns days in Gregorian calendar and rebase them - * via conversion to local date in Julian calendar for dates before 1582-10-15 - * in read/write for backward compatibility with Spark 2.4 and earlier versions. - * - * This is a clone of `org.apache.spark.sql.execution.datasources.DaysWritable`. - * The class is cloned because Hive ORC v1.2 uses different `DateWritable`: - * - v1.2: `org.apache.orc.storage.serde2.io.DateWritable` - * - v2.3 and `HiveInspectors`: `org.apache.hadoop.hive.serde2.io.DateWritable` - * - * @param gregorianDays The number of days since the epoch 1970-01-01 in - * Gregorian calendar. - * @param julianDays The number of days since the epoch 1970-01-01 in - * Julian calendar. - */ -class DaysWritable( - var gregorianDays: Int, - var julianDays: Int) - extends DateWritable { - - def this() = this(0, 0) - def this(gregorianDays: Int) = - this(gregorianDays, rebaseGregorianToJulianDays(gregorianDays)) - def this(dateWritable: DateWritable) = { - this( - gregorianDays = dateWritable match { - case daysWritable: DaysWritable => daysWritable.gregorianDays - case dateWritable: DateWritable => - rebaseJulianToGregorianDays(dateWritable.getDays) - }, - julianDays = dateWritable.getDays) - } - - override def getDays: Int = julianDays - override def get(): Date = new Date(DateWritable.daysToMillis(julianDays)) - - override def set(d: Int): Unit = { - gregorianDays = d - julianDays = rebaseGregorianToJulianDays(d) - } - - @throws[IOException] - override def write(out: DataOutput): Unit = { - WritableUtils.writeVInt(out, julianDays) - } - - @throws[IOException] - override def readFields(in: DataInput): Unit = { - julianDays = WritableUtils.readVInt(in) - gregorianDays = rebaseJulianToGregorianDays(julianDays) - } -} diff --git a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala deleted file mode 100644 index 0e657bfe66238..0000000000000 --- a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ /dev/null @@ -1,275 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.orc - -import java.time.{Instant, LocalDate} - -import org.apache.orc.storage.common.`type`.HiveDecimal -import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} -import org.apache.orc.storage.ql.io.sarg.SearchArgument.Builder -import org.apache.orc.storage.ql.io.sarg.SearchArgumentFactory.newBuilder -import org.apache.orc.storage.serde2.io.HiveDecimalWritable - -import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.util.DateTimeUtils.{instantToMicros, localDateToDays, toJavaDate, toJavaTimestamp} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types._ - -/** - * Helper object for building ORC `SearchArgument`s, which are used for ORC predicate push-down. - * - * Due to limitation of ORC `SearchArgument` builder, we had to implement separate checking and - * conversion passes through the Filter to make sure we only convert predicates that are known - * to be convertible. - * - * An ORC `SearchArgument` must be built in one pass using a single builder. For example, you can't - * build `a = 1` and `b = 2` first, and then combine them into `a = 1 AND b = 2`. This is quite - * different from the cases in Spark SQL or Parquet, where complex filters can be easily built using - * existing simpler ones. - * - * The annoying part is that, `SearchArgument` builder methods like `startAnd()`, `startOr()`, and - * `startNot()` mutate internal state of the builder instance. This forces us to translate all - * convertible filters with a single builder instance. However, if we try to translate a filter - * before checking whether it can be converted or not, we may end up with a builder whose internal - * state is inconsistent in the case of an inconvertible filter. - * - * For example, to convert an `And` filter with builder `b`, we call `b.startAnd()` first, and then - * try to convert its children. Say we convert `left` child successfully, but find that `right` - * child is inconvertible. Alas, `b.startAnd()` call can't be rolled back, and `b` is inconsistent - * now. - * - * The workaround employed here is to trim the Spark filters before trying to convert them. This - * way, we can only do the actual conversion on the part of the Filter that is known to be - * convertible. - * - * P.S.: Hive seems to use `SearchArgument` together with `ExprNodeGenericFuncDesc` only. Usage of - * builder methods mentioned above can only be found in test code, where all tested filters are - * known to be convertible. - */ -private[sql] object OrcFilters extends OrcFiltersBase { - - /** - * Create ORC filter as a SearchArgument instance. - */ - def createFilter(schema: StructType, filters: Seq[Filter]): Option[SearchArgument] = { - val dataTypeMap = OrcFilters.getSearchableTypeMap(schema, SQLConf.get.caseSensitiveAnalysis) - // Combines all convertible filters using `And` to produce a single conjunction - val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, filters)) - conjunctionOptional.map { conjunction => - // Then tries to build a single ORC `SearchArgument` for the conjunction predicate. - // The input predicate is fully convertible. There should not be any empty result in the - // following recursive method call `buildSearchArgument`. - buildSearchArgument(dataTypeMap, conjunction, newBuilder).build() - } - } - - def convertibleFilters( - schema: StructType, - dataTypeMap: Map[String, OrcPrimitiveField], - filters: Seq[Filter]): Seq[Filter] = { - import org.apache.spark.sql.sources._ - - def convertibleFiltersHelper( - filter: Filter, - canPartialPushDown: Boolean): Option[Filter] = filter match { - // At here, it is not safe to just convert one side and remove the other side - // if we do not understand what the parent filters are. - // - // Here is an example used to explain the reason. - // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to - // convert b in ('1'). If we only convert a = 2, we will end up with a filter - // NOT(a = 2), which will generate wrong results. - // - // Pushing one side of AND down is only safe to do at the top level or in the child - // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate - // can be safely removed. - case And(left, right) => - val leftResultOptional = convertibleFiltersHelper(left, canPartialPushDown) - val rightResultOptional = convertibleFiltersHelper(right, canPartialPushDown) - (leftResultOptional, rightResultOptional) match { - case (Some(leftResult), Some(rightResult)) => Some(And(leftResult, rightResult)) - case (Some(leftResult), None) if canPartialPushDown => Some(leftResult) - case (None, Some(rightResult)) if canPartialPushDown => Some(rightResult) - case _ => None - } - - // The Or predicate is convertible when both of its children can be pushed down. - // That is to say, if one/both of the children can be partially pushed down, the Or - // predicate can be partially pushed down as well. - // - // Here is an example used to explain the reason. - // Let's say we have - // (a1 AND a2) OR (b1 AND b2), - // a1 and b1 is convertible, while a2 and b2 is not. - // The predicate can be converted as - // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) - // As per the logical in And predicate, we can push down (a1 OR b1). - case Or(left, right) => - for { - lhs <- convertibleFiltersHelper(left, canPartialPushDown) - rhs <- convertibleFiltersHelper(right, canPartialPushDown) - } yield Or(lhs, rhs) - case Not(pred) => - val childResultOptional = convertibleFiltersHelper(pred, canPartialPushDown = false) - childResultOptional.map(Not) - case other => - for (_ <- buildLeafSearchArgument(dataTypeMap, other, newBuilder())) yield other - } - filters.flatMap { filter => - convertibleFiltersHelper(filter, true) - } - } - - /** - * Get PredicateLeafType which is corresponding to the given DataType. - */ - def getPredicateLeafType(dataType: DataType): PredicateLeaf.Type = dataType match { - case BooleanType => PredicateLeaf.Type.BOOLEAN - case ByteType | ShortType | IntegerType | LongType => PredicateLeaf.Type.LONG - case FloatType | DoubleType => PredicateLeaf.Type.FLOAT - case StringType => PredicateLeaf.Type.STRING - case DateType => PredicateLeaf.Type.DATE - case TimestampType => PredicateLeaf.Type.TIMESTAMP - case _: DecimalType => PredicateLeaf.Type.DECIMAL - case _ => throw new UnsupportedOperationException(s"DataType: ${dataType.catalogString}") - } - - /** - * Cast literal values for filters. - * - * We need to cast to long because ORC raises exceptions - * at 'checkLiteralType' of SearchArgumentImpl.java. - */ - private def castLiteralValue(value: Any, dataType: DataType): Any = dataType match { - case ByteType | ShortType | IntegerType | LongType => - value.asInstanceOf[Number].longValue - case FloatType | DoubleType => - value.asInstanceOf[Number].doubleValue() - case _: DecimalType => - new HiveDecimalWritable(HiveDecimal.create(value.asInstanceOf[java.math.BigDecimal])) - case _: DateType if value.isInstanceOf[LocalDate] => - toJavaDate(localDateToDays(value.asInstanceOf[LocalDate])) - case _: TimestampType if value.isInstanceOf[Instant] => - toJavaTimestamp(instantToMicros(value.asInstanceOf[Instant])) - case _ => value - } - - /** - * Build a SearchArgument and return the builder so far. - * - * @param dataTypeMap a map from the attribute name to its data type. - * @param expression the input predicates, which should be fully convertible to SearchArgument. - * @param builder the input SearchArgument.Builder. - * @return the builder so far. - */ - private def buildSearchArgument( - dataTypeMap: Map[String, OrcPrimitiveField], - expression: Filter, - builder: Builder): Builder = { - import org.apache.spark.sql.sources._ - - expression match { - case And(left, right) => - val lhs = buildSearchArgument(dataTypeMap, left, builder.startAnd()) - val rhs = buildSearchArgument(dataTypeMap, right, lhs) - rhs.end() - - case Or(left, right) => - val lhs = buildSearchArgument(dataTypeMap, left, builder.startOr()) - val rhs = buildSearchArgument(dataTypeMap, right, lhs) - rhs.end() - - case Not(child) => - buildSearchArgument(dataTypeMap, child, builder.startNot()).end() - - case other => - buildLeafSearchArgument(dataTypeMap, other, builder).getOrElse { - throw new SparkException( - "The input filter of OrcFilters.buildSearchArgument should be fully convertible.") - } - } - } - - /** - * Build a SearchArgument for a leaf predicate and return the builder so far. - * - * @param dataTypeMap a map from the attribute name to its data type. - * @param expression the input filter predicates. - * @param builder the input SearchArgument.Builder. - * @return the builder so far. - */ - private def buildLeafSearchArgument( - dataTypeMap: Map[String, OrcPrimitiveField], - expression: Filter, - builder: Builder): Option[Builder] = { - def getType(attribute: String): PredicateLeaf.Type = - getPredicateLeafType(dataTypeMap(attribute).fieldType) - - import org.apache.spark.sql.sources._ - - // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` - // call is mandatory. ORC `SearchArgument` builder requires that all leaf predicates must be - // wrapped by a "parent" predicate (`And`, `Or`, or `Not`). - expression match { - case EqualTo(name, value) if dataTypeMap.contains(name) => - val castedValue = castLiteralValue(value, dataTypeMap(name).fieldType) - Some(builder.startAnd() - .equals(dataTypeMap(name).fieldName, getType(name), castedValue).end()) - - case EqualNullSafe(name, value) if dataTypeMap.contains(name) => - val castedValue = castLiteralValue(value, dataTypeMap(name).fieldType) - Some(builder.startAnd() - .nullSafeEquals(dataTypeMap(name).fieldName, getType(name), castedValue).end()) - - case LessThan(name, value) if dataTypeMap.contains(name) => - val castedValue = castLiteralValue(value, dataTypeMap(name).fieldType) - Some(builder.startAnd() - .lessThan(dataTypeMap(name).fieldName, getType(name), castedValue).end()) - - case LessThanOrEqual(name, value) if dataTypeMap.contains(name) => - val castedValue = castLiteralValue(value, dataTypeMap(name).fieldType) - Some(builder.startAnd() - .lessThanEquals(dataTypeMap(name).fieldName, getType(name), castedValue).end()) - - case GreaterThan(name, value) if dataTypeMap.contains(name) => - val castedValue = castLiteralValue(value, dataTypeMap(name).fieldType) - Some(builder.startNot() - .lessThanEquals(dataTypeMap(name).fieldName, getType(name), castedValue).end()) - - case GreaterThanOrEqual(name, value) if dataTypeMap.contains(name) => - val castedValue = castLiteralValue(value, dataTypeMap(name).fieldType) - Some(builder.startNot() - .lessThan(dataTypeMap(name).fieldName, getType(name), castedValue).end()) - - case IsNull(name) if dataTypeMap.contains(name) => - Some(builder.startAnd().isNull(dataTypeMap(name).fieldName, getType(name)).end()) - - case IsNotNull(name) if dataTypeMap.contains(name) => - Some(builder.startNot().isNull(dataTypeMap(name).fieldName, getType(name)).end()) - - case In(name, values) if dataTypeMap.contains(name) => - val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(name).fieldType)) - Some(builder.startAnd().in(dataTypeMap(name).fieldName, getType(name), - castedValues.map(_.asInstanceOf[AnyRef]): _*).end()) - - case _ => None - } - } -} - diff --git a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala deleted file mode 100644 index 7fbc1cd205b13..0000000000000 --- a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.orc - -import org.apache.orc.storage.common.`type`.HiveDecimal -import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch -import org.apache.orc.storage.ql.io.sarg.{SearchArgument => OrcSearchArgument} -import org.apache.orc.storage.ql.io.sarg.PredicateLeaf.{Operator => OrcOperator} -import org.apache.orc.storage.serde2.io.{DateWritable, HiveDecimalWritable} - -import org.apache.spark.sql.catalyst.expressions.SpecializedGetters -import org.apache.spark.sql.types.Decimal - -/** - * Various utilities for ORC used to upgrade the built-in Hive. - */ -private[sql] object OrcShimUtils { - - class VectorizedRowBatchWrap(val batch: VectorizedRowBatch) {} - - private[sql] type Operator = OrcOperator - private[sql] type SearchArgument = OrcSearchArgument - - def getGregorianDays(value: Any): Int = { - new DaysWritable(value.asInstanceOf[DateWritable]).gregorianDays - } - - def getDecimal(value: Any): Decimal = { - val decimal = value.asInstanceOf[HiveDecimalWritable].getHiveDecimal() - Decimal(decimal.bigDecimalValue, decimal.precision(), decimal.scale()) - } - - def getDateWritable(reuseObj: Boolean): (SpecializedGetters, Int) => DateWritable = { - if (reuseObj) { - val result = new DaysWritable() - (getter, ordinal) => - result.set(getter.getInt(ordinal)) - result - } else { - (getter: SpecializedGetters, ordinal: Int) => - new DaysWritable(getter.getInt(ordinal)) - } - } - - def getHiveDecimalWritable(precision: Int, scale: Int): - (SpecializedGetters, Int) => HiveDecimalWritable = { - (getter, ordinal) => - val d = getter.getDecimal(ordinal, precision, scale) - new HiveDecimalWritable(HiveDecimal.create(d.toJavaBigDecimal)) - } -} diff --git a/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala deleted file mode 100644 index e159a0588dfff..0000000000000 --- a/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ /dev/null @@ -1,676 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.orc - -import java.math.MathContext -import java.nio.charset.StandardCharsets -import java.sql.{Date, Timestamp} - -import scala.collection.JavaConverters._ - -import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} -import org.apache.orc.storage.ql.io.sarg.SearchArgumentFactory.newBuilder - -import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Row} -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation -import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types._ - -/** - * A test suite that tests Apache ORC filter API based filter pushdown optimization. - * OrcFilterSuite and HiveOrcFilterSuite is logically duplicated to provide the same test coverage. - * The difference are the packages containing 'Predicate' and 'SearchArgument' classes. - * - OrcFilterSuite uses 'org.apache.orc.storage.ql.io.sarg' package. - * - HiveOrcFilterSuite uses 'org.apache.hadoop.hive.ql.io.sarg' package. - */ -class OrcFilterSuite extends OrcTest with SharedSparkSession { - - override protected def sparkConf: SparkConf = - super - .sparkConf - .set(SQLConf.USE_V1_SOURCE_LIST, "") - - protected def checkFilterPredicate( - df: DataFrame, - predicate: Predicate, - checker: (SearchArgument) => Unit): Unit = { - val output = predicate.collect { case a: Attribute => a }.distinct - val query = df - .select(output.map(e => Column(e)): _*) - .where(Column(predicate)) - - query.queryExecution.optimizedPlan match { - case PhysicalOperation(_, filters, DataSourceV2ScanRelation(_, o: OrcScan, _)) => - assert(filters.nonEmpty, "No filter is analyzed from the given query") - assert(o.pushedFilters.nonEmpty, "No filter is pushed down") - val maybeFilter = OrcFilters.createFilter(query.schema, o.pushedFilters) - assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for ${o.pushedFilters}") - checker(maybeFilter.get) - - case _ => - throw new AnalysisException("Can not match OrcTable in the query.") - } - } - - protected def checkFilterPredicate - (predicate: Predicate, filterOperator: PredicateLeaf.Operator) - (implicit df: DataFrame): Unit = { - def checkComparisonOperator(filter: SearchArgument) = { - val operator = filter.getLeaves.asScala - assert(operator.map(_.getOperator).contains(filterOperator)) - } - checkFilterPredicate(df, predicate, checkComparisonOperator) - } - - protected def checkFilterPredicate - (predicate: Predicate, stringExpr: String) - (implicit df: DataFrame): Unit = { - def checkLogicalOperator(filter: SearchArgument) = { - assert(filter.toString == stringExpr) - } - checkFilterPredicate(df, predicate, checkLogicalOperator) - } - - test("filter pushdown - integer") { - withNestedOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { case (inputDF, colName, _) => - implicit val df: DataFrame = inputDF - - val intAttr = df(colName).expr - assert(df(colName).expr.dataType === IntegerType) - - checkFilterPredicate(intAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(intAttr === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(intAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(intAttr < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(intAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(intAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(intAttr >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === intAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> intAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > intAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < intAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= intAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= intAttr, PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - long") { - withNestedOrcDataFrame( - (1 to 4).map(i => Tuple1(Option(i.toLong)))) { case (inputDF, colName, _) => - implicit val df: DataFrame = inputDF - - val longAttr = df(colName).expr - assert(df(colName).expr.dataType === LongType) - - checkFilterPredicate(longAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(longAttr === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(longAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(longAttr < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(longAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(longAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(longAttr >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === longAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> longAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > longAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < longAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= longAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= longAttr, PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - float") { - withNestedOrcDataFrame( - (1 to 4).map(i => Tuple1(Option(i.toFloat)))) { case (inputDF, colName, _) => - implicit val df: DataFrame = inputDF - - val floatAttr = df(colName).expr - assert(df(colName).expr.dataType === FloatType) - - checkFilterPredicate(floatAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(floatAttr === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(floatAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(floatAttr < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(floatAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(floatAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(floatAttr >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === floatAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> floatAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > floatAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < floatAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= floatAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= floatAttr, PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - double") { - withNestedOrcDataFrame( - (1 to 4).map(i => Tuple1(Option(i.toDouble)))) { case (inputDF, colName, _) => - implicit val df: DataFrame = inputDF - - val doubleAttr = df(colName).expr - assert(df(colName).expr.dataType === DoubleType) - - checkFilterPredicate(doubleAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(doubleAttr === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(doubleAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(doubleAttr < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(doubleAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(doubleAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(doubleAttr >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === doubleAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> doubleAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > doubleAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < doubleAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= doubleAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= doubleAttr, PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - string") { - withNestedOrcDataFrame((1 to 4).map(i => Tuple1(i.toString))) { case (inputDF, colName, _) => - implicit val df: DataFrame = inputDF - - val strAttr = df(colName).expr - assert(df(colName).expr.dataType === StringType) - - checkFilterPredicate(strAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(strAttr === "1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(strAttr <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(strAttr < "2", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(strAttr > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(strAttr <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(strAttr >= "4", PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal("1") === strAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal("1") <=> strAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal("2") > strAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal("3") < strAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("1") >= strAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("4") <= strAttr, PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - boolean") { - withNestedOrcDataFrame( - (true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { case (inputDF, colName, _) => - implicit val df: DataFrame = inputDF - - val booleanAttr = df(colName).expr - assert(df(colName).expr.dataType === BooleanType) - - checkFilterPredicate(booleanAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(booleanAttr === true, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(booleanAttr <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(booleanAttr < true, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(booleanAttr > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(booleanAttr <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(booleanAttr >= false, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(false) === booleanAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(false) <=> booleanAttr, - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(false) > booleanAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(true) < booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) >= booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) <= booleanAttr, PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - decimal") { - withNestedOrcDataFrame( - (1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i)))) { case (inputDF, colName, _) => - implicit val df: DataFrame = inputDF - - val decimalAttr = df(colName).expr - assert(df(colName).expr.dataType === DecimalType(38, 18)) - - checkFilterPredicate(decimalAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(decimalAttr === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(decimalAttr <=> BigDecimal.valueOf(1), - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(decimalAttr < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(decimalAttr > BigDecimal.valueOf(3), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(decimalAttr <= BigDecimal.valueOf(1), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(decimalAttr >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) === decimalAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) <=> decimalAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(2)) > decimalAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate( - Literal(BigDecimal.valueOf(3)) < decimalAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) >= decimalAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(4)) <= decimalAttr, PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - timestamp") { - val input = Seq( - "1000-01-01 01:02:03", - "1582-10-01 00:11:22", - "1900-01-01 23:59:59", - "2020-05-25 10:11:12").map(Timestamp.valueOf) - - withOrcFile(input.map(Tuple1(_))) { path => - Seq(false, true).foreach { java8Api => - withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString) { - readFile(path) { implicit df => - val timestamps = input.map(Literal(_)) - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === timestamps(0), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < timestamps(1), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(timestamps(0)) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate( - Literal(timestamps(0)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(timestamps(1)) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate( - Literal(timestamps(2)) < $"_1", - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate( - Literal(timestamps(0)) >= $"_1", - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(3)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) - } - } - } - } - } - - test("filter pushdown - combinations with logical operators") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => - checkFilterPredicate( - $"_1".isNotNull, - "leaf-0 = (IS_NULL _1), expr = (not leaf-0)" - ) - checkFilterPredicate( - $"_1" =!= 1, - "leaf-0 = (IS_NULL _1), leaf-1 = (EQUALS _1 1), expr = (and (not leaf-0) (not leaf-1))" - ) - checkFilterPredicate( - !($"_1" < 4), - "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 4), expr = (and (not leaf-0) (not leaf-1))" - ) - checkFilterPredicate( - $"_1" < 2 || $"_1" > 3, - "leaf-0 = (LESS_THAN _1 2), leaf-1 = (LESS_THAN_EQUALS _1 3), " + - "expr = (or leaf-0 (not leaf-1))" - ) - checkFilterPredicate( - $"_1" < 2 && $"_1" > 3, - "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 2), leaf-2 = (LESS_THAN_EQUALS _1 3), " + - "expr = (and (not leaf-0) leaf-1 (not leaf-2))" - ) - } - } - - test("filter pushdown - date") { - val input = Seq("2017-08-18", "2017-08-19", "2017-08-20", "2017-08-21").map { day => - Date.valueOf(day) - } - withOrcFile(input.map(Tuple1(_))) { path => - Seq(false, true).foreach { java8Api => - withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString) { - readFile(path) { implicit df => - val dates = input.map(Literal(_)) - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === dates(0), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> dates(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < dates(1), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > dates(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= dates(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= dates(3), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(dates(0) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(dates(0) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(dates(1) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(dates(2) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(dates(0) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(dates(3) <= $"_1", PredicateLeaf.Operator.LESS_THAN) - } - } - } - } - } - - test("no filter pushdown - non-supported types") { - implicit class IntToBinary(int: Int) { - def b: Array[Byte] = int.toString.getBytes(StandardCharsets.UTF_8) - } - // ArrayType - withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => - checkNoFilterPredicate($"_1".isNull, noneSupported = true) - } - // BinaryType - withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => - checkNoFilterPredicate($"_1" <=> 1.b, noneSupported = true) - } - // MapType - withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => - checkNoFilterPredicate($"_1".isNotNull, noneSupported = true) - } - } - - test("SPARK-12218 and SPARK-25699 Converting conjunctions into ORC SearchArguments") { - import org.apache.spark.sql.sources._ - // The `LessThan` should be converted while the `StringContains` shouldn't - val schema = new StructType( - Array( - StructField("a", IntegerType, nullable = true), - StructField("b", StringType, nullable = true))) - assertResult("leaf-0 = (LESS_THAN a 10), expr = leaf-0") { - OrcFilters.createFilter(schema, Array( - LessThan("a", 10), - StringContains("b", "prefix") - )).get.toString - } - - // The `LessThan` should be converted while the whole inner `And` shouldn't - assertResult("leaf-0 = (LESS_THAN a 10), expr = leaf-0") { - OrcFilters.createFilter(schema, Array( - LessThan("a", 10), - Not(And( - GreaterThan("a", 1), - StringContains("b", "prefix") - )) - )).get.toString - } - - // Safely remove unsupported `StringContains` predicate and push down `LessThan` - assertResult("leaf-0 = (LESS_THAN a 10), expr = leaf-0") { - OrcFilters.createFilter(schema, Array( - And( - LessThan("a", 10), - StringContains("b", "prefix") - ) - )).get.toString - } - - // Safely remove unsupported `StringContains` predicate, push down `LessThan` and `GreaterThan`. - assertResult("leaf-0 = (LESS_THAN a 10), leaf-1 = (LESS_THAN_EQUALS a 1)," + - " expr = (and leaf-0 (not leaf-1))") { - OrcFilters.createFilter(schema, Array( - And( - And( - LessThan("a", 10), - StringContains("b", "prefix") - ), - GreaterThan("a", 1) - ) - )).get.toString - } - } - - test("SPARK-27699 Converting disjunctions into ORC SearchArguments") { - import org.apache.spark.sql.sources._ - // The `LessThan` should be converted while the `StringContains` shouldn't - val schema = new StructType( - Array( - StructField("a", IntegerType, nullable = true), - StructField("b", StringType, nullable = true))) - - // The predicate `StringContains` predicate is not able to be pushed down. - assertResult("leaf-0 = (LESS_THAN_EQUALS a 10), leaf-1 = (LESS_THAN a 1)," + - " expr = (or (not leaf-0) leaf-1)") { - OrcFilters.createFilter(schema, Array( - Or( - GreaterThan("a", 10), - And( - StringContains("b", "prefix"), - LessThan("a", 1) - ) - ) - )).get.toString - } - - assertResult("leaf-0 = (LESS_THAN_EQUALS a 10), leaf-1 = (LESS_THAN a 1)," + - " expr = (or (not leaf-0) leaf-1)") { - OrcFilters.createFilter(schema, Array( - Or( - And( - GreaterThan("a", 10), - StringContains("b", "foobar") - ), - And( - StringContains("b", "prefix"), - LessThan("a", 1) - ) - ) - )).get.toString - } - - assert(OrcFilters.createFilter(schema, Array( - Or( - StringContains("b", "foobar"), - And( - StringContains("b", "prefix"), - LessThan("a", 1) - ) - ) - )).isEmpty) - } - - test("SPARK-27160: Fix casting of the DecimalType literal") { - import org.apache.spark.sql.sources._ - val schema = StructType(Array(StructField("a", DecimalType(3, 2)))) - assertResult("leaf-0 = (LESS_THAN a 3.14), expr = leaf-0") { - OrcFilters.createFilter(schema, Array( - LessThan( - "a", - new java.math.BigDecimal(3.14, MathContext.DECIMAL64).setScale(2))) - ).get.toString - } - } - - test("SPARK-32622: case sensitivity in predicate pushdown") { - withTempPath { dir => - val count = 10 - val tableName = "spark_32622" - val tableDir1 = dir.getAbsoluteFile + "/table1" - - // Physical ORC files have both `A` and `a` fields. - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - spark.range(count).repartition(count).selectExpr("id - 1 as A", "id as a") - .write.mode("overwrite").orc(tableDir1) - } - - // Metastore table has both `A` and `a` fields too. - withTable(tableName) { - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - sql( - s""" - |CREATE TABLE $tableName (A LONG, a LONG) USING ORC LOCATION '$tableDir1' - """.stripMargin) - - checkAnswer(sql(s"select a, A from $tableName"), (0 until count).map(c => Row(c, c - 1))) - - val actual1 = stripSparkFilter(sql(s"select A from $tableName where A < 0")) - assert(actual1.count() == 1) - - val actual2 = stripSparkFilter(sql(s"select A from $tableName where a < 0")) - assert(actual2.count() == 0) - } - - // Exception thrown for ambiguous case. - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { - val e = intercept[AnalysisException] { - sql(s"select a from $tableName where a < 0").collect() - } - assert(e.getMessage.contains( - "Reference 'a' is ambiguous")) - } - } - - // Metastore table has only `A` field. - withTable(tableName) { - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { - sql( - s""" - |CREATE TABLE $tableName (A LONG) USING ORC LOCATION '$tableDir1' - """.stripMargin) - - val e = intercept[SparkException] { - sql(s"select A from $tableName where A < 0").collect() - } - assert(e.getCause.isInstanceOf[RuntimeException] && e.getCause.getMessage.contains( - """Found duplicate field(s) "A": [A, a] in case-insensitive mode""")) - } - } - - // Physical ORC files have only `A` field. - val tableDir2 = dir.getAbsoluteFile + "/table2" - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - spark.range(count).repartition(count).selectExpr("id - 1 as A") - .write.mode("overwrite").orc(tableDir2) - } - - withTable(tableName) { - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { - sql( - s""" - |CREATE TABLE $tableName (a LONG) USING ORC LOCATION '$tableDir2' - """.stripMargin) - - checkAnswer(sql(s"select a from $tableName"), (0 until count).map(c => Row(c - 1))) - - val actual = stripSparkFilter(sql(s"select a from $tableName where a < 0")) - assert(actual.count() == 1) - } - } - - withTable(tableName) { - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - sql( - s""" - |CREATE TABLE $tableName (A LONG) USING ORC LOCATION '$tableDir2' - """.stripMargin) - - checkAnswer(sql(s"select A from $tableName"), (0 until count).map(c => Row(c - 1))) - - val actual = stripSparkFilter(sql(s"select A from $tableName where A < 0")) - assert(actual.count() == 1) - } - } - } - } - - test("SPARK-32646: Case-insensitive field resolution for pushdown when reading ORC") { - import org.apache.spark.sql.sources._ - - def getOrcFilter( - schema: StructType, - filters: Seq[Filter], - caseSensitive: String): Option[SearchArgument] = { - var orcFilter: Option[SearchArgument] = None - withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { - orcFilter = - OrcFilters.createFilter(schema, filters) - } - orcFilter - } - - def testFilter( - schema: StructType, - filters: Seq[Filter], - expected: SearchArgument): Unit = { - val caseSensitiveFilters = getOrcFilter(schema, filters, "true") - val caseInsensitiveFilters = getOrcFilter(schema, filters, "false") - - assert(caseSensitiveFilters.isEmpty) - assert(caseInsensitiveFilters.isDefined) - - assert(caseInsensitiveFilters.get.getLeaves().size() > 0) - assert(caseInsensitiveFilters.get.getLeaves().size() == expected.getLeaves().size()) - (0 until expected.getLeaves().size()).foreach { index => - assert(caseInsensitiveFilters.get.getLeaves().get(index) == expected.getLeaves().get(index)) - } - } - - val schema1 = StructType(Seq(StructField("cint", IntegerType))) - testFilter(schema1, Seq(GreaterThan("CINT", 1)), - newBuilder.startNot() - .lessThanEquals("cint", OrcFilters.getPredicateLeafType(IntegerType), 1L).`end`().build()) - testFilter(schema1, Seq( - And(GreaterThan("CINT", 1), EqualTo("Cint", 2))), - newBuilder.startAnd() - .startNot() - .lessThanEquals("cint", OrcFilters.getPredicateLeafType(IntegerType), 1L).`end`() - .equals("cint", OrcFilters.getPredicateLeafType(IntegerType), 2L) - .`end`().build()) - - // Nested column case - val schema2 = StructType(Seq(StructField("a", - StructType(Seq(StructField("cint", IntegerType)))))) - - testFilter(schema2, Seq(GreaterThan("A.CINT", 1)), - newBuilder.startNot() - .lessThanEquals("a.cint", OrcFilters.getPredicateLeafType(IntegerType), 1L).`end`().build()) - testFilter(schema2, Seq(GreaterThan("a.CINT", 1)), - newBuilder.startNot() - .lessThanEquals("a.cint", OrcFilters.getPredicateLeafType(IntegerType), 1L).`end`().build()) - testFilter(schema2, Seq(GreaterThan("A.cint", 1)), - newBuilder.startNot() - .lessThanEquals("a.cint", OrcFilters.getPredicateLeafType(IntegerType), 1L).`end`().build()) - testFilter(schema2, Seq( - And(GreaterThan("a.CINT", 1), EqualTo("a.Cint", 2))), - newBuilder.startAnd() - .startNot() - .lessThanEquals("a.cint", OrcFilters.getPredicateLeafType(IntegerType), 1L).`end`() - .equals("a.cint", OrcFilters.getPredicateLeafType(IntegerType), 2L) - .`end`().build()) - } -} - diff --git a/sql/hive-thriftserver/v2.3/if/TCLIService.thrift b/sql/hive-thriftserver/if/TCLIService.thrift similarity index 100% rename from sql/hive-thriftserver/v2.3/if/TCLIService.thrift rename to sql/hive-thriftserver/if/TCLIService.thrift diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 5bf20b209aff7..4a96afe9df20a 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -146,9 +146,7 @@ - v${hive.version.short}/src/gen/java - v${hive.version.short}/src/main/java - v${hive.version.short}/src/main/scala + src/gen/java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/AbstractService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/AbstractService.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/CompositeService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/CompositeService.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/CookieSigner.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/CookieSigner.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/ServiceOperations.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/ServiceOperations.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/ServiceUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/ServiceUtils.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/CLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/CLIService.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnValue.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnValue.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/FetchOrientation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchOrientation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/FetchOrientation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchOrientation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/GetInfoType.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoType.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/GetInfoType.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoType.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/GetInfoValue.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/GetInfoValue.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoValue.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/Handle.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Handle.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/Handle.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Handle.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/HiveSQLException.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/HiveSQLException.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ICLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ICLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ICLIService.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ICLIService.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationHandle.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationHandle.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationHandle.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationHandle.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationState.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationState.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationType.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationType.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationType.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationType.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowBasedSet.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSet.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowSet.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSet.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowSetFactory.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSetFactory.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowSetFactory.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSetFactory.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/SessionHandle.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/SessionHandle.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/SessionHandle.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/SessionHandle.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TableSchema.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TableSchema.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TableSchema.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TableSchema.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/Operation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSession.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSession.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSession.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSession.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/SessionManager.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/server/HiveServer2.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/server/HiveServer2.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index d30951f89cf6b..ec2c795e95c83 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -118,8 +118,7 @@ private[hive] class SparkExecuteStatementOperation( validateDefaultFetchOrientation(order) assertState(OperationState.FINISHED) setHasResultSet(true) - val resultRowSet: RowSet = - ThriftserverShimUtils.resultRowSet(getResultSetSchema, getProtocolVersion) + val resultRowSet: RowSet = RowSetFactory.create(getResultSetSchema, getProtocolVersion, false) // Reset iter when FETCH_FIRST or FETCH_PRIOR if ((order.equals(FetchOrientation.FETCH_FIRST) || diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index 0a46c837183e8..88aebb36633f6 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -130,7 +130,8 @@ private[hive] class SparkGetColumnsOperation( * For array, map, string, and binaries, the column size is variable, return null as unknown. */ private def getColumnSize(typ: DataType): Option[Int] = typ match { - case dt @ (BooleanType | _: NumericType | DateType | TimestampType | CalendarIntervalType) => + case dt @ (BooleanType | _: NumericType | DateType | TimestampType | + CalendarIntervalType | NullType) => Some(dt.defaultSize) case StructType(fields) => val sizeArr = fields.map(f => getColumnSize(f.dataType)) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index 16fd502048e80..e58357a415545 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -77,7 +77,8 @@ private[hive] class SparkGetSchemasOperation( val globalTempViewDb = sqlContext.sessionState.catalog.globalTempViewManager.database val databasePattern = Pattern.compile(CLIServiceUtils.patternToRegex(schemaName)) - if (databasePattern.matcher(globalTempViewDb).matches()) { + if (schemaName == null || schemaName.isEmpty || + databasePattern.matcher(globalTempViewDb).matches()) { rowSet.addRow(Array[AnyRef](globalTempViewDb, DEFAULT_HIVE_CATALOG)) } setState(OperationState.FINISHED) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala index 0d4b9b392f074..bccad865be27a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -125,10 +125,6 @@ private[hive] class SparkGetTablesOperation( tableType, comment.getOrElse("")) // Since HIVE-7575(Hive 2.0.0), adds 5 additional columns to the ResultSet of GetTables. - if (HiveUtils.isHive23) { - rowSet.addRow(rowData ++ Array(null, null, null, null, null)) - } else { - rowSet.addRow(rowData) - } + rowSet.addRow(rowData ++ Array(null, null, null, null, null)) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala index c2568ad4ada0a..26b5f8ad8cee1 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.hive.thriftserver import java.util.UUID import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType +import org.apache.hadoop.hive.serde2.thrift.Type +import org.apache.hadoop.hive.serde2.thrift.Type._ import org.apache.hive.service.cli.OperationState import org.apache.hive.service.cli.operation.GetTypeInfoOperation import org.apache.hive.service.cli.session.HiveSession @@ -61,7 +63,7 @@ private[hive] class SparkGetTypeInfoOperation( parentSession.getUsername) try { - ThriftserverShimUtils.supportedType().foreach(typeInfo => { + SparkGetTypeInfoUtil.supportedType.foreach(typeInfo => { val rowData = Array[AnyRef]( typeInfo.getName, // TYPE_NAME typeInfo.toJavaSQLType.asInstanceOf[AnyRef], // DATA_TYPE @@ -90,3 +92,13 @@ private[hive] class SparkGetTypeInfoOperation( HiveThriftServer2.eventManager.onStatementFinish(statementId) } } + +private[hive] object SparkGetTypeInfoUtil { + val supportedType: Seq[Type] = { + Seq(NULL_TYPE, BOOLEAN_TYPE, STRING_TYPE, BINARY_TYPE, + TINYINT_TYPE, SMALLINT_TYPE, INT_TYPE, BIGINT_TYPE, + FLOAT_TYPE, DOUBLE_TYPE, DECIMAL_TYPE, + DATE_TYPE, TIMESTAMP_TYPE, + ARRAY_TYPE, MAP_TYPE, STRUCT_TYPE) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 99aa623441206..965f28ebe0840 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -35,7 +35,9 @@ import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.log4j.Level import org.apache.thrift.transport.TSocket +import org.slf4j.LoggerFactory import sun.misc.{Signal, SignalHandler} import org.apache.spark.SparkConf @@ -306,7 +308,9 @@ private[hive] object SparkSQLCLIDriver extends Logging { private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { private val sessionState = SessionState.get().asInstanceOf[CliSessionState] - private val console = ThriftserverShimUtils.getConsole + private val LOG = LoggerFactory.getLogger(classOf[SparkSQLCLIDriver]) + + private val console = new SessionState.LogHelper(LOG) private val isRemoteMode = { SparkSQLCLIDriver.isRemoteMode(sessionState) @@ -319,6 +323,9 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { // because the Hive unit tests do not go through the main() code path. if (!isRemoteMode) { SparkSQLEnv.init() + if (sessionState.getIsSilent) { + SparkSQLEnv.sparkContext.setLogLevel(Level.WARN.toString) + } } else { // Hive 1.2 + not supported in CLI throw new RuntimeException("Remote operations not supported") diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala index 984625c76e057..c39d2ecdd7923 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -113,17 +113,10 @@ private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, sqlContext: SQLC private[thriftserver] trait ReflectedCompositeService { this: AbstractService => - private val logInfo = (msg: String) => if (HiveUtils.isHive23) { - getAncestorField[Logger](this, 3, "LOG").info(msg) - } else { - getAncestorField[Log](this, 3, "LOG").info(msg) - } + private val logInfo = (msg: String) => getAncestorField[Logger](this, 3, "LOG").info(msg) - private val logError = (msg: String, e: Throwable) => if (HiveUtils.isHive23) { + private val logError = (msg: String, e: Throwable) => getAncestorField[Logger](this, 3, "LOG").error(msg, e) - } else { - getAncestorField[Log](this, 3, "LOG").error(msg, e) - } def initCompositeService(hiveConf: HiveConf): Unit = { // Emulating `CompositeService.init(hiveConf)` diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index 806b6146b2db1..e4559e69e7585 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.service.cli.SessionHandle import org.apache.hive.service.cli.session.SessionManager +import org.apache.hive.service.rpc.thrift.TProtocolVersion import org.apache.hive.service.server.HiveServer2 import org.apache.spark.sql.SQLContext @@ -45,7 +46,7 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: } override def openSession( - protocol: ThriftserverShimUtils.TProtocolVersion, + protocol: TProtocolVersion, username: String, passwd: String, ipAddress: String, diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreLazyInitializationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreLazyInitializationSuite.scala index 277df548aefd0..951f92793732f 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreLazyInitializationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreLazyInitializationSuite.scala @@ -31,6 +31,7 @@ class HiveMetastoreLazyInitializationSuite extends SparkFunSuite { .config("spark.hadoop.hive.metastore.uris", "thrift://127.0.0.1:11111") .getOrCreate() val originalLevel = org.apache.log4j.Logger.getRootLogger().getLevel + val originalClassLoader = Thread.currentThread().getContextClassLoader try { // Avoid outputting a lot of expected warning logs spark.sparkContext.setLogLevel("error") @@ -64,6 +65,7 @@ class HiveMetastoreLazyInitializationSuite extends SparkFunSuite { exceptionString.contains(msg) } } finally { + Thread.currentThread().setContextClassLoader(originalClassLoader) spark.sparkContext.setLogLevel(originalLevel.toString) spark.stop() } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala index 764f1690d5a66..1bc9aaf672c3b 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala @@ -22,8 +22,7 @@ import java.util.UUID import org.apache.hive.service.cli.OperationHandle import org.apache.hive.service.cli.operation.GetCatalogsOperation import org.apache.hive.service.cli.session.HiveSession - -import org.apache.spark.sql.hive.thriftserver.ThriftserverShimUtils.{THandleIdentifier, TOperationHandle, TOperationType} +import org.apache.hive.service.rpc.thrift.{THandleIdentifier, TOperationHandle, TOperationType} class GetCatalogsOperationMock(parentSession: HiveSession) extends GetCatalogsOperation(parentSession) { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala index 47db7e34a5a2c..13dc74b92d4b3 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hive.service.cli.OperationHandle import org.apache.hive.service.cli.operation.{GetCatalogsOperation, Operation, OperationManager} import org.apache.hive.service.cli.session.{HiveSession, HiveSessionImpl, SessionManager} +import org.apache.hive.service.rpc.thrift.TProtocolVersion import org.apache.spark.SparkFunSuite @@ -39,7 +40,7 @@ class HiveSessionImplSuite extends SparkFunSuite { operationManager = new OperationManagerMock() session = new HiveSessionImpl( - ThriftserverShimUtils.testedProtocolVersions.head, + TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V1, "", "", new HiveConf(), diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index ad0f97cae3f8e..75c00000dee47 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -37,6 +37,7 @@ import org.apache.hive.jdbc.HiveDriver import org.apache.hive.service.auth.PlainSaslHelper import org.apache.hive.service.cli.{FetchOrientation, FetchType, GetInfoType, RowSet} import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient +import org.apache.hive.service.rpc.thrift.TCLIService.Client import org.apache.thrift.protocol.TBinaryProtocol import org.apache.thrift.transport.TSocket import org.scalatest.BeforeAndAfterAll @@ -67,7 +68,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { val user = System.getProperty("user.name") val transport = PlainSaslHelper.getPlainTransport(user, "anonymous", rawTransport) val protocol = new TBinaryProtocol(transport) - val client = new ThriftCLIServiceClient(new ThriftserverShimUtils.Client(protocol)) + val client = new ThriftCLIServiceClient(new Client(protocol)) transport.open() try f(client) finally transport.close() @@ -544,11 +545,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { conf += resultSet.getString(1) -> resultSet.getString(2) } - if (HiveUtils.isHive23) { - assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("2.3.7")) - } else { - assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("1.2.1")) - } + assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("2.3.7")) } } @@ -561,11 +558,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { conf += resultSet.getString(1) -> resultSet.getString(2) } - if (HiveUtils.isHive23) { - assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("2.3.7")) - } else { - assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("1.2.1")) - } + assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("2.3.7")) } } @@ -643,11 +636,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { val sessionHandle = client.openSession(user, "") val sessionID = sessionHandle.getSessionId - if (HiveUtils.isHive23) { - assert(pipeoutFileList(sessionID).length == 2) - } else { - assert(pipeoutFileList(sessionID).length == 1) - } + assert(pipeoutFileList(sessionID).length == 2) client.closeSession(sessionHandle) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala index 5f17607585521..8f61268c838fe 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala @@ -31,6 +31,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.jdbc.HttpBasicAuthInterceptor import org.apache.hive.service.auth.PlainSaslHelper import org.apache.hive.service.cli.thrift.{ThriftCLIService, ThriftCLIServiceClient} +import org.apache.hive.service.rpc.thrift.TCLIService.Client import org.apache.http.impl.client.HttpClientBuilder import org.apache.thrift.protocol.TBinaryProtocol import org.apache.thrift.transport.{THttpClient, TSocket} @@ -115,7 +116,7 @@ trait SharedThriftServer extends SharedSparkSession { } val protocol = new TBinaryProtocol(transport) - val client = new ThriftCLIServiceClient(new ThriftserverShimUtils.Client(protocol)) + val client = new ThriftCLIServiceClient(new Client(protocol)) transport.open() try f(client) finally transport.close() diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala index 4c2f29e0bf394..ca1f9a2f74244 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala @@ -25,6 +25,7 @@ import scala.concurrent.duration._ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hive.service.cli.OperationState import org.apache.hive.service.cli.session.{HiveSession, HiveSessionImpl} +import org.apache.hive.service.rpc.thrift.TProtocolVersion import org.mockito.Mockito.{doReturn, mock, spy, when, RETURNS_DEEP_STUBS} import org.mockito.invocation.InvocationOnMock @@ -64,7 +65,7 @@ class SparkExecuteStatementOperationSuite extends SparkFunSuite with SharedSpark ).foreach { case (finalState, transition) => test("SPARK-32057 SparkExecuteStatementOperation should not transiently become ERROR " + s"before being set to $finalState") { - val hiveSession = new HiveSessionImpl(ThriftserverShimUtils.testedProtocolVersions.head, + val hiveSession = new HiveSessionImpl(TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V1, "username", "password", new HiveConf, "ip address") hiveSession.open(new util.HashMap) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala index 7fefb1ee6a415..b94d819326d16 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.hive.thriftserver import java.sql.{DatabaseMetaData, ResultSet} +import org.apache.hive.service.cli.HiveSQLException + import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.types._ @@ -28,23 +30,40 @@ class SparkMetadataOperationSuite extends HiveThriftJdbcTest { test("Spark's own GetSchemasOperation(SparkGetSchemasOperation)") { def checkResult(rs: ResultSet, dbNames: Seq[String]): Unit = { - for (i <- dbNames.indices) { - assert(rs.next()) - assert(rs.getString("TABLE_SCHEM") === dbNames(i)) + val expected = dbNames.iterator + while(rs.next() || expected.hasNext) { + assert(rs.getString("TABLE_SCHEM") === expected.next) + assert(rs.getString("TABLE_CATALOG").isEmpty) } // Make sure there are no more elements assert(!rs.next()) + assert(!expected.hasNext, "All expected schemas should be visited") } - withDatabase("db1", "db2") { statement => - Seq("CREATE DATABASE db1", "CREATE DATABASE db2").foreach(statement.execute) - + val dbs = Seq("db1", "db2", "db33", "db44") + val dbDflts = Seq("default", "global_temp") + withDatabase(dbs: _*) { statement => + dbs.foreach( db => statement.execute(s"CREATE DATABASE IF NOT EXISTS $db")) val metaData = statement.getConnection.getMetaData - checkResult(metaData.getSchemas(null, "%"), Seq("db1", "db2", "default", "global_temp")) + Seq("", "%", null, ".*", "_*", "_%", ".%") foreach { pattern => + checkResult(metaData.getSchemas(null, pattern), dbs ++ dbDflts) + } + + Seq("db%", "db*") foreach { pattern => + checkResult(metaData.getSchemas(null, pattern), dbs) + } + + Seq("db_", "db.") foreach { pattern => + checkResult(metaData.getSchemas(null, pattern), dbs.take(2)) + } + checkResult(metaData.getSchemas(null, "db1"), Seq("db1")) checkResult(metaData.getSchemas(null, "db_not_exist"), Seq.empty) - checkResult(metaData.getSchemas(null, "db*"), Seq("db1", "db2")) + + val e = intercept[HiveSQLException](metaData.getSchemas(null, "*")) + assert(e.getCause.getMessage === + "Error operating GET_SCHEMAS Dangling meta character '*' near index 0\n*\n^") } } @@ -236,7 +255,7 @@ class SparkMetadataOperationSuite extends HiveThriftJdbcTest { withJdbcStatement() { statement => val metaData = statement.getConnection.getMetaData - checkResult(metaData.getTypeInfo, ThriftserverShimUtils.supportedType().map(_.getName)) + checkResult(metaData.getTypeInfo, SparkGetTypeInfoUtil.supportedType.map(_.getName)) } } @@ -350,4 +369,31 @@ class SparkMetadataOperationSuite extends HiveThriftJdbcTest { } } } + + test("handling null in view for get columns operations") { + val viewName = "view_null" + val ddl = s"CREATE GLOBAL TEMP VIEW $viewName as select null as n" + + withJdbcStatement(viewName) { statement => + statement.execute(ddl) + val data = statement.getConnection.getMetaData + val rowSet = data.getColumns("", "global_temp", viewName, "n") + while (rowSet.next()) { + assert(rowSet.getString("TABLE_CAT") === null) + assert(rowSet.getString("TABLE_SCHEM") === "global_temp") + assert(rowSet.getString("TABLE_NAME") === viewName) + assert(rowSet.getString("COLUMN_NAME") === "n") + assert(rowSet.getInt("DATA_TYPE") === java.sql.Types.NULL) + assert(rowSet.getString("TYPE_NAME").equalsIgnoreCase(NullType.sql)) + assert(rowSet.getInt("COLUMN_SIZE") === 1) + assert(rowSet.getInt("DECIMAL_DIGITS") === 0) + assert(rowSet.getInt("NUM_PREC_RADIX") === 0) + assert(rowSet.getInt("NULLABLE") === 1) + assert(rowSet.getString("REMARKS") === "") + assert(rowSet.getInt("ORDINAL_POSITION") === 0) + assert(rowSet.getString("IS_NULLABLE") === "YES") + assert(rowSet.getString("IS_AUTO_INCREMENT") === "NO") + } + } + } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala index fd45e7a48c0eb..52cf429441d16 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala @@ -23,6 +23,8 @@ import java.util.{List => JList, Properties} import org.apache.hive.jdbc.{HiveConnection, HiveQueryResultSet} import org.apache.hive.service.auth.PlainSaslHelper import org.apache.hive.service.cli.GetInfoType +import org.apache.hive.service.rpc.thrift.{TExecuteStatementReq, TGetInfoReq, TGetTablesReq, TOpenSessionReq, TProtocolVersion} +import org.apache.hive.service.rpc.thrift.TCLIService.Client import org.apache.thrift.protocol.TBinaryProtocol import org.apache.thrift.transport.TSocket @@ -34,20 +36,20 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { override def mode: ServerMode.Value = ServerMode.binary def testExecuteStatementWithProtocolVersion( - version: ThriftserverShimUtils.TProtocolVersion, + version: TProtocolVersion, sql: String)(f: HiveQueryResultSet => Unit): Unit = { val rawTransport = new TSocket("localhost", serverPort) val connection = new HiveConnection(s"jdbc:hive2://localhost:$serverPort", new Properties) val user = System.getProperty("user.name") val transport = PlainSaslHelper.getPlainTransport(user, "anonymous", rawTransport) - val client = new ThriftserverShimUtils.Client(new TBinaryProtocol(transport)) + val client = new Client(new TBinaryProtocol(transport)) transport.open() var rs: HiveQueryResultSet = null try { - val clientProtocol = new ThriftserverShimUtils.TOpenSessionReq(version) + val clientProtocol = new TOpenSessionReq(version) val openResp = client.OpenSession(clientProtocol) val sessHandle = openResp.getSessionHandle - val execReq = new ThriftserverShimUtils.TExecuteStatementReq(sessHandle, sql) + val execReq = new TExecuteStatementReq(sessHandle, sql) val execResp = client.ExecuteStatement(execReq) val stmtHandle = execResp.getOperationHandle @@ -72,23 +74,21 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { } } - def testGetInfoWithProtocolVersion(version: ThriftserverShimUtils.TProtocolVersion): Unit = { + def testGetInfoWithProtocolVersion(version: TProtocolVersion): Unit = { val rawTransport = new TSocket("localhost", serverPort) val connection = new HiveConnection(s"jdbc:hive2://localhost:$serverPort", new Properties) val transport = PlainSaslHelper.getPlainTransport(user, "anonymous", rawTransport) - val client = new ThriftserverShimUtils.Client(new TBinaryProtocol(transport)) + val client = new Client(new TBinaryProtocol(transport)) transport.open() try { - val clientProtocol = new ThriftserverShimUtils.TOpenSessionReq(version) + val clientProtocol = new TOpenSessionReq(version) val openResp = client.OpenSession(clientProtocol) val sessHandle = openResp.getSessionHandle - val dbVersionReq = - new ThriftserverShimUtils.TGetInfoReq(sessHandle, GetInfoType.CLI_DBMS_VER.toTGetInfoType) + val dbVersionReq = new TGetInfoReq(sessHandle, GetInfoType.CLI_DBMS_VER.toTGetInfoType) val dbVersion = client.GetInfo(dbVersionReq).getInfoValue.getStringValue - val dbNameReq = - new ThriftserverShimUtils.TGetInfoReq(sessHandle, GetInfoType.CLI_DBMS_NAME.toTGetInfoType) + val dbNameReq = new TGetInfoReq(sessHandle, GetInfoType.CLI_DBMS_NAME.toTGetInfoType) val dbName = client.GetInfo(dbNameReq).getInfoValue.getStringValue assert(dbVersion === org.apache.spark.SPARK_VERSION) @@ -101,21 +101,21 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { } def testGetTablesWithProtocolVersion( - version: ThriftserverShimUtils.TProtocolVersion, + version: TProtocolVersion, schema: String, tableNamePattern: String, tableTypes: JList[String])(f: HiveQueryResultSet => Unit): Unit = { val rawTransport = new TSocket("localhost", serverPort) val connection = new HiveConnection(s"jdbc:hive2://localhost:$serverPort", new Properties) val transport = PlainSaslHelper.getPlainTransport(user, "anonymous", rawTransport) - val client = new ThriftserverShimUtils.Client(new TBinaryProtocol(transport)) + val client = new Client(new TBinaryProtocol(transport)) transport.open() var rs: HiveQueryResultSet = null try { - val clientProtocol = new ThriftserverShimUtils.TOpenSessionReq(version) + val clientProtocol = new TOpenSessionReq(version) val openResp = client.OpenSession(clientProtocol) val sessHandle = openResp.getSessionHandle - val getTableReq = new ThriftserverShimUtils.TGetTablesReq(sessHandle) + val getTableReq = new TGetTablesReq(sessHandle) getTableReq.setSchemaName(schema) getTableReq.setTableName(tableNamePattern) getTableReq.setTableTypes(tableTypes) @@ -143,11 +143,17 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { } } - ThriftserverShimUtils.testedProtocolVersions.foreach { version => + TProtocolVersion.values().foreach { version => test(s"$version get byte type") { testExecuteStatementWithProtocolVersion(version, "SELECT cast(1 as byte)") { rs => assert(rs.next()) assert(rs.getByte(1) === 1.toByte) + val metaData = rs.getMetaData + assert(metaData.getColumnName(1) === "CAST(1 AS TINYINT)") + assert(metaData.getColumnTypeName(1) === "tinyint") + assert(metaData.getColumnType(1) === java.sql.Types.TINYINT) + assert(metaData.getPrecision(1) === 3) + assert(metaData.getScale(1) === 0) } } @@ -155,6 +161,12 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { testExecuteStatementWithProtocolVersion(version, "SELECT cast(1 as short)") { rs => assert(rs.next()) assert(rs.getShort(1) === 1.toShort) + val metaData = rs.getMetaData + assert(metaData.getColumnName(1) === "CAST(1 AS SMALLINT)") + assert(metaData.getColumnTypeName(1) === "smallint") + assert(metaData.getColumnType(1) === java.sql.Types.SMALLINT) + assert(metaData.getPrecision(1) === 5) + assert(metaData.getScale(1) === 0) } } @@ -162,6 +174,12 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { testExecuteStatementWithProtocolVersion(version, "SELECT 1") { rs => assert(rs.next()) assert(rs.getInt(1) === 1) + val metaData = rs.getMetaData + assert(metaData.getColumnName(1) === "1") + assert(metaData.getColumnTypeName(1) === "int") + assert(metaData.getColumnType(1) === java.sql.Types.INTEGER) + assert(metaData.getPrecision(1) === 10) + assert(metaData.getScale(1) === 0) } } @@ -169,6 +187,12 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { testExecuteStatementWithProtocolVersion(version, "SELECT cast(1 as bigint)") { rs => assert(rs.next()) assert(rs.getLong(1) === 1L) + val metaData = rs.getMetaData + assert(metaData.getColumnName(1) === "CAST(1 AS BIGINT)") + assert(metaData.getColumnTypeName(1) === "bigint") + assert(metaData.getColumnType(1) === java.sql.Types.BIGINT) + assert(metaData.getPrecision(1) === 19) + assert(metaData.getScale(1) === 0) } } @@ -176,6 +200,12 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { testExecuteStatementWithProtocolVersion(version, "SELECT cast(1.2 as float)") { rs => assert(rs.next()) assert(rs.getFloat(1) === 1.2F) + val metaData = rs.getMetaData + assert(metaData.getColumnName(1) === "CAST(1.2 AS FLOAT)") + assert(metaData.getColumnTypeName(1) === "float") + assert(metaData.getColumnType(1) === java.sql.Types.FLOAT) + assert(metaData.getPrecision(1) === 7) + assert(metaData.getScale(1) === 7) } } @@ -183,14 +213,30 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { testExecuteStatementWithProtocolVersion(version, "SELECT cast(1.2 as double)") { rs => assert(rs.next()) assert(rs.getDouble(1) === 1.2D) + val metaData = rs.getMetaData + assert(metaData.getColumnName(1) === "CAST(1.2 AS DOUBLE)") + assert(metaData.getColumnTypeName(1) === "double") + assert(metaData.getColumnType(1) === java.sql.Types.DOUBLE) + assert(metaData.getPrecision(1) === 15) + assert(metaData.getScale(1) === 15) } } test(s"$version get decimal type") { testExecuteStatementWithProtocolVersion(version, - "SELECT cast(1 as decimal(18, 2)) as c") { rs => + "SELECT cast(1 as decimal(9, 1)) as col0, 1234.56BD as col1, 0.123 as col2") { rs => assert(rs.next()) - assert(rs.getBigDecimal(1) === new java.math.BigDecimal("1.00")) + assert(rs.getBigDecimal(1) === new java.math.BigDecimal("1.0")) + assert(rs.getBigDecimal("col1") === new java.math.BigDecimal("1234.56")) + assert(rs.getBigDecimal("col2") === new java.math.BigDecimal("0.123")) + val metaData = rs.getMetaData + (1 to 3) foreach { i => + assert(metaData.getColumnName(i) === s"col${i - 1}") + assert(metaData.getColumnTypeName(i) === "decimal") + assert(metaData.getColumnType(i) === java.sql.Types.DECIMAL) + assert(metaData.getPrecision(i) == 12 - i * 3) + assert(metaData.getScale(i) == i) + } } testExecuteStatementWithProtocolVersion(version, "SELECT cast(null as decimal) ") { rs => @@ -203,6 +249,12 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { testExecuteStatementWithProtocolVersion(version, "SELECT 'str'") { rs => assert(rs.next()) assert(rs.getString(1) === "str") + val metaData = rs.getMetaData + assert(metaData.getColumnName(1) ==="str") + assert(metaData.getColumnTypeName(1) === "string") + assert(metaData.getColumnType(1) === java.sql.Types.VARCHAR) + assert(metaData.getPrecision(1) === Int.MaxValue) + assert(metaData.getScale(1) === 0) } } @@ -211,6 +263,12 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { "SELECT cast('char-str' as char(10))") { rs => assert(rs.next()) assert(rs.getString(1) === "char-str") + val metaData = rs.getMetaData + assert(metaData.getColumnName(1) ==="CAST(char-str AS STRING)") + assert(metaData.getColumnTypeName(1) === "string") + assert(metaData.getColumnType(1) === java.sql.Types.VARCHAR) + assert(metaData.getPrecision(1) === Int.MaxValue) + assert(metaData.getScale(1) === 0) } } @@ -219,6 +277,12 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { "SELECT cast('varchar-str' as varchar(10))") { rs => assert(rs.next()) assert(rs.getString(1) === "varchar-str") + val metaData = rs.getMetaData + assert(metaData.getColumnName(1) ==="CAST(varchar-str AS STRING)") + assert(metaData.getColumnTypeName(1) === "string") + assert(metaData.getColumnType(1) === java.sql.Types.VARCHAR) + assert(metaData.getPrecision(1) === Int.MaxValue) + assert(metaData.getScale(1) === 0) } } @@ -226,6 +290,12 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { testExecuteStatementWithProtocolVersion(version, "SELECT cast('ABC' as binary)") { rs => assert(rs.next()) assert(rs.getString(1) === "ABC") + val metaData = rs.getMetaData + assert(metaData.getColumnName(1) === "CAST(ABC AS BINARY)") + assert(metaData.getColumnTypeName(1) === "binary") + assert(metaData.getColumnType(1) === java.sql.Types.BINARY) + assert(metaData.getPrecision(1) === Int.MaxValue) + assert(metaData.getScale(1) === 0) } testExecuteStatementWithProtocolVersion(version, "SELECT cast(49960 as binary)") { rs => assert(rs.next()) @@ -241,6 +311,12 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { testExecuteStatementWithProtocolVersion(version, "SELECT true") { rs => assert(rs.next()) assert(rs.getBoolean(1) === true) + val metaData = rs.getMetaData + assert(metaData.getColumnName(1) === "true") + assert(metaData.getColumnTypeName(1) === "boolean") + assert(metaData.getColumnType(1) === java.sql.Types.BOOLEAN) + assert(metaData.getPrecision(1) === 1) + assert(metaData.getScale(1) === 0) } } @@ -248,6 +324,12 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { testExecuteStatementWithProtocolVersion(version, "SELECT cast('2019-07-22' as date)") { rs => assert(rs.next()) assert(rs.getDate(1) === Date.valueOf("2019-07-22")) + val metaData = rs.getMetaData + assert(metaData.getColumnName(1) === "CAST(2019-07-22 AS DATE)") + assert(metaData.getColumnTypeName(1) === "date") + assert(metaData.getColumnType(1) === java.sql.Types.DATE) + assert(metaData.getPrecision(1) === 10) + assert(metaData.getScale(1) === 0) } } @@ -256,6 +338,12 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { "SELECT cast('2019-07-22 18:14:00' as timestamp)") { rs => assert(rs.next()) assert(rs.getTimestamp(1) === Timestamp.valueOf("2019-07-22 18:14:00")) + val metaData = rs.getMetaData + assert(metaData.getColumnName(1) === "CAST(2019-07-22 18:14:00 AS TIMESTAMP)") + assert(metaData.getColumnTypeName(1) === "timestamp") + assert(metaData.getColumnType(1) === java.sql.Types.TIMESTAMP) + assert(metaData.getPrecision(1) === 29) + assert(metaData.getScale(1) === 9) } } @@ -263,6 +351,12 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { testExecuteStatementWithProtocolVersion(version, "SELECT null") { rs => assert(rs.next()) assert(rs.getString(1) === null) + val metaData = rs.getMetaData + assert(metaData.getColumnName(1) === "NULL") + assert(metaData.getColumnTypeName(1) === "void") + assert(metaData.getColumnType(1) === java.sql.Types.NULL) + assert(metaData.getPrecision(1) === 0) + assert(metaData.getScale(1) === 0) } } @@ -270,28 +364,67 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { testExecuteStatementWithProtocolVersion(version, "SELECT interval '1' year '2' day") { rs => assert(rs.next()) assert(rs.getString(1) === "1 years 2 days") + val metaData = rs.getMetaData + assert(metaData.getColumnName(1) === "INTERVAL '1 years 2 days'") + assert(metaData.getColumnTypeName(1) === "string") + assert(metaData.getColumnType(1) === java.sql.Types.VARCHAR) + assert(metaData.getPrecision(1) === Int.MaxValue) + assert(metaData.getScale(1) === 0) } } test(s"$version get array type") { - testExecuteStatementWithProtocolVersion(version, "SELECT array(1, 2)") { rs => + testExecuteStatementWithProtocolVersion( + version, "SELECT array() AS col1, array(1, 2) AS col2") { rs => assert(rs.next()) - assert(rs.getString(1) === "[1,2]") + assert(rs.getString(2) === "[1,2]") + assert(rs.getObject("col1") === "[]") + assert(rs.getObject("col2") === "[1,2]") + val metaData = rs.getMetaData + (1 to 2) foreach { i => + assert(metaData.getColumnName(i) === s"col$i") + assert(metaData.getColumnTypeName(i) === "array") + assert(metaData.getColumnType(i) === java.sql.Types.ARRAY) + assert(metaData.getPrecision(i) === Int.MaxValue) + assert(metaData.getScale(i) == 0) + } } } test(s"$version get map type") { - testExecuteStatementWithProtocolVersion(version, "SELECT map(1, 2)") { rs => + testExecuteStatementWithProtocolVersion(version, + "SELECT map(), map(1, 2, 3, 4)") { rs => assert(rs.next()) - assert(rs.getString(1) === "{1:2}") + assert(rs.getObject(1) === "{}") + assert(rs.getObject(2) === "{1:2,3:4}") + assert(rs.getString(2) === "{1:2,3:4}") + val metaData = rs.getMetaData + (1 to 2) foreach { i => + assert(metaData.getColumnName(i).startsWith("map(")) + assert(metaData.getColumnTypeName(1) === "map") + assert(metaData.getColumnType(i) === java.sql.Types.JAVA_OBJECT) + assert(metaData.getPrecision(i) === Int.MaxValue) + assert(metaData.getScale(i) == 0) + } } } test(s"$version get struct type") { testExecuteStatementWithProtocolVersion(version, - "SELECT struct('alpha' AS A, 'beta' AS B)") { rs => + "SELECT struct('alpha' AS A, 'beta' AS B) as col0," + + " struct('1', '2') AS col1, named_struct('a', 2, 'b', 4) AS col2") { rs => assert(rs.next()) assert(rs.getString(1) === """{"A":"alpha","B":"beta"}""") + assert(rs.getObject("col1") === """{"col1":"1","col2":"2"}""") + assert(rs.getObject("col2") === """{"a":2,"b":4}""") + val metaData = rs.getMetaData + (1 to 3) foreach { i => + assert(metaData.getColumnName(i) === s"col${i - 1}") + assert(metaData.getColumnTypeName(1) === "struct") + assert(metaData.getColumnType(i) === java.sql.Types.STRUCT) + assert(metaData.getPrecision(i) === Int.MaxValue) + assert(metaData.getScale(i) == 0) + } } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index ecc7ce71d950e..be42497113469 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -39,12 +39,12 @@ import org.apache.spark.sql.types._ * * To run the entire test suite: * {{{ - * build/sbt "hive-thriftserver/test-only *ThriftServerQueryTestSuite" -Phive-thriftserver + * build/sbt "hive-thriftserver/testOnly *ThriftServerQueryTestSuite" -Phive-thriftserver * }}} * * This test suite won't generate golden files. To re-generate golden files for entire suite, run: * {{{ - * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *SQLQueryTestSuite" + * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite" * }}} * * TODO: diff --git a/sql/hive-thriftserver/v1.2/if/TCLIService.thrift b/sql/hive-thriftserver/v1.2/if/TCLIService.thrift deleted file mode 100644 index 225e319737811..0000000000000 --- a/sql/hive-thriftserver/v1.2/if/TCLIService.thrift +++ /dev/null @@ -1,1173 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -// Coding Conventions for this file: -// -// Structs/Enums/Unions -// * Struct, Enum, and Union names begin with a "T", -// and use a capital letter for each new word, with no underscores. -// * All fields should be declared as either optional or required. -// -// Functions -// * Function names start with a capital letter and have a capital letter for -// each new word, with no underscores. -// * Each function should take exactly one parameter, named TFunctionNameReq, -// and should return either void or TFunctionNameResp. This convention allows -// incremental updates. -// -// Services -// * Service names begin with the letter "T", use a capital letter for each -// new word (with no underscores), and end with the word "Service". - -namespace java org.apache.hive.service.cli.thrift -namespace cpp apache.hive.service.cli.thrift - -// List of protocol versions. A new token should be -// added to the end of this list every time a change is made. -enum TProtocolVersion { - HIVE_CLI_SERVICE_PROTOCOL_V1, - - // V2 adds support for asynchronous execution - HIVE_CLI_SERVICE_PROTOCOL_V2 - - // V3 add varchar type, primitive type qualifiers - HIVE_CLI_SERVICE_PROTOCOL_V3 - - // V4 add decimal precision/scale, char type - HIVE_CLI_SERVICE_PROTOCOL_V4 - - // V5 adds error details when GetOperationStatus returns in error state - HIVE_CLI_SERVICE_PROTOCOL_V5 - - // V6 uses binary type for binary payload (was string) and uses columnar result set - HIVE_CLI_SERVICE_PROTOCOL_V6 - - // V7 adds support for delegation token based connection - HIVE_CLI_SERVICE_PROTOCOL_V7 - - // V8 adds support for interval types - HIVE_CLI_SERVICE_PROTOCOL_V8 -} - -enum TTypeId { - BOOLEAN_TYPE, - TINYINT_TYPE, - SMALLINT_TYPE, - INT_TYPE, - BIGINT_TYPE, - FLOAT_TYPE, - DOUBLE_TYPE, - STRING_TYPE, - TIMESTAMP_TYPE, - BINARY_TYPE, - ARRAY_TYPE, - MAP_TYPE, - STRUCT_TYPE, - UNION_TYPE, - USER_DEFINED_TYPE, - DECIMAL_TYPE, - NULL_TYPE, - DATE_TYPE, - VARCHAR_TYPE, - CHAR_TYPE, - INTERVAL_YEAR_MONTH_TYPE, - INTERVAL_DAY_TIME_TYPE -} - -const set PRIMITIVE_TYPES = [ - TTypeId.BOOLEAN_TYPE, - TTypeId.TINYINT_TYPE, - TTypeId.SMALLINT_TYPE, - TTypeId.INT_TYPE, - TTypeId.BIGINT_TYPE, - TTypeId.FLOAT_TYPE, - TTypeId.DOUBLE_TYPE, - TTypeId.STRING_TYPE, - TTypeId.TIMESTAMP_TYPE, - TTypeId.BINARY_TYPE, - TTypeId.DECIMAL_TYPE, - TTypeId.NULL_TYPE, - TTypeId.DATE_TYPE, - TTypeId.VARCHAR_TYPE, - TTypeId.CHAR_TYPE, - TTypeId.INTERVAL_YEAR_MONTH_TYPE, - TTypeId.INTERVAL_DAY_TIME_TYPE -] - -const set COMPLEX_TYPES = [ - TTypeId.ARRAY_TYPE - TTypeId.MAP_TYPE - TTypeId.STRUCT_TYPE - TTypeId.UNION_TYPE - TTypeId.USER_DEFINED_TYPE -] - -const set COLLECTION_TYPES = [ - TTypeId.ARRAY_TYPE - TTypeId.MAP_TYPE -] - -const map TYPE_NAMES = { - TTypeId.BOOLEAN_TYPE: "BOOLEAN", - TTypeId.TINYINT_TYPE: "TINYINT", - TTypeId.SMALLINT_TYPE: "SMALLINT", - TTypeId.INT_TYPE: "INT", - TTypeId.BIGINT_TYPE: "BIGINT", - TTypeId.FLOAT_TYPE: "FLOAT", - TTypeId.DOUBLE_TYPE: "DOUBLE", - TTypeId.STRING_TYPE: "STRING", - TTypeId.TIMESTAMP_TYPE: "TIMESTAMP", - TTypeId.BINARY_TYPE: "BINARY", - TTypeId.ARRAY_TYPE: "ARRAY", - TTypeId.MAP_TYPE: "MAP", - TTypeId.STRUCT_TYPE: "STRUCT", - TTypeId.UNION_TYPE: "UNIONTYPE", - TTypeId.DECIMAL_TYPE: "DECIMAL", - TTypeId.NULL_TYPE: "NULL" - TTypeId.DATE_TYPE: "DATE" - TTypeId.VARCHAR_TYPE: "VARCHAR" - TTypeId.CHAR_TYPE: "CHAR" - TTypeId.INTERVAL_YEAR_MONTH_TYPE: "INTERVAL_YEAR_MONTH" - TTypeId.INTERVAL_DAY_TIME_TYPE: "INTERVAL_DAY_TIME" -} - -// Thrift does not support recursively defined types or forward declarations, -// which makes it difficult to represent Hive's nested types. -// To get around these limitations TTypeDesc employs a type list that maps -// integer "pointers" to TTypeEntry objects. The following examples show -// how different types are represented using this scheme: -// -// "INT": -// TTypeDesc { -// types = [ -// TTypeEntry.primitive_entry { -// type = INT_TYPE -// } -// ] -// } -// -// "ARRAY": -// TTypeDesc { -// types = [ -// TTypeEntry.array_entry { -// object_type_ptr = 1 -// }, -// TTypeEntry.primitive_entry { -// type = INT_TYPE -// } -// ] -// } -// -// "MAP": -// TTypeDesc { -// types = [ -// TTypeEntry.map_entry { -// key_type_ptr = 1 -// value_type_ptr = 2 -// }, -// TTypeEntry.primitive_entry { -// type = INT_TYPE -// }, -// TTypeEntry.primitive_entry { -// type = STRING_TYPE -// } -// ] -// } - -typedef i32 TTypeEntryPtr - -// Valid TTypeQualifiers key names -const string CHARACTER_MAXIMUM_LENGTH = "characterMaximumLength" - -// Type qualifier key name for decimal -const string PRECISION = "precision" -const string SCALE = "scale" - -union TTypeQualifierValue { - 1: optional i32 i32Value - 2: optional string stringValue -} - -// Type qualifiers for primitive type. -struct TTypeQualifiers { - 1: required map qualifiers -} - -// Type entry for a primitive type. -struct TPrimitiveTypeEntry { - // The primitive type token. This must satisfy the condition - // that type is in the PRIMITIVE_TYPES set. - 1: required TTypeId type - 2: optional TTypeQualifiers typeQualifiers -} - -// Type entry for an ARRAY type. -struct TArrayTypeEntry { - 1: required TTypeEntryPtr objectTypePtr -} - -// Type entry for a MAP type. -struct TMapTypeEntry { - 1: required TTypeEntryPtr keyTypePtr - 2: required TTypeEntryPtr valueTypePtr -} - -// Type entry for a STRUCT type. -struct TStructTypeEntry { - 1: required map nameToTypePtr -} - -// Type entry for a UNIONTYPE type. -struct TUnionTypeEntry { - 1: required map nameToTypePtr -} - -struct TUserDefinedTypeEntry { - // The fully qualified name of the class implementing this type. - 1: required string typeClassName -} - -// We use a union here since Thrift does not support inheritance. -union TTypeEntry { - 1: TPrimitiveTypeEntry primitiveEntry - 2: TArrayTypeEntry arrayEntry - 3: TMapTypeEntry mapEntry - 4: TStructTypeEntry structEntry - 5: TUnionTypeEntry unionEntry - 6: TUserDefinedTypeEntry userDefinedTypeEntry -} - -// Type descriptor for columns. -struct TTypeDesc { - // The "top" type is always the first element of the list. - // If the top type is an ARRAY, MAP, STRUCT, or UNIONTYPE - // type, then subsequent elements represent nested types. - 1: required list types -} - -// A result set column descriptor. -struct TColumnDesc { - // The name of the column - 1: required string columnName - - // The type descriptor for this column - 2: required TTypeDesc typeDesc - - // The ordinal position of this column in the schema - 3: required i32 position - - 4: optional string comment -} - -// Metadata used to describe the schema (column names, types, comments) -// of result sets. -struct TTableSchema { - 1: required list columns -} - -// A Boolean column value. -struct TBoolValue { - // NULL if value is unset. - 1: optional bool value -} - -// A Byte column value. -struct TByteValue { - // NULL if value is unset. - 1: optional byte value -} - -// A signed, 16 bit column value. -struct TI16Value { - // NULL if value is unset - 1: optional i16 value -} - -// A signed, 32 bit column value -struct TI32Value { - // NULL if value is unset - 1: optional i32 value -} - -// A signed 64 bit column value -struct TI64Value { - // NULL if value is unset - 1: optional i64 value -} - -// A floating point 64 bit column value -struct TDoubleValue { - // NULL if value is unset - 1: optional double value -} - -struct TStringValue { - // NULL if value is unset - 1: optional string value -} - -// A single column value in a result set. -// Note that Hive's type system is richer than Thrift's, -// so in some cases we have to map multiple Hive types -// to the same Thrift type. On the client-side this is -// disambiguated by looking at the Schema of the -// result set. -union TColumnValue { - 1: TBoolValue boolVal // BOOLEAN - 2: TByteValue byteVal // TINYINT - 3: TI16Value i16Val // SMALLINT - 4: TI32Value i32Val // INT - 5: TI64Value i64Val // BIGINT, TIMESTAMP - 6: TDoubleValue doubleVal // FLOAT, DOUBLE - 7: TStringValue stringVal // STRING, LIST, MAP, STRUCT, UNIONTYPE, BINARY, DECIMAL, NULL, INTERVAL_YEAR_MONTH, INTERVAL_DAY_TIME -} - -// Represents a row in a rowset. -struct TRow { - 1: required list colVals -} - -struct TBoolColumn { - 1: required list values - 2: required binary nulls -} - -struct TByteColumn { - 1: required list values - 2: required binary nulls -} - -struct TI16Column { - 1: required list values - 2: required binary nulls -} - -struct TI32Column { - 1: required list values - 2: required binary nulls -} - -struct TI64Column { - 1: required list values - 2: required binary nulls -} - -struct TDoubleColumn { - 1: required list values - 2: required binary nulls -} - -struct TStringColumn { - 1: required list values - 2: required binary nulls -} - -struct TBinaryColumn { - 1: required list values - 2: required binary nulls -} - -// Note that Hive's type system is richer than Thrift's, -// so in some cases we have to map multiple Hive types -// to the same Thrift type. On the client-side this is -// disambiguated by looking at the Schema of the -// result set. -union TColumn { - 1: TBoolColumn boolVal // BOOLEAN - 2: TByteColumn byteVal // TINYINT - 3: TI16Column i16Val // SMALLINT - 4: TI32Column i32Val // INT - 5: TI64Column i64Val // BIGINT, TIMESTAMP - 6: TDoubleColumn doubleVal // FLOAT, DOUBLE - 7: TStringColumn stringVal // STRING, LIST, MAP, STRUCT, UNIONTYPE, DECIMAL, NULL - 8: TBinaryColumn binaryVal // BINARY -} - -// Represents a rowset -struct TRowSet { - // The starting row offset of this rowset. - 1: required i64 startRowOffset - 2: required list rows - 3: optional list columns -} - -// The return status code contained in each response. -enum TStatusCode { - SUCCESS_STATUS, - SUCCESS_WITH_INFO_STATUS, - STILL_EXECUTING_STATUS, - ERROR_STATUS, - INVALID_HANDLE_STATUS -} - -// The return status of a remote request -struct TStatus { - 1: required TStatusCode statusCode - - // If status is SUCCESS_WITH_INFO, info_msgs may be populated with - // additional diagnostic information. - 2: optional list infoMessages - - // If status is ERROR, then the following fields may be set - 3: optional string sqlState // as defined in the ISO/IEF CLI specification - 4: optional i32 errorCode // internal error code - 5: optional string errorMessage -} - -// The state of an operation (i.e. a query or other -// asynchronous operation that generates a result set) -// on the server. -enum TOperationState { - // The operation has been initialized - INITIALIZED_STATE, - - // The operation is running. In this state the result - // set is not available. - RUNNING_STATE, - - // The operation has completed. When an operation is in - // this state its result set may be fetched. - FINISHED_STATE, - - // The operation was canceled by a client - CANCELED_STATE, - - // The operation was closed by a client - CLOSED_STATE, - - // The operation failed due to an error - ERROR_STATE, - - // The operation is in an unrecognized state - UKNOWN_STATE, - - // The operation is in an pending state - PENDING_STATE, -} - -// A string identifier. This is interpreted literally. -typedef string TIdentifier - -// A search pattern. -// -// Valid search pattern characters: -// '_': Any single character. -// '%': Any sequence of zero or more characters. -// '\': Escape character used to include special characters, -// e.g. '_', '%', '\'. If a '\' precedes a non-special -// character it has no special meaning and is interpreted -// literally. -typedef string TPattern - - -// A search pattern or identifier. Used as input -// parameter for many of the catalog functions. -typedef string TPatternOrIdentifier - -struct THandleIdentifier { - // 16 byte globally unique identifier - // This is the public ID of the handle and - // can be used for reporting. - 1: required binary guid, - - // 16 byte secret generated by the server - // and used to verify that the handle is not - // being hijacked by another user. - 2: required binary secret, -} - -// Client-side handle to persistent -// session information on the server-side. -struct TSessionHandle { - 1: required THandleIdentifier sessionId -} - -// The subtype of an OperationHandle. -enum TOperationType { - EXECUTE_STATEMENT, - GET_TYPE_INFO, - GET_CATALOGS, - GET_SCHEMAS, - GET_TABLES, - GET_TABLE_TYPES, - GET_COLUMNS, - GET_FUNCTIONS, - UNKNOWN, -} - -// Client-side reference to a task running -// asynchronously on the server. -struct TOperationHandle { - 1: required THandleIdentifier operationId - 2: required TOperationType operationType - - // If hasResultSet = TRUE, then this operation - // generates a result set that can be fetched. - // Note that the result set may be empty. - // - // If hasResultSet = FALSE, then this operation - // does not generate a result set, and calling - // GetResultSetMetadata or FetchResults against - // this OperationHandle will generate an error. - 3: required bool hasResultSet - - // For operations that don't generate result sets, - // modifiedRowCount is either: - // - // 1) The number of rows that were modified by - // the DML operation (e.g. number of rows inserted, - // number of rows deleted, etc). - // - // 2) 0 for operations that don't modify or add rows. - // - // 3) < 0 if the operation is capable of modifiying rows, - // but Hive is unable to determine how many rows were - // modified. For example, Hive's LOAD DATA command - // doesn't generate row count information because - // Hive doesn't inspect the data as it is loaded. - // - // modifiedRowCount is unset if the operation generates - // a result set. - 4: optional double modifiedRowCount -} - - -// OpenSession() -// -// Open a session (connection) on the server against -// which operations may be executed. -struct TOpenSessionReq { - // The version of the HiveServer2 protocol that the client is using. - 1: required TProtocolVersion client_protocol = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8 - - // Username and password for authentication. - // Depending on the authentication scheme being used, - // this information may instead be provided by a lower - // protocol layer, in which case these fields may be - // left unset. - 2: optional string username - 3: optional string password - - // Configuration overlay which is applied when the session is - // first created. - 4: optional map configuration -} - -struct TOpenSessionResp { - 1: required TStatus status - - // The protocol version that the server is using. - 2: required TProtocolVersion serverProtocolVersion = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8 - - // Session Handle - 3: optional TSessionHandle sessionHandle - - // The configuration settings for this session. - 4: optional map configuration -} - - -// CloseSession() -// -// Closes the specified session and frees any resources -// currently allocated to that session. Any open -// operations in that session will be canceled. -struct TCloseSessionReq { - 1: required TSessionHandle sessionHandle -} - -struct TCloseSessionResp { - 1: required TStatus status -} - - - -enum TGetInfoType { - CLI_MAX_DRIVER_CONNECTIONS = 0, - CLI_MAX_CONCURRENT_ACTIVITIES = 1, - CLI_DATA_SOURCE_NAME = 2, - CLI_FETCH_DIRECTION = 8, - CLI_SERVER_NAME = 13, - CLI_SEARCH_PATTERN_ESCAPE = 14, - CLI_DBMS_NAME = 17, - CLI_DBMS_VER = 18, - CLI_ACCESSIBLE_TABLES = 19, - CLI_ACCESSIBLE_PROCEDURES = 20, - CLI_CURSOR_COMMIT_BEHAVIOR = 23, - CLI_DATA_SOURCE_READ_ONLY = 25, - CLI_DEFAULT_TXN_ISOLATION = 26, - CLI_IDENTIFIER_CASE = 28, - CLI_IDENTIFIER_QUOTE_CHAR = 29, - CLI_MAX_COLUMN_NAME_LEN = 30, - CLI_MAX_CURSOR_NAME_LEN = 31, - CLI_MAX_SCHEMA_NAME_LEN = 32, - CLI_MAX_CATALOG_NAME_LEN = 34, - CLI_MAX_TABLE_NAME_LEN = 35, - CLI_SCROLL_CONCURRENCY = 43, - CLI_TXN_CAPABLE = 46, - CLI_USER_NAME = 47, - CLI_TXN_ISOLATION_OPTION = 72, - CLI_INTEGRITY = 73, - CLI_GETDATA_EXTENSIONS = 81, - CLI_NULL_COLLATION = 85, - CLI_ALTER_TABLE = 86, - CLI_ORDER_BY_COLUMNS_IN_SELECT = 90, - CLI_SPECIAL_CHARACTERS = 94, - CLI_MAX_COLUMNS_IN_GROUP_BY = 97, - CLI_MAX_COLUMNS_IN_INDEX = 98, - CLI_MAX_COLUMNS_IN_ORDER_BY = 99, - CLI_MAX_COLUMNS_IN_SELECT = 100, - CLI_MAX_COLUMNS_IN_TABLE = 101, - CLI_MAX_INDEX_SIZE = 102, - CLI_MAX_ROW_SIZE = 104, - CLI_MAX_STATEMENT_LEN = 105, - CLI_MAX_TABLES_IN_SELECT = 106, - CLI_MAX_USER_NAME_LEN = 107, - CLI_OJ_CAPABILITIES = 115, - - CLI_XOPEN_CLI_YEAR = 10000, - CLI_CURSOR_SENSITIVITY = 10001, - CLI_DESCRIBE_PARAMETER = 10002, - CLI_CATALOG_NAME = 10003, - CLI_COLLATION_SEQ = 10004, - CLI_MAX_IDENTIFIER_LEN = 10005, -} - -union TGetInfoValue { - 1: string stringValue - 2: i16 smallIntValue - 3: i32 integerBitmask - 4: i32 integerFlag - 5: i32 binaryValue - 6: i64 lenValue -} - -// GetInfo() -// -// This function is based on ODBC's CLIGetInfo() function. -// The function returns general information about the data source -// using the same keys as ODBC. -struct TGetInfoReq { - // The session to run this request against - 1: required TSessionHandle sessionHandle - - 2: required TGetInfoType infoType -} - -struct TGetInfoResp { - 1: required TStatus status - - 2: required TGetInfoValue infoValue -} - - -// ExecuteStatement() -// -// Execute a statement. -// The returned OperationHandle can be used to check on the -// status of the statement, and to fetch results once the -// statement has finished executing. -struct TExecuteStatementReq { - // The session to execute the statement against - 1: required TSessionHandle sessionHandle - - // The statement to be executed (DML, DDL, SET, etc) - 2: required string statement - - // Configuration properties that are overlayed on top of the - // the existing session configuration before this statement - // is executed. These properties apply to this statement - // only and will not affect the subsequent state of the Session. - 3: optional map confOverlay - - // Execute asynchronously when runAsync is true - 4: optional bool runAsync = false -} - -struct TExecuteStatementResp { - 1: required TStatus status - 2: optional TOperationHandle operationHandle -} - -// GetTypeInfo() -// -// Get information about types supported by the HiveServer instance. -// The information is returned as a result set which can be fetched -// using the OperationHandle provided in the response. -// -// Refer to the documentation for ODBC's CLIGetTypeInfo function for -// the format of the result set. -struct TGetTypeInfoReq { - // The session to run this request against. - 1: required TSessionHandle sessionHandle -} - -struct TGetTypeInfoResp { - 1: required TStatus status - 2: optional TOperationHandle operationHandle -} - - -// GetCatalogs() -// -// Returns the list of catalogs (databases) -// Results are ordered by TABLE_CATALOG -// -// Resultset columns : -// col1 -// name: TABLE_CAT -// type: STRING -// desc: Catalog name. NULL if not applicable. -// -struct TGetCatalogsReq { - // Session to run this request against - 1: required TSessionHandle sessionHandle -} - -struct TGetCatalogsResp { - 1: required TStatus status - 2: optional TOperationHandle operationHandle -} - - -// GetSchemas() -// -// Retrieves the schema names available in this database. -// The results are ordered by TABLE_CATALOG and TABLE_SCHEM. -// col1 -// name: TABLE_SCHEM -// type: STRING -// desc: schema name -// col2 -// name: TABLE_CATALOG -// type: STRING -// desc: catalog name -struct TGetSchemasReq { - // Session to run this request against - 1: required TSessionHandle sessionHandle - - // Name of the catalog. Must not contain a search pattern. - 2: optional TIdentifier catalogName - - // schema name or pattern - 3: optional TPatternOrIdentifier schemaName -} - -struct TGetSchemasResp { - 1: required TStatus status - 2: optional TOperationHandle operationHandle -} - - -// GetTables() -// -// Returns a list of tables with catalog, schema, and table -// type information. The information is returned as a result -// set which can be fetched using the OperationHandle -// provided in the response. -// Results are ordered by TABLE_TYPE, TABLE_CAT, TABLE_SCHEM, and TABLE_NAME -// -// Result Set Columns: -// -// col1 -// name: TABLE_CAT -// type: STRING -// desc: Catalog name. NULL if not applicable. -// -// col2 -// name: TABLE_SCHEM -// type: STRING -// desc: Schema name. -// -// col3 -// name: TABLE_NAME -// type: STRING -// desc: Table name. -// -// col4 -// name: TABLE_TYPE -// type: STRING -// desc: The table type, e.g. "TABLE", "VIEW", etc. -// -// col5 -// name: REMARKS -// type: STRING -// desc: Comments about the table -// -struct TGetTablesReq { - // Session to run this request against - 1: required TSessionHandle sessionHandle - - // Name of the catalog or a search pattern. - 2: optional TPatternOrIdentifier catalogName - - // Name of the schema or a search pattern. - 3: optional TPatternOrIdentifier schemaName - - // Name of the table or a search pattern. - 4: optional TPatternOrIdentifier tableName - - // List of table types to match - // e.g. "TABLE", "VIEW", "SYSTEM TABLE", "GLOBAL TEMPORARY", - // "LOCAL TEMPORARY", "ALIAS", "SYNONYM", etc. - 5: optional list tableTypes -} - -struct TGetTablesResp { - 1: required TStatus status - 2: optional TOperationHandle operationHandle -} - - -// GetTableTypes() -// -// Returns the table types available in this database. -// The results are ordered by table type. -// -// col1 -// name: TABLE_TYPE -// type: STRING -// desc: Table type name. -struct TGetTableTypesReq { - // Session to run this request against - 1: required TSessionHandle sessionHandle -} - -struct TGetTableTypesResp { - 1: required TStatus status - 2: optional TOperationHandle operationHandle -} - - -// GetColumns() -// -// Returns a list of columns in the specified tables. -// The information is returned as a result set which can be fetched -// using the OperationHandle provided in the response. -// Results are ordered by TABLE_CAT, TABLE_SCHEM, TABLE_NAME, -// and ORDINAL_POSITION. -// -// Result Set Columns are the same as those for the ODBC CLIColumns -// function. -// -struct TGetColumnsReq { - // Session to run this request against - 1: required TSessionHandle sessionHandle - - // Name of the catalog. Must not contain a search pattern. - 2: optional TIdentifier catalogName - - // Schema name or search pattern - 3: optional TPatternOrIdentifier schemaName - - // Table name or search pattern - 4: optional TPatternOrIdentifier tableName - - // Column name or search pattern - 5: optional TPatternOrIdentifier columnName -} - -struct TGetColumnsResp { - 1: required TStatus status - 2: optional TOperationHandle operationHandle -} - - -// GetFunctions() -// -// Returns a list of functions supported by the data source. The -// behavior of this function matches -// java.sql.DatabaseMetaData.getFunctions() both in terms of -// inputs and outputs. -// -// Result Set Columns: -// -// col1 -// name: FUNCTION_CAT -// type: STRING -// desc: Function catalog (may be null) -// -// col2 -// name: FUNCTION_SCHEM -// type: STRING -// desc: Function schema (may be null) -// -// col3 -// name: FUNCTION_NAME -// type: STRING -// desc: Function name. This is the name used to invoke the function. -// -// col4 -// name: REMARKS -// type: STRING -// desc: Explanatory comment on the function. -// -// col5 -// name: FUNCTION_TYPE -// type: SMALLINT -// desc: Kind of function. One of: -// * functionResultUnknown - Cannot determine if a return value or a table -// will be returned. -// * functionNoTable - Does not a return a table. -// * functionReturnsTable - Returns a table. -// -// col6 -// name: SPECIFIC_NAME -// type: STRING -// desc: The name which uniquely identifies this function within its schema. -// In this case this is the fully qualified class name of the class -// that implements this function. -// -struct TGetFunctionsReq { - // Session to run this request against - 1: required TSessionHandle sessionHandle - - // A catalog name; must match the catalog name as it is stored in the - // database; "" retrieves those without a catalog; null means - // that the catalog name should not be used to narrow the search. - 2: optional TIdentifier catalogName - - // A schema name pattern; must match the schema name as it is stored - // in the database; "" retrieves those without a schema; null means - // that the schema name should not be used to narrow the search. - 3: optional TPatternOrIdentifier schemaName - - // A function name pattern; must match the function name as it is stored - // in the database. - 4: required TPatternOrIdentifier functionName -} - -struct TGetFunctionsResp { - 1: required TStatus status - 2: optional TOperationHandle operationHandle -} - - -// GetOperationStatus() -// -// Get the status of an operation running on the server. -struct TGetOperationStatusReq { - // Session to run this request against - 1: required TOperationHandle operationHandle -} - -struct TGetOperationStatusResp { - 1: required TStatus status - 2: optional TOperationState operationState - - // If operationState is ERROR_STATE, then the following fields may be set - // sqlState as defined in the ISO/IEF CLI specification - 3: optional string sqlState - - // Internal error code - 4: optional i32 errorCode - - // Error message - 5: optional string errorMessage -} - - -// CancelOperation() -// -// Cancels processing on the specified operation handle and -// frees any resources which were allocated. -struct TCancelOperationReq { - // Operation to cancel - 1: required TOperationHandle operationHandle -} - -struct TCancelOperationResp { - 1: required TStatus status -} - - -// CloseOperation() -// -// Given an operation in the FINISHED, CANCELED, -// or ERROR states, CloseOperation() will free -// all of the resources which were allocated on -// the server to service the operation. -struct TCloseOperationReq { - 1: required TOperationHandle operationHandle -} - -struct TCloseOperationResp { - 1: required TStatus status -} - - -// GetResultSetMetadata() -// -// Retrieves schema information for the specified operation -struct TGetResultSetMetadataReq { - // Operation for which to fetch result set schema information - 1: required TOperationHandle operationHandle -} - -struct TGetResultSetMetadataResp { - 1: required TStatus status - 2: optional TTableSchema schema -} - - -enum TFetchOrientation { - // Get the next rowset. The fetch offset is ignored. - FETCH_NEXT, - - // Get the previous rowset. The fetch offset is ignored. - FETCH_PRIOR, - - // Return the rowset at the given fetch offset relative - // to the current rowset. - // NOT SUPPORTED - FETCH_RELATIVE, - - // Return the rowset at the specified fetch offset. - // NOT SUPPORTED - FETCH_ABSOLUTE, - - // Get the first rowset in the result set. - FETCH_FIRST, - - // Get the last rowset in the result set. - // NOT SUPPORTED - FETCH_LAST -} - -// FetchResults() -// -// Fetch rows from the server corresponding to -// a particular OperationHandle. -struct TFetchResultsReq { - // Operation from which to fetch results. - 1: required TOperationHandle operationHandle - - // The fetch orientation. This must be either - // FETCH_NEXT, FETCH_PRIOR or FETCH_FIRST. Defaults to FETCH_NEXT. - 2: required TFetchOrientation orientation = TFetchOrientation.FETCH_NEXT - - // Max number of rows that should be returned in - // the rowset. - 3: required i64 maxRows - - // The type of a fetch results request. 0 represents Query output. 1 represents Log - 4: optional i16 fetchType = 0 -} - -struct TFetchResultsResp { - 1: required TStatus status - - // TRUE if there are more rows left to fetch from the server. - 2: optional bool hasMoreRows - - // The rowset. This is optional so that we have the - // option in the future of adding alternate formats for - // representing result set data, e.g. delimited strings, - // binary encoded, etc. - 3: optional TRowSet results -} - -// GetDelegationToken() -// Retrieve delegation token for the current user -struct TGetDelegationTokenReq { - // session handle - 1: required TSessionHandle sessionHandle - - // userid for the proxy user - 2: required string owner - - // designated renewer userid - 3: required string renewer -} - -struct TGetDelegationTokenResp { - // status of the request - 1: required TStatus status - - // delegation token string - 2: optional string delegationToken -} - -// CancelDelegationToken() -// Cancel the given delegation token -struct TCancelDelegationTokenReq { - // session handle - 1: required TSessionHandle sessionHandle - - // delegation token to cancel - 2: required string delegationToken -} - -struct TCancelDelegationTokenResp { - // status of the request - 1: required TStatus status -} - -// RenewDelegationToken() -// Renew the given delegation token -struct TRenewDelegationTokenReq { - // session handle - 1: required TSessionHandle sessionHandle - - // delegation token to renew - 2: required string delegationToken -} - -struct TRenewDelegationTokenResp { - // status of the request - 1: required TStatus status -} - -service TCLIService { - - TOpenSessionResp OpenSession(1:TOpenSessionReq req); - - TCloseSessionResp CloseSession(1:TCloseSessionReq req); - - TGetInfoResp GetInfo(1:TGetInfoReq req); - - TExecuteStatementResp ExecuteStatement(1:TExecuteStatementReq req); - - TGetTypeInfoResp GetTypeInfo(1:TGetTypeInfoReq req); - - TGetCatalogsResp GetCatalogs(1:TGetCatalogsReq req); - - TGetSchemasResp GetSchemas(1:TGetSchemasReq req); - - TGetTablesResp GetTables(1:TGetTablesReq req); - - TGetTableTypesResp GetTableTypes(1:TGetTableTypesReq req); - - TGetColumnsResp GetColumns(1:TGetColumnsReq req); - - TGetFunctionsResp GetFunctions(1:TGetFunctionsReq req); - - TGetOperationStatusResp GetOperationStatus(1:TGetOperationStatusReq req); - - TCancelOperationResp CancelOperation(1:TCancelOperationReq req); - - TCloseOperationResp CloseOperation(1:TCloseOperationReq req); - - TGetResultSetMetadataResp GetResultSetMetadata(1:TGetResultSetMetadataReq req); - - TFetchResultsResp FetchResults(1:TFetchResultsReq req); - - TGetDelegationTokenResp GetDelegationToken(1:TGetDelegationTokenReq req); - - TCancelDelegationTokenResp CancelDelegationToken(1:TCancelDelegationTokenReq req); - - TRenewDelegationTokenResp RenewDelegationToken(1:TRenewDelegationTokenReq req); -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java deleted file mode 100644 index 6323d34eac734..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java +++ /dev/null @@ -1,383 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TArrayTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TArrayTypeEntry"); - - private static final org.apache.thrift.protocol.TField OBJECT_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("objectTypePtr", org.apache.thrift.protocol.TType.I32, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TArrayTypeEntryStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TArrayTypeEntryTupleSchemeFactory()); - } - - private int objectTypePtr; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - OBJECT_TYPE_PTR((short)1, "objectTypePtr"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // OBJECT_TYPE_PTR - return OBJECT_TYPE_PTR; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __OBJECTTYPEPTR_ISSET_ID = 0; - private byte __isset_bitfield = 0; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.OBJECT_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("objectTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr"))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TArrayTypeEntry.class, metaDataMap); - } - - public TArrayTypeEntry() { - } - - public TArrayTypeEntry( - int objectTypePtr) - { - this(); - this.objectTypePtr = objectTypePtr; - setObjectTypePtrIsSet(true); - } - - /** - * Performs a deep copy on other. - */ - public TArrayTypeEntry(TArrayTypeEntry other) { - __isset_bitfield = other.__isset_bitfield; - this.objectTypePtr = other.objectTypePtr; - } - - public TArrayTypeEntry deepCopy() { - return new TArrayTypeEntry(this); - } - - @Override - public void clear() { - setObjectTypePtrIsSet(false); - this.objectTypePtr = 0; - } - - public int getObjectTypePtr() { - return this.objectTypePtr; - } - - public void setObjectTypePtr(int objectTypePtr) { - this.objectTypePtr = objectTypePtr; - setObjectTypePtrIsSet(true); - } - - public void unsetObjectTypePtr() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __OBJECTTYPEPTR_ISSET_ID); - } - - /** Returns true if field objectTypePtr is set (has been assigned a value) and false otherwise */ - public boolean isSetObjectTypePtr() { - return EncodingUtils.testBit(__isset_bitfield, __OBJECTTYPEPTR_ISSET_ID); - } - - public void setObjectTypePtrIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __OBJECTTYPEPTR_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case OBJECT_TYPE_PTR: - if (value == null) { - unsetObjectTypePtr(); - } else { - setObjectTypePtr((Integer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case OBJECT_TYPE_PTR: - return Integer.valueOf(getObjectTypePtr()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case OBJECT_TYPE_PTR: - return isSetObjectTypePtr(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TArrayTypeEntry) - return this.equals((TArrayTypeEntry)that); - return false; - } - - public boolean equals(TArrayTypeEntry that) { - if (that == null) - return false; - - boolean this_present_objectTypePtr = true; - boolean that_present_objectTypePtr = true; - if (this_present_objectTypePtr || that_present_objectTypePtr) { - if (!(this_present_objectTypePtr && that_present_objectTypePtr)) - return false; - if (this.objectTypePtr != that.objectTypePtr) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_objectTypePtr = true; - builder.append(present_objectTypePtr); - if (present_objectTypePtr) - builder.append(objectTypePtr); - - return builder.toHashCode(); - } - - public int compareTo(TArrayTypeEntry other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TArrayTypeEntry typedOther = (TArrayTypeEntry)other; - - lastComparison = Boolean.valueOf(isSetObjectTypePtr()).compareTo(typedOther.isSetObjectTypePtr()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetObjectTypePtr()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.objectTypePtr, typedOther.objectTypePtr); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TArrayTypeEntry("); - boolean first = true; - - sb.append("objectTypePtr:"); - sb.append(this.objectTypePtr); - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetObjectTypePtr()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'objectTypePtr' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TArrayTypeEntryStandardSchemeFactory implements SchemeFactory { - public TArrayTypeEntryStandardScheme getScheme() { - return new TArrayTypeEntryStandardScheme(); - } - } - - private static class TArrayTypeEntryStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TArrayTypeEntry struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // OBJECT_TYPE_PTR - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.objectTypePtr = iprot.readI32(); - struct.setObjectTypePtrIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TArrayTypeEntry struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(OBJECT_TYPE_PTR_FIELD_DESC); - oprot.writeI32(struct.objectTypePtr); - oprot.writeFieldEnd(); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TArrayTypeEntryTupleSchemeFactory implements SchemeFactory { - public TArrayTypeEntryTupleScheme getScheme() { - return new TArrayTypeEntryTupleScheme(); - } - } - - private static class TArrayTypeEntryTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TArrayTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeI32(struct.objectTypePtr); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TArrayTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.objectTypePtr = iprot.readI32(); - struct.setObjectTypePtrIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java deleted file mode 100644 index 6b1b054d1acad..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java +++ /dev/null @@ -1,550 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TBinaryColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBinaryColumn"); - - private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); - private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TBinaryColumnStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TBinaryColumnTupleSchemeFactory()); - } - - private List values; // required - private ByteBuffer nulls; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUES((short)1, "values"), - NULLS((short)2, "nulls"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUES - return VALUES; - case 2: // NULLS - return NULLS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)))); - tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TBinaryColumn.class, metaDataMap); - } - - public TBinaryColumn() { - } - - public TBinaryColumn( - List values, - ByteBuffer nulls) - { - this(); - this.values = values; - this.nulls = nulls; - } - - /** - * Performs a deep copy on other. - */ - public TBinaryColumn(TBinaryColumn other) { - if (other.isSetValues()) { - List __this__values = new ArrayList(); - for (ByteBuffer other_element : other.values) { - ByteBuffer temp_binary_element = org.apache.thrift.TBaseHelper.copyBinary(other_element); -; - __this__values.add(temp_binary_element); - } - this.values = __this__values; - } - if (other.isSetNulls()) { - this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); -; - } - } - - public TBinaryColumn deepCopy() { - return new TBinaryColumn(this); - } - - @Override - public void clear() { - this.values = null; - this.nulls = null; - } - - public int getValuesSize() { - return (this.values == null) ? 0 : this.values.size(); - } - - public java.util.Iterator getValuesIterator() { - return (this.values == null) ? null : this.values.iterator(); - } - - public void addToValues(ByteBuffer elem) { - if (this.values == null) { - this.values = new ArrayList(); - } - this.values.add(elem); - } - - public List getValues() { - return this.values; - } - - public void setValues(List values) { - this.values = values; - } - - public void unsetValues() { - this.values = null; - } - - /** Returns true if field values is set (has been assigned a value) and false otherwise */ - public boolean isSetValues() { - return this.values != null; - } - - public void setValuesIsSet(boolean value) { - if (!value) { - this.values = null; - } - } - - public byte[] getNulls() { - setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); - return nulls == null ? null : nulls.array(); - } - - public ByteBuffer bufferForNulls() { - return nulls; - } - - public void setNulls(byte[] nulls) { - setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); - } - - public void setNulls(ByteBuffer nulls) { - this.nulls = nulls; - } - - public void unsetNulls() { - this.nulls = null; - } - - /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ - public boolean isSetNulls() { - return this.nulls != null; - } - - public void setNullsIsSet(boolean value) { - if (!value) { - this.nulls = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUES: - if (value == null) { - unsetValues(); - } else { - setValues((List)value); - } - break; - - case NULLS: - if (value == null) { - unsetNulls(); - } else { - setNulls((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUES: - return getValues(); - - case NULLS: - return getNulls(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUES: - return isSetValues(); - case NULLS: - return isSetNulls(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TBinaryColumn) - return this.equals((TBinaryColumn)that); - return false; - } - - public boolean equals(TBinaryColumn that) { - if (that == null) - return false; - - boolean this_present_values = true && this.isSetValues(); - boolean that_present_values = true && that.isSetValues(); - if (this_present_values || that_present_values) { - if (!(this_present_values && that_present_values)) - return false; - if (!this.values.equals(that.values)) - return false; - } - - boolean this_present_nulls = true && this.isSetNulls(); - boolean that_present_nulls = true && that.isSetNulls(); - if (this_present_nulls || that_present_nulls) { - if (!(this_present_nulls && that_present_nulls)) - return false; - if (!this.nulls.equals(that.nulls)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_values = true && (isSetValues()); - builder.append(present_values); - if (present_values) - builder.append(values); - - boolean present_nulls = true && (isSetNulls()); - builder.append(present_nulls); - if (present_nulls) - builder.append(nulls); - - return builder.toHashCode(); - } - - public int compareTo(TBinaryColumn other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TBinaryColumn typedOther = (TBinaryColumn)other; - - lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNulls()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TBinaryColumn("); - boolean first = true; - - sb.append("values:"); - if (this.values == null) { - sb.append("null"); - } else { - sb.append(this.values); - } - first = false; - if (!first) sb.append(", "); - sb.append("nulls:"); - if (this.nulls == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.nulls, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetValues()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); - } - - if (!isSetNulls()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TBinaryColumnStandardSchemeFactory implements SchemeFactory { - public TBinaryColumnStandardScheme getScheme() { - return new TBinaryColumnStandardScheme(); - } - } - - private static class TBinaryColumnStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TBinaryColumn struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list110 = iprot.readListBegin(); - struct.values = new ArrayList(_list110.size); - for (int _i111 = 0; _i111 < _list110.size; ++_i111) - { - ByteBuffer _elem112; // optional - _elem112 = iprot.readBinary(); - struct.values.add(_elem112); - } - iprot.readListEnd(); - } - struct.setValuesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // NULLS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TBinaryColumn struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.values != null) { - oprot.writeFieldBegin(VALUES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.values.size())); - for (ByteBuffer _iter113 : struct.values) - { - oprot.writeBinary(_iter113); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.nulls != null) { - oprot.writeFieldBegin(NULLS_FIELD_DESC); - oprot.writeBinary(struct.nulls); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TBinaryColumnTupleSchemeFactory implements SchemeFactory { - public TBinaryColumnTupleScheme getScheme() { - return new TBinaryColumnTupleScheme(); - } - } - - private static class TBinaryColumnTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TBinaryColumn struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.values.size()); - for (ByteBuffer _iter114 : struct.values) - { - oprot.writeBinary(_iter114); - } - } - oprot.writeBinary(struct.nulls); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TBinaryColumn struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list115 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.values = new ArrayList(_list115.size); - for (int _i116 = 0; _i116 < _list115.size; ++_i116) - { - ByteBuffer _elem117; // optional - _elem117 = iprot.readBinary(); - struct.values.add(_elem117); - } - } - struct.setValuesIsSet(true); - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java deleted file mode 100644 index efd571cfdfbbf..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java +++ /dev/null @@ -1,548 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TBoolColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBoolColumn"); - - private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); - private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TBoolColumnStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TBoolColumnTupleSchemeFactory()); - } - - private List values; // required - private ByteBuffer nulls; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUES((short)1, "values"), - NULLS((short)2, "nulls"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUES - return VALUES; - case 2: // NULLS - return NULLS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)))); - tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TBoolColumn.class, metaDataMap); - } - - public TBoolColumn() { - } - - public TBoolColumn( - List values, - ByteBuffer nulls) - { - this(); - this.values = values; - this.nulls = nulls; - } - - /** - * Performs a deep copy on other. - */ - public TBoolColumn(TBoolColumn other) { - if (other.isSetValues()) { - List __this__values = new ArrayList(); - for (Boolean other_element : other.values) { - __this__values.add(other_element); - } - this.values = __this__values; - } - if (other.isSetNulls()) { - this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); -; - } - } - - public TBoolColumn deepCopy() { - return new TBoolColumn(this); - } - - @Override - public void clear() { - this.values = null; - this.nulls = null; - } - - public int getValuesSize() { - return (this.values == null) ? 0 : this.values.size(); - } - - public java.util.Iterator getValuesIterator() { - return (this.values == null) ? null : this.values.iterator(); - } - - public void addToValues(boolean elem) { - if (this.values == null) { - this.values = new ArrayList(); - } - this.values.add(elem); - } - - public List getValues() { - return this.values; - } - - public void setValues(List values) { - this.values = values; - } - - public void unsetValues() { - this.values = null; - } - - /** Returns true if field values is set (has been assigned a value) and false otherwise */ - public boolean isSetValues() { - return this.values != null; - } - - public void setValuesIsSet(boolean value) { - if (!value) { - this.values = null; - } - } - - public byte[] getNulls() { - setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); - return nulls == null ? null : nulls.array(); - } - - public ByteBuffer bufferForNulls() { - return nulls; - } - - public void setNulls(byte[] nulls) { - setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); - } - - public void setNulls(ByteBuffer nulls) { - this.nulls = nulls; - } - - public void unsetNulls() { - this.nulls = null; - } - - /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ - public boolean isSetNulls() { - return this.nulls != null; - } - - public void setNullsIsSet(boolean value) { - if (!value) { - this.nulls = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUES: - if (value == null) { - unsetValues(); - } else { - setValues((List)value); - } - break; - - case NULLS: - if (value == null) { - unsetNulls(); - } else { - setNulls((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUES: - return getValues(); - - case NULLS: - return getNulls(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUES: - return isSetValues(); - case NULLS: - return isSetNulls(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TBoolColumn) - return this.equals((TBoolColumn)that); - return false; - } - - public boolean equals(TBoolColumn that) { - if (that == null) - return false; - - boolean this_present_values = true && this.isSetValues(); - boolean that_present_values = true && that.isSetValues(); - if (this_present_values || that_present_values) { - if (!(this_present_values && that_present_values)) - return false; - if (!this.values.equals(that.values)) - return false; - } - - boolean this_present_nulls = true && this.isSetNulls(); - boolean that_present_nulls = true && that.isSetNulls(); - if (this_present_nulls || that_present_nulls) { - if (!(this_present_nulls && that_present_nulls)) - return false; - if (!this.nulls.equals(that.nulls)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_values = true && (isSetValues()); - builder.append(present_values); - if (present_values) - builder.append(values); - - boolean present_nulls = true && (isSetNulls()); - builder.append(present_nulls); - if (present_nulls) - builder.append(nulls); - - return builder.toHashCode(); - } - - public int compareTo(TBoolColumn other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TBoolColumn typedOther = (TBoolColumn)other; - - lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNulls()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TBoolColumn("); - boolean first = true; - - sb.append("values:"); - if (this.values == null) { - sb.append("null"); - } else { - sb.append(this.values); - } - first = false; - if (!first) sb.append(", "); - sb.append("nulls:"); - if (this.nulls == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.nulls, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetValues()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); - } - - if (!isSetNulls()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TBoolColumnStandardSchemeFactory implements SchemeFactory { - public TBoolColumnStandardScheme getScheme() { - return new TBoolColumnStandardScheme(); - } - } - - private static class TBoolColumnStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TBoolColumn struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list54 = iprot.readListBegin(); - struct.values = new ArrayList(_list54.size); - for (int _i55 = 0; _i55 < _list54.size; ++_i55) - { - boolean _elem56; // optional - _elem56 = iprot.readBool(); - struct.values.add(_elem56); - } - iprot.readListEnd(); - } - struct.setValuesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // NULLS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TBoolColumn struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.values != null) { - oprot.writeFieldBegin(VALUES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BOOL, struct.values.size())); - for (boolean _iter57 : struct.values) - { - oprot.writeBool(_iter57); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.nulls != null) { - oprot.writeFieldBegin(NULLS_FIELD_DESC); - oprot.writeBinary(struct.nulls); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TBoolColumnTupleSchemeFactory implements SchemeFactory { - public TBoolColumnTupleScheme getScheme() { - return new TBoolColumnTupleScheme(); - } - } - - private static class TBoolColumnTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TBoolColumn struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.values.size()); - for (boolean _iter58 : struct.values) - { - oprot.writeBool(_iter58); - } - } - oprot.writeBinary(struct.nulls); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TBoolColumn struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list59 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BOOL, iprot.readI32()); - struct.values = new ArrayList(_list59.size); - for (int _i60 = 0; _i60 < _list59.size; ++_i60) - { - boolean _elem61; // optional - _elem61 = iprot.readBool(); - struct.values.add(_elem61); - } - } - struct.setValuesIsSet(true); - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java deleted file mode 100644 index c7495ee79e4b5..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java +++ /dev/null @@ -1,386 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TBoolValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBoolValue"); - - private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.BOOL, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TBoolValueStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TBoolValueTupleSchemeFactory()); - } - - private boolean value; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUE((short)1, "value"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUE - return VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __VALUE_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.VALUE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TBoolValue.class, metaDataMap); - } - - public TBoolValue() { - } - - /** - * Performs a deep copy on other. - */ - public TBoolValue(TBoolValue other) { - __isset_bitfield = other.__isset_bitfield; - this.value = other.value; - } - - public TBoolValue deepCopy() { - return new TBoolValue(this); - } - - @Override - public void clear() { - setValueIsSet(false); - this.value = false; - } - - public boolean isValue() { - return this.value; - } - - public void setValue(boolean value) { - this.value = value; - setValueIsSet(true); - } - - public void unsetValue() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - /** Returns true if field value is set (has been assigned a value) and false otherwise */ - public boolean isSetValue() { - return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - public void setValueIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUE: - if (value == null) { - unsetValue(); - } else { - setValue((Boolean)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUE: - return Boolean.valueOf(isValue()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUE: - return isSetValue(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TBoolValue) - return this.equals((TBoolValue)that); - return false; - } - - public boolean equals(TBoolValue that) { - if (that == null) - return false; - - boolean this_present_value = true && this.isSetValue(); - boolean that_present_value = true && that.isSetValue(); - if (this_present_value || that_present_value) { - if (!(this_present_value && that_present_value)) - return false; - if (this.value != that.value) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_value = true && (isSetValue()); - builder.append(present_value); - if (present_value) - builder.append(value); - - return builder.toHashCode(); - } - - public int compareTo(TBoolValue other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TBoolValue typedOther = (TBoolValue)other; - - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TBoolValue("); - boolean first = true; - - if (isSetValue()) { - sb.append("value:"); - sb.append(this.value); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TBoolValueStandardSchemeFactory implements SchemeFactory { - public TBoolValueStandardScheme getScheme() { - return new TBoolValueStandardScheme(); - } - } - - private static class TBoolValueStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TBoolValue struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUE - if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { - struct.value = iprot.readBool(); - struct.setValueIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TBoolValue struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetValue()) { - oprot.writeFieldBegin(VALUE_FIELD_DESC); - oprot.writeBool(struct.value); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TBoolValueTupleSchemeFactory implements SchemeFactory { - public TBoolValueTupleScheme getScheme() { - return new TBoolValueTupleScheme(); - } - } - - private static class TBoolValueTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TBoolValue struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetValue()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetValue()) { - oprot.writeBool(struct.value); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TBoolValue struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.value = iprot.readBool(); - struct.setValueIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java deleted file mode 100644 index 169bfdeab3eea..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java +++ /dev/null @@ -1,548 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TByteColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TByteColumn"); - - private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); - private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TByteColumnStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TByteColumnTupleSchemeFactory()); - } - - private List values; // required - private ByteBuffer nulls; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUES((short)1, "values"), - NULLS((short)2, "nulls"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUES - return VALUES; - case 2: // NULLS - return NULLS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE)))); - tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TByteColumn.class, metaDataMap); - } - - public TByteColumn() { - } - - public TByteColumn( - List values, - ByteBuffer nulls) - { - this(); - this.values = values; - this.nulls = nulls; - } - - /** - * Performs a deep copy on other. - */ - public TByteColumn(TByteColumn other) { - if (other.isSetValues()) { - List __this__values = new ArrayList(); - for (Byte other_element : other.values) { - __this__values.add(other_element); - } - this.values = __this__values; - } - if (other.isSetNulls()) { - this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); -; - } - } - - public TByteColumn deepCopy() { - return new TByteColumn(this); - } - - @Override - public void clear() { - this.values = null; - this.nulls = null; - } - - public int getValuesSize() { - return (this.values == null) ? 0 : this.values.size(); - } - - public java.util.Iterator getValuesIterator() { - return (this.values == null) ? null : this.values.iterator(); - } - - public void addToValues(byte elem) { - if (this.values == null) { - this.values = new ArrayList(); - } - this.values.add(elem); - } - - public List getValues() { - return this.values; - } - - public void setValues(List values) { - this.values = values; - } - - public void unsetValues() { - this.values = null; - } - - /** Returns true if field values is set (has been assigned a value) and false otherwise */ - public boolean isSetValues() { - return this.values != null; - } - - public void setValuesIsSet(boolean value) { - if (!value) { - this.values = null; - } - } - - public byte[] getNulls() { - setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); - return nulls == null ? null : nulls.array(); - } - - public ByteBuffer bufferForNulls() { - return nulls; - } - - public void setNulls(byte[] nulls) { - setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); - } - - public void setNulls(ByteBuffer nulls) { - this.nulls = nulls; - } - - public void unsetNulls() { - this.nulls = null; - } - - /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ - public boolean isSetNulls() { - return this.nulls != null; - } - - public void setNullsIsSet(boolean value) { - if (!value) { - this.nulls = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUES: - if (value == null) { - unsetValues(); - } else { - setValues((List)value); - } - break; - - case NULLS: - if (value == null) { - unsetNulls(); - } else { - setNulls((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUES: - return getValues(); - - case NULLS: - return getNulls(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUES: - return isSetValues(); - case NULLS: - return isSetNulls(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TByteColumn) - return this.equals((TByteColumn)that); - return false; - } - - public boolean equals(TByteColumn that) { - if (that == null) - return false; - - boolean this_present_values = true && this.isSetValues(); - boolean that_present_values = true && that.isSetValues(); - if (this_present_values || that_present_values) { - if (!(this_present_values && that_present_values)) - return false; - if (!this.values.equals(that.values)) - return false; - } - - boolean this_present_nulls = true && this.isSetNulls(); - boolean that_present_nulls = true && that.isSetNulls(); - if (this_present_nulls || that_present_nulls) { - if (!(this_present_nulls && that_present_nulls)) - return false; - if (!this.nulls.equals(that.nulls)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_values = true && (isSetValues()); - builder.append(present_values); - if (present_values) - builder.append(values); - - boolean present_nulls = true && (isSetNulls()); - builder.append(present_nulls); - if (present_nulls) - builder.append(nulls); - - return builder.toHashCode(); - } - - public int compareTo(TByteColumn other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TByteColumn typedOther = (TByteColumn)other; - - lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNulls()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TByteColumn("); - boolean first = true; - - sb.append("values:"); - if (this.values == null) { - sb.append("null"); - } else { - sb.append(this.values); - } - first = false; - if (!first) sb.append(", "); - sb.append("nulls:"); - if (this.nulls == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.nulls, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetValues()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); - } - - if (!isSetNulls()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TByteColumnStandardSchemeFactory implements SchemeFactory { - public TByteColumnStandardScheme getScheme() { - return new TByteColumnStandardScheme(); - } - } - - private static class TByteColumnStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TByteColumn struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list62 = iprot.readListBegin(); - struct.values = new ArrayList(_list62.size); - for (int _i63 = 0; _i63 < _list62.size; ++_i63) - { - byte _elem64; // optional - _elem64 = iprot.readByte(); - struct.values.add(_elem64); - } - iprot.readListEnd(); - } - struct.setValuesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // NULLS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TByteColumn struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.values != null) { - oprot.writeFieldBegin(VALUES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BYTE, struct.values.size())); - for (byte _iter65 : struct.values) - { - oprot.writeByte(_iter65); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.nulls != null) { - oprot.writeFieldBegin(NULLS_FIELD_DESC); - oprot.writeBinary(struct.nulls); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TByteColumnTupleSchemeFactory implements SchemeFactory { - public TByteColumnTupleScheme getScheme() { - return new TByteColumnTupleScheme(); - } - } - - private static class TByteColumnTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TByteColumn struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.values.size()); - for (byte _iter66 : struct.values) - { - oprot.writeByte(_iter66); - } - } - oprot.writeBinary(struct.nulls); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TByteColumn struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list67 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BYTE, iprot.readI32()); - struct.values = new ArrayList(_list67.size); - for (int _i68 = 0; _i68 < _list67.size; ++_i68) - { - byte _elem69; // optional - _elem69 = iprot.readByte(); - struct.values.add(_elem69); - } - } - struct.setValuesIsSet(true); - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java deleted file mode 100644 index 23d9693759968..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java +++ /dev/null @@ -1,386 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TByteValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TByteValue"); - - private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.BYTE, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TByteValueStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TByteValueTupleSchemeFactory()); - } - - private byte value; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUE((short)1, "value"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUE - return VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __VALUE_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.VALUE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TByteValue.class, metaDataMap); - } - - public TByteValue() { - } - - /** - * Performs a deep copy on other. - */ - public TByteValue(TByteValue other) { - __isset_bitfield = other.__isset_bitfield; - this.value = other.value; - } - - public TByteValue deepCopy() { - return new TByteValue(this); - } - - @Override - public void clear() { - setValueIsSet(false); - this.value = 0; - } - - public byte getValue() { - return this.value; - } - - public void setValue(byte value) { - this.value = value; - setValueIsSet(true); - } - - public void unsetValue() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - /** Returns true if field value is set (has been assigned a value) and false otherwise */ - public boolean isSetValue() { - return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - public void setValueIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUE: - if (value == null) { - unsetValue(); - } else { - setValue((Byte)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUE: - return Byte.valueOf(getValue()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUE: - return isSetValue(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TByteValue) - return this.equals((TByteValue)that); - return false; - } - - public boolean equals(TByteValue that) { - if (that == null) - return false; - - boolean this_present_value = true && this.isSetValue(); - boolean that_present_value = true && that.isSetValue(); - if (this_present_value || that_present_value) { - if (!(this_present_value && that_present_value)) - return false; - if (this.value != that.value) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_value = true && (isSetValue()); - builder.append(present_value); - if (present_value) - builder.append(value); - - return builder.toHashCode(); - } - - public int compareTo(TByteValue other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TByteValue typedOther = (TByteValue)other; - - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TByteValue("); - boolean first = true; - - if (isSetValue()) { - sb.append("value:"); - sb.append(this.value); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TByteValueStandardSchemeFactory implements SchemeFactory { - public TByteValueStandardScheme getScheme() { - return new TByteValueStandardScheme(); - } - } - - private static class TByteValueStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TByteValue struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUE - if (schemeField.type == org.apache.thrift.protocol.TType.BYTE) { - struct.value = iprot.readByte(); - struct.setValueIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TByteValue struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetValue()) { - oprot.writeFieldBegin(VALUE_FIELD_DESC); - oprot.writeByte(struct.value); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TByteValueTupleSchemeFactory implements SchemeFactory { - public TByteValueTupleScheme getScheme() { - return new TByteValueTupleScheme(); - } - } - - private static class TByteValueTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TByteValue struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetValue()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetValue()) { - oprot.writeByte(struct.value); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TByteValue struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.value = iprot.readByte(); - struct.setValueIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java deleted file mode 100644 index 54851b8d51317..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java +++ /dev/null @@ -1,15414 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCLIService { - - public interface Iface { - - public TOpenSessionResp OpenSession(TOpenSessionReq req) throws org.apache.thrift.TException; - - public TCloseSessionResp CloseSession(TCloseSessionReq req) throws org.apache.thrift.TException; - - public TGetInfoResp GetInfo(TGetInfoReq req) throws org.apache.thrift.TException; - - public TExecuteStatementResp ExecuteStatement(TExecuteStatementReq req) throws org.apache.thrift.TException; - - public TGetTypeInfoResp GetTypeInfo(TGetTypeInfoReq req) throws org.apache.thrift.TException; - - public TGetCatalogsResp GetCatalogs(TGetCatalogsReq req) throws org.apache.thrift.TException; - - public TGetSchemasResp GetSchemas(TGetSchemasReq req) throws org.apache.thrift.TException; - - public TGetTablesResp GetTables(TGetTablesReq req) throws org.apache.thrift.TException; - - public TGetTableTypesResp GetTableTypes(TGetTableTypesReq req) throws org.apache.thrift.TException; - - public TGetColumnsResp GetColumns(TGetColumnsReq req) throws org.apache.thrift.TException; - - public TGetFunctionsResp GetFunctions(TGetFunctionsReq req) throws org.apache.thrift.TException; - - public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) throws org.apache.thrift.TException; - - public TCancelOperationResp CancelOperation(TCancelOperationReq req) throws org.apache.thrift.TException; - - public TCloseOperationResp CloseOperation(TCloseOperationReq req) throws org.apache.thrift.TException; - - public TGetResultSetMetadataResp GetResultSetMetadata(TGetResultSetMetadataReq req) throws org.apache.thrift.TException; - - public TFetchResultsResp FetchResults(TFetchResultsReq req) throws org.apache.thrift.TException; - - public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) throws org.apache.thrift.TException; - - public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) throws org.apache.thrift.TException; - - public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) throws org.apache.thrift.TException; - - } - - public interface AsyncIface { - - public void OpenSession(TOpenSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void CloseSession(TCloseSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetInfo(TGetInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void ExecuteStatement(TExecuteStatementReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetTypeInfo(TGetTypeInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetCatalogs(TGetCatalogsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetSchemas(TGetSchemasReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetTables(TGetTablesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetTableTypes(TGetTableTypesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetColumns(TGetColumnsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetFunctions(TGetFunctionsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetOperationStatus(TGetOperationStatusReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void CancelOperation(TCancelOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void CloseOperation(TCloseOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetResultSetMetadata(TGetResultSetMetadataReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void FetchResults(TFetchResultsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetDelegationToken(TGetDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void CancelDelegationToken(TCancelDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void RenewDelegationToken(TRenewDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - } - - public static class Client extends org.apache.thrift.TServiceClient implements Iface { - public static class Factory implements org.apache.thrift.TServiceClientFactory { - public Factory() {} - public Client getClient(org.apache.thrift.protocol.TProtocol prot) { - return new Client(prot); - } - public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { - return new Client(iprot, oprot); - } - } - - public Client(org.apache.thrift.protocol.TProtocol prot) - { - super(prot, prot); - } - - public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { - super(iprot, oprot); - } - - public TOpenSessionResp OpenSession(TOpenSessionReq req) throws org.apache.thrift.TException - { - send_OpenSession(req); - return recv_OpenSession(); - } - - public void send_OpenSession(TOpenSessionReq req) throws org.apache.thrift.TException - { - OpenSession_args args = new OpenSession_args(); - args.setReq(req); - sendBase("OpenSession", args); - } - - public TOpenSessionResp recv_OpenSession() throws org.apache.thrift.TException - { - OpenSession_result result = new OpenSession_result(); - receiveBase(result, "OpenSession"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "OpenSession failed: unknown result"); - } - - public TCloseSessionResp CloseSession(TCloseSessionReq req) throws org.apache.thrift.TException - { - send_CloseSession(req); - return recv_CloseSession(); - } - - public void send_CloseSession(TCloseSessionReq req) throws org.apache.thrift.TException - { - CloseSession_args args = new CloseSession_args(); - args.setReq(req); - sendBase("CloseSession", args); - } - - public TCloseSessionResp recv_CloseSession() throws org.apache.thrift.TException - { - CloseSession_result result = new CloseSession_result(); - receiveBase(result, "CloseSession"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CloseSession failed: unknown result"); - } - - public TGetInfoResp GetInfo(TGetInfoReq req) throws org.apache.thrift.TException - { - send_GetInfo(req); - return recv_GetInfo(); - } - - public void send_GetInfo(TGetInfoReq req) throws org.apache.thrift.TException - { - GetInfo_args args = new GetInfo_args(); - args.setReq(req); - sendBase("GetInfo", args); - } - - public TGetInfoResp recv_GetInfo() throws org.apache.thrift.TException - { - GetInfo_result result = new GetInfo_result(); - receiveBase(result, "GetInfo"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetInfo failed: unknown result"); - } - - public TExecuteStatementResp ExecuteStatement(TExecuteStatementReq req) throws org.apache.thrift.TException - { - send_ExecuteStatement(req); - return recv_ExecuteStatement(); - } - - public void send_ExecuteStatement(TExecuteStatementReq req) throws org.apache.thrift.TException - { - ExecuteStatement_args args = new ExecuteStatement_args(); - args.setReq(req); - sendBase("ExecuteStatement", args); - } - - public TExecuteStatementResp recv_ExecuteStatement() throws org.apache.thrift.TException - { - ExecuteStatement_result result = new ExecuteStatement_result(); - receiveBase(result, "ExecuteStatement"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "ExecuteStatement failed: unknown result"); - } - - public TGetTypeInfoResp GetTypeInfo(TGetTypeInfoReq req) throws org.apache.thrift.TException - { - send_GetTypeInfo(req); - return recv_GetTypeInfo(); - } - - public void send_GetTypeInfo(TGetTypeInfoReq req) throws org.apache.thrift.TException - { - GetTypeInfo_args args = new GetTypeInfo_args(); - args.setReq(req); - sendBase("GetTypeInfo", args); - } - - public TGetTypeInfoResp recv_GetTypeInfo() throws org.apache.thrift.TException - { - GetTypeInfo_result result = new GetTypeInfo_result(); - receiveBase(result, "GetTypeInfo"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetTypeInfo failed: unknown result"); - } - - public TGetCatalogsResp GetCatalogs(TGetCatalogsReq req) throws org.apache.thrift.TException - { - send_GetCatalogs(req); - return recv_GetCatalogs(); - } - - public void send_GetCatalogs(TGetCatalogsReq req) throws org.apache.thrift.TException - { - GetCatalogs_args args = new GetCatalogs_args(); - args.setReq(req); - sendBase("GetCatalogs", args); - } - - public TGetCatalogsResp recv_GetCatalogs() throws org.apache.thrift.TException - { - GetCatalogs_result result = new GetCatalogs_result(); - receiveBase(result, "GetCatalogs"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetCatalogs failed: unknown result"); - } - - public TGetSchemasResp GetSchemas(TGetSchemasReq req) throws org.apache.thrift.TException - { - send_GetSchemas(req); - return recv_GetSchemas(); - } - - public void send_GetSchemas(TGetSchemasReq req) throws org.apache.thrift.TException - { - GetSchemas_args args = new GetSchemas_args(); - args.setReq(req); - sendBase("GetSchemas", args); - } - - public TGetSchemasResp recv_GetSchemas() throws org.apache.thrift.TException - { - GetSchemas_result result = new GetSchemas_result(); - receiveBase(result, "GetSchemas"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetSchemas failed: unknown result"); - } - - public TGetTablesResp GetTables(TGetTablesReq req) throws org.apache.thrift.TException - { - send_GetTables(req); - return recv_GetTables(); - } - - public void send_GetTables(TGetTablesReq req) throws org.apache.thrift.TException - { - GetTables_args args = new GetTables_args(); - args.setReq(req); - sendBase("GetTables", args); - } - - public TGetTablesResp recv_GetTables() throws org.apache.thrift.TException - { - GetTables_result result = new GetTables_result(); - receiveBase(result, "GetTables"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetTables failed: unknown result"); - } - - public TGetTableTypesResp GetTableTypes(TGetTableTypesReq req) throws org.apache.thrift.TException - { - send_GetTableTypes(req); - return recv_GetTableTypes(); - } - - public void send_GetTableTypes(TGetTableTypesReq req) throws org.apache.thrift.TException - { - GetTableTypes_args args = new GetTableTypes_args(); - args.setReq(req); - sendBase("GetTableTypes", args); - } - - public TGetTableTypesResp recv_GetTableTypes() throws org.apache.thrift.TException - { - GetTableTypes_result result = new GetTableTypes_result(); - receiveBase(result, "GetTableTypes"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetTableTypes failed: unknown result"); - } - - public TGetColumnsResp GetColumns(TGetColumnsReq req) throws org.apache.thrift.TException - { - send_GetColumns(req); - return recv_GetColumns(); - } - - public void send_GetColumns(TGetColumnsReq req) throws org.apache.thrift.TException - { - GetColumns_args args = new GetColumns_args(); - args.setReq(req); - sendBase("GetColumns", args); - } - - public TGetColumnsResp recv_GetColumns() throws org.apache.thrift.TException - { - GetColumns_result result = new GetColumns_result(); - receiveBase(result, "GetColumns"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetColumns failed: unknown result"); - } - - public TGetFunctionsResp GetFunctions(TGetFunctionsReq req) throws org.apache.thrift.TException - { - send_GetFunctions(req); - return recv_GetFunctions(); - } - - public void send_GetFunctions(TGetFunctionsReq req) throws org.apache.thrift.TException - { - GetFunctions_args args = new GetFunctions_args(); - args.setReq(req); - sendBase("GetFunctions", args); - } - - public TGetFunctionsResp recv_GetFunctions() throws org.apache.thrift.TException - { - GetFunctions_result result = new GetFunctions_result(); - receiveBase(result, "GetFunctions"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetFunctions failed: unknown result"); - } - - public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) throws org.apache.thrift.TException - { - send_GetOperationStatus(req); - return recv_GetOperationStatus(); - } - - public void send_GetOperationStatus(TGetOperationStatusReq req) throws org.apache.thrift.TException - { - GetOperationStatus_args args = new GetOperationStatus_args(); - args.setReq(req); - sendBase("GetOperationStatus", args); - } - - public TGetOperationStatusResp recv_GetOperationStatus() throws org.apache.thrift.TException - { - GetOperationStatus_result result = new GetOperationStatus_result(); - receiveBase(result, "GetOperationStatus"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetOperationStatus failed: unknown result"); - } - - public TCancelOperationResp CancelOperation(TCancelOperationReq req) throws org.apache.thrift.TException - { - send_CancelOperation(req); - return recv_CancelOperation(); - } - - public void send_CancelOperation(TCancelOperationReq req) throws org.apache.thrift.TException - { - CancelOperation_args args = new CancelOperation_args(); - args.setReq(req); - sendBase("CancelOperation", args); - } - - public TCancelOperationResp recv_CancelOperation() throws org.apache.thrift.TException - { - CancelOperation_result result = new CancelOperation_result(); - receiveBase(result, "CancelOperation"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CancelOperation failed: unknown result"); - } - - public TCloseOperationResp CloseOperation(TCloseOperationReq req) throws org.apache.thrift.TException - { - send_CloseOperation(req); - return recv_CloseOperation(); - } - - public void send_CloseOperation(TCloseOperationReq req) throws org.apache.thrift.TException - { - CloseOperation_args args = new CloseOperation_args(); - args.setReq(req); - sendBase("CloseOperation", args); - } - - public TCloseOperationResp recv_CloseOperation() throws org.apache.thrift.TException - { - CloseOperation_result result = new CloseOperation_result(); - receiveBase(result, "CloseOperation"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CloseOperation failed: unknown result"); - } - - public TGetResultSetMetadataResp GetResultSetMetadata(TGetResultSetMetadataReq req) throws org.apache.thrift.TException - { - send_GetResultSetMetadata(req); - return recv_GetResultSetMetadata(); - } - - public void send_GetResultSetMetadata(TGetResultSetMetadataReq req) throws org.apache.thrift.TException - { - GetResultSetMetadata_args args = new GetResultSetMetadata_args(); - args.setReq(req); - sendBase("GetResultSetMetadata", args); - } - - public TGetResultSetMetadataResp recv_GetResultSetMetadata() throws org.apache.thrift.TException - { - GetResultSetMetadata_result result = new GetResultSetMetadata_result(); - receiveBase(result, "GetResultSetMetadata"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetResultSetMetadata failed: unknown result"); - } - - public TFetchResultsResp FetchResults(TFetchResultsReq req) throws org.apache.thrift.TException - { - send_FetchResults(req); - return recv_FetchResults(); - } - - public void send_FetchResults(TFetchResultsReq req) throws org.apache.thrift.TException - { - FetchResults_args args = new FetchResults_args(); - args.setReq(req); - sendBase("FetchResults", args); - } - - public TFetchResultsResp recv_FetchResults() throws org.apache.thrift.TException - { - FetchResults_result result = new FetchResults_result(); - receiveBase(result, "FetchResults"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "FetchResults failed: unknown result"); - } - - public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) throws org.apache.thrift.TException - { - send_GetDelegationToken(req); - return recv_GetDelegationToken(); - } - - public void send_GetDelegationToken(TGetDelegationTokenReq req) throws org.apache.thrift.TException - { - GetDelegationToken_args args = new GetDelegationToken_args(); - args.setReq(req); - sendBase("GetDelegationToken", args); - } - - public TGetDelegationTokenResp recv_GetDelegationToken() throws org.apache.thrift.TException - { - GetDelegationToken_result result = new GetDelegationToken_result(); - receiveBase(result, "GetDelegationToken"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetDelegationToken failed: unknown result"); - } - - public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) throws org.apache.thrift.TException - { - send_CancelDelegationToken(req); - return recv_CancelDelegationToken(); - } - - public void send_CancelDelegationToken(TCancelDelegationTokenReq req) throws org.apache.thrift.TException - { - CancelDelegationToken_args args = new CancelDelegationToken_args(); - args.setReq(req); - sendBase("CancelDelegationToken", args); - } - - public TCancelDelegationTokenResp recv_CancelDelegationToken() throws org.apache.thrift.TException - { - CancelDelegationToken_result result = new CancelDelegationToken_result(); - receiveBase(result, "CancelDelegationToken"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CancelDelegationToken failed: unknown result"); - } - - public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) throws org.apache.thrift.TException - { - send_RenewDelegationToken(req); - return recv_RenewDelegationToken(); - } - - public void send_RenewDelegationToken(TRenewDelegationTokenReq req) throws org.apache.thrift.TException - { - RenewDelegationToken_args args = new RenewDelegationToken_args(); - args.setReq(req); - sendBase("RenewDelegationToken", args); - } - - public TRenewDelegationTokenResp recv_RenewDelegationToken() throws org.apache.thrift.TException - { - RenewDelegationToken_result result = new RenewDelegationToken_result(); - receiveBase(result, "RenewDelegationToken"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "RenewDelegationToken failed: unknown result"); - } - - } - public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { - public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { - private org.apache.thrift.async.TAsyncClientManager clientManager; - private org.apache.thrift.protocol.TProtocolFactory protocolFactory; - public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) { - this.clientManager = clientManager; - this.protocolFactory = protocolFactory; - } - public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) { - return new AsyncClient(protocolFactory, clientManager, transport); - } - } - - public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) { - super(protocolFactory, clientManager, transport); - } - - public void OpenSession(TOpenSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - OpenSession_call method_call = new OpenSession_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class OpenSession_call extends org.apache.thrift.async.TAsyncMethodCall { - private TOpenSessionReq req; - public OpenSession_call(TOpenSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("OpenSession", org.apache.thrift.protocol.TMessageType.CALL, 0)); - OpenSession_args args = new OpenSession_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TOpenSessionResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_OpenSession(); - } - } - - public void CloseSession(TCloseSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - CloseSession_call method_call = new CloseSession_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class CloseSession_call extends org.apache.thrift.async.TAsyncMethodCall { - private TCloseSessionReq req; - public CloseSession_call(TCloseSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CloseSession", org.apache.thrift.protocol.TMessageType.CALL, 0)); - CloseSession_args args = new CloseSession_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TCloseSessionResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_CloseSession(); - } - } - - public void GetInfo(TGetInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetInfo_call method_call = new GetInfo_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetInfo_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetInfoReq req; - public GetInfo_call(TGetInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetInfo", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetInfo_args args = new GetInfo_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetInfoResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetInfo(); - } - } - - public void ExecuteStatement(TExecuteStatementReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - ExecuteStatement_call method_call = new ExecuteStatement_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class ExecuteStatement_call extends org.apache.thrift.async.TAsyncMethodCall { - private TExecuteStatementReq req; - public ExecuteStatement_call(TExecuteStatementReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("ExecuteStatement", org.apache.thrift.protocol.TMessageType.CALL, 0)); - ExecuteStatement_args args = new ExecuteStatement_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TExecuteStatementResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_ExecuteStatement(); - } - } - - public void GetTypeInfo(TGetTypeInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetTypeInfo_call method_call = new GetTypeInfo_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetTypeInfo_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetTypeInfoReq req; - public GetTypeInfo_call(TGetTypeInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetTypeInfo", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetTypeInfo_args args = new GetTypeInfo_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetTypeInfoResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetTypeInfo(); - } - } - - public void GetCatalogs(TGetCatalogsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetCatalogs_call method_call = new GetCatalogs_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetCatalogs_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetCatalogsReq req; - public GetCatalogs_call(TGetCatalogsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetCatalogs", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetCatalogs_args args = new GetCatalogs_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetCatalogsResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetCatalogs(); - } - } - - public void GetSchemas(TGetSchemasReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetSchemas_call method_call = new GetSchemas_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetSchemas_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetSchemasReq req; - public GetSchemas_call(TGetSchemasReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetSchemas", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetSchemas_args args = new GetSchemas_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetSchemasResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetSchemas(); - } - } - - public void GetTables(TGetTablesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetTables_call method_call = new GetTables_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetTables_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetTablesReq req; - public GetTables_call(TGetTablesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetTables", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetTables_args args = new GetTables_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetTablesResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetTables(); - } - } - - public void GetTableTypes(TGetTableTypesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetTableTypes_call method_call = new GetTableTypes_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetTableTypes_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetTableTypesReq req; - public GetTableTypes_call(TGetTableTypesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetTableTypes", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetTableTypes_args args = new GetTableTypes_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetTableTypesResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetTableTypes(); - } - } - - public void GetColumns(TGetColumnsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetColumns_call method_call = new GetColumns_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetColumns_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetColumnsReq req; - public GetColumns_call(TGetColumnsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetColumns", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetColumns_args args = new GetColumns_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetColumnsResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetColumns(); - } - } - - public void GetFunctions(TGetFunctionsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetFunctions_call method_call = new GetFunctions_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetFunctions_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetFunctionsReq req; - public GetFunctions_call(TGetFunctionsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetFunctions", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetFunctions_args args = new GetFunctions_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetFunctionsResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetFunctions(); - } - } - - public void GetOperationStatus(TGetOperationStatusReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetOperationStatus_call method_call = new GetOperationStatus_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetOperationStatus_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetOperationStatusReq req; - public GetOperationStatus_call(TGetOperationStatusReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetOperationStatus", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetOperationStatus_args args = new GetOperationStatus_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetOperationStatusResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetOperationStatus(); - } - } - - public void CancelOperation(TCancelOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - CancelOperation_call method_call = new CancelOperation_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class CancelOperation_call extends org.apache.thrift.async.TAsyncMethodCall { - private TCancelOperationReq req; - public CancelOperation_call(TCancelOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CancelOperation", org.apache.thrift.protocol.TMessageType.CALL, 0)); - CancelOperation_args args = new CancelOperation_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TCancelOperationResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_CancelOperation(); - } - } - - public void CloseOperation(TCloseOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - CloseOperation_call method_call = new CloseOperation_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class CloseOperation_call extends org.apache.thrift.async.TAsyncMethodCall { - private TCloseOperationReq req; - public CloseOperation_call(TCloseOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CloseOperation", org.apache.thrift.protocol.TMessageType.CALL, 0)); - CloseOperation_args args = new CloseOperation_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TCloseOperationResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_CloseOperation(); - } - } - - public void GetResultSetMetadata(TGetResultSetMetadataReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetResultSetMetadata_call method_call = new GetResultSetMetadata_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetResultSetMetadata_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetResultSetMetadataReq req; - public GetResultSetMetadata_call(TGetResultSetMetadataReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetResultSetMetadata", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetResultSetMetadata_args args = new GetResultSetMetadata_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetResultSetMetadataResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetResultSetMetadata(); - } - } - - public void FetchResults(TFetchResultsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - FetchResults_call method_call = new FetchResults_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class FetchResults_call extends org.apache.thrift.async.TAsyncMethodCall { - private TFetchResultsReq req; - public FetchResults_call(TFetchResultsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("FetchResults", org.apache.thrift.protocol.TMessageType.CALL, 0)); - FetchResults_args args = new FetchResults_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TFetchResultsResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_FetchResults(); - } - } - - public void GetDelegationToken(TGetDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetDelegationToken_call method_call = new GetDelegationToken_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetDelegationTokenReq req; - public GetDelegationToken_call(TGetDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetDelegationToken", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetDelegationToken_args args = new GetDelegationToken_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetDelegationTokenResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetDelegationToken(); - } - } - - public void CancelDelegationToken(TCancelDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - CancelDelegationToken_call method_call = new CancelDelegationToken_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class CancelDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall { - private TCancelDelegationTokenReq req; - public CancelDelegationToken_call(TCancelDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CancelDelegationToken", org.apache.thrift.protocol.TMessageType.CALL, 0)); - CancelDelegationToken_args args = new CancelDelegationToken_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TCancelDelegationTokenResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_CancelDelegationToken(); - } - } - - public void RenewDelegationToken(TRenewDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - RenewDelegationToken_call method_call = new RenewDelegationToken_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class RenewDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall { - private TRenewDelegationTokenReq req; - public RenewDelegationToken_call(TRenewDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("RenewDelegationToken", org.apache.thrift.protocol.TMessageType.CALL, 0)); - RenewDelegationToken_args args = new RenewDelegationToken_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TRenewDelegationTokenResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_RenewDelegationToken(); - } - } - - } - - public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor { - private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName()); - public Processor(I iface) { - super(iface, getProcessMap(new HashMap>())); - } - - protected Processor(I iface, Map> processMap) { - super(iface, getProcessMap(processMap)); - } - - private static Map> getProcessMap(Map> processMap) { - processMap.put("OpenSession", new OpenSession()); - processMap.put("CloseSession", new CloseSession()); - processMap.put("GetInfo", new GetInfo()); - processMap.put("ExecuteStatement", new ExecuteStatement()); - processMap.put("GetTypeInfo", new GetTypeInfo()); - processMap.put("GetCatalogs", new GetCatalogs()); - processMap.put("GetSchemas", new GetSchemas()); - processMap.put("GetTables", new GetTables()); - processMap.put("GetTableTypes", new GetTableTypes()); - processMap.put("GetColumns", new GetColumns()); - processMap.put("GetFunctions", new GetFunctions()); - processMap.put("GetOperationStatus", new GetOperationStatus()); - processMap.put("CancelOperation", new CancelOperation()); - processMap.put("CloseOperation", new CloseOperation()); - processMap.put("GetResultSetMetadata", new GetResultSetMetadata()); - processMap.put("FetchResults", new FetchResults()); - processMap.put("GetDelegationToken", new GetDelegationToken()); - processMap.put("CancelDelegationToken", new CancelDelegationToken()); - processMap.put("RenewDelegationToken", new RenewDelegationToken()); - return processMap; - } - - public static class OpenSession extends org.apache.thrift.ProcessFunction { - public OpenSession() { - super("OpenSession"); - } - - public OpenSession_args getEmptyArgsInstance() { - return new OpenSession_args(); - } - - protected boolean isOneway() { - return false; - } - - public OpenSession_result getResult(I iface, OpenSession_args args) throws org.apache.thrift.TException { - OpenSession_result result = new OpenSession_result(); - result.success = iface.OpenSession(args.req); - return result; - } - } - - public static class CloseSession extends org.apache.thrift.ProcessFunction { - public CloseSession() { - super("CloseSession"); - } - - public CloseSession_args getEmptyArgsInstance() { - return new CloseSession_args(); - } - - protected boolean isOneway() { - return false; - } - - public CloseSession_result getResult(I iface, CloseSession_args args) throws org.apache.thrift.TException { - CloseSession_result result = new CloseSession_result(); - result.success = iface.CloseSession(args.req); - return result; - } - } - - public static class GetInfo extends org.apache.thrift.ProcessFunction { - public GetInfo() { - super("GetInfo"); - } - - public GetInfo_args getEmptyArgsInstance() { - return new GetInfo_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetInfo_result getResult(I iface, GetInfo_args args) throws org.apache.thrift.TException { - GetInfo_result result = new GetInfo_result(); - result.success = iface.GetInfo(args.req); - return result; - } - } - - public static class ExecuteStatement extends org.apache.thrift.ProcessFunction { - public ExecuteStatement() { - super("ExecuteStatement"); - } - - public ExecuteStatement_args getEmptyArgsInstance() { - return new ExecuteStatement_args(); - } - - protected boolean isOneway() { - return false; - } - - public ExecuteStatement_result getResult(I iface, ExecuteStatement_args args) throws org.apache.thrift.TException { - ExecuteStatement_result result = new ExecuteStatement_result(); - result.success = iface.ExecuteStatement(args.req); - return result; - } - } - - public static class GetTypeInfo extends org.apache.thrift.ProcessFunction { - public GetTypeInfo() { - super("GetTypeInfo"); - } - - public GetTypeInfo_args getEmptyArgsInstance() { - return new GetTypeInfo_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetTypeInfo_result getResult(I iface, GetTypeInfo_args args) throws org.apache.thrift.TException { - GetTypeInfo_result result = new GetTypeInfo_result(); - result.success = iface.GetTypeInfo(args.req); - return result; - } - } - - public static class GetCatalogs extends org.apache.thrift.ProcessFunction { - public GetCatalogs() { - super("GetCatalogs"); - } - - public GetCatalogs_args getEmptyArgsInstance() { - return new GetCatalogs_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetCatalogs_result getResult(I iface, GetCatalogs_args args) throws org.apache.thrift.TException { - GetCatalogs_result result = new GetCatalogs_result(); - result.success = iface.GetCatalogs(args.req); - return result; - } - } - - public static class GetSchemas extends org.apache.thrift.ProcessFunction { - public GetSchemas() { - super("GetSchemas"); - } - - public GetSchemas_args getEmptyArgsInstance() { - return new GetSchemas_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetSchemas_result getResult(I iface, GetSchemas_args args) throws org.apache.thrift.TException { - GetSchemas_result result = new GetSchemas_result(); - result.success = iface.GetSchemas(args.req); - return result; - } - } - - public static class GetTables extends org.apache.thrift.ProcessFunction { - public GetTables() { - super("GetTables"); - } - - public GetTables_args getEmptyArgsInstance() { - return new GetTables_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetTables_result getResult(I iface, GetTables_args args) throws org.apache.thrift.TException { - GetTables_result result = new GetTables_result(); - result.success = iface.GetTables(args.req); - return result; - } - } - - public static class GetTableTypes extends org.apache.thrift.ProcessFunction { - public GetTableTypes() { - super("GetTableTypes"); - } - - public GetTableTypes_args getEmptyArgsInstance() { - return new GetTableTypes_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetTableTypes_result getResult(I iface, GetTableTypes_args args) throws org.apache.thrift.TException { - GetTableTypes_result result = new GetTableTypes_result(); - result.success = iface.GetTableTypes(args.req); - return result; - } - } - - public static class GetColumns extends org.apache.thrift.ProcessFunction { - public GetColumns() { - super("GetColumns"); - } - - public GetColumns_args getEmptyArgsInstance() { - return new GetColumns_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetColumns_result getResult(I iface, GetColumns_args args) throws org.apache.thrift.TException { - GetColumns_result result = new GetColumns_result(); - result.success = iface.GetColumns(args.req); - return result; - } - } - - public static class GetFunctions extends org.apache.thrift.ProcessFunction { - public GetFunctions() { - super("GetFunctions"); - } - - public GetFunctions_args getEmptyArgsInstance() { - return new GetFunctions_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetFunctions_result getResult(I iface, GetFunctions_args args) throws org.apache.thrift.TException { - GetFunctions_result result = new GetFunctions_result(); - result.success = iface.GetFunctions(args.req); - return result; - } - } - - public static class GetOperationStatus extends org.apache.thrift.ProcessFunction { - public GetOperationStatus() { - super("GetOperationStatus"); - } - - public GetOperationStatus_args getEmptyArgsInstance() { - return new GetOperationStatus_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetOperationStatus_result getResult(I iface, GetOperationStatus_args args) throws org.apache.thrift.TException { - GetOperationStatus_result result = new GetOperationStatus_result(); - result.success = iface.GetOperationStatus(args.req); - return result; - } - } - - public static class CancelOperation extends org.apache.thrift.ProcessFunction { - public CancelOperation() { - super("CancelOperation"); - } - - public CancelOperation_args getEmptyArgsInstance() { - return new CancelOperation_args(); - } - - protected boolean isOneway() { - return false; - } - - public CancelOperation_result getResult(I iface, CancelOperation_args args) throws org.apache.thrift.TException { - CancelOperation_result result = new CancelOperation_result(); - result.success = iface.CancelOperation(args.req); - return result; - } - } - - public static class CloseOperation extends org.apache.thrift.ProcessFunction { - public CloseOperation() { - super("CloseOperation"); - } - - public CloseOperation_args getEmptyArgsInstance() { - return new CloseOperation_args(); - } - - protected boolean isOneway() { - return false; - } - - public CloseOperation_result getResult(I iface, CloseOperation_args args) throws org.apache.thrift.TException { - CloseOperation_result result = new CloseOperation_result(); - result.success = iface.CloseOperation(args.req); - return result; - } - } - - public static class GetResultSetMetadata extends org.apache.thrift.ProcessFunction { - public GetResultSetMetadata() { - super("GetResultSetMetadata"); - } - - public GetResultSetMetadata_args getEmptyArgsInstance() { - return new GetResultSetMetadata_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetResultSetMetadata_result getResult(I iface, GetResultSetMetadata_args args) throws org.apache.thrift.TException { - GetResultSetMetadata_result result = new GetResultSetMetadata_result(); - result.success = iface.GetResultSetMetadata(args.req); - return result; - } - } - - public static class FetchResults extends org.apache.thrift.ProcessFunction { - public FetchResults() { - super("FetchResults"); - } - - public FetchResults_args getEmptyArgsInstance() { - return new FetchResults_args(); - } - - protected boolean isOneway() { - return false; - } - - public FetchResults_result getResult(I iface, FetchResults_args args) throws org.apache.thrift.TException { - FetchResults_result result = new FetchResults_result(); - result.success = iface.FetchResults(args.req); - return result; - } - } - - public static class GetDelegationToken extends org.apache.thrift.ProcessFunction { - public GetDelegationToken() { - super("GetDelegationToken"); - } - - public GetDelegationToken_args getEmptyArgsInstance() { - return new GetDelegationToken_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetDelegationToken_result getResult(I iface, GetDelegationToken_args args) throws org.apache.thrift.TException { - GetDelegationToken_result result = new GetDelegationToken_result(); - result.success = iface.GetDelegationToken(args.req); - return result; - } - } - - public static class CancelDelegationToken extends org.apache.thrift.ProcessFunction { - public CancelDelegationToken() { - super("CancelDelegationToken"); - } - - public CancelDelegationToken_args getEmptyArgsInstance() { - return new CancelDelegationToken_args(); - } - - protected boolean isOneway() { - return false; - } - - public CancelDelegationToken_result getResult(I iface, CancelDelegationToken_args args) throws org.apache.thrift.TException { - CancelDelegationToken_result result = new CancelDelegationToken_result(); - result.success = iface.CancelDelegationToken(args.req); - return result; - } - } - - public static class RenewDelegationToken extends org.apache.thrift.ProcessFunction { - public RenewDelegationToken() { - super("RenewDelegationToken"); - } - - public RenewDelegationToken_args getEmptyArgsInstance() { - return new RenewDelegationToken_args(); - } - - protected boolean isOneway() { - return false; - } - - public RenewDelegationToken_result getResult(I iface, RenewDelegationToken_args args) throws org.apache.thrift.TException { - RenewDelegationToken_result result = new RenewDelegationToken_result(); - result.success = iface.RenewDelegationToken(args.req); - return result; - } - } - - } - - public static class OpenSession_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OpenSession_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new OpenSession_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new OpenSession_argsTupleSchemeFactory()); - } - - private TOpenSessionReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOpenSessionReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(OpenSession_args.class, metaDataMap); - } - - public OpenSession_args() { - } - - public OpenSession_args( - TOpenSessionReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public OpenSession_args(OpenSession_args other) { - if (other.isSetReq()) { - this.req = new TOpenSessionReq(other.req); - } - } - - public OpenSession_args deepCopy() { - return new OpenSession_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TOpenSessionReq getReq() { - return this.req; - } - - public void setReq(TOpenSessionReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TOpenSessionReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof OpenSession_args) - return this.equals((OpenSession_args)that); - return false; - } - - public boolean equals(OpenSession_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(OpenSession_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - OpenSession_args typedOther = (OpenSession_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("OpenSession_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class OpenSession_argsStandardSchemeFactory implements SchemeFactory { - public OpenSession_argsStandardScheme getScheme() { - return new OpenSession_argsStandardScheme(); - } - } - - private static class OpenSession_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, OpenSession_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TOpenSessionReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, OpenSession_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class OpenSession_argsTupleSchemeFactory implements SchemeFactory { - public OpenSession_argsTupleScheme getScheme() { - return new OpenSession_argsTupleScheme(); - } - } - - private static class OpenSession_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, OpenSession_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, OpenSession_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TOpenSessionReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class OpenSession_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OpenSession_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new OpenSession_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new OpenSession_resultTupleSchemeFactory()); - } - - private TOpenSessionResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOpenSessionResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(OpenSession_result.class, metaDataMap); - } - - public OpenSession_result() { - } - - public OpenSession_result( - TOpenSessionResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public OpenSession_result(OpenSession_result other) { - if (other.isSetSuccess()) { - this.success = new TOpenSessionResp(other.success); - } - } - - public OpenSession_result deepCopy() { - return new OpenSession_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TOpenSessionResp getSuccess() { - return this.success; - } - - public void setSuccess(TOpenSessionResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TOpenSessionResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof OpenSession_result) - return this.equals((OpenSession_result)that); - return false; - } - - public boolean equals(OpenSession_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(OpenSession_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - OpenSession_result typedOther = (OpenSession_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("OpenSession_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class OpenSession_resultStandardSchemeFactory implements SchemeFactory { - public OpenSession_resultStandardScheme getScheme() { - return new OpenSession_resultStandardScheme(); - } - } - - private static class OpenSession_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, OpenSession_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TOpenSessionResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, OpenSession_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class OpenSession_resultTupleSchemeFactory implements SchemeFactory { - public OpenSession_resultTupleScheme getScheme() { - return new OpenSession_resultTupleScheme(); - } - } - - private static class OpenSession_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, OpenSession_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, OpenSession_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TOpenSessionResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class CloseSession_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CloseSession_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new CloseSession_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new CloseSession_argsTupleSchemeFactory()); - } - - private TCloseSessionReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCloseSessionReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CloseSession_args.class, metaDataMap); - } - - public CloseSession_args() { - } - - public CloseSession_args( - TCloseSessionReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public CloseSession_args(CloseSession_args other) { - if (other.isSetReq()) { - this.req = new TCloseSessionReq(other.req); - } - } - - public CloseSession_args deepCopy() { - return new CloseSession_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TCloseSessionReq getReq() { - return this.req; - } - - public void setReq(TCloseSessionReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TCloseSessionReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof CloseSession_args) - return this.equals((CloseSession_args)that); - return false; - } - - public boolean equals(CloseSession_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(CloseSession_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - CloseSession_args typedOther = (CloseSession_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("CloseSession_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class CloseSession_argsStandardSchemeFactory implements SchemeFactory { - public CloseSession_argsStandardScheme getScheme() { - return new CloseSession_argsStandardScheme(); - } - } - - private static class CloseSession_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, CloseSession_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TCloseSessionReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, CloseSession_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class CloseSession_argsTupleSchemeFactory implements SchemeFactory { - public CloseSession_argsTupleScheme getScheme() { - return new CloseSession_argsTupleScheme(); - } - } - - private static class CloseSession_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, CloseSession_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, CloseSession_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TCloseSessionReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class CloseSession_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CloseSession_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new CloseSession_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new CloseSession_resultTupleSchemeFactory()); - } - - private TCloseSessionResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCloseSessionResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CloseSession_result.class, metaDataMap); - } - - public CloseSession_result() { - } - - public CloseSession_result( - TCloseSessionResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public CloseSession_result(CloseSession_result other) { - if (other.isSetSuccess()) { - this.success = new TCloseSessionResp(other.success); - } - } - - public CloseSession_result deepCopy() { - return new CloseSession_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TCloseSessionResp getSuccess() { - return this.success; - } - - public void setSuccess(TCloseSessionResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TCloseSessionResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof CloseSession_result) - return this.equals((CloseSession_result)that); - return false; - } - - public boolean equals(CloseSession_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(CloseSession_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - CloseSession_result typedOther = (CloseSession_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("CloseSession_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class CloseSession_resultStandardSchemeFactory implements SchemeFactory { - public CloseSession_resultStandardScheme getScheme() { - return new CloseSession_resultStandardScheme(); - } - } - - private static class CloseSession_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, CloseSession_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TCloseSessionResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, CloseSession_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class CloseSession_resultTupleSchemeFactory implements SchemeFactory { - public CloseSession_resultTupleScheme getScheme() { - return new CloseSession_resultTupleScheme(); - } - } - - private static class CloseSession_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, CloseSession_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, CloseSession_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TCloseSessionResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetInfo_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetInfo_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetInfo_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetInfo_argsTupleSchemeFactory()); - } - - private TGetInfoReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetInfoReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetInfo_args.class, metaDataMap); - } - - public GetInfo_args() { - } - - public GetInfo_args( - TGetInfoReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetInfo_args(GetInfo_args other) { - if (other.isSetReq()) { - this.req = new TGetInfoReq(other.req); - } - } - - public GetInfo_args deepCopy() { - return new GetInfo_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetInfoReq getReq() { - return this.req; - } - - public void setReq(TGetInfoReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetInfoReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetInfo_args) - return this.equals((GetInfo_args)that); - return false; - } - - public boolean equals(GetInfo_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetInfo_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetInfo_args typedOther = (GetInfo_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetInfo_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetInfo_argsStandardSchemeFactory implements SchemeFactory { - public GetInfo_argsStandardScheme getScheme() { - return new GetInfo_argsStandardScheme(); - } - } - - private static class GetInfo_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetInfo_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetInfoReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetInfo_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetInfo_argsTupleSchemeFactory implements SchemeFactory { - public GetInfo_argsTupleScheme getScheme() { - return new GetInfo_argsTupleScheme(); - } - } - - private static class GetInfo_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetInfo_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetInfo_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetInfoReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetInfo_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetInfo_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetInfo_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetInfo_resultTupleSchemeFactory()); - } - - private TGetInfoResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetInfoResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetInfo_result.class, metaDataMap); - } - - public GetInfo_result() { - } - - public GetInfo_result( - TGetInfoResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetInfo_result(GetInfo_result other) { - if (other.isSetSuccess()) { - this.success = new TGetInfoResp(other.success); - } - } - - public GetInfo_result deepCopy() { - return new GetInfo_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetInfoResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetInfoResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetInfoResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetInfo_result) - return this.equals((GetInfo_result)that); - return false; - } - - public boolean equals(GetInfo_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetInfo_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetInfo_result typedOther = (GetInfo_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetInfo_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetInfo_resultStandardSchemeFactory implements SchemeFactory { - public GetInfo_resultStandardScheme getScheme() { - return new GetInfo_resultStandardScheme(); - } - } - - private static class GetInfo_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetInfo_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetInfoResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetInfo_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetInfo_resultTupleSchemeFactory implements SchemeFactory { - public GetInfo_resultTupleScheme getScheme() { - return new GetInfo_resultTupleScheme(); - } - } - - private static class GetInfo_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetInfo_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetInfo_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetInfoResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class ExecuteStatement_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecuteStatement_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new ExecuteStatement_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new ExecuteStatement_argsTupleSchemeFactory()); - } - - private TExecuteStatementReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TExecuteStatementReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecuteStatement_args.class, metaDataMap); - } - - public ExecuteStatement_args() { - } - - public ExecuteStatement_args( - TExecuteStatementReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public ExecuteStatement_args(ExecuteStatement_args other) { - if (other.isSetReq()) { - this.req = new TExecuteStatementReq(other.req); - } - } - - public ExecuteStatement_args deepCopy() { - return new ExecuteStatement_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TExecuteStatementReq getReq() { - return this.req; - } - - public void setReq(TExecuteStatementReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TExecuteStatementReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof ExecuteStatement_args) - return this.equals((ExecuteStatement_args)that); - return false; - } - - public boolean equals(ExecuteStatement_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(ExecuteStatement_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - ExecuteStatement_args typedOther = (ExecuteStatement_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("ExecuteStatement_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class ExecuteStatement_argsStandardSchemeFactory implements SchemeFactory { - public ExecuteStatement_argsStandardScheme getScheme() { - return new ExecuteStatement_argsStandardScheme(); - } - } - - private static class ExecuteStatement_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, ExecuteStatement_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TExecuteStatementReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, ExecuteStatement_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class ExecuteStatement_argsTupleSchemeFactory implements SchemeFactory { - public ExecuteStatement_argsTupleScheme getScheme() { - return new ExecuteStatement_argsTupleScheme(); - } - } - - private static class ExecuteStatement_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, ExecuteStatement_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, ExecuteStatement_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TExecuteStatementReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class ExecuteStatement_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecuteStatement_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new ExecuteStatement_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new ExecuteStatement_resultTupleSchemeFactory()); - } - - private TExecuteStatementResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TExecuteStatementResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecuteStatement_result.class, metaDataMap); - } - - public ExecuteStatement_result() { - } - - public ExecuteStatement_result( - TExecuteStatementResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public ExecuteStatement_result(ExecuteStatement_result other) { - if (other.isSetSuccess()) { - this.success = new TExecuteStatementResp(other.success); - } - } - - public ExecuteStatement_result deepCopy() { - return new ExecuteStatement_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TExecuteStatementResp getSuccess() { - return this.success; - } - - public void setSuccess(TExecuteStatementResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TExecuteStatementResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof ExecuteStatement_result) - return this.equals((ExecuteStatement_result)that); - return false; - } - - public boolean equals(ExecuteStatement_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(ExecuteStatement_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - ExecuteStatement_result typedOther = (ExecuteStatement_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("ExecuteStatement_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class ExecuteStatement_resultStandardSchemeFactory implements SchemeFactory { - public ExecuteStatement_resultStandardScheme getScheme() { - return new ExecuteStatement_resultStandardScheme(); - } - } - - private static class ExecuteStatement_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, ExecuteStatement_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TExecuteStatementResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, ExecuteStatement_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class ExecuteStatement_resultTupleSchemeFactory implements SchemeFactory { - public ExecuteStatement_resultTupleScheme getScheme() { - return new ExecuteStatement_resultTupleScheme(); - } - } - - private static class ExecuteStatement_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, ExecuteStatement_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, ExecuteStatement_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TExecuteStatementResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetTypeInfo_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTypeInfo_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetTypeInfo_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetTypeInfo_argsTupleSchemeFactory()); - } - - private TGetTypeInfoReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTypeInfoReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTypeInfo_args.class, metaDataMap); - } - - public GetTypeInfo_args() { - } - - public GetTypeInfo_args( - TGetTypeInfoReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetTypeInfo_args(GetTypeInfo_args other) { - if (other.isSetReq()) { - this.req = new TGetTypeInfoReq(other.req); - } - } - - public GetTypeInfo_args deepCopy() { - return new GetTypeInfo_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetTypeInfoReq getReq() { - return this.req; - } - - public void setReq(TGetTypeInfoReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetTypeInfoReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetTypeInfo_args) - return this.equals((GetTypeInfo_args)that); - return false; - } - - public boolean equals(GetTypeInfo_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetTypeInfo_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetTypeInfo_args typedOther = (GetTypeInfo_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetTypeInfo_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetTypeInfo_argsStandardSchemeFactory implements SchemeFactory { - public GetTypeInfo_argsStandardScheme getScheme() { - return new GetTypeInfo_argsStandardScheme(); - } - } - - private static class GetTypeInfo_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetTypeInfo_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetTypeInfoReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetTypeInfo_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetTypeInfo_argsTupleSchemeFactory implements SchemeFactory { - public GetTypeInfo_argsTupleScheme getScheme() { - return new GetTypeInfo_argsTupleScheme(); - } - } - - private static class GetTypeInfo_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetTypeInfo_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetTypeInfo_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetTypeInfoReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetTypeInfo_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTypeInfo_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetTypeInfo_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetTypeInfo_resultTupleSchemeFactory()); - } - - private TGetTypeInfoResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTypeInfoResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTypeInfo_result.class, metaDataMap); - } - - public GetTypeInfo_result() { - } - - public GetTypeInfo_result( - TGetTypeInfoResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetTypeInfo_result(GetTypeInfo_result other) { - if (other.isSetSuccess()) { - this.success = new TGetTypeInfoResp(other.success); - } - } - - public GetTypeInfo_result deepCopy() { - return new GetTypeInfo_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetTypeInfoResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetTypeInfoResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetTypeInfoResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetTypeInfo_result) - return this.equals((GetTypeInfo_result)that); - return false; - } - - public boolean equals(GetTypeInfo_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetTypeInfo_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetTypeInfo_result typedOther = (GetTypeInfo_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetTypeInfo_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetTypeInfo_resultStandardSchemeFactory implements SchemeFactory { - public GetTypeInfo_resultStandardScheme getScheme() { - return new GetTypeInfo_resultStandardScheme(); - } - } - - private static class GetTypeInfo_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetTypeInfo_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetTypeInfoResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetTypeInfo_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetTypeInfo_resultTupleSchemeFactory implements SchemeFactory { - public GetTypeInfo_resultTupleScheme getScheme() { - return new GetTypeInfo_resultTupleScheme(); - } - } - - private static class GetTypeInfo_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetTypeInfo_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetTypeInfo_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetTypeInfoResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetCatalogs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetCatalogs_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetCatalogs_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetCatalogs_argsTupleSchemeFactory()); - } - - private TGetCatalogsReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetCatalogsReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetCatalogs_args.class, metaDataMap); - } - - public GetCatalogs_args() { - } - - public GetCatalogs_args( - TGetCatalogsReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetCatalogs_args(GetCatalogs_args other) { - if (other.isSetReq()) { - this.req = new TGetCatalogsReq(other.req); - } - } - - public GetCatalogs_args deepCopy() { - return new GetCatalogs_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetCatalogsReq getReq() { - return this.req; - } - - public void setReq(TGetCatalogsReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetCatalogsReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetCatalogs_args) - return this.equals((GetCatalogs_args)that); - return false; - } - - public boolean equals(GetCatalogs_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetCatalogs_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetCatalogs_args typedOther = (GetCatalogs_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetCatalogs_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetCatalogs_argsStandardSchemeFactory implements SchemeFactory { - public GetCatalogs_argsStandardScheme getScheme() { - return new GetCatalogs_argsStandardScheme(); - } - } - - private static class GetCatalogs_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetCatalogs_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetCatalogsReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetCatalogs_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetCatalogs_argsTupleSchemeFactory implements SchemeFactory { - public GetCatalogs_argsTupleScheme getScheme() { - return new GetCatalogs_argsTupleScheme(); - } - } - - private static class GetCatalogs_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetCatalogs_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetCatalogs_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetCatalogsReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetCatalogs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetCatalogs_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetCatalogs_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetCatalogs_resultTupleSchemeFactory()); - } - - private TGetCatalogsResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetCatalogsResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetCatalogs_result.class, metaDataMap); - } - - public GetCatalogs_result() { - } - - public GetCatalogs_result( - TGetCatalogsResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetCatalogs_result(GetCatalogs_result other) { - if (other.isSetSuccess()) { - this.success = new TGetCatalogsResp(other.success); - } - } - - public GetCatalogs_result deepCopy() { - return new GetCatalogs_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetCatalogsResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetCatalogsResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetCatalogsResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetCatalogs_result) - return this.equals((GetCatalogs_result)that); - return false; - } - - public boolean equals(GetCatalogs_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetCatalogs_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetCatalogs_result typedOther = (GetCatalogs_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetCatalogs_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetCatalogs_resultStandardSchemeFactory implements SchemeFactory { - public GetCatalogs_resultStandardScheme getScheme() { - return new GetCatalogs_resultStandardScheme(); - } - } - - private static class GetCatalogs_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetCatalogs_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetCatalogsResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetCatalogs_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetCatalogs_resultTupleSchemeFactory implements SchemeFactory { - public GetCatalogs_resultTupleScheme getScheme() { - return new GetCatalogs_resultTupleScheme(); - } - } - - private static class GetCatalogs_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetCatalogs_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetCatalogs_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetCatalogsResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetSchemas_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetSchemas_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetSchemas_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetSchemas_argsTupleSchemeFactory()); - } - - private TGetSchemasReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetSchemasReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetSchemas_args.class, metaDataMap); - } - - public GetSchemas_args() { - } - - public GetSchemas_args( - TGetSchemasReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetSchemas_args(GetSchemas_args other) { - if (other.isSetReq()) { - this.req = new TGetSchemasReq(other.req); - } - } - - public GetSchemas_args deepCopy() { - return new GetSchemas_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetSchemasReq getReq() { - return this.req; - } - - public void setReq(TGetSchemasReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetSchemasReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetSchemas_args) - return this.equals((GetSchemas_args)that); - return false; - } - - public boolean equals(GetSchemas_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetSchemas_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetSchemas_args typedOther = (GetSchemas_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetSchemas_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetSchemas_argsStandardSchemeFactory implements SchemeFactory { - public GetSchemas_argsStandardScheme getScheme() { - return new GetSchemas_argsStandardScheme(); - } - } - - private static class GetSchemas_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetSchemas_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetSchemasReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetSchemas_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetSchemas_argsTupleSchemeFactory implements SchemeFactory { - public GetSchemas_argsTupleScheme getScheme() { - return new GetSchemas_argsTupleScheme(); - } - } - - private static class GetSchemas_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetSchemas_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetSchemas_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetSchemasReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetSchemas_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetSchemas_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetSchemas_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetSchemas_resultTupleSchemeFactory()); - } - - private TGetSchemasResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetSchemasResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetSchemas_result.class, metaDataMap); - } - - public GetSchemas_result() { - } - - public GetSchemas_result( - TGetSchemasResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetSchemas_result(GetSchemas_result other) { - if (other.isSetSuccess()) { - this.success = new TGetSchemasResp(other.success); - } - } - - public GetSchemas_result deepCopy() { - return new GetSchemas_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetSchemasResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetSchemasResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetSchemasResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetSchemas_result) - return this.equals((GetSchemas_result)that); - return false; - } - - public boolean equals(GetSchemas_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetSchemas_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetSchemas_result typedOther = (GetSchemas_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetSchemas_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetSchemas_resultStandardSchemeFactory implements SchemeFactory { - public GetSchemas_resultStandardScheme getScheme() { - return new GetSchemas_resultStandardScheme(); - } - } - - private static class GetSchemas_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetSchemas_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetSchemasResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetSchemas_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetSchemas_resultTupleSchemeFactory implements SchemeFactory { - public GetSchemas_resultTupleScheme getScheme() { - return new GetSchemas_resultTupleScheme(); - } - } - - private static class GetSchemas_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetSchemas_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetSchemas_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetSchemasResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetTables_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTables_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetTables_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetTables_argsTupleSchemeFactory()); - } - - private TGetTablesReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTablesReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTables_args.class, metaDataMap); - } - - public GetTables_args() { - } - - public GetTables_args( - TGetTablesReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetTables_args(GetTables_args other) { - if (other.isSetReq()) { - this.req = new TGetTablesReq(other.req); - } - } - - public GetTables_args deepCopy() { - return new GetTables_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetTablesReq getReq() { - return this.req; - } - - public void setReq(TGetTablesReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetTablesReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetTables_args) - return this.equals((GetTables_args)that); - return false; - } - - public boolean equals(GetTables_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetTables_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetTables_args typedOther = (GetTables_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetTables_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetTables_argsStandardSchemeFactory implements SchemeFactory { - public GetTables_argsStandardScheme getScheme() { - return new GetTables_argsStandardScheme(); - } - } - - private static class GetTables_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetTables_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetTablesReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetTables_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetTables_argsTupleSchemeFactory implements SchemeFactory { - public GetTables_argsTupleScheme getScheme() { - return new GetTables_argsTupleScheme(); - } - } - - private static class GetTables_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetTables_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetTables_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetTablesReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetTables_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTables_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetTables_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetTables_resultTupleSchemeFactory()); - } - - private TGetTablesResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTablesResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTables_result.class, metaDataMap); - } - - public GetTables_result() { - } - - public GetTables_result( - TGetTablesResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetTables_result(GetTables_result other) { - if (other.isSetSuccess()) { - this.success = new TGetTablesResp(other.success); - } - } - - public GetTables_result deepCopy() { - return new GetTables_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetTablesResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetTablesResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetTablesResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetTables_result) - return this.equals((GetTables_result)that); - return false; - } - - public boolean equals(GetTables_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetTables_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetTables_result typedOther = (GetTables_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetTables_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetTables_resultStandardSchemeFactory implements SchemeFactory { - public GetTables_resultStandardScheme getScheme() { - return new GetTables_resultStandardScheme(); - } - } - - private static class GetTables_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetTables_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetTablesResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetTables_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetTables_resultTupleSchemeFactory implements SchemeFactory { - public GetTables_resultTupleScheme getScheme() { - return new GetTables_resultTupleScheme(); - } - } - - private static class GetTables_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetTables_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetTables_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetTablesResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetTableTypes_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTableTypes_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetTableTypes_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetTableTypes_argsTupleSchemeFactory()); - } - - private TGetTableTypesReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTableTypesReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTableTypes_args.class, metaDataMap); - } - - public GetTableTypes_args() { - } - - public GetTableTypes_args( - TGetTableTypesReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetTableTypes_args(GetTableTypes_args other) { - if (other.isSetReq()) { - this.req = new TGetTableTypesReq(other.req); - } - } - - public GetTableTypes_args deepCopy() { - return new GetTableTypes_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetTableTypesReq getReq() { - return this.req; - } - - public void setReq(TGetTableTypesReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetTableTypesReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetTableTypes_args) - return this.equals((GetTableTypes_args)that); - return false; - } - - public boolean equals(GetTableTypes_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetTableTypes_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetTableTypes_args typedOther = (GetTableTypes_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetTableTypes_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetTableTypes_argsStandardSchemeFactory implements SchemeFactory { - public GetTableTypes_argsStandardScheme getScheme() { - return new GetTableTypes_argsStandardScheme(); - } - } - - private static class GetTableTypes_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetTableTypes_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetTableTypesReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetTableTypes_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetTableTypes_argsTupleSchemeFactory implements SchemeFactory { - public GetTableTypes_argsTupleScheme getScheme() { - return new GetTableTypes_argsTupleScheme(); - } - } - - private static class GetTableTypes_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetTableTypes_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetTableTypes_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetTableTypesReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetTableTypes_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTableTypes_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetTableTypes_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetTableTypes_resultTupleSchemeFactory()); - } - - private TGetTableTypesResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTableTypesResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTableTypes_result.class, metaDataMap); - } - - public GetTableTypes_result() { - } - - public GetTableTypes_result( - TGetTableTypesResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetTableTypes_result(GetTableTypes_result other) { - if (other.isSetSuccess()) { - this.success = new TGetTableTypesResp(other.success); - } - } - - public GetTableTypes_result deepCopy() { - return new GetTableTypes_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetTableTypesResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetTableTypesResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetTableTypesResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetTableTypes_result) - return this.equals((GetTableTypes_result)that); - return false; - } - - public boolean equals(GetTableTypes_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetTableTypes_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetTableTypes_result typedOther = (GetTableTypes_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetTableTypes_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetTableTypes_resultStandardSchemeFactory implements SchemeFactory { - public GetTableTypes_resultStandardScheme getScheme() { - return new GetTableTypes_resultStandardScheme(); - } - } - - private static class GetTableTypes_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetTableTypes_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetTableTypesResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetTableTypes_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetTableTypes_resultTupleSchemeFactory implements SchemeFactory { - public GetTableTypes_resultTupleScheme getScheme() { - return new GetTableTypes_resultTupleScheme(); - } - } - - private static class GetTableTypes_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetTableTypes_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetTableTypes_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetTableTypesResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetColumns_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetColumns_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetColumns_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetColumns_argsTupleSchemeFactory()); - } - - private TGetColumnsReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetColumnsReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetColumns_args.class, metaDataMap); - } - - public GetColumns_args() { - } - - public GetColumns_args( - TGetColumnsReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetColumns_args(GetColumns_args other) { - if (other.isSetReq()) { - this.req = new TGetColumnsReq(other.req); - } - } - - public GetColumns_args deepCopy() { - return new GetColumns_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetColumnsReq getReq() { - return this.req; - } - - public void setReq(TGetColumnsReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetColumnsReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetColumns_args) - return this.equals((GetColumns_args)that); - return false; - } - - public boolean equals(GetColumns_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetColumns_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetColumns_args typedOther = (GetColumns_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetColumns_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetColumns_argsStandardSchemeFactory implements SchemeFactory { - public GetColumns_argsStandardScheme getScheme() { - return new GetColumns_argsStandardScheme(); - } - } - - private static class GetColumns_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetColumns_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetColumnsReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetColumns_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetColumns_argsTupleSchemeFactory implements SchemeFactory { - public GetColumns_argsTupleScheme getScheme() { - return new GetColumns_argsTupleScheme(); - } - } - - private static class GetColumns_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetColumns_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetColumns_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetColumnsReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetColumns_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetColumns_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetColumns_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetColumns_resultTupleSchemeFactory()); - } - - private TGetColumnsResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetColumnsResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetColumns_result.class, metaDataMap); - } - - public GetColumns_result() { - } - - public GetColumns_result( - TGetColumnsResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetColumns_result(GetColumns_result other) { - if (other.isSetSuccess()) { - this.success = new TGetColumnsResp(other.success); - } - } - - public GetColumns_result deepCopy() { - return new GetColumns_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetColumnsResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetColumnsResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetColumnsResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetColumns_result) - return this.equals((GetColumns_result)that); - return false; - } - - public boolean equals(GetColumns_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetColumns_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetColumns_result typedOther = (GetColumns_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetColumns_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetColumns_resultStandardSchemeFactory implements SchemeFactory { - public GetColumns_resultStandardScheme getScheme() { - return new GetColumns_resultStandardScheme(); - } - } - - private static class GetColumns_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetColumns_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetColumnsResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetColumns_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetColumns_resultTupleSchemeFactory implements SchemeFactory { - public GetColumns_resultTupleScheme getScheme() { - return new GetColumns_resultTupleScheme(); - } - } - - private static class GetColumns_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetColumns_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetColumns_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetColumnsResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetFunctions_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetFunctions_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetFunctions_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetFunctions_argsTupleSchemeFactory()); - } - - private TGetFunctionsReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetFunctionsReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetFunctions_args.class, metaDataMap); - } - - public GetFunctions_args() { - } - - public GetFunctions_args( - TGetFunctionsReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetFunctions_args(GetFunctions_args other) { - if (other.isSetReq()) { - this.req = new TGetFunctionsReq(other.req); - } - } - - public GetFunctions_args deepCopy() { - return new GetFunctions_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetFunctionsReq getReq() { - return this.req; - } - - public void setReq(TGetFunctionsReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetFunctionsReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetFunctions_args) - return this.equals((GetFunctions_args)that); - return false; - } - - public boolean equals(GetFunctions_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetFunctions_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetFunctions_args typedOther = (GetFunctions_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetFunctions_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetFunctions_argsStandardSchemeFactory implements SchemeFactory { - public GetFunctions_argsStandardScheme getScheme() { - return new GetFunctions_argsStandardScheme(); - } - } - - private static class GetFunctions_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetFunctions_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetFunctionsReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetFunctions_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetFunctions_argsTupleSchemeFactory implements SchemeFactory { - public GetFunctions_argsTupleScheme getScheme() { - return new GetFunctions_argsTupleScheme(); - } - } - - private static class GetFunctions_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetFunctions_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetFunctions_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetFunctionsReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetFunctions_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetFunctions_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetFunctions_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetFunctions_resultTupleSchemeFactory()); - } - - private TGetFunctionsResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetFunctionsResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetFunctions_result.class, metaDataMap); - } - - public GetFunctions_result() { - } - - public GetFunctions_result( - TGetFunctionsResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetFunctions_result(GetFunctions_result other) { - if (other.isSetSuccess()) { - this.success = new TGetFunctionsResp(other.success); - } - } - - public GetFunctions_result deepCopy() { - return new GetFunctions_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetFunctionsResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetFunctionsResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetFunctionsResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetFunctions_result) - return this.equals((GetFunctions_result)that); - return false; - } - - public boolean equals(GetFunctions_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetFunctions_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetFunctions_result typedOther = (GetFunctions_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetFunctions_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetFunctions_resultStandardSchemeFactory implements SchemeFactory { - public GetFunctions_resultStandardScheme getScheme() { - return new GetFunctions_resultStandardScheme(); - } - } - - private static class GetFunctions_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetFunctions_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetFunctionsResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetFunctions_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetFunctions_resultTupleSchemeFactory implements SchemeFactory { - public GetFunctions_resultTupleScheme getScheme() { - return new GetFunctions_resultTupleScheme(); - } - } - - private static class GetFunctions_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetFunctions_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetFunctions_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetFunctionsResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetOperationStatus_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetOperationStatus_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetOperationStatus_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetOperationStatus_argsTupleSchemeFactory()); - } - - private TGetOperationStatusReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetOperationStatusReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetOperationStatus_args.class, metaDataMap); - } - - public GetOperationStatus_args() { - } - - public GetOperationStatus_args( - TGetOperationStatusReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetOperationStatus_args(GetOperationStatus_args other) { - if (other.isSetReq()) { - this.req = new TGetOperationStatusReq(other.req); - } - } - - public GetOperationStatus_args deepCopy() { - return new GetOperationStatus_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetOperationStatusReq getReq() { - return this.req; - } - - public void setReq(TGetOperationStatusReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetOperationStatusReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetOperationStatus_args) - return this.equals((GetOperationStatus_args)that); - return false; - } - - public boolean equals(GetOperationStatus_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetOperationStatus_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetOperationStatus_args typedOther = (GetOperationStatus_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetOperationStatus_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetOperationStatus_argsStandardSchemeFactory implements SchemeFactory { - public GetOperationStatus_argsStandardScheme getScheme() { - return new GetOperationStatus_argsStandardScheme(); - } - } - - private static class GetOperationStatus_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetOperationStatus_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetOperationStatusReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetOperationStatus_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetOperationStatus_argsTupleSchemeFactory implements SchemeFactory { - public GetOperationStatus_argsTupleScheme getScheme() { - return new GetOperationStatus_argsTupleScheme(); - } - } - - private static class GetOperationStatus_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetOperationStatus_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetOperationStatus_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetOperationStatusReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetOperationStatus_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetOperationStatus_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetOperationStatus_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetOperationStatus_resultTupleSchemeFactory()); - } - - private TGetOperationStatusResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetOperationStatusResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetOperationStatus_result.class, metaDataMap); - } - - public GetOperationStatus_result() { - } - - public GetOperationStatus_result( - TGetOperationStatusResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetOperationStatus_result(GetOperationStatus_result other) { - if (other.isSetSuccess()) { - this.success = new TGetOperationStatusResp(other.success); - } - } - - public GetOperationStatus_result deepCopy() { - return new GetOperationStatus_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetOperationStatusResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetOperationStatusResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetOperationStatusResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetOperationStatus_result) - return this.equals((GetOperationStatus_result)that); - return false; - } - - public boolean equals(GetOperationStatus_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetOperationStatus_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetOperationStatus_result typedOther = (GetOperationStatus_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetOperationStatus_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetOperationStatus_resultStandardSchemeFactory implements SchemeFactory { - public GetOperationStatus_resultStandardScheme getScheme() { - return new GetOperationStatus_resultStandardScheme(); - } - } - - private static class GetOperationStatus_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetOperationStatus_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetOperationStatusResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetOperationStatus_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetOperationStatus_resultTupleSchemeFactory implements SchemeFactory { - public GetOperationStatus_resultTupleScheme getScheme() { - return new GetOperationStatus_resultTupleScheme(); - } - } - - private static class GetOperationStatus_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetOperationStatus_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetOperationStatus_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetOperationStatusResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class CancelOperation_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CancelOperation_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new CancelOperation_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new CancelOperation_argsTupleSchemeFactory()); - } - - private TCancelOperationReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCancelOperationReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CancelOperation_args.class, metaDataMap); - } - - public CancelOperation_args() { - } - - public CancelOperation_args( - TCancelOperationReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public CancelOperation_args(CancelOperation_args other) { - if (other.isSetReq()) { - this.req = new TCancelOperationReq(other.req); - } - } - - public CancelOperation_args deepCopy() { - return new CancelOperation_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TCancelOperationReq getReq() { - return this.req; - } - - public void setReq(TCancelOperationReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TCancelOperationReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof CancelOperation_args) - return this.equals((CancelOperation_args)that); - return false; - } - - public boolean equals(CancelOperation_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(CancelOperation_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - CancelOperation_args typedOther = (CancelOperation_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("CancelOperation_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class CancelOperation_argsStandardSchemeFactory implements SchemeFactory { - public CancelOperation_argsStandardScheme getScheme() { - return new CancelOperation_argsStandardScheme(); - } - } - - private static class CancelOperation_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, CancelOperation_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TCancelOperationReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, CancelOperation_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class CancelOperation_argsTupleSchemeFactory implements SchemeFactory { - public CancelOperation_argsTupleScheme getScheme() { - return new CancelOperation_argsTupleScheme(); - } - } - - private static class CancelOperation_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, CancelOperation_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, CancelOperation_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TCancelOperationReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class CancelOperation_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CancelOperation_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new CancelOperation_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new CancelOperation_resultTupleSchemeFactory()); - } - - private TCancelOperationResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCancelOperationResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CancelOperation_result.class, metaDataMap); - } - - public CancelOperation_result() { - } - - public CancelOperation_result( - TCancelOperationResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public CancelOperation_result(CancelOperation_result other) { - if (other.isSetSuccess()) { - this.success = new TCancelOperationResp(other.success); - } - } - - public CancelOperation_result deepCopy() { - return new CancelOperation_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TCancelOperationResp getSuccess() { - return this.success; - } - - public void setSuccess(TCancelOperationResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TCancelOperationResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof CancelOperation_result) - return this.equals((CancelOperation_result)that); - return false; - } - - public boolean equals(CancelOperation_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(CancelOperation_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - CancelOperation_result typedOther = (CancelOperation_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("CancelOperation_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class CancelOperation_resultStandardSchemeFactory implements SchemeFactory { - public CancelOperation_resultStandardScheme getScheme() { - return new CancelOperation_resultStandardScheme(); - } - } - - private static class CancelOperation_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, CancelOperation_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TCancelOperationResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, CancelOperation_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class CancelOperation_resultTupleSchemeFactory implements SchemeFactory { - public CancelOperation_resultTupleScheme getScheme() { - return new CancelOperation_resultTupleScheme(); - } - } - - private static class CancelOperation_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, CancelOperation_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, CancelOperation_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TCancelOperationResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class CloseOperation_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CloseOperation_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new CloseOperation_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new CloseOperation_argsTupleSchemeFactory()); - } - - private TCloseOperationReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCloseOperationReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CloseOperation_args.class, metaDataMap); - } - - public CloseOperation_args() { - } - - public CloseOperation_args( - TCloseOperationReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public CloseOperation_args(CloseOperation_args other) { - if (other.isSetReq()) { - this.req = new TCloseOperationReq(other.req); - } - } - - public CloseOperation_args deepCopy() { - return new CloseOperation_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TCloseOperationReq getReq() { - return this.req; - } - - public void setReq(TCloseOperationReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TCloseOperationReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof CloseOperation_args) - return this.equals((CloseOperation_args)that); - return false; - } - - public boolean equals(CloseOperation_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(CloseOperation_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - CloseOperation_args typedOther = (CloseOperation_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("CloseOperation_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class CloseOperation_argsStandardSchemeFactory implements SchemeFactory { - public CloseOperation_argsStandardScheme getScheme() { - return new CloseOperation_argsStandardScheme(); - } - } - - private static class CloseOperation_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, CloseOperation_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TCloseOperationReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, CloseOperation_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class CloseOperation_argsTupleSchemeFactory implements SchemeFactory { - public CloseOperation_argsTupleScheme getScheme() { - return new CloseOperation_argsTupleScheme(); - } - } - - private static class CloseOperation_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, CloseOperation_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, CloseOperation_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TCloseOperationReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class CloseOperation_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CloseOperation_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new CloseOperation_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new CloseOperation_resultTupleSchemeFactory()); - } - - private TCloseOperationResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCloseOperationResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CloseOperation_result.class, metaDataMap); - } - - public CloseOperation_result() { - } - - public CloseOperation_result( - TCloseOperationResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public CloseOperation_result(CloseOperation_result other) { - if (other.isSetSuccess()) { - this.success = new TCloseOperationResp(other.success); - } - } - - public CloseOperation_result deepCopy() { - return new CloseOperation_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TCloseOperationResp getSuccess() { - return this.success; - } - - public void setSuccess(TCloseOperationResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TCloseOperationResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof CloseOperation_result) - return this.equals((CloseOperation_result)that); - return false; - } - - public boolean equals(CloseOperation_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(CloseOperation_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - CloseOperation_result typedOther = (CloseOperation_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("CloseOperation_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class CloseOperation_resultStandardSchemeFactory implements SchemeFactory { - public CloseOperation_resultStandardScheme getScheme() { - return new CloseOperation_resultStandardScheme(); - } - } - - private static class CloseOperation_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, CloseOperation_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TCloseOperationResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, CloseOperation_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class CloseOperation_resultTupleSchemeFactory implements SchemeFactory { - public CloseOperation_resultTupleScheme getScheme() { - return new CloseOperation_resultTupleScheme(); - } - } - - private static class CloseOperation_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, CloseOperation_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, CloseOperation_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TCloseOperationResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetResultSetMetadata_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetResultSetMetadata_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetResultSetMetadata_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetResultSetMetadata_argsTupleSchemeFactory()); - } - - private TGetResultSetMetadataReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetResultSetMetadataReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetResultSetMetadata_args.class, metaDataMap); - } - - public GetResultSetMetadata_args() { - } - - public GetResultSetMetadata_args( - TGetResultSetMetadataReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetResultSetMetadata_args(GetResultSetMetadata_args other) { - if (other.isSetReq()) { - this.req = new TGetResultSetMetadataReq(other.req); - } - } - - public GetResultSetMetadata_args deepCopy() { - return new GetResultSetMetadata_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetResultSetMetadataReq getReq() { - return this.req; - } - - public void setReq(TGetResultSetMetadataReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetResultSetMetadataReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetResultSetMetadata_args) - return this.equals((GetResultSetMetadata_args)that); - return false; - } - - public boolean equals(GetResultSetMetadata_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetResultSetMetadata_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetResultSetMetadata_args typedOther = (GetResultSetMetadata_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetResultSetMetadata_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetResultSetMetadata_argsStandardSchemeFactory implements SchemeFactory { - public GetResultSetMetadata_argsStandardScheme getScheme() { - return new GetResultSetMetadata_argsStandardScheme(); - } - } - - private static class GetResultSetMetadata_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetResultSetMetadata_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetResultSetMetadataReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetResultSetMetadata_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetResultSetMetadata_argsTupleSchemeFactory implements SchemeFactory { - public GetResultSetMetadata_argsTupleScheme getScheme() { - return new GetResultSetMetadata_argsTupleScheme(); - } - } - - private static class GetResultSetMetadata_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetResultSetMetadata_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetResultSetMetadata_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetResultSetMetadataReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetResultSetMetadata_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetResultSetMetadata_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetResultSetMetadata_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetResultSetMetadata_resultTupleSchemeFactory()); - } - - private TGetResultSetMetadataResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetResultSetMetadataResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetResultSetMetadata_result.class, metaDataMap); - } - - public GetResultSetMetadata_result() { - } - - public GetResultSetMetadata_result( - TGetResultSetMetadataResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetResultSetMetadata_result(GetResultSetMetadata_result other) { - if (other.isSetSuccess()) { - this.success = new TGetResultSetMetadataResp(other.success); - } - } - - public GetResultSetMetadata_result deepCopy() { - return new GetResultSetMetadata_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetResultSetMetadataResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetResultSetMetadataResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetResultSetMetadataResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetResultSetMetadata_result) - return this.equals((GetResultSetMetadata_result)that); - return false; - } - - public boolean equals(GetResultSetMetadata_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetResultSetMetadata_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetResultSetMetadata_result typedOther = (GetResultSetMetadata_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetResultSetMetadata_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetResultSetMetadata_resultStandardSchemeFactory implements SchemeFactory { - public GetResultSetMetadata_resultStandardScheme getScheme() { - return new GetResultSetMetadata_resultStandardScheme(); - } - } - - private static class GetResultSetMetadata_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetResultSetMetadata_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetResultSetMetadataResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetResultSetMetadata_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetResultSetMetadata_resultTupleSchemeFactory implements SchemeFactory { - public GetResultSetMetadata_resultTupleScheme getScheme() { - return new GetResultSetMetadata_resultTupleScheme(); - } - } - - private static class GetResultSetMetadata_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetResultSetMetadata_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetResultSetMetadata_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetResultSetMetadataResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class FetchResults_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("FetchResults_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new FetchResults_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new FetchResults_argsTupleSchemeFactory()); - } - - private TFetchResultsReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFetchResultsReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(FetchResults_args.class, metaDataMap); - } - - public FetchResults_args() { - } - - public FetchResults_args( - TFetchResultsReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public FetchResults_args(FetchResults_args other) { - if (other.isSetReq()) { - this.req = new TFetchResultsReq(other.req); - } - } - - public FetchResults_args deepCopy() { - return new FetchResults_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TFetchResultsReq getReq() { - return this.req; - } - - public void setReq(TFetchResultsReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TFetchResultsReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof FetchResults_args) - return this.equals((FetchResults_args)that); - return false; - } - - public boolean equals(FetchResults_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(FetchResults_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - FetchResults_args typedOther = (FetchResults_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("FetchResults_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class FetchResults_argsStandardSchemeFactory implements SchemeFactory { - public FetchResults_argsStandardScheme getScheme() { - return new FetchResults_argsStandardScheme(); - } - } - - private static class FetchResults_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, FetchResults_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TFetchResultsReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, FetchResults_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class FetchResults_argsTupleSchemeFactory implements SchemeFactory { - public FetchResults_argsTupleScheme getScheme() { - return new FetchResults_argsTupleScheme(); - } - } - - private static class FetchResults_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, FetchResults_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, FetchResults_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TFetchResultsReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class FetchResults_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("FetchResults_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new FetchResults_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new FetchResults_resultTupleSchemeFactory()); - } - - private TFetchResultsResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFetchResultsResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(FetchResults_result.class, metaDataMap); - } - - public FetchResults_result() { - } - - public FetchResults_result( - TFetchResultsResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public FetchResults_result(FetchResults_result other) { - if (other.isSetSuccess()) { - this.success = new TFetchResultsResp(other.success); - } - } - - public FetchResults_result deepCopy() { - return new FetchResults_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TFetchResultsResp getSuccess() { - return this.success; - } - - public void setSuccess(TFetchResultsResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TFetchResultsResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof FetchResults_result) - return this.equals((FetchResults_result)that); - return false; - } - - public boolean equals(FetchResults_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(FetchResults_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - FetchResults_result typedOther = (FetchResults_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("FetchResults_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class FetchResults_resultStandardSchemeFactory implements SchemeFactory { - public FetchResults_resultStandardScheme getScheme() { - return new FetchResults_resultStandardScheme(); - } - } - - private static class FetchResults_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, FetchResults_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TFetchResultsResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, FetchResults_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class FetchResults_resultTupleSchemeFactory implements SchemeFactory { - public FetchResults_resultTupleScheme getScheme() { - return new FetchResults_resultTupleScheme(); - } - } - - private static class FetchResults_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, FetchResults_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, FetchResults_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TFetchResultsResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetDelegationToken_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetDelegationToken_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetDelegationToken_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetDelegationToken_argsTupleSchemeFactory()); - } - - private TGetDelegationTokenReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetDelegationTokenReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetDelegationToken_args.class, metaDataMap); - } - - public GetDelegationToken_args() { - } - - public GetDelegationToken_args( - TGetDelegationTokenReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetDelegationToken_args(GetDelegationToken_args other) { - if (other.isSetReq()) { - this.req = new TGetDelegationTokenReq(other.req); - } - } - - public GetDelegationToken_args deepCopy() { - return new GetDelegationToken_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetDelegationTokenReq getReq() { - return this.req; - } - - public void setReq(TGetDelegationTokenReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetDelegationTokenReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetDelegationToken_args) - return this.equals((GetDelegationToken_args)that); - return false; - } - - public boolean equals(GetDelegationToken_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetDelegationToken_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetDelegationToken_args typedOther = (GetDelegationToken_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetDelegationToken_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetDelegationToken_argsStandardSchemeFactory implements SchemeFactory { - public GetDelegationToken_argsStandardScheme getScheme() { - return new GetDelegationToken_argsStandardScheme(); - } - } - - private static class GetDelegationToken_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetDelegationToken_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetDelegationTokenReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetDelegationToken_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetDelegationToken_argsTupleSchemeFactory implements SchemeFactory { - public GetDelegationToken_argsTupleScheme getScheme() { - return new GetDelegationToken_argsTupleScheme(); - } - } - - private static class GetDelegationToken_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetDelegationToken_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetDelegationToken_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetDelegationTokenReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetDelegationToken_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetDelegationToken_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetDelegationToken_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetDelegationToken_resultTupleSchemeFactory()); - } - - private TGetDelegationTokenResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetDelegationTokenResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetDelegationToken_result.class, metaDataMap); - } - - public GetDelegationToken_result() { - } - - public GetDelegationToken_result( - TGetDelegationTokenResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetDelegationToken_result(GetDelegationToken_result other) { - if (other.isSetSuccess()) { - this.success = new TGetDelegationTokenResp(other.success); - } - } - - public GetDelegationToken_result deepCopy() { - return new GetDelegationToken_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetDelegationTokenResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetDelegationTokenResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetDelegationTokenResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetDelegationToken_result) - return this.equals((GetDelegationToken_result)that); - return false; - } - - public boolean equals(GetDelegationToken_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetDelegationToken_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetDelegationToken_result typedOther = (GetDelegationToken_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetDelegationToken_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetDelegationToken_resultStandardSchemeFactory implements SchemeFactory { - public GetDelegationToken_resultStandardScheme getScheme() { - return new GetDelegationToken_resultStandardScheme(); - } - } - - private static class GetDelegationToken_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetDelegationToken_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetDelegationTokenResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetDelegationToken_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetDelegationToken_resultTupleSchemeFactory implements SchemeFactory { - public GetDelegationToken_resultTupleScheme getScheme() { - return new GetDelegationToken_resultTupleScheme(); - } - } - - private static class GetDelegationToken_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetDelegationToken_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetDelegationToken_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetDelegationTokenResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class CancelDelegationToken_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CancelDelegationToken_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new CancelDelegationToken_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new CancelDelegationToken_argsTupleSchemeFactory()); - } - - private TCancelDelegationTokenReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCancelDelegationTokenReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CancelDelegationToken_args.class, metaDataMap); - } - - public CancelDelegationToken_args() { - } - - public CancelDelegationToken_args( - TCancelDelegationTokenReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public CancelDelegationToken_args(CancelDelegationToken_args other) { - if (other.isSetReq()) { - this.req = new TCancelDelegationTokenReq(other.req); - } - } - - public CancelDelegationToken_args deepCopy() { - return new CancelDelegationToken_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TCancelDelegationTokenReq getReq() { - return this.req; - } - - public void setReq(TCancelDelegationTokenReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TCancelDelegationTokenReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof CancelDelegationToken_args) - return this.equals((CancelDelegationToken_args)that); - return false; - } - - public boolean equals(CancelDelegationToken_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(CancelDelegationToken_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - CancelDelegationToken_args typedOther = (CancelDelegationToken_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("CancelDelegationToken_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class CancelDelegationToken_argsStandardSchemeFactory implements SchemeFactory { - public CancelDelegationToken_argsStandardScheme getScheme() { - return new CancelDelegationToken_argsStandardScheme(); - } - } - - private static class CancelDelegationToken_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, CancelDelegationToken_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TCancelDelegationTokenReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, CancelDelegationToken_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class CancelDelegationToken_argsTupleSchemeFactory implements SchemeFactory { - public CancelDelegationToken_argsTupleScheme getScheme() { - return new CancelDelegationToken_argsTupleScheme(); - } - } - - private static class CancelDelegationToken_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, CancelDelegationToken_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, CancelDelegationToken_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TCancelDelegationTokenReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class CancelDelegationToken_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CancelDelegationToken_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new CancelDelegationToken_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new CancelDelegationToken_resultTupleSchemeFactory()); - } - - private TCancelDelegationTokenResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCancelDelegationTokenResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CancelDelegationToken_result.class, metaDataMap); - } - - public CancelDelegationToken_result() { - } - - public CancelDelegationToken_result( - TCancelDelegationTokenResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public CancelDelegationToken_result(CancelDelegationToken_result other) { - if (other.isSetSuccess()) { - this.success = new TCancelDelegationTokenResp(other.success); - } - } - - public CancelDelegationToken_result deepCopy() { - return new CancelDelegationToken_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TCancelDelegationTokenResp getSuccess() { - return this.success; - } - - public void setSuccess(TCancelDelegationTokenResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TCancelDelegationTokenResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof CancelDelegationToken_result) - return this.equals((CancelDelegationToken_result)that); - return false; - } - - public boolean equals(CancelDelegationToken_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(CancelDelegationToken_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - CancelDelegationToken_result typedOther = (CancelDelegationToken_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("CancelDelegationToken_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class CancelDelegationToken_resultStandardSchemeFactory implements SchemeFactory { - public CancelDelegationToken_resultStandardScheme getScheme() { - return new CancelDelegationToken_resultStandardScheme(); - } - } - - private static class CancelDelegationToken_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, CancelDelegationToken_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TCancelDelegationTokenResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, CancelDelegationToken_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class CancelDelegationToken_resultTupleSchemeFactory implements SchemeFactory { - public CancelDelegationToken_resultTupleScheme getScheme() { - return new CancelDelegationToken_resultTupleScheme(); - } - } - - private static class CancelDelegationToken_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, CancelDelegationToken_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, CancelDelegationToken_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TCancelDelegationTokenResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class RenewDelegationToken_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RenewDelegationToken_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new RenewDelegationToken_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new RenewDelegationToken_argsTupleSchemeFactory()); - } - - private TRenewDelegationTokenReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRenewDelegationTokenReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(RenewDelegationToken_args.class, metaDataMap); - } - - public RenewDelegationToken_args() { - } - - public RenewDelegationToken_args( - TRenewDelegationTokenReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public RenewDelegationToken_args(RenewDelegationToken_args other) { - if (other.isSetReq()) { - this.req = new TRenewDelegationTokenReq(other.req); - } - } - - public RenewDelegationToken_args deepCopy() { - return new RenewDelegationToken_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TRenewDelegationTokenReq getReq() { - return this.req; - } - - public void setReq(TRenewDelegationTokenReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TRenewDelegationTokenReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof RenewDelegationToken_args) - return this.equals((RenewDelegationToken_args)that); - return false; - } - - public boolean equals(RenewDelegationToken_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(RenewDelegationToken_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - RenewDelegationToken_args typedOther = (RenewDelegationToken_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("RenewDelegationToken_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class RenewDelegationToken_argsStandardSchemeFactory implements SchemeFactory { - public RenewDelegationToken_argsStandardScheme getScheme() { - return new RenewDelegationToken_argsStandardScheme(); - } - } - - private static class RenewDelegationToken_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, RenewDelegationToken_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TRenewDelegationTokenReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, RenewDelegationToken_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class RenewDelegationToken_argsTupleSchemeFactory implements SchemeFactory { - public RenewDelegationToken_argsTupleScheme getScheme() { - return new RenewDelegationToken_argsTupleScheme(); - } - } - - private static class RenewDelegationToken_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TRenewDelegationTokenReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class RenewDelegationToken_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RenewDelegationToken_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new RenewDelegationToken_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new RenewDelegationToken_resultTupleSchemeFactory()); - } - - private TRenewDelegationTokenResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRenewDelegationTokenResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(RenewDelegationToken_result.class, metaDataMap); - } - - public RenewDelegationToken_result() { - } - - public RenewDelegationToken_result( - TRenewDelegationTokenResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public RenewDelegationToken_result(RenewDelegationToken_result other) { - if (other.isSetSuccess()) { - this.success = new TRenewDelegationTokenResp(other.success); - } - } - - public RenewDelegationToken_result deepCopy() { - return new RenewDelegationToken_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TRenewDelegationTokenResp getSuccess() { - return this.success; - } - - public void setSuccess(TRenewDelegationTokenResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TRenewDelegationTokenResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof RenewDelegationToken_result) - return this.equals((RenewDelegationToken_result)that); - return false; - } - - public boolean equals(RenewDelegationToken_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(RenewDelegationToken_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - RenewDelegationToken_result typedOther = (RenewDelegationToken_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("RenewDelegationToken_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class RenewDelegationToken_resultStandardSchemeFactory implements SchemeFactory { - public RenewDelegationToken_resultStandardScheme getScheme() { - return new RenewDelegationToken_resultStandardScheme(); - } - } - - private static class RenewDelegationToken_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, RenewDelegationToken_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TRenewDelegationTokenResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, RenewDelegationToken_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class RenewDelegationToken_resultTupleSchemeFactory implements SchemeFactory { - public RenewDelegationToken_resultTupleScheme getScheme() { - return new RenewDelegationToken_resultTupleScheme(); - } - } - - private static class RenewDelegationToken_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TRenewDelegationTokenResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java deleted file mode 100644 index 25a38b178428a..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java +++ /dev/null @@ -1,103 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCLIServiceConstants { - - public static final Set PRIMITIVE_TYPES = new HashSet(); - static { - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.BOOLEAN_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.TINYINT_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.SMALLINT_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.INT_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.BIGINT_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.FLOAT_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.DOUBLE_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.STRING_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.TIMESTAMP_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.BINARY_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.DECIMAL_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.NULL_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.DATE_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.VARCHAR_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.CHAR_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.INTERVAL_YEAR_MONTH_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.INTERVAL_DAY_TIME_TYPE); - } - - public static final Set COMPLEX_TYPES = new HashSet(); - static { - COMPLEX_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.ARRAY_TYPE); - COMPLEX_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.MAP_TYPE); - COMPLEX_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.STRUCT_TYPE); - COMPLEX_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.UNION_TYPE); - COMPLEX_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.USER_DEFINED_TYPE); - } - - public static final Set COLLECTION_TYPES = new HashSet(); - static { - COLLECTION_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.ARRAY_TYPE); - COLLECTION_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.MAP_TYPE); - } - - public static final Map TYPE_NAMES = new HashMap(); - static { - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.BOOLEAN_TYPE, "BOOLEAN"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.TINYINT_TYPE, "TINYINT"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.SMALLINT_TYPE, "SMALLINT"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.INT_TYPE, "INT"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.BIGINT_TYPE, "BIGINT"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.FLOAT_TYPE, "FLOAT"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.DOUBLE_TYPE, "DOUBLE"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.STRING_TYPE, "STRING"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.TIMESTAMP_TYPE, "TIMESTAMP"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.BINARY_TYPE, "BINARY"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.ARRAY_TYPE, "ARRAY"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.MAP_TYPE, "MAP"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.STRUCT_TYPE, "STRUCT"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.UNION_TYPE, "UNIONTYPE"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.DECIMAL_TYPE, "DECIMAL"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.NULL_TYPE, "NULL"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.DATE_TYPE, "DATE"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.VARCHAR_TYPE, "VARCHAR"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.CHAR_TYPE, "CHAR"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.INTERVAL_YEAR_MONTH_TYPE, "INTERVAL_YEAR_MONTH"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.INTERVAL_DAY_TIME_TYPE, "INTERVAL_DAY_TIME"); - } - - public static final String CHARACTER_MAXIMUM_LENGTH = "characterMaximumLength"; - - public static final String PRECISION = "precision"; - - public static final String SCALE = "scale"; - -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java deleted file mode 100644 index e23fcdd77a1a4..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java +++ /dev/null @@ -1,491 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCancelDelegationTokenReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelDelegationTokenReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField DELEGATION_TOKEN_FIELD_DESC = new org.apache.thrift.protocol.TField("delegationToken", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TCancelDelegationTokenReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TCancelDelegationTokenReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private String delegationToken; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - DELEGATION_TOKEN((short)2, "delegationToken"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // DELEGATION_TOKEN - return DELEGATION_TOKEN; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.DELEGATION_TOKEN, new org.apache.thrift.meta_data.FieldMetaData("delegationToken", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCancelDelegationTokenReq.class, metaDataMap); - } - - public TCancelDelegationTokenReq() { - } - - public TCancelDelegationTokenReq( - TSessionHandle sessionHandle, - String delegationToken) - { - this(); - this.sessionHandle = sessionHandle; - this.delegationToken = delegationToken; - } - - /** - * Performs a deep copy on other. - */ - public TCancelDelegationTokenReq(TCancelDelegationTokenReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetDelegationToken()) { - this.delegationToken = other.delegationToken; - } - } - - public TCancelDelegationTokenReq deepCopy() { - return new TCancelDelegationTokenReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.delegationToken = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public String getDelegationToken() { - return this.delegationToken; - } - - public void setDelegationToken(String delegationToken) { - this.delegationToken = delegationToken; - } - - public void unsetDelegationToken() { - this.delegationToken = null; - } - - /** Returns true if field delegationToken is set (has been assigned a value) and false otherwise */ - public boolean isSetDelegationToken() { - return this.delegationToken != null; - } - - public void setDelegationTokenIsSet(boolean value) { - if (!value) { - this.delegationToken = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case DELEGATION_TOKEN: - if (value == null) { - unsetDelegationToken(); - } else { - setDelegationToken((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case DELEGATION_TOKEN: - return getDelegationToken(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case DELEGATION_TOKEN: - return isSetDelegationToken(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TCancelDelegationTokenReq) - return this.equals((TCancelDelegationTokenReq)that); - return false; - } - - public boolean equals(TCancelDelegationTokenReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_delegationToken = true && this.isSetDelegationToken(); - boolean that_present_delegationToken = true && that.isSetDelegationToken(); - if (this_present_delegationToken || that_present_delegationToken) { - if (!(this_present_delegationToken && that_present_delegationToken)) - return false; - if (!this.delegationToken.equals(that.delegationToken)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_delegationToken = true && (isSetDelegationToken()); - builder.append(present_delegationToken); - if (present_delegationToken) - builder.append(delegationToken); - - return builder.toHashCode(); - } - - public int compareTo(TCancelDelegationTokenReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TCancelDelegationTokenReq typedOther = (TCancelDelegationTokenReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetDelegationToken()).compareTo(typedOther.isSetDelegationToken()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetDelegationToken()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.delegationToken, typedOther.delegationToken); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TCancelDelegationTokenReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (!first) sb.append(", "); - sb.append("delegationToken:"); - if (this.delegationToken == null) { - sb.append("null"); - } else { - sb.append(this.delegationToken); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - if (!isSetDelegationToken()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'delegationToken' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TCancelDelegationTokenReqStandardSchemeFactory implements SchemeFactory { - public TCancelDelegationTokenReqStandardScheme getScheme() { - return new TCancelDelegationTokenReqStandardScheme(); - } - } - - private static class TCancelDelegationTokenReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TCancelDelegationTokenReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // DELEGATION_TOKEN - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.delegationToken = iprot.readString(); - struct.setDelegationTokenIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TCancelDelegationTokenReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.delegationToken != null) { - oprot.writeFieldBegin(DELEGATION_TOKEN_FIELD_DESC); - oprot.writeString(struct.delegationToken); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TCancelDelegationTokenReqTupleSchemeFactory implements SchemeFactory { - public TCancelDelegationTokenReqTupleScheme getScheme() { - return new TCancelDelegationTokenReqTupleScheme(); - } - } - - private static class TCancelDelegationTokenReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TCancelDelegationTokenReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - oprot.writeString(struct.delegationToken); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TCancelDelegationTokenReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - struct.delegationToken = iprot.readString(); - struct.setDelegationTokenIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java deleted file mode 100644 index 77c9ee77ec59b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCancelDelegationTokenResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelDelegationTokenResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TCancelDelegationTokenRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TCancelDelegationTokenRespTupleSchemeFactory()); - } - - private TStatus status; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCancelDelegationTokenResp.class, metaDataMap); - } - - public TCancelDelegationTokenResp() { - } - - public TCancelDelegationTokenResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TCancelDelegationTokenResp(TCancelDelegationTokenResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - } - - public TCancelDelegationTokenResp deepCopy() { - return new TCancelDelegationTokenResp(this); - } - - @Override - public void clear() { - this.status = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TCancelDelegationTokenResp) - return this.equals((TCancelDelegationTokenResp)that); - return false; - } - - public boolean equals(TCancelDelegationTokenResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - return builder.toHashCode(); - } - - public int compareTo(TCancelDelegationTokenResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TCancelDelegationTokenResp typedOther = (TCancelDelegationTokenResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TCancelDelegationTokenResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TCancelDelegationTokenRespStandardSchemeFactory implements SchemeFactory { - public TCancelDelegationTokenRespStandardScheme getScheme() { - return new TCancelDelegationTokenRespStandardScheme(); - } - } - - private static class TCancelDelegationTokenRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TCancelDelegationTokenResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TCancelDelegationTokenResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TCancelDelegationTokenRespTupleSchemeFactory implements SchemeFactory { - public TCancelDelegationTokenRespTupleScheme getScheme() { - return new TCancelDelegationTokenRespTupleScheme(); - } - } - - private static class TCancelDelegationTokenRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TCancelDelegationTokenResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TCancelDelegationTokenResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java deleted file mode 100644 index 45eac48ab12d3..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCancelOperationReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelOperationReq"); - - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TCancelOperationReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TCancelOperationReqTupleSchemeFactory()); - } - - private TOperationHandle operationHandle; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - OPERATION_HANDLE((short)1, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCancelOperationReq.class, metaDataMap); - } - - public TCancelOperationReq() { - } - - public TCancelOperationReq( - TOperationHandle operationHandle) - { - this(); - this.operationHandle = operationHandle; - } - - /** - * Performs a deep copy on other. - */ - public TCancelOperationReq(TCancelOperationReq other) { - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TCancelOperationReq deepCopy() { - return new TCancelOperationReq(this); - } - - @Override - public void clear() { - this.operationHandle = null; - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TCancelOperationReq) - return this.equals((TCancelOperationReq)that); - return false; - } - - public boolean equals(TCancelOperationReq that) { - if (that == null) - return false; - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TCancelOperationReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TCancelOperationReq typedOther = (TCancelOperationReq)other; - - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TCancelOperationReq("); - boolean first = true; - - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetOperationHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TCancelOperationReqStandardSchemeFactory implements SchemeFactory { - public TCancelOperationReqStandardScheme getScheme() { - return new TCancelOperationReqStandardScheme(); - } - } - - private static class TCancelOperationReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TCancelOperationReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TCancelOperationReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.operationHandle != null) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TCancelOperationReqTupleSchemeFactory implements SchemeFactory { - public TCancelOperationReqTupleScheme getScheme() { - return new TCancelOperationReqTupleScheme(); - } - } - - private static class TCancelOperationReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TCancelOperationReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.operationHandle.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TCancelOperationReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java deleted file mode 100644 index 2a39414d601aa..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCancelOperationResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelOperationResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TCancelOperationRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TCancelOperationRespTupleSchemeFactory()); - } - - private TStatus status; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCancelOperationResp.class, metaDataMap); - } - - public TCancelOperationResp() { - } - - public TCancelOperationResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TCancelOperationResp(TCancelOperationResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - } - - public TCancelOperationResp deepCopy() { - return new TCancelOperationResp(this); - } - - @Override - public void clear() { - this.status = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TCancelOperationResp) - return this.equals((TCancelOperationResp)that); - return false; - } - - public boolean equals(TCancelOperationResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - return builder.toHashCode(); - } - - public int compareTo(TCancelOperationResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TCancelOperationResp typedOther = (TCancelOperationResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TCancelOperationResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TCancelOperationRespStandardSchemeFactory implements SchemeFactory { - public TCancelOperationRespStandardScheme getScheme() { - return new TCancelOperationRespStandardScheme(); - } - } - - private static class TCancelOperationRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TCancelOperationResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TCancelOperationResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TCancelOperationRespTupleSchemeFactory implements SchemeFactory { - public TCancelOperationRespTupleScheme getScheme() { - return new TCancelOperationRespTupleScheme(); - } - } - - private static class TCancelOperationRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TCancelOperationResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TCancelOperationResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java deleted file mode 100644 index 0cbb7ccced073..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCloseOperationReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseOperationReq"); - - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TCloseOperationReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TCloseOperationReqTupleSchemeFactory()); - } - - private TOperationHandle operationHandle; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - OPERATION_HANDLE((short)1, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCloseOperationReq.class, metaDataMap); - } - - public TCloseOperationReq() { - } - - public TCloseOperationReq( - TOperationHandle operationHandle) - { - this(); - this.operationHandle = operationHandle; - } - - /** - * Performs a deep copy on other. - */ - public TCloseOperationReq(TCloseOperationReq other) { - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TCloseOperationReq deepCopy() { - return new TCloseOperationReq(this); - } - - @Override - public void clear() { - this.operationHandle = null; - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TCloseOperationReq) - return this.equals((TCloseOperationReq)that); - return false; - } - - public boolean equals(TCloseOperationReq that) { - if (that == null) - return false; - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TCloseOperationReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TCloseOperationReq typedOther = (TCloseOperationReq)other; - - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TCloseOperationReq("); - boolean first = true; - - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetOperationHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TCloseOperationReqStandardSchemeFactory implements SchemeFactory { - public TCloseOperationReqStandardScheme getScheme() { - return new TCloseOperationReqStandardScheme(); - } - } - - private static class TCloseOperationReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TCloseOperationReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TCloseOperationReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.operationHandle != null) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TCloseOperationReqTupleSchemeFactory implements SchemeFactory { - public TCloseOperationReqTupleScheme getScheme() { - return new TCloseOperationReqTupleScheme(); - } - } - - private static class TCloseOperationReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TCloseOperationReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.operationHandle.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TCloseOperationReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java deleted file mode 100644 index 7334d67173d7b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCloseOperationResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseOperationResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TCloseOperationRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TCloseOperationRespTupleSchemeFactory()); - } - - private TStatus status; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCloseOperationResp.class, metaDataMap); - } - - public TCloseOperationResp() { - } - - public TCloseOperationResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TCloseOperationResp(TCloseOperationResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - } - - public TCloseOperationResp deepCopy() { - return new TCloseOperationResp(this); - } - - @Override - public void clear() { - this.status = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TCloseOperationResp) - return this.equals((TCloseOperationResp)that); - return false; - } - - public boolean equals(TCloseOperationResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - return builder.toHashCode(); - } - - public int compareTo(TCloseOperationResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TCloseOperationResp typedOther = (TCloseOperationResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TCloseOperationResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TCloseOperationRespStandardSchemeFactory implements SchemeFactory { - public TCloseOperationRespStandardScheme getScheme() { - return new TCloseOperationRespStandardScheme(); - } - } - - private static class TCloseOperationRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TCloseOperationResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TCloseOperationResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TCloseOperationRespTupleSchemeFactory implements SchemeFactory { - public TCloseOperationRespTupleScheme getScheme() { - return new TCloseOperationRespTupleScheme(); - } - } - - private static class TCloseOperationRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TCloseOperationResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TCloseOperationResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java deleted file mode 100644 index 027e8295436b0..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCloseSessionReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseSessionReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TCloseSessionReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TCloseSessionReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCloseSessionReq.class, metaDataMap); - } - - public TCloseSessionReq() { - } - - public TCloseSessionReq( - TSessionHandle sessionHandle) - { - this(); - this.sessionHandle = sessionHandle; - } - - /** - * Performs a deep copy on other. - */ - public TCloseSessionReq(TCloseSessionReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - } - - public TCloseSessionReq deepCopy() { - return new TCloseSessionReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TCloseSessionReq) - return this.equals((TCloseSessionReq)that); - return false; - } - - public boolean equals(TCloseSessionReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - return builder.toHashCode(); - } - - public int compareTo(TCloseSessionReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TCloseSessionReq typedOther = (TCloseSessionReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TCloseSessionReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TCloseSessionReqStandardSchemeFactory implements SchemeFactory { - public TCloseSessionReqStandardScheme getScheme() { - return new TCloseSessionReqStandardScheme(); - } - } - - private static class TCloseSessionReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TCloseSessionReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TCloseSessionReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TCloseSessionReqTupleSchemeFactory implements SchemeFactory { - public TCloseSessionReqTupleScheme getScheme() { - return new TCloseSessionReqTupleScheme(); - } - } - - private static class TCloseSessionReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TCloseSessionReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TCloseSessionReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java deleted file mode 100644 index 168c8fc775e33..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCloseSessionResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseSessionResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TCloseSessionRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TCloseSessionRespTupleSchemeFactory()); - } - - private TStatus status; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCloseSessionResp.class, metaDataMap); - } - - public TCloseSessionResp() { - } - - public TCloseSessionResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TCloseSessionResp(TCloseSessionResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - } - - public TCloseSessionResp deepCopy() { - return new TCloseSessionResp(this); - } - - @Override - public void clear() { - this.status = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TCloseSessionResp) - return this.equals((TCloseSessionResp)that); - return false; - } - - public boolean equals(TCloseSessionResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - return builder.toHashCode(); - } - - public int compareTo(TCloseSessionResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TCloseSessionResp typedOther = (TCloseSessionResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TCloseSessionResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TCloseSessionRespStandardSchemeFactory implements SchemeFactory { - public TCloseSessionRespStandardScheme getScheme() { - return new TCloseSessionRespStandardScheme(); - } - } - - private static class TCloseSessionRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TCloseSessionResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TCloseSessionResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TCloseSessionRespTupleSchemeFactory implements SchemeFactory { - public TCloseSessionRespTupleScheme getScheme() { - return new TCloseSessionRespTupleScheme(); - } - } - - private static class TCloseSessionRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TCloseSessionResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TCloseSessionResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java deleted file mode 100644 index fc2171dc99e4c..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java +++ /dev/null @@ -1,732 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TColumn extends org.apache.thrift.TUnion { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn"); - private static final org.apache.thrift.protocol.TField BOOL_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("boolVal", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField BYTE_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("byteVal", org.apache.thrift.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift.protocol.TField I16_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i16Val", org.apache.thrift.protocol.TType.STRUCT, (short)3); - private static final org.apache.thrift.protocol.TField I32_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i32Val", org.apache.thrift.protocol.TType.STRUCT, (short)4); - private static final org.apache.thrift.protocol.TField I64_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i64Val", org.apache.thrift.protocol.TType.STRUCT, (short)5); - private static final org.apache.thrift.protocol.TField DOUBLE_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("doubleVal", org.apache.thrift.protocol.TType.STRUCT, (short)6); - private static final org.apache.thrift.protocol.TField STRING_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("stringVal", org.apache.thrift.protocol.TType.STRUCT, (short)7); - private static final org.apache.thrift.protocol.TField BINARY_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("binaryVal", org.apache.thrift.protocol.TType.STRUCT, (short)8); - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - BOOL_VAL((short)1, "boolVal"), - BYTE_VAL((short)2, "byteVal"), - I16_VAL((short)3, "i16Val"), - I32_VAL((short)4, "i32Val"), - I64_VAL((short)5, "i64Val"), - DOUBLE_VAL((short)6, "doubleVal"), - STRING_VAL((short)7, "stringVal"), - BINARY_VAL((short)8, "binaryVal"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // BOOL_VAL - return BOOL_VAL; - case 2: // BYTE_VAL - return BYTE_VAL; - case 3: // I16_VAL - return I16_VAL; - case 4: // I32_VAL - return I32_VAL; - case 5: // I64_VAL - return I64_VAL; - case 6: // DOUBLE_VAL - return DOUBLE_VAL; - case 7: // STRING_VAL - return STRING_VAL; - case 8: // BINARY_VAL - return BINARY_VAL; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.BOOL_VAL, new org.apache.thrift.meta_data.FieldMetaData("boolVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TBoolColumn.class))); - tmpMap.put(_Fields.BYTE_VAL, new org.apache.thrift.meta_data.FieldMetaData("byteVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TByteColumn.class))); - tmpMap.put(_Fields.I16_VAL, new org.apache.thrift.meta_data.FieldMetaData("i16Val", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI16Column.class))); - tmpMap.put(_Fields.I32_VAL, new org.apache.thrift.meta_data.FieldMetaData("i32Val", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI32Column.class))); - tmpMap.put(_Fields.I64_VAL, new org.apache.thrift.meta_data.FieldMetaData("i64Val", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI64Column.class))); - tmpMap.put(_Fields.DOUBLE_VAL, new org.apache.thrift.meta_data.FieldMetaData("doubleVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TDoubleColumn.class))); - tmpMap.put(_Fields.STRING_VAL, new org.apache.thrift.meta_data.FieldMetaData("stringVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStringColumn.class))); - tmpMap.put(_Fields.BINARY_VAL, new org.apache.thrift.meta_data.FieldMetaData("binaryVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TBinaryColumn.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumn.class, metaDataMap); - } - - public TColumn() { - super(); - } - - public TColumn(TColumn._Fields setField, Object value) { - super(setField, value); - } - - public TColumn(TColumn other) { - super(other); - } - public TColumn deepCopy() { - return new TColumn(this); - } - - public static TColumn boolVal(TBoolColumn value) { - TColumn x = new TColumn(); - x.setBoolVal(value); - return x; - } - - public static TColumn byteVal(TByteColumn value) { - TColumn x = new TColumn(); - x.setByteVal(value); - return x; - } - - public static TColumn i16Val(TI16Column value) { - TColumn x = new TColumn(); - x.setI16Val(value); - return x; - } - - public static TColumn i32Val(TI32Column value) { - TColumn x = new TColumn(); - x.setI32Val(value); - return x; - } - - public static TColumn i64Val(TI64Column value) { - TColumn x = new TColumn(); - x.setI64Val(value); - return x; - } - - public static TColumn doubleVal(TDoubleColumn value) { - TColumn x = new TColumn(); - x.setDoubleVal(value); - return x; - } - - public static TColumn stringVal(TStringColumn value) { - TColumn x = new TColumn(); - x.setStringVal(value); - return x; - } - - public static TColumn binaryVal(TBinaryColumn value) { - TColumn x = new TColumn(); - x.setBinaryVal(value); - return x; - } - - - @Override - protected void checkType(_Fields setField, Object value) throws ClassCastException { - switch (setField) { - case BOOL_VAL: - if (value instanceof TBoolColumn) { - break; - } - throw new ClassCastException("Was expecting value of type TBoolColumn for field 'boolVal', but got " + value.getClass().getSimpleName()); - case BYTE_VAL: - if (value instanceof TByteColumn) { - break; - } - throw new ClassCastException("Was expecting value of type TByteColumn for field 'byteVal', but got " + value.getClass().getSimpleName()); - case I16_VAL: - if (value instanceof TI16Column) { - break; - } - throw new ClassCastException("Was expecting value of type TI16Column for field 'i16Val', but got " + value.getClass().getSimpleName()); - case I32_VAL: - if (value instanceof TI32Column) { - break; - } - throw new ClassCastException("Was expecting value of type TI32Column for field 'i32Val', but got " + value.getClass().getSimpleName()); - case I64_VAL: - if (value instanceof TI64Column) { - break; - } - throw new ClassCastException("Was expecting value of type TI64Column for field 'i64Val', but got " + value.getClass().getSimpleName()); - case DOUBLE_VAL: - if (value instanceof TDoubleColumn) { - break; - } - throw new ClassCastException("Was expecting value of type TDoubleColumn for field 'doubleVal', but got " + value.getClass().getSimpleName()); - case STRING_VAL: - if (value instanceof TStringColumn) { - break; - } - throw new ClassCastException("Was expecting value of type TStringColumn for field 'stringVal', but got " + value.getClass().getSimpleName()); - case BINARY_VAL: - if (value instanceof TBinaryColumn) { - break; - } - throw new ClassCastException("Was expecting value of type TBinaryColumn for field 'binaryVal', but got " + value.getClass().getSimpleName()); - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(field.id); - if (setField != null) { - switch (setField) { - case BOOL_VAL: - if (field.type == BOOL_VAL_FIELD_DESC.type) { - TBoolColumn boolVal; - boolVal = new TBoolColumn(); - boolVal.read(iprot); - return boolVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case BYTE_VAL: - if (field.type == BYTE_VAL_FIELD_DESC.type) { - TByteColumn byteVal; - byteVal = new TByteColumn(); - byteVal.read(iprot); - return byteVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case I16_VAL: - if (field.type == I16_VAL_FIELD_DESC.type) { - TI16Column i16Val; - i16Val = new TI16Column(); - i16Val.read(iprot); - return i16Val; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case I32_VAL: - if (field.type == I32_VAL_FIELD_DESC.type) { - TI32Column i32Val; - i32Val = new TI32Column(); - i32Val.read(iprot); - return i32Val; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case I64_VAL: - if (field.type == I64_VAL_FIELD_DESC.type) { - TI64Column i64Val; - i64Val = new TI64Column(); - i64Val.read(iprot); - return i64Val; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case DOUBLE_VAL: - if (field.type == DOUBLE_VAL_FIELD_DESC.type) { - TDoubleColumn doubleVal; - doubleVal = new TDoubleColumn(); - doubleVal.read(iprot); - return doubleVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case STRING_VAL: - if (field.type == STRING_VAL_FIELD_DESC.type) { - TStringColumn stringVal; - stringVal = new TStringColumn(); - stringVal.read(iprot); - return stringVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case BINARY_VAL: - if (field.type == BINARY_VAL_FIELD_DESC.type) { - TBinaryColumn binaryVal; - binaryVal = new TBinaryColumn(); - binaryVal.read(iprot); - return binaryVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - return null; - } - } - - @Override - protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case BOOL_VAL: - TBoolColumn boolVal = (TBoolColumn)value_; - boolVal.write(oprot); - return; - case BYTE_VAL: - TByteColumn byteVal = (TByteColumn)value_; - byteVal.write(oprot); - return; - case I16_VAL: - TI16Column i16Val = (TI16Column)value_; - i16Val.write(oprot); - return; - case I32_VAL: - TI32Column i32Val = (TI32Column)value_; - i32Val.write(oprot); - return; - case I64_VAL: - TI64Column i64Val = (TI64Column)value_; - i64Val.write(oprot); - return; - case DOUBLE_VAL: - TDoubleColumn doubleVal = (TDoubleColumn)value_; - doubleVal.write(oprot); - return; - case STRING_VAL: - TStringColumn stringVal = (TStringColumn)value_; - stringVal.write(oprot); - return; - case BINARY_VAL: - TBinaryColumn binaryVal = (TBinaryColumn)value_; - binaryVal.write(oprot); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(fieldID); - if (setField != null) { - switch (setField) { - case BOOL_VAL: - TBoolColumn boolVal; - boolVal = new TBoolColumn(); - boolVal.read(iprot); - return boolVal; - case BYTE_VAL: - TByteColumn byteVal; - byteVal = new TByteColumn(); - byteVal.read(iprot); - return byteVal; - case I16_VAL: - TI16Column i16Val; - i16Val = new TI16Column(); - i16Val.read(iprot); - return i16Val; - case I32_VAL: - TI32Column i32Val; - i32Val = new TI32Column(); - i32Val.read(iprot); - return i32Val; - case I64_VAL: - TI64Column i64Val; - i64Val = new TI64Column(); - i64Val.read(iprot); - return i64Val; - case DOUBLE_VAL: - TDoubleColumn doubleVal; - doubleVal = new TDoubleColumn(); - doubleVal.read(iprot); - return doubleVal; - case STRING_VAL: - TStringColumn stringVal; - stringVal = new TStringColumn(); - stringVal.read(iprot); - return stringVal; - case BINARY_VAL: - TBinaryColumn binaryVal; - binaryVal = new TBinaryColumn(); - binaryVal.read(iprot); - return binaryVal; - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - throw new TProtocolException("Couldn't find a field with field id " + fieldID); - } - } - - @Override - protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case BOOL_VAL: - TBoolColumn boolVal = (TBoolColumn)value_; - boolVal.write(oprot); - return; - case BYTE_VAL: - TByteColumn byteVal = (TByteColumn)value_; - byteVal.write(oprot); - return; - case I16_VAL: - TI16Column i16Val = (TI16Column)value_; - i16Val.write(oprot); - return; - case I32_VAL: - TI32Column i32Val = (TI32Column)value_; - i32Val.write(oprot); - return; - case I64_VAL: - TI64Column i64Val = (TI64Column)value_; - i64Val.write(oprot); - return; - case DOUBLE_VAL: - TDoubleColumn doubleVal = (TDoubleColumn)value_; - doubleVal.write(oprot); - return; - case STRING_VAL: - TStringColumn stringVal = (TStringColumn)value_; - stringVal.write(oprot); - return; - case BINARY_VAL: - TBinaryColumn binaryVal = (TBinaryColumn)value_; - binaryVal.write(oprot); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { - switch (setField) { - case BOOL_VAL: - return BOOL_VAL_FIELD_DESC; - case BYTE_VAL: - return BYTE_VAL_FIELD_DESC; - case I16_VAL: - return I16_VAL_FIELD_DESC; - case I32_VAL: - return I32_VAL_FIELD_DESC; - case I64_VAL: - return I64_VAL_FIELD_DESC; - case DOUBLE_VAL: - return DOUBLE_VAL_FIELD_DESC; - case STRING_VAL: - return STRING_VAL_FIELD_DESC; - case BINARY_VAL: - return BINARY_VAL_FIELD_DESC; - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected org.apache.thrift.protocol.TStruct getStructDesc() { - return STRUCT_DESC; - } - - @Override - protected _Fields enumForId(short id) { - return _Fields.findByThriftIdOrThrow(id); - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - - public TBoolColumn getBoolVal() { - if (getSetField() == _Fields.BOOL_VAL) { - return (TBoolColumn)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'boolVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setBoolVal(TBoolColumn value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.BOOL_VAL; - value_ = value; - } - - public TByteColumn getByteVal() { - if (getSetField() == _Fields.BYTE_VAL) { - return (TByteColumn)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'byteVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setByteVal(TByteColumn value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.BYTE_VAL; - value_ = value; - } - - public TI16Column getI16Val() { - if (getSetField() == _Fields.I16_VAL) { - return (TI16Column)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'i16Val' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setI16Val(TI16Column value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.I16_VAL; - value_ = value; - } - - public TI32Column getI32Val() { - if (getSetField() == _Fields.I32_VAL) { - return (TI32Column)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'i32Val' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setI32Val(TI32Column value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.I32_VAL; - value_ = value; - } - - public TI64Column getI64Val() { - if (getSetField() == _Fields.I64_VAL) { - return (TI64Column)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'i64Val' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setI64Val(TI64Column value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.I64_VAL; - value_ = value; - } - - public TDoubleColumn getDoubleVal() { - if (getSetField() == _Fields.DOUBLE_VAL) { - return (TDoubleColumn)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'doubleVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setDoubleVal(TDoubleColumn value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.DOUBLE_VAL; - value_ = value; - } - - public TStringColumn getStringVal() { - if (getSetField() == _Fields.STRING_VAL) { - return (TStringColumn)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'stringVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setStringVal(TStringColumn value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.STRING_VAL; - value_ = value; - } - - public TBinaryColumn getBinaryVal() { - if (getSetField() == _Fields.BINARY_VAL) { - return (TBinaryColumn)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'binaryVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setBinaryVal(TBinaryColumn value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.BINARY_VAL; - value_ = value; - } - - public boolean isSetBoolVal() { - return setField_ == _Fields.BOOL_VAL; - } - - - public boolean isSetByteVal() { - return setField_ == _Fields.BYTE_VAL; - } - - - public boolean isSetI16Val() { - return setField_ == _Fields.I16_VAL; - } - - - public boolean isSetI32Val() { - return setField_ == _Fields.I32_VAL; - } - - - public boolean isSetI64Val() { - return setField_ == _Fields.I64_VAL; - } - - - public boolean isSetDoubleVal() { - return setField_ == _Fields.DOUBLE_VAL; - } - - - public boolean isSetStringVal() { - return setField_ == _Fields.STRING_VAL; - } - - - public boolean isSetBinaryVal() { - return setField_ == _Fields.BINARY_VAL; - } - - - public boolean equals(Object other) { - if (other instanceof TColumn) { - return equals((TColumn)other); - } else { - return false; - } - } - - public boolean equals(TColumn other) { - return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); - } - - @Override - public int compareTo(TColumn other) { - int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); - if (lastComparison == 0) { - return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); - } - return lastComparison; - } - - - @Override - public int hashCode() { - HashCodeBuilder hcb = new HashCodeBuilder(); - hcb.append(this.getClass().getName()); - org.apache.thrift.TFieldIdEnum setField = getSetField(); - if (setField != null) { - hcb.append(setField.getThriftFieldId()); - Object value = getFieldValue(); - if (value instanceof org.apache.thrift.TEnum) { - hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue()); - } else { - hcb.append(value); - } - } - return hcb.toHashCode(); - } - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java deleted file mode 100644 index 247db6489457f..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java +++ /dev/null @@ -1,700 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TColumnDesc implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnDesc"); - - private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("columnName", org.apache.thrift.protocol.TType.STRING, (short)1); - private static final org.apache.thrift.protocol.TField TYPE_DESC_FIELD_DESC = new org.apache.thrift.protocol.TField("typeDesc", org.apache.thrift.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift.protocol.TField POSITION_FIELD_DESC = new org.apache.thrift.protocol.TField("position", org.apache.thrift.protocol.TType.I32, (short)3); - private static final org.apache.thrift.protocol.TField COMMENT_FIELD_DESC = new org.apache.thrift.protocol.TField("comment", org.apache.thrift.protocol.TType.STRING, (short)4); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TColumnDescStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TColumnDescTupleSchemeFactory()); - } - - private String columnName; // required - private TTypeDesc typeDesc; // required - private int position; // required - private String comment; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - COLUMN_NAME((short)1, "columnName"), - TYPE_DESC((short)2, "typeDesc"), - POSITION((short)3, "position"), - COMMENT((short)4, "comment"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // COLUMN_NAME - return COLUMN_NAME; - case 2: // TYPE_DESC - return TYPE_DESC; - case 3: // POSITION - return POSITION; - case 4: // COMMENT - return COMMENT; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __POSITION_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.COMMENT}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.COLUMN_NAME, new org.apache.thrift.meta_data.FieldMetaData("columnName", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.TYPE_DESC, new org.apache.thrift.meta_data.FieldMetaData("typeDesc", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTypeDesc.class))); - tmpMap.put(_Fields.POSITION, new org.apache.thrift.meta_data.FieldMetaData("position", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); - tmpMap.put(_Fields.COMMENT, new org.apache.thrift.meta_data.FieldMetaData("comment", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumnDesc.class, metaDataMap); - } - - public TColumnDesc() { - } - - public TColumnDesc( - String columnName, - TTypeDesc typeDesc, - int position) - { - this(); - this.columnName = columnName; - this.typeDesc = typeDesc; - this.position = position; - setPositionIsSet(true); - } - - /** - * Performs a deep copy on other. - */ - public TColumnDesc(TColumnDesc other) { - __isset_bitfield = other.__isset_bitfield; - if (other.isSetColumnName()) { - this.columnName = other.columnName; - } - if (other.isSetTypeDesc()) { - this.typeDesc = new TTypeDesc(other.typeDesc); - } - this.position = other.position; - if (other.isSetComment()) { - this.comment = other.comment; - } - } - - public TColumnDesc deepCopy() { - return new TColumnDesc(this); - } - - @Override - public void clear() { - this.columnName = null; - this.typeDesc = null; - setPositionIsSet(false); - this.position = 0; - this.comment = null; - } - - public String getColumnName() { - return this.columnName; - } - - public void setColumnName(String columnName) { - this.columnName = columnName; - } - - public void unsetColumnName() { - this.columnName = null; - } - - /** Returns true if field columnName is set (has been assigned a value) and false otherwise */ - public boolean isSetColumnName() { - return this.columnName != null; - } - - public void setColumnNameIsSet(boolean value) { - if (!value) { - this.columnName = null; - } - } - - public TTypeDesc getTypeDesc() { - return this.typeDesc; - } - - public void setTypeDesc(TTypeDesc typeDesc) { - this.typeDesc = typeDesc; - } - - public void unsetTypeDesc() { - this.typeDesc = null; - } - - /** Returns true if field typeDesc is set (has been assigned a value) and false otherwise */ - public boolean isSetTypeDesc() { - return this.typeDesc != null; - } - - public void setTypeDescIsSet(boolean value) { - if (!value) { - this.typeDesc = null; - } - } - - public int getPosition() { - return this.position; - } - - public void setPosition(int position) { - this.position = position; - setPositionIsSet(true); - } - - public void unsetPosition() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __POSITION_ISSET_ID); - } - - /** Returns true if field position is set (has been assigned a value) and false otherwise */ - public boolean isSetPosition() { - return EncodingUtils.testBit(__isset_bitfield, __POSITION_ISSET_ID); - } - - public void setPositionIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __POSITION_ISSET_ID, value); - } - - public String getComment() { - return this.comment; - } - - public void setComment(String comment) { - this.comment = comment; - } - - public void unsetComment() { - this.comment = null; - } - - /** Returns true if field comment is set (has been assigned a value) and false otherwise */ - public boolean isSetComment() { - return this.comment != null; - } - - public void setCommentIsSet(boolean value) { - if (!value) { - this.comment = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case COLUMN_NAME: - if (value == null) { - unsetColumnName(); - } else { - setColumnName((String)value); - } - break; - - case TYPE_DESC: - if (value == null) { - unsetTypeDesc(); - } else { - setTypeDesc((TTypeDesc)value); - } - break; - - case POSITION: - if (value == null) { - unsetPosition(); - } else { - setPosition((Integer)value); - } - break; - - case COMMENT: - if (value == null) { - unsetComment(); - } else { - setComment((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case COLUMN_NAME: - return getColumnName(); - - case TYPE_DESC: - return getTypeDesc(); - - case POSITION: - return Integer.valueOf(getPosition()); - - case COMMENT: - return getComment(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case COLUMN_NAME: - return isSetColumnName(); - case TYPE_DESC: - return isSetTypeDesc(); - case POSITION: - return isSetPosition(); - case COMMENT: - return isSetComment(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TColumnDesc) - return this.equals((TColumnDesc)that); - return false; - } - - public boolean equals(TColumnDesc that) { - if (that == null) - return false; - - boolean this_present_columnName = true && this.isSetColumnName(); - boolean that_present_columnName = true && that.isSetColumnName(); - if (this_present_columnName || that_present_columnName) { - if (!(this_present_columnName && that_present_columnName)) - return false; - if (!this.columnName.equals(that.columnName)) - return false; - } - - boolean this_present_typeDesc = true && this.isSetTypeDesc(); - boolean that_present_typeDesc = true && that.isSetTypeDesc(); - if (this_present_typeDesc || that_present_typeDesc) { - if (!(this_present_typeDesc && that_present_typeDesc)) - return false; - if (!this.typeDesc.equals(that.typeDesc)) - return false; - } - - boolean this_present_position = true; - boolean that_present_position = true; - if (this_present_position || that_present_position) { - if (!(this_present_position && that_present_position)) - return false; - if (this.position != that.position) - return false; - } - - boolean this_present_comment = true && this.isSetComment(); - boolean that_present_comment = true && that.isSetComment(); - if (this_present_comment || that_present_comment) { - if (!(this_present_comment && that_present_comment)) - return false; - if (!this.comment.equals(that.comment)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_columnName = true && (isSetColumnName()); - builder.append(present_columnName); - if (present_columnName) - builder.append(columnName); - - boolean present_typeDesc = true && (isSetTypeDesc()); - builder.append(present_typeDesc); - if (present_typeDesc) - builder.append(typeDesc); - - boolean present_position = true; - builder.append(present_position); - if (present_position) - builder.append(position); - - boolean present_comment = true && (isSetComment()); - builder.append(present_comment); - if (present_comment) - builder.append(comment); - - return builder.toHashCode(); - } - - public int compareTo(TColumnDesc other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TColumnDesc typedOther = (TColumnDesc)other; - - lastComparison = Boolean.valueOf(isSetColumnName()).compareTo(typedOther.isSetColumnName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetColumnName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columnName, typedOther.columnName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetTypeDesc()).compareTo(typedOther.isSetTypeDesc()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetTypeDesc()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.typeDesc, typedOther.typeDesc); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetPosition()).compareTo(typedOther.isSetPosition()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetPosition()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.position, typedOther.position); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetComment()).compareTo(typedOther.isSetComment()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetComment()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.comment, typedOther.comment); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TColumnDesc("); - boolean first = true; - - sb.append("columnName:"); - if (this.columnName == null) { - sb.append("null"); - } else { - sb.append(this.columnName); - } - first = false; - if (!first) sb.append(", "); - sb.append("typeDesc:"); - if (this.typeDesc == null) { - sb.append("null"); - } else { - sb.append(this.typeDesc); - } - first = false; - if (!first) sb.append(", "); - sb.append("position:"); - sb.append(this.position); - first = false; - if (isSetComment()) { - if (!first) sb.append(", "); - sb.append("comment:"); - if (this.comment == null) { - sb.append("null"); - } else { - sb.append(this.comment); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetColumnName()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'columnName' is unset! Struct:" + toString()); - } - - if (!isSetTypeDesc()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'typeDesc' is unset! Struct:" + toString()); - } - - if (!isSetPosition()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'position' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (typeDesc != null) { - typeDesc.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TColumnDescStandardSchemeFactory implements SchemeFactory { - public TColumnDescStandardScheme getScheme() { - return new TColumnDescStandardScheme(); - } - } - - private static class TColumnDescStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TColumnDesc struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // COLUMN_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.columnName = iprot.readString(); - struct.setColumnNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // TYPE_DESC - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.typeDesc = new TTypeDesc(); - struct.typeDesc.read(iprot); - struct.setTypeDescIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // POSITION - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.position = iprot.readI32(); - struct.setPositionIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // COMMENT - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.comment = iprot.readString(); - struct.setCommentIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TColumnDesc struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.columnName != null) { - oprot.writeFieldBegin(COLUMN_NAME_FIELD_DESC); - oprot.writeString(struct.columnName); - oprot.writeFieldEnd(); - } - if (struct.typeDesc != null) { - oprot.writeFieldBegin(TYPE_DESC_FIELD_DESC); - struct.typeDesc.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldBegin(POSITION_FIELD_DESC); - oprot.writeI32(struct.position); - oprot.writeFieldEnd(); - if (struct.comment != null) { - if (struct.isSetComment()) { - oprot.writeFieldBegin(COMMENT_FIELD_DESC); - oprot.writeString(struct.comment); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TColumnDescTupleSchemeFactory implements SchemeFactory { - public TColumnDescTupleScheme getScheme() { - return new TColumnDescTupleScheme(); - } - } - - private static class TColumnDescTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TColumnDesc struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeString(struct.columnName); - struct.typeDesc.write(oprot); - oprot.writeI32(struct.position); - BitSet optionals = new BitSet(); - if (struct.isSetComment()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetComment()) { - oprot.writeString(struct.comment); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TColumnDesc struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.columnName = iprot.readString(); - struct.setColumnNameIsSet(true); - struct.typeDesc = new TTypeDesc(); - struct.typeDesc.read(iprot); - struct.setTypeDescIsSet(true); - struct.position = iprot.readI32(); - struct.setPositionIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.comment = iprot.readString(); - struct.setCommentIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java deleted file mode 100644 index 8504c6d608d42..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java +++ /dev/null @@ -1,671 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TColumnValue extends org.apache.thrift.TUnion { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnValue"); - private static final org.apache.thrift.protocol.TField BOOL_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("boolVal", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField BYTE_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("byteVal", org.apache.thrift.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift.protocol.TField I16_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i16Val", org.apache.thrift.protocol.TType.STRUCT, (short)3); - private static final org.apache.thrift.protocol.TField I32_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i32Val", org.apache.thrift.protocol.TType.STRUCT, (short)4); - private static final org.apache.thrift.protocol.TField I64_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i64Val", org.apache.thrift.protocol.TType.STRUCT, (short)5); - private static final org.apache.thrift.protocol.TField DOUBLE_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("doubleVal", org.apache.thrift.protocol.TType.STRUCT, (short)6); - private static final org.apache.thrift.protocol.TField STRING_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("stringVal", org.apache.thrift.protocol.TType.STRUCT, (short)7); - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - BOOL_VAL((short)1, "boolVal"), - BYTE_VAL((short)2, "byteVal"), - I16_VAL((short)3, "i16Val"), - I32_VAL((short)4, "i32Val"), - I64_VAL((short)5, "i64Val"), - DOUBLE_VAL((short)6, "doubleVal"), - STRING_VAL((short)7, "stringVal"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // BOOL_VAL - return BOOL_VAL; - case 2: // BYTE_VAL - return BYTE_VAL; - case 3: // I16_VAL - return I16_VAL; - case 4: // I32_VAL - return I32_VAL; - case 5: // I64_VAL - return I64_VAL; - case 6: // DOUBLE_VAL - return DOUBLE_VAL; - case 7: // STRING_VAL - return STRING_VAL; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.BOOL_VAL, new org.apache.thrift.meta_data.FieldMetaData("boolVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TBoolValue.class))); - tmpMap.put(_Fields.BYTE_VAL, new org.apache.thrift.meta_data.FieldMetaData("byteVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TByteValue.class))); - tmpMap.put(_Fields.I16_VAL, new org.apache.thrift.meta_data.FieldMetaData("i16Val", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI16Value.class))); - tmpMap.put(_Fields.I32_VAL, new org.apache.thrift.meta_data.FieldMetaData("i32Val", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI32Value.class))); - tmpMap.put(_Fields.I64_VAL, new org.apache.thrift.meta_data.FieldMetaData("i64Val", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI64Value.class))); - tmpMap.put(_Fields.DOUBLE_VAL, new org.apache.thrift.meta_data.FieldMetaData("doubleVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TDoubleValue.class))); - tmpMap.put(_Fields.STRING_VAL, new org.apache.thrift.meta_data.FieldMetaData("stringVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStringValue.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumnValue.class, metaDataMap); - } - - public TColumnValue() { - super(); - } - - public TColumnValue(TColumnValue._Fields setField, Object value) { - super(setField, value); - } - - public TColumnValue(TColumnValue other) { - super(other); - } - public TColumnValue deepCopy() { - return new TColumnValue(this); - } - - public static TColumnValue boolVal(TBoolValue value) { - TColumnValue x = new TColumnValue(); - x.setBoolVal(value); - return x; - } - - public static TColumnValue byteVal(TByteValue value) { - TColumnValue x = new TColumnValue(); - x.setByteVal(value); - return x; - } - - public static TColumnValue i16Val(TI16Value value) { - TColumnValue x = new TColumnValue(); - x.setI16Val(value); - return x; - } - - public static TColumnValue i32Val(TI32Value value) { - TColumnValue x = new TColumnValue(); - x.setI32Val(value); - return x; - } - - public static TColumnValue i64Val(TI64Value value) { - TColumnValue x = new TColumnValue(); - x.setI64Val(value); - return x; - } - - public static TColumnValue doubleVal(TDoubleValue value) { - TColumnValue x = new TColumnValue(); - x.setDoubleVal(value); - return x; - } - - public static TColumnValue stringVal(TStringValue value) { - TColumnValue x = new TColumnValue(); - x.setStringVal(value); - return x; - } - - - @Override - protected void checkType(_Fields setField, Object value) throws ClassCastException { - switch (setField) { - case BOOL_VAL: - if (value instanceof TBoolValue) { - break; - } - throw new ClassCastException("Was expecting value of type TBoolValue for field 'boolVal', but got " + value.getClass().getSimpleName()); - case BYTE_VAL: - if (value instanceof TByteValue) { - break; - } - throw new ClassCastException("Was expecting value of type TByteValue for field 'byteVal', but got " + value.getClass().getSimpleName()); - case I16_VAL: - if (value instanceof TI16Value) { - break; - } - throw new ClassCastException("Was expecting value of type TI16Value for field 'i16Val', but got " + value.getClass().getSimpleName()); - case I32_VAL: - if (value instanceof TI32Value) { - break; - } - throw new ClassCastException("Was expecting value of type TI32Value for field 'i32Val', but got " + value.getClass().getSimpleName()); - case I64_VAL: - if (value instanceof TI64Value) { - break; - } - throw new ClassCastException("Was expecting value of type TI64Value for field 'i64Val', but got " + value.getClass().getSimpleName()); - case DOUBLE_VAL: - if (value instanceof TDoubleValue) { - break; - } - throw new ClassCastException("Was expecting value of type TDoubleValue for field 'doubleVal', but got " + value.getClass().getSimpleName()); - case STRING_VAL: - if (value instanceof TStringValue) { - break; - } - throw new ClassCastException("Was expecting value of type TStringValue for field 'stringVal', but got " + value.getClass().getSimpleName()); - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(field.id); - if (setField != null) { - switch (setField) { - case BOOL_VAL: - if (field.type == BOOL_VAL_FIELD_DESC.type) { - TBoolValue boolVal; - boolVal = new TBoolValue(); - boolVal.read(iprot); - return boolVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case BYTE_VAL: - if (field.type == BYTE_VAL_FIELD_DESC.type) { - TByteValue byteVal; - byteVal = new TByteValue(); - byteVal.read(iprot); - return byteVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case I16_VAL: - if (field.type == I16_VAL_FIELD_DESC.type) { - TI16Value i16Val; - i16Val = new TI16Value(); - i16Val.read(iprot); - return i16Val; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case I32_VAL: - if (field.type == I32_VAL_FIELD_DESC.type) { - TI32Value i32Val; - i32Val = new TI32Value(); - i32Val.read(iprot); - return i32Val; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case I64_VAL: - if (field.type == I64_VAL_FIELD_DESC.type) { - TI64Value i64Val; - i64Val = new TI64Value(); - i64Val.read(iprot); - return i64Val; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case DOUBLE_VAL: - if (field.type == DOUBLE_VAL_FIELD_DESC.type) { - TDoubleValue doubleVal; - doubleVal = new TDoubleValue(); - doubleVal.read(iprot); - return doubleVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case STRING_VAL: - if (field.type == STRING_VAL_FIELD_DESC.type) { - TStringValue stringVal; - stringVal = new TStringValue(); - stringVal.read(iprot); - return stringVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - return null; - } - } - - @Override - protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case BOOL_VAL: - TBoolValue boolVal = (TBoolValue)value_; - boolVal.write(oprot); - return; - case BYTE_VAL: - TByteValue byteVal = (TByteValue)value_; - byteVal.write(oprot); - return; - case I16_VAL: - TI16Value i16Val = (TI16Value)value_; - i16Val.write(oprot); - return; - case I32_VAL: - TI32Value i32Val = (TI32Value)value_; - i32Val.write(oprot); - return; - case I64_VAL: - TI64Value i64Val = (TI64Value)value_; - i64Val.write(oprot); - return; - case DOUBLE_VAL: - TDoubleValue doubleVal = (TDoubleValue)value_; - doubleVal.write(oprot); - return; - case STRING_VAL: - TStringValue stringVal = (TStringValue)value_; - stringVal.write(oprot); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(fieldID); - if (setField != null) { - switch (setField) { - case BOOL_VAL: - TBoolValue boolVal; - boolVal = new TBoolValue(); - boolVal.read(iprot); - return boolVal; - case BYTE_VAL: - TByteValue byteVal; - byteVal = new TByteValue(); - byteVal.read(iprot); - return byteVal; - case I16_VAL: - TI16Value i16Val; - i16Val = new TI16Value(); - i16Val.read(iprot); - return i16Val; - case I32_VAL: - TI32Value i32Val; - i32Val = new TI32Value(); - i32Val.read(iprot); - return i32Val; - case I64_VAL: - TI64Value i64Val; - i64Val = new TI64Value(); - i64Val.read(iprot); - return i64Val; - case DOUBLE_VAL: - TDoubleValue doubleVal; - doubleVal = new TDoubleValue(); - doubleVal.read(iprot); - return doubleVal; - case STRING_VAL: - TStringValue stringVal; - stringVal = new TStringValue(); - stringVal.read(iprot); - return stringVal; - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - throw new TProtocolException("Couldn't find a field with field id " + fieldID); - } - } - - @Override - protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case BOOL_VAL: - TBoolValue boolVal = (TBoolValue)value_; - boolVal.write(oprot); - return; - case BYTE_VAL: - TByteValue byteVal = (TByteValue)value_; - byteVal.write(oprot); - return; - case I16_VAL: - TI16Value i16Val = (TI16Value)value_; - i16Val.write(oprot); - return; - case I32_VAL: - TI32Value i32Val = (TI32Value)value_; - i32Val.write(oprot); - return; - case I64_VAL: - TI64Value i64Val = (TI64Value)value_; - i64Val.write(oprot); - return; - case DOUBLE_VAL: - TDoubleValue doubleVal = (TDoubleValue)value_; - doubleVal.write(oprot); - return; - case STRING_VAL: - TStringValue stringVal = (TStringValue)value_; - stringVal.write(oprot); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { - switch (setField) { - case BOOL_VAL: - return BOOL_VAL_FIELD_DESC; - case BYTE_VAL: - return BYTE_VAL_FIELD_DESC; - case I16_VAL: - return I16_VAL_FIELD_DESC; - case I32_VAL: - return I32_VAL_FIELD_DESC; - case I64_VAL: - return I64_VAL_FIELD_DESC; - case DOUBLE_VAL: - return DOUBLE_VAL_FIELD_DESC; - case STRING_VAL: - return STRING_VAL_FIELD_DESC; - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected org.apache.thrift.protocol.TStruct getStructDesc() { - return STRUCT_DESC; - } - - @Override - protected _Fields enumForId(short id) { - return _Fields.findByThriftIdOrThrow(id); - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - - public TBoolValue getBoolVal() { - if (getSetField() == _Fields.BOOL_VAL) { - return (TBoolValue)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'boolVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setBoolVal(TBoolValue value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.BOOL_VAL; - value_ = value; - } - - public TByteValue getByteVal() { - if (getSetField() == _Fields.BYTE_VAL) { - return (TByteValue)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'byteVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setByteVal(TByteValue value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.BYTE_VAL; - value_ = value; - } - - public TI16Value getI16Val() { - if (getSetField() == _Fields.I16_VAL) { - return (TI16Value)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'i16Val' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setI16Val(TI16Value value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.I16_VAL; - value_ = value; - } - - public TI32Value getI32Val() { - if (getSetField() == _Fields.I32_VAL) { - return (TI32Value)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'i32Val' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setI32Val(TI32Value value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.I32_VAL; - value_ = value; - } - - public TI64Value getI64Val() { - if (getSetField() == _Fields.I64_VAL) { - return (TI64Value)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'i64Val' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setI64Val(TI64Value value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.I64_VAL; - value_ = value; - } - - public TDoubleValue getDoubleVal() { - if (getSetField() == _Fields.DOUBLE_VAL) { - return (TDoubleValue)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'doubleVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setDoubleVal(TDoubleValue value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.DOUBLE_VAL; - value_ = value; - } - - public TStringValue getStringVal() { - if (getSetField() == _Fields.STRING_VAL) { - return (TStringValue)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'stringVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setStringVal(TStringValue value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.STRING_VAL; - value_ = value; - } - - public boolean isSetBoolVal() { - return setField_ == _Fields.BOOL_VAL; - } - - - public boolean isSetByteVal() { - return setField_ == _Fields.BYTE_VAL; - } - - - public boolean isSetI16Val() { - return setField_ == _Fields.I16_VAL; - } - - - public boolean isSetI32Val() { - return setField_ == _Fields.I32_VAL; - } - - - public boolean isSetI64Val() { - return setField_ == _Fields.I64_VAL; - } - - - public boolean isSetDoubleVal() { - return setField_ == _Fields.DOUBLE_VAL; - } - - - public boolean isSetStringVal() { - return setField_ == _Fields.STRING_VAL; - } - - - public boolean equals(Object other) { - if (other instanceof TColumnValue) { - return equals((TColumnValue)other); - } else { - return false; - } - } - - public boolean equals(TColumnValue other) { - return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); - } - - @Override - public int compareTo(TColumnValue other) { - int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); - if (lastComparison == 0) { - return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); - } - return lastComparison; - } - - - @Override - public int hashCode() { - HashCodeBuilder hcb = new HashCodeBuilder(); - hcb.append(this.getClass().getName()); - org.apache.thrift.TFieldIdEnum setField = getSetField(); - if (setField != null) { - hcb.append(setField.getThriftFieldId()); - Object value = getFieldValue(); - if (value instanceof org.apache.thrift.TEnum) { - hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue()); - } else { - hcb.append(value); - } - } - return hcb.toHashCode(); - } - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java deleted file mode 100644 index 4fc54544c1bea..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java +++ /dev/null @@ -1,548 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TDoubleColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDoubleColumn"); - - private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); - private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TDoubleColumnStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TDoubleColumnTupleSchemeFactory()); - } - - private List values; // required - private ByteBuffer nulls; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUES((short)1, "values"), - NULLS((short)2, "nulls"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUES - return VALUES; - case 2: // NULLS - return NULLS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)))); - tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TDoubleColumn.class, metaDataMap); - } - - public TDoubleColumn() { - } - - public TDoubleColumn( - List values, - ByteBuffer nulls) - { - this(); - this.values = values; - this.nulls = nulls; - } - - /** - * Performs a deep copy on other. - */ - public TDoubleColumn(TDoubleColumn other) { - if (other.isSetValues()) { - List __this__values = new ArrayList(); - for (Double other_element : other.values) { - __this__values.add(other_element); - } - this.values = __this__values; - } - if (other.isSetNulls()) { - this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); -; - } - } - - public TDoubleColumn deepCopy() { - return new TDoubleColumn(this); - } - - @Override - public void clear() { - this.values = null; - this.nulls = null; - } - - public int getValuesSize() { - return (this.values == null) ? 0 : this.values.size(); - } - - public java.util.Iterator getValuesIterator() { - return (this.values == null) ? null : this.values.iterator(); - } - - public void addToValues(double elem) { - if (this.values == null) { - this.values = new ArrayList(); - } - this.values.add(elem); - } - - public List getValues() { - return this.values; - } - - public void setValues(List values) { - this.values = values; - } - - public void unsetValues() { - this.values = null; - } - - /** Returns true if field values is set (has been assigned a value) and false otherwise */ - public boolean isSetValues() { - return this.values != null; - } - - public void setValuesIsSet(boolean value) { - if (!value) { - this.values = null; - } - } - - public byte[] getNulls() { - setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); - return nulls == null ? null : nulls.array(); - } - - public ByteBuffer bufferForNulls() { - return nulls; - } - - public void setNulls(byte[] nulls) { - setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); - } - - public void setNulls(ByteBuffer nulls) { - this.nulls = nulls; - } - - public void unsetNulls() { - this.nulls = null; - } - - /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ - public boolean isSetNulls() { - return this.nulls != null; - } - - public void setNullsIsSet(boolean value) { - if (!value) { - this.nulls = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUES: - if (value == null) { - unsetValues(); - } else { - setValues((List)value); - } - break; - - case NULLS: - if (value == null) { - unsetNulls(); - } else { - setNulls((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUES: - return getValues(); - - case NULLS: - return getNulls(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUES: - return isSetValues(); - case NULLS: - return isSetNulls(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TDoubleColumn) - return this.equals((TDoubleColumn)that); - return false; - } - - public boolean equals(TDoubleColumn that) { - if (that == null) - return false; - - boolean this_present_values = true && this.isSetValues(); - boolean that_present_values = true && that.isSetValues(); - if (this_present_values || that_present_values) { - if (!(this_present_values && that_present_values)) - return false; - if (!this.values.equals(that.values)) - return false; - } - - boolean this_present_nulls = true && this.isSetNulls(); - boolean that_present_nulls = true && that.isSetNulls(); - if (this_present_nulls || that_present_nulls) { - if (!(this_present_nulls && that_present_nulls)) - return false; - if (!this.nulls.equals(that.nulls)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_values = true && (isSetValues()); - builder.append(present_values); - if (present_values) - builder.append(values); - - boolean present_nulls = true && (isSetNulls()); - builder.append(present_nulls); - if (present_nulls) - builder.append(nulls); - - return builder.toHashCode(); - } - - public int compareTo(TDoubleColumn other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TDoubleColumn typedOther = (TDoubleColumn)other; - - lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNulls()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TDoubleColumn("); - boolean first = true; - - sb.append("values:"); - if (this.values == null) { - sb.append("null"); - } else { - sb.append(this.values); - } - first = false; - if (!first) sb.append(", "); - sb.append("nulls:"); - if (this.nulls == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.nulls, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetValues()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); - } - - if (!isSetNulls()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TDoubleColumnStandardSchemeFactory implements SchemeFactory { - public TDoubleColumnStandardScheme getScheme() { - return new TDoubleColumnStandardScheme(); - } - } - - private static class TDoubleColumnStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TDoubleColumn struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list94 = iprot.readListBegin(); - struct.values = new ArrayList(_list94.size); - for (int _i95 = 0; _i95 < _list94.size; ++_i95) - { - double _elem96; // optional - _elem96 = iprot.readDouble(); - struct.values.add(_elem96); - } - iprot.readListEnd(); - } - struct.setValuesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // NULLS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TDoubleColumn struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.values != null) { - oprot.writeFieldBegin(VALUES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.DOUBLE, struct.values.size())); - for (double _iter97 : struct.values) - { - oprot.writeDouble(_iter97); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.nulls != null) { - oprot.writeFieldBegin(NULLS_FIELD_DESC); - oprot.writeBinary(struct.nulls); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TDoubleColumnTupleSchemeFactory implements SchemeFactory { - public TDoubleColumnTupleScheme getScheme() { - return new TDoubleColumnTupleScheme(); - } - } - - private static class TDoubleColumnTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TDoubleColumn struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.values.size()); - for (double _iter98 : struct.values) - { - oprot.writeDouble(_iter98); - } - } - oprot.writeBinary(struct.nulls); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TDoubleColumn struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list99 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32()); - struct.values = new ArrayList(_list99.size); - for (int _i100 = 0; _i100 < _list99.size; ++_i100) - { - double _elem101; // optional - _elem101 = iprot.readDouble(); - struct.values.add(_elem101); - } - } - struct.setValuesIsSet(true); - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java deleted file mode 100644 index d21573633ef51..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java +++ /dev/null @@ -1,386 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TDoubleValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDoubleValue"); - - private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.DOUBLE, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TDoubleValueStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TDoubleValueTupleSchemeFactory()); - } - - private double value; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUE((short)1, "value"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUE - return VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __VALUE_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.VALUE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TDoubleValue.class, metaDataMap); - } - - public TDoubleValue() { - } - - /** - * Performs a deep copy on other. - */ - public TDoubleValue(TDoubleValue other) { - __isset_bitfield = other.__isset_bitfield; - this.value = other.value; - } - - public TDoubleValue deepCopy() { - return new TDoubleValue(this); - } - - @Override - public void clear() { - setValueIsSet(false); - this.value = 0.0; - } - - public double getValue() { - return this.value; - } - - public void setValue(double value) { - this.value = value; - setValueIsSet(true); - } - - public void unsetValue() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - /** Returns true if field value is set (has been assigned a value) and false otherwise */ - public boolean isSetValue() { - return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - public void setValueIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUE: - if (value == null) { - unsetValue(); - } else { - setValue((Double)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUE: - return Double.valueOf(getValue()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUE: - return isSetValue(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TDoubleValue) - return this.equals((TDoubleValue)that); - return false; - } - - public boolean equals(TDoubleValue that) { - if (that == null) - return false; - - boolean this_present_value = true && this.isSetValue(); - boolean that_present_value = true && that.isSetValue(); - if (this_present_value || that_present_value) { - if (!(this_present_value && that_present_value)) - return false; - if (this.value != that.value) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_value = true && (isSetValue()); - builder.append(present_value); - if (present_value) - builder.append(value); - - return builder.toHashCode(); - } - - public int compareTo(TDoubleValue other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TDoubleValue typedOther = (TDoubleValue)other; - - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TDoubleValue("); - boolean first = true; - - if (isSetValue()) { - sb.append("value:"); - sb.append(this.value); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TDoubleValueStandardSchemeFactory implements SchemeFactory { - public TDoubleValueStandardScheme getScheme() { - return new TDoubleValueStandardScheme(); - } - } - - private static class TDoubleValueStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TDoubleValue struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUE - if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { - struct.value = iprot.readDouble(); - struct.setValueIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TDoubleValue struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetValue()) { - oprot.writeFieldBegin(VALUE_FIELD_DESC); - oprot.writeDouble(struct.value); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TDoubleValueTupleSchemeFactory implements SchemeFactory { - public TDoubleValueTupleScheme getScheme() { - return new TDoubleValueTupleScheme(); - } - } - - private static class TDoubleValueTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TDoubleValue struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetValue()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetValue()) { - oprot.writeDouble(struct.value); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TDoubleValue struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.value = iprot.readDouble(); - struct.setValueIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java deleted file mode 100644 index 4f157ad5a6450..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java +++ /dev/null @@ -1,769 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TExecuteStatementReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TExecuteStatementReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField STATEMENT_FIELD_DESC = new org.apache.thrift.protocol.TField("statement", org.apache.thrift.protocol.TType.STRING, (short)2); - private static final org.apache.thrift.protocol.TField CONF_OVERLAY_FIELD_DESC = new org.apache.thrift.protocol.TField("confOverlay", org.apache.thrift.protocol.TType.MAP, (short)3); - private static final org.apache.thrift.protocol.TField RUN_ASYNC_FIELD_DESC = new org.apache.thrift.protocol.TField("runAsync", org.apache.thrift.protocol.TType.BOOL, (short)4); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TExecuteStatementReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TExecuteStatementReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private String statement; // required - private Map confOverlay; // optional - private boolean runAsync; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - STATEMENT((short)2, "statement"), - CONF_OVERLAY((short)3, "confOverlay"), - RUN_ASYNC((short)4, "runAsync"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // STATEMENT - return STATEMENT; - case 3: // CONF_OVERLAY - return CONF_OVERLAY; - case 4: // RUN_ASYNC - return RUN_ASYNC; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __RUNASYNC_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.CONF_OVERLAY,_Fields.RUN_ASYNC}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.STATEMENT, new org.apache.thrift.meta_data.FieldMetaData("statement", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.CONF_OVERLAY, new org.apache.thrift.meta_data.FieldMetaData("confOverlay", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); - tmpMap.put(_Fields.RUN_ASYNC, new org.apache.thrift.meta_data.FieldMetaData("runAsync", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TExecuteStatementReq.class, metaDataMap); - } - - public TExecuteStatementReq() { - this.runAsync = false; - - } - - public TExecuteStatementReq( - TSessionHandle sessionHandle, - String statement) - { - this(); - this.sessionHandle = sessionHandle; - this.statement = statement; - } - - /** - * Performs a deep copy on other. - */ - public TExecuteStatementReq(TExecuteStatementReq other) { - __isset_bitfield = other.__isset_bitfield; - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetStatement()) { - this.statement = other.statement; - } - if (other.isSetConfOverlay()) { - Map __this__confOverlay = new HashMap(); - for (Map.Entry other_element : other.confOverlay.entrySet()) { - - String other_element_key = other_element.getKey(); - String other_element_value = other_element.getValue(); - - String __this__confOverlay_copy_key = other_element_key; - - String __this__confOverlay_copy_value = other_element_value; - - __this__confOverlay.put(__this__confOverlay_copy_key, __this__confOverlay_copy_value); - } - this.confOverlay = __this__confOverlay; - } - this.runAsync = other.runAsync; - } - - public TExecuteStatementReq deepCopy() { - return new TExecuteStatementReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.statement = null; - this.confOverlay = null; - this.runAsync = false; - - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public String getStatement() { - return this.statement; - } - - public void setStatement(String statement) { - this.statement = statement; - } - - public void unsetStatement() { - this.statement = null; - } - - /** Returns true if field statement is set (has been assigned a value) and false otherwise */ - public boolean isSetStatement() { - return this.statement != null; - } - - public void setStatementIsSet(boolean value) { - if (!value) { - this.statement = null; - } - } - - public int getConfOverlaySize() { - return (this.confOverlay == null) ? 0 : this.confOverlay.size(); - } - - public void putToConfOverlay(String key, String val) { - if (this.confOverlay == null) { - this.confOverlay = new HashMap(); - } - this.confOverlay.put(key, val); - } - - public Map getConfOverlay() { - return this.confOverlay; - } - - public void setConfOverlay(Map confOverlay) { - this.confOverlay = confOverlay; - } - - public void unsetConfOverlay() { - this.confOverlay = null; - } - - /** Returns true if field confOverlay is set (has been assigned a value) and false otherwise */ - public boolean isSetConfOverlay() { - return this.confOverlay != null; - } - - public void setConfOverlayIsSet(boolean value) { - if (!value) { - this.confOverlay = null; - } - } - - public boolean isRunAsync() { - return this.runAsync; - } - - public void setRunAsync(boolean runAsync) { - this.runAsync = runAsync; - setRunAsyncIsSet(true); - } - - public void unsetRunAsync() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RUNASYNC_ISSET_ID); - } - - /** Returns true if field runAsync is set (has been assigned a value) and false otherwise */ - public boolean isSetRunAsync() { - return EncodingUtils.testBit(__isset_bitfield, __RUNASYNC_ISSET_ID); - } - - public void setRunAsyncIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RUNASYNC_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case STATEMENT: - if (value == null) { - unsetStatement(); - } else { - setStatement((String)value); - } - break; - - case CONF_OVERLAY: - if (value == null) { - unsetConfOverlay(); - } else { - setConfOverlay((Map)value); - } - break; - - case RUN_ASYNC: - if (value == null) { - unsetRunAsync(); - } else { - setRunAsync((Boolean)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case STATEMENT: - return getStatement(); - - case CONF_OVERLAY: - return getConfOverlay(); - - case RUN_ASYNC: - return Boolean.valueOf(isRunAsync()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case STATEMENT: - return isSetStatement(); - case CONF_OVERLAY: - return isSetConfOverlay(); - case RUN_ASYNC: - return isSetRunAsync(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TExecuteStatementReq) - return this.equals((TExecuteStatementReq)that); - return false; - } - - public boolean equals(TExecuteStatementReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_statement = true && this.isSetStatement(); - boolean that_present_statement = true && that.isSetStatement(); - if (this_present_statement || that_present_statement) { - if (!(this_present_statement && that_present_statement)) - return false; - if (!this.statement.equals(that.statement)) - return false; - } - - boolean this_present_confOverlay = true && this.isSetConfOverlay(); - boolean that_present_confOverlay = true && that.isSetConfOverlay(); - if (this_present_confOverlay || that_present_confOverlay) { - if (!(this_present_confOverlay && that_present_confOverlay)) - return false; - if (!this.confOverlay.equals(that.confOverlay)) - return false; - } - - boolean this_present_runAsync = true && this.isSetRunAsync(); - boolean that_present_runAsync = true && that.isSetRunAsync(); - if (this_present_runAsync || that_present_runAsync) { - if (!(this_present_runAsync && that_present_runAsync)) - return false; - if (this.runAsync != that.runAsync) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_statement = true && (isSetStatement()); - builder.append(present_statement); - if (present_statement) - builder.append(statement); - - boolean present_confOverlay = true && (isSetConfOverlay()); - builder.append(present_confOverlay); - if (present_confOverlay) - builder.append(confOverlay); - - boolean present_runAsync = true && (isSetRunAsync()); - builder.append(present_runAsync); - if (present_runAsync) - builder.append(runAsync); - - return builder.toHashCode(); - } - - public int compareTo(TExecuteStatementReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TExecuteStatementReq typedOther = (TExecuteStatementReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetStatement()).compareTo(typedOther.isSetStatement()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatement()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.statement, typedOther.statement); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetConfOverlay()).compareTo(typedOther.isSetConfOverlay()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetConfOverlay()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.confOverlay, typedOther.confOverlay); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetRunAsync()).compareTo(typedOther.isSetRunAsync()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetRunAsync()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.runAsync, typedOther.runAsync); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TExecuteStatementReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (!first) sb.append(", "); - sb.append("statement:"); - if (this.statement == null) { - sb.append("null"); - } else { - sb.append(this.statement); - } - first = false; - if (isSetConfOverlay()) { - if (!first) sb.append(", "); - sb.append("confOverlay:"); - if (this.confOverlay == null) { - sb.append("null"); - } else { - sb.append(this.confOverlay); - } - first = false; - } - if (isSetRunAsync()) { - if (!first) sb.append(", "); - sb.append("runAsync:"); - sb.append(this.runAsync); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - if (!isSetStatement()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'statement' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TExecuteStatementReqStandardSchemeFactory implements SchemeFactory { - public TExecuteStatementReqStandardScheme getScheme() { - return new TExecuteStatementReqStandardScheme(); - } - } - - private static class TExecuteStatementReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TExecuteStatementReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // STATEMENT - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.statement = iprot.readString(); - struct.setStatementIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // CONF_OVERLAY - if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { - { - org.apache.thrift.protocol.TMap _map162 = iprot.readMapBegin(); - struct.confOverlay = new HashMap(2*_map162.size); - for (int _i163 = 0; _i163 < _map162.size; ++_i163) - { - String _key164; // required - String _val165; // required - _key164 = iprot.readString(); - _val165 = iprot.readString(); - struct.confOverlay.put(_key164, _val165); - } - iprot.readMapEnd(); - } - struct.setConfOverlayIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // RUN_ASYNC - if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { - struct.runAsync = iprot.readBool(); - struct.setRunAsyncIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TExecuteStatementReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.statement != null) { - oprot.writeFieldBegin(STATEMENT_FIELD_DESC); - oprot.writeString(struct.statement); - oprot.writeFieldEnd(); - } - if (struct.confOverlay != null) { - if (struct.isSetConfOverlay()) { - oprot.writeFieldBegin(CONF_OVERLAY_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.confOverlay.size())); - for (Map.Entry _iter166 : struct.confOverlay.entrySet()) - { - oprot.writeString(_iter166.getKey()); - oprot.writeString(_iter166.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - } - if (struct.isSetRunAsync()) { - oprot.writeFieldBegin(RUN_ASYNC_FIELD_DESC); - oprot.writeBool(struct.runAsync); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TExecuteStatementReqTupleSchemeFactory implements SchemeFactory { - public TExecuteStatementReqTupleScheme getScheme() { - return new TExecuteStatementReqTupleScheme(); - } - } - - private static class TExecuteStatementReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - oprot.writeString(struct.statement); - BitSet optionals = new BitSet(); - if (struct.isSetConfOverlay()) { - optionals.set(0); - } - if (struct.isSetRunAsync()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); - if (struct.isSetConfOverlay()) { - { - oprot.writeI32(struct.confOverlay.size()); - for (Map.Entry _iter167 : struct.confOverlay.entrySet()) - { - oprot.writeString(_iter167.getKey()); - oprot.writeString(_iter167.getValue()); - } - } - } - if (struct.isSetRunAsync()) { - oprot.writeBool(struct.runAsync); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - struct.statement = iprot.readString(); - struct.setStatementIsSet(true); - BitSet incoming = iprot.readBitSet(2); - if (incoming.get(0)) { - { - org.apache.thrift.protocol.TMap _map168 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.confOverlay = new HashMap(2*_map168.size); - for (int _i169 = 0; _i169 < _map168.size; ++_i169) - { - String _key170; // required - String _val171; // required - _key170 = iprot.readString(); - _val171 = iprot.readString(); - struct.confOverlay.put(_key170, _val171); - } - } - struct.setConfOverlayIsSet(true); - } - if (incoming.get(1)) { - struct.runAsync = iprot.readBool(); - struct.setRunAsyncIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java deleted file mode 100644 index fdde51e70f783..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TExecuteStatementResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TExecuteStatementResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TExecuteStatementRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TExecuteStatementRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationHandle operationHandle; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - OPERATION_HANDLE((short)2, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TExecuteStatementResp.class, metaDataMap); - } - - public TExecuteStatementResp() { - } - - public TExecuteStatementResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TExecuteStatementResp(TExecuteStatementResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TExecuteStatementResp deepCopy() { - return new TExecuteStatementResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationHandle = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TExecuteStatementResp) - return this.equals((TExecuteStatementResp)that); - return false; - } - - public boolean equals(TExecuteStatementResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TExecuteStatementResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TExecuteStatementResp typedOther = (TExecuteStatementResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TExecuteStatementResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationHandle()) { - if (!first) sb.append(", "); - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TExecuteStatementRespStandardSchemeFactory implements SchemeFactory { - public TExecuteStatementRespStandardScheme getScheme() { - return new TExecuteStatementRespStandardScheme(); - } - } - - private static class TExecuteStatementRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TExecuteStatementResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TExecuteStatementResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationHandle != null) { - if (struct.isSetOperationHandle()) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TExecuteStatementRespTupleSchemeFactory implements SchemeFactory { - public TExecuteStatementRespTupleScheme getScheme() { - return new TExecuteStatementRespTupleScheme(); - } - } - - private static class TExecuteStatementRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationHandle()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetOperationHandle()) { - struct.operationHandle.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java deleted file mode 100644 index b2a22effd91af..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java +++ /dev/null @@ -1,57 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - - -import java.util.Map; -import java.util.HashMap; -import org.apache.thrift.TEnum; - -public enum TFetchOrientation implements org.apache.thrift.TEnum { - FETCH_NEXT(0), - FETCH_PRIOR(1), - FETCH_RELATIVE(2), - FETCH_ABSOLUTE(3), - FETCH_FIRST(4), - FETCH_LAST(5); - - private final int value; - - private TFetchOrientation(int value) { - this.value = value; - } - - /** - * Get the integer value of this enum value, as defined in the Thrift IDL. - */ - public int getValue() { - return value; - } - - /** - * Find a the enum type by its integer value, as defined in the Thrift IDL. - * @return null if the value is not found. - */ - public static TFetchOrientation findByValue(int value) { - switch (value) { - case 0: - return FETCH_NEXT; - case 1: - return FETCH_PRIOR; - case 2: - return FETCH_RELATIVE; - case 3: - return FETCH_ABSOLUTE; - case 4: - return FETCH_FIRST; - case 5: - return FETCH_LAST; - default: - return null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java deleted file mode 100644 index 068711fc44440..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java +++ /dev/null @@ -1,710 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TFetchResultsReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TFetchResultsReq"); - - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField ORIENTATION_FIELD_DESC = new org.apache.thrift.protocol.TField("orientation", org.apache.thrift.protocol.TType.I32, (short)2); - private static final org.apache.thrift.protocol.TField MAX_ROWS_FIELD_DESC = new org.apache.thrift.protocol.TField("maxRows", org.apache.thrift.protocol.TType.I64, (short)3); - private static final org.apache.thrift.protocol.TField FETCH_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("fetchType", org.apache.thrift.protocol.TType.I16, (short)4); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TFetchResultsReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TFetchResultsReqTupleSchemeFactory()); - } - - private TOperationHandle operationHandle; // required - private TFetchOrientation orientation; // required - private long maxRows; // required - private short fetchType; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - OPERATION_HANDLE((short)1, "operationHandle"), - /** - * - * @see TFetchOrientation - */ - ORIENTATION((short)2, "orientation"), - MAX_ROWS((short)3, "maxRows"), - FETCH_TYPE((short)4, "fetchType"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // OPERATION_HANDLE - return OPERATION_HANDLE; - case 2: // ORIENTATION - return ORIENTATION; - case 3: // MAX_ROWS - return MAX_ROWS; - case 4: // FETCH_TYPE - return FETCH_TYPE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __MAXROWS_ISSET_ID = 0; - private static final int __FETCHTYPE_ISSET_ID = 1; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.FETCH_TYPE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - tmpMap.put(_Fields.ORIENTATION, new org.apache.thrift.meta_data.FieldMetaData("orientation", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TFetchOrientation.class))); - tmpMap.put(_Fields.MAX_ROWS, new org.apache.thrift.meta_data.FieldMetaData("maxRows", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - tmpMap.put(_Fields.FETCH_TYPE, new org.apache.thrift.meta_data.FieldMetaData("fetchType", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TFetchResultsReq.class, metaDataMap); - } - - public TFetchResultsReq() { - this.orientation = org.apache.hive.service.cli.thrift.TFetchOrientation.FETCH_NEXT; - - this.fetchType = (short)0; - - } - - public TFetchResultsReq( - TOperationHandle operationHandle, - TFetchOrientation orientation, - long maxRows) - { - this(); - this.operationHandle = operationHandle; - this.orientation = orientation; - this.maxRows = maxRows; - setMaxRowsIsSet(true); - } - - /** - * Performs a deep copy on other. - */ - public TFetchResultsReq(TFetchResultsReq other) { - __isset_bitfield = other.__isset_bitfield; - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - if (other.isSetOrientation()) { - this.orientation = other.orientation; - } - this.maxRows = other.maxRows; - this.fetchType = other.fetchType; - } - - public TFetchResultsReq deepCopy() { - return new TFetchResultsReq(this); - } - - @Override - public void clear() { - this.operationHandle = null; - this.orientation = org.apache.hive.service.cli.thrift.TFetchOrientation.FETCH_NEXT; - - setMaxRowsIsSet(false); - this.maxRows = 0; - this.fetchType = (short)0; - - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - /** - * - * @see TFetchOrientation - */ - public TFetchOrientation getOrientation() { - return this.orientation; - } - - /** - * - * @see TFetchOrientation - */ - public void setOrientation(TFetchOrientation orientation) { - this.orientation = orientation; - } - - public void unsetOrientation() { - this.orientation = null; - } - - /** Returns true if field orientation is set (has been assigned a value) and false otherwise */ - public boolean isSetOrientation() { - return this.orientation != null; - } - - public void setOrientationIsSet(boolean value) { - if (!value) { - this.orientation = null; - } - } - - public long getMaxRows() { - return this.maxRows; - } - - public void setMaxRows(long maxRows) { - this.maxRows = maxRows; - setMaxRowsIsSet(true); - } - - public void unsetMaxRows() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAXROWS_ISSET_ID); - } - - /** Returns true if field maxRows is set (has been assigned a value) and false otherwise */ - public boolean isSetMaxRows() { - return EncodingUtils.testBit(__isset_bitfield, __MAXROWS_ISSET_ID); - } - - public void setMaxRowsIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAXROWS_ISSET_ID, value); - } - - public short getFetchType() { - return this.fetchType; - } - - public void setFetchType(short fetchType) { - this.fetchType = fetchType; - setFetchTypeIsSet(true); - } - - public void unsetFetchType() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __FETCHTYPE_ISSET_ID); - } - - /** Returns true if field fetchType is set (has been assigned a value) and false otherwise */ - public boolean isSetFetchType() { - return EncodingUtils.testBit(__isset_bitfield, __FETCHTYPE_ISSET_ID); - } - - public void setFetchTypeIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __FETCHTYPE_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - case ORIENTATION: - if (value == null) { - unsetOrientation(); - } else { - setOrientation((TFetchOrientation)value); - } - break; - - case MAX_ROWS: - if (value == null) { - unsetMaxRows(); - } else { - setMaxRows((Long)value); - } - break; - - case FETCH_TYPE: - if (value == null) { - unsetFetchType(); - } else { - setFetchType((Short)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case OPERATION_HANDLE: - return getOperationHandle(); - - case ORIENTATION: - return getOrientation(); - - case MAX_ROWS: - return Long.valueOf(getMaxRows()); - - case FETCH_TYPE: - return Short.valueOf(getFetchType()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case OPERATION_HANDLE: - return isSetOperationHandle(); - case ORIENTATION: - return isSetOrientation(); - case MAX_ROWS: - return isSetMaxRows(); - case FETCH_TYPE: - return isSetFetchType(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TFetchResultsReq) - return this.equals((TFetchResultsReq)that); - return false; - } - - public boolean equals(TFetchResultsReq that) { - if (that == null) - return false; - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - boolean this_present_orientation = true && this.isSetOrientation(); - boolean that_present_orientation = true && that.isSetOrientation(); - if (this_present_orientation || that_present_orientation) { - if (!(this_present_orientation && that_present_orientation)) - return false; - if (!this.orientation.equals(that.orientation)) - return false; - } - - boolean this_present_maxRows = true; - boolean that_present_maxRows = true; - if (this_present_maxRows || that_present_maxRows) { - if (!(this_present_maxRows && that_present_maxRows)) - return false; - if (this.maxRows != that.maxRows) - return false; - } - - boolean this_present_fetchType = true && this.isSetFetchType(); - boolean that_present_fetchType = true && that.isSetFetchType(); - if (this_present_fetchType || that_present_fetchType) { - if (!(this_present_fetchType && that_present_fetchType)) - return false; - if (this.fetchType != that.fetchType) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - boolean present_orientation = true && (isSetOrientation()); - builder.append(present_orientation); - if (present_orientation) - builder.append(orientation.getValue()); - - boolean present_maxRows = true; - builder.append(present_maxRows); - if (present_maxRows) - builder.append(maxRows); - - boolean present_fetchType = true && (isSetFetchType()); - builder.append(present_fetchType); - if (present_fetchType) - builder.append(fetchType); - - return builder.toHashCode(); - } - - public int compareTo(TFetchResultsReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TFetchResultsReq typedOther = (TFetchResultsReq)other; - - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOrientation()).compareTo(typedOther.isSetOrientation()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOrientation()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.orientation, typedOther.orientation); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetMaxRows()).compareTo(typedOther.isSetMaxRows()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetMaxRows()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maxRows, typedOther.maxRows); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetFetchType()).compareTo(typedOther.isSetFetchType()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetFetchType()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fetchType, typedOther.fetchType); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TFetchResultsReq("); - boolean first = true; - - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - if (!first) sb.append(", "); - sb.append("orientation:"); - if (this.orientation == null) { - sb.append("null"); - } else { - sb.append(this.orientation); - } - first = false; - if (!first) sb.append(", "); - sb.append("maxRows:"); - sb.append(this.maxRows); - first = false; - if (isSetFetchType()) { - if (!first) sb.append(", "); - sb.append("fetchType:"); - sb.append(this.fetchType); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetOperationHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); - } - - if (!isSetOrientation()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'orientation' is unset! Struct:" + toString()); - } - - if (!isSetMaxRows()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'maxRows' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TFetchResultsReqStandardSchemeFactory implements SchemeFactory { - public TFetchResultsReqStandardScheme getScheme() { - return new TFetchResultsReqStandardScheme(); - } - } - - private static class TFetchResultsReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TFetchResultsReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // ORIENTATION - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.orientation = TFetchOrientation.findByValue(iprot.readI32()); - struct.setOrientationIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // MAX_ROWS - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.maxRows = iprot.readI64(); - struct.setMaxRowsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // FETCH_TYPE - if (schemeField.type == org.apache.thrift.protocol.TType.I16) { - struct.fetchType = iprot.readI16(); - struct.setFetchTypeIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TFetchResultsReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.operationHandle != null) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.orientation != null) { - oprot.writeFieldBegin(ORIENTATION_FIELD_DESC); - oprot.writeI32(struct.orientation.getValue()); - oprot.writeFieldEnd(); - } - oprot.writeFieldBegin(MAX_ROWS_FIELD_DESC); - oprot.writeI64(struct.maxRows); - oprot.writeFieldEnd(); - if (struct.isSetFetchType()) { - oprot.writeFieldBegin(FETCH_TYPE_FIELD_DESC); - oprot.writeI16(struct.fetchType); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TFetchResultsReqTupleSchemeFactory implements SchemeFactory { - public TFetchResultsReqTupleScheme getScheme() { - return new TFetchResultsReqTupleScheme(); - } - } - - private static class TFetchResultsReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TFetchResultsReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.operationHandle.write(oprot); - oprot.writeI32(struct.orientation.getValue()); - oprot.writeI64(struct.maxRows); - BitSet optionals = new BitSet(); - if (struct.isSetFetchType()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetFetchType()) { - oprot.writeI16(struct.fetchType); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TFetchResultsReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - struct.orientation = TFetchOrientation.findByValue(iprot.readI32()); - struct.setOrientationIsSet(true); - struct.maxRows = iprot.readI64(); - struct.setMaxRowsIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.fetchType = iprot.readI16(); - struct.setFetchTypeIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java deleted file mode 100644 index 19991f1da3eb3..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java +++ /dev/null @@ -1,608 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TFetchResultsResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TFetchResultsResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField HAS_MORE_ROWS_FIELD_DESC = new org.apache.thrift.protocol.TField("hasMoreRows", org.apache.thrift.protocol.TType.BOOL, (short)2); - private static final org.apache.thrift.protocol.TField RESULTS_FIELD_DESC = new org.apache.thrift.protocol.TField("results", org.apache.thrift.protocol.TType.STRUCT, (short)3); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TFetchResultsRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TFetchResultsRespTupleSchemeFactory()); - } - - private TStatus status; // required - private boolean hasMoreRows; // optional - private TRowSet results; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - HAS_MORE_ROWS((short)2, "hasMoreRows"), - RESULTS((short)3, "results"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // HAS_MORE_ROWS - return HAS_MORE_ROWS; - case 3: // RESULTS - return RESULTS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __HASMOREROWS_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.HAS_MORE_ROWS,_Fields.RESULTS}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.HAS_MORE_ROWS, new org.apache.thrift.meta_data.FieldMetaData("hasMoreRows", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); - tmpMap.put(_Fields.RESULTS, new org.apache.thrift.meta_data.FieldMetaData("results", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRowSet.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TFetchResultsResp.class, metaDataMap); - } - - public TFetchResultsResp() { - } - - public TFetchResultsResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TFetchResultsResp(TFetchResultsResp other) { - __isset_bitfield = other.__isset_bitfield; - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - this.hasMoreRows = other.hasMoreRows; - if (other.isSetResults()) { - this.results = new TRowSet(other.results); - } - } - - public TFetchResultsResp deepCopy() { - return new TFetchResultsResp(this); - } - - @Override - public void clear() { - this.status = null; - setHasMoreRowsIsSet(false); - this.hasMoreRows = false; - this.results = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public boolean isHasMoreRows() { - return this.hasMoreRows; - } - - public void setHasMoreRows(boolean hasMoreRows) { - this.hasMoreRows = hasMoreRows; - setHasMoreRowsIsSet(true); - } - - public void unsetHasMoreRows() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __HASMOREROWS_ISSET_ID); - } - - /** Returns true if field hasMoreRows is set (has been assigned a value) and false otherwise */ - public boolean isSetHasMoreRows() { - return EncodingUtils.testBit(__isset_bitfield, __HASMOREROWS_ISSET_ID); - } - - public void setHasMoreRowsIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __HASMOREROWS_ISSET_ID, value); - } - - public TRowSet getResults() { - return this.results; - } - - public void setResults(TRowSet results) { - this.results = results; - } - - public void unsetResults() { - this.results = null; - } - - /** Returns true if field results is set (has been assigned a value) and false otherwise */ - public boolean isSetResults() { - return this.results != null; - } - - public void setResultsIsSet(boolean value) { - if (!value) { - this.results = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case HAS_MORE_ROWS: - if (value == null) { - unsetHasMoreRows(); - } else { - setHasMoreRows((Boolean)value); - } - break; - - case RESULTS: - if (value == null) { - unsetResults(); - } else { - setResults((TRowSet)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case HAS_MORE_ROWS: - return Boolean.valueOf(isHasMoreRows()); - - case RESULTS: - return getResults(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case HAS_MORE_ROWS: - return isSetHasMoreRows(); - case RESULTS: - return isSetResults(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TFetchResultsResp) - return this.equals((TFetchResultsResp)that); - return false; - } - - public boolean equals(TFetchResultsResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_hasMoreRows = true && this.isSetHasMoreRows(); - boolean that_present_hasMoreRows = true && that.isSetHasMoreRows(); - if (this_present_hasMoreRows || that_present_hasMoreRows) { - if (!(this_present_hasMoreRows && that_present_hasMoreRows)) - return false; - if (this.hasMoreRows != that.hasMoreRows) - return false; - } - - boolean this_present_results = true && this.isSetResults(); - boolean that_present_results = true && that.isSetResults(); - if (this_present_results || that_present_results) { - if (!(this_present_results && that_present_results)) - return false; - if (!this.results.equals(that.results)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_hasMoreRows = true && (isSetHasMoreRows()); - builder.append(present_hasMoreRows); - if (present_hasMoreRows) - builder.append(hasMoreRows); - - boolean present_results = true && (isSetResults()); - builder.append(present_results); - if (present_results) - builder.append(results); - - return builder.toHashCode(); - } - - public int compareTo(TFetchResultsResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TFetchResultsResp typedOther = (TFetchResultsResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetHasMoreRows()).compareTo(typedOther.isSetHasMoreRows()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetHasMoreRows()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hasMoreRows, typedOther.hasMoreRows); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetResults()).compareTo(typedOther.isSetResults()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetResults()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.results, typedOther.results); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TFetchResultsResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetHasMoreRows()) { - if (!first) sb.append(", "); - sb.append("hasMoreRows:"); - sb.append(this.hasMoreRows); - first = false; - } - if (isSetResults()) { - if (!first) sb.append(", "); - sb.append("results:"); - if (this.results == null) { - sb.append("null"); - } else { - sb.append(this.results); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (results != null) { - results.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TFetchResultsRespStandardSchemeFactory implements SchemeFactory { - public TFetchResultsRespStandardScheme getScheme() { - return new TFetchResultsRespStandardScheme(); - } - } - - private static class TFetchResultsRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TFetchResultsResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // HAS_MORE_ROWS - if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { - struct.hasMoreRows = iprot.readBool(); - struct.setHasMoreRowsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // RESULTS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.results = new TRowSet(); - struct.results.read(iprot); - struct.setResultsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TFetchResultsResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.isSetHasMoreRows()) { - oprot.writeFieldBegin(HAS_MORE_ROWS_FIELD_DESC); - oprot.writeBool(struct.hasMoreRows); - oprot.writeFieldEnd(); - } - if (struct.results != null) { - if (struct.isSetResults()) { - oprot.writeFieldBegin(RESULTS_FIELD_DESC); - struct.results.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TFetchResultsRespTupleSchemeFactory implements SchemeFactory { - public TFetchResultsRespTupleScheme getScheme() { - return new TFetchResultsRespTupleScheme(); - } - } - - private static class TFetchResultsRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TFetchResultsResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetHasMoreRows()) { - optionals.set(0); - } - if (struct.isSetResults()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); - if (struct.isSetHasMoreRows()) { - oprot.writeBool(struct.hasMoreRows); - } - if (struct.isSetResults()) { - struct.results.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TFetchResultsResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(2); - if (incoming.get(0)) { - struct.hasMoreRows = iprot.readBool(); - struct.setHasMoreRowsIsSet(true); - } - if (incoming.get(1)) { - struct.results = new TRowSet(); - struct.results.read(iprot); - struct.setResultsIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java deleted file mode 100644 index cfd157f701b26..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetCatalogsReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetCatalogsReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetCatalogsReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetCatalogsReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetCatalogsReq.class, metaDataMap); - } - - public TGetCatalogsReq() { - } - - public TGetCatalogsReq( - TSessionHandle sessionHandle) - { - this(); - this.sessionHandle = sessionHandle; - } - - /** - * Performs a deep copy on other. - */ - public TGetCatalogsReq(TGetCatalogsReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - } - - public TGetCatalogsReq deepCopy() { - return new TGetCatalogsReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetCatalogsReq) - return this.equals((TGetCatalogsReq)that); - return false; - } - - public boolean equals(TGetCatalogsReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetCatalogsReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetCatalogsReq typedOther = (TGetCatalogsReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetCatalogsReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetCatalogsReqStandardSchemeFactory implements SchemeFactory { - public TGetCatalogsReqStandardScheme getScheme() { - return new TGetCatalogsReqStandardScheme(); - } - } - - private static class TGetCatalogsReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetCatalogsReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetCatalogsReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetCatalogsReqTupleSchemeFactory implements SchemeFactory { - public TGetCatalogsReqTupleScheme getScheme() { - return new TGetCatalogsReqTupleScheme(); - } - } - - private static class TGetCatalogsReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetCatalogsReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetCatalogsReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java deleted file mode 100644 index 1c5a35437d416..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetCatalogsResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetCatalogsResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetCatalogsRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetCatalogsRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationHandle operationHandle; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - OPERATION_HANDLE((short)2, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetCatalogsResp.class, metaDataMap); - } - - public TGetCatalogsResp() { - } - - public TGetCatalogsResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetCatalogsResp(TGetCatalogsResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetCatalogsResp deepCopy() { - return new TGetCatalogsResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationHandle = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetCatalogsResp) - return this.equals((TGetCatalogsResp)that); - return false; - } - - public boolean equals(TGetCatalogsResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetCatalogsResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetCatalogsResp typedOther = (TGetCatalogsResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetCatalogsResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationHandle()) { - if (!first) sb.append(", "); - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetCatalogsRespStandardSchemeFactory implements SchemeFactory { - public TGetCatalogsRespStandardScheme getScheme() { - return new TGetCatalogsRespStandardScheme(); - } - } - - private static class TGetCatalogsRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetCatalogsResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetCatalogsResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationHandle != null) { - if (struct.isSetOperationHandle()) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetCatalogsRespTupleSchemeFactory implements SchemeFactory { - public TGetCatalogsRespTupleScheme getScheme() { - return new TGetCatalogsRespTupleScheme(); - } - } - - private static class TGetCatalogsRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetCatalogsResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationHandle()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetOperationHandle()) { - struct.operationHandle.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetCatalogsResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java deleted file mode 100644 index a2c793bd95927..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java +++ /dev/null @@ -1,818 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetColumnsReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetColumnsReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); - private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); - private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)4); - private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("columnName", org.apache.thrift.protocol.TType.STRING, (short)5); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetColumnsReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetColumnsReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private String catalogName; // optional - private String schemaName; // optional - private String tableName; // optional - private String columnName; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - CATALOG_NAME((short)2, "catalogName"), - SCHEMA_NAME((short)3, "schemaName"), - TABLE_NAME((short)4, "tableName"), - COLUMN_NAME((short)5, "columnName"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // CATALOG_NAME - return CATALOG_NAME; - case 3: // SCHEMA_NAME - return SCHEMA_NAME; - case 4: // TABLE_NAME - return TABLE_NAME; - case 5: // COLUMN_NAME - return COLUMN_NAME; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME,_Fields.TABLE_NAME,_Fields.COLUMN_NAME}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); - tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - tmpMap.put(_Fields.COLUMN_NAME, new org.apache.thrift.meta_data.FieldMetaData("columnName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetColumnsReq.class, metaDataMap); - } - - public TGetColumnsReq() { - } - - public TGetColumnsReq( - TSessionHandle sessionHandle) - { - this(); - this.sessionHandle = sessionHandle; - } - - /** - * Performs a deep copy on other. - */ - public TGetColumnsReq(TGetColumnsReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetCatalogName()) { - this.catalogName = other.catalogName; - } - if (other.isSetSchemaName()) { - this.schemaName = other.schemaName; - } - if (other.isSetTableName()) { - this.tableName = other.tableName; - } - if (other.isSetColumnName()) { - this.columnName = other.columnName; - } - } - - public TGetColumnsReq deepCopy() { - return new TGetColumnsReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.catalogName = null; - this.schemaName = null; - this.tableName = null; - this.columnName = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public String getCatalogName() { - return this.catalogName; - } - - public void setCatalogName(String catalogName) { - this.catalogName = catalogName; - } - - public void unsetCatalogName() { - this.catalogName = null; - } - - /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ - public boolean isSetCatalogName() { - return this.catalogName != null; - } - - public void setCatalogNameIsSet(boolean value) { - if (!value) { - this.catalogName = null; - } - } - - public String getSchemaName() { - return this.schemaName; - } - - public void setSchemaName(String schemaName) { - this.schemaName = schemaName; - } - - public void unsetSchemaName() { - this.schemaName = null; - } - - /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ - public boolean isSetSchemaName() { - return this.schemaName != null; - } - - public void setSchemaNameIsSet(boolean value) { - if (!value) { - this.schemaName = null; - } - } - - public String getTableName() { - return this.tableName; - } - - public void setTableName(String tableName) { - this.tableName = tableName; - } - - public void unsetTableName() { - this.tableName = null; - } - - /** Returns true if field tableName is set (has been assigned a value) and false otherwise */ - public boolean isSetTableName() { - return this.tableName != null; - } - - public void setTableNameIsSet(boolean value) { - if (!value) { - this.tableName = null; - } - } - - public String getColumnName() { - return this.columnName; - } - - public void setColumnName(String columnName) { - this.columnName = columnName; - } - - public void unsetColumnName() { - this.columnName = null; - } - - /** Returns true if field columnName is set (has been assigned a value) and false otherwise */ - public boolean isSetColumnName() { - return this.columnName != null; - } - - public void setColumnNameIsSet(boolean value) { - if (!value) { - this.columnName = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case CATALOG_NAME: - if (value == null) { - unsetCatalogName(); - } else { - setCatalogName((String)value); - } - break; - - case SCHEMA_NAME: - if (value == null) { - unsetSchemaName(); - } else { - setSchemaName((String)value); - } - break; - - case TABLE_NAME: - if (value == null) { - unsetTableName(); - } else { - setTableName((String)value); - } - break; - - case COLUMN_NAME: - if (value == null) { - unsetColumnName(); - } else { - setColumnName((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case CATALOG_NAME: - return getCatalogName(); - - case SCHEMA_NAME: - return getSchemaName(); - - case TABLE_NAME: - return getTableName(); - - case COLUMN_NAME: - return getColumnName(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case CATALOG_NAME: - return isSetCatalogName(); - case SCHEMA_NAME: - return isSetSchemaName(); - case TABLE_NAME: - return isSetTableName(); - case COLUMN_NAME: - return isSetColumnName(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetColumnsReq) - return this.equals((TGetColumnsReq)that); - return false; - } - - public boolean equals(TGetColumnsReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_catalogName = true && this.isSetCatalogName(); - boolean that_present_catalogName = true && that.isSetCatalogName(); - if (this_present_catalogName || that_present_catalogName) { - if (!(this_present_catalogName && that_present_catalogName)) - return false; - if (!this.catalogName.equals(that.catalogName)) - return false; - } - - boolean this_present_schemaName = true && this.isSetSchemaName(); - boolean that_present_schemaName = true && that.isSetSchemaName(); - if (this_present_schemaName || that_present_schemaName) { - if (!(this_present_schemaName && that_present_schemaName)) - return false; - if (!this.schemaName.equals(that.schemaName)) - return false; - } - - boolean this_present_tableName = true && this.isSetTableName(); - boolean that_present_tableName = true && that.isSetTableName(); - if (this_present_tableName || that_present_tableName) { - if (!(this_present_tableName && that_present_tableName)) - return false; - if (!this.tableName.equals(that.tableName)) - return false; - } - - boolean this_present_columnName = true && this.isSetColumnName(); - boolean that_present_columnName = true && that.isSetColumnName(); - if (this_present_columnName || that_present_columnName) { - if (!(this_present_columnName && that_present_columnName)) - return false; - if (!this.columnName.equals(that.columnName)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_catalogName = true && (isSetCatalogName()); - builder.append(present_catalogName); - if (present_catalogName) - builder.append(catalogName); - - boolean present_schemaName = true && (isSetSchemaName()); - builder.append(present_schemaName); - if (present_schemaName) - builder.append(schemaName); - - boolean present_tableName = true && (isSetTableName()); - builder.append(present_tableName); - if (present_tableName) - builder.append(tableName); - - boolean present_columnName = true && (isSetColumnName()); - builder.append(present_columnName); - if (present_columnName) - builder.append(columnName); - - return builder.toHashCode(); - } - - public int compareTo(TGetColumnsReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetColumnsReq typedOther = (TGetColumnsReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(typedOther.isSetCatalogName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetCatalogName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, typedOther.catalogName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(typedOther.isSetSchemaName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSchemaName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, typedOther.schemaName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetColumnName()).compareTo(typedOther.isSetColumnName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetColumnName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columnName, typedOther.columnName); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetColumnsReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (isSetCatalogName()) { - if (!first) sb.append(", "); - sb.append("catalogName:"); - if (this.catalogName == null) { - sb.append("null"); - } else { - sb.append(this.catalogName); - } - first = false; - } - if (isSetSchemaName()) { - if (!first) sb.append(", "); - sb.append("schemaName:"); - if (this.schemaName == null) { - sb.append("null"); - } else { - sb.append(this.schemaName); - } - first = false; - } - if (isSetTableName()) { - if (!first) sb.append(", "); - sb.append("tableName:"); - if (this.tableName == null) { - sb.append("null"); - } else { - sb.append(this.tableName); - } - first = false; - } - if (isSetColumnName()) { - if (!first) sb.append(", "); - sb.append("columnName:"); - if (this.columnName == null) { - sb.append("null"); - } else { - sb.append(this.columnName); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetColumnsReqStandardSchemeFactory implements SchemeFactory { - public TGetColumnsReqStandardScheme getScheme() { - return new TGetColumnsReqStandardScheme(); - } - } - - private static class TGetColumnsReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetColumnsReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // CATALOG_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.catalogName = iprot.readString(); - struct.setCatalogNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // SCHEMA_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.schemaName = iprot.readString(); - struct.setSchemaNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // TABLE_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.tableName = iprot.readString(); - struct.setTableNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 5: // COLUMN_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.columnName = iprot.readString(); - struct.setColumnNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetColumnsReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.catalogName != null) { - if (struct.isSetCatalogName()) { - oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); - oprot.writeString(struct.catalogName); - oprot.writeFieldEnd(); - } - } - if (struct.schemaName != null) { - if (struct.isSetSchemaName()) { - oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); - oprot.writeString(struct.schemaName); - oprot.writeFieldEnd(); - } - } - if (struct.tableName != null) { - if (struct.isSetTableName()) { - oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); - oprot.writeString(struct.tableName); - oprot.writeFieldEnd(); - } - } - if (struct.columnName != null) { - if (struct.isSetColumnName()) { - oprot.writeFieldBegin(COLUMN_NAME_FIELD_DESC); - oprot.writeString(struct.columnName); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetColumnsReqTupleSchemeFactory implements SchemeFactory { - public TGetColumnsReqTupleScheme getScheme() { - return new TGetColumnsReqTupleScheme(); - } - } - - private static class TGetColumnsReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetColumnsReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetCatalogName()) { - optionals.set(0); - } - if (struct.isSetSchemaName()) { - optionals.set(1); - } - if (struct.isSetTableName()) { - optionals.set(2); - } - if (struct.isSetColumnName()) { - optionals.set(3); - } - oprot.writeBitSet(optionals, 4); - if (struct.isSetCatalogName()) { - oprot.writeString(struct.catalogName); - } - if (struct.isSetSchemaName()) { - oprot.writeString(struct.schemaName); - } - if (struct.isSetTableName()) { - oprot.writeString(struct.tableName); - } - if (struct.isSetColumnName()) { - oprot.writeString(struct.columnName); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetColumnsReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - BitSet incoming = iprot.readBitSet(4); - if (incoming.get(0)) { - struct.catalogName = iprot.readString(); - struct.setCatalogNameIsSet(true); - } - if (incoming.get(1)) { - struct.schemaName = iprot.readString(); - struct.setSchemaNameIsSet(true); - } - if (incoming.get(2)) { - struct.tableName = iprot.readString(); - struct.setTableNameIsSet(true); - } - if (incoming.get(3)) { - struct.columnName = iprot.readString(); - struct.setColumnNameIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java deleted file mode 100644 index d6cf1be6d304b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetColumnsResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetColumnsResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetColumnsRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetColumnsRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationHandle operationHandle; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - OPERATION_HANDLE((short)2, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetColumnsResp.class, metaDataMap); - } - - public TGetColumnsResp() { - } - - public TGetColumnsResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetColumnsResp(TGetColumnsResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetColumnsResp deepCopy() { - return new TGetColumnsResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationHandle = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetColumnsResp) - return this.equals((TGetColumnsResp)that); - return false; - } - - public boolean equals(TGetColumnsResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetColumnsResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetColumnsResp typedOther = (TGetColumnsResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetColumnsResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationHandle()) { - if (!first) sb.append(", "); - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetColumnsRespStandardSchemeFactory implements SchemeFactory { - public TGetColumnsRespStandardScheme getScheme() { - return new TGetColumnsRespStandardScheme(); - } - } - - private static class TGetColumnsRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetColumnsResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetColumnsResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationHandle != null) { - if (struct.isSetOperationHandle()) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetColumnsRespTupleSchemeFactory implements SchemeFactory { - public TGetColumnsRespTupleScheme getScheme() { - return new TGetColumnsRespTupleScheme(); - } - } - - private static class TGetColumnsRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetColumnsResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationHandle()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetOperationHandle()) { - struct.operationHandle.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetColumnsResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java deleted file mode 100644 index 6c6bb00e43e43..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java +++ /dev/null @@ -1,592 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetDelegationTokenReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetDelegationTokenReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)2); - private static final org.apache.thrift.protocol.TField RENEWER_FIELD_DESC = new org.apache.thrift.protocol.TField("renewer", org.apache.thrift.protocol.TType.STRING, (short)3); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetDelegationTokenReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetDelegationTokenReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private String owner; // required - private String renewer; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - OWNER((short)2, "owner"), - RENEWER((short)3, "renewer"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // OWNER - return OWNER; - case 3: // RENEWER - return RENEWER; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.RENEWER, new org.apache.thrift.meta_data.FieldMetaData("renewer", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetDelegationTokenReq.class, metaDataMap); - } - - public TGetDelegationTokenReq() { - } - - public TGetDelegationTokenReq( - TSessionHandle sessionHandle, - String owner, - String renewer) - { - this(); - this.sessionHandle = sessionHandle; - this.owner = owner; - this.renewer = renewer; - } - - /** - * Performs a deep copy on other. - */ - public TGetDelegationTokenReq(TGetDelegationTokenReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetOwner()) { - this.owner = other.owner; - } - if (other.isSetRenewer()) { - this.renewer = other.renewer; - } - } - - public TGetDelegationTokenReq deepCopy() { - return new TGetDelegationTokenReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.owner = null; - this.renewer = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public String getOwner() { - return this.owner; - } - - public void setOwner(String owner) { - this.owner = owner; - } - - public void unsetOwner() { - this.owner = null; - } - - /** Returns true if field owner is set (has been assigned a value) and false otherwise */ - public boolean isSetOwner() { - return this.owner != null; - } - - public void setOwnerIsSet(boolean value) { - if (!value) { - this.owner = null; - } - } - - public String getRenewer() { - return this.renewer; - } - - public void setRenewer(String renewer) { - this.renewer = renewer; - } - - public void unsetRenewer() { - this.renewer = null; - } - - /** Returns true if field renewer is set (has been assigned a value) and false otherwise */ - public boolean isSetRenewer() { - return this.renewer != null; - } - - public void setRenewerIsSet(boolean value) { - if (!value) { - this.renewer = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case OWNER: - if (value == null) { - unsetOwner(); - } else { - setOwner((String)value); - } - break; - - case RENEWER: - if (value == null) { - unsetRenewer(); - } else { - setRenewer((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case OWNER: - return getOwner(); - - case RENEWER: - return getRenewer(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case OWNER: - return isSetOwner(); - case RENEWER: - return isSetRenewer(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetDelegationTokenReq) - return this.equals((TGetDelegationTokenReq)that); - return false; - } - - public boolean equals(TGetDelegationTokenReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_owner = true && this.isSetOwner(); - boolean that_present_owner = true && that.isSetOwner(); - if (this_present_owner || that_present_owner) { - if (!(this_present_owner && that_present_owner)) - return false; - if (!this.owner.equals(that.owner)) - return false; - } - - boolean this_present_renewer = true && this.isSetRenewer(); - boolean that_present_renewer = true && that.isSetRenewer(); - if (this_present_renewer || that_present_renewer) { - if (!(this_present_renewer && that_present_renewer)) - return false; - if (!this.renewer.equals(that.renewer)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_owner = true && (isSetOwner()); - builder.append(present_owner); - if (present_owner) - builder.append(owner); - - boolean present_renewer = true && (isSetRenewer()); - builder.append(present_renewer); - if (present_renewer) - builder.append(renewer); - - return builder.toHashCode(); - } - - public int compareTo(TGetDelegationTokenReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetDelegationTokenReq typedOther = (TGetDelegationTokenReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOwner()).compareTo(typedOther.isSetOwner()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOwner()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, typedOther.owner); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetRenewer()).compareTo(typedOther.isSetRenewer()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetRenewer()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.renewer, typedOther.renewer); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetDelegationTokenReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (!first) sb.append(", "); - sb.append("owner:"); - if (this.owner == null) { - sb.append("null"); - } else { - sb.append(this.owner); - } - first = false; - if (!first) sb.append(", "); - sb.append("renewer:"); - if (this.renewer == null) { - sb.append("null"); - } else { - sb.append(this.renewer); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - if (!isSetOwner()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'owner' is unset! Struct:" + toString()); - } - - if (!isSetRenewer()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'renewer' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetDelegationTokenReqStandardSchemeFactory implements SchemeFactory { - public TGetDelegationTokenReqStandardScheme getScheme() { - return new TGetDelegationTokenReqStandardScheme(); - } - } - - private static class TGetDelegationTokenReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetDelegationTokenReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OWNER - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.owner = iprot.readString(); - struct.setOwnerIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // RENEWER - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.renewer = iprot.readString(); - struct.setRenewerIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetDelegationTokenReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.owner != null) { - oprot.writeFieldBegin(OWNER_FIELD_DESC); - oprot.writeString(struct.owner); - oprot.writeFieldEnd(); - } - if (struct.renewer != null) { - oprot.writeFieldBegin(RENEWER_FIELD_DESC); - oprot.writeString(struct.renewer); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetDelegationTokenReqTupleSchemeFactory implements SchemeFactory { - public TGetDelegationTokenReqTupleScheme getScheme() { - return new TGetDelegationTokenReqTupleScheme(); - } - } - - private static class TGetDelegationTokenReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetDelegationTokenReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - oprot.writeString(struct.owner); - oprot.writeString(struct.renewer); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetDelegationTokenReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - struct.owner = iprot.readString(); - struct.setOwnerIsSet(true); - struct.renewer = iprot.readString(); - struct.setRenewerIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java deleted file mode 100644 index d14c5e029a35d..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java +++ /dev/null @@ -1,500 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetDelegationTokenResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetDelegationTokenResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField DELEGATION_TOKEN_FIELD_DESC = new org.apache.thrift.protocol.TField("delegationToken", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetDelegationTokenRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetDelegationTokenRespTupleSchemeFactory()); - } - - private TStatus status; // required - private String delegationToken; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - DELEGATION_TOKEN((short)2, "delegationToken"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // DELEGATION_TOKEN - return DELEGATION_TOKEN; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.DELEGATION_TOKEN}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.DELEGATION_TOKEN, new org.apache.thrift.meta_data.FieldMetaData("delegationToken", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetDelegationTokenResp.class, metaDataMap); - } - - public TGetDelegationTokenResp() { - } - - public TGetDelegationTokenResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetDelegationTokenResp(TGetDelegationTokenResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetDelegationToken()) { - this.delegationToken = other.delegationToken; - } - } - - public TGetDelegationTokenResp deepCopy() { - return new TGetDelegationTokenResp(this); - } - - @Override - public void clear() { - this.status = null; - this.delegationToken = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public String getDelegationToken() { - return this.delegationToken; - } - - public void setDelegationToken(String delegationToken) { - this.delegationToken = delegationToken; - } - - public void unsetDelegationToken() { - this.delegationToken = null; - } - - /** Returns true if field delegationToken is set (has been assigned a value) and false otherwise */ - public boolean isSetDelegationToken() { - return this.delegationToken != null; - } - - public void setDelegationTokenIsSet(boolean value) { - if (!value) { - this.delegationToken = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case DELEGATION_TOKEN: - if (value == null) { - unsetDelegationToken(); - } else { - setDelegationToken((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case DELEGATION_TOKEN: - return getDelegationToken(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case DELEGATION_TOKEN: - return isSetDelegationToken(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetDelegationTokenResp) - return this.equals((TGetDelegationTokenResp)that); - return false; - } - - public boolean equals(TGetDelegationTokenResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_delegationToken = true && this.isSetDelegationToken(); - boolean that_present_delegationToken = true && that.isSetDelegationToken(); - if (this_present_delegationToken || that_present_delegationToken) { - if (!(this_present_delegationToken && that_present_delegationToken)) - return false; - if (!this.delegationToken.equals(that.delegationToken)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_delegationToken = true && (isSetDelegationToken()); - builder.append(present_delegationToken); - if (present_delegationToken) - builder.append(delegationToken); - - return builder.toHashCode(); - } - - public int compareTo(TGetDelegationTokenResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetDelegationTokenResp typedOther = (TGetDelegationTokenResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetDelegationToken()).compareTo(typedOther.isSetDelegationToken()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetDelegationToken()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.delegationToken, typedOther.delegationToken); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetDelegationTokenResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetDelegationToken()) { - if (!first) sb.append(", "); - sb.append("delegationToken:"); - if (this.delegationToken == null) { - sb.append("null"); - } else { - sb.append(this.delegationToken); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetDelegationTokenRespStandardSchemeFactory implements SchemeFactory { - public TGetDelegationTokenRespStandardScheme getScheme() { - return new TGetDelegationTokenRespStandardScheme(); - } - } - - private static class TGetDelegationTokenRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetDelegationTokenResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // DELEGATION_TOKEN - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.delegationToken = iprot.readString(); - struct.setDelegationTokenIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetDelegationTokenResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.delegationToken != null) { - if (struct.isSetDelegationToken()) { - oprot.writeFieldBegin(DELEGATION_TOKEN_FIELD_DESC); - oprot.writeString(struct.delegationToken); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetDelegationTokenRespTupleSchemeFactory implements SchemeFactory { - public TGetDelegationTokenRespTupleScheme getScheme() { - return new TGetDelegationTokenRespTupleScheme(); - } - } - - private static class TGetDelegationTokenRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetDelegationTokenResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetDelegationToken()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetDelegationToken()) { - oprot.writeString(struct.delegationToken); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetDelegationTokenResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.delegationToken = iprot.readString(); - struct.setDelegationTokenIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java deleted file mode 100644 index ff45ee0386cb9..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java +++ /dev/null @@ -1,707 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetFunctionsReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetFunctionsReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); - private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); - private static final org.apache.thrift.protocol.TField FUNCTION_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("functionName", org.apache.thrift.protocol.TType.STRING, (short)4); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetFunctionsReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetFunctionsReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private String catalogName; // optional - private String schemaName; // optional - private String functionName; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - CATALOG_NAME((short)2, "catalogName"), - SCHEMA_NAME((short)3, "schemaName"), - FUNCTION_NAME((short)4, "functionName"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // CATALOG_NAME - return CATALOG_NAME; - case 3: // SCHEMA_NAME - return SCHEMA_NAME; - case 4: // FUNCTION_NAME - return FUNCTION_NAME; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); - tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - tmpMap.put(_Fields.FUNCTION_NAME, new org.apache.thrift.meta_data.FieldMetaData("functionName", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetFunctionsReq.class, metaDataMap); - } - - public TGetFunctionsReq() { - } - - public TGetFunctionsReq( - TSessionHandle sessionHandle, - String functionName) - { - this(); - this.sessionHandle = sessionHandle; - this.functionName = functionName; - } - - /** - * Performs a deep copy on other. - */ - public TGetFunctionsReq(TGetFunctionsReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetCatalogName()) { - this.catalogName = other.catalogName; - } - if (other.isSetSchemaName()) { - this.schemaName = other.schemaName; - } - if (other.isSetFunctionName()) { - this.functionName = other.functionName; - } - } - - public TGetFunctionsReq deepCopy() { - return new TGetFunctionsReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.catalogName = null; - this.schemaName = null; - this.functionName = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public String getCatalogName() { - return this.catalogName; - } - - public void setCatalogName(String catalogName) { - this.catalogName = catalogName; - } - - public void unsetCatalogName() { - this.catalogName = null; - } - - /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ - public boolean isSetCatalogName() { - return this.catalogName != null; - } - - public void setCatalogNameIsSet(boolean value) { - if (!value) { - this.catalogName = null; - } - } - - public String getSchemaName() { - return this.schemaName; - } - - public void setSchemaName(String schemaName) { - this.schemaName = schemaName; - } - - public void unsetSchemaName() { - this.schemaName = null; - } - - /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ - public boolean isSetSchemaName() { - return this.schemaName != null; - } - - public void setSchemaNameIsSet(boolean value) { - if (!value) { - this.schemaName = null; - } - } - - public String getFunctionName() { - return this.functionName; - } - - public void setFunctionName(String functionName) { - this.functionName = functionName; - } - - public void unsetFunctionName() { - this.functionName = null; - } - - /** Returns true if field functionName is set (has been assigned a value) and false otherwise */ - public boolean isSetFunctionName() { - return this.functionName != null; - } - - public void setFunctionNameIsSet(boolean value) { - if (!value) { - this.functionName = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case CATALOG_NAME: - if (value == null) { - unsetCatalogName(); - } else { - setCatalogName((String)value); - } - break; - - case SCHEMA_NAME: - if (value == null) { - unsetSchemaName(); - } else { - setSchemaName((String)value); - } - break; - - case FUNCTION_NAME: - if (value == null) { - unsetFunctionName(); - } else { - setFunctionName((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case CATALOG_NAME: - return getCatalogName(); - - case SCHEMA_NAME: - return getSchemaName(); - - case FUNCTION_NAME: - return getFunctionName(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case CATALOG_NAME: - return isSetCatalogName(); - case SCHEMA_NAME: - return isSetSchemaName(); - case FUNCTION_NAME: - return isSetFunctionName(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetFunctionsReq) - return this.equals((TGetFunctionsReq)that); - return false; - } - - public boolean equals(TGetFunctionsReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_catalogName = true && this.isSetCatalogName(); - boolean that_present_catalogName = true && that.isSetCatalogName(); - if (this_present_catalogName || that_present_catalogName) { - if (!(this_present_catalogName && that_present_catalogName)) - return false; - if (!this.catalogName.equals(that.catalogName)) - return false; - } - - boolean this_present_schemaName = true && this.isSetSchemaName(); - boolean that_present_schemaName = true && that.isSetSchemaName(); - if (this_present_schemaName || that_present_schemaName) { - if (!(this_present_schemaName && that_present_schemaName)) - return false; - if (!this.schemaName.equals(that.schemaName)) - return false; - } - - boolean this_present_functionName = true && this.isSetFunctionName(); - boolean that_present_functionName = true && that.isSetFunctionName(); - if (this_present_functionName || that_present_functionName) { - if (!(this_present_functionName && that_present_functionName)) - return false; - if (!this.functionName.equals(that.functionName)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_catalogName = true && (isSetCatalogName()); - builder.append(present_catalogName); - if (present_catalogName) - builder.append(catalogName); - - boolean present_schemaName = true && (isSetSchemaName()); - builder.append(present_schemaName); - if (present_schemaName) - builder.append(schemaName); - - boolean present_functionName = true && (isSetFunctionName()); - builder.append(present_functionName); - if (present_functionName) - builder.append(functionName); - - return builder.toHashCode(); - } - - public int compareTo(TGetFunctionsReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetFunctionsReq typedOther = (TGetFunctionsReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(typedOther.isSetCatalogName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetCatalogName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, typedOther.catalogName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(typedOther.isSetSchemaName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSchemaName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, typedOther.schemaName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetFunctionName()).compareTo(typedOther.isSetFunctionName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetFunctionName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.functionName, typedOther.functionName); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetFunctionsReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (isSetCatalogName()) { - if (!first) sb.append(", "); - sb.append("catalogName:"); - if (this.catalogName == null) { - sb.append("null"); - } else { - sb.append(this.catalogName); - } - first = false; - } - if (isSetSchemaName()) { - if (!first) sb.append(", "); - sb.append("schemaName:"); - if (this.schemaName == null) { - sb.append("null"); - } else { - sb.append(this.schemaName); - } - first = false; - } - if (!first) sb.append(", "); - sb.append("functionName:"); - if (this.functionName == null) { - sb.append("null"); - } else { - sb.append(this.functionName); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - if (!isSetFunctionName()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'functionName' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetFunctionsReqStandardSchemeFactory implements SchemeFactory { - public TGetFunctionsReqStandardScheme getScheme() { - return new TGetFunctionsReqStandardScheme(); - } - } - - private static class TGetFunctionsReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetFunctionsReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // CATALOG_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.catalogName = iprot.readString(); - struct.setCatalogNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // SCHEMA_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.schemaName = iprot.readString(); - struct.setSchemaNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // FUNCTION_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.functionName = iprot.readString(); - struct.setFunctionNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetFunctionsReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.catalogName != null) { - if (struct.isSetCatalogName()) { - oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); - oprot.writeString(struct.catalogName); - oprot.writeFieldEnd(); - } - } - if (struct.schemaName != null) { - if (struct.isSetSchemaName()) { - oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); - oprot.writeString(struct.schemaName); - oprot.writeFieldEnd(); - } - } - if (struct.functionName != null) { - oprot.writeFieldBegin(FUNCTION_NAME_FIELD_DESC); - oprot.writeString(struct.functionName); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetFunctionsReqTupleSchemeFactory implements SchemeFactory { - public TGetFunctionsReqTupleScheme getScheme() { - return new TGetFunctionsReqTupleScheme(); - } - } - - private static class TGetFunctionsReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetFunctionsReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - oprot.writeString(struct.functionName); - BitSet optionals = new BitSet(); - if (struct.isSetCatalogName()) { - optionals.set(0); - } - if (struct.isSetSchemaName()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); - if (struct.isSetCatalogName()) { - oprot.writeString(struct.catalogName); - } - if (struct.isSetSchemaName()) { - oprot.writeString(struct.schemaName); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetFunctionsReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - struct.functionName = iprot.readString(); - struct.setFunctionNameIsSet(true); - BitSet incoming = iprot.readBitSet(2); - if (incoming.get(0)) { - struct.catalogName = iprot.readString(); - struct.setCatalogNameIsSet(true); - } - if (incoming.get(1)) { - struct.schemaName = iprot.readString(); - struct.setSchemaNameIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java deleted file mode 100644 index 3adafdacb54ef..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetFunctionsResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetFunctionsResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetFunctionsRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetFunctionsRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationHandle operationHandle; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - OPERATION_HANDLE((short)2, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetFunctionsResp.class, metaDataMap); - } - - public TGetFunctionsResp() { - } - - public TGetFunctionsResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetFunctionsResp(TGetFunctionsResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetFunctionsResp deepCopy() { - return new TGetFunctionsResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationHandle = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetFunctionsResp) - return this.equals((TGetFunctionsResp)that); - return false; - } - - public boolean equals(TGetFunctionsResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetFunctionsResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetFunctionsResp typedOther = (TGetFunctionsResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetFunctionsResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationHandle()) { - if (!first) sb.append(", "); - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetFunctionsRespStandardSchemeFactory implements SchemeFactory { - public TGetFunctionsRespStandardScheme getScheme() { - return new TGetFunctionsRespStandardScheme(); - } - } - - private static class TGetFunctionsRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetFunctionsResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetFunctionsResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationHandle != null) { - if (struct.isSetOperationHandle()) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetFunctionsRespTupleSchemeFactory implements SchemeFactory { - public TGetFunctionsRespTupleScheme getScheme() { - return new TGetFunctionsRespTupleScheme(); - } - } - - private static class TGetFunctionsRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetFunctionsResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationHandle()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetOperationHandle()) { - struct.operationHandle.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetFunctionsResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java deleted file mode 100644 index 0139bf04ec7db..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java +++ /dev/null @@ -1,503 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetInfoReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetInfoReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField INFO_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("infoType", org.apache.thrift.protocol.TType.I32, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetInfoReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetInfoReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private TGetInfoType infoType; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - /** - * - * @see TGetInfoType - */ - INFO_TYPE((short)2, "infoType"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // INFO_TYPE - return INFO_TYPE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.INFO_TYPE, new org.apache.thrift.meta_data.FieldMetaData("infoType", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TGetInfoType.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetInfoReq.class, metaDataMap); - } - - public TGetInfoReq() { - } - - public TGetInfoReq( - TSessionHandle sessionHandle, - TGetInfoType infoType) - { - this(); - this.sessionHandle = sessionHandle; - this.infoType = infoType; - } - - /** - * Performs a deep copy on other. - */ - public TGetInfoReq(TGetInfoReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetInfoType()) { - this.infoType = other.infoType; - } - } - - public TGetInfoReq deepCopy() { - return new TGetInfoReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.infoType = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - /** - * - * @see TGetInfoType - */ - public TGetInfoType getInfoType() { - return this.infoType; - } - - /** - * - * @see TGetInfoType - */ - public void setInfoType(TGetInfoType infoType) { - this.infoType = infoType; - } - - public void unsetInfoType() { - this.infoType = null; - } - - /** Returns true if field infoType is set (has been assigned a value) and false otherwise */ - public boolean isSetInfoType() { - return this.infoType != null; - } - - public void setInfoTypeIsSet(boolean value) { - if (!value) { - this.infoType = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case INFO_TYPE: - if (value == null) { - unsetInfoType(); - } else { - setInfoType((TGetInfoType)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case INFO_TYPE: - return getInfoType(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case INFO_TYPE: - return isSetInfoType(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetInfoReq) - return this.equals((TGetInfoReq)that); - return false; - } - - public boolean equals(TGetInfoReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_infoType = true && this.isSetInfoType(); - boolean that_present_infoType = true && that.isSetInfoType(); - if (this_present_infoType || that_present_infoType) { - if (!(this_present_infoType && that_present_infoType)) - return false; - if (!this.infoType.equals(that.infoType)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_infoType = true && (isSetInfoType()); - builder.append(present_infoType); - if (present_infoType) - builder.append(infoType.getValue()); - - return builder.toHashCode(); - } - - public int compareTo(TGetInfoReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetInfoReq typedOther = (TGetInfoReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetInfoType()).compareTo(typedOther.isSetInfoType()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetInfoType()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.infoType, typedOther.infoType); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetInfoReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (!first) sb.append(", "); - sb.append("infoType:"); - if (this.infoType == null) { - sb.append("null"); - } else { - sb.append(this.infoType); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - if (!isSetInfoType()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'infoType' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetInfoReqStandardSchemeFactory implements SchemeFactory { - public TGetInfoReqStandardScheme getScheme() { - return new TGetInfoReqStandardScheme(); - } - } - - private static class TGetInfoReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetInfoReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // INFO_TYPE - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.infoType = TGetInfoType.findByValue(iprot.readI32()); - struct.setInfoTypeIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetInfoReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.infoType != null) { - oprot.writeFieldBegin(INFO_TYPE_FIELD_DESC); - oprot.writeI32(struct.infoType.getValue()); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetInfoReqTupleSchemeFactory implements SchemeFactory { - public TGetInfoReqTupleScheme getScheme() { - return new TGetInfoReqTupleScheme(); - } - } - - private static class TGetInfoReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetInfoReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - oprot.writeI32(struct.infoType.getValue()); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetInfoReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - struct.infoType = TGetInfoType.findByValue(iprot.readI32()); - struct.setInfoTypeIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java deleted file mode 100644 index 2faaa9211b3ba..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java +++ /dev/null @@ -1,493 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetInfoResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetInfoResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField INFO_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("infoValue", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetInfoRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetInfoRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TGetInfoValue infoValue; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - INFO_VALUE((short)2, "infoValue"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // INFO_VALUE - return INFO_VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.INFO_VALUE, new org.apache.thrift.meta_data.FieldMetaData("infoValue", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetInfoValue.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetInfoResp.class, metaDataMap); - } - - public TGetInfoResp() { - } - - public TGetInfoResp( - TStatus status, - TGetInfoValue infoValue) - { - this(); - this.status = status; - this.infoValue = infoValue; - } - - /** - * Performs a deep copy on other. - */ - public TGetInfoResp(TGetInfoResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetInfoValue()) { - this.infoValue = new TGetInfoValue(other.infoValue); - } - } - - public TGetInfoResp deepCopy() { - return new TGetInfoResp(this); - } - - @Override - public void clear() { - this.status = null; - this.infoValue = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TGetInfoValue getInfoValue() { - return this.infoValue; - } - - public void setInfoValue(TGetInfoValue infoValue) { - this.infoValue = infoValue; - } - - public void unsetInfoValue() { - this.infoValue = null; - } - - /** Returns true if field infoValue is set (has been assigned a value) and false otherwise */ - public boolean isSetInfoValue() { - return this.infoValue != null; - } - - public void setInfoValueIsSet(boolean value) { - if (!value) { - this.infoValue = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case INFO_VALUE: - if (value == null) { - unsetInfoValue(); - } else { - setInfoValue((TGetInfoValue)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case INFO_VALUE: - return getInfoValue(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case INFO_VALUE: - return isSetInfoValue(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetInfoResp) - return this.equals((TGetInfoResp)that); - return false; - } - - public boolean equals(TGetInfoResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_infoValue = true && this.isSetInfoValue(); - boolean that_present_infoValue = true && that.isSetInfoValue(); - if (this_present_infoValue || that_present_infoValue) { - if (!(this_present_infoValue && that_present_infoValue)) - return false; - if (!this.infoValue.equals(that.infoValue)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_infoValue = true && (isSetInfoValue()); - builder.append(present_infoValue); - if (present_infoValue) - builder.append(infoValue); - - return builder.toHashCode(); - } - - public int compareTo(TGetInfoResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetInfoResp typedOther = (TGetInfoResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetInfoValue()).compareTo(typedOther.isSetInfoValue()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetInfoValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.infoValue, typedOther.infoValue); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetInfoResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (!first) sb.append(", "); - sb.append("infoValue:"); - if (this.infoValue == null) { - sb.append("null"); - } else { - sb.append(this.infoValue); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - if (!isSetInfoValue()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'infoValue' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetInfoRespStandardSchemeFactory implements SchemeFactory { - public TGetInfoRespStandardScheme getScheme() { - return new TGetInfoRespStandardScheme(); - } - } - - private static class TGetInfoRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetInfoResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // INFO_VALUE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.infoValue = new TGetInfoValue(); - struct.infoValue.read(iprot); - struct.setInfoValueIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetInfoResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.infoValue != null) { - oprot.writeFieldBegin(INFO_VALUE_FIELD_DESC); - struct.infoValue.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetInfoRespTupleSchemeFactory implements SchemeFactory { - public TGetInfoRespTupleScheme getScheme() { - return new TGetInfoRespTupleScheme(); - } - } - - private static class TGetInfoRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetInfoResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - struct.infoValue.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetInfoResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - struct.infoValue = new TGetInfoValue(); - struct.infoValue.read(iprot); - struct.setInfoValueIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java deleted file mode 100644 index d9dd62414f001..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java +++ /dev/null @@ -1,180 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - - -import java.util.Map; -import java.util.HashMap; -import org.apache.thrift.TEnum; - -public enum TGetInfoType implements org.apache.thrift.TEnum { - CLI_MAX_DRIVER_CONNECTIONS(0), - CLI_MAX_CONCURRENT_ACTIVITIES(1), - CLI_DATA_SOURCE_NAME(2), - CLI_FETCH_DIRECTION(8), - CLI_SERVER_NAME(13), - CLI_SEARCH_PATTERN_ESCAPE(14), - CLI_DBMS_NAME(17), - CLI_DBMS_VER(18), - CLI_ACCESSIBLE_TABLES(19), - CLI_ACCESSIBLE_PROCEDURES(20), - CLI_CURSOR_COMMIT_BEHAVIOR(23), - CLI_DATA_SOURCE_READ_ONLY(25), - CLI_DEFAULT_TXN_ISOLATION(26), - CLI_IDENTIFIER_CASE(28), - CLI_IDENTIFIER_QUOTE_CHAR(29), - CLI_MAX_COLUMN_NAME_LEN(30), - CLI_MAX_CURSOR_NAME_LEN(31), - CLI_MAX_SCHEMA_NAME_LEN(32), - CLI_MAX_CATALOG_NAME_LEN(34), - CLI_MAX_TABLE_NAME_LEN(35), - CLI_SCROLL_CONCURRENCY(43), - CLI_TXN_CAPABLE(46), - CLI_USER_NAME(47), - CLI_TXN_ISOLATION_OPTION(72), - CLI_INTEGRITY(73), - CLI_GETDATA_EXTENSIONS(81), - CLI_NULL_COLLATION(85), - CLI_ALTER_TABLE(86), - CLI_ORDER_BY_COLUMNS_IN_SELECT(90), - CLI_SPECIAL_CHARACTERS(94), - CLI_MAX_COLUMNS_IN_GROUP_BY(97), - CLI_MAX_COLUMNS_IN_INDEX(98), - CLI_MAX_COLUMNS_IN_ORDER_BY(99), - CLI_MAX_COLUMNS_IN_SELECT(100), - CLI_MAX_COLUMNS_IN_TABLE(101), - CLI_MAX_INDEX_SIZE(102), - CLI_MAX_ROW_SIZE(104), - CLI_MAX_STATEMENT_LEN(105), - CLI_MAX_TABLES_IN_SELECT(106), - CLI_MAX_USER_NAME_LEN(107), - CLI_OJ_CAPABILITIES(115), - CLI_XOPEN_CLI_YEAR(10000), - CLI_CURSOR_SENSITIVITY(10001), - CLI_DESCRIBE_PARAMETER(10002), - CLI_CATALOG_NAME(10003), - CLI_COLLATION_SEQ(10004), - CLI_MAX_IDENTIFIER_LEN(10005); - - private final int value; - - private TGetInfoType(int value) { - this.value = value; - } - - /** - * Get the integer value of this enum value, as defined in the Thrift IDL. - */ - public int getValue() { - return value; - } - - /** - * Find a the enum type by its integer value, as defined in the Thrift IDL. - * @return null if the value is not found. - */ - public static TGetInfoType findByValue(int value) { - switch (value) { - case 0: - return CLI_MAX_DRIVER_CONNECTIONS; - case 1: - return CLI_MAX_CONCURRENT_ACTIVITIES; - case 2: - return CLI_DATA_SOURCE_NAME; - case 8: - return CLI_FETCH_DIRECTION; - case 13: - return CLI_SERVER_NAME; - case 14: - return CLI_SEARCH_PATTERN_ESCAPE; - case 17: - return CLI_DBMS_NAME; - case 18: - return CLI_DBMS_VER; - case 19: - return CLI_ACCESSIBLE_TABLES; - case 20: - return CLI_ACCESSIBLE_PROCEDURES; - case 23: - return CLI_CURSOR_COMMIT_BEHAVIOR; - case 25: - return CLI_DATA_SOURCE_READ_ONLY; - case 26: - return CLI_DEFAULT_TXN_ISOLATION; - case 28: - return CLI_IDENTIFIER_CASE; - case 29: - return CLI_IDENTIFIER_QUOTE_CHAR; - case 30: - return CLI_MAX_COLUMN_NAME_LEN; - case 31: - return CLI_MAX_CURSOR_NAME_LEN; - case 32: - return CLI_MAX_SCHEMA_NAME_LEN; - case 34: - return CLI_MAX_CATALOG_NAME_LEN; - case 35: - return CLI_MAX_TABLE_NAME_LEN; - case 43: - return CLI_SCROLL_CONCURRENCY; - case 46: - return CLI_TXN_CAPABLE; - case 47: - return CLI_USER_NAME; - case 72: - return CLI_TXN_ISOLATION_OPTION; - case 73: - return CLI_INTEGRITY; - case 81: - return CLI_GETDATA_EXTENSIONS; - case 85: - return CLI_NULL_COLLATION; - case 86: - return CLI_ALTER_TABLE; - case 90: - return CLI_ORDER_BY_COLUMNS_IN_SELECT; - case 94: - return CLI_SPECIAL_CHARACTERS; - case 97: - return CLI_MAX_COLUMNS_IN_GROUP_BY; - case 98: - return CLI_MAX_COLUMNS_IN_INDEX; - case 99: - return CLI_MAX_COLUMNS_IN_ORDER_BY; - case 100: - return CLI_MAX_COLUMNS_IN_SELECT; - case 101: - return CLI_MAX_COLUMNS_IN_TABLE; - case 102: - return CLI_MAX_INDEX_SIZE; - case 104: - return CLI_MAX_ROW_SIZE; - case 105: - return CLI_MAX_STATEMENT_LEN; - case 106: - return CLI_MAX_TABLES_IN_SELECT; - case 107: - return CLI_MAX_USER_NAME_LEN; - case 115: - return CLI_OJ_CAPABILITIES; - case 10000: - return CLI_XOPEN_CLI_YEAR; - case 10001: - return CLI_CURSOR_SENSITIVITY; - case 10002: - return CLI_DESCRIBE_PARAMETER; - case 10003: - return CLI_CATALOG_NAME; - case 10004: - return CLI_COLLATION_SEQ; - case 10005: - return CLI_MAX_IDENTIFIER_LEN; - default: - return null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java deleted file mode 100644 index fe2a211c46309..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java +++ /dev/null @@ -1,593 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetInfoValue extends org.apache.thrift.TUnion { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetInfoValue"); - private static final org.apache.thrift.protocol.TField STRING_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("stringValue", org.apache.thrift.protocol.TType.STRING, (short)1); - private static final org.apache.thrift.protocol.TField SMALL_INT_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("smallIntValue", org.apache.thrift.protocol.TType.I16, (short)2); - private static final org.apache.thrift.protocol.TField INTEGER_BITMASK_FIELD_DESC = new org.apache.thrift.protocol.TField("integerBitmask", org.apache.thrift.protocol.TType.I32, (short)3); - private static final org.apache.thrift.protocol.TField INTEGER_FLAG_FIELD_DESC = new org.apache.thrift.protocol.TField("integerFlag", org.apache.thrift.protocol.TType.I32, (short)4); - private static final org.apache.thrift.protocol.TField BINARY_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("binaryValue", org.apache.thrift.protocol.TType.I32, (short)5); - private static final org.apache.thrift.protocol.TField LEN_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("lenValue", org.apache.thrift.protocol.TType.I64, (short)6); - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STRING_VALUE((short)1, "stringValue"), - SMALL_INT_VALUE((short)2, "smallIntValue"), - INTEGER_BITMASK((short)3, "integerBitmask"), - INTEGER_FLAG((short)4, "integerFlag"), - BINARY_VALUE((short)5, "binaryValue"), - LEN_VALUE((short)6, "lenValue"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STRING_VALUE - return STRING_VALUE; - case 2: // SMALL_INT_VALUE - return SMALL_INT_VALUE; - case 3: // INTEGER_BITMASK - return INTEGER_BITMASK; - case 4: // INTEGER_FLAG - return INTEGER_FLAG; - case 5: // BINARY_VALUE - return BINARY_VALUE; - case 6: // LEN_VALUE - return LEN_VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STRING_VALUE, new org.apache.thrift.meta_data.FieldMetaData("stringValue", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.SMALL_INT_VALUE, new org.apache.thrift.meta_data.FieldMetaData("smallIntValue", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); - tmpMap.put(_Fields.INTEGER_BITMASK, new org.apache.thrift.meta_data.FieldMetaData("integerBitmask", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); - tmpMap.put(_Fields.INTEGER_FLAG, new org.apache.thrift.meta_data.FieldMetaData("integerFlag", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); - tmpMap.put(_Fields.BINARY_VALUE, new org.apache.thrift.meta_data.FieldMetaData("binaryValue", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); - tmpMap.put(_Fields.LEN_VALUE, new org.apache.thrift.meta_data.FieldMetaData("lenValue", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetInfoValue.class, metaDataMap); - } - - public TGetInfoValue() { - super(); - } - - public TGetInfoValue(TGetInfoValue._Fields setField, Object value) { - super(setField, value); - } - - public TGetInfoValue(TGetInfoValue other) { - super(other); - } - public TGetInfoValue deepCopy() { - return new TGetInfoValue(this); - } - - public static TGetInfoValue stringValue(String value) { - TGetInfoValue x = new TGetInfoValue(); - x.setStringValue(value); - return x; - } - - public static TGetInfoValue smallIntValue(short value) { - TGetInfoValue x = new TGetInfoValue(); - x.setSmallIntValue(value); - return x; - } - - public static TGetInfoValue integerBitmask(int value) { - TGetInfoValue x = new TGetInfoValue(); - x.setIntegerBitmask(value); - return x; - } - - public static TGetInfoValue integerFlag(int value) { - TGetInfoValue x = new TGetInfoValue(); - x.setIntegerFlag(value); - return x; - } - - public static TGetInfoValue binaryValue(int value) { - TGetInfoValue x = new TGetInfoValue(); - x.setBinaryValue(value); - return x; - } - - public static TGetInfoValue lenValue(long value) { - TGetInfoValue x = new TGetInfoValue(); - x.setLenValue(value); - return x; - } - - - @Override - protected void checkType(_Fields setField, Object value) throws ClassCastException { - switch (setField) { - case STRING_VALUE: - if (value instanceof String) { - break; - } - throw new ClassCastException("Was expecting value of type String for field 'stringValue', but got " + value.getClass().getSimpleName()); - case SMALL_INT_VALUE: - if (value instanceof Short) { - break; - } - throw new ClassCastException("Was expecting value of type Short for field 'smallIntValue', but got " + value.getClass().getSimpleName()); - case INTEGER_BITMASK: - if (value instanceof Integer) { - break; - } - throw new ClassCastException("Was expecting value of type Integer for field 'integerBitmask', but got " + value.getClass().getSimpleName()); - case INTEGER_FLAG: - if (value instanceof Integer) { - break; - } - throw new ClassCastException("Was expecting value of type Integer for field 'integerFlag', but got " + value.getClass().getSimpleName()); - case BINARY_VALUE: - if (value instanceof Integer) { - break; - } - throw new ClassCastException("Was expecting value of type Integer for field 'binaryValue', but got " + value.getClass().getSimpleName()); - case LEN_VALUE: - if (value instanceof Long) { - break; - } - throw new ClassCastException("Was expecting value of type Long for field 'lenValue', but got " + value.getClass().getSimpleName()); - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(field.id); - if (setField != null) { - switch (setField) { - case STRING_VALUE: - if (field.type == STRING_VALUE_FIELD_DESC.type) { - String stringValue; - stringValue = iprot.readString(); - return stringValue; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case SMALL_INT_VALUE: - if (field.type == SMALL_INT_VALUE_FIELD_DESC.type) { - Short smallIntValue; - smallIntValue = iprot.readI16(); - return smallIntValue; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case INTEGER_BITMASK: - if (field.type == INTEGER_BITMASK_FIELD_DESC.type) { - Integer integerBitmask; - integerBitmask = iprot.readI32(); - return integerBitmask; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case INTEGER_FLAG: - if (field.type == INTEGER_FLAG_FIELD_DESC.type) { - Integer integerFlag; - integerFlag = iprot.readI32(); - return integerFlag; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case BINARY_VALUE: - if (field.type == BINARY_VALUE_FIELD_DESC.type) { - Integer binaryValue; - binaryValue = iprot.readI32(); - return binaryValue; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case LEN_VALUE: - if (field.type == LEN_VALUE_FIELD_DESC.type) { - Long lenValue; - lenValue = iprot.readI64(); - return lenValue; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - return null; - } - } - - @Override - protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case STRING_VALUE: - String stringValue = (String)value_; - oprot.writeString(stringValue); - return; - case SMALL_INT_VALUE: - Short smallIntValue = (Short)value_; - oprot.writeI16(smallIntValue); - return; - case INTEGER_BITMASK: - Integer integerBitmask = (Integer)value_; - oprot.writeI32(integerBitmask); - return; - case INTEGER_FLAG: - Integer integerFlag = (Integer)value_; - oprot.writeI32(integerFlag); - return; - case BINARY_VALUE: - Integer binaryValue = (Integer)value_; - oprot.writeI32(binaryValue); - return; - case LEN_VALUE: - Long lenValue = (Long)value_; - oprot.writeI64(lenValue); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(fieldID); - if (setField != null) { - switch (setField) { - case STRING_VALUE: - String stringValue; - stringValue = iprot.readString(); - return stringValue; - case SMALL_INT_VALUE: - Short smallIntValue; - smallIntValue = iprot.readI16(); - return smallIntValue; - case INTEGER_BITMASK: - Integer integerBitmask; - integerBitmask = iprot.readI32(); - return integerBitmask; - case INTEGER_FLAG: - Integer integerFlag; - integerFlag = iprot.readI32(); - return integerFlag; - case BINARY_VALUE: - Integer binaryValue; - binaryValue = iprot.readI32(); - return binaryValue; - case LEN_VALUE: - Long lenValue; - lenValue = iprot.readI64(); - return lenValue; - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - throw new TProtocolException("Couldn't find a field with field id " + fieldID); - } - } - - @Override - protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case STRING_VALUE: - String stringValue = (String)value_; - oprot.writeString(stringValue); - return; - case SMALL_INT_VALUE: - Short smallIntValue = (Short)value_; - oprot.writeI16(smallIntValue); - return; - case INTEGER_BITMASK: - Integer integerBitmask = (Integer)value_; - oprot.writeI32(integerBitmask); - return; - case INTEGER_FLAG: - Integer integerFlag = (Integer)value_; - oprot.writeI32(integerFlag); - return; - case BINARY_VALUE: - Integer binaryValue = (Integer)value_; - oprot.writeI32(binaryValue); - return; - case LEN_VALUE: - Long lenValue = (Long)value_; - oprot.writeI64(lenValue); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { - switch (setField) { - case STRING_VALUE: - return STRING_VALUE_FIELD_DESC; - case SMALL_INT_VALUE: - return SMALL_INT_VALUE_FIELD_DESC; - case INTEGER_BITMASK: - return INTEGER_BITMASK_FIELD_DESC; - case INTEGER_FLAG: - return INTEGER_FLAG_FIELD_DESC; - case BINARY_VALUE: - return BINARY_VALUE_FIELD_DESC; - case LEN_VALUE: - return LEN_VALUE_FIELD_DESC; - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected org.apache.thrift.protocol.TStruct getStructDesc() { - return STRUCT_DESC; - } - - @Override - protected _Fields enumForId(short id) { - return _Fields.findByThriftIdOrThrow(id); - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - - public String getStringValue() { - if (getSetField() == _Fields.STRING_VALUE) { - return (String)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'stringValue' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setStringValue(String value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.STRING_VALUE; - value_ = value; - } - - public short getSmallIntValue() { - if (getSetField() == _Fields.SMALL_INT_VALUE) { - return (Short)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'smallIntValue' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setSmallIntValue(short value) { - setField_ = _Fields.SMALL_INT_VALUE; - value_ = value; - } - - public int getIntegerBitmask() { - if (getSetField() == _Fields.INTEGER_BITMASK) { - return (Integer)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'integerBitmask' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setIntegerBitmask(int value) { - setField_ = _Fields.INTEGER_BITMASK; - value_ = value; - } - - public int getIntegerFlag() { - if (getSetField() == _Fields.INTEGER_FLAG) { - return (Integer)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'integerFlag' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setIntegerFlag(int value) { - setField_ = _Fields.INTEGER_FLAG; - value_ = value; - } - - public int getBinaryValue() { - if (getSetField() == _Fields.BINARY_VALUE) { - return (Integer)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'binaryValue' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setBinaryValue(int value) { - setField_ = _Fields.BINARY_VALUE; - value_ = value; - } - - public long getLenValue() { - if (getSetField() == _Fields.LEN_VALUE) { - return (Long)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'lenValue' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setLenValue(long value) { - setField_ = _Fields.LEN_VALUE; - value_ = value; - } - - public boolean isSetStringValue() { - return setField_ == _Fields.STRING_VALUE; - } - - - public boolean isSetSmallIntValue() { - return setField_ == _Fields.SMALL_INT_VALUE; - } - - - public boolean isSetIntegerBitmask() { - return setField_ == _Fields.INTEGER_BITMASK; - } - - - public boolean isSetIntegerFlag() { - return setField_ == _Fields.INTEGER_FLAG; - } - - - public boolean isSetBinaryValue() { - return setField_ == _Fields.BINARY_VALUE; - } - - - public boolean isSetLenValue() { - return setField_ == _Fields.LEN_VALUE; - } - - - public boolean equals(Object other) { - if (other instanceof TGetInfoValue) { - return equals((TGetInfoValue)other); - } else { - return false; - } - } - - public boolean equals(TGetInfoValue other) { - return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); - } - - @Override - public int compareTo(TGetInfoValue other) { - int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); - if (lastComparison == 0) { - return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); - } - return lastComparison; - } - - - @Override - public int hashCode() { - HashCodeBuilder hcb = new HashCodeBuilder(); - hcb.append(this.getClass().getName()); - org.apache.thrift.TFieldIdEnum setField = getSetField(); - if (setField != null) { - hcb.append(setField.getThriftFieldId()); - Object value = getFieldValue(); - if (value instanceof org.apache.thrift.TEnum) { - hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue()); - } else { - hcb.append(value); - } - } - return hcb.toHashCode(); - } - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java deleted file mode 100644 index b88591ea1945b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetOperationStatusReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetOperationStatusReq"); - - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetOperationStatusReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetOperationStatusReqTupleSchemeFactory()); - } - - private TOperationHandle operationHandle; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - OPERATION_HANDLE((short)1, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetOperationStatusReq.class, metaDataMap); - } - - public TGetOperationStatusReq() { - } - - public TGetOperationStatusReq( - TOperationHandle operationHandle) - { - this(); - this.operationHandle = operationHandle; - } - - /** - * Performs a deep copy on other. - */ - public TGetOperationStatusReq(TGetOperationStatusReq other) { - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetOperationStatusReq deepCopy() { - return new TGetOperationStatusReq(this); - } - - @Override - public void clear() { - this.operationHandle = null; - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetOperationStatusReq) - return this.equals((TGetOperationStatusReq)that); - return false; - } - - public boolean equals(TGetOperationStatusReq that) { - if (that == null) - return false; - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetOperationStatusReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetOperationStatusReq typedOther = (TGetOperationStatusReq)other; - - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetOperationStatusReq("); - boolean first = true; - - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetOperationHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetOperationStatusReqStandardSchemeFactory implements SchemeFactory { - public TGetOperationStatusReqStandardScheme getScheme() { - return new TGetOperationStatusReqStandardScheme(); - } - } - - private static class TGetOperationStatusReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetOperationStatusReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetOperationStatusReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.operationHandle != null) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetOperationStatusReqTupleSchemeFactory implements SchemeFactory { - public TGetOperationStatusReqTupleScheme getScheme() { - return new TGetOperationStatusReqTupleScheme(); - } - } - - private static class TGetOperationStatusReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetOperationStatusReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.operationHandle.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetOperationStatusReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java deleted file mode 100644 index 94ba6bb1146de..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java +++ /dev/null @@ -1,827 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetOperationStatusResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetOperationStatusResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationState", org.apache.thrift.protocol.TType.I32, (short)2); - private static final org.apache.thrift.protocol.TField SQL_STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("sqlState", org.apache.thrift.protocol.TType.STRING, (short)3); - private static final org.apache.thrift.protocol.TField ERROR_CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorCode", org.apache.thrift.protocol.TType.I32, (short)4); - private static final org.apache.thrift.protocol.TField ERROR_MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorMessage", org.apache.thrift.protocol.TType.STRING, (short)5); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetOperationStatusRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetOperationStatusRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationState operationState; // optional - private String sqlState; // optional - private int errorCode; // optional - private String errorMessage; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - /** - * - * @see TOperationState - */ - OPERATION_STATE((short)2, "operationState"), - SQL_STATE((short)3, "sqlState"), - ERROR_CODE((short)4, "errorCode"), - ERROR_MESSAGE((short)5, "errorMessage"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_STATE - return OPERATION_STATE; - case 3: // SQL_STATE - return SQL_STATE; - case 4: // ERROR_CODE - return ERROR_CODE; - case 5: // ERROR_MESSAGE - return ERROR_MESSAGE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __ERRORCODE_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.OPERATION_STATE,_Fields.SQL_STATE,_Fields.ERROR_CODE,_Fields.ERROR_MESSAGE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_STATE, new org.apache.thrift.meta_data.FieldMetaData("operationState", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TOperationState.class))); - tmpMap.put(_Fields.SQL_STATE, new org.apache.thrift.meta_data.FieldMetaData("sqlState", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.ERROR_CODE, new org.apache.thrift.meta_data.FieldMetaData("errorCode", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); - tmpMap.put(_Fields.ERROR_MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("errorMessage", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetOperationStatusResp.class, metaDataMap); - } - - public TGetOperationStatusResp() { - } - - public TGetOperationStatusResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetOperationStatusResp(TGetOperationStatusResp other) { - __isset_bitfield = other.__isset_bitfield; - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationState()) { - this.operationState = other.operationState; - } - if (other.isSetSqlState()) { - this.sqlState = other.sqlState; - } - this.errorCode = other.errorCode; - if (other.isSetErrorMessage()) { - this.errorMessage = other.errorMessage; - } - } - - public TGetOperationStatusResp deepCopy() { - return new TGetOperationStatusResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationState = null; - this.sqlState = null; - setErrorCodeIsSet(false); - this.errorCode = 0; - this.errorMessage = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - /** - * - * @see TOperationState - */ - public TOperationState getOperationState() { - return this.operationState; - } - - /** - * - * @see TOperationState - */ - public void setOperationState(TOperationState operationState) { - this.operationState = operationState; - } - - public void unsetOperationState() { - this.operationState = null; - } - - /** Returns true if field operationState is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationState() { - return this.operationState != null; - } - - public void setOperationStateIsSet(boolean value) { - if (!value) { - this.operationState = null; - } - } - - public String getSqlState() { - return this.sqlState; - } - - public void setSqlState(String sqlState) { - this.sqlState = sqlState; - } - - public void unsetSqlState() { - this.sqlState = null; - } - - /** Returns true if field sqlState is set (has been assigned a value) and false otherwise */ - public boolean isSetSqlState() { - return this.sqlState != null; - } - - public void setSqlStateIsSet(boolean value) { - if (!value) { - this.sqlState = null; - } - } - - public int getErrorCode() { - return this.errorCode; - } - - public void setErrorCode(int errorCode) { - this.errorCode = errorCode; - setErrorCodeIsSet(true); - } - - public void unsetErrorCode() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ERRORCODE_ISSET_ID); - } - - /** Returns true if field errorCode is set (has been assigned a value) and false otherwise */ - public boolean isSetErrorCode() { - return EncodingUtils.testBit(__isset_bitfield, __ERRORCODE_ISSET_ID); - } - - public void setErrorCodeIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ERRORCODE_ISSET_ID, value); - } - - public String getErrorMessage() { - return this.errorMessage; - } - - public void setErrorMessage(String errorMessage) { - this.errorMessage = errorMessage; - } - - public void unsetErrorMessage() { - this.errorMessage = null; - } - - /** Returns true if field errorMessage is set (has been assigned a value) and false otherwise */ - public boolean isSetErrorMessage() { - return this.errorMessage != null; - } - - public void setErrorMessageIsSet(boolean value) { - if (!value) { - this.errorMessage = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_STATE: - if (value == null) { - unsetOperationState(); - } else { - setOperationState((TOperationState)value); - } - break; - - case SQL_STATE: - if (value == null) { - unsetSqlState(); - } else { - setSqlState((String)value); - } - break; - - case ERROR_CODE: - if (value == null) { - unsetErrorCode(); - } else { - setErrorCode((Integer)value); - } - break; - - case ERROR_MESSAGE: - if (value == null) { - unsetErrorMessage(); - } else { - setErrorMessage((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_STATE: - return getOperationState(); - - case SQL_STATE: - return getSqlState(); - - case ERROR_CODE: - return Integer.valueOf(getErrorCode()); - - case ERROR_MESSAGE: - return getErrorMessage(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_STATE: - return isSetOperationState(); - case SQL_STATE: - return isSetSqlState(); - case ERROR_CODE: - return isSetErrorCode(); - case ERROR_MESSAGE: - return isSetErrorMessage(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetOperationStatusResp) - return this.equals((TGetOperationStatusResp)that); - return false; - } - - public boolean equals(TGetOperationStatusResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationState = true && this.isSetOperationState(); - boolean that_present_operationState = true && that.isSetOperationState(); - if (this_present_operationState || that_present_operationState) { - if (!(this_present_operationState && that_present_operationState)) - return false; - if (!this.operationState.equals(that.operationState)) - return false; - } - - boolean this_present_sqlState = true && this.isSetSqlState(); - boolean that_present_sqlState = true && that.isSetSqlState(); - if (this_present_sqlState || that_present_sqlState) { - if (!(this_present_sqlState && that_present_sqlState)) - return false; - if (!this.sqlState.equals(that.sqlState)) - return false; - } - - boolean this_present_errorCode = true && this.isSetErrorCode(); - boolean that_present_errorCode = true && that.isSetErrorCode(); - if (this_present_errorCode || that_present_errorCode) { - if (!(this_present_errorCode && that_present_errorCode)) - return false; - if (this.errorCode != that.errorCode) - return false; - } - - boolean this_present_errorMessage = true && this.isSetErrorMessage(); - boolean that_present_errorMessage = true && that.isSetErrorMessage(); - if (this_present_errorMessage || that_present_errorMessage) { - if (!(this_present_errorMessage && that_present_errorMessage)) - return false; - if (!this.errorMessage.equals(that.errorMessage)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationState = true && (isSetOperationState()); - builder.append(present_operationState); - if (present_operationState) - builder.append(operationState.getValue()); - - boolean present_sqlState = true && (isSetSqlState()); - builder.append(present_sqlState); - if (present_sqlState) - builder.append(sqlState); - - boolean present_errorCode = true && (isSetErrorCode()); - builder.append(present_errorCode); - if (present_errorCode) - builder.append(errorCode); - - boolean present_errorMessage = true && (isSetErrorMessage()); - builder.append(present_errorMessage); - if (present_errorMessage) - builder.append(errorMessage); - - return builder.toHashCode(); - } - - public int compareTo(TGetOperationStatusResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetOperationStatusResp typedOther = (TGetOperationStatusResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationState()).compareTo(typedOther.isSetOperationState()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationState()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationState, typedOther.operationState); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSqlState()).compareTo(typedOther.isSetSqlState()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSqlState()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sqlState, typedOther.sqlState); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetErrorCode()).compareTo(typedOther.isSetErrorCode()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetErrorCode()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorCode, typedOther.errorCode); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetErrorMessage()).compareTo(typedOther.isSetErrorMessage()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetErrorMessage()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorMessage, typedOther.errorMessage); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetOperationStatusResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationState()) { - if (!first) sb.append(", "); - sb.append("operationState:"); - if (this.operationState == null) { - sb.append("null"); - } else { - sb.append(this.operationState); - } - first = false; - } - if (isSetSqlState()) { - if (!first) sb.append(", "); - sb.append("sqlState:"); - if (this.sqlState == null) { - sb.append("null"); - } else { - sb.append(this.sqlState); - } - first = false; - } - if (isSetErrorCode()) { - if (!first) sb.append(", "); - sb.append("errorCode:"); - sb.append(this.errorCode); - first = false; - } - if (isSetErrorMessage()) { - if (!first) sb.append(", "); - sb.append("errorMessage:"); - if (this.errorMessage == null) { - sb.append("null"); - } else { - sb.append(this.errorMessage); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetOperationStatusRespStandardSchemeFactory implements SchemeFactory { - public TGetOperationStatusRespStandardScheme getScheme() { - return new TGetOperationStatusRespStandardScheme(); - } - } - - private static class TGetOperationStatusRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetOperationStatusResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_STATE - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.operationState = TOperationState.findByValue(iprot.readI32()); - struct.setOperationStateIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // SQL_STATE - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.sqlState = iprot.readString(); - struct.setSqlStateIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // ERROR_CODE - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.errorCode = iprot.readI32(); - struct.setErrorCodeIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 5: // ERROR_MESSAGE - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.errorMessage = iprot.readString(); - struct.setErrorMessageIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetOperationStatusResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationState != null) { - if (struct.isSetOperationState()) { - oprot.writeFieldBegin(OPERATION_STATE_FIELD_DESC); - oprot.writeI32(struct.operationState.getValue()); - oprot.writeFieldEnd(); - } - } - if (struct.sqlState != null) { - if (struct.isSetSqlState()) { - oprot.writeFieldBegin(SQL_STATE_FIELD_DESC); - oprot.writeString(struct.sqlState); - oprot.writeFieldEnd(); - } - } - if (struct.isSetErrorCode()) { - oprot.writeFieldBegin(ERROR_CODE_FIELD_DESC); - oprot.writeI32(struct.errorCode); - oprot.writeFieldEnd(); - } - if (struct.errorMessage != null) { - if (struct.isSetErrorMessage()) { - oprot.writeFieldBegin(ERROR_MESSAGE_FIELD_DESC); - oprot.writeString(struct.errorMessage); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetOperationStatusRespTupleSchemeFactory implements SchemeFactory { - public TGetOperationStatusRespTupleScheme getScheme() { - return new TGetOperationStatusRespTupleScheme(); - } - } - - private static class TGetOperationStatusRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetOperationStatusResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationState()) { - optionals.set(0); - } - if (struct.isSetSqlState()) { - optionals.set(1); - } - if (struct.isSetErrorCode()) { - optionals.set(2); - } - if (struct.isSetErrorMessage()) { - optionals.set(3); - } - oprot.writeBitSet(optionals, 4); - if (struct.isSetOperationState()) { - oprot.writeI32(struct.operationState.getValue()); - } - if (struct.isSetSqlState()) { - oprot.writeString(struct.sqlState); - } - if (struct.isSetErrorCode()) { - oprot.writeI32(struct.errorCode); - } - if (struct.isSetErrorMessage()) { - oprot.writeString(struct.errorMessage); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetOperationStatusResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(4); - if (incoming.get(0)) { - struct.operationState = TOperationState.findByValue(iprot.readI32()); - struct.setOperationStateIsSet(true); - } - if (incoming.get(1)) { - struct.sqlState = iprot.readString(); - struct.setSqlStateIsSet(true); - } - if (incoming.get(2)) { - struct.errorCode = iprot.readI32(); - struct.setErrorCodeIsSet(true); - } - if (incoming.get(3)) { - struct.errorMessage = iprot.readString(); - struct.setErrorMessageIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java deleted file mode 100644 index 3bf363c958468..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetResultSetMetadataReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetResultSetMetadataReq"); - - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetResultSetMetadataReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetResultSetMetadataReqTupleSchemeFactory()); - } - - private TOperationHandle operationHandle; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - OPERATION_HANDLE((short)1, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetResultSetMetadataReq.class, metaDataMap); - } - - public TGetResultSetMetadataReq() { - } - - public TGetResultSetMetadataReq( - TOperationHandle operationHandle) - { - this(); - this.operationHandle = operationHandle; - } - - /** - * Performs a deep copy on other. - */ - public TGetResultSetMetadataReq(TGetResultSetMetadataReq other) { - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetResultSetMetadataReq deepCopy() { - return new TGetResultSetMetadataReq(this); - } - - @Override - public void clear() { - this.operationHandle = null; - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetResultSetMetadataReq) - return this.equals((TGetResultSetMetadataReq)that); - return false; - } - - public boolean equals(TGetResultSetMetadataReq that) { - if (that == null) - return false; - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetResultSetMetadataReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetResultSetMetadataReq typedOther = (TGetResultSetMetadataReq)other; - - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetResultSetMetadataReq("); - boolean first = true; - - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetOperationHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetResultSetMetadataReqStandardSchemeFactory implements SchemeFactory { - public TGetResultSetMetadataReqStandardScheme getScheme() { - return new TGetResultSetMetadataReqStandardScheme(); - } - } - - private static class TGetResultSetMetadataReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetResultSetMetadataReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetResultSetMetadataReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.operationHandle != null) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetResultSetMetadataReqTupleSchemeFactory implements SchemeFactory { - public TGetResultSetMetadataReqTupleScheme getScheme() { - return new TGetResultSetMetadataReqTupleScheme(); - } - } - - private static class TGetResultSetMetadataReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetResultSetMetadataReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.operationHandle.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetResultSetMetadataReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java deleted file mode 100644 index a9bef9f722c16..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetResultSetMetadataResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetResultSetMetadataResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField SCHEMA_FIELD_DESC = new org.apache.thrift.protocol.TField("schema", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetResultSetMetadataRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetResultSetMetadataRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TTableSchema schema; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - SCHEMA((short)2, "schema"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // SCHEMA - return SCHEMA; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.SCHEMA}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.SCHEMA, new org.apache.thrift.meta_data.FieldMetaData("schema", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableSchema.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetResultSetMetadataResp.class, metaDataMap); - } - - public TGetResultSetMetadataResp() { - } - - public TGetResultSetMetadataResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetResultSetMetadataResp(TGetResultSetMetadataResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetSchema()) { - this.schema = new TTableSchema(other.schema); - } - } - - public TGetResultSetMetadataResp deepCopy() { - return new TGetResultSetMetadataResp(this); - } - - @Override - public void clear() { - this.status = null; - this.schema = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TTableSchema getSchema() { - return this.schema; - } - - public void setSchema(TTableSchema schema) { - this.schema = schema; - } - - public void unsetSchema() { - this.schema = null; - } - - /** Returns true if field schema is set (has been assigned a value) and false otherwise */ - public boolean isSetSchema() { - return this.schema != null; - } - - public void setSchemaIsSet(boolean value) { - if (!value) { - this.schema = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case SCHEMA: - if (value == null) { - unsetSchema(); - } else { - setSchema((TTableSchema)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case SCHEMA: - return getSchema(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case SCHEMA: - return isSetSchema(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetResultSetMetadataResp) - return this.equals((TGetResultSetMetadataResp)that); - return false; - } - - public boolean equals(TGetResultSetMetadataResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_schema = true && this.isSetSchema(); - boolean that_present_schema = true && that.isSetSchema(); - if (this_present_schema || that_present_schema) { - if (!(this_present_schema && that_present_schema)) - return false; - if (!this.schema.equals(that.schema)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_schema = true && (isSetSchema()); - builder.append(present_schema); - if (present_schema) - builder.append(schema); - - return builder.toHashCode(); - } - - public int compareTo(TGetResultSetMetadataResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetResultSetMetadataResp typedOther = (TGetResultSetMetadataResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSchema()).compareTo(typedOther.isSetSchema()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSchema()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schema, typedOther.schema); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetResultSetMetadataResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetSchema()) { - if (!first) sb.append(", "); - sb.append("schema:"); - if (this.schema == null) { - sb.append("null"); - } else { - sb.append(this.schema); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (schema != null) { - schema.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetResultSetMetadataRespStandardSchemeFactory implements SchemeFactory { - public TGetResultSetMetadataRespStandardScheme getScheme() { - return new TGetResultSetMetadataRespStandardScheme(); - } - } - - private static class TGetResultSetMetadataRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetResultSetMetadataResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // SCHEMA - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.schema = new TTableSchema(); - struct.schema.read(iprot); - struct.setSchemaIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetResultSetMetadataResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.schema != null) { - if (struct.isSetSchema()) { - oprot.writeFieldBegin(SCHEMA_FIELD_DESC); - struct.schema.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetResultSetMetadataRespTupleSchemeFactory implements SchemeFactory { - public TGetResultSetMetadataRespTupleScheme getScheme() { - return new TGetResultSetMetadataRespTupleScheme(); - } - } - - private static class TGetResultSetMetadataRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetResultSetMetadataResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetSchema()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSchema()) { - struct.schema.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetResultSetMetadataResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.schema = new TTableSchema(); - struct.schema.read(iprot); - struct.setSchemaIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java deleted file mode 100644 index c2aadaa49a1e9..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java +++ /dev/null @@ -1,606 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetSchemasReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetSchemasReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); - private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetSchemasReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetSchemasReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private String catalogName; // optional - private String schemaName; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - CATALOG_NAME((short)2, "catalogName"), - SCHEMA_NAME((short)3, "schemaName"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // CATALOG_NAME - return CATALOG_NAME; - case 3: // SCHEMA_NAME - return SCHEMA_NAME; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); - tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetSchemasReq.class, metaDataMap); - } - - public TGetSchemasReq() { - } - - public TGetSchemasReq( - TSessionHandle sessionHandle) - { - this(); - this.sessionHandle = sessionHandle; - } - - /** - * Performs a deep copy on other. - */ - public TGetSchemasReq(TGetSchemasReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetCatalogName()) { - this.catalogName = other.catalogName; - } - if (other.isSetSchemaName()) { - this.schemaName = other.schemaName; - } - } - - public TGetSchemasReq deepCopy() { - return new TGetSchemasReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.catalogName = null; - this.schemaName = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public String getCatalogName() { - return this.catalogName; - } - - public void setCatalogName(String catalogName) { - this.catalogName = catalogName; - } - - public void unsetCatalogName() { - this.catalogName = null; - } - - /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ - public boolean isSetCatalogName() { - return this.catalogName != null; - } - - public void setCatalogNameIsSet(boolean value) { - if (!value) { - this.catalogName = null; - } - } - - public String getSchemaName() { - return this.schemaName; - } - - public void setSchemaName(String schemaName) { - this.schemaName = schemaName; - } - - public void unsetSchemaName() { - this.schemaName = null; - } - - /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ - public boolean isSetSchemaName() { - return this.schemaName != null; - } - - public void setSchemaNameIsSet(boolean value) { - if (!value) { - this.schemaName = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case CATALOG_NAME: - if (value == null) { - unsetCatalogName(); - } else { - setCatalogName((String)value); - } - break; - - case SCHEMA_NAME: - if (value == null) { - unsetSchemaName(); - } else { - setSchemaName((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case CATALOG_NAME: - return getCatalogName(); - - case SCHEMA_NAME: - return getSchemaName(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case CATALOG_NAME: - return isSetCatalogName(); - case SCHEMA_NAME: - return isSetSchemaName(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetSchemasReq) - return this.equals((TGetSchemasReq)that); - return false; - } - - public boolean equals(TGetSchemasReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_catalogName = true && this.isSetCatalogName(); - boolean that_present_catalogName = true && that.isSetCatalogName(); - if (this_present_catalogName || that_present_catalogName) { - if (!(this_present_catalogName && that_present_catalogName)) - return false; - if (!this.catalogName.equals(that.catalogName)) - return false; - } - - boolean this_present_schemaName = true && this.isSetSchemaName(); - boolean that_present_schemaName = true && that.isSetSchemaName(); - if (this_present_schemaName || that_present_schemaName) { - if (!(this_present_schemaName && that_present_schemaName)) - return false; - if (!this.schemaName.equals(that.schemaName)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_catalogName = true && (isSetCatalogName()); - builder.append(present_catalogName); - if (present_catalogName) - builder.append(catalogName); - - boolean present_schemaName = true && (isSetSchemaName()); - builder.append(present_schemaName); - if (present_schemaName) - builder.append(schemaName); - - return builder.toHashCode(); - } - - public int compareTo(TGetSchemasReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetSchemasReq typedOther = (TGetSchemasReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(typedOther.isSetCatalogName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetCatalogName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, typedOther.catalogName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(typedOther.isSetSchemaName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSchemaName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, typedOther.schemaName); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetSchemasReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (isSetCatalogName()) { - if (!first) sb.append(", "); - sb.append("catalogName:"); - if (this.catalogName == null) { - sb.append("null"); - } else { - sb.append(this.catalogName); - } - first = false; - } - if (isSetSchemaName()) { - if (!first) sb.append(", "); - sb.append("schemaName:"); - if (this.schemaName == null) { - sb.append("null"); - } else { - sb.append(this.schemaName); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetSchemasReqStandardSchemeFactory implements SchemeFactory { - public TGetSchemasReqStandardScheme getScheme() { - return new TGetSchemasReqStandardScheme(); - } - } - - private static class TGetSchemasReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetSchemasReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // CATALOG_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.catalogName = iprot.readString(); - struct.setCatalogNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // SCHEMA_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.schemaName = iprot.readString(); - struct.setSchemaNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetSchemasReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.catalogName != null) { - if (struct.isSetCatalogName()) { - oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); - oprot.writeString(struct.catalogName); - oprot.writeFieldEnd(); - } - } - if (struct.schemaName != null) { - if (struct.isSetSchemaName()) { - oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); - oprot.writeString(struct.schemaName); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetSchemasReqTupleSchemeFactory implements SchemeFactory { - public TGetSchemasReqTupleScheme getScheme() { - return new TGetSchemasReqTupleScheme(); - } - } - - private static class TGetSchemasReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetSchemasReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetCatalogName()) { - optionals.set(0); - } - if (struct.isSetSchemaName()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); - if (struct.isSetCatalogName()) { - oprot.writeString(struct.catalogName); - } - if (struct.isSetSchemaName()) { - oprot.writeString(struct.schemaName); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetSchemasReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - BitSet incoming = iprot.readBitSet(2); - if (incoming.get(0)) { - struct.catalogName = iprot.readString(); - struct.setCatalogNameIsSet(true); - } - if (incoming.get(1)) { - struct.schemaName = iprot.readString(); - struct.setSchemaNameIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java deleted file mode 100644 index ac1ea3e7cc7af..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetSchemasResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetSchemasResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetSchemasRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetSchemasRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationHandle operationHandle; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - OPERATION_HANDLE((short)2, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetSchemasResp.class, metaDataMap); - } - - public TGetSchemasResp() { - } - - public TGetSchemasResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetSchemasResp(TGetSchemasResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetSchemasResp deepCopy() { - return new TGetSchemasResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationHandle = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetSchemasResp) - return this.equals((TGetSchemasResp)that); - return false; - } - - public boolean equals(TGetSchemasResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetSchemasResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetSchemasResp typedOther = (TGetSchemasResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetSchemasResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationHandle()) { - if (!first) sb.append(", "); - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetSchemasRespStandardSchemeFactory implements SchemeFactory { - public TGetSchemasRespStandardScheme getScheme() { - return new TGetSchemasRespStandardScheme(); - } - } - - private static class TGetSchemasRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetSchemasResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetSchemasResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationHandle != null) { - if (struct.isSetOperationHandle()) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetSchemasRespTupleSchemeFactory implements SchemeFactory { - public TGetSchemasRespTupleScheme getScheme() { - return new TGetSchemasRespTupleScheme(); - } - } - - private static class TGetSchemasRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetSchemasResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationHandle()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetOperationHandle()) { - struct.operationHandle.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetSchemasResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java deleted file mode 100644 index 6f2c713e0be6a..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetTableTypesReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTableTypesReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetTableTypesReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetTableTypesReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTableTypesReq.class, metaDataMap); - } - - public TGetTableTypesReq() { - } - - public TGetTableTypesReq( - TSessionHandle sessionHandle) - { - this(); - this.sessionHandle = sessionHandle; - } - - /** - * Performs a deep copy on other. - */ - public TGetTableTypesReq(TGetTableTypesReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - } - - public TGetTableTypesReq deepCopy() { - return new TGetTableTypesReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetTableTypesReq) - return this.equals((TGetTableTypesReq)that); - return false; - } - - public boolean equals(TGetTableTypesReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetTableTypesReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetTableTypesReq typedOther = (TGetTableTypesReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetTableTypesReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetTableTypesReqStandardSchemeFactory implements SchemeFactory { - public TGetTableTypesReqStandardScheme getScheme() { - return new TGetTableTypesReqStandardScheme(); - } - } - - private static class TGetTableTypesReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTableTypesReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTableTypesReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetTableTypesReqTupleSchemeFactory implements SchemeFactory { - public TGetTableTypesReqTupleScheme getScheme() { - return new TGetTableTypesReqTupleScheme(); - } - } - - private static class TGetTableTypesReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetTableTypesReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetTableTypesReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java deleted file mode 100644 index 6f33fbcf5dadc..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetTableTypesResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTableTypesResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetTableTypesRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetTableTypesRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationHandle operationHandle; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - OPERATION_HANDLE((short)2, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTableTypesResp.class, metaDataMap); - } - - public TGetTableTypesResp() { - } - - public TGetTableTypesResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetTableTypesResp(TGetTableTypesResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetTableTypesResp deepCopy() { - return new TGetTableTypesResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationHandle = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetTableTypesResp) - return this.equals((TGetTableTypesResp)that); - return false; - } - - public boolean equals(TGetTableTypesResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetTableTypesResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetTableTypesResp typedOther = (TGetTableTypesResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetTableTypesResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationHandle()) { - if (!first) sb.append(", "); - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetTableTypesRespStandardSchemeFactory implements SchemeFactory { - public TGetTableTypesRespStandardScheme getScheme() { - return new TGetTableTypesRespStandardScheme(); - } - } - - private static class TGetTableTypesRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTableTypesResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTableTypesResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationHandle != null) { - if (struct.isSetOperationHandle()) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetTableTypesRespTupleSchemeFactory implements SchemeFactory { - public TGetTableTypesRespTupleScheme getScheme() { - return new TGetTableTypesRespTupleScheme(); - } - } - - private static class TGetTableTypesRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetTableTypesResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationHandle()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetOperationHandle()) { - struct.operationHandle.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetTableTypesResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java deleted file mode 100644 index c973fcc24cb10..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java +++ /dev/null @@ -1,870 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetTablesReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTablesReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); - private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); - private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)4); - private static final org.apache.thrift.protocol.TField TABLE_TYPES_FIELD_DESC = new org.apache.thrift.protocol.TField("tableTypes", org.apache.thrift.protocol.TType.LIST, (short)5); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetTablesReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetTablesReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private String catalogName; // optional - private String schemaName; // optional - private String tableName; // optional - private List tableTypes; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - CATALOG_NAME((short)2, "catalogName"), - SCHEMA_NAME((short)3, "schemaName"), - TABLE_NAME((short)4, "tableName"), - TABLE_TYPES((short)5, "tableTypes"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // CATALOG_NAME - return CATALOG_NAME; - case 3: // SCHEMA_NAME - return SCHEMA_NAME; - case 4: // TABLE_NAME - return TABLE_NAME; - case 5: // TABLE_TYPES - return TABLE_TYPES; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME,_Fields.TABLE_NAME,_Fields.TABLE_TYPES}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - tmpMap.put(_Fields.TABLE_TYPES, new org.apache.thrift.meta_data.FieldMetaData("tableTypes", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTablesReq.class, metaDataMap); - } - - public TGetTablesReq() { - } - - public TGetTablesReq( - TSessionHandle sessionHandle) - { - this(); - this.sessionHandle = sessionHandle; - } - - /** - * Performs a deep copy on other. - */ - public TGetTablesReq(TGetTablesReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetCatalogName()) { - this.catalogName = other.catalogName; - } - if (other.isSetSchemaName()) { - this.schemaName = other.schemaName; - } - if (other.isSetTableName()) { - this.tableName = other.tableName; - } - if (other.isSetTableTypes()) { - List __this__tableTypes = new ArrayList(); - for (String other_element : other.tableTypes) { - __this__tableTypes.add(other_element); - } - this.tableTypes = __this__tableTypes; - } - } - - public TGetTablesReq deepCopy() { - return new TGetTablesReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.catalogName = null; - this.schemaName = null; - this.tableName = null; - this.tableTypes = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public String getCatalogName() { - return this.catalogName; - } - - public void setCatalogName(String catalogName) { - this.catalogName = catalogName; - } - - public void unsetCatalogName() { - this.catalogName = null; - } - - /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ - public boolean isSetCatalogName() { - return this.catalogName != null; - } - - public void setCatalogNameIsSet(boolean value) { - if (!value) { - this.catalogName = null; - } - } - - public String getSchemaName() { - return this.schemaName; - } - - public void setSchemaName(String schemaName) { - this.schemaName = schemaName; - } - - public void unsetSchemaName() { - this.schemaName = null; - } - - /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ - public boolean isSetSchemaName() { - return this.schemaName != null; - } - - public void setSchemaNameIsSet(boolean value) { - if (!value) { - this.schemaName = null; - } - } - - public String getTableName() { - return this.tableName; - } - - public void setTableName(String tableName) { - this.tableName = tableName; - } - - public void unsetTableName() { - this.tableName = null; - } - - /** Returns true if field tableName is set (has been assigned a value) and false otherwise */ - public boolean isSetTableName() { - return this.tableName != null; - } - - public void setTableNameIsSet(boolean value) { - if (!value) { - this.tableName = null; - } - } - - public int getTableTypesSize() { - return (this.tableTypes == null) ? 0 : this.tableTypes.size(); - } - - public java.util.Iterator getTableTypesIterator() { - return (this.tableTypes == null) ? null : this.tableTypes.iterator(); - } - - public void addToTableTypes(String elem) { - if (this.tableTypes == null) { - this.tableTypes = new ArrayList(); - } - this.tableTypes.add(elem); - } - - public List getTableTypes() { - return this.tableTypes; - } - - public void setTableTypes(List tableTypes) { - this.tableTypes = tableTypes; - } - - public void unsetTableTypes() { - this.tableTypes = null; - } - - /** Returns true if field tableTypes is set (has been assigned a value) and false otherwise */ - public boolean isSetTableTypes() { - return this.tableTypes != null; - } - - public void setTableTypesIsSet(boolean value) { - if (!value) { - this.tableTypes = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case CATALOG_NAME: - if (value == null) { - unsetCatalogName(); - } else { - setCatalogName((String)value); - } - break; - - case SCHEMA_NAME: - if (value == null) { - unsetSchemaName(); - } else { - setSchemaName((String)value); - } - break; - - case TABLE_NAME: - if (value == null) { - unsetTableName(); - } else { - setTableName((String)value); - } - break; - - case TABLE_TYPES: - if (value == null) { - unsetTableTypes(); - } else { - setTableTypes((List)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case CATALOG_NAME: - return getCatalogName(); - - case SCHEMA_NAME: - return getSchemaName(); - - case TABLE_NAME: - return getTableName(); - - case TABLE_TYPES: - return getTableTypes(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case CATALOG_NAME: - return isSetCatalogName(); - case SCHEMA_NAME: - return isSetSchemaName(); - case TABLE_NAME: - return isSetTableName(); - case TABLE_TYPES: - return isSetTableTypes(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetTablesReq) - return this.equals((TGetTablesReq)that); - return false; - } - - public boolean equals(TGetTablesReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_catalogName = true && this.isSetCatalogName(); - boolean that_present_catalogName = true && that.isSetCatalogName(); - if (this_present_catalogName || that_present_catalogName) { - if (!(this_present_catalogName && that_present_catalogName)) - return false; - if (!this.catalogName.equals(that.catalogName)) - return false; - } - - boolean this_present_schemaName = true && this.isSetSchemaName(); - boolean that_present_schemaName = true && that.isSetSchemaName(); - if (this_present_schemaName || that_present_schemaName) { - if (!(this_present_schemaName && that_present_schemaName)) - return false; - if (!this.schemaName.equals(that.schemaName)) - return false; - } - - boolean this_present_tableName = true && this.isSetTableName(); - boolean that_present_tableName = true && that.isSetTableName(); - if (this_present_tableName || that_present_tableName) { - if (!(this_present_tableName && that_present_tableName)) - return false; - if (!this.tableName.equals(that.tableName)) - return false; - } - - boolean this_present_tableTypes = true && this.isSetTableTypes(); - boolean that_present_tableTypes = true && that.isSetTableTypes(); - if (this_present_tableTypes || that_present_tableTypes) { - if (!(this_present_tableTypes && that_present_tableTypes)) - return false; - if (!this.tableTypes.equals(that.tableTypes)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_catalogName = true && (isSetCatalogName()); - builder.append(present_catalogName); - if (present_catalogName) - builder.append(catalogName); - - boolean present_schemaName = true && (isSetSchemaName()); - builder.append(present_schemaName); - if (present_schemaName) - builder.append(schemaName); - - boolean present_tableName = true && (isSetTableName()); - builder.append(present_tableName); - if (present_tableName) - builder.append(tableName); - - boolean present_tableTypes = true && (isSetTableTypes()); - builder.append(present_tableTypes); - if (present_tableTypes) - builder.append(tableTypes); - - return builder.toHashCode(); - } - - public int compareTo(TGetTablesReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetTablesReq typedOther = (TGetTablesReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(typedOther.isSetCatalogName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetCatalogName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, typedOther.catalogName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(typedOther.isSetSchemaName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSchemaName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, typedOther.schemaName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetTableTypes()).compareTo(typedOther.isSetTableTypes()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetTableTypes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableTypes, typedOther.tableTypes); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetTablesReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (isSetCatalogName()) { - if (!first) sb.append(", "); - sb.append("catalogName:"); - if (this.catalogName == null) { - sb.append("null"); - } else { - sb.append(this.catalogName); - } - first = false; - } - if (isSetSchemaName()) { - if (!first) sb.append(", "); - sb.append("schemaName:"); - if (this.schemaName == null) { - sb.append("null"); - } else { - sb.append(this.schemaName); - } - first = false; - } - if (isSetTableName()) { - if (!first) sb.append(", "); - sb.append("tableName:"); - if (this.tableName == null) { - sb.append("null"); - } else { - sb.append(this.tableName); - } - first = false; - } - if (isSetTableTypes()) { - if (!first) sb.append(", "); - sb.append("tableTypes:"); - if (this.tableTypes == null) { - sb.append("null"); - } else { - sb.append(this.tableTypes); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetTablesReqStandardSchemeFactory implements SchemeFactory { - public TGetTablesReqStandardScheme getScheme() { - return new TGetTablesReqStandardScheme(); - } - } - - private static class TGetTablesReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTablesReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // CATALOG_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.catalogName = iprot.readString(); - struct.setCatalogNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // SCHEMA_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.schemaName = iprot.readString(); - struct.setSchemaNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // TABLE_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.tableName = iprot.readString(); - struct.setTableNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 5: // TABLE_TYPES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list172 = iprot.readListBegin(); - struct.tableTypes = new ArrayList(_list172.size); - for (int _i173 = 0; _i173 < _list172.size; ++_i173) - { - String _elem174; // optional - _elem174 = iprot.readString(); - struct.tableTypes.add(_elem174); - } - iprot.readListEnd(); - } - struct.setTableTypesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTablesReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.catalogName != null) { - if (struct.isSetCatalogName()) { - oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); - oprot.writeString(struct.catalogName); - oprot.writeFieldEnd(); - } - } - if (struct.schemaName != null) { - if (struct.isSetSchemaName()) { - oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); - oprot.writeString(struct.schemaName); - oprot.writeFieldEnd(); - } - } - if (struct.tableName != null) { - if (struct.isSetTableName()) { - oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); - oprot.writeString(struct.tableName); - oprot.writeFieldEnd(); - } - } - if (struct.tableTypes != null) { - if (struct.isSetTableTypes()) { - oprot.writeFieldBegin(TABLE_TYPES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tableTypes.size())); - for (String _iter175 : struct.tableTypes) - { - oprot.writeString(_iter175); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetTablesReqTupleSchemeFactory implements SchemeFactory { - public TGetTablesReqTupleScheme getScheme() { - return new TGetTablesReqTupleScheme(); - } - } - - private static class TGetTablesReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetTablesReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetCatalogName()) { - optionals.set(0); - } - if (struct.isSetSchemaName()) { - optionals.set(1); - } - if (struct.isSetTableName()) { - optionals.set(2); - } - if (struct.isSetTableTypes()) { - optionals.set(3); - } - oprot.writeBitSet(optionals, 4); - if (struct.isSetCatalogName()) { - oprot.writeString(struct.catalogName); - } - if (struct.isSetSchemaName()) { - oprot.writeString(struct.schemaName); - } - if (struct.isSetTableName()) { - oprot.writeString(struct.tableName); - } - if (struct.isSetTableTypes()) { - { - oprot.writeI32(struct.tableTypes.size()); - for (String _iter176 : struct.tableTypes) - { - oprot.writeString(_iter176); - } - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetTablesReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - BitSet incoming = iprot.readBitSet(4); - if (incoming.get(0)) { - struct.catalogName = iprot.readString(); - struct.setCatalogNameIsSet(true); - } - if (incoming.get(1)) { - struct.schemaName = iprot.readString(); - struct.setSchemaNameIsSet(true); - } - if (incoming.get(2)) { - struct.tableName = iprot.readString(); - struct.setTableNameIsSet(true); - } - if (incoming.get(3)) { - { - org.apache.thrift.protocol.TList _list177 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.tableTypes = new ArrayList(_list177.size); - for (int _i178 = 0; _i178 < _list177.size; ++_i178) - { - String _elem179; // optional - _elem179 = iprot.readString(); - struct.tableTypes.add(_elem179); - } - } - struct.setTableTypesIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java deleted file mode 100644 index d526f4478a24e..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetTablesResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTablesResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetTablesRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetTablesRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationHandle operationHandle; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - OPERATION_HANDLE((short)2, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTablesResp.class, metaDataMap); - } - - public TGetTablesResp() { - } - - public TGetTablesResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetTablesResp(TGetTablesResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetTablesResp deepCopy() { - return new TGetTablesResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationHandle = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetTablesResp) - return this.equals((TGetTablesResp)that); - return false; - } - - public boolean equals(TGetTablesResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetTablesResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetTablesResp typedOther = (TGetTablesResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetTablesResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationHandle()) { - if (!first) sb.append(", "); - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetTablesRespStandardSchemeFactory implements SchemeFactory { - public TGetTablesRespStandardScheme getScheme() { - return new TGetTablesRespStandardScheme(); - } - } - - private static class TGetTablesRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTablesResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTablesResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationHandle != null) { - if (struct.isSetOperationHandle()) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetTablesRespTupleSchemeFactory implements SchemeFactory { - public TGetTablesRespTupleScheme getScheme() { - return new TGetTablesRespTupleScheme(); - } - } - - private static class TGetTablesRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetTablesResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationHandle()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetOperationHandle()) { - struct.operationHandle.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetTablesResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java deleted file mode 100644 index d40115e83ec45..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetTypeInfoReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTypeInfoReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetTypeInfoReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetTypeInfoReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTypeInfoReq.class, metaDataMap); - } - - public TGetTypeInfoReq() { - } - - public TGetTypeInfoReq( - TSessionHandle sessionHandle) - { - this(); - this.sessionHandle = sessionHandle; - } - - /** - * Performs a deep copy on other. - */ - public TGetTypeInfoReq(TGetTypeInfoReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - } - - public TGetTypeInfoReq deepCopy() { - return new TGetTypeInfoReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetTypeInfoReq) - return this.equals((TGetTypeInfoReq)that); - return false; - } - - public boolean equals(TGetTypeInfoReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetTypeInfoReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetTypeInfoReq typedOther = (TGetTypeInfoReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetTypeInfoReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetTypeInfoReqStandardSchemeFactory implements SchemeFactory { - public TGetTypeInfoReqStandardScheme getScheme() { - return new TGetTypeInfoReqStandardScheme(); - } - } - - private static class TGetTypeInfoReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTypeInfoReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTypeInfoReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetTypeInfoReqTupleSchemeFactory implements SchemeFactory { - public TGetTypeInfoReqTupleScheme getScheme() { - return new TGetTypeInfoReqTupleScheme(); - } - } - - private static class TGetTypeInfoReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetTypeInfoReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetTypeInfoReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java deleted file mode 100644 index 59be1a33b55e2..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetTypeInfoResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTypeInfoResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetTypeInfoRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetTypeInfoRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationHandle operationHandle; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - OPERATION_HANDLE((short)2, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTypeInfoResp.class, metaDataMap); - } - - public TGetTypeInfoResp() { - } - - public TGetTypeInfoResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetTypeInfoResp(TGetTypeInfoResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetTypeInfoResp deepCopy() { - return new TGetTypeInfoResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationHandle = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetTypeInfoResp) - return this.equals((TGetTypeInfoResp)that); - return false; - } - - public boolean equals(TGetTypeInfoResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetTypeInfoResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetTypeInfoResp typedOther = (TGetTypeInfoResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetTypeInfoResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationHandle()) { - if (!first) sb.append(", "); - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetTypeInfoRespStandardSchemeFactory implements SchemeFactory { - public TGetTypeInfoRespStandardScheme getScheme() { - return new TGetTypeInfoRespStandardScheme(); - } - } - - private static class TGetTypeInfoRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTypeInfoResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTypeInfoResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationHandle != null) { - if (struct.isSetOperationHandle()) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetTypeInfoRespTupleSchemeFactory implements SchemeFactory { - public TGetTypeInfoRespTupleScheme getScheme() { - return new TGetTypeInfoRespTupleScheme(); - } - } - - private static class TGetTypeInfoRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetTypeInfoResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationHandle()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetOperationHandle()) { - struct.operationHandle.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetTypeInfoResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java deleted file mode 100644 index 368273c341c7b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java +++ /dev/null @@ -1,506 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class THandleIdentifier implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THandleIdentifier"); - - private static final org.apache.thrift.protocol.TField GUID_FIELD_DESC = new org.apache.thrift.protocol.TField("guid", org.apache.thrift.protocol.TType.STRING, (short)1); - private static final org.apache.thrift.protocol.TField SECRET_FIELD_DESC = new org.apache.thrift.protocol.TField("secret", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new THandleIdentifierStandardSchemeFactory()); - schemes.put(TupleScheme.class, new THandleIdentifierTupleSchemeFactory()); - } - - private ByteBuffer guid; // required - private ByteBuffer secret; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - GUID((short)1, "guid"), - SECRET((short)2, "secret"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // GUID - return GUID; - case 2: // SECRET - return SECRET; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.GUID, new org.apache.thrift.meta_data.FieldMetaData("guid", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - tmpMap.put(_Fields.SECRET, new org.apache.thrift.meta_data.FieldMetaData("secret", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(THandleIdentifier.class, metaDataMap); - } - - public THandleIdentifier() { - } - - public THandleIdentifier( - ByteBuffer guid, - ByteBuffer secret) - { - this(); - this.guid = guid; - this.secret = secret; - } - - /** - * Performs a deep copy on other. - */ - public THandleIdentifier(THandleIdentifier other) { - if (other.isSetGuid()) { - this.guid = org.apache.thrift.TBaseHelper.copyBinary(other.guid); -; - } - if (other.isSetSecret()) { - this.secret = org.apache.thrift.TBaseHelper.copyBinary(other.secret); -; - } - } - - public THandleIdentifier deepCopy() { - return new THandleIdentifier(this); - } - - @Override - public void clear() { - this.guid = null; - this.secret = null; - } - - public byte[] getGuid() { - setGuid(org.apache.thrift.TBaseHelper.rightSize(guid)); - return guid == null ? null : guid.array(); - } - - public ByteBuffer bufferForGuid() { - return guid; - } - - public void setGuid(byte[] guid) { - setGuid(guid == null ? (ByteBuffer)null : ByteBuffer.wrap(guid)); - } - - public void setGuid(ByteBuffer guid) { - this.guid = guid; - } - - public void unsetGuid() { - this.guid = null; - } - - /** Returns true if field guid is set (has been assigned a value) and false otherwise */ - public boolean isSetGuid() { - return this.guid != null; - } - - public void setGuidIsSet(boolean value) { - if (!value) { - this.guid = null; - } - } - - public byte[] getSecret() { - setSecret(org.apache.thrift.TBaseHelper.rightSize(secret)); - return secret == null ? null : secret.array(); - } - - public ByteBuffer bufferForSecret() { - return secret; - } - - public void setSecret(byte[] secret) { - setSecret(secret == null ? (ByteBuffer)null : ByteBuffer.wrap(secret)); - } - - public void setSecret(ByteBuffer secret) { - this.secret = secret; - } - - public void unsetSecret() { - this.secret = null; - } - - /** Returns true if field secret is set (has been assigned a value) and false otherwise */ - public boolean isSetSecret() { - return this.secret != null; - } - - public void setSecretIsSet(boolean value) { - if (!value) { - this.secret = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case GUID: - if (value == null) { - unsetGuid(); - } else { - setGuid((ByteBuffer)value); - } - break; - - case SECRET: - if (value == null) { - unsetSecret(); - } else { - setSecret((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case GUID: - return getGuid(); - - case SECRET: - return getSecret(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case GUID: - return isSetGuid(); - case SECRET: - return isSetSecret(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof THandleIdentifier) - return this.equals((THandleIdentifier)that); - return false; - } - - public boolean equals(THandleIdentifier that) { - if (that == null) - return false; - - boolean this_present_guid = true && this.isSetGuid(); - boolean that_present_guid = true && that.isSetGuid(); - if (this_present_guid || that_present_guid) { - if (!(this_present_guid && that_present_guid)) - return false; - if (!this.guid.equals(that.guid)) - return false; - } - - boolean this_present_secret = true && this.isSetSecret(); - boolean that_present_secret = true && that.isSetSecret(); - if (this_present_secret || that_present_secret) { - if (!(this_present_secret && that_present_secret)) - return false; - if (!this.secret.equals(that.secret)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_guid = true && (isSetGuid()); - builder.append(present_guid); - if (present_guid) - builder.append(guid); - - boolean present_secret = true && (isSetSecret()); - builder.append(present_secret); - if (present_secret) - builder.append(secret); - - return builder.toHashCode(); - } - - public int compareTo(THandleIdentifier other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - THandleIdentifier typedOther = (THandleIdentifier)other; - - lastComparison = Boolean.valueOf(isSetGuid()).compareTo(typedOther.isSetGuid()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetGuid()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.guid, typedOther.guid); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSecret()).compareTo(typedOther.isSetSecret()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSecret()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.secret, typedOther.secret); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("THandleIdentifier("); - boolean first = true; - - sb.append("guid:"); - if (this.guid == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.guid, sb); - } - first = false; - if (!first) sb.append(", "); - sb.append("secret:"); - if (this.secret == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.secret, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetGuid()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'guid' is unset! Struct:" + toString()); - } - - if (!isSetSecret()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'secret' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class THandleIdentifierStandardSchemeFactory implements SchemeFactory { - public THandleIdentifierStandardScheme getScheme() { - return new THandleIdentifierStandardScheme(); - } - } - - private static class THandleIdentifierStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, THandleIdentifier struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // GUID - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.guid = iprot.readBinary(); - struct.setGuidIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // SECRET - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.secret = iprot.readBinary(); - struct.setSecretIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, THandleIdentifier struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.guid != null) { - oprot.writeFieldBegin(GUID_FIELD_DESC); - oprot.writeBinary(struct.guid); - oprot.writeFieldEnd(); - } - if (struct.secret != null) { - oprot.writeFieldBegin(SECRET_FIELD_DESC); - oprot.writeBinary(struct.secret); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class THandleIdentifierTupleSchemeFactory implements SchemeFactory { - public THandleIdentifierTupleScheme getScheme() { - return new THandleIdentifierTupleScheme(); - } - } - - private static class THandleIdentifierTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, THandleIdentifier struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeBinary(struct.guid); - oprot.writeBinary(struct.secret); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, THandleIdentifier struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.guid = iprot.readBinary(); - struct.setGuidIsSet(true); - struct.secret = iprot.readBinary(); - struct.setSecretIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java deleted file mode 100644 index c83663072f877..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java +++ /dev/null @@ -1,548 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TI16Column implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI16Column"); - - private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); - private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TI16ColumnStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TI16ColumnTupleSchemeFactory()); - } - - private List values; // required - private ByteBuffer nulls; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUES((short)1, "values"), - NULLS((short)2, "nulls"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUES - return VALUES; - case 2: // NULLS - return NULLS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16)))); - tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI16Column.class, metaDataMap); - } - - public TI16Column() { - } - - public TI16Column( - List values, - ByteBuffer nulls) - { - this(); - this.values = values; - this.nulls = nulls; - } - - /** - * Performs a deep copy on other. - */ - public TI16Column(TI16Column other) { - if (other.isSetValues()) { - List __this__values = new ArrayList(); - for (Short other_element : other.values) { - __this__values.add(other_element); - } - this.values = __this__values; - } - if (other.isSetNulls()) { - this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); -; - } - } - - public TI16Column deepCopy() { - return new TI16Column(this); - } - - @Override - public void clear() { - this.values = null; - this.nulls = null; - } - - public int getValuesSize() { - return (this.values == null) ? 0 : this.values.size(); - } - - public java.util.Iterator getValuesIterator() { - return (this.values == null) ? null : this.values.iterator(); - } - - public void addToValues(short elem) { - if (this.values == null) { - this.values = new ArrayList(); - } - this.values.add(elem); - } - - public List getValues() { - return this.values; - } - - public void setValues(List values) { - this.values = values; - } - - public void unsetValues() { - this.values = null; - } - - /** Returns true if field values is set (has been assigned a value) and false otherwise */ - public boolean isSetValues() { - return this.values != null; - } - - public void setValuesIsSet(boolean value) { - if (!value) { - this.values = null; - } - } - - public byte[] getNulls() { - setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); - return nulls == null ? null : nulls.array(); - } - - public ByteBuffer bufferForNulls() { - return nulls; - } - - public void setNulls(byte[] nulls) { - setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); - } - - public void setNulls(ByteBuffer nulls) { - this.nulls = nulls; - } - - public void unsetNulls() { - this.nulls = null; - } - - /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ - public boolean isSetNulls() { - return this.nulls != null; - } - - public void setNullsIsSet(boolean value) { - if (!value) { - this.nulls = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUES: - if (value == null) { - unsetValues(); - } else { - setValues((List)value); - } - break; - - case NULLS: - if (value == null) { - unsetNulls(); - } else { - setNulls((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUES: - return getValues(); - - case NULLS: - return getNulls(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUES: - return isSetValues(); - case NULLS: - return isSetNulls(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TI16Column) - return this.equals((TI16Column)that); - return false; - } - - public boolean equals(TI16Column that) { - if (that == null) - return false; - - boolean this_present_values = true && this.isSetValues(); - boolean that_present_values = true && that.isSetValues(); - if (this_present_values || that_present_values) { - if (!(this_present_values && that_present_values)) - return false; - if (!this.values.equals(that.values)) - return false; - } - - boolean this_present_nulls = true && this.isSetNulls(); - boolean that_present_nulls = true && that.isSetNulls(); - if (this_present_nulls || that_present_nulls) { - if (!(this_present_nulls && that_present_nulls)) - return false; - if (!this.nulls.equals(that.nulls)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_values = true && (isSetValues()); - builder.append(present_values); - if (present_values) - builder.append(values); - - boolean present_nulls = true && (isSetNulls()); - builder.append(present_nulls); - if (present_nulls) - builder.append(nulls); - - return builder.toHashCode(); - } - - public int compareTo(TI16Column other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TI16Column typedOther = (TI16Column)other; - - lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNulls()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TI16Column("); - boolean first = true; - - sb.append("values:"); - if (this.values == null) { - sb.append("null"); - } else { - sb.append(this.values); - } - first = false; - if (!first) sb.append(", "); - sb.append("nulls:"); - if (this.nulls == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.nulls, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetValues()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); - } - - if (!isSetNulls()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TI16ColumnStandardSchemeFactory implements SchemeFactory { - public TI16ColumnStandardScheme getScheme() { - return new TI16ColumnStandardScheme(); - } - } - - private static class TI16ColumnStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TI16Column struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list70 = iprot.readListBegin(); - struct.values = new ArrayList(_list70.size); - for (int _i71 = 0; _i71 < _list70.size; ++_i71) - { - short _elem72; // optional - _elem72 = iprot.readI16(); - struct.values.add(_elem72); - } - iprot.readListEnd(); - } - struct.setValuesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // NULLS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TI16Column struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.values != null) { - oprot.writeFieldBegin(VALUES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I16, struct.values.size())); - for (short _iter73 : struct.values) - { - oprot.writeI16(_iter73); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.nulls != null) { - oprot.writeFieldBegin(NULLS_FIELD_DESC); - oprot.writeBinary(struct.nulls); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TI16ColumnTupleSchemeFactory implements SchemeFactory { - public TI16ColumnTupleScheme getScheme() { - return new TI16ColumnTupleScheme(); - } - } - - private static class TI16ColumnTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TI16Column struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.values.size()); - for (short _iter74 : struct.values) - { - oprot.writeI16(_iter74); - } - } - oprot.writeBinary(struct.nulls); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TI16Column struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list75 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I16, iprot.readI32()); - struct.values = new ArrayList(_list75.size); - for (int _i76 = 0; _i76 < _list75.size; ++_i76) - { - short _elem77; // optional - _elem77 = iprot.readI16(); - struct.values.add(_elem77); - } - } - struct.setValuesIsSet(true); - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java deleted file mode 100644 index bb5ae9609de86..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java +++ /dev/null @@ -1,386 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TI16Value implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI16Value"); - - private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.I16, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TI16ValueStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TI16ValueTupleSchemeFactory()); - } - - private short value; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUE((short)1, "value"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUE - return VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __VALUE_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.VALUE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI16Value.class, metaDataMap); - } - - public TI16Value() { - } - - /** - * Performs a deep copy on other. - */ - public TI16Value(TI16Value other) { - __isset_bitfield = other.__isset_bitfield; - this.value = other.value; - } - - public TI16Value deepCopy() { - return new TI16Value(this); - } - - @Override - public void clear() { - setValueIsSet(false); - this.value = 0; - } - - public short getValue() { - return this.value; - } - - public void setValue(short value) { - this.value = value; - setValueIsSet(true); - } - - public void unsetValue() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - /** Returns true if field value is set (has been assigned a value) and false otherwise */ - public boolean isSetValue() { - return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - public void setValueIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUE: - if (value == null) { - unsetValue(); - } else { - setValue((Short)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUE: - return Short.valueOf(getValue()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUE: - return isSetValue(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TI16Value) - return this.equals((TI16Value)that); - return false; - } - - public boolean equals(TI16Value that) { - if (that == null) - return false; - - boolean this_present_value = true && this.isSetValue(); - boolean that_present_value = true && that.isSetValue(); - if (this_present_value || that_present_value) { - if (!(this_present_value && that_present_value)) - return false; - if (this.value != that.value) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_value = true && (isSetValue()); - builder.append(present_value); - if (present_value) - builder.append(value); - - return builder.toHashCode(); - } - - public int compareTo(TI16Value other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TI16Value typedOther = (TI16Value)other; - - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TI16Value("); - boolean first = true; - - if (isSetValue()) { - sb.append("value:"); - sb.append(this.value); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TI16ValueStandardSchemeFactory implements SchemeFactory { - public TI16ValueStandardScheme getScheme() { - return new TI16ValueStandardScheme(); - } - } - - private static class TI16ValueStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TI16Value struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUE - if (schemeField.type == org.apache.thrift.protocol.TType.I16) { - struct.value = iprot.readI16(); - struct.setValueIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TI16Value struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetValue()) { - oprot.writeFieldBegin(VALUE_FIELD_DESC); - oprot.writeI16(struct.value); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TI16ValueTupleSchemeFactory implements SchemeFactory { - public TI16ValueTupleScheme getScheme() { - return new TI16ValueTupleScheme(); - } - } - - private static class TI16ValueTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TI16Value struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetValue()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetValue()) { - oprot.writeI16(struct.value); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TI16Value struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.value = iprot.readI16(); - struct.setValueIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java deleted file mode 100644 index 6c6c5f35b7c8e..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java +++ /dev/null @@ -1,548 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TI32Column implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI32Column"); - - private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); - private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TI32ColumnStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TI32ColumnTupleSchemeFactory()); - } - - private List values; // required - private ByteBuffer nulls; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUES((short)1, "values"), - NULLS((short)2, "nulls"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUES - return VALUES; - case 2: // NULLS - return NULLS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)))); - tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI32Column.class, metaDataMap); - } - - public TI32Column() { - } - - public TI32Column( - List values, - ByteBuffer nulls) - { - this(); - this.values = values; - this.nulls = nulls; - } - - /** - * Performs a deep copy on other. - */ - public TI32Column(TI32Column other) { - if (other.isSetValues()) { - List __this__values = new ArrayList(); - for (Integer other_element : other.values) { - __this__values.add(other_element); - } - this.values = __this__values; - } - if (other.isSetNulls()) { - this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); -; - } - } - - public TI32Column deepCopy() { - return new TI32Column(this); - } - - @Override - public void clear() { - this.values = null; - this.nulls = null; - } - - public int getValuesSize() { - return (this.values == null) ? 0 : this.values.size(); - } - - public java.util.Iterator getValuesIterator() { - return (this.values == null) ? null : this.values.iterator(); - } - - public void addToValues(int elem) { - if (this.values == null) { - this.values = new ArrayList(); - } - this.values.add(elem); - } - - public List getValues() { - return this.values; - } - - public void setValues(List values) { - this.values = values; - } - - public void unsetValues() { - this.values = null; - } - - /** Returns true if field values is set (has been assigned a value) and false otherwise */ - public boolean isSetValues() { - return this.values != null; - } - - public void setValuesIsSet(boolean value) { - if (!value) { - this.values = null; - } - } - - public byte[] getNulls() { - setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); - return nulls == null ? null : nulls.array(); - } - - public ByteBuffer bufferForNulls() { - return nulls; - } - - public void setNulls(byte[] nulls) { - setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); - } - - public void setNulls(ByteBuffer nulls) { - this.nulls = nulls; - } - - public void unsetNulls() { - this.nulls = null; - } - - /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ - public boolean isSetNulls() { - return this.nulls != null; - } - - public void setNullsIsSet(boolean value) { - if (!value) { - this.nulls = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUES: - if (value == null) { - unsetValues(); - } else { - setValues((List)value); - } - break; - - case NULLS: - if (value == null) { - unsetNulls(); - } else { - setNulls((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUES: - return getValues(); - - case NULLS: - return getNulls(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUES: - return isSetValues(); - case NULLS: - return isSetNulls(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TI32Column) - return this.equals((TI32Column)that); - return false; - } - - public boolean equals(TI32Column that) { - if (that == null) - return false; - - boolean this_present_values = true && this.isSetValues(); - boolean that_present_values = true && that.isSetValues(); - if (this_present_values || that_present_values) { - if (!(this_present_values && that_present_values)) - return false; - if (!this.values.equals(that.values)) - return false; - } - - boolean this_present_nulls = true && this.isSetNulls(); - boolean that_present_nulls = true && that.isSetNulls(); - if (this_present_nulls || that_present_nulls) { - if (!(this_present_nulls && that_present_nulls)) - return false; - if (!this.nulls.equals(that.nulls)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_values = true && (isSetValues()); - builder.append(present_values); - if (present_values) - builder.append(values); - - boolean present_nulls = true && (isSetNulls()); - builder.append(present_nulls); - if (present_nulls) - builder.append(nulls); - - return builder.toHashCode(); - } - - public int compareTo(TI32Column other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TI32Column typedOther = (TI32Column)other; - - lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNulls()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TI32Column("); - boolean first = true; - - sb.append("values:"); - if (this.values == null) { - sb.append("null"); - } else { - sb.append(this.values); - } - first = false; - if (!first) sb.append(", "); - sb.append("nulls:"); - if (this.nulls == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.nulls, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetValues()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); - } - - if (!isSetNulls()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TI32ColumnStandardSchemeFactory implements SchemeFactory { - public TI32ColumnStandardScheme getScheme() { - return new TI32ColumnStandardScheme(); - } - } - - private static class TI32ColumnStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TI32Column struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list78 = iprot.readListBegin(); - struct.values = new ArrayList(_list78.size); - for (int _i79 = 0; _i79 < _list78.size; ++_i79) - { - int _elem80; // optional - _elem80 = iprot.readI32(); - struct.values.add(_elem80); - } - iprot.readListEnd(); - } - struct.setValuesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // NULLS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TI32Column struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.values != null) { - oprot.writeFieldBegin(VALUES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.values.size())); - for (int _iter81 : struct.values) - { - oprot.writeI32(_iter81); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.nulls != null) { - oprot.writeFieldBegin(NULLS_FIELD_DESC); - oprot.writeBinary(struct.nulls); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TI32ColumnTupleSchemeFactory implements SchemeFactory { - public TI32ColumnTupleScheme getScheme() { - return new TI32ColumnTupleScheme(); - } - } - - private static class TI32ColumnTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TI32Column struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.values.size()); - for (int _iter82 : struct.values) - { - oprot.writeI32(_iter82); - } - } - oprot.writeBinary(struct.nulls); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TI32Column struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list83 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32()); - struct.values = new ArrayList(_list83.size); - for (int _i84 = 0; _i84 < _list83.size; ++_i84) - { - int _elem85; // optional - _elem85 = iprot.readI32(); - struct.values.add(_elem85); - } - } - struct.setValuesIsSet(true); - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java deleted file mode 100644 index 059408b96c8ce..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java +++ /dev/null @@ -1,386 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TI32Value implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI32Value"); - - private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.I32, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TI32ValueStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TI32ValueTupleSchemeFactory()); - } - - private int value; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUE((short)1, "value"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUE - return VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __VALUE_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.VALUE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI32Value.class, metaDataMap); - } - - public TI32Value() { - } - - /** - * Performs a deep copy on other. - */ - public TI32Value(TI32Value other) { - __isset_bitfield = other.__isset_bitfield; - this.value = other.value; - } - - public TI32Value deepCopy() { - return new TI32Value(this); - } - - @Override - public void clear() { - setValueIsSet(false); - this.value = 0; - } - - public int getValue() { - return this.value; - } - - public void setValue(int value) { - this.value = value; - setValueIsSet(true); - } - - public void unsetValue() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - /** Returns true if field value is set (has been assigned a value) and false otherwise */ - public boolean isSetValue() { - return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - public void setValueIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUE: - if (value == null) { - unsetValue(); - } else { - setValue((Integer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUE: - return Integer.valueOf(getValue()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUE: - return isSetValue(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TI32Value) - return this.equals((TI32Value)that); - return false; - } - - public boolean equals(TI32Value that) { - if (that == null) - return false; - - boolean this_present_value = true && this.isSetValue(); - boolean that_present_value = true && that.isSetValue(); - if (this_present_value || that_present_value) { - if (!(this_present_value && that_present_value)) - return false; - if (this.value != that.value) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_value = true && (isSetValue()); - builder.append(present_value); - if (present_value) - builder.append(value); - - return builder.toHashCode(); - } - - public int compareTo(TI32Value other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TI32Value typedOther = (TI32Value)other; - - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TI32Value("); - boolean first = true; - - if (isSetValue()) { - sb.append("value:"); - sb.append(this.value); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TI32ValueStandardSchemeFactory implements SchemeFactory { - public TI32ValueStandardScheme getScheme() { - return new TI32ValueStandardScheme(); - } - } - - private static class TI32ValueStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TI32Value struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUE - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.value = iprot.readI32(); - struct.setValueIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TI32Value struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetValue()) { - oprot.writeFieldBegin(VALUE_FIELD_DESC); - oprot.writeI32(struct.value); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TI32ValueTupleSchemeFactory implements SchemeFactory { - public TI32ValueTupleScheme getScheme() { - return new TI32ValueTupleScheme(); - } - } - - private static class TI32ValueTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TI32Value struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetValue()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetValue()) { - oprot.writeI32(struct.value); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TI32Value struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.value = iprot.readI32(); - struct.setValueIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java deleted file mode 100644 index cc383ed089fa4..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java +++ /dev/null @@ -1,548 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TI64Column implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI64Column"); - - private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); - private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TI64ColumnStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TI64ColumnTupleSchemeFactory()); - } - - private List values; // required - private ByteBuffer nulls; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUES((short)1, "values"), - NULLS((short)2, "nulls"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUES - return VALUES; - case 2: // NULLS - return NULLS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))); - tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI64Column.class, metaDataMap); - } - - public TI64Column() { - } - - public TI64Column( - List values, - ByteBuffer nulls) - { - this(); - this.values = values; - this.nulls = nulls; - } - - /** - * Performs a deep copy on other. - */ - public TI64Column(TI64Column other) { - if (other.isSetValues()) { - List __this__values = new ArrayList(); - for (Long other_element : other.values) { - __this__values.add(other_element); - } - this.values = __this__values; - } - if (other.isSetNulls()) { - this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); -; - } - } - - public TI64Column deepCopy() { - return new TI64Column(this); - } - - @Override - public void clear() { - this.values = null; - this.nulls = null; - } - - public int getValuesSize() { - return (this.values == null) ? 0 : this.values.size(); - } - - public java.util.Iterator getValuesIterator() { - return (this.values == null) ? null : this.values.iterator(); - } - - public void addToValues(long elem) { - if (this.values == null) { - this.values = new ArrayList(); - } - this.values.add(elem); - } - - public List getValues() { - return this.values; - } - - public void setValues(List values) { - this.values = values; - } - - public void unsetValues() { - this.values = null; - } - - /** Returns true if field values is set (has been assigned a value) and false otherwise */ - public boolean isSetValues() { - return this.values != null; - } - - public void setValuesIsSet(boolean value) { - if (!value) { - this.values = null; - } - } - - public byte[] getNulls() { - setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); - return nulls == null ? null : nulls.array(); - } - - public ByteBuffer bufferForNulls() { - return nulls; - } - - public void setNulls(byte[] nulls) { - setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); - } - - public void setNulls(ByteBuffer nulls) { - this.nulls = nulls; - } - - public void unsetNulls() { - this.nulls = null; - } - - /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ - public boolean isSetNulls() { - return this.nulls != null; - } - - public void setNullsIsSet(boolean value) { - if (!value) { - this.nulls = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUES: - if (value == null) { - unsetValues(); - } else { - setValues((List)value); - } - break; - - case NULLS: - if (value == null) { - unsetNulls(); - } else { - setNulls((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUES: - return getValues(); - - case NULLS: - return getNulls(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUES: - return isSetValues(); - case NULLS: - return isSetNulls(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TI64Column) - return this.equals((TI64Column)that); - return false; - } - - public boolean equals(TI64Column that) { - if (that == null) - return false; - - boolean this_present_values = true && this.isSetValues(); - boolean that_present_values = true && that.isSetValues(); - if (this_present_values || that_present_values) { - if (!(this_present_values && that_present_values)) - return false; - if (!this.values.equals(that.values)) - return false; - } - - boolean this_present_nulls = true && this.isSetNulls(); - boolean that_present_nulls = true && that.isSetNulls(); - if (this_present_nulls || that_present_nulls) { - if (!(this_present_nulls && that_present_nulls)) - return false; - if (!this.nulls.equals(that.nulls)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_values = true && (isSetValues()); - builder.append(present_values); - if (present_values) - builder.append(values); - - boolean present_nulls = true && (isSetNulls()); - builder.append(present_nulls); - if (present_nulls) - builder.append(nulls); - - return builder.toHashCode(); - } - - public int compareTo(TI64Column other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TI64Column typedOther = (TI64Column)other; - - lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNulls()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TI64Column("); - boolean first = true; - - sb.append("values:"); - if (this.values == null) { - sb.append("null"); - } else { - sb.append(this.values); - } - first = false; - if (!first) sb.append(", "); - sb.append("nulls:"); - if (this.nulls == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.nulls, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetValues()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); - } - - if (!isSetNulls()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TI64ColumnStandardSchemeFactory implements SchemeFactory { - public TI64ColumnStandardScheme getScheme() { - return new TI64ColumnStandardScheme(); - } - } - - private static class TI64ColumnStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TI64Column struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list86 = iprot.readListBegin(); - struct.values = new ArrayList(_list86.size); - for (int _i87 = 0; _i87 < _list86.size; ++_i87) - { - long _elem88; // optional - _elem88 = iprot.readI64(); - struct.values.add(_elem88); - } - iprot.readListEnd(); - } - struct.setValuesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // NULLS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TI64Column struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.values != null) { - oprot.writeFieldBegin(VALUES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.values.size())); - for (long _iter89 : struct.values) - { - oprot.writeI64(_iter89); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.nulls != null) { - oprot.writeFieldBegin(NULLS_FIELD_DESC); - oprot.writeBinary(struct.nulls); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TI64ColumnTupleSchemeFactory implements SchemeFactory { - public TI64ColumnTupleScheme getScheme() { - return new TI64ColumnTupleScheme(); - } - } - - private static class TI64ColumnTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TI64Column struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.values.size()); - for (long _iter90 : struct.values) - { - oprot.writeI64(_iter90); - } - } - oprot.writeBinary(struct.nulls); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TI64Column struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list91 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - struct.values = new ArrayList(_list91.size); - for (int _i92 = 0; _i92 < _list91.size; ++_i92) - { - long _elem93; // optional - _elem93 = iprot.readI64(); - struct.values.add(_elem93); - } - } - struct.setValuesIsSet(true); - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java deleted file mode 100644 index 9a941cce0c077..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java +++ /dev/null @@ -1,386 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TI64Value implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI64Value"); - - private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.I64, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TI64ValueStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TI64ValueTupleSchemeFactory()); - } - - private long value; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUE((short)1, "value"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUE - return VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __VALUE_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.VALUE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI64Value.class, metaDataMap); - } - - public TI64Value() { - } - - /** - * Performs a deep copy on other. - */ - public TI64Value(TI64Value other) { - __isset_bitfield = other.__isset_bitfield; - this.value = other.value; - } - - public TI64Value deepCopy() { - return new TI64Value(this); - } - - @Override - public void clear() { - setValueIsSet(false); - this.value = 0; - } - - public long getValue() { - return this.value; - } - - public void setValue(long value) { - this.value = value; - setValueIsSet(true); - } - - public void unsetValue() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - /** Returns true if field value is set (has been assigned a value) and false otherwise */ - public boolean isSetValue() { - return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - public void setValueIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUE: - if (value == null) { - unsetValue(); - } else { - setValue((Long)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUE: - return Long.valueOf(getValue()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUE: - return isSetValue(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TI64Value) - return this.equals((TI64Value)that); - return false; - } - - public boolean equals(TI64Value that) { - if (that == null) - return false; - - boolean this_present_value = true && this.isSetValue(); - boolean that_present_value = true && that.isSetValue(); - if (this_present_value || that_present_value) { - if (!(this_present_value && that_present_value)) - return false; - if (this.value != that.value) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_value = true && (isSetValue()); - builder.append(present_value); - if (present_value) - builder.append(value); - - return builder.toHashCode(); - } - - public int compareTo(TI64Value other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TI64Value typedOther = (TI64Value)other; - - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TI64Value("); - boolean first = true; - - if (isSetValue()) { - sb.append("value:"); - sb.append(this.value); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TI64ValueStandardSchemeFactory implements SchemeFactory { - public TI64ValueStandardScheme getScheme() { - return new TI64ValueStandardScheme(); - } - } - - private static class TI64ValueStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TI64Value struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUE - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.value = iprot.readI64(); - struct.setValueIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TI64Value struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetValue()) { - oprot.writeFieldBegin(VALUE_FIELD_DESC); - oprot.writeI64(struct.value); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TI64ValueTupleSchemeFactory implements SchemeFactory { - public TI64ValueTupleScheme getScheme() { - return new TI64ValueTupleScheme(); - } - } - - private static class TI64ValueTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TI64Value struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetValue()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetValue()) { - oprot.writeI64(struct.value); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TI64Value struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.value = iprot.readI64(); - struct.setValueIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java deleted file mode 100644 index 425603cbdecbd..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java +++ /dev/null @@ -1,478 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TMapTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TMapTypeEntry"); - - private static final org.apache.thrift.protocol.TField KEY_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("keyTypePtr", org.apache.thrift.protocol.TType.I32, (short)1); - private static final org.apache.thrift.protocol.TField VALUE_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("valueTypePtr", org.apache.thrift.protocol.TType.I32, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TMapTypeEntryStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TMapTypeEntryTupleSchemeFactory()); - } - - private int keyTypePtr; // required - private int valueTypePtr; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - KEY_TYPE_PTR((short)1, "keyTypePtr"), - VALUE_TYPE_PTR((short)2, "valueTypePtr"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // KEY_TYPE_PTR - return KEY_TYPE_PTR; - case 2: // VALUE_TYPE_PTR - return VALUE_TYPE_PTR; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __KEYTYPEPTR_ISSET_ID = 0; - private static final int __VALUETYPEPTR_ISSET_ID = 1; - private byte __isset_bitfield = 0; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.KEY_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("keyTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr"))); - tmpMap.put(_Fields.VALUE_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("valueTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr"))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TMapTypeEntry.class, metaDataMap); - } - - public TMapTypeEntry() { - } - - public TMapTypeEntry( - int keyTypePtr, - int valueTypePtr) - { - this(); - this.keyTypePtr = keyTypePtr; - setKeyTypePtrIsSet(true); - this.valueTypePtr = valueTypePtr; - setValueTypePtrIsSet(true); - } - - /** - * Performs a deep copy on other. - */ - public TMapTypeEntry(TMapTypeEntry other) { - __isset_bitfield = other.__isset_bitfield; - this.keyTypePtr = other.keyTypePtr; - this.valueTypePtr = other.valueTypePtr; - } - - public TMapTypeEntry deepCopy() { - return new TMapTypeEntry(this); - } - - @Override - public void clear() { - setKeyTypePtrIsSet(false); - this.keyTypePtr = 0; - setValueTypePtrIsSet(false); - this.valueTypePtr = 0; - } - - public int getKeyTypePtr() { - return this.keyTypePtr; - } - - public void setKeyTypePtr(int keyTypePtr) { - this.keyTypePtr = keyTypePtr; - setKeyTypePtrIsSet(true); - } - - public void unsetKeyTypePtr() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __KEYTYPEPTR_ISSET_ID); - } - - /** Returns true if field keyTypePtr is set (has been assigned a value) and false otherwise */ - public boolean isSetKeyTypePtr() { - return EncodingUtils.testBit(__isset_bitfield, __KEYTYPEPTR_ISSET_ID); - } - - public void setKeyTypePtrIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __KEYTYPEPTR_ISSET_ID, value); - } - - public int getValueTypePtr() { - return this.valueTypePtr; - } - - public void setValueTypePtr(int valueTypePtr) { - this.valueTypePtr = valueTypePtr; - setValueTypePtrIsSet(true); - } - - public void unsetValueTypePtr() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUETYPEPTR_ISSET_ID); - } - - /** Returns true if field valueTypePtr is set (has been assigned a value) and false otherwise */ - public boolean isSetValueTypePtr() { - return EncodingUtils.testBit(__isset_bitfield, __VALUETYPEPTR_ISSET_ID); - } - - public void setValueTypePtrIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUETYPEPTR_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case KEY_TYPE_PTR: - if (value == null) { - unsetKeyTypePtr(); - } else { - setKeyTypePtr((Integer)value); - } - break; - - case VALUE_TYPE_PTR: - if (value == null) { - unsetValueTypePtr(); - } else { - setValueTypePtr((Integer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case KEY_TYPE_PTR: - return Integer.valueOf(getKeyTypePtr()); - - case VALUE_TYPE_PTR: - return Integer.valueOf(getValueTypePtr()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case KEY_TYPE_PTR: - return isSetKeyTypePtr(); - case VALUE_TYPE_PTR: - return isSetValueTypePtr(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TMapTypeEntry) - return this.equals((TMapTypeEntry)that); - return false; - } - - public boolean equals(TMapTypeEntry that) { - if (that == null) - return false; - - boolean this_present_keyTypePtr = true; - boolean that_present_keyTypePtr = true; - if (this_present_keyTypePtr || that_present_keyTypePtr) { - if (!(this_present_keyTypePtr && that_present_keyTypePtr)) - return false; - if (this.keyTypePtr != that.keyTypePtr) - return false; - } - - boolean this_present_valueTypePtr = true; - boolean that_present_valueTypePtr = true; - if (this_present_valueTypePtr || that_present_valueTypePtr) { - if (!(this_present_valueTypePtr && that_present_valueTypePtr)) - return false; - if (this.valueTypePtr != that.valueTypePtr) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_keyTypePtr = true; - builder.append(present_keyTypePtr); - if (present_keyTypePtr) - builder.append(keyTypePtr); - - boolean present_valueTypePtr = true; - builder.append(present_valueTypePtr); - if (present_valueTypePtr) - builder.append(valueTypePtr); - - return builder.toHashCode(); - } - - public int compareTo(TMapTypeEntry other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TMapTypeEntry typedOther = (TMapTypeEntry)other; - - lastComparison = Boolean.valueOf(isSetKeyTypePtr()).compareTo(typedOther.isSetKeyTypePtr()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetKeyTypePtr()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.keyTypePtr, typedOther.keyTypePtr); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetValueTypePtr()).compareTo(typedOther.isSetValueTypePtr()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValueTypePtr()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.valueTypePtr, typedOther.valueTypePtr); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TMapTypeEntry("); - boolean first = true; - - sb.append("keyTypePtr:"); - sb.append(this.keyTypePtr); - first = false; - if (!first) sb.append(", "); - sb.append("valueTypePtr:"); - sb.append(this.valueTypePtr); - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetKeyTypePtr()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'keyTypePtr' is unset! Struct:" + toString()); - } - - if (!isSetValueTypePtr()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'valueTypePtr' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TMapTypeEntryStandardSchemeFactory implements SchemeFactory { - public TMapTypeEntryStandardScheme getScheme() { - return new TMapTypeEntryStandardScheme(); - } - } - - private static class TMapTypeEntryStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TMapTypeEntry struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // KEY_TYPE_PTR - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.keyTypePtr = iprot.readI32(); - struct.setKeyTypePtrIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // VALUE_TYPE_PTR - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.valueTypePtr = iprot.readI32(); - struct.setValueTypePtrIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TMapTypeEntry struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(KEY_TYPE_PTR_FIELD_DESC); - oprot.writeI32(struct.keyTypePtr); - oprot.writeFieldEnd(); - oprot.writeFieldBegin(VALUE_TYPE_PTR_FIELD_DESC); - oprot.writeI32(struct.valueTypePtr); - oprot.writeFieldEnd(); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TMapTypeEntryTupleSchemeFactory implements SchemeFactory { - public TMapTypeEntryTupleScheme getScheme() { - return new TMapTypeEntryTupleScheme(); - } - } - - private static class TMapTypeEntryTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TMapTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeI32(struct.keyTypePtr); - oprot.writeI32(struct.valueTypePtr); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TMapTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.keyTypePtr = iprot.readI32(); - struct.setKeyTypePtrIsSet(true); - struct.valueTypePtr = iprot.readI32(); - struct.setValueTypePtrIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java deleted file mode 100644 index c0481615b06d3..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java +++ /dev/null @@ -1,785 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TOpenSessionReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOpenSessionReq"); - - private static final org.apache.thrift.protocol.TField CLIENT_PROTOCOL_FIELD_DESC = new org.apache.thrift.protocol.TField("client_protocol", org.apache.thrift.protocol.TType.I32, (short)1); - private static final org.apache.thrift.protocol.TField USERNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("username", org.apache.thrift.protocol.TType.STRING, (short)2); - private static final org.apache.thrift.protocol.TField PASSWORD_FIELD_DESC = new org.apache.thrift.protocol.TField("password", org.apache.thrift.protocol.TType.STRING, (short)3); - private static final org.apache.thrift.protocol.TField CONFIGURATION_FIELD_DESC = new org.apache.thrift.protocol.TField("configuration", org.apache.thrift.protocol.TType.MAP, (short)4); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TOpenSessionReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TOpenSessionReqTupleSchemeFactory()); - } - - private TProtocolVersion client_protocol; // required - private String username; // optional - private String password; // optional - private Map configuration; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - /** - * - * @see TProtocolVersion - */ - CLIENT_PROTOCOL((short)1, "client_protocol"), - USERNAME((short)2, "username"), - PASSWORD((short)3, "password"), - CONFIGURATION((short)4, "configuration"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // CLIENT_PROTOCOL - return CLIENT_PROTOCOL; - case 2: // USERNAME - return USERNAME; - case 3: // PASSWORD - return PASSWORD; - case 4: // CONFIGURATION - return CONFIGURATION; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.USERNAME,_Fields.PASSWORD,_Fields.CONFIGURATION}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.CLIENT_PROTOCOL, new org.apache.thrift.meta_data.FieldMetaData("client_protocol", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TProtocolVersion.class))); - tmpMap.put(_Fields.USERNAME, new org.apache.thrift.meta_data.FieldMetaData("username", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.PASSWORD, new org.apache.thrift.meta_data.FieldMetaData("password", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.CONFIGURATION, new org.apache.thrift.meta_data.FieldMetaData("configuration", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TOpenSessionReq.class, metaDataMap); - } - - public TOpenSessionReq() { - this.client_protocol = org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8; - - } - - public TOpenSessionReq( - TProtocolVersion client_protocol) - { - this(); - this.client_protocol = client_protocol; - } - - /** - * Performs a deep copy on other. - */ - public TOpenSessionReq(TOpenSessionReq other) { - if (other.isSetClient_protocol()) { - this.client_protocol = other.client_protocol; - } - if (other.isSetUsername()) { - this.username = other.username; - } - if (other.isSetPassword()) { - this.password = other.password; - } - if (other.isSetConfiguration()) { - Map __this__configuration = new HashMap(); - for (Map.Entry other_element : other.configuration.entrySet()) { - - String other_element_key = other_element.getKey(); - String other_element_value = other_element.getValue(); - - String __this__configuration_copy_key = other_element_key; - - String __this__configuration_copy_value = other_element_value; - - __this__configuration.put(__this__configuration_copy_key, __this__configuration_copy_value); - } - this.configuration = __this__configuration; - } - } - - public TOpenSessionReq deepCopy() { - return new TOpenSessionReq(this); - } - - @Override - public void clear() { - this.client_protocol = org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8; - - this.username = null; - this.password = null; - this.configuration = null; - } - - /** - * - * @see TProtocolVersion - */ - public TProtocolVersion getClient_protocol() { - return this.client_protocol; - } - - /** - * - * @see TProtocolVersion - */ - public void setClient_protocol(TProtocolVersion client_protocol) { - this.client_protocol = client_protocol; - } - - public void unsetClient_protocol() { - this.client_protocol = null; - } - - /** Returns true if field client_protocol is set (has been assigned a value) and false otherwise */ - public boolean isSetClient_protocol() { - return this.client_protocol != null; - } - - public void setClient_protocolIsSet(boolean value) { - if (!value) { - this.client_protocol = null; - } - } - - public String getUsername() { - return this.username; - } - - public void setUsername(String username) { - this.username = username; - } - - public void unsetUsername() { - this.username = null; - } - - /** Returns true if field username is set (has been assigned a value) and false otherwise */ - public boolean isSetUsername() { - return this.username != null; - } - - public void setUsernameIsSet(boolean value) { - if (!value) { - this.username = null; - } - } - - public String getPassword() { - return this.password; - } - - public void setPassword(String password) { - this.password = password; - } - - public void unsetPassword() { - this.password = null; - } - - /** Returns true if field password is set (has been assigned a value) and false otherwise */ - public boolean isSetPassword() { - return this.password != null; - } - - public void setPasswordIsSet(boolean value) { - if (!value) { - this.password = null; - } - } - - public int getConfigurationSize() { - return (this.configuration == null) ? 0 : this.configuration.size(); - } - - public void putToConfiguration(String key, String val) { - if (this.configuration == null) { - this.configuration = new HashMap(); - } - this.configuration.put(key, val); - } - - public Map getConfiguration() { - return this.configuration; - } - - public void setConfiguration(Map configuration) { - this.configuration = configuration; - } - - public void unsetConfiguration() { - this.configuration = null; - } - - /** Returns true if field configuration is set (has been assigned a value) and false otherwise */ - public boolean isSetConfiguration() { - return this.configuration != null; - } - - public void setConfigurationIsSet(boolean value) { - if (!value) { - this.configuration = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case CLIENT_PROTOCOL: - if (value == null) { - unsetClient_protocol(); - } else { - setClient_protocol((TProtocolVersion)value); - } - break; - - case USERNAME: - if (value == null) { - unsetUsername(); - } else { - setUsername((String)value); - } - break; - - case PASSWORD: - if (value == null) { - unsetPassword(); - } else { - setPassword((String)value); - } - break; - - case CONFIGURATION: - if (value == null) { - unsetConfiguration(); - } else { - setConfiguration((Map)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case CLIENT_PROTOCOL: - return getClient_protocol(); - - case USERNAME: - return getUsername(); - - case PASSWORD: - return getPassword(); - - case CONFIGURATION: - return getConfiguration(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case CLIENT_PROTOCOL: - return isSetClient_protocol(); - case USERNAME: - return isSetUsername(); - case PASSWORD: - return isSetPassword(); - case CONFIGURATION: - return isSetConfiguration(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TOpenSessionReq) - return this.equals((TOpenSessionReq)that); - return false; - } - - public boolean equals(TOpenSessionReq that) { - if (that == null) - return false; - - boolean this_present_client_protocol = true && this.isSetClient_protocol(); - boolean that_present_client_protocol = true && that.isSetClient_protocol(); - if (this_present_client_protocol || that_present_client_protocol) { - if (!(this_present_client_protocol && that_present_client_protocol)) - return false; - if (!this.client_protocol.equals(that.client_protocol)) - return false; - } - - boolean this_present_username = true && this.isSetUsername(); - boolean that_present_username = true && that.isSetUsername(); - if (this_present_username || that_present_username) { - if (!(this_present_username && that_present_username)) - return false; - if (!this.username.equals(that.username)) - return false; - } - - boolean this_present_password = true && this.isSetPassword(); - boolean that_present_password = true && that.isSetPassword(); - if (this_present_password || that_present_password) { - if (!(this_present_password && that_present_password)) - return false; - if (!this.password.equals(that.password)) - return false; - } - - boolean this_present_configuration = true && this.isSetConfiguration(); - boolean that_present_configuration = true && that.isSetConfiguration(); - if (this_present_configuration || that_present_configuration) { - if (!(this_present_configuration && that_present_configuration)) - return false; - if (!this.configuration.equals(that.configuration)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_client_protocol = true && (isSetClient_protocol()); - builder.append(present_client_protocol); - if (present_client_protocol) - builder.append(client_protocol.getValue()); - - boolean present_username = true && (isSetUsername()); - builder.append(present_username); - if (present_username) - builder.append(username); - - boolean present_password = true && (isSetPassword()); - builder.append(present_password); - if (present_password) - builder.append(password); - - boolean present_configuration = true && (isSetConfiguration()); - builder.append(present_configuration); - if (present_configuration) - builder.append(configuration); - - return builder.toHashCode(); - } - - public int compareTo(TOpenSessionReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TOpenSessionReq typedOther = (TOpenSessionReq)other; - - lastComparison = Boolean.valueOf(isSetClient_protocol()).compareTo(typedOther.isSetClient_protocol()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetClient_protocol()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.client_protocol, typedOther.client_protocol); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetUsername()).compareTo(typedOther.isSetUsername()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetUsername()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.username, typedOther.username); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetPassword()).compareTo(typedOther.isSetPassword()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetPassword()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.password, typedOther.password); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetConfiguration()).compareTo(typedOther.isSetConfiguration()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetConfiguration()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.configuration, typedOther.configuration); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TOpenSessionReq("); - boolean first = true; - - sb.append("client_protocol:"); - if (this.client_protocol == null) { - sb.append("null"); - } else { - sb.append(this.client_protocol); - } - first = false; - if (isSetUsername()) { - if (!first) sb.append(", "); - sb.append("username:"); - if (this.username == null) { - sb.append("null"); - } else { - sb.append(this.username); - } - first = false; - } - if (isSetPassword()) { - if (!first) sb.append(", "); - sb.append("password:"); - if (this.password == null) { - sb.append("null"); - } else { - sb.append(this.password); - } - first = false; - } - if (isSetConfiguration()) { - if (!first) sb.append(", "); - sb.append("configuration:"); - if (this.configuration == null) { - sb.append("null"); - } else { - sb.append(this.configuration); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetClient_protocol()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'client_protocol' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TOpenSessionReqStandardSchemeFactory implements SchemeFactory { - public TOpenSessionReqStandardScheme getScheme() { - return new TOpenSessionReqStandardScheme(); - } - } - - private static class TOpenSessionReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TOpenSessionReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // CLIENT_PROTOCOL - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.client_protocol = TProtocolVersion.findByValue(iprot.readI32()); - struct.setClient_protocolIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // USERNAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.username = iprot.readString(); - struct.setUsernameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // PASSWORD - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.password = iprot.readString(); - struct.setPasswordIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // CONFIGURATION - if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { - { - org.apache.thrift.protocol.TMap _map142 = iprot.readMapBegin(); - struct.configuration = new HashMap(2*_map142.size); - for (int _i143 = 0; _i143 < _map142.size; ++_i143) - { - String _key144; // required - String _val145; // required - _key144 = iprot.readString(); - _val145 = iprot.readString(); - struct.configuration.put(_key144, _val145); - } - iprot.readMapEnd(); - } - struct.setConfigurationIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TOpenSessionReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.client_protocol != null) { - oprot.writeFieldBegin(CLIENT_PROTOCOL_FIELD_DESC); - oprot.writeI32(struct.client_protocol.getValue()); - oprot.writeFieldEnd(); - } - if (struct.username != null) { - if (struct.isSetUsername()) { - oprot.writeFieldBegin(USERNAME_FIELD_DESC); - oprot.writeString(struct.username); - oprot.writeFieldEnd(); - } - } - if (struct.password != null) { - if (struct.isSetPassword()) { - oprot.writeFieldBegin(PASSWORD_FIELD_DESC); - oprot.writeString(struct.password); - oprot.writeFieldEnd(); - } - } - if (struct.configuration != null) { - if (struct.isSetConfiguration()) { - oprot.writeFieldBegin(CONFIGURATION_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.configuration.size())); - for (Map.Entry _iter146 : struct.configuration.entrySet()) - { - oprot.writeString(_iter146.getKey()); - oprot.writeString(_iter146.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TOpenSessionReqTupleSchemeFactory implements SchemeFactory { - public TOpenSessionReqTupleScheme getScheme() { - return new TOpenSessionReqTupleScheme(); - } - } - - private static class TOpenSessionReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TOpenSessionReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeI32(struct.client_protocol.getValue()); - BitSet optionals = new BitSet(); - if (struct.isSetUsername()) { - optionals.set(0); - } - if (struct.isSetPassword()) { - optionals.set(1); - } - if (struct.isSetConfiguration()) { - optionals.set(2); - } - oprot.writeBitSet(optionals, 3); - if (struct.isSetUsername()) { - oprot.writeString(struct.username); - } - if (struct.isSetPassword()) { - oprot.writeString(struct.password); - } - if (struct.isSetConfiguration()) { - { - oprot.writeI32(struct.configuration.size()); - for (Map.Entry _iter147 : struct.configuration.entrySet()) - { - oprot.writeString(_iter147.getKey()); - oprot.writeString(_iter147.getValue()); - } - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TOpenSessionReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.client_protocol = TProtocolVersion.findByValue(iprot.readI32()); - struct.setClient_protocolIsSet(true); - BitSet incoming = iprot.readBitSet(3); - if (incoming.get(0)) { - struct.username = iprot.readString(); - struct.setUsernameIsSet(true); - } - if (incoming.get(1)) { - struct.password = iprot.readString(); - struct.setPasswordIsSet(true); - } - if (incoming.get(2)) { - { - org.apache.thrift.protocol.TMap _map148 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.configuration = new HashMap(2*_map148.size); - for (int _i149 = 0; _i149 < _map148.size; ++_i149) - { - String _key150; // required - String _val151; // required - _key150 = iprot.readString(); - _val151 = iprot.readString(); - struct.configuration.put(_key150, _val151); - } - } - struct.setConfigurationIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java deleted file mode 100644 index 351f78b2de20c..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java +++ /dev/null @@ -1,790 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TOpenSessionResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOpenSessionResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField SERVER_PROTOCOL_VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("serverProtocolVersion", org.apache.thrift.protocol.TType.I32, (short)2); - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)3); - private static final org.apache.thrift.protocol.TField CONFIGURATION_FIELD_DESC = new org.apache.thrift.protocol.TField("configuration", org.apache.thrift.protocol.TType.MAP, (short)4); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TOpenSessionRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TOpenSessionRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TProtocolVersion serverProtocolVersion; // required - private TSessionHandle sessionHandle; // optional - private Map configuration; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - /** - * - * @see TProtocolVersion - */ - SERVER_PROTOCOL_VERSION((short)2, "serverProtocolVersion"), - SESSION_HANDLE((short)3, "sessionHandle"), - CONFIGURATION((short)4, "configuration"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // SERVER_PROTOCOL_VERSION - return SERVER_PROTOCOL_VERSION; - case 3: // SESSION_HANDLE - return SESSION_HANDLE; - case 4: // CONFIGURATION - return CONFIGURATION; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.SESSION_HANDLE,_Fields.CONFIGURATION}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.SERVER_PROTOCOL_VERSION, new org.apache.thrift.meta_data.FieldMetaData("serverProtocolVersion", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TProtocolVersion.class))); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.CONFIGURATION, new org.apache.thrift.meta_data.FieldMetaData("configuration", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TOpenSessionResp.class, metaDataMap); - } - - public TOpenSessionResp() { - this.serverProtocolVersion = org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8; - - } - - public TOpenSessionResp( - TStatus status, - TProtocolVersion serverProtocolVersion) - { - this(); - this.status = status; - this.serverProtocolVersion = serverProtocolVersion; - } - - /** - * Performs a deep copy on other. - */ - public TOpenSessionResp(TOpenSessionResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetServerProtocolVersion()) { - this.serverProtocolVersion = other.serverProtocolVersion; - } - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetConfiguration()) { - Map __this__configuration = new HashMap(); - for (Map.Entry other_element : other.configuration.entrySet()) { - - String other_element_key = other_element.getKey(); - String other_element_value = other_element.getValue(); - - String __this__configuration_copy_key = other_element_key; - - String __this__configuration_copy_value = other_element_value; - - __this__configuration.put(__this__configuration_copy_key, __this__configuration_copy_value); - } - this.configuration = __this__configuration; - } - } - - public TOpenSessionResp deepCopy() { - return new TOpenSessionResp(this); - } - - @Override - public void clear() { - this.status = null; - this.serverProtocolVersion = org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8; - - this.sessionHandle = null; - this.configuration = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - /** - * - * @see TProtocolVersion - */ - public TProtocolVersion getServerProtocolVersion() { - return this.serverProtocolVersion; - } - - /** - * - * @see TProtocolVersion - */ - public void setServerProtocolVersion(TProtocolVersion serverProtocolVersion) { - this.serverProtocolVersion = serverProtocolVersion; - } - - public void unsetServerProtocolVersion() { - this.serverProtocolVersion = null; - } - - /** Returns true if field serverProtocolVersion is set (has been assigned a value) and false otherwise */ - public boolean isSetServerProtocolVersion() { - return this.serverProtocolVersion != null; - } - - public void setServerProtocolVersionIsSet(boolean value) { - if (!value) { - this.serverProtocolVersion = null; - } - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public int getConfigurationSize() { - return (this.configuration == null) ? 0 : this.configuration.size(); - } - - public void putToConfiguration(String key, String val) { - if (this.configuration == null) { - this.configuration = new HashMap(); - } - this.configuration.put(key, val); - } - - public Map getConfiguration() { - return this.configuration; - } - - public void setConfiguration(Map configuration) { - this.configuration = configuration; - } - - public void unsetConfiguration() { - this.configuration = null; - } - - /** Returns true if field configuration is set (has been assigned a value) and false otherwise */ - public boolean isSetConfiguration() { - return this.configuration != null; - } - - public void setConfigurationIsSet(boolean value) { - if (!value) { - this.configuration = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case SERVER_PROTOCOL_VERSION: - if (value == null) { - unsetServerProtocolVersion(); - } else { - setServerProtocolVersion((TProtocolVersion)value); - } - break; - - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case CONFIGURATION: - if (value == null) { - unsetConfiguration(); - } else { - setConfiguration((Map)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case SERVER_PROTOCOL_VERSION: - return getServerProtocolVersion(); - - case SESSION_HANDLE: - return getSessionHandle(); - - case CONFIGURATION: - return getConfiguration(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case SERVER_PROTOCOL_VERSION: - return isSetServerProtocolVersion(); - case SESSION_HANDLE: - return isSetSessionHandle(); - case CONFIGURATION: - return isSetConfiguration(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TOpenSessionResp) - return this.equals((TOpenSessionResp)that); - return false; - } - - public boolean equals(TOpenSessionResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_serverProtocolVersion = true && this.isSetServerProtocolVersion(); - boolean that_present_serverProtocolVersion = true && that.isSetServerProtocolVersion(); - if (this_present_serverProtocolVersion || that_present_serverProtocolVersion) { - if (!(this_present_serverProtocolVersion && that_present_serverProtocolVersion)) - return false; - if (!this.serverProtocolVersion.equals(that.serverProtocolVersion)) - return false; - } - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_configuration = true && this.isSetConfiguration(); - boolean that_present_configuration = true && that.isSetConfiguration(); - if (this_present_configuration || that_present_configuration) { - if (!(this_present_configuration && that_present_configuration)) - return false; - if (!this.configuration.equals(that.configuration)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_serverProtocolVersion = true && (isSetServerProtocolVersion()); - builder.append(present_serverProtocolVersion); - if (present_serverProtocolVersion) - builder.append(serverProtocolVersion.getValue()); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_configuration = true && (isSetConfiguration()); - builder.append(present_configuration); - if (present_configuration) - builder.append(configuration); - - return builder.toHashCode(); - } - - public int compareTo(TOpenSessionResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TOpenSessionResp typedOther = (TOpenSessionResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetServerProtocolVersion()).compareTo(typedOther.isSetServerProtocolVersion()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetServerProtocolVersion()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.serverProtocolVersion, typedOther.serverProtocolVersion); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetConfiguration()).compareTo(typedOther.isSetConfiguration()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetConfiguration()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.configuration, typedOther.configuration); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TOpenSessionResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (!first) sb.append(", "); - sb.append("serverProtocolVersion:"); - if (this.serverProtocolVersion == null) { - sb.append("null"); - } else { - sb.append(this.serverProtocolVersion); - } - first = false; - if (isSetSessionHandle()) { - if (!first) sb.append(", "); - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - } - if (isSetConfiguration()) { - if (!first) sb.append(", "); - sb.append("configuration:"); - if (this.configuration == null) { - sb.append("null"); - } else { - sb.append(this.configuration); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - if (!isSetServerProtocolVersion()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'serverProtocolVersion' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TOpenSessionRespStandardSchemeFactory implements SchemeFactory { - public TOpenSessionRespStandardScheme getScheme() { - return new TOpenSessionRespStandardScheme(); - } - } - - private static class TOpenSessionRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TOpenSessionResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // SERVER_PROTOCOL_VERSION - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.serverProtocolVersion = TProtocolVersion.findByValue(iprot.readI32()); - struct.setServerProtocolVersionIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // CONFIGURATION - if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { - { - org.apache.thrift.protocol.TMap _map152 = iprot.readMapBegin(); - struct.configuration = new HashMap(2*_map152.size); - for (int _i153 = 0; _i153 < _map152.size; ++_i153) - { - String _key154; // required - String _val155; // required - _key154 = iprot.readString(); - _val155 = iprot.readString(); - struct.configuration.put(_key154, _val155); - } - iprot.readMapEnd(); - } - struct.setConfigurationIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TOpenSessionResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.serverProtocolVersion != null) { - oprot.writeFieldBegin(SERVER_PROTOCOL_VERSION_FIELD_DESC); - oprot.writeI32(struct.serverProtocolVersion.getValue()); - oprot.writeFieldEnd(); - } - if (struct.sessionHandle != null) { - if (struct.isSetSessionHandle()) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - if (struct.configuration != null) { - if (struct.isSetConfiguration()) { - oprot.writeFieldBegin(CONFIGURATION_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.configuration.size())); - for (Map.Entry _iter156 : struct.configuration.entrySet()) - { - oprot.writeString(_iter156.getKey()); - oprot.writeString(_iter156.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TOpenSessionRespTupleSchemeFactory implements SchemeFactory { - public TOpenSessionRespTupleScheme getScheme() { - return new TOpenSessionRespTupleScheme(); - } - } - - private static class TOpenSessionRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TOpenSessionResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - oprot.writeI32(struct.serverProtocolVersion.getValue()); - BitSet optionals = new BitSet(); - if (struct.isSetSessionHandle()) { - optionals.set(0); - } - if (struct.isSetConfiguration()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); - if (struct.isSetSessionHandle()) { - struct.sessionHandle.write(oprot); - } - if (struct.isSetConfiguration()) { - { - oprot.writeI32(struct.configuration.size()); - for (Map.Entry _iter157 : struct.configuration.entrySet()) - { - oprot.writeString(_iter157.getKey()); - oprot.writeString(_iter157.getValue()); - } - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TOpenSessionResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - struct.serverProtocolVersion = TProtocolVersion.findByValue(iprot.readI32()); - struct.setServerProtocolVersionIsSet(true); - BitSet incoming = iprot.readBitSet(2); - if (incoming.get(0)) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } - if (incoming.get(1)) { - { - org.apache.thrift.protocol.TMap _map158 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.configuration = new HashMap(2*_map158.size); - for (int _i159 = 0; _i159 < _map158.size; ++_i159) - { - String _key160; // required - String _val161; // required - _key160 = iprot.readString(); - _val161 = iprot.readString(); - struct.configuration.put(_key160, _val161); - } - } - struct.setConfigurationIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java deleted file mode 100644 index 8fbd8752eaca6..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java +++ /dev/null @@ -1,705 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TOperationHandle implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOperationHandle"); - - private static final org.apache.thrift.protocol.TField OPERATION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("operationId", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationType", org.apache.thrift.protocol.TType.I32, (short)2); - private static final org.apache.thrift.protocol.TField HAS_RESULT_SET_FIELD_DESC = new org.apache.thrift.protocol.TField("hasResultSet", org.apache.thrift.protocol.TType.BOOL, (short)3); - private static final org.apache.thrift.protocol.TField MODIFIED_ROW_COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("modifiedRowCount", org.apache.thrift.protocol.TType.DOUBLE, (short)4); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TOperationHandleStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TOperationHandleTupleSchemeFactory()); - } - - private THandleIdentifier operationId; // required - private TOperationType operationType; // required - private boolean hasResultSet; // required - private double modifiedRowCount; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - OPERATION_ID((short)1, "operationId"), - /** - * - * @see TOperationType - */ - OPERATION_TYPE((short)2, "operationType"), - HAS_RESULT_SET((short)3, "hasResultSet"), - MODIFIED_ROW_COUNT((short)4, "modifiedRowCount"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // OPERATION_ID - return OPERATION_ID; - case 2: // OPERATION_TYPE - return OPERATION_TYPE; - case 3: // HAS_RESULT_SET - return HAS_RESULT_SET; - case 4: // MODIFIED_ROW_COUNT - return MODIFIED_ROW_COUNT; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __HASRESULTSET_ISSET_ID = 0; - private static final int __MODIFIEDROWCOUNT_ISSET_ID = 1; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.MODIFIED_ROW_COUNT}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.OPERATION_ID, new org.apache.thrift.meta_data.FieldMetaData("operationId", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, THandleIdentifier.class))); - tmpMap.put(_Fields.OPERATION_TYPE, new org.apache.thrift.meta_data.FieldMetaData("operationType", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TOperationType.class))); - tmpMap.put(_Fields.HAS_RESULT_SET, new org.apache.thrift.meta_data.FieldMetaData("hasResultSet", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); - tmpMap.put(_Fields.MODIFIED_ROW_COUNT, new org.apache.thrift.meta_data.FieldMetaData("modifiedRowCount", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TOperationHandle.class, metaDataMap); - } - - public TOperationHandle() { - } - - public TOperationHandle( - THandleIdentifier operationId, - TOperationType operationType, - boolean hasResultSet) - { - this(); - this.operationId = operationId; - this.operationType = operationType; - this.hasResultSet = hasResultSet; - setHasResultSetIsSet(true); - } - - /** - * Performs a deep copy on other. - */ - public TOperationHandle(TOperationHandle other) { - __isset_bitfield = other.__isset_bitfield; - if (other.isSetOperationId()) { - this.operationId = new THandleIdentifier(other.operationId); - } - if (other.isSetOperationType()) { - this.operationType = other.operationType; - } - this.hasResultSet = other.hasResultSet; - this.modifiedRowCount = other.modifiedRowCount; - } - - public TOperationHandle deepCopy() { - return new TOperationHandle(this); - } - - @Override - public void clear() { - this.operationId = null; - this.operationType = null; - setHasResultSetIsSet(false); - this.hasResultSet = false; - setModifiedRowCountIsSet(false); - this.modifiedRowCount = 0.0; - } - - public THandleIdentifier getOperationId() { - return this.operationId; - } - - public void setOperationId(THandleIdentifier operationId) { - this.operationId = operationId; - } - - public void unsetOperationId() { - this.operationId = null; - } - - /** Returns true if field operationId is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationId() { - return this.operationId != null; - } - - public void setOperationIdIsSet(boolean value) { - if (!value) { - this.operationId = null; - } - } - - /** - * - * @see TOperationType - */ - public TOperationType getOperationType() { - return this.operationType; - } - - /** - * - * @see TOperationType - */ - public void setOperationType(TOperationType operationType) { - this.operationType = operationType; - } - - public void unsetOperationType() { - this.operationType = null; - } - - /** Returns true if field operationType is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationType() { - return this.operationType != null; - } - - public void setOperationTypeIsSet(boolean value) { - if (!value) { - this.operationType = null; - } - } - - public boolean isHasResultSet() { - return this.hasResultSet; - } - - public void setHasResultSet(boolean hasResultSet) { - this.hasResultSet = hasResultSet; - setHasResultSetIsSet(true); - } - - public void unsetHasResultSet() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __HASRESULTSET_ISSET_ID); - } - - /** Returns true if field hasResultSet is set (has been assigned a value) and false otherwise */ - public boolean isSetHasResultSet() { - return EncodingUtils.testBit(__isset_bitfield, __HASRESULTSET_ISSET_ID); - } - - public void setHasResultSetIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __HASRESULTSET_ISSET_ID, value); - } - - public double getModifiedRowCount() { - return this.modifiedRowCount; - } - - public void setModifiedRowCount(double modifiedRowCount) { - this.modifiedRowCount = modifiedRowCount; - setModifiedRowCountIsSet(true); - } - - public void unsetModifiedRowCount() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MODIFIEDROWCOUNT_ISSET_ID); - } - - /** Returns true if field modifiedRowCount is set (has been assigned a value) and false otherwise */ - public boolean isSetModifiedRowCount() { - return EncodingUtils.testBit(__isset_bitfield, __MODIFIEDROWCOUNT_ISSET_ID); - } - - public void setModifiedRowCountIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MODIFIEDROWCOUNT_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case OPERATION_ID: - if (value == null) { - unsetOperationId(); - } else { - setOperationId((THandleIdentifier)value); - } - break; - - case OPERATION_TYPE: - if (value == null) { - unsetOperationType(); - } else { - setOperationType((TOperationType)value); - } - break; - - case HAS_RESULT_SET: - if (value == null) { - unsetHasResultSet(); - } else { - setHasResultSet((Boolean)value); - } - break; - - case MODIFIED_ROW_COUNT: - if (value == null) { - unsetModifiedRowCount(); - } else { - setModifiedRowCount((Double)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case OPERATION_ID: - return getOperationId(); - - case OPERATION_TYPE: - return getOperationType(); - - case HAS_RESULT_SET: - return Boolean.valueOf(isHasResultSet()); - - case MODIFIED_ROW_COUNT: - return Double.valueOf(getModifiedRowCount()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case OPERATION_ID: - return isSetOperationId(); - case OPERATION_TYPE: - return isSetOperationType(); - case HAS_RESULT_SET: - return isSetHasResultSet(); - case MODIFIED_ROW_COUNT: - return isSetModifiedRowCount(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TOperationHandle) - return this.equals((TOperationHandle)that); - return false; - } - - public boolean equals(TOperationHandle that) { - if (that == null) - return false; - - boolean this_present_operationId = true && this.isSetOperationId(); - boolean that_present_operationId = true && that.isSetOperationId(); - if (this_present_operationId || that_present_operationId) { - if (!(this_present_operationId && that_present_operationId)) - return false; - if (!this.operationId.equals(that.operationId)) - return false; - } - - boolean this_present_operationType = true && this.isSetOperationType(); - boolean that_present_operationType = true && that.isSetOperationType(); - if (this_present_operationType || that_present_operationType) { - if (!(this_present_operationType && that_present_operationType)) - return false; - if (!this.operationType.equals(that.operationType)) - return false; - } - - boolean this_present_hasResultSet = true; - boolean that_present_hasResultSet = true; - if (this_present_hasResultSet || that_present_hasResultSet) { - if (!(this_present_hasResultSet && that_present_hasResultSet)) - return false; - if (this.hasResultSet != that.hasResultSet) - return false; - } - - boolean this_present_modifiedRowCount = true && this.isSetModifiedRowCount(); - boolean that_present_modifiedRowCount = true && that.isSetModifiedRowCount(); - if (this_present_modifiedRowCount || that_present_modifiedRowCount) { - if (!(this_present_modifiedRowCount && that_present_modifiedRowCount)) - return false; - if (this.modifiedRowCount != that.modifiedRowCount) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_operationId = true && (isSetOperationId()); - builder.append(present_operationId); - if (present_operationId) - builder.append(operationId); - - boolean present_operationType = true && (isSetOperationType()); - builder.append(present_operationType); - if (present_operationType) - builder.append(operationType.getValue()); - - boolean present_hasResultSet = true; - builder.append(present_hasResultSet); - if (present_hasResultSet) - builder.append(hasResultSet); - - boolean present_modifiedRowCount = true && (isSetModifiedRowCount()); - builder.append(present_modifiedRowCount); - if (present_modifiedRowCount) - builder.append(modifiedRowCount); - - return builder.toHashCode(); - } - - public int compareTo(TOperationHandle other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TOperationHandle typedOther = (TOperationHandle)other; - - lastComparison = Boolean.valueOf(isSetOperationId()).compareTo(typedOther.isSetOperationId()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationId, typedOther.operationId); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationType()).compareTo(typedOther.isSetOperationType()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationType()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationType, typedOther.operationType); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetHasResultSet()).compareTo(typedOther.isSetHasResultSet()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetHasResultSet()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hasResultSet, typedOther.hasResultSet); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetModifiedRowCount()).compareTo(typedOther.isSetModifiedRowCount()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetModifiedRowCount()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.modifiedRowCount, typedOther.modifiedRowCount); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TOperationHandle("); - boolean first = true; - - sb.append("operationId:"); - if (this.operationId == null) { - sb.append("null"); - } else { - sb.append(this.operationId); - } - first = false; - if (!first) sb.append(", "); - sb.append("operationType:"); - if (this.operationType == null) { - sb.append("null"); - } else { - sb.append(this.operationType); - } - first = false; - if (!first) sb.append(", "); - sb.append("hasResultSet:"); - sb.append(this.hasResultSet); - first = false; - if (isSetModifiedRowCount()) { - if (!first) sb.append(", "); - sb.append("modifiedRowCount:"); - sb.append(this.modifiedRowCount); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetOperationId()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationId' is unset! Struct:" + toString()); - } - - if (!isSetOperationType()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationType' is unset! Struct:" + toString()); - } - - if (!isSetHasResultSet()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'hasResultSet' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (operationId != null) { - operationId.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TOperationHandleStandardSchemeFactory implements SchemeFactory { - public TOperationHandleStandardScheme getScheme() { - return new TOperationHandleStandardScheme(); - } - } - - private static class TOperationHandleStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TOperationHandle struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // OPERATION_ID - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationId = new THandleIdentifier(); - struct.operationId.read(iprot); - struct.setOperationIdIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_TYPE - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.operationType = TOperationType.findByValue(iprot.readI32()); - struct.setOperationTypeIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // HAS_RESULT_SET - if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { - struct.hasResultSet = iprot.readBool(); - struct.setHasResultSetIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // MODIFIED_ROW_COUNT - if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { - struct.modifiedRowCount = iprot.readDouble(); - struct.setModifiedRowCountIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TOperationHandle struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.operationId != null) { - oprot.writeFieldBegin(OPERATION_ID_FIELD_DESC); - struct.operationId.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationType != null) { - oprot.writeFieldBegin(OPERATION_TYPE_FIELD_DESC); - oprot.writeI32(struct.operationType.getValue()); - oprot.writeFieldEnd(); - } - oprot.writeFieldBegin(HAS_RESULT_SET_FIELD_DESC); - oprot.writeBool(struct.hasResultSet); - oprot.writeFieldEnd(); - if (struct.isSetModifiedRowCount()) { - oprot.writeFieldBegin(MODIFIED_ROW_COUNT_FIELD_DESC); - oprot.writeDouble(struct.modifiedRowCount); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TOperationHandleTupleSchemeFactory implements SchemeFactory { - public TOperationHandleTupleScheme getScheme() { - return new TOperationHandleTupleScheme(); - } - } - - private static class TOperationHandleTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TOperationHandle struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.operationId.write(oprot); - oprot.writeI32(struct.operationType.getValue()); - oprot.writeBool(struct.hasResultSet); - BitSet optionals = new BitSet(); - if (struct.isSetModifiedRowCount()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetModifiedRowCount()) { - oprot.writeDouble(struct.modifiedRowCount); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TOperationHandle struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.operationId = new THandleIdentifier(); - struct.operationId.read(iprot); - struct.setOperationIdIsSet(true); - struct.operationType = TOperationType.findByValue(iprot.readI32()); - struct.setOperationTypeIsSet(true); - struct.hasResultSet = iprot.readBool(); - struct.setHasResultSetIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.modifiedRowCount = iprot.readDouble(); - struct.setModifiedRowCountIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java deleted file mode 100644 index 219866223a6b0..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java +++ /dev/null @@ -1,63 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - - -import java.util.Map; -import java.util.HashMap; -import org.apache.thrift.TEnum; - -public enum TOperationState implements org.apache.thrift.TEnum { - INITIALIZED_STATE(0), - RUNNING_STATE(1), - FINISHED_STATE(2), - CANCELED_STATE(3), - CLOSED_STATE(4), - ERROR_STATE(5), - UKNOWN_STATE(6), - PENDING_STATE(7); - - private final int value; - - private TOperationState(int value) { - this.value = value; - } - - /** - * Get the integer value of this enum value, as defined in the Thrift IDL. - */ - public int getValue() { - return value; - } - - /** - * Find a the enum type by its integer value, as defined in the Thrift IDL. - * @return null if the value is not found. - */ - public static TOperationState findByValue(int value) { - switch (value) { - case 0: - return INITIALIZED_STATE; - case 1: - return RUNNING_STATE; - case 2: - return FINISHED_STATE; - case 3: - return CANCELED_STATE; - case 4: - return CLOSED_STATE; - case 5: - return ERROR_STATE; - case 6: - return UKNOWN_STATE; - case 7: - return PENDING_STATE; - default: - return null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java deleted file mode 100644 index b6d4b2fab9f96..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java +++ /dev/null @@ -1,66 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - - -import java.util.Map; -import java.util.HashMap; -import org.apache.thrift.TEnum; - -public enum TOperationType implements org.apache.thrift.TEnum { - EXECUTE_STATEMENT(0), - GET_TYPE_INFO(1), - GET_CATALOGS(2), - GET_SCHEMAS(3), - GET_TABLES(4), - GET_TABLE_TYPES(5), - GET_COLUMNS(6), - GET_FUNCTIONS(7), - UNKNOWN(8); - - private final int value; - - private TOperationType(int value) { - this.value = value; - } - - /** - * Get the integer value of this enum value, as defined in the Thrift IDL. - */ - public int getValue() { - return value; - } - - /** - * Find a the enum type by its integer value, as defined in the Thrift IDL. - * @return null if the value is not found. - */ - public static TOperationType findByValue(int value) { - switch (value) { - case 0: - return EXECUTE_STATEMENT; - case 1: - return GET_TYPE_INFO; - case 2: - return GET_CATALOGS; - case 3: - return GET_SCHEMAS; - case 4: - return GET_TABLES; - case 5: - return GET_TABLE_TYPES; - case 6: - return GET_COLUMNS; - case 7: - return GET_FUNCTIONS; - case 8: - return UNKNOWN; - default: - return null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java deleted file mode 100644 index 9d2abf2b3b084..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java +++ /dev/null @@ -1,512 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TPrimitiveTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TPrimitiveTypeEntry"); - - private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1); - private static final org.apache.thrift.protocol.TField TYPE_QUALIFIERS_FIELD_DESC = new org.apache.thrift.protocol.TField("typeQualifiers", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TPrimitiveTypeEntryStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TPrimitiveTypeEntryTupleSchemeFactory()); - } - - private TTypeId type; // required - private TTypeQualifiers typeQualifiers; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - /** - * - * @see TTypeId - */ - TYPE((short)1, "type"), - TYPE_QUALIFIERS((short)2, "typeQualifiers"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // TYPE - return TYPE; - case 2: // TYPE_QUALIFIERS - return TYPE_QUALIFIERS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.TYPE_QUALIFIERS}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TTypeId.class))); - tmpMap.put(_Fields.TYPE_QUALIFIERS, new org.apache.thrift.meta_data.FieldMetaData("typeQualifiers", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTypeQualifiers.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TPrimitiveTypeEntry.class, metaDataMap); - } - - public TPrimitiveTypeEntry() { - } - - public TPrimitiveTypeEntry( - TTypeId type) - { - this(); - this.type = type; - } - - /** - * Performs a deep copy on other. - */ - public TPrimitiveTypeEntry(TPrimitiveTypeEntry other) { - if (other.isSetType()) { - this.type = other.type; - } - if (other.isSetTypeQualifiers()) { - this.typeQualifiers = new TTypeQualifiers(other.typeQualifiers); - } - } - - public TPrimitiveTypeEntry deepCopy() { - return new TPrimitiveTypeEntry(this); - } - - @Override - public void clear() { - this.type = null; - this.typeQualifiers = null; - } - - /** - * - * @see TTypeId - */ - public TTypeId getType() { - return this.type; - } - - /** - * - * @see TTypeId - */ - public void setType(TTypeId type) { - this.type = type; - } - - public void unsetType() { - this.type = null; - } - - /** Returns true if field type is set (has been assigned a value) and false otherwise */ - public boolean isSetType() { - return this.type != null; - } - - public void setTypeIsSet(boolean value) { - if (!value) { - this.type = null; - } - } - - public TTypeQualifiers getTypeQualifiers() { - return this.typeQualifiers; - } - - public void setTypeQualifiers(TTypeQualifiers typeQualifiers) { - this.typeQualifiers = typeQualifiers; - } - - public void unsetTypeQualifiers() { - this.typeQualifiers = null; - } - - /** Returns true if field typeQualifiers is set (has been assigned a value) and false otherwise */ - public boolean isSetTypeQualifiers() { - return this.typeQualifiers != null; - } - - public void setTypeQualifiersIsSet(boolean value) { - if (!value) { - this.typeQualifiers = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case TYPE: - if (value == null) { - unsetType(); - } else { - setType((TTypeId)value); - } - break; - - case TYPE_QUALIFIERS: - if (value == null) { - unsetTypeQualifiers(); - } else { - setTypeQualifiers((TTypeQualifiers)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case TYPE: - return getType(); - - case TYPE_QUALIFIERS: - return getTypeQualifiers(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case TYPE: - return isSetType(); - case TYPE_QUALIFIERS: - return isSetTypeQualifiers(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TPrimitiveTypeEntry) - return this.equals((TPrimitiveTypeEntry)that); - return false; - } - - public boolean equals(TPrimitiveTypeEntry that) { - if (that == null) - return false; - - boolean this_present_type = true && this.isSetType(); - boolean that_present_type = true && that.isSetType(); - if (this_present_type || that_present_type) { - if (!(this_present_type && that_present_type)) - return false; - if (!this.type.equals(that.type)) - return false; - } - - boolean this_present_typeQualifiers = true && this.isSetTypeQualifiers(); - boolean that_present_typeQualifiers = true && that.isSetTypeQualifiers(); - if (this_present_typeQualifiers || that_present_typeQualifiers) { - if (!(this_present_typeQualifiers && that_present_typeQualifiers)) - return false; - if (!this.typeQualifiers.equals(that.typeQualifiers)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_type = true && (isSetType()); - builder.append(present_type); - if (present_type) - builder.append(type.getValue()); - - boolean present_typeQualifiers = true && (isSetTypeQualifiers()); - builder.append(present_typeQualifiers); - if (present_typeQualifiers) - builder.append(typeQualifiers); - - return builder.toHashCode(); - } - - public int compareTo(TPrimitiveTypeEntry other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TPrimitiveTypeEntry typedOther = (TPrimitiveTypeEntry)other; - - lastComparison = Boolean.valueOf(isSetType()).compareTo(typedOther.isSetType()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetType()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, typedOther.type); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetTypeQualifiers()).compareTo(typedOther.isSetTypeQualifiers()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetTypeQualifiers()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.typeQualifiers, typedOther.typeQualifiers); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TPrimitiveTypeEntry("); - boolean first = true; - - sb.append("type:"); - if (this.type == null) { - sb.append("null"); - } else { - sb.append(this.type); - } - first = false; - if (isSetTypeQualifiers()) { - if (!first) sb.append(", "); - sb.append("typeQualifiers:"); - if (this.typeQualifiers == null) { - sb.append("null"); - } else { - sb.append(this.typeQualifiers); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetType()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'type' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (typeQualifiers != null) { - typeQualifiers.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TPrimitiveTypeEntryStandardSchemeFactory implements SchemeFactory { - public TPrimitiveTypeEntryStandardScheme getScheme() { - return new TPrimitiveTypeEntryStandardScheme(); - } - } - - private static class TPrimitiveTypeEntryStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TPrimitiveTypeEntry struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // TYPE - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.type = TTypeId.findByValue(iprot.readI32()); - struct.setTypeIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // TYPE_QUALIFIERS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.typeQualifiers = new TTypeQualifiers(); - struct.typeQualifiers.read(iprot); - struct.setTypeQualifiersIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TPrimitiveTypeEntry struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.type != null) { - oprot.writeFieldBegin(TYPE_FIELD_DESC); - oprot.writeI32(struct.type.getValue()); - oprot.writeFieldEnd(); - } - if (struct.typeQualifiers != null) { - if (struct.isSetTypeQualifiers()) { - oprot.writeFieldBegin(TYPE_QUALIFIERS_FIELD_DESC); - struct.typeQualifiers.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TPrimitiveTypeEntryTupleSchemeFactory implements SchemeFactory { - public TPrimitiveTypeEntryTupleScheme getScheme() { - return new TPrimitiveTypeEntryTupleScheme(); - } - } - - private static class TPrimitiveTypeEntryTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TPrimitiveTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeI32(struct.type.getValue()); - BitSet optionals = new BitSet(); - if (struct.isSetTypeQualifiers()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetTypeQualifiers()) { - struct.typeQualifiers.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TPrimitiveTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.type = TTypeId.findByValue(iprot.readI32()); - struct.setTypeIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.typeQualifiers = new TTypeQualifiers(); - struct.typeQualifiers.read(iprot); - struct.setTypeQualifiersIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java deleted file mode 100644 index a4279d29f662e..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java +++ /dev/null @@ -1,63 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - - -import java.util.Map; -import java.util.HashMap; -import org.apache.thrift.TEnum; - -public enum TProtocolVersion implements org.apache.thrift.TEnum { - HIVE_CLI_SERVICE_PROTOCOL_V1(0), - HIVE_CLI_SERVICE_PROTOCOL_V2(1), - HIVE_CLI_SERVICE_PROTOCOL_V3(2), - HIVE_CLI_SERVICE_PROTOCOL_V4(3), - HIVE_CLI_SERVICE_PROTOCOL_V5(4), - HIVE_CLI_SERVICE_PROTOCOL_V6(5), - HIVE_CLI_SERVICE_PROTOCOL_V7(6), - HIVE_CLI_SERVICE_PROTOCOL_V8(7); - - private final int value; - - private TProtocolVersion(int value) { - this.value = value; - } - - /** - * Get the integer value of this enum value, as defined in the Thrift IDL. - */ - public int getValue() { - return value; - } - - /** - * Find a the enum type by its integer value, as defined in the Thrift IDL. - * @return null if the value is not found. - */ - public static TProtocolVersion findByValue(int value) { - switch (value) { - case 0: - return HIVE_CLI_SERVICE_PROTOCOL_V1; - case 1: - return HIVE_CLI_SERVICE_PROTOCOL_V2; - case 2: - return HIVE_CLI_SERVICE_PROTOCOL_V3; - case 3: - return HIVE_CLI_SERVICE_PROTOCOL_V4; - case 4: - return HIVE_CLI_SERVICE_PROTOCOL_V5; - case 5: - return HIVE_CLI_SERVICE_PROTOCOL_V6; - case 6: - return HIVE_CLI_SERVICE_PROTOCOL_V7; - case 7: - return HIVE_CLI_SERVICE_PROTOCOL_V8; - default: - return null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java deleted file mode 100644 index a3e39c8cdf321..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java +++ /dev/null @@ -1,491 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TRenewDelegationTokenReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRenewDelegationTokenReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField DELEGATION_TOKEN_FIELD_DESC = new org.apache.thrift.protocol.TField("delegationToken", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TRenewDelegationTokenReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TRenewDelegationTokenReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private String delegationToken; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - DELEGATION_TOKEN((short)2, "delegationToken"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // DELEGATION_TOKEN - return DELEGATION_TOKEN; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.DELEGATION_TOKEN, new org.apache.thrift.meta_data.FieldMetaData("delegationToken", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRenewDelegationTokenReq.class, metaDataMap); - } - - public TRenewDelegationTokenReq() { - } - - public TRenewDelegationTokenReq( - TSessionHandle sessionHandle, - String delegationToken) - { - this(); - this.sessionHandle = sessionHandle; - this.delegationToken = delegationToken; - } - - /** - * Performs a deep copy on other. - */ - public TRenewDelegationTokenReq(TRenewDelegationTokenReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetDelegationToken()) { - this.delegationToken = other.delegationToken; - } - } - - public TRenewDelegationTokenReq deepCopy() { - return new TRenewDelegationTokenReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.delegationToken = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public String getDelegationToken() { - return this.delegationToken; - } - - public void setDelegationToken(String delegationToken) { - this.delegationToken = delegationToken; - } - - public void unsetDelegationToken() { - this.delegationToken = null; - } - - /** Returns true if field delegationToken is set (has been assigned a value) and false otherwise */ - public boolean isSetDelegationToken() { - return this.delegationToken != null; - } - - public void setDelegationTokenIsSet(boolean value) { - if (!value) { - this.delegationToken = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case DELEGATION_TOKEN: - if (value == null) { - unsetDelegationToken(); - } else { - setDelegationToken((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case DELEGATION_TOKEN: - return getDelegationToken(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case DELEGATION_TOKEN: - return isSetDelegationToken(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TRenewDelegationTokenReq) - return this.equals((TRenewDelegationTokenReq)that); - return false; - } - - public boolean equals(TRenewDelegationTokenReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_delegationToken = true && this.isSetDelegationToken(); - boolean that_present_delegationToken = true && that.isSetDelegationToken(); - if (this_present_delegationToken || that_present_delegationToken) { - if (!(this_present_delegationToken && that_present_delegationToken)) - return false; - if (!this.delegationToken.equals(that.delegationToken)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_delegationToken = true && (isSetDelegationToken()); - builder.append(present_delegationToken); - if (present_delegationToken) - builder.append(delegationToken); - - return builder.toHashCode(); - } - - public int compareTo(TRenewDelegationTokenReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TRenewDelegationTokenReq typedOther = (TRenewDelegationTokenReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetDelegationToken()).compareTo(typedOther.isSetDelegationToken()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetDelegationToken()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.delegationToken, typedOther.delegationToken); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TRenewDelegationTokenReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (!first) sb.append(", "); - sb.append("delegationToken:"); - if (this.delegationToken == null) { - sb.append("null"); - } else { - sb.append(this.delegationToken); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - if (!isSetDelegationToken()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'delegationToken' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TRenewDelegationTokenReqStandardSchemeFactory implements SchemeFactory { - public TRenewDelegationTokenReqStandardScheme getScheme() { - return new TRenewDelegationTokenReqStandardScheme(); - } - } - - private static class TRenewDelegationTokenReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TRenewDelegationTokenReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // DELEGATION_TOKEN - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.delegationToken = iprot.readString(); - struct.setDelegationTokenIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TRenewDelegationTokenReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.delegationToken != null) { - oprot.writeFieldBegin(DELEGATION_TOKEN_FIELD_DESC); - oprot.writeString(struct.delegationToken); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TRenewDelegationTokenReqTupleSchemeFactory implements SchemeFactory { - public TRenewDelegationTokenReqTupleScheme getScheme() { - return new TRenewDelegationTokenReqTupleScheme(); - } - } - - private static class TRenewDelegationTokenReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TRenewDelegationTokenReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - oprot.writeString(struct.delegationToken); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TRenewDelegationTokenReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - struct.delegationToken = iprot.readString(); - struct.setDelegationTokenIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java deleted file mode 100644 index 5f3eb6c4d4b90..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TRenewDelegationTokenResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRenewDelegationTokenResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TRenewDelegationTokenRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TRenewDelegationTokenRespTupleSchemeFactory()); - } - - private TStatus status; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRenewDelegationTokenResp.class, metaDataMap); - } - - public TRenewDelegationTokenResp() { - } - - public TRenewDelegationTokenResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TRenewDelegationTokenResp(TRenewDelegationTokenResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - } - - public TRenewDelegationTokenResp deepCopy() { - return new TRenewDelegationTokenResp(this); - } - - @Override - public void clear() { - this.status = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TRenewDelegationTokenResp) - return this.equals((TRenewDelegationTokenResp)that); - return false; - } - - public boolean equals(TRenewDelegationTokenResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - return builder.toHashCode(); - } - - public int compareTo(TRenewDelegationTokenResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TRenewDelegationTokenResp typedOther = (TRenewDelegationTokenResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TRenewDelegationTokenResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TRenewDelegationTokenRespStandardSchemeFactory implements SchemeFactory { - public TRenewDelegationTokenRespStandardScheme getScheme() { - return new TRenewDelegationTokenRespStandardScheme(); - } - } - - private static class TRenewDelegationTokenRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TRenewDelegationTokenResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TRenewDelegationTokenResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TRenewDelegationTokenRespTupleSchemeFactory implements SchemeFactory { - public TRenewDelegationTokenRespTupleScheme getScheme() { - return new TRenewDelegationTokenRespTupleScheme(); - } - } - - private static class TRenewDelegationTokenRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TRenewDelegationTokenResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TRenewDelegationTokenResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java deleted file mode 100644 index a44cfb08ff01a..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java +++ /dev/null @@ -1,439 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TRow implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRow"); - - private static final org.apache.thrift.protocol.TField COL_VALS_FIELD_DESC = new org.apache.thrift.protocol.TField("colVals", org.apache.thrift.protocol.TType.LIST, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TRowStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TRowTupleSchemeFactory()); - } - - private List colVals; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - COL_VALS((short)1, "colVals"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // COL_VALS - return COL_VALS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.COL_VALS, new org.apache.thrift.meta_data.FieldMetaData("colVals", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnValue.class)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRow.class, metaDataMap); - } - - public TRow() { - } - - public TRow( - List colVals) - { - this(); - this.colVals = colVals; - } - - /** - * Performs a deep copy on other. - */ - public TRow(TRow other) { - if (other.isSetColVals()) { - List __this__colVals = new ArrayList(); - for (TColumnValue other_element : other.colVals) { - __this__colVals.add(new TColumnValue(other_element)); - } - this.colVals = __this__colVals; - } - } - - public TRow deepCopy() { - return new TRow(this); - } - - @Override - public void clear() { - this.colVals = null; - } - - public int getColValsSize() { - return (this.colVals == null) ? 0 : this.colVals.size(); - } - - public java.util.Iterator getColValsIterator() { - return (this.colVals == null) ? null : this.colVals.iterator(); - } - - public void addToColVals(TColumnValue elem) { - if (this.colVals == null) { - this.colVals = new ArrayList(); - } - this.colVals.add(elem); - } - - public List getColVals() { - return this.colVals; - } - - public void setColVals(List colVals) { - this.colVals = colVals; - } - - public void unsetColVals() { - this.colVals = null; - } - - /** Returns true if field colVals is set (has been assigned a value) and false otherwise */ - public boolean isSetColVals() { - return this.colVals != null; - } - - public void setColValsIsSet(boolean value) { - if (!value) { - this.colVals = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case COL_VALS: - if (value == null) { - unsetColVals(); - } else { - setColVals((List)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case COL_VALS: - return getColVals(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case COL_VALS: - return isSetColVals(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TRow) - return this.equals((TRow)that); - return false; - } - - public boolean equals(TRow that) { - if (that == null) - return false; - - boolean this_present_colVals = true && this.isSetColVals(); - boolean that_present_colVals = true && that.isSetColVals(); - if (this_present_colVals || that_present_colVals) { - if (!(this_present_colVals && that_present_colVals)) - return false; - if (!this.colVals.equals(that.colVals)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_colVals = true && (isSetColVals()); - builder.append(present_colVals); - if (present_colVals) - builder.append(colVals); - - return builder.toHashCode(); - } - - public int compareTo(TRow other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TRow typedOther = (TRow)other; - - lastComparison = Boolean.valueOf(isSetColVals()).compareTo(typedOther.isSetColVals()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetColVals()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.colVals, typedOther.colVals); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TRow("); - boolean first = true; - - sb.append("colVals:"); - if (this.colVals == null) { - sb.append("null"); - } else { - sb.append(this.colVals); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetColVals()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'colVals' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TRowStandardSchemeFactory implements SchemeFactory { - public TRowStandardScheme getScheme() { - return new TRowStandardScheme(); - } - } - - private static class TRowStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TRow struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // COL_VALS - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list46 = iprot.readListBegin(); - struct.colVals = new ArrayList(_list46.size); - for (int _i47 = 0; _i47 < _list46.size; ++_i47) - { - TColumnValue _elem48; // optional - _elem48 = new TColumnValue(); - _elem48.read(iprot); - struct.colVals.add(_elem48); - } - iprot.readListEnd(); - } - struct.setColValsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TRow struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.colVals != null) { - oprot.writeFieldBegin(COL_VALS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.colVals.size())); - for (TColumnValue _iter49 : struct.colVals) - { - _iter49.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TRowTupleSchemeFactory implements SchemeFactory { - public TRowTupleScheme getScheme() { - return new TRowTupleScheme(); - } - } - - private static class TRowTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TRow struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.colVals.size()); - for (TColumnValue _iter50 : struct.colVals) - { - _iter50.write(oprot); - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TRow struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list51 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.colVals = new ArrayList(_list51.size); - for (int _i52 = 0; _i52 < _list51.size; ++_i52) - { - TColumnValue _elem53; // optional - _elem53 = new TColumnValue(); - _elem53.read(iprot); - struct.colVals.add(_elem53); - } - } - struct.setColValsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java deleted file mode 100644 index d16c8a4bb32da..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java +++ /dev/null @@ -1,702 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TRowSet implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowSet"); - - private static final org.apache.thrift.protocol.TField START_ROW_OFFSET_FIELD_DESC = new org.apache.thrift.protocol.TField("startRowOffset", org.apache.thrift.protocol.TType.I64, (short)1); - private static final org.apache.thrift.protocol.TField ROWS_FIELD_DESC = new org.apache.thrift.protocol.TField("rows", org.apache.thrift.protocol.TType.LIST, (short)2); - private static final org.apache.thrift.protocol.TField COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.LIST, (short)3); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TRowSetStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TRowSetTupleSchemeFactory()); - } - - private long startRowOffset; // required - private List rows; // required - private List columns; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - START_ROW_OFFSET((short)1, "startRowOffset"), - ROWS((short)2, "rows"), - COLUMNS((short)3, "columns"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // START_ROW_OFFSET - return START_ROW_OFFSET; - case 2: // ROWS - return ROWS; - case 3: // COLUMNS - return COLUMNS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __STARTROWOFFSET_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.COLUMNS}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.START_ROW_OFFSET, new org.apache.thrift.meta_data.FieldMetaData("startRowOffset", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - tmpMap.put(_Fields.ROWS, new org.apache.thrift.meta_data.FieldMetaData("rows", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRow.class)))); - tmpMap.put(_Fields.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumn.class)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRowSet.class, metaDataMap); - } - - public TRowSet() { - } - - public TRowSet( - long startRowOffset, - List rows) - { - this(); - this.startRowOffset = startRowOffset; - setStartRowOffsetIsSet(true); - this.rows = rows; - } - - /** - * Performs a deep copy on other. - */ - public TRowSet(TRowSet other) { - __isset_bitfield = other.__isset_bitfield; - this.startRowOffset = other.startRowOffset; - if (other.isSetRows()) { - List __this__rows = new ArrayList(); - for (TRow other_element : other.rows) { - __this__rows.add(new TRow(other_element)); - } - this.rows = __this__rows; - } - if (other.isSetColumns()) { - List __this__columns = new ArrayList(); - for (TColumn other_element : other.columns) { - __this__columns.add(new TColumn(other_element)); - } - this.columns = __this__columns; - } - } - - public TRowSet deepCopy() { - return new TRowSet(this); - } - - @Override - public void clear() { - setStartRowOffsetIsSet(false); - this.startRowOffset = 0; - this.rows = null; - this.columns = null; - } - - public long getStartRowOffset() { - return this.startRowOffset; - } - - public void setStartRowOffset(long startRowOffset) { - this.startRowOffset = startRowOffset; - setStartRowOffsetIsSet(true); - } - - public void unsetStartRowOffset() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __STARTROWOFFSET_ISSET_ID); - } - - /** Returns true if field startRowOffset is set (has been assigned a value) and false otherwise */ - public boolean isSetStartRowOffset() { - return EncodingUtils.testBit(__isset_bitfield, __STARTROWOFFSET_ISSET_ID); - } - - public void setStartRowOffsetIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STARTROWOFFSET_ISSET_ID, value); - } - - public int getRowsSize() { - return (this.rows == null) ? 0 : this.rows.size(); - } - - public java.util.Iterator getRowsIterator() { - return (this.rows == null) ? null : this.rows.iterator(); - } - - public void addToRows(TRow elem) { - if (this.rows == null) { - this.rows = new ArrayList(); - } - this.rows.add(elem); - } - - public List getRows() { - return this.rows; - } - - public void setRows(List rows) { - this.rows = rows; - } - - public void unsetRows() { - this.rows = null; - } - - /** Returns true if field rows is set (has been assigned a value) and false otherwise */ - public boolean isSetRows() { - return this.rows != null; - } - - public void setRowsIsSet(boolean value) { - if (!value) { - this.rows = null; - } - } - - public int getColumnsSize() { - return (this.columns == null) ? 0 : this.columns.size(); - } - - public java.util.Iterator getColumnsIterator() { - return (this.columns == null) ? null : this.columns.iterator(); - } - - public void addToColumns(TColumn elem) { - if (this.columns == null) { - this.columns = new ArrayList(); - } - this.columns.add(elem); - } - - public List getColumns() { - return this.columns; - } - - public void setColumns(List columns) { - this.columns = columns; - } - - public void unsetColumns() { - this.columns = null; - } - - /** Returns true if field columns is set (has been assigned a value) and false otherwise */ - public boolean isSetColumns() { - return this.columns != null; - } - - public void setColumnsIsSet(boolean value) { - if (!value) { - this.columns = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case START_ROW_OFFSET: - if (value == null) { - unsetStartRowOffset(); - } else { - setStartRowOffset((Long)value); - } - break; - - case ROWS: - if (value == null) { - unsetRows(); - } else { - setRows((List)value); - } - break; - - case COLUMNS: - if (value == null) { - unsetColumns(); - } else { - setColumns((List)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case START_ROW_OFFSET: - return Long.valueOf(getStartRowOffset()); - - case ROWS: - return getRows(); - - case COLUMNS: - return getColumns(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case START_ROW_OFFSET: - return isSetStartRowOffset(); - case ROWS: - return isSetRows(); - case COLUMNS: - return isSetColumns(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TRowSet) - return this.equals((TRowSet)that); - return false; - } - - public boolean equals(TRowSet that) { - if (that == null) - return false; - - boolean this_present_startRowOffset = true; - boolean that_present_startRowOffset = true; - if (this_present_startRowOffset || that_present_startRowOffset) { - if (!(this_present_startRowOffset && that_present_startRowOffset)) - return false; - if (this.startRowOffset != that.startRowOffset) - return false; - } - - boolean this_present_rows = true && this.isSetRows(); - boolean that_present_rows = true && that.isSetRows(); - if (this_present_rows || that_present_rows) { - if (!(this_present_rows && that_present_rows)) - return false; - if (!this.rows.equals(that.rows)) - return false; - } - - boolean this_present_columns = true && this.isSetColumns(); - boolean that_present_columns = true && that.isSetColumns(); - if (this_present_columns || that_present_columns) { - if (!(this_present_columns && that_present_columns)) - return false; - if (!this.columns.equals(that.columns)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_startRowOffset = true; - builder.append(present_startRowOffset); - if (present_startRowOffset) - builder.append(startRowOffset); - - boolean present_rows = true && (isSetRows()); - builder.append(present_rows); - if (present_rows) - builder.append(rows); - - boolean present_columns = true && (isSetColumns()); - builder.append(present_columns); - if (present_columns) - builder.append(columns); - - return builder.toHashCode(); - } - - public int compareTo(TRowSet other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TRowSet typedOther = (TRowSet)other; - - lastComparison = Boolean.valueOf(isSetStartRowOffset()).compareTo(typedOther.isSetStartRowOffset()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStartRowOffset()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRowOffset, typedOther.startRowOffset); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetRows()).compareTo(typedOther.isSetRows()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetRows()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rows, typedOther.rows); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TRowSet("); - boolean first = true; - - sb.append("startRowOffset:"); - sb.append(this.startRowOffset); - first = false; - if (!first) sb.append(", "); - sb.append("rows:"); - if (this.rows == null) { - sb.append("null"); - } else { - sb.append(this.rows); - } - first = false; - if (isSetColumns()) { - if (!first) sb.append(", "); - sb.append("columns:"); - if (this.columns == null) { - sb.append("null"); - } else { - sb.append(this.columns); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStartRowOffset()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'startRowOffset' is unset! Struct:" + toString()); - } - - if (!isSetRows()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'rows' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TRowSetStandardSchemeFactory implements SchemeFactory { - public TRowSetStandardScheme getScheme() { - return new TRowSetStandardScheme(); - } - } - - private static class TRowSetStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TRowSet struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // START_ROW_OFFSET - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.startRowOffset = iprot.readI64(); - struct.setStartRowOffsetIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // ROWS - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list118 = iprot.readListBegin(); - struct.rows = new ArrayList(_list118.size); - for (int _i119 = 0; _i119 < _list118.size; ++_i119) - { - TRow _elem120; // optional - _elem120 = new TRow(); - _elem120.read(iprot); - struct.rows.add(_elem120); - } - iprot.readListEnd(); - } - struct.setRowsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // COLUMNS - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list121 = iprot.readListBegin(); - struct.columns = new ArrayList(_list121.size); - for (int _i122 = 0; _i122 < _list121.size; ++_i122) - { - TColumn _elem123; // optional - _elem123 = new TColumn(); - _elem123.read(iprot); - struct.columns.add(_elem123); - } - iprot.readListEnd(); - } - struct.setColumnsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TRowSet struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(START_ROW_OFFSET_FIELD_DESC); - oprot.writeI64(struct.startRowOffset); - oprot.writeFieldEnd(); - if (struct.rows != null) { - oprot.writeFieldBegin(ROWS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.rows.size())); - for (TRow _iter124 : struct.rows) - { - _iter124.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.columns != null) { - if (struct.isSetColumns()) { - oprot.writeFieldBegin(COLUMNS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size())); - for (TColumn _iter125 : struct.columns) - { - _iter125.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TRowSetTupleSchemeFactory implements SchemeFactory { - public TRowSetTupleScheme getScheme() { - return new TRowSetTupleScheme(); - } - } - - private static class TRowSetTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TRowSet struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeI64(struct.startRowOffset); - { - oprot.writeI32(struct.rows.size()); - for (TRow _iter126 : struct.rows) - { - _iter126.write(oprot); - } - } - BitSet optionals = new BitSet(); - if (struct.isSetColumns()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetColumns()) { - { - oprot.writeI32(struct.columns.size()); - for (TColumn _iter127 : struct.columns) - { - _iter127.write(oprot); - } - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TRowSet struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.startRowOffset = iprot.readI64(); - struct.setStartRowOffsetIsSet(true); - { - org.apache.thrift.protocol.TList _list128 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.rows = new ArrayList(_list128.size); - for (int _i129 = 0; _i129 < _list128.size; ++_i129) - { - TRow _elem130; // optional - _elem130 = new TRow(); - _elem130.read(iprot); - struct.rows.add(_elem130); - } - } - struct.setRowsIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - { - org.apache.thrift.protocol.TList _list131 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.columns = new ArrayList(_list131.size); - for (int _i132 = 0; _i132 < _list131.size; ++_i132) - { - TColumn _elem133; // optional - _elem133 = new TColumn(); - _elem133.read(iprot); - struct.columns.add(_elem133); - } - } - struct.setColumnsIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java deleted file mode 100644 index 82c00dd68a98b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TSessionHandle implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TSessionHandle"); - - private static final org.apache.thrift.protocol.TField SESSION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionId", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TSessionHandleStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TSessionHandleTupleSchemeFactory()); - } - - private THandleIdentifier sessionId; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_ID((short)1, "sessionId"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_ID - return SESSION_ID; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_ID, new org.apache.thrift.meta_data.FieldMetaData("sessionId", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, THandleIdentifier.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TSessionHandle.class, metaDataMap); - } - - public TSessionHandle() { - } - - public TSessionHandle( - THandleIdentifier sessionId) - { - this(); - this.sessionId = sessionId; - } - - /** - * Performs a deep copy on other. - */ - public TSessionHandle(TSessionHandle other) { - if (other.isSetSessionId()) { - this.sessionId = new THandleIdentifier(other.sessionId); - } - } - - public TSessionHandle deepCopy() { - return new TSessionHandle(this); - } - - @Override - public void clear() { - this.sessionId = null; - } - - public THandleIdentifier getSessionId() { - return this.sessionId; - } - - public void setSessionId(THandleIdentifier sessionId) { - this.sessionId = sessionId; - } - - public void unsetSessionId() { - this.sessionId = null; - } - - /** Returns true if field sessionId is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionId() { - return this.sessionId != null; - } - - public void setSessionIdIsSet(boolean value) { - if (!value) { - this.sessionId = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_ID: - if (value == null) { - unsetSessionId(); - } else { - setSessionId((THandleIdentifier)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_ID: - return getSessionId(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_ID: - return isSetSessionId(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TSessionHandle) - return this.equals((TSessionHandle)that); - return false; - } - - public boolean equals(TSessionHandle that) { - if (that == null) - return false; - - boolean this_present_sessionId = true && this.isSetSessionId(); - boolean that_present_sessionId = true && that.isSetSessionId(); - if (this_present_sessionId || that_present_sessionId) { - if (!(this_present_sessionId && that_present_sessionId)) - return false; - if (!this.sessionId.equals(that.sessionId)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionId = true && (isSetSessionId()); - builder.append(present_sessionId); - if (present_sessionId) - builder.append(sessionId); - - return builder.toHashCode(); - } - - public int compareTo(TSessionHandle other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TSessionHandle typedOther = (TSessionHandle)other; - - lastComparison = Boolean.valueOf(isSetSessionId()).compareTo(typedOther.isSetSessionId()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionId, typedOther.sessionId); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TSessionHandle("); - boolean first = true; - - sb.append("sessionId:"); - if (this.sessionId == null) { - sb.append("null"); - } else { - sb.append(this.sessionId); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionId()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionId' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionId != null) { - sessionId.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TSessionHandleStandardSchemeFactory implements SchemeFactory { - public TSessionHandleStandardScheme getScheme() { - return new TSessionHandleStandardScheme(); - } - } - - private static class TSessionHandleStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TSessionHandle struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_ID - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionId = new THandleIdentifier(); - struct.sessionId.read(iprot); - struct.setSessionIdIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TSessionHandle struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionId != null) { - oprot.writeFieldBegin(SESSION_ID_FIELD_DESC); - struct.sessionId.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TSessionHandleTupleSchemeFactory implements SchemeFactory { - public TSessionHandleTupleScheme getScheme() { - return new TSessionHandleTupleScheme(); - } - } - - private static class TSessionHandleTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TSessionHandle struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionId.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TSessionHandle struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionId = new THandleIdentifier(); - struct.sessionId.read(iprot); - struct.setSessionIdIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java deleted file mode 100644 index 24a746e94965d..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java +++ /dev/null @@ -1,874 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TStatus implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStatus"); - - private static final org.apache.thrift.protocol.TField STATUS_CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("statusCode", org.apache.thrift.protocol.TType.I32, (short)1); - private static final org.apache.thrift.protocol.TField INFO_MESSAGES_FIELD_DESC = new org.apache.thrift.protocol.TField("infoMessages", org.apache.thrift.protocol.TType.LIST, (short)2); - private static final org.apache.thrift.protocol.TField SQL_STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("sqlState", org.apache.thrift.protocol.TType.STRING, (short)3); - private static final org.apache.thrift.protocol.TField ERROR_CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorCode", org.apache.thrift.protocol.TType.I32, (short)4); - private static final org.apache.thrift.protocol.TField ERROR_MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorMessage", org.apache.thrift.protocol.TType.STRING, (short)5); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TStatusStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TStatusTupleSchemeFactory()); - } - - private TStatusCode statusCode; // required - private List infoMessages; // optional - private String sqlState; // optional - private int errorCode; // optional - private String errorMessage; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - /** - * - * @see TStatusCode - */ - STATUS_CODE((short)1, "statusCode"), - INFO_MESSAGES((short)2, "infoMessages"), - SQL_STATE((short)3, "sqlState"), - ERROR_CODE((short)4, "errorCode"), - ERROR_MESSAGE((short)5, "errorMessage"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS_CODE - return STATUS_CODE; - case 2: // INFO_MESSAGES - return INFO_MESSAGES; - case 3: // SQL_STATE - return SQL_STATE; - case 4: // ERROR_CODE - return ERROR_CODE; - case 5: // ERROR_MESSAGE - return ERROR_MESSAGE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __ERRORCODE_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.INFO_MESSAGES,_Fields.SQL_STATE,_Fields.ERROR_CODE,_Fields.ERROR_MESSAGE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS_CODE, new org.apache.thrift.meta_data.FieldMetaData("statusCode", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TStatusCode.class))); - tmpMap.put(_Fields.INFO_MESSAGES, new org.apache.thrift.meta_data.FieldMetaData("infoMessages", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); - tmpMap.put(_Fields.SQL_STATE, new org.apache.thrift.meta_data.FieldMetaData("sqlState", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.ERROR_CODE, new org.apache.thrift.meta_data.FieldMetaData("errorCode", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); - tmpMap.put(_Fields.ERROR_MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("errorMessage", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TStatus.class, metaDataMap); - } - - public TStatus() { - } - - public TStatus( - TStatusCode statusCode) - { - this(); - this.statusCode = statusCode; - } - - /** - * Performs a deep copy on other. - */ - public TStatus(TStatus other) { - __isset_bitfield = other.__isset_bitfield; - if (other.isSetStatusCode()) { - this.statusCode = other.statusCode; - } - if (other.isSetInfoMessages()) { - List __this__infoMessages = new ArrayList(); - for (String other_element : other.infoMessages) { - __this__infoMessages.add(other_element); - } - this.infoMessages = __this__infoMessages; - } - if (other.isSetSqlState()) { - this.sqlState = other.sqlState; - } - this.errorCode = other.errorCode; - if (other.isSetErrorMessage()) { - this.errorMessage = other.errorMessage; - } - } - - public TStatus deepCopy() { - return new TStatus(this); - } - - @Override - public void clear() { - this.statusCode = null; - this.infoMessages = null; - this.sqlState = null; - setErrorCodeIsSet(false); - this.errorCode = 0; - this.errorMessage = null; - } - - /** - * - * @see TStatusCode - */ - public TStatusCode getStatusCode() { - return this.statusCode; - } - - /** - * - * @see TStatusCode - */ - public void setStatusCode(TStatusCode statusCode) { - this.statusCode = statusCode; - } - - public void unsetStatusCode() { - this.statusCode = null; - } - - /** Returns true if field statusCode is set (has been assigned a value) and false otherwise */ - public boolean isSetStatusCode() { - return this.statusCode != null; - } - - public void setStatusCodeIsSet(boolean value) { - if (!value) { - this.statusCode = null; - } - } - - public int getInfoMessagesSize() { - return (this.infoMessages == null) ? 0 : this.infoMessages.size(); - } - - public java.util.Iterator getInfoMessagesIterator() { - return (this.infoMessages == null) ? null : this.infoMessages.iterator(); - } - - public void addToInfoMessages(String elem) { - if (this.infoMessages == null) { - this.infoMessages = new ArrayList(); - } - this.infoMessages.add(elem); - } - - public List getInfoMessages() { - return this.infoMessages; - } - - public void setInfoMessages(List infoMessages) { - this.infoMessages = infoMessages; - } - - public void unsetInfoMessages() { - this.infoMessages = null; - } - - /** Returns true if field infoMessages is set (has been assigned a value) and false otherwise */ - public boolean isSetInfoMessages() { - return this.infoMessages != null; - } - - public void setInfoMessagesIsSet(boolean value) { - if (!value) { - this.infoMessages = null; - } - } - - public String getSqlState() { - return this.sqlState; - } - - public void setSqlState(String sqlState) { - this.sqlState = sqlState; - } - - public void unsetSqlState() { - this.sqlState = null; - } - - /** Returns true if field sqlState is set (has been assigned a value) and false otherwise */ - public boolean isSetSqlState() { - return this.sqlState != null; - } - - public void setSqlStateIsSet(boolean value) { - if (!value) { - this.sqlState = null; - } - } - - public int getErrorCode() { - return this.errorCode; - } - - public void setErrorCode(int errorCode) { - this.errorCode = errorCode; - setErrorCodeIsSet(true); - } - - public void unsetErrorCode() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ERRORCODE_ISSET_ID); - } - - /** Returns true if field errorCode is set (has been assigned a value) and false otherwise */ - public boolean isSetErrorCode() { - return EncodingUtils.testBit(__isset_bitfield, __ERRORCODE_ISSET_ID); - } - - public void setErrorCodeIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ERRORCODE_ISSET_ID, value); - } - - public String getErrorMessage() { - return this.errorMessage; - } - - public void setErrorMessage(String errorMessage) { - this.errorMessage = errorMessage; - } - - public void unsetErrorMessage() { - this.errorMessage = null; - } - - /** Returns true if field errorMessage is set (has been assigned a value) and false otherwise */ - public boolean isSetErrorMessage() { - return this.errorMessage != null; - } - - public void setErrorMessageIsSet(boolean value) { - if (!value) { - this.errorMessage = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS_CODE: - if (value == null) { - unsetStatusCode(); - } else { - setStatusCode((TStatusCode)value); - } - break; - - case INFO_MESSAGES: - if (value == null) { - unsetInfoMessages(); - } else { - setInfoMessages((List)value); - } - break; - - case SQL_STATE: - if (value == null) { - unsetSqlState(); - } else { - setSqlState((String)value); - } - break; - - case ERROR_CODE: - if (value == null) { - unsetErrorCode(); - } else { - setErrorCode((Integer)value); - } - break; - - case ERROR_MESSAGE: - if (value == null) { - unsetErrorMessage(); - } else { - setErrorMessage((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS_CODE: - return getStatusCode(); - - case INFO_MESSAGES: - return getInfoMessages(); - - case SQL_STATE: - return getSqlState(); - - case ERROR_CODE: - return Integer.valueOf(getErrorCode()); - - case ERROR_MESSAGE: - return getErrorMessage(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS_CODE: - return isSetStatusCode(); - case INFO_MESSAGES: - return isSetInfoMessages(); - case SQL_STATE: - return isSetSqlState(); - case ERROR_CODE: - return isSetErrorCode(); - case ERROR_MESSAGE: - return isSetErrorMessage(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TStatus) - return this.equals((TStatus)that); - return false; - } - - public boolean equals(TStatus that) { - if (that == null) - return false; - - boolean this_present_statusCode = true && this.isSetStatusCode(); - boolean that_present_statusCode = true && that.isSetStatusCode(); - if (this_present_statusCode || that_present_statusCode) { - if (!(this_present_statusCode && that_present_statusCode)) - return false; - if (!this.statusCode.equals(that.statusCode)) - return false; - } - - boolean this_present_infoMessages = true && this.isSetInfoMessages(); - boolean that_present_infoMessages = true && that.isSetInfoMessages(); - if (this_present_infoMessages || that_present_infoMessages) { - if (!(this_present_infoMessages && that_present_infoMessages)) - return false; - if (!this.infoMessages.equals(that.infoMessages)) - return false; - } - - boolean this_present_sqlState = true && this.isSetSqlState(); - boolean that_present_sqlState = true && that.isSetSqlState(); - if (this_present_sqlState || that_present_sqlState) { - if (!(this_present_sqlState && that_present_sqlState)) - return false; - if (!this.sqlState.equals(that.sqlState)) - return false; - } - - boolean this_present_errorCode = true && this.isSetErrorCode(); - boolean that_present_errorCode = true && that.isSetErrorCode(); - if (this_present_errorCode || that_present_errorCode) { - if (!(this_present_errorCode && that_present_errorCode)) - return false; - if (this.errorCode != that.errorCode) - return false; - } - - boolean this_present_errorMessage = true && this.isSetErrorMessage(); - boolean that_present_errorMessage = true && that.isSetErrorMessage(); - if (this_present_errorMessage || that_present_errorMessage) { - if (!(this_present_errorMessage && that_present_errorMessage)) - return false; - if (!this.errorMessage.equals(that.errorMessage)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_statusCode = true && (isSetStatusCode()); - builder.append(present_statusCode); - if (present_statusCode) - builder.append(statusCode.getValue()); - - boolean present_infoMessages = true && (isSetInfoMessages()); - builder.append(present_infoMessages); - if (present_infoMessages) - builder.append(infoMessages); - - boolean present_sqlState = true && (isSetSqlState()); - builder.append(present_sqlState); - if (present_sqlState) - builder.append(sqlState); - - boolean present_errorCode = true && (isSetErrorCode()); - builder.append(present_errorCode); - if (present_errorCode) - builder.append(errorCode); - - boolean present_errorMessage = true && (isSetErrorMessage()); - builder.append(present_errorMessage); - if (present_errorMessage) - builder.append(errorMessage); - - return builder.toHashCode(); - } - - public int compareTo(TStatus other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TStatus typedOther = (TStatus)other; - - lastComparison = Boolean.valueOf(isSetStatusCode()).compareTo(typedOther.isSetStatusCode()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatusCode()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.statusCode, typedOther.statusCode); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetInfoMessages()).compareTo(typedOther.isSetInfoMessages()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetInfoMessages()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.infoMessages, typedOther.infoMessages); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSqlState()).compareTo(typedOther.isSetSqlState()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSqlState()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sqlState, typedOther.sqlState); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetErrorCode()).compareTo(typedOther.isSetErrorCode()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetErrorCode()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorCode, typedOther.errorCode); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetErrorMessage()).compareTo(typedOther.isSetErrorMessage()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetErrorMessage()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorMessage, typedOther.errorMessage); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TStatus("); - boolean first = true; - - sb.append("statusCode:"); - if (this.statusCode == null) { - sb.append("null"); - } else { - sb.append(this.statusCode); - } - first = false; - if (isSetInfoMessages()) { - if (!first) sb.append(", "); - sb.append("infoMessages:"); - if (this.infoMessages == null) { - sb.append("null"); - } else { - sb.append(this.infoMessages); - } - first = false; - } - if (isSetSqlState()) { - if (!first) sb.append(", "); - sb.append("sqlState:"); - if (this.sqlState == null) { - sb.append("null"); - } else { - sb.append(this.sqlState); - } - first = false; - } - if (isSetErrorCode()) { - if (!first) sb.append(", "); - sb.append("errorCode:"); - sb.append(this.errorCode); - first = false; - } - if (isSetErrorMessage()) { - if (!first) sb.append(", "); - sb.append("errorMessage:"); - if (this.errorMessage == null) { - sb.append("null"); - } else { - sb.append(this.errorMessage); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatusCode()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'statusCode' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TStatusStandardSchemeFactory implements SchemeFactory { - public TStatusStandardScheme getScheme() { - return new TStatusStandardScheme(); - } - } - - private static class TStatusStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TStatus struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS_CODE - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.statusCode = TStatusCode.findByValue(iprot.readI32()); - struct.setStatusCodeIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // INFO_MESSAGES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list134 = iprot.readListBegin(); - struct.infoMessages = new ArrayList(_list134.size); - for (int _i135 = 0; _i135 < _list134.size; ++_i135) - { - String _elem136; // optional - _elem136 = iprot.readString(); - struct.infoMessages.add(_elem136); - } - iprot.readListEnd(); - } - struct.setInfoMessagesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // SQL_STATE - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.sqlState = iprot.readString(); - struct.setSqlStateIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // ERROR_CODE - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.errorCode = iprot.readI32(); - struct.setErrorCodeIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 5: // ERROR_MESSAGE - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.errorMessage = iprot.readString(); - struct.setErrorMessageIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TStatus struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.statusCode != null) { - oprot.writeFieldBegin(STATUS_CODE_FIELD_DESC); - oprot.writeI32(struct.statusCode.getValue()); - oprot.writeFieldEnd(); - } - if (struct.infoMessages != null) { - if (struct.isSetInfoMessages()) { - oprot.writeFieldBegin(INFO_MESSAGES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.infoMessages.size())); - for (String _iter137 : struct.infoMessages) - { - oprot.writeString(_iter137); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - } - if (struct.sqlState != null) { - if (struct.isSetSqlState()) { - oprot.writeFieldBegin(SQL_STATE_FIELD_DESC); - oprot.writeString(struct.sqlState); - oprot.writeFieldEnd(); - } - } - if (struct.isSetErrorCode()) { - oprot.writeFieldBegin(ERROR_CODE_FIELD_DESC); - oprot.writeI32(struct.errorCode); - oprot.writeFieldEnd(); - } - if (struct.errorMessage != null) { - if (struct.isSetErrorMessage()) { - oprot.writeFieldBegin(ERROR_MESSAGE_FIELD_DESC); - oprot.writeString(struct.errorMessage); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TStatusTupleSchemeFactory implements SchemeFactory { - public TStatusTupleScheme getScheme() { - return new TStatusTupleScheme(); - } - } - - private static class TStatusTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TStatus struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeI32(struct.statusCode.getValue()); - BitSet optionals = new BitSet(); - if (struct.isSetInfoMessages()) { - optionals.set(0); - } - if (struct.isSetSqlState()) { - optionals.set(1); - } - if (struct.isSetErrorCode()) { - optionals.set(2); - } - if (struct.isSetErrorMessage()) { - optionals.set(3); - } - oprot.writeBitSet(optionals, 4); - if (struct.isSetInfoMessages()) { - { - oprot.writeI32(struct.infoMessages.size()); - for (String _iter138 : struct.infoMessages) - { - oprot.writeString(_iter138); - } - } - } - if (struct.isSetSqlState()) { - oprot.writeString(struct.sqlState); - } - if (struct.isSetErrorCode()) { - oprot.writeI32(struct.errorCode); - } - if (struct.isSetErrorMessage()) { - oprot.writeString(struct.errorMessage); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TStatus struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.statusCode = TStatusCode.findByValue(iprot.readI32()); - struct.setStatusCodeIsSet(true); - BitSet incoming = iprot.readBitSet(4); - if (incoming.get(0)) { - { - org.apache.thrift.protocol.TList _list139 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.infoMessages = new ArrayList(_list139.size); - for (int _i140 = 0; _i140 < _list139.size; ++_i140) - { - String _elem141; // optional - _elem141 = iprot.readString(); - struct.infoMessages.add(_elem141); - } - } - struct.setInfoMessagesIsSet(true); - } - if (incoming.get(1)) { - struct.sqlState = iprot.readString(); - struct.setSqlStateIsSet(true); - } - if (incoming.get(2)) { - struct.errorCode = iprot.readI32(); - struct.setErrorCodeIsSet(true); - } - if (incoming.get(3)) { - struct.errorMessage = iprot.readString(); - struct.setErrorMessageIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java deleted file mode 100644 index e7fde45fd131a..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java +++ /dev/null @@ -1,54 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - - -import java.util.Map; -import java.util.HashMap; -import org.apache.thrift.TEnum; - -public enum TStatusCode implements org.apache.thrift.TEnum { - SUCCESS_STATUS(0), - SUCCESS_WITH_INFO_STATUS(1), - STILL_EXECUTING_STATUS(2), - ERROR_STATUS(3), - INVALID_HANDLE_STATUS(4); - - private final int value; - - private TStatusCode(int value) { - this.value = value; - } - - /** - * Get the integer value of this enum value, as defined in the Thrift IDL. - */ - public int getValue() { - return value; - } - - /** - * Find a the enum type by its integer value, as defined in the Thrift IDL. - * @return null if the value is not found. - */ - public static TStatusCode findByValue(int value) { - switch (value) { - case 0: - return SUCCESS_STATUS; - case 1: - return SUCCESS_WITH_INFO_STATUS; - case 2: - return STILL_EXECUTING_STATUS; - case 3: - return ERROR_STATUS; - case 4: - return INVALID_HANDLE_STATUS; - default: - return null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java deleted file mode 100644 index 3dae460c8621d..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java +++ /dev/null @@ -1,548 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TStringColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStringColumn"); - - private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); - private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TStringColumnStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TStringColumnTupleSchemeFactory()); - } - - private List values; // required - private ByteBuffer nulls; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUES((short)1, "values"), - NULLS((short)2, "nulls"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUES - return VALUES; - case 2: // NULLS - return NULLS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); - tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TStringColumn.class, metaDataMap); - } - - public TStringColumn() { - } - - public TStringColumn( - List values, - ByteBuffer nulls) - { - this(); - this.values = values; - this.nulls = nulls; - } - - /** - * Performs a deep copy on other. - */ - public TStringColumn(TStringColumn other) { - if (other.isSetValues()) { - List __this__values = new ArrayList(); - for (String other_element : other.values) { - __this__values.add(other_element); - } - this.values = __this__values; - } - if (other.isSetNulls()) { - this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); -; - } - } - - public TStringColumn deepCopy() { - return new TStringColumn(this); - } - - @Override - public void clear() { - this.values = null; - this.nulls = null; - } - - public int getValuesSize() { - return (this.values == null) ? 0 : this.values.size(); - } - - public java.util.Iterator getValuesIterator() { - return (this.values == null) ? null : this.values.iterator(); - } - - public void addToValues(String elem) { - if (this.values == null) { - this.values = new ArrayList(); - } - this.values.add(elem); - } - - public List getValues() { - return this.values; - } - - public void setValues(List values) { - this.values = values; - } - - public void unsetValues() { - this.values = null; - } - - /** Returns true if field values is set (has been assigned a value) and false otherwise */ - public boolean isSetValues() { - return this.values != null; - } - - public void setValuesIsSet(boolean value) { - if (!value) { - this.values = null; - } - } - - public byte[] getNulls() { - setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); - return nulls == null ? null : nulls.array(); - } - - public ByteBuffer bufferForNulls() { - return nulls; - } - - public void setNulls(byte[] nulls) { - setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); - } - - public void setNulls(ByteBuffer nulls) { - this.nulls = nulls; - } - - public void unsetNulls() { - this.nulls = null; - } - - /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ - public boolean isSetNulls() { - return this.nulls != null; - } - - public void setNullsIsSet(boolean value) { - if (!value) { - this.nulls = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUES: - if (value == null) { - unsetValues(); - } else { - setValues((List)value); - } - break; - - case NULLS: - if (value == null) { - unsetNulls(); - } else { - setNulls((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUES: - return getValues(); - - case NULLS: - return getNulls(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUES: - return isSetValues(); - case NULLS: - return isSetNulls(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TStringColumn) - return this.equals((TStringColumn)that); - return false; - } - - public boolean equals(TStringColumn that) { - if (that == null) - return false; - - boolean this_present_values = true && this.isSetValues(); - boolean that_present_values = true && that.isSetValues(); - if (this_present_values || that_present_values) { - if (!(this_present_values && that_present_values)) - return false; - if (!this.values.equals(that.values)) - return false; - } - - boolean this_present_nulls = true && this.isSetNulls(); - boolean that_present_nulls = true && that.isSetNulls(); - if (this_present_nulls || that_present_nulls) { - if (!(this_present_nulls && that_present_nulls)) - return false; - if (!this.nulls.equals(that.nulls)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_values = true && (isSetValues()); - builder.append(present_values); - if (present_values) - builder.append(values); - - boolean present_nulls = true && (isSetNulls()); - builder.append(present_nulls); - if (present_nulls) - builder.append(nulls); - - return builder.toHashCode(); - } - - public int compareTo(TStringColumn other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TStringColumn typedOther = (TStringColumn)other; - - lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNulls()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TStringColumn("); - boolean first = true; - - sb.append("values:"); - if (this.values == null) { - sb.append("null"); - } else { - sb.append(this.values); - } - first = false; - if (!first) sb.append(", "); - sb.append("nulls:"); - if (this.nulls == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.nulls, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetValues()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); - } - - if (!isSetNulls()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TStringColumnStandardSchemeFactory implements SchemeFactory { - public TStringColumnStandardScheme getScheme() { - return new TStringColumnStandardScheme(); - } - } - - private static class TStringColumnStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TStringColumn struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list102 = iprot.readListBegin(); - struct.values = new ArrayList(_list102.size); - for (int _i103 = 0; _i103 < _list102.size; ++_i103) - { - String _elem104; // optional - _elem104 = iprot.readString(); - struct.values.add(_elem104); - } - iprot.readListEnd(); - } - struct.setValuesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // NULLS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TStringColumn struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.values != null) { - oprot.writeFieldBegin(VALUES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.values.size())); - for (String _iter105 : struct.values) - { - oprot.writeString(_iter105); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.nulls != null) { - oprot.writeFieldBegin(NULLS_FIELD_DESC); - oprot.writeBinary(struct.nulls); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TStringColumnTupleSchemeFactory implements SchemeFactory { - public TStringColumnTupleScheme getScheme() { - return new TStringColumnTupleScheme(); - } - } - - private static class TStringColumnTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TStringColumn struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.values.size()); - for (String _iter106 : struct.values) - { - oprot.writeString(_iter106); - } - } - oprot.writeBinary(struct.nulls); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TStringColumn struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list107 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.values = new ArrayList(_list107.size); - for (int _i108 = 0; _i108 < _list107.size; ++_i108) - { - String _elem109; // optional - _elem109 = iprot.readString(); - struct.values.add(_elem109); - } - } - struct.setValuesIsSet(true); - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java deleted file mode 100644 index af7a109775a8b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java +++ /dev/null @@ -1,389 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TStringValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStringValue"); - - private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.STRING, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TStringValueStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TStringValueTupleSchemeFactory()); - } - - private String value; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUE((short)1, "value"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUE - return VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.VALUE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TStringValue.class, metaDataMap); - } - - public TStringValue() { - } - - /** - * Performs a deep copy on other. - */ - public TStringValue(TStringValue other) { - if (other.isSetValue()) { - this.value = other.value; - } - } - - public TStringValue deepCopy() { - return new TStringValue(this); - } - - @Override - public void clear() { - this.value = null; - } - - public String getValue() { - return this.value; - } - - public void setValue(String value) { - this.value = value; - } - - public void unsetValue() { - this.value = null; - } - - /** Returns true if field value is set (has been assigned a value) and false otherwise */ - public boolean isSetValue() { - return this.value != null; - } - - public void setValueIsSet(boolean value) { - if (!value) { - this.value = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUE: - if (value == null) { - unsetValue(); - } else { - setValue((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUE: - return getValue(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUE: - return isSetValue(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TStringValue) - return this.equals((TStringValue)that); - return false; - } - - public boolean equals(TStringValue that) { - if (that == null) - return false; - - boolean this_present_value = true && this.isSetValue(); - boolean that_present_value = true && that.isSetValue(); - if (this_present_value || that_present_value) { - if (!(this_present_value && that_present_value)) - return false; - if (!this.value.equals(that.value)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_value = true && (isSetValue()); - builder.append(present_value); - if (present_value) - builder.append(value); - - return builder.toHashCode(); - } - - public int compareTo(TStringValue other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TStringValue typedOther = (TStringValue)other; - - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TStringValue("); - boolean first = true; - - if (isSetValue()) { - sb.append("value:"); - if (this.value == null) { - sb.append("null"); - } else { - sb.append(this.value); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TStringValueStandardSchemeFactory implements SchemeFactory { - public TStringValueStandardScheme getScheme() { - return new TStringValueStandardScheme(); - } - } - - private static class TStringValueStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TStringValue struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUE - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.value = iprot.readString(); - struct.setValueIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TStringValue struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.value != null) { - if (struct.isSetValue()) { - oprot.writeFieldBegin(VALUE_FIELD_DESC); - oprot.writeString(struct.value); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TStringValueTupleSchemeFactory implements SchemeFactory { - public TStringValueTupleScheme getScheme() { - return new TStringValueTupleScheme(); - } - } - - private static class TStringValueTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TStringValue struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetValue()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetValue()) { - oprot.writeString(struct.value); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TStringValue struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.value = iprot.readString(); - struct.setValueIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java deleted file mode 100644 index 20f5fb6c29073..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java +++ /dev/null @@ -1,448 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TStructTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStructTypeEntry"); - - private static final org.apache.thrift.protocol.TField NAME_TO_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("nameToTypePtr", org.apache.thrift.protocol.TType.MAP, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TStructTypeEntryStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TStructTypeEntryTupleSchemeFactory()); - } - - private Map nameToTypePtr; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - NAME_TO_TYPE_PTR((short)1, "nameToTypePtr"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // NAME_TO_TYPE_PTR - return NAME_TO_TYPE_PTR; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.NAME_TO_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("nameToTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr")))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TStructTypeEntry.class, metaDataMap); - } - - public TStructTypeEntry() { - } - - public TStructTypeEntry( - Map nameToTypePtr) - { - this(); - this.nameToTypePtr = nameToTypePtr; - } - - /** - * Performs a deep copy on other. - */ - public TStructTypeEntry(TStructTypeEntry other) { - if (other.isSetNameToTypePtr()) { - Map __this__nameToTypePtr = new HashMap(); - for (Map.Entry other_element : other.nameToTypePtr.entrySet()) { - - String other_element_key = other_element.getKey(); - Integer other_element_value = other_element.getValue(); - - String __this__nameToTypePtr_copy_key = other_element_key; - - Integer __this__nameToTypePtr_copy_value = other_element_value; - - __this__nameToTypePtr.put(__this__nameToTypePtr_copy_key, __this__nameToTypePtr_copy_value); - } - this.nameToTypePtr = __this__nameToTypePtr; - } - } - - public TStructTypeEntry deepCopy() { - return new TStructTypeEntry(this); - } - - @Override - public void clear() { - this.nameToTypePtr = null; - } - - public int getNameToTypePtrSize() { - return (this.nameToTypePtr == null) ? 0 : this.nameToTypePtr.size(); - } - - public void putToNameToTypePtr(String key, int val) { - if (this.nameToTypePtr == null) { - this.nameToTypePtr = new HashMap(); - } - this.nameToTypePtr.put(key, val); - } - - public Map getNameToTypePtr() { - return this.nameToTypePtr; - } - - public void setNameToTypePtr(Map nameToTypePtr) { - this.nameToTypePtr = nameToTypePtr; - } - - public void unsetNameToTypePtr() { - this.nameToTypePtr = null; - } - - /** Returns true if field nameToTypePtr is set (has been assigned a value) and false otherwise */ - public boolean isSetNameToTypePtr() { - return this.nameToTypePtr != null; - } - - public void setNameToTypePtrIsSet(boolean value) { - if (!value) { - this.nameToTypePtr = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case NAME_TO_TYPE_PTR: - if (value == null) { - unsetNameToTypePtr(); - } else { - setNameToTypePtr((Map)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case NAME_TO_TYPE_PTR: - return getNameToTypePtr(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case NAME_TO_TYPE_PTR: - return isSetNameToTypePtr(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TStructTypeEntry) - return this.equals((TStructTypeEntry)that); - return false; - } - - public boolean equals(TStructTypeEntry that) { - if (that == null) - return false; - - boolean this_present_nameToTypePtr = true && this.isSetNameToTypePtr(); - boolean that_present_nameToTypePtr = true && that.isSetNameToTypePtr(); - if (this_present_nameToTypePtr || that_present_nameToTypePtr) { - if (!(this_present_nameToTypePtr && that_present_nameToTypePtr)) - return false; - if (!this.nameToTypePtr.equals(that.nameToTypePtr)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_nameToTypePtr = true && (isSetNameToTypePtr()); - builder.append(present_nameToTypePtr); - if (present_nameToTypePtr) - builder.append(nameToTypePtr); - - return builder.toHashCode(); - } - - public int compareTo(TStructTypeEntry other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TStructTypeEntry typedOther = (TStructTypeEntry)other; - - lastComparison = Boolean.valueOf(isSetNameToTypePtr()).compareTo(typedOther.isSetNameToTypePtr()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNameToTypePtr()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nameToTypePtr, typedOther.nameToTypePtr); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TStructTypeEntry("); - boolean first = true; - - sb.append("nameToTypePtr:"); - if (this.nameToTypePtr == null) { - sb.append("null"); - } else { - sb.append(this.nameToTypePtr); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetNameToTypePtr()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nameToTypePtr' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TStructTypeEntryStandardSchemeFactory implements SchemeFactory { - public TStructTypeEntryStandardScheme getScheme() { - return new TStructTypeEntryStandardScheme(); - } - } - - private static class TStructTypeEntryStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TStructTypeEntry struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // NAME_TO_TYPE_PTR - if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { - { - org.apache.thrift.protocol.TMap _map10 = iprot.readMapBegin(); - struct.nameToTypePtr = new HashMap(2*_map10.size); - for (int _i11 = 0; _i11 < _map10.size; ++_i11) - { - String _key12; // required - int _val13; // required - _key12 = iprot.readString(); - _val13 = iprot.readI32(); - struct.nameToTypePtr.put(_key12, _val13); - } - iprot.readMapEnd(); - } - struct.setNameToTypePtrIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TStructTypeEntry struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.nameToTypePtr != null) { - oprot.writeFieldBegin(NAME_TO_TYPE_PTR_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.nameToTypePtr.size())); - for (Map.Entry _iter14 : struct.nameToTypePtr.entrySet()) - { - oprot.writeString(_iter14.getKey()); - oprot.writeI32(_iter14.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TStructTypeEntryTupleSchemeFactory implements SchemeFactory { - public TStructTypeEntryTupleScheme getScheme() { - return new TStructTypeEntryTupleScheme(); - } - } - - private static class TStructTypeEntryTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TStructTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.nameToTypePtr.size()); - for (Map.Entry _iter15 : struct.nameToTypePtr.entrySet()) - { - oprot.writeString(_iter15.getKey()); - oprot.writeI32(_iter15.getValue()); - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TStructTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TMap _map16 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); - struct.nameToTypePtr = new HashMap(2*_map16.size); - for (int _i17 = 0; _i17 < _map16.size; ++_i17) - { - String _key18; // required - int _val19; // required - _key18 = iprot.readString(); - _val19 = iprot.readI32(); - struct.nameToTypePtr.put(_key18, _val19); - } - } - struct.setNameToTypePtrIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java deleted file mode 100644 index ff5e54db7c16c..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java +++ /dev/null @@ -1,439 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TTableSchema implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableSchema"); - - private static final org.apache.thrift.protocol.TField COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.LIST, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TTableSchemaStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TTableSchemaTupleSchemeFactory()); - } - - private List columns; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - COLUMNS((short)1, "columns"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // COLUMNS - return COLUMNS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnDesc.class)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTableSchema.class, metaDataMap); - } - - public TTableSchema() { - } - - public TTableSchema( - List columns) - { - this(); - this.columns = columns; - } - - /** - * Performs a deep copy on other. - */ - public TTableSchema(TTableSchema other) { - if (other.isSetColumns()) { - List __this__columns = new ArrayList(); - for (TColumnDesc other_element : other.columns) { - __this__columns.add(new TColumnDesc(other_element)); - } - this.columns = __this__columns; - } - } - - public TTableSchema deepCopy() { - return new TTableSchema(this); - } - - @Override - public void clear() { - this.columns = null; - } - - public int getColumnsSize() { - return (this.columns == null) ? 0 : this.columns.size(); - } - - public java.util.Iterator getColumnsIterator() { - return (this.columns == null) ? null : this.columns.iterator(); - } - - public void addToColumns(TColumnDesc elem) { - if (this.columns == null) { - this.columns = new ArrayList(); - } - this.columns.add(elem); - } - - public List getColumns() { - return this.columns; - } - - public void setColumns(List columns) { - this.columns = columns; - } - - public void unsetColumns() { - this.columns = null; - } - - /** Returns true if field columns is set (has been assigned a value) and false otherwise */ - public boolean isSetColumns() { - return this.columns != null; - } - - public void setColumnsIsSet(boolean value) { - if (!value) { - this.columns = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case COLUMNS: - if (value == null) { - unsetColumns(); - } else { - setColumns((List)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case COLUMNS: - return getColumns(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case COLUMNS: - return isSetColumns(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TTableSchema) - return this.equals((TTableSchema)that); - return false; - } - - public boolean equals(TTableSchema that) { - if (that == null) - return false; - - boolean this_present_columns = true && this.isSetColumns(); - boolean that_present_columns = true && that.isSetColumns(); - if (this_present_columns || that_present_columns) { - if (!(this_present_columns && that_present_columns)) - return false; - if (!this.columns.equals(that.columns)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_columns = true && (isSetColumns()); - builder.append(present_columns); - if (present_columns) - builder.append(columns); - - return builder.toHashCode(); - } - - public int compareTo(TTableSchema other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TTableSchema typedOther = (TTableSchema)other; - - lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TTableSchema("); - boolean first = true; - - sb.append("columns:"); - if (this.columns == null) { - sb.append("null"); - } else { - sb.append(this.columns); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetColumns()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'columns' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TTableSchemaStandardSchemeFactory implements SchemeFactory { - public TTableSchemaStandardScheme getScheme() { - return new TTableSchemaStandardScheme(); - } - } - - private static class TTableSchemaStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TTableSchema struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // COLUMNS - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list38 = iprot.readListBegin(); - struct.columns = new ArrayList(_list38.size); - for (int _i39 = 0; _i39 < _list38.size; ++_i39) - { - TColumnDesc _elem40; // optional - _elem40 = new TColumnDesc(); - _elem40.read(iprot); - struct.columns.add(_elem40); - } - iprot.readListEnd(); - } - struct.setColumnsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TTableSchema struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.columns != null) { - oprot.writeFieldBegin(COLUMNS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size())); - for (TColumnDesc _iter41 : struct.columns) - { - _iter41.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TTableSchemaTupleSchemeFactory implements SchemeFactory { - public TTableSchemaTupleScheme getScheme() { - return new TTableSchemaTupleScheme(); - } - } - - private static class TTableSchemaTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TTableSchema struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.columns.size()); - for (TColumnDesc _iter42 : struct.columns) - { - _iter42.write(oprot); - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TTableSchema struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list43 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.columns = new ArrayList(_list43.size); - for (int _i44 = 0; _i44 < _list43.size; ++_i44) - { - TColumnDesc _elem45; // optional - _elem45 = new TColumnDesc(); - _elem45.read(iprot); - struct.columns.add(_elem45); - } - } - struct.setColumnsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java deleted file mode 100644 index 251f86a914719..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java +++ /dev/null @@ -1,439 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TTypeDesc implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeDesc"); - - private static final org.apache.thrift.protocol.TField TYPES_FIELD_DESC = new org.apache.thrift.protocol.TField("types", org.apache.thrift.protocol.TType.LIST, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TTypeDescStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TTypeDescTupleSchemeFactory()); - } - - private List types; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - TYPES((short)1, "types"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // TYPES - return TYPES; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.TYPES, new org.apache.thrift.meta_data.FieldMetaData("types", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTypeEntry.class)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTypeDesc.class, metaDataMap); - } - - public TTypeDesc() { - } - - public TTypeDesc( - List types) - { - this(); - this.types = types; - } - - /** - * Performs a deep copy on other. - */ - public TTypeDesc(TTypeDesc other) { - if (other.isSetTypes()) { - List __this__types = new ArrayList(); - for (TTypeEntry other_element : other.types) { - __this__types.add(new TTypeEntry(other_element)); - } - this.types = __this__types; - } - } - - public TTypeDesc deepCopy() { - return new TTypeDesc(this); - } - - @Override - public void clear() { - this.types = null; - } - - public int getTypesSize() { - return (this.types == null) ? 0 : this.types.size(); - } - - public java.util.Iterator getTypesIterator() { - return (this.types == null) ? null : this.types.iterator(); - } - - public void addToTypes(TTypeEntry elem) { - if (this.types == null) { - this.types = new ArrayList(); - } - this.types.add(elem); - } - - public List getTypes() { - return this.types; - } - - public void setTypes(List types) { - this.types = types; - } - - public void unsetTypes() { - this.types = null; - } - - /** Returns true if field types is set (has been assigned a value) and false otherwise */ - public boolean isSetTypes() { - return this.types != null; - } - - public void setTypesIsSet(boolean value) { - if (!value) { - this.types = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case TYPES: - if (value == null) { - unsetTypes(); - } else { - setTypes((List)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case TYPES: - return getTypes(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case TYPES: - return isSetTypes(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TTypeDesc) - return this.equals((TTypeDesc)that); - return false; - } - - public boolean equals(TTypeDesc that) { - if (that == null) - return false; - - boolean this_present_types = true && this.isSetTypes(); - boolean that_present_types = true && that.isSetTypes(); - if (this_present_types || that_present_types) { - if (!(this_present_types && that_present_types)) - return false; - if (!this.types.equals(that.types)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_types = true && (isSetTypes()); - builder.append(present_types); - if (present_types) - builder.append(types); - - return builder.toHashCode(); - } - - public int compareTo(TTypeDesc other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TTypeDesc typedOther = (TTypeDesc)other; - - lastComparison = Boolean.valueOf(isSetTypes()).compareTo(typedOther.isSetTypes()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetTypes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.types, typedOther.types); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TTypeDesc("); - boolean first = true; - - sb.append("types:"); - if (this.types == null) { - sb.append("null"); - } else { - sb.append(this.types); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetTypes()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'types' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TTypeDescStandardSchemeFactory implements SchemeFactory { - public TTypeDescStandardScheme getScheme() { - return new TTypeDescStandardScheme(); - } - } - - private static class TTypeDescStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TTypeDesc struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // TYPES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list30 = iprot.readListBegin(); - struct.types = new ArrayList(_list30.size); - for (int _i31 = 0; _i31 < _list30.size; ++_i31) - { - TTypeEntry _elem32; // optional - _elem32 = new TTypeEntry(); - _elem32.read(iprot); - struct.types.add(_elem32); - } - iprot.readListEnd(); - } - struct.setTypesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TTypeDesc struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.types != null) { - oprot.writeFieldBegin(TYPES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.types.size())); - for (TTypeEntry _iter33 : struct.types) - { - _iter33.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TTypeDescTupleSchemeFactory implements SchemeFactory { - public TTypeDescTupleScheme getScheme() { - return new TTypeDescTupleScheme(); - } - } - - private static class TTypeDescTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TTypeDesc struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.types.size()); - for (TTypeEntry _iter34 : struct.types) - { - _iter34.write(oprot); - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TTypeDesc struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list35 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.types = new ArrayList(_list35.size); - for (int _i36 = 0; _i36 < _list35.size; ++_i36) - { - TTypeEntry _elem37; // optional - _elem37 = new TTypeEntry(); - _elem37.read(iprot); - struct.types.add(_elem37); - } - } - struct.setTypesIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java deleted file mode 100644 index d0d70c1279572..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java +++ /dev/null @@ -1,610 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TTypeEntry extends org.apache.thrift.TUnion { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeEntry"); - private static final org.apache.thrift.protocol.TField PRIMITIVE_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("primitiveEntry", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField ARRAY_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("arrayEntry", org.apache.thrift.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift.protocol.TField MAP_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("mapEntry", org.apache.thrift.protocol.TType.STRUCT, (short)3); - private static final org.apache.thrift.protocol.TField STRUCT_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("structEntry", org.apache.thrift.protocol.TType.STRUCT, (short)4); - private static final org.apache.thrift.protocol.TField UNION_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("unionEntry", org.apache.thrift.protocol.TType.STRUCT, (short)5); - private static final org.apache.thrift.protocol.TField USER_DEFINED_TYPE_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("userDefinedTypeEntry", org.apache.thrift.protocol.TType.STRUCT, (short)6); - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - PRIMITIVE_ENTRY((short)1, "primitiveEntry"), - ARRAY_ENTRY((short)2, "arrayEntry"), - MAP_ENTRY((short)3, "mapEntry"), - STRUCT_ENTRY((short)4, "structEntry"), - UNION_ENTRY((short)5, "unionEntry"), - USER_DEFINED_TYPE_ENTRY((short)6, "userDefinedTypeEntry"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // PRIMITIVE_ENTRY - return PRIMITIVE_ENTRY; - case 2: // ARRAY_ENTRY - return ARRAY_ENTRY; - case 3: // MAP_ENTRY - return MAP_ENTRY; - case 4: // STRUCT_ENTRY - return STRUCT_ENTRY; - case 5: // UNION_ENTRY - return UNION_ENTRY; - case 6: // USER_DEFINED_TYPE_ENTRY - return USER_DEFINED_TYPE_ENTRY; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.PRIMITIVE_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("primitiveEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TPrimitiveTypeEntry.class))); - tmpMap.put(_Fields.ARRAY_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("arrayEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TArrayTypeEntry.class))); - tmpMap.put(_Fields.MAP_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("mapEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TMapTypeEntry.class))); - tmpMap.put(_Fields.STRUCT_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("structEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStructTypeEntry.class))); - tmpMap.put(_Fields.UNION_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("unionEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TUnionTypeEntry.class))); - tmpMap.put(_Fields.USER_DEFINED_TYPE_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("userDefinedTypeEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TUserDefinedTypeEntry.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTypeEntry.class, metaDataMap); - } - - public TTypeEntry() { - super(); - } - - public TTypeEntry(TTypeEntry._Fields setField, Object value) { - super(setField, value); - } - - public TTypeEntry(TTypeEntry other) { - super(other); - } - public TTypeEntry deepCopy() { - return new TTypeEntry(this); - } - - public static TTypeEntry primitiveEntry(TPrimitiveTypeEntry value) { - TTypeEntry x = new TTypeEntry(); - x.setPrimitiveEntry(value); - return x; - } - - public static TTypeEntry arrayEntry(TArrayTypeEntry value) { - TTypeEntry x = new TTypeEntry(); - x.setArrayEntry(value); - return x; - } - - public static TTypeEntry mapEntry(TMapTypeEntry value) { - TTypeEntry x = new TTypeEntry(); - x.setMapEntry(value); - return x; - } - - public static TTypeEntry structEntry(TStructTypeEntry value) { - TTypeEntry x = new TTypeEntry(); - x.setStructEntry(value); - return x; - } - - public static TTypeEntry unionEntry(TUnionTypeEntry value) { - TTypeEntry x = new TTypeEntry(); - x.setUnionEntry(value); - return x; - } - - public static TTypeEntry userDefinedTypeEntry(TUserDefinedTypeEntry value) { - TTypeEntry x = new TTypeEntry(); - x.setUserDefinedTypeEntry(value); - return x; - } - - - @Override - protected void checkType(_Fields setField, Object value) throws ClassCastException { - switch (setField) { - case PRIMITIVE_ENTRY: - if (value instanceof TPrimitiveTypeEntry) { - break; - } - throw new ClassCastException("Was expecting value of type TPrimitiveTypeEntry for field 'primitiveEntry', but got " + value.getClass().getSimpleName()); - case ARRAY_ENTRY: - if (value instanceof TArrayTypeEntry) { - break; - } - throw new ClassCastException("Was expecting value of type TArrayTypeEntry for field 'arrayEntry', but got " + value.getClass().getSimpleName()); - case MAP_ENTRY: - if (value instanceof TMapTypeEntry) { - break; - } - throw new ClassCastException("Was expecting value of type TMapTypeEntry for field 'mapEntry', but got " + value.getClass().getSimpleName()); - case STRUCT_ENTRY: - if (value instanceof TStructTypeEntry) { - break; - } - throw new ClassCastException("Was expecting value of type TStructTypeEntry for field 'structEntry', but got " + value.getClass().getSimpleName()); - case UNION_ENTRY: - if (value instanceof TUnionTypeEntry) { - break; - } - throw new ClassCastException("Was expecting value of type TUnionTypeEntry for field 'unionEntry', but got " + value.getClass().getSimpleName()); - case USER_DEFINED_TYPE_ENTRY: - if (value instanceof TUserDefinedTypeEntry) { - break; - } - throw new ClassCastException("Was expecting value of type TUserDefinedTypeEntry for field 'userDefinedTypeEntry', but got " + value.getClass().getSimpleName()); - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(field.id); - if (setField != null) { - switch (setField) { - case PRIMITIVE_ENTRY: - if (field.type == PRIMITIVE_ENTRY_FIELD_DESC.type) { - TPrimitiveTypeEntry primitiveEntry; - primitiveEntry = new TPrimitiveTypeEntry(); - primitiveEntry.read(iprot); - return primitiveEntry; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case ARRAY_ENTRY: - if (field.type == ARRAY_ENTRY_FIELD_DESC.type) { - TArrayTypeEntry arrayEntry; - arrayEntry = new TArrayTypeEntry(); - arrayEntry.read(iprot); - return arrayEntry; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case MAP_ENTRY: - if (field.type == MAP_ENTRY_FIELD_DESC.type) { - TMapTypeEntry mapEntry; - mapEntry = new TMapTypeEntry(); - mapEntry.read(iprot); - return mapEntry; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case STRUCT_ENTRY: - if (field.type == STRUCT_ENTRY_FIELD_DESC.type) { - TStructTypeEntry structEntry; - structEntry = new TStructTypeEntry(); - structEntry.read(iprot); - return structEntry; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case UNION_ENTRY: - if (field.type == UNION_ENTRY_FIELD_DESC.type) { - TUnionTypeEntry unionEntry; - unionEntry = new TUnionTypeEntry(); - unionEntry.read(iprot); - return unionEntry; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case USER_DEFINED_TYPE_ENTRY: - if (field.type == USER_DEFINED_TYPE_ENTRY_FIELD_DESC.type) { - TUserDefinedTypeEntry userDefinedTypeEntry; - userDefinedTypeEntry = new TUserDefinedTypeEntry(); - userDefinedTypeEntry.read(iprot); - return userDefinedTypeEntry; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - return null; - } - } - - @Override - protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case PRIMITIVE_ENTRY: - TPrimitiveTypeEntry primitiveEntry = (TPrimitiveTypeEntry)value_; - primitiveEntry.write(oprot); - return; - case ARRAY_ENTRY: - TArrayTypeEntry arrayEntry = (TArrayTypeEntry)value_; - arrayEntry.write(oprot); - return; - case MAP_ENTRY: - TMapTypeEntry mapEntry = (TMapTypeEntry)value_; - mapEntry.write(oprot); - return; - case STRUCT_ENTRY: - TStructTypeEntry structEntry = (TStructTypeEntry)value_; - structEntry.write(oprot); - return; - case UNION_ENTRY: - TUnionTypeEntry unionEntry = (TUnionTypeEntry)value_; - unionEntry.write(oprot); - return; - case USER_DEFINED_TYPE_ENTRY: - TUserDefinedTypeEntry userDefinedTypeEntry = (TUserDefinedTypeEntry)value_; - userDefinedTypeEntry.write(oprot); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(fieldID); - if (setField != null) { - switch (setField) { - case PRIMITIVE_ENTRY: - TPrimitiveTypeEntry primitiveEntry; - primitiveEntry = new TPrimitiveTypeEntry(); - primitiveEntry.read(iprot); - return primitiveEntry; - case ARRAY_ENTRY: - TArrayTypeEntry arrayEntry; - arrayEntry = new TArrayTypeEntry(); - arrayEntry.read(iprot); - return arrayEntry; - case MAP_ENTRY: - TMapTypeEntry mapEntry; - mapEntry = new TMapTypeEntry(); - mapEntry.read(iprot); - return mapEntry; - case STRUCT_ENTRY: - TStructTypeEntry structEntry; - structEntry = new TStructTypeEntry(); - structEntry.read(iprot); - return structEntry; - case UNION_ENTRY: - TUnionTypeEntry unionEntry; - unionEntry = new TUnionTypeEntry(); - unionEntry.read(iprot); - return unionEntry; - case USER_DEFINED_TYPE_ENTRY: - TUserDefinedTypeEntry userDefinedTypeEntry; - userDefinedTypeEntry = new TUserDefinedTypeEntry(); - userDefinedTypeEntry.read(iprot); - return userDefinedTypeEntry; - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - throw new TProtocolException("Couldn't find a field with field id " + fieldID); - } - } - - @Override - protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case PRIMITIVE_ENTRY: - TPrimitiveTypeEntry primitiveEntry = (TPrimitiveTypeEntry)value_; - primitiveEntry.write(oprot); - return; - case ARRAY_ENTRY: - TArrayTypeEntry arrayEntry = (TArrayTypeEntry)value_; - arrayEntry.write(oprot); - return; - case MAP_ENTRY: - TMapTypeEntry mapEntry = (TMapTypeEntry)value_; - mapEntry.write(oprot); - return; - case STRUCT_ENTRY: - TStructTypeEntry structEntry = (TStructTypeEntry)value_; - structEntry.write(oprot); - return; - case UNION_ENTRY: - TUnionTypeEntry unionEntry = (TUnionTypeEntry)value_; - unionEntry.write(oprot); - return; - case USER_DEFINED_TYPE_ENTRY: - TUserDefinedTypeEntry userDefinedTypeEntry = (TUserDefinedTypeEntry)value_; - userDefinedTypeEntry.write(oprot); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { - switch (setField) { - case PRIMITIVE_ENTRY: - return PRIMITIVE_ENTRY_FIELD_DESC; - case ARRAY_ENTRY: - return ARRAY_ENTRY_FIELD_DESC; - case MAP_ENTRY: - return MAP_ENTRY_FIELD_DESC; - case STRUCT_ENTRY: - return STRUCT_ENTRY_FIELD_DESC; - case UNION_ENTRY: - return UNION_ENTRY_FIELD_DESC; - case USER_DEFINED_TYPE_ENTRY: - return USER_DEFINED_TYPE_ENTRY_FIELD_DESC; - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected org.apache.thrift.protocol.TStruct getStructDesc() { - return STRUCT_DESC; - } - - @Override - protected _Fields enumForId(short id) { - return _Fields.findByThriftIdOrThrow(id); - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - - public TPrimitiveTypeEntry getPrimitiveEntry() { - if (getSetField() == _Fields.PRIMITIVE_ENTRY) { - return (TPrimitiveTypeEntry)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'primitiveEntry' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setPrimitiveEntry(TPrimitiveTypeEntry value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.PRIMITIVE_ENTRY; - value_ = value; - } - - public TArrayTypeEntry getArrayEntry() { - if (getSetField() == _Fields.ARRAY_ENTRY) { - return (TArrayTypeEntry)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'arrayEntry' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setArrayEntry(TArrayTypeEntry value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.ARRAY_ENTRY; - value_ = value; - } - - public TMapTypeEntry getMapEntry() { - if (getSetField() == _Fields.MAP_ENTRY) { - return (TMapTypeEntry)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'mapEntry' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setMapEntry(TMapTypeEntry value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.MAP_ENTRY; - value_ = value; - } - - public TStructTypeEntry getStructEntry() { - if (getSetField() == _Fields.STRUCT_ENTRY) { - return (TStructTypeEntry)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'structEntry' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setStructEntry(TStructTypeEntry value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.STRUCT_ENTRY; - value_ = value; - } - - public TUnionTypeEntry getUnionEntry() { - if (getSetField() == _Fields.UNION_ENTRY) { - return (TUnionTypeEntry)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'unionEntry' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setUnionEntry(TUnionTypeEntry value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.UNION_ENTRY; - value_ = value; - } - - public TUserDefinedTypeEntry getUserDefinedTypeEntry() { - if (getSetField() == _Fields.USER_DEFINED_TYPE_ENTRY) { - return (TUserDefinedTypeEntry)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'userDefinedTypeEntry' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setUserDefinedTypeEntry(TUserDefinedTypeEntry value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.USER_DEFINED_TYPE_ENTRY; - value_ = value; - } - - public boolean isSetPrimitiveEntry() { - return setField_ == _Fields.PRIMITIVE_ENTRY; - } - - - public boolean isSetArrayEntry() { - return setField_ == _Fields.ARRAY_ENTRY; - } - - - public boolean isSetMapEntry() { - return setField_ == _Fields.MAP_ENTRY; - } - - - public boolean isSetStructEntry() { - return setField_ == _Fields.STRUCT_ENTRY; - } - - - public boolean isSetUnionEntry() { - return setField_ == _Fields.UNION_ENTRY; - } - - - public boolean isSetUserDefinedTypeEntry() { - return setField_ == _Fields.USER_DEFINED_TYPE_ENTRY; - } - - - public boolean equals(Object other) { - if (other instanceof TTypeEntry) { - return equals((TTypeEntry)other); - } else { - return false; - } - } - - public boolean equals(TTypeEntry other) { - return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); - } - - @Override - public int compareTo(TTypeEntry other) { - int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); - if (lastComparison == 0) { - return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); - } - return lastComparison; - } - - - @Override - public int hashCode() { - HashCodeBuilder hcb = new HashCodeBuilder(); - hcb.append(this.getClass().getName()); - org.apache.thrift.TFieldIdEnum setField = getSetField(); - if (setField != null) { - hcb.append(setField.getThriftFieldId()); - Object value = getFieldValue(); - if (value instanceof org.apache.thrift.TEnum) { - hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue()); - } else { - hcb.append(value); - } - } - return hcb.toHashCode(); - } - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java deleted file mode 100644 index 40f05894623c0..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java +++ /dev/null @@ -1,105 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - - -import java.util.Map; -import java.util.HashMap; -import org.apache.thrift.TEnum; - -public enum TTypeId implements org.apache.thrift.TEnum { - BOOLEAN_TYPE(0), - TINYINT_TYPE(1), - SMALLINT_TYPE(2), - INT_TYPE(3), - BIGINT_TYPE(4), - FLOAT_TYPE(5), - DOUBLE_TYPE(6), - STRING_TYPE(7), - TIMESTAMP_TYPE(8), - BINARY_TYPE(9), - ARRAY_TYPE(10), - MAP_TYPE(11), - STRUCT_TYPE(12), - UNION_TYPE(13), - USER_DEFINED_TYPE(14), - DECIMAL_TYPE(15), - NULL_TYPE(16), - DATE_TYPE(17), - VARCHAR_TYPE(18), - CHAR_TYPE(19), - INTERVAL_YEAR_MONTH_TYPE(20), - INTERVAL_DAY_TIME_TYPE(21); - - private final int value; - - private TTypeId(int value) { - this.value = value; - } - - /** - * Get the integer value of this enum value, as defined in the Thrift IDL. - */ - public int getValue() { - return value; - } - - /** - * Find a the enum type by its integer value, as defined in the Thrift IDL. - * @return null if the value is not found. - */ - public static TTypeId findByValue(int value) { - switch (value) { - case 0: - return BOOLEAN_TYPE; - case 1: - return TINYINT_TYPE; - case 2: - return SMALLINT_TYPE; - case 3: - return INT_TYPE; - case 4: - return BIGINT_TYPE; - case 5: - return FLOAT_TYPE; - case 6: - return DOUBLE_TYPE; - case 7: - return STRING_TYPE; - case 8: - return TIMESTAMP_TYPE; - case 9: - return BINARY_TYPE; - case 10: - return ARRAY_TYPE; - case 11: - return MAP_TYPE; - case 12: - return STRUCT_TYPE; - case 13: - return UNION_TYPE; - case 14: - return USER_DEFINED_TYPE; - case 15: - return DECIMAL_TYPE; - case 16: - return NULL_TYPE; - case 17: - return DATE_TYPE; - case 18: - return VARCHAR_TYPE; - case 19: - return CHAR_TYPE; - case 20: - return INTERVAL_YEAR_MONTH_TYPE; - case 21: - return INTERVAL_DAY_TIME_TYPE; - default: - return null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java deleted file mode 100644 index a3e3829372276..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java +++ /dev/null @@ -1,361 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TTypeQualifierValue extends org.apache.thrift.TUnion { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeQualifierValue"); - private static final org.apache.thrift.protocol.TField I32_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("i32Value", org.apache.thrift.protocol.TType.I32, (short)1); - private static final org.apache.thrift.protocol.TField STRING_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("stringValue", org.apache.thrift.protocol.TType.STRING, (short)2); - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - I32_VALUE((short)1, "i32Value"), - STRING_VALUE((short)2, "stringValue"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // I32_VALUE - return I32_VALUE; - case 2: // STRING_VALUE - return STRING_VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.I32_VALUE, new org.apache.thrift.meta_data.FieldMetaData("i32Value", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); - tmpMap.put(_Fields.STRING_VALUE, new org.apache.thrift.meta_data.FieldMetaData("stringValue", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTypeQualifierValue.class, metaDataMap); - } - - public TTypeQualifierValue() { - super(); - } - - public TTypeQualifierValue(TTypeQualifierValue._Fields setField, Object value) { - super(setField, value); - } - - public TTypeQualifierValue(TTypeQualifierValue other) { - super(other); - } - public TTypeQualifierValue deepCopy() { - return new TTypeQualifierValue(this); - } - - public static TTypeQualifierValue i32Value(int value) { - TTypeQualifierValue x = new TTypeQualifierValue(); - x.setI32Value(value); - return x; - } - - public static TTypeQualifierValue stringValue(String value) { - TTypeQualifierValue x = new TTypeQualifierValue(); - x.setStringValue(value); - return x; - } - - - @Override - protected void checkType(_Fields setField, Object value) throws ClassCastException { - switch (setField) { - case I32_VALUE: - if (value instanceof Integer) { - break; - } - throw new ClassCastException("Was expecting value of type Integer for field 'i32Value', but got " + value.getClass().getSimpleName()); - case STRING_VALUE: - if (value instanceof String) { - break; - } - throw new ClassCastException("Was expecting value of type String for field 'stringValue', but got " + value.getClass().getSimpleName()); - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(field.id); - if (setField != null) { - switch (setField) { - case I32_VALUE: - if (field.type == I32_VALUE_FIELD_DESC.type) { - Integer i32Value; - i32Value = iprot.readI32(); - return i32Value; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case STRING_VALUE: - if (field.type == STRING_VALUE_FIELD_DESC.type) { - String stringValue; - stringValue = iprot.readString(); - return stringValue; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - return null; - } - } - - @Override - protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case I32_VALUE: - Integer i32Value = (Integer)value_; - oprot.writeI32(i32Value); - return; - case STRING_VALUE: - String stringValue = (String)value_; - oprot.writeString(stringValue); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(fieldID); - if (setField != null) { - switch (setField) { - case I32_VALUE: - Integer i32Value; - i32Value = iprot.readI32(); - return i32Value; - case STRING_VALUE: - String stringValue; - stringValue = iprot.readString(); - return stringValue; - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - throw new TProtocolException("Couldn't find a field with field id " + fieldID); - } - } - - @Override - protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case I32_VALUE: - Integer i32Value = (Integer)value_; - oprot.writeI32(i32Value); - return; - case STRING_VALUE: - String stringValue = (String)value_; - oprot.writeString(stringValue); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { - switch (setField) { - case I32_VALUE: - return I32_VALUE_FIELD_DESC; - case STRING_VALUE: - return STRING_VALUE_FIELD_DESC; - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected org.apache.thrift.protocol.TStruct getStructDesc() { - return STRUCT_DESC; - } - - @Override - protected _Fields enumForId(short id) { - return _Fields.findByThriftIdOrThrow(id); - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - - public int getI32Value() { - if (getSetField() == _Fields.I32_VALUE) { - return (Integer)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'i32Value' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setI32Value(int value) { - setField_ = _Fields.I32_VALUE; - value_ = value; - } - - public String getStringValue() { - if (getSetField() == _Fields.STRING_VALUE) { - return (String)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'stringValue' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setStringValue(String value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.STRING_VALUE; - value_ = value; - } - - public boolean isSetI32Value() { - return setField_ == _Fields.I32_VALUE; - } - - - public boolean isSetStringValue() { - return setField_ == _Fields.STRING_VALUE; - } - - - public boolean equals(Object other) { - if (other instanceof TTypeQualifierValue) { - return equals((TTypeQualifierValue)other); - } else { - return false; - } - } - - public boolean equals(TTypeQualifierValue other) { - return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); - } - - @Override - public int compareTo(TTypeQualifierValue other) { - int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); - if (lastComparison == 0) { - return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); - } - return lastComparison; - } - - - @Override - public int hashCode() { - HashCodeBuilder hcb = new HashCodeBuilder(); - hcb.append(this.getClass().getName()); - org.apache.thrift.TFieldIdEnum setField = getSetField(); - if (setField != null) { - hcb.append(setField.getThriftFieldId()); - Object value = getFieldValue(); - if (value instanceof org.apache.thrift.TEnum) { - hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue()); - } else { - hcb.append(value); - } - } - return hcb.toHashCode(); - } - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java deleted file mode 100644 index 39355551d3722..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java +++ /dev/null @@ -1,450 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TTypeQualifiers implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeQualifiers"); - - private static final org.apache.thrift.protocol.TField QUALIFIERS_FIELD_DESC = new org.apache.thrift.protocol.TField("qualifiers", org.apache.thrift.protocol.TType.MAP, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TTypeQualifiersStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TTypeQualifiersTupleSchemeFactory()); - } - - private Map qualifiers; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - QUALIFIERS((short)1, "qualifiers"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // QUALIFIERS - return QUALIFIERS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.QUALIFIERS, new org.apache.thrift.meta_data.FieldMetaData("qualifiers", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTypeQualifierValue.class)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTypeQualifiers.class, metaDataMap); - } - - public TTypeQualifiers() { - } - - public TTypeQualifiers( - Map qualifiers) - { - this(); - this.qualifiers = qualifiers; - } - - /** - * Performs a deep copy on other. - */ - public TTypeQualifiers(TTypeQualifiers other) { - if (other.isSetQualifiers()) { - Map __this__qualifiers = new HashMap(); - for (Map.Entry other_element : other.qualifiers.entrySet()) { - - String other_element_key = other_element.getKey(); - TTypeQualifierValue other_element_value = other_element.getValue(); - - String __this__qualifiers_copy_key = other_element_key; - - TTypeQualifierValue __this__qualifiers_copy_value = new TTypeQualifierValue(other_element_value); - - __this__qualifiers.put(__this__qualifiers_copy_key, __this__qualifiers_copy_value); - } - this.qualifiers = __this__qualifiers; - } - } - - public TTypeQualifiers deepCopy() { - return new TTypeQualifiers(this); - } - - @Override - public void clear() { - this.qualifiers = null; - } - - public int getQualifiersSize() { - return (this.qualifiers == null) ? 0 : this.qualifiers.size(); - } - - public void putToQualifiers(String key, TTypeQualifierValue val) { - if (this.qualifiers == null) { - this.qualifiers = new HashMap(); - } - this.qualifiers.put(key, val); - } - - public Map getQualifiers() { - return this.qualifiers; - } - - public void setQualifiers(Map qualifiers) { - this.qualifiers = qualifiers; - } - - public void unsetQualifiers() { - this.qualifiers = null; - } - - /** Returns true if field qualifiers is set (has been assigned a value) and false otherwise */ - public boolean isSetQualifiers() { - return this.qualifiers != null; - } - - public void setQualifiersIsSet(boolean value) { - if (!value) { - this.qualifiers = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case QUALIFIERS: - if (value == null) { - unsetQualifiers(); - } else { - setQualifiers((Map)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case QUALIFIERS: - return getQualifiers(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case QUALIFIERS: - return isSetQualifiers(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TTypeQualifiers) - return this.equals((TTypeQualifiers)that); - return false; - } - - public boolean equals(TTypeQualifiers that) { - if (that == null) - return false; - - boolean this_present_qualifiers = true && this.isSetQualifiers(); - boolean that_present_qualifiers = true && that.isSetQualifiers(); - if (this_present_qualifiers || that_present_qualifiers) { - if (!(this_present_qualifiers && that_present_qualifiers)) - return false; - if (!this.qualifiers.equals(that.qualifiers)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_qualifiers = true && (isSetQualifiers()); - builder.append(present_qualifiers); - if (present_qualifiers) - builder.append(qualifiers); - - return builder.toHashCode(); - } - - public int compareTo(TTypeQualifiers other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TTypeQualifiers typedOther = (TTypeQualifiers)other; - - lastComparison = Boolean.valueOf(isSetQualifiers()).compareTo(typedOther.isSetQualifiers()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetQualifiers()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.qualifiers, typedOther.qualifiers); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TTypeQualifiers("); - boolean first = true; - - sb.append("qualifiers:"); - if (this.qualifiers == null) { - sb.append("null"); - } else { - sb.append(this.qualifiers); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetQualifiers()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'qualifiers' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TTypeQualifiersStandardSchemeFactory implements SchemeFactory { - public TTypeQualifiersStandardScheme getScheme() { - return new TTypeQualifiersStandardScheme(); - } - } - - private static class TTypeQualifiersStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TTypeQualifiers struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // QUALIFIERS - if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { - { - org.apache.thrift.protocol.TMap _map0 = iprot.readMapBegin(); - struct.qualifiers = new HashMap(2*_map0.size); - for (int _i1 = 0; _i1 < _map0.size; ++_i1) - { - String _key2; // required - TTypeQualifierValue _val3; // required - _key2 = iprot.readString(); - _val3 = new TTypeQualifierValue(); - _val3.read(iprot); - struct.qualifiers.put(_key2, _val3); - } - iprot.readMapEnd(); - } - struct.setQualifiersIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TTypeQualifiers struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.qualifiers != null) { - oprot.writeFieldBegin(QUALIFIERS_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.qualifiers.size())); - for (Map.Entry _iter4 : struct.qualifiers.entrySet()) - { - oprot.writeString(_iter4.getKey()); - _iter4.getValue().write(oprot); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TTypeQualifiersTupleSchemeFactory implements SchemeFactory { - public TTypeQualifiersTupleScheme getScheme() { - return new TTypeQualifiersTupleScheme(); - } - } - - private static class TTypeQualifiersTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TTypeQualifiers struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.qualifiers.size()); - for (Map.Entry _iter5 : struct.qualifiers.entrySet()) - { - oprot.writeString(_iter5.getKey()); - _iter5.getValue().write(oprot); - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TTypeQualifiers struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TMap _map6 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.qualifiers = new HashMap(2*_map6.size); - for (int _i7 = 0; _i7 < _map6.size; ++_i7) - { - String _key8; // required - TTypeQualifierValue _val9; // required - _key8 = iprot.readString(); - _val9 = new TTypeQualifierValue(); - _val9.read(iprot); - struct.qualifiers.put(_key8, _val9); - } - } - struct.setQualifiersIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java deleted file mode 100644 index 73dd45d3dd01a..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java +++ /dev/null @@ -1,448 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TUnionTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TUnionTypeEntry"); - - private static final org.apache.thrift.protocol.TField NAME_TO_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("nameToTypePtr", org.apache.thrift.protocol.TType.MAP, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TUnionTypeEntryStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TUnionTypeEntryTupleSchemeFactory()); - } - - private Map nameToTypePtr; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - NAME_TO_TYPE_PTR((short)1, "nameToTypePtr"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // NAME_TO_TYPE_PTR - return NAME_TO_TYPE_PTR; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.NAME_TO_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("nameToTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr")))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TUnionTypeEntry.class, metaDataMap); - } - - public TUnionTypeEntry() { - } - - public TUnionTypeEntry( - Map nameToTypePtr) - { - this(); - this.nameToTypePtr = nameToTypePtr; - } - - /** - * Performs a deep copy on other. - */ - public TUnionTypeEntry(TUnionTypeEntry other) { - if (other.isSetNameToTypePtr()) { - Map __this__nameToTypePtr = new HashMap(); - for (Map.Entry other_element : other.nameToTypePtr.entrySet()) { - - String other_element_key = other_element.getKey(); - Integer other_element_value = other_element.getValue(); - - String __this__nameToTypePtr_copy_key = other_element_key; - - Integer __this__nameToTypePtr_copy_value = other_element_value; - - __this__nameToTypePtr.put(__this__nameToTypePtr_copy_key, __this__nameToTypePtr_copy_value); - } - this.nameToTypePtr = __this__nameToTypePtr; - } - } - - public TUnionTypeEntry deepCopy() { - return new TUnionTypeEntry(this); - } - - @Override - public void clear() { - this.nameToTypePtr = null; - } - - public int getNameToTypePtrSize() { - return (this.nameToTypePtr == null) ? 0 : this.nameToTypePtr.size(); - } - - public void putToNameToTypePtr(String key, int val) { - if (this.nameToTypePtr == null) { - this.nameToTypePtr = new HashMap(); - } - this.nameToTypePtr.put(key, val); - } - - public Map getNameToTypePtr() { - return this.nameToTypePtr; - } - - public void setNameToTypePtr(Map nameToTypePtr) { - this.nameToTypePtr = nameToTypePtr; - } - - public void unsetNameToTypePtr() { - this.nameToTypePtr = null; - } - - /** Returns true if field nameToTypePtr is set (has been assigned a value) and false otherwise */ - public boolean isSetNameToTypePtr() { - return this.nameToTypePtr != null; - } - - public void setNameToTypePtrIsSet(boolean value) { - if (!value) { - this.nameToTypePtr = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case NAME_TO_TYPE_PTR: - if (value == null) { - unsetNameToTypePtr(); - } else { - setNameToTypePtr((Map)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case NAME_TO_TYPE_PTR: - return getNameToTypePtr(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case NAME_TO_TYPE_PTR: - return isSetNameToTypePtr(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TUnionTypeEntry) - return this.equals((TUnionTypeEntry)that); - return false; - } - - public boolean equals(TUnionTypeEntry that) { - if (that == null) - return false; - - boolean this_present_nameToTypePtr = true && this.isSetNameToTypePtr(); - boolean that_present_nameToTypePtr = true && that.isSetNameToTypePtr(); - if (this_present_nameToTypePtr || that_present_nameToTypePtr) { - if (!(this_present_nameToTypePtr && that_present_nameToTypePtr)) - return false; - if (!this.nameToTypePtr.equals(that.nameToTypePtr)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_nameToTypePtr = true && (isSetNameToTypePtr()); - builder.append(present_nameToTypePtr); - if (present_nameToTypePtr) - builder.append(nameToTypePtr); - - return builder.toHashCode(); - } - - public int compareTo(TUnionTypeEntry other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TUnionTypeEntry typedOther = (TUnionTypeEntry)other; - - lastComparison = Boolean.valueOf(isSetNameToTypePtr()).compareTo(typedOther.isSetNameToTypePtr()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNameToTypePtr()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nameToTypePtr, typedOther.nameToTypePtr); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TUnionTypeEntry("); - boolean first = true; - - sb.append("nameToTypePtr:"); - if (this.nameToTypePtr == null) { - sb.append("null"); - } else { - sb.append(this.nameToTypePtr); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetNameToTypePtr()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nameToTypePtr' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TUnionTypeEntryStandardSchemeFactory implements SchemeFactory { - public TUnionTypeEntryStandardScheme getScheme() { - return new TUnionTypeEntryStandardScheme(); - } - } - - private static class TUnionTypeEntryStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TUnionTypeEntry struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // NAME_TO_TYPE_PTR - if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { - { - org.apache.thrift.protocol.TMap _map20 = iprot.readMapBegin(); - struct.nameToTypePtr = new HashMap(2*_map20.size); - for (int _i21 = 0; _i21 < _map20.size; ++_i21) - { - String _key22; // required - int _val23; // required - _key22 = iprot.readString(); - _val23 = iprot.readI32(); - struct.nameToTypePtr.put(_key22, _val23); - } - iprot.readMapEnd(); - } - struct.setNameToTypePtrIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TUnionTypeEntry struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.nameToTypePtr != null) { - oprot.writeFieldBegin(NAME_TO_TYPE_PTR_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.nameToTypePtr.size())); - for (Map.Entry _iter24 : struct.nameToTypePtr.entrySet()) - { - oprot.writeString(_iter24.getKey()); - oprot.writeI32(_iter24.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TUnionTypeEntryTupleSchemeFactory implements SchemeFactory { - public TUnionTypeEntryTupleScheme getScheme() { - return new TUnionTypeEntryTupleScheme(); - } - } - - private static class TUnionTypeEntryTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TUnionTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.nameToTypePtr.size()); - for (Map.Entry _iter25 : struct.nameToTypePtr.entrySet()) - { - oprot.writeString(_iter25.getKey()); - oprot.writeI32(_iter25.getValue()); - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TUnionTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TMap _map26 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); - struct.nameToTypePtr = new HashMap(2*_map26.size); - for (int _i27 = 0; _i27 < _map26.size; ++_i27) - { - String _key28; // required - int _val29; // required - _key28 = iprot.readString(); - _val29 = iprot.readI32(); - struct.nameToTypePtr.put(_key28, _val29); - } - } - struct.setNameToTypePtrIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java deleted file mode 100644 index 3a111a2c8c2c6..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java +++ /dev/null @@ -1,385 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TUserDefinedTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TUserDefinedTypeEntry"); - - private static final org.apache.thrift.protocol.TField TYPE_CLASS_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("typeClassName", org.apache.thrift.protocol.TType.STRING, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TUserDefinedTypeEntryStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TUserDefinedTypeEntryTupleSchemeFactory()); - } - - private String typeClassName; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - TYPE_CLASS_NAME((short)1, "typeClassName"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // TYPE_CLASS_NAME - return TYPE_CLASS_NAME; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.TYPE_CLASS_NAME, new org.apache.thrift.meta_data.FieldMetaData("typeClassName", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TUserDefinedTypeEntry.class, metaDataMap); - } - - public TUserDefinedTypeEntry() { - } - - public TUserDefinedTypeEntry( - String typeClassName) - { - this(); - this.typeClassName = typeClassName; - } - - /** - * Performs a deep copy on other. - */ - public TUserDefinedTypeEntry(TUserDefinedTypeEntry other) { - if (other.isSetTypeClassName()) { - this.typeClassName = other.typeClassName; - } - } - - public TUserDefinedTypeEntry deepCopy() { - return new TUserDefinedTypeEntry(this); - } - - @Override - public void clear() { - this.typeClassName = null; - } - - public String getTypeClassName() { - return this.typeClassName; - } - - public void setTypeClassName(String typeClassName) { - this.typeClassName = typeClassName; - } - - public void unsetTypeClassName() { - this.typeClassName = null; - } - - /** Returns true if field typeClassName is set (has been assigned a value) and false otherwise */ - public boolean isSetTypeClassName() { - return this.typeClassName != null; - } - - public void setTypeClassNameIsSet(boolean value) { - if (!value) { - this.typeClassName = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case TYPE_CLASS_NAME: - if (value == null) { - unsetTypeClassName(); - } else { - setTypeClassName((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case TYPE_CLASS_NAME: - return getTypeClassName(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case TYPE_CLASS_NAME: - return isSetTypeClassName(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TUserDefinedTypeEntry) - return this.equals((TUserDefinedTypeEntry)that); - return false; - } - - public boolean equals(TUserDefinedTypeEntry that) { - if (that == null) - return false; - - boolean this_present_typeClassName = true && this.isSetTypeClassName(); - boolean that_present_typeClassName = true && that.isSetTypeClassName(); - if (this_present_typeClassName || that_present_typeClassName) { - if (!(this_present_typeClassName && that_present_typeClassName)) - return false; - if (!this.typeClassName.equals(that.typeClassName)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_typeClassName = true && (isSetTypeClassName()); - builder.append(present_typeClassName); - if (present_typeClassName) - builder.append(typeClassName); - - return builder.toHashCode(); - } - - public int compareTo(TUserDefinedTypeEntry other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TUserDefinedTypeEntry typedOther = (TUserDefinedTypeEntry)other; - - lastComparison = Boolean.valueOf(isSetTypeClassName()).compareTo(typedOther.isSetTypeClassName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetTypeClassName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.typeClassName, typedOther.typeClassName); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TUserDefinedTypeEntry("); - boolean first = true; - - sb.append("typeClassName:"); - if (this.typeClassName == null) { - sb.append("null"); - } else { - sb.append(this.typeClassName); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetTypeClassName()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'typeClassName' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TUserDefinedTypeEntryStandardSchemeFactory implements SchemeFactory { - public TUserDefinedTypeEntryStandardScheme getScheme() { - return new TUserDefinedTypeEntryStandardScheme(); - } - } - - private static class TUserDefinedTypeEntryStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TUserDefinedTypeEntry struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // TYPE_CLASS_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.typeClassName = iprot.readString(); - struct.setTypeClassNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TUserDefinedTypeEntry struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.typeClassName != null) { - oprot.writeFieldBegin(TYPE_CLASS_NAME_FIELD_DESC); - oprot.writeString(struct.typeClassName); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TUserDefinedTypeEntryTupleSchemeFactory implements SchemeFactory { - public TUserDefinedTypeEntryTupleScheme getScheme() { - return new TUserDefinedTypeEntryTupleScheme(); - } - } - - private static class TUserDefinedTypeEntryTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TUserDefinedTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeString(struct.typeClassName); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TUserDefinedTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.typeClassName = iprot.readString(); - struct.setTypeClassNameIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/AbstractService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/AbstractService.java deleted file mode 100644 index 7e557aeccf5b0..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/AbstractService.java +++ /dev/null @@ -1,184 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; - -/** - * AbstractService. - * - */ -public abstract class AbstractService implements Service { - - private static final Log LOG = LogFactory.getLog(AbstractService.class); - - /** - * Service state: initially {@link STATE#NOTINITED}. - */ - private Service.STATE state = STATE.NOTINITED; - - /** - * Service name. - */ - private final String name; - /** - * Service start time. Will be zero until the service is started. - */ - private long startTime; - - /** - * The configuration. Will be null until the service is initialized. - */ - private HiveConf hiveConf; - - /** - * List of state change listeners; it is final to ensure - * that it will never be null. - */ - private final List listeners = - new ArrayList(); - - /** - * Construct the service. - * - * @param name - * service name - */ - public AbstractService(String name) { - this.name = name; - } - - @Override - public synchronized Service.STATE getServiceState() { - return state; - } - - /** - * {@inheritDoc} - * - * @throws IllegalStateException - * if the current service state does not permit - * this action - */ - @Override - public synchronized void init(HiveConf hiveConf) { - ensureCurrentState(STATE.NOTINITED); - this.hiveConf = hiveConf; - changeState(STATE.INITED); - LOG.info("Service:" + getName() + " is inited."); - } - - /** - * {@inheritDoc} - * - * @throws IllegalStateException - * if the current service state does not permit - * this action - */ - @Override - public synchronized void start() { - startTime = System.currentTimeMillis(); - ensureCurrentState(STATE.INITED); - changeState(STATE.STARTED); - LOG.info("Service:" + getName() + " is started."); - } - - /** - * {@inheritDoc} - * - * @throws IllegalStateException - * if the current service state does not permit - * this action - */ - @Override - public synchronized void stop() { - if (state == STATE.STOPPED || - state == STATE.INITED || - state == STATE.NOTINITED) { - // already stopped, or else it was never - // started (eg another service failing canceled startup) - return; - } - ensureCurrentState(STATE.STARTED); - changeState(STATE.STOPPED); - LOG.info("Service:" + getName() + " is stopped."); - } - - @Override - public synchronized void register(ServiceStateChangeListener l) { - listeners.add(l); - } - - @Override - public synchronized void unregister(ServiceStateChangeListener l) { - listeners.remove(l); - } - - @Override - public String getName() { - return name; - } - - @Override - public synchronized HiveConf getHiveConf() { - return hiveConf; - } - - @Override - public long getStartTime() { - return startTime; - } - - /** - * Verify that a service is in a given state. - * - * @param currentState - * the desired state - * @throws IllegalStateException - * if the service state is different from - * the desired state - */ - private void ensureCurrentState(Service.STATE currentState) { - ServiceOperations.ensureCurrentState(state, currentState); - } - - /** - * Change to a new state and notify all listeners. - * This is a private method that is only invoked from synchronized methods, - * which avoid having to clone the listener list. It does imply that - * the state change listener methods should be short lived, as they - * will delay the state transition. - * - * @param newState - * new service state - */ - private void changeState(Service.STATE newState) { - state = newState; - // notify listeners - for (ServiceStateChangeListener l : listeners) { - l.stateChanged(this); - } - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CompositeService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CompositeService.java deleted file mode 100644 index 897911872b80f..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CompositeService.java +++ /dev/null @@ -1,133 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; - -/** - * CompositeService. - * - */ -public class CompositeService extends AbstractService { - - private static final Log LOG = LogFactory.getLog(CompositeService.class); - - private final List serviceList = new ArrayList(); - - public CompositeService(String name) { - super(name); - } - - public Collection getServices() { - return Collections.unmodifiableList(serviceList); - } - - protected synchronized void addService(Service service) { - serviceList.add(service); - } - - protected synchronized boolean removeService(Service service) { - return serviceList.remove(service); - } - - @Override - public synchronized void init(HiveConf hiveConf) { - for (Service service : serviceList) { - service.init(hiveConf); - } - super.init(hiveConf); - } - - @Override - public synchronized void start() { - int i = 0; - try { - for (int n = serviceList.size(); i < n; i++) { - Service service = serviceList.get(i); - service.start(); - } - super.start(); - } catch (Throwable e) { - LOG.error("Error starting services " + getName(), e); - // Note that the state of the failed service is still INITED and not - // STARTED. Even though the last service is not started completely, still - // call stop() on all services including failed service to make sure cleanup - // happens. - stop(i); - throw new ServiceException("Failed to Start " + getName(), e); - } - - } - - @Override - public synchronized void stop() { - if (this.getServiceState() == STATE.STOPPED) { - // The base composite-service is already stopped, don't do anything again. - return; - } - if (serviceList.size() > 0) { - stop(serviceList.size() - 1); - } - super.stop(); - } - - private synchronized void stop(int numOfServicesStarted) { - // stop in reserve order of start - for (int i = numOfServicesStarted; i >= 0; i--) { - Service service = serviceList.get(i); - try { - service.stop(); - } catch (Throwable t) { - LOG.info("Error stopping " + service.getName(), t); - } - } - } - - /** - * JVM Shutdown hook for CompositeService which will stop the given - * CompositeService gracefully in case of JVM shutdown. - */ - public static class CompositeServiceShutdownHook implements Runnable { - - private final CompositeService compositeService; - - public CompositeServiceShutdownHook(CompositeService compositeService) { - this.compositeService = compositeService; - } - - @Override - public void run() { - try { - // Stop the Composite Service - compositeService.stop(); - } catch (Throwable t) { - LOG.info("Error stopping " + compositeService.getName(), t); - } - } - } - - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CookieSigner.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CookieSigner.java deleted file mode 100644 index f2a80c9d5ffbc..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CookieSigner.java +++ /dev/null @@ -1,108 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service; - -import org.apache.commons.codec.binary.Base64; -import org.apache.commons.logging.LogFactory; -import org.apache.commons.logging.Log; - -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; - -/** - * The cookie signer generates a signature based on SHA digest - * and appends it to the cookie value generated at the - * server side. It uses SHA digest algorithm to sign and verify signatures. - */ -public class CookieSigner { - private static final String SIGNATURE = "&s="; - private static final String SHA_STRING = "SHA"; - private byte[] secretBytes; - private static final Log LOG = LogFactory.getLog(CookieSigner.class); - - /** - * Constructor - * @param secret Secret Bytes - */ - public CookieSigner(byte[] secret) { - if (secret == null) { - throw new IllegalArgumentException(" NULL Secret Bytes"); - } - this.secretBytes = secret.clone(); - } - - /** - * Sign the cookie given the string token as input. - * @param str Input token - * @return Signed token that can be used to create a cookie - */ - public String signCookie(String str) { - if (str == null || str.isEmpty()) { - throw new IllegalArgumentException("NULL or empty string to sign"); - } - String signature = getSignature(str); - - if (LOG.isDebugEnabled()) { - LOG.debug("Signature generated for " + str + " is " + signature); - } - return str + SIGNATURE + signature; - } - - /** - * Verify a signed string and extracts the original string. - * @param signedStr The already signed string - * @return Raw Value of the string without the signature - */ - public String verifyAndExtract(String signedStr) { - int index = signedStr.lastIndexOf(SIGNATURE); - if (index == -1) { - throw new IllegalArgumentException("Invalid input sign: " + signedStr); - } - String originalSignature = signedStr.substring(index + SIGNATURE.length()); - String rawValue = signedStr.substring(0, index); - String currentSignature = getSignature(rawValue); - - if (LOG.isDebugEnabled()) { - LOG.debug("Signature generated for " + rawValue + " inside verify is " + currentSignature); - } - if (!MessageDigest.isEqual(originalSignature.getBytes(), currentSignature.getBytes())) { - throw new IllegalArgumentException("Invalid sign, original = " + originalSignature + - " current = " + currentSignature); - } - return rawValue; - } - - /** - * Get the signature of the input string based on SHA digest algorithm. - * @param str Input token - * @return Signed String - */ - private String getSignature(String str) { - try { - MessageDigest md = MessageDigest.getInstance(SHA_STRING); - md.update(str.getBytes()); - md.update(secretBytes); - byte[] digest = md.digest(); - return new Base64(0).encodeToString(digest); - } catch (NoSuchAlgorithmException ex) { - throw new RuntimeException("Invalid SHA digest String: " + SHA_STRING + - " " + ex.getMessage(), ex); - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceOperations.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceOperations.java deleted file mode 100644 index f16863c1b41aa..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceOperations.java +++ /dev/null @@ -1,141 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; - -/** - * ServiceOperations. - * - */ -public final class ServiceOperations { - private static final Log LOG = LogFactory.getLog(ServiceOperations.class); - - private ServiceOperations() { - } - - /** - * Verify that a service is in a given state. - * @param state the actual state a service is in - * @param expectedState the desired state - * @throws IllegalStateException if the service state is different from - * the desired state - */ - public static void ensureCurrentState(Service.STATE state, - Service.STATE expectedState) { - if (state != expectedState) { - throw new IllegalStateException("For this operation, the " + - "current service state must be " - + expectedState - + " instead of " + state); - } - } - - /** - * Initialize a service. - * - * The service state is checked before the operation begins. - * This process is not thread safe. - * @param service a service that must be in the state - * {@link Service.STATE#NOTINITED} - * @param configuration the configuration to initialize the service with - * @throws RuntimeException on a state change failure - * @throws IllegalStateException if the service is in the wrong state - */ - - public static void init(Service service, HiveConf configuration) { - Service.STATE state = service.getServiceState(); - ensureCurrentState(state, Service.STATE.NOTINITED); - service.init(configuration); - } - - /** - * Start a service. - * - * The service state is checked before the operation begins. - * This process is not thread safe. - * @param service a service that must be in the state - * {@link Service.STATE#INITED} - * @throws RuntimeException on a state change failure - * @throws IllegalStateException if the service is in the wrong state - */ - - public static void start(Service service) { - Service.STATE state = service.getServiceState(); - ensureCurrentState(state, Service.STATE.INITED); - service.start(); - } - - /** - * Initialize then start a service. - * - * The service state is checked before the operation begins. - * This process is not thread safe. - * @param service a service that must be in the state - * {@link Service.STATE#NOTINITED} - * @param configuration the configuration to initialize the service with - * @throws RuntimeException on a state change failure - * @throws IllegalStateException if the service is in the wrong state - */ - public static void deploy(Service service, HiveConf configuration) { - init(service, configuration); - start(service); - } - - /** - * Stop a service. - * - * Do nothing if the service is null or not in a state in which it can be/needs to be stopped. - * - * The service state is checked before the operation begins. - * This process is not thread safe. - * @param service a service or null - */ - public static void stop(Service service) { - if (service != null) { - Service.STATE state = service.getServiceState(); - if (state == Service.STATE.STARTED) { - service.stop(); - } - } - } - - /** - * Stop a service; if it is null do nothing. Exceptions are caught and - * logged at warn level. (but not Throwables). This operation is intended to - * be used in cleanup operations - * - * @param service a service; may be null - * @return any exception that was caught; null if none was. - */ - public static Exception stopQuietly(Service service) { - try { - stop(service); - } catch (Exception e) { - LOG.warn("When stopping the service " + service.getName() - + " : " + e, - e); - return e; - } - return null; - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceUtils.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceUtils.java deleted file mode 100644 index edb5eff9615bf..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceUtils.java +++ /dev/null @@ -1,44 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.service; - -public class ServiceUtils { - - /* - * Get the index separating the user name from domain name (the user's name up - * to the first '/' or '@'). - * - * @param userName full user name. - * @return index of domain match or -1 if not found - */ - public static int indexOfDomainMatch(String userName) { - if (userName == null) { - return -1; - } - - int idx = userName.indexOf('/'); - int idx2 = userName.indexOf('@'); - int endIdx = Math.min(idx, idx2); // Use the earlier match. - // Unless at least one of '/' or '@' was not found, in - // which case, user the latter match. - if (endIdx == -1) { - endIdx = Math.max(idx, idx2); - } - return endIdx; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java deleted file mode 100644 index 10000f12ab329..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java +++ /dev/null @@ -1,419 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.service.auth; - -import java.io.IOException; -import java.lang.reflect.Field; -import java.lang.reflect.Method; -import java.net.InetSocketAddress; -import java.net.UnknownHostException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Objects; - -import javax.net.ssl.SSLServerSocket; -import javax.security.auth.login.LoginException; -import javax.security.sasl.Sasl; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.metastore.HiveMetaStore; -import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.shims.HadoopShims.KerberosNameShim; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.hive.thrift.DBTokenStore; -import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge; -import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge.Server.ServerMode; -import org.apache.hadoop.security.SecurityUtil; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.authorize.ProxyUsers; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.thrift.ThriftCLIService; -import org.apache.thrift.TProcessorFactory; -import org.apache.thrift.transport.TSSLTransportFactory; -import org.apache.thrift.transport.TServerSocket; -import org.apache.thrift.transport.TSocket; -import org.apache.thrift.transport.TTransport; -import org.apache.thrift.transport.TTransportException; -import org.apache.thrift.transport.TTransportFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This class helps in some aspects of authentication. It creates the proper Thrift classes for the - * given configuration as well as helps with authenticating requests. - */ -public class HiveAuthFactory { - private static final Logger LOG = LoggerFactory.getLogger(HiveAuthFactory.class); - - - public enum AuthTypes { - NOSASL("NOSASL"), - NONE("NONE"), - LDAP("LDAP"), - KERBEROS("KERBEROS"), - CUSTOM("CUSTOM"), - PAM("PAM"); - - private final String authType; - - AuthTypes(String authType) { - this.authType = authType; - } - - public String getAuthName() { - return authType; - } - - } - - private HadoopThriftAuthBridge.Server saslServer; - private String authTypeStr; - private final String transportMode; - private final HiveConf conf; - - public static final String HS2_PROXY_USER = "hive.server2.proxy.user"; - public static final String HS2_CLIENT_TOKEN = "hiveserver2ClientToken"; - - private static Field keytabFile = null; - private static Method getKeytab = null; - static { - Class clz = UserGroupInformation.class; - try { - keytabFile = clz.getDeclaredField("keytabFile"); - keytabFile.setAccessible(true); - } catch (NoSuchFieldException nfe) { - LOG.debug("Cannot find private field \"keytabFile\" in class: " + - UserGroupInformation.class.getCanonicalName(), nfe); - keytabFile = null; - } - - try { - getKeytab = clz.getDeclaredMethod("getKeytab"); - getKeytab.setAccessible(true); - } catch(NoSuchMethodException nme) { - LOG.debug("Cannot find private method \"getKeytab\" in class:" + - UserGroupInformation.class.getCanonicalName(), nme); - getKeytab = null; - } - } - - public HiveAuthFactory(HiveConf conf) throws TTransportException, IOException { - this.conf = conf; - transportMode = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE); - authTypeStr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_AUTHENTICATION); - - // In http mode we use NOSASL as the default auth type - if ("http".equalsIgnoreCase(transportMode)) { - if (authTypeStr == null) { - authTypeStr = AuthTypes.NOSASL.getAuthName(); - } - } else { - if (authTypeStr == null) { - authTypeStr = AuthTypes.NONE.getAuthName(); - } - if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) { - String principal = conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL); - String keytab = conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB); - if (needUgiLogin(UserGroupInformation.getCurrentUser(), - SecurityUtil.getServerPrincipal(principal, "0.0.0.0"), keytab)) { - saslServer = ShimLoader.getHadoopThriftAuthBridge().createServer(principal, keytab); - } else { - // Using the default constructor to avoid unnecessary UGI login. - saslServer = new HadoopThriftAuthBridge.Server(); - } - - // start delegation token manager - try { - // rawStore is only necessary for DBTokenStore - Object rawStore = null; - String tokenStoreClass = conf.getVar(HiveConf.ConfVars.METASTORE_CLUSTER_DELEGATION_TOKEN_STORE_CLS); - - if (tokenStoreClass.equals(DBTokenStore.class.getName())) { - HMSHandler baseHandler = new HiveMetaStore.HMSHandler( - "new db based metaserver", conf, true); - rawStore = baseHandler.getMS(); - } - - saslServer.startDelegationTokenSecretManager(conf, rawStore, ServerMode.HIVESERVER2); - } - catch (MetaException|IOException e) { - throw new TTransportException("Failed to start token manager", e); - } - } - } - } - - public Map getSaslProperties() { - Map saslProps = new HashMap(); - SaslQOP saslQOP = SaslQOP.fromString(conf.getVar(ConfVars.HIVE_SERVER2_THRIFT_SASL_QOP)); - saslProps.put(Sasl.QOP, saslQOP.toString()); - saslProps.put(Sasl.SERVER_AUTH, "true"); - return saslProps; - } - - public TTransportFactory getAuthTransFactory() throws LoginException { - TTransportFactory transportFactory; - if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) { - try { - transportFactory = saslServer.createTransportFactory(getSaslProperties()); - } catch (TTransportException e) { - throw new LoginException(e.getMessage()); - } - } else if (authTypeStr.equalsIgnoreCase(AuthTypes.NONE.getAuthName())) { - transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); - } else if (authTypeStr.equalsIgnoreCase(AuthTypes.LDAP.getAuthName())) { - transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); - } else if (authTypeStr.equalsIgnoreCase(AuthTypes.PAM.getAuthName())) { - transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); - } else if (authTypeStr.equalsIgnoreCase(AuthTypes.NOSASL.getAuthName())) { - transportFactory = new TTransportFactory(); - } else if (authTypeStr.equalsIgnoreCase(AuthTypes.CUSTOM.getAuthName())) { - transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); - } else { - throw new LoginException("Unsupported authentication type " + authTypeStr); - } - return transportFactory; - } - - /** - * Returns the thrift processor factory for HiveServer2 running in binary mode - * @param service - * @return - * @throws LoginException - */ - public TProcessorFactory getAuthProcFactory(ThriftCLIService service) throws LoginException { - if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) { - return KerberosSaslHelper.getKerberosProcessorFactory(saslServer, service); - } else { - return PlainSaslHelper.getPlainProcessorFactory(service); - } - } - - public String getRemoteUser() { - return saslServer == null ? null : saslServer.getRemoteUser(); - } - - public String getIpAddress() { - if (saslServer == null || saslServer.getRemoteAddress() == null) { - return null; - } else { - return saslServer.getRemoteAddress().getHostAddress(); - } - } - - // Perform kerberos login using the hadoop shim API if the configuration is available - public static void loginFromKeytab(HiveConf hiveConf) throws IOException { - String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL); - String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB); - if (principal.isEmpty() || keyTabFile.isEmpty()) { - throw new IOException("HiveServer2 Kerberos principal or keytab is not correctly configured"); - } else { - UserGroupInformation.loginUserFromKeytab(SecurityUtil.getServerPrincipal(principal, "0.0.0.0"), keyTabFile); - } - } - - // Perform SPNEGO login using the hadoop shim API if the configuration is available - public static UserGroupInformation loginFromSpnegoKeytabAndReturnUGI(HiveConf hiveConf) - throws IOException { - String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_PRINCIPAL); - String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_KEYTAB); - if (principal.isEmpty() || keyTabFile.isEmpty()) { - throw new IOException("HiveServer2 SPNEGO principal or keytab is not correctly configured"); - } else { - return UserGroupInformation.loginUserFromKeytabAndReturnUGI(SecurityUtil.getServerPrincipal(principal, "0.0.0.0"), keyTabFile); - } - } - - public static TTransport getSocketTransport(String host, int port, int loginTimeout) { - return new TSocket(host, port, loginTimeout); - } - - public static TTransport getSSLSocket(String host, int port, int loginTimeout) - throws TTransportException { - return TSSLTransportFactory.getClientSocket(host, port, loginTimeout); - } - - public static TTransport getSSLSocket(String host, int port, int loginTimeout, - String trustStorePath, String trustStorePassWord) throws TTransportException { - TSSLTransportFactory.TSSLTransportParameters params = - new TSSLTransportFactory.TSSLTransportParameters(); - params.setTrustStore(trustStorePath, trustStorePassWord); - params.requireClientAuth(true); - return TSSLTransportFactory.getClientSocket(host, port, loginTimeout, params); - } - - public static TServerSocket getServerSocket(String hiveHost, int portNum) - throws TTransportException { - InetSocketAddress serverAddress; - if (hiveHost == null || hiveHost.isEmpty()) { - // Wildcard bind - serverAddress = new InetSocketAddress(portNum); - } else { - serverAddress = new InetSocketAddress(hiveHost, portNum); - } - return new TServerSocket(serverAddress); - } - - public static TServerSocket getServerSSLSocket(String hiveHost, int portNum, String keyStorePath, - String keyStorePassWord, List sslVersionBlacklist) throws TTransportException, - UnknownHostException { - TSSLTransportFactory.TSSLTransportParameters params = - new TSSLTransportFactory.TSSLTransportParameters(); - params.setKeyStore(keyStorePath, keyStorePassWord); - InetSocketAddress serverAddress; - if (hiveHost == null || hiveHost.isEmpty()) { - // Wildcard bind - serverAddress = new InetSocketAddress(portNum); - } else { - serverAddress = new InetSocketAddress(hiveHost, portNum); - } - TServerSocket thriftServerSocket = - TSSLTransportFactory.getServerSocket(portNum, 0, serverAddress.getAddress(), params); - if (thriftServerSocket.getServerSocket() instanceof SSLServerSocket) { - List sslVersionBlacklistLocal = new ArrayList(); - for (String sslVersion : sslVersionBlacklist) { - sslVersionBlacklistLocal.add(sslVersion.trim().toLowerCase(Locale.ROOT)); - } - SSLServerSocket sslServerSocket = (SSLServerSocket) thriftServerSocket.getServerSocket(); - List enabledProtocols = new ArrayList(); - for (String protocol : sslServerSocket.getEnabledProtocols()) { - if (sslVersionBlacklistLocal.contains(protocol.toLowerCase(Locale.ROOT))) { - LOG.debug("Disabling SSL Protocol: " + protocol); - } else { - enabledProtocols.add(protocol); - } - } - sslServerSocket.setEnabledProtocols(enabledProtocols.toArray(new String[0])); - LOG.info("SSL Server Socket Enabled Protocols: " - + Arrays.toString(sslServerSocket.getEnabledProtocols())); - } - return thriftServerSocket; - } - - // retrieve delegation token for the given user - public String getDelegationToken(String owner, String renewer) throws HiveSQLException { - if (saslServer == null) { - throw new HiveSQLException( - "Delegation token only supported over kerberos authentication", "08S01"); - } - - try { - String tokenStr = saslServer.getDelegationTokenWithService(owner, renewer, HS2_CLIENT_TOKEN); - if (tokenStr == null || tokenStr.isEmpty()) { - throw new HiveSQLException( - "Received empty retrieving delegation token for user " + owner, "08S01"); - } - return tokenStr; - } catch (IOException e) { - throw new HiveSQLException( - "Error retrieving delegation token for user " + owner, "08S01", e); - } catch (InterruptedException e) { - throw new HiveSQLException("delegation token retrieval interrupted", "08S01", e); - } - } - - // cancel given delegation token - public void cancelDelegationToken(String delegationToken) throws HiveSQLException { - if (saslServer == null) { - throw new HiveSQLException( - "Delegation token only supported over kerberos authentication", "08S01"); - } - try { - saslServer.cancelDelegationToken(delegationToken); - } catch (IOException e) { - throw new HiveSQLException( - "Error canceling delegation token " + delegationToken, "08S01", e); - } - } - - public void renewDelegationToken(String delegationToken) throws HiveSQLException { - if (saslServer == null) { - throw new HiveSQLException( - "Delegation token only supported over kerberos authentication", "08S01"); - } - try { - saslServer.renewDelegationToken(delegationToken); - } catch (IOException e) { - throw new HiveSQLException( - "Error renewing delegation token " + delegationToken, "08S01", e); - } - } - - public String getUserFromToken(String delegationToken) throws HiveSQLException { - if (saslServer == null) { - throw new HiveSQLException( - "Delegation token only supported over kerberos authentication", "08S01"); - } - try { - return saslServer.getUserFromToken(delegationToken); - } catch (IOException e) { - throw new HiveSQLException( - "Error extracting user from delegation token " + delegationToken, "08S01", e); - } - } - - public static void verifyProxyAccess(String realUser, String proxyUser, String ipAddress, - HiveConf hiveConf) throws HiveSQLException { - try { - UserGroupInformation sessionUgi; - if (UserGroupInformation.isSecurityEnabled()) { - KerberosNameShim kerbName = ShimLoader.getHadoopShims().getKerberosNameShim(realUser); - sessionUgi = UserGroupInformation.createProxyUser( - kerbName.getServiceName(), UserGroupInformation.getLoginUser()); - } else { - sessionUgi = UserGroupInformation.createRemoteUser(realUser); - } - if (!proxyUser.equalsIgnoreCase(realUser)) { - ProxyUsers.refreshSuperUserGroupsConfiguration(hiveConf); - ProxyUsers.authorize(UserGroupInformation.createProxyUser(proxyUser, sessionUgi), - ipAddress, hiveConf); - } - } catch (IOException e) { - throw new HiveSQLException( - "Failed to validate proxy privilege of " + realUser + " for " + proxyUser, "08S01", e); - } - } - - public static boolean needUgiLogin(UserGroupInformation ugi, String principal, String keytab) { - return null == ugi || !ugi.hasKerberosCredentials() || !ugi.getUserName().equals(principal) || - !Objects.equals(keytab, getKeytabFromUgi()); - } - - private static String getKeytabFromUgi() { - synchronized (UserGroupInformation.class) { - try { - if (keytabFile != null) { - return (String) keytabFile.get(null); - } else if (getKeytab != null) { - return (String) getKeytab.invoke(UserGroupInformation.getCurrentUser()); - } else { - return null; - } - } catch (Exception e) { - LOG.debug("Fail to get keytabFile path via reflection", e); - return null; - } - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java deleted file mode 100644 index f7375ee707830..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java +++ /dev/null @@ -1,189 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.auth; - -import java.security.AccessControlContext; -import java.security.AccessController; -import java.security.PrivilegedExceptionAction; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Random; -import java.util.Set; -import java.util.StringTokenizer; - -import javax.security.auth.Subject; - -import org.apache.commons.codec.binary.Base64; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.http.protocol.BasicHttpContext; -import org.apache.http.protocol.HttpContext; -import org.ietf.jgss.GSSContext; -import org.ietf.jgss.GSSManager; -import org.ietf.jgss.GSSName; -import org.ietf.jgss.Oid; - -/** - * Utility functions for HTTP mode authentication. - */ -public final class HttpAuthUtils { - public static final String WWW_AUTHENTICATE = "WWW-Authenticate"; - public static final String AUTHORIZATION = "Authorization"; - public static final String BASIC = "Basic"; - public static final String NEGOTIATE = "Negotiate"; - private static final Log LOG = LogFactory.getLog(HttpAuthUtils.class); - private static final String COOKIE_ATTR_SEPARATOR = "&"; - private static final String COOKIE_CLIENT_USER_NAME = "cu"; - private static final String COOKIE_CLIENT_RAND_NUMBER = "rn"; - private static final String COOKIE_KEY_VALUE_SEPARATOR = "="; - private static final Set COOKIE_ATTRIBUTES = - new HashSet(Arrays.asList(COOKIE_CLIENT_USER_NAME, COOKIE_CLIENT_RAND_NUMBER)); - - /** - * @return Stringified Base64 encoded kerberosAuthHeader on success - * @throws Exception - */ - public static String getKerberosServiceTicket(String principal, String host, - String serverHttpUrl, boolean assumeSubject) throws Exception { - String serverPrincipal = - ShimLoader.getHadoopThriftAuthBridge().getServerPrincipal(principal, host); - if (assumeSubject) { - // With this option, we're assuming that the external application, - // using the JDBC driver has done a JAAS kerberos login already - AccessControlContext context = AccessController.getContext(); - Subject subject = Subject.getSubject(context); - if (subject == null) { - throw new Exception("The Subject is not set"); - } - return Subject.doAs(subject, new HttpKerberosClientAction(serverPrincipal, serverHttpUrl)); - } else { - // JAAS login from ticket cache to setup the client UserGroupInformation - UserGroupInformation clientUGI = - ShimLoader.getHadoopThriftAuthBridge().getCurrentUGIWithConf("kerberos"); - return clientUGI.doAs(new HttpKerberosClientAction(serverPrincipal, serverHttpUrl)); - } - } - - /** - * Creates and returns a HS2 cookie token. - * @param clientUserName Client User name. - * @return An unsigned cookie token generated from input parameters. - * The final cookie generated is of the following format : - * {@code cu=&rn=&s=} - */ - public static String createCookieToken(String clientUserName) { - StringBuffer sb = new StringBuffer(); - sb.append(COOKIE_CLIENT_USER_NAME).append(COOKIE_KEY_VALUE_SEPARATOR).append(clientUserName) - .append(COOKIE_ATTR_SEPARATOR); - sb.append(COOKIE_CLIENT_RAND_NUMBER).append(COOKIE_KEY_VALUE_SEPARATOR) - .append((new Random(System.currentTimeMillis())).nextLong()); - return sb.toString(); - } - - /** - * Parses a cookie token to retrieve client user name. - * @param tokenStr Token String. - * @return A valid user name if input is of valid format, else returns null. - */ - public static String getUserNameFromCookieToken(String tokenStr) { - Map map = splitCookieToken(tokenStr); - - if (!map.keySet().equals(COOKIE_ATTRIBUTES)) { - LOG.error("Invalid token with missing attributes " + tokenStr); - return null; - } - return map.get(COOKIE_CLIENT_USER_NAME); - } - - /** - * Splits the cookie token into attributes pairs. - * @param str input token. - * @return a map with the attribute pairs of the token if the input is valid. - * Else, returns null. - */ - private static Map splitCookieToken(String tokenStr) { - Map map = new HashMap(); - StringTokenizer st = new StringTokenizer(tokenStr, COOKIE_ATTR_SEPARATOR); - - while (st.hasMoreTokens()) { - String part = st.nextToken(); - int separator = part.indexOf(COOKIE_KEY_VALUE_SEPARATOR); - if (separator == -1) { - LOG.error("Invalid token string " + tokenStr); - return null; - } - String key = part.substring(0, separator); - String value = part.substring(separator + 1); - map.put(key, value); - } - return map; - } - - - private HttpAuthUtils() { - throw new UnsupportedOperationException("Can't initialize class"); - } - - /** - * We'll create an instance of this class within a doAs block so that the client's TGT credentials - * can be read from the Subject - */ - public static class HttpKerberosClientAction implements PrivilegedExceptionAction { - public static final String HTTP_RESPONSE = "HTTP_RESPONSE"; - public static final String SERVER_HTTP_URL = "SERVER_HTTP_URL"; - private final String serverPrincipal; - private final String serverHttpUrl; - private final Base64 base64codec; - private final HttpContext httpContext; - - public HttpKerberosClientAction(String serverPrincipal, String serverHttpUrl) { - this.serverPrincipal = serverPrincipal; - this.serverHttpUrl = serverHttpUrl; - base64codec = new Base64(0); - httpContext = new BasicHttpContext(); - httpContext.setAttribute(SERVER_HTTP_URL, serverHttpUrl); - } - - @Override - public String run() throws Exception { - // This Oid for Kerberos GSS-API mechanism. - Oid mechOid = new Oid("1.2.840.113554.1.2.2"); - // Oid for kerberos principal name - Oid krb5PrincipalOid = new Oid("1.2.840.113554.1.2.2.1"); - GSSManager manager = GSSManager.getInstance(); - // GSS name for server - GSSName serverName = manager.createName(serverPrincipal, krb5PrincipalOid); - // Create a GSSContext for authentication with the service. - // We're passing client credentials as null since we want them to be read from the Subject. - GSSContext gssContext = - manager.createContext(serverName, mechOid, null, GSSContext.DEFAULT_LIFETIME); - gssContext.requestMutualAuth(false); - // Establish context - byte[] inToken = new byte[0]; - byte[] outToken = gssContext.initSecContext(inToken, 0, inToken.length); - gssContext.dispose(); - // Base64 encoded and stringified token for server - return new String(base64codec.encode(outToken)); - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java deleted file mode 100644 index 52eb752f1e026..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java +++ /dev/null @@ -1,111 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.service.auth; - -import java.io.IOException; -import java.util.Map; -import javax.security.sasl.SaslException; - -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge; -import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge.Server; -import org.apache.hive.service.cli.thrift.TCLIService; -import org.apache.hive.service.cli.thrift.TCLIService.Iface; -import org.apache.hive.service.cli.thrift.ThriftCLIService; -import org.apache.thrift.TProcessor; -import org.apache.thrift.TProcessorFactory; -import org.apache.thrift.transport.TSaslClientTransport; -import org.apache.thrift.transport.TTransport; - -public final class KerberosSaslHelper { - - public static TProcessorFactory getKerberosProcessorFactory(Server saslServer, - ThriftCLIService service) { - return new CLIServiceProcessorFactory(saslServer, service); - } - - public static TTransport getKerberosTransport(String principal, String host, - TTransport underlyingTransport, Map saslProps, boolean assumeSubject) - throws SaslException { - try { - String[] names = principal.split("[/@]"); - if (names.length != 3) { - throw new IllegalArgumentException("Kerberos principal should have 3 parts: " + principal); - } - - if (assumeSubject) { - return createSubjectAssumedTransport(principal, underlyingTransport, saslProps); - } else { - HadoopThriftAuthBridge.Client authBridge = - ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos"); - return authBridge.createClientTransport(principal, host, "KERBEROS", null, - underlyingTransport, saslProps); - } - } catch (IOException e) { - throw new SaslException("Failed to open client transport", e); - } - } - - public static TTransport createSubjectAssumedTransport(String principal, - TTransport underlyingTransport, Map saslProps) throws IOException { - String[] names = principal.split("[/@]"); - try { - TTransport saslTransport = - new TSaslClientTransport("GSSAPI", null, names[0], names[1], saslProps, null, - underlyingTransport); - return new TSubjectAssumingTransport(saslTransport); - } catch (SaslException se) { - throw new IOException("Could not instantiate SASL transport", se); - } - } - - public static TTransport getTokenTransport(String tokenStr, String host, - TTransport underlyingTransport, Map saslProps) throws SaslException { - HadoopThriftAuthBridge.Client authBridge = - ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos"); - - try { - return authBridge.createClientTransport(null, host, "DIGEST", tokenStr, underlyingTransport, - saslProps); - } catch (IOException e) { - throw new SaslException("Failed to open client transport", e); - } - } - - private KerberosSaslHelper() { - throw new UnsupportedOperationException("Can't initialize class"); - } - - private static class CLIServiceProcessorFactory extends TProcessorFactory { - - private final ThriftCLIService service; - private final Server saslServer; - - CLIServiceProcessorFactory(Server saslServer, ThriftCLIService service) { - super(null); - this.service = service; - this.saslServer = saslServer; - } - - @Override - public TProcessor getProcessor(TTransport trans) { - TProcessor sqlProcessor = new TCLIService.Processor(service); - return saslServer.wrapNonAssumingProcessor(sqlProcessor); - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java deleted file mode 100644 index afc144199f1e8..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java +++ /dev/null @@ -1,154 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.service.auth; - -import java.io.IOException; -import java.security.Security; -import java.util.HashMap; -import javax.security.auth.callback.Callback; -import javax.security.auth.callback.CallbackHandler; -import javax.security.auth.callback.NameCallback; -import javax.security.auth.callback.PasswordCallback; -import javax.security.auth.callback.UnsupportedCallbackException; -import javax.security.auth.login.LoginException; -import javax.security.sasl.AuthenticationException; -import javax.security.sasl.AuthorizeCallback; -import javax.security.sasl.SaslException; - -import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods; -import org.apache.hive.service.auth.PlainSaslServer.SaslPlainProvider; -import org.apache.hive.service.cli.thrift.TCLIService.Iface; -import org.apache.hive.service.cli.thrift.ThriftCLIService; -import org.apache.thrift.TProcessor; -import org.apache.thrift.TProcessorFactory; -import org.apache.thrift.transport.TSaslClientTransport; -import org.apache.thrift.transport.TSaslServerTransport; -import org.apache.thrift.transport.TTransport; -import org.apache.thrift.transport.TTransportFactory; - -public final class PlainSaslHelper { - - public static TProcessorFactory getPlainProcessorFactory(ThriftCLIService service) { - return new SQLPlainProcessorFactory(service); - } - - // Register Plain SASL server provider - static { - Security.addProvider(new SaslPlainProvider()); - } - - public static TTransportFactory getPlainTransportFactory(String authTypeStr) - throws LoginException { - TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory(); - try { - saslFactory.addServerDefinition("PLAIN", authTypeStr, null, new HashMap(), - new PlainServerCallbackHandler(authTypeStr)); - } catch (AuthenticationException e) { - throw new LoginException("Error setting callback handler" + e); - } - return saslFactory; - } - - public static TTransport getPlainTransport(String username, String password, - TTransport underlyingTransport) throws SaslException { - return new TSaslClientTransport("PLAIN", null, null, null, new HashMap(), - new PlainCallbackHandler(username, password), underlyingTransport); - } - - private PlainSaslHelper() { - throw new UnsupportedOperationException("Can't initialize class"); - } - - private static final class PlainServerCallbackHandler implements CallbackHandler { - - private final AuthMethods authMethod; - - PlainServerCallbackHandler(String authMethodStr) throws AuthenticationException { - authMethod = AuthMethods.getValidAuthMethod(authMethodStr); - } - - @Override - public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { - String username = null; - String password = null; - AuthorizeCallback ac = null; - - for (Callback callback : callbacks) { - if (callback instanceof NameCallback) { - NameCallback nc = (NameCallback) callback; - username = nc.getName(); - } else if (callback instanceof PasswordCallback) { - PasswordCallback pc = (PasswordCallback) callback; - password = new String(pc.getPassword()); - } else if (callback instanceof AuthorizeCallback) { - ac = (AuthorizeCallback) callback; - } else { - throw new UnsupportedCallbackException(callback); - } - } - PasswdAuthenticationProvider provider = - AuthenticationProviderFactory.getAuthenticationProvider(authMethod); - provider.Authenticate(username, password); - if (ac != null) { - ac.setAuthorized(true); - } - } - } - - public static class PlainCallbackHandler implements CallbackHandler { - - private final String username; - private final String password; - - public PlainCallbackHandler(String username, String password) { - this.username = username; - this.password = password; - } - - @Override - public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { - for (Callback callback : callbacks) { - if (callback instanceof NameCallback) { - NameCallback nameCallback = (NameCallback) callback; - nameCallback.setName(username); - } else if (callback instanceof PasswordCallback) { - PasswordCallback passCallback = (PasswordCallback) callback; - passCallback.setPassword(password.toCharArray()); - } else { - throw new UnsupportedCallbackException(callback); - } - } - } - } - - private static final class SQLPlainProcessorFactory extends TProcessorFactory { - - private final ThriftCLIService service; - - SQLPlainProcessorFactory(ThriftCLIService service) { - super(null); - this.service = service; - } - - @Override - public TProcessor getProcessor(TTransport trans) { - return new TSetIpAddressProcessor(service); - } - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java deleted file mode 100644 index 9a61ad49942c8..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java +++ /dev/null @@ -1,114 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.auth; - -import org.apache.hive.service.cli.thrift.TCLIService; -import org.apache.hive.service.cli.thrift.TCLIService.Iface; -import org.apache.thrift.TException; -import org.apache.thrift.protocol.TProtocol; -import org.apache.thrift.transport.TSaslClientTransport; -import org.apache.thrift.transport.TSaslServerTransport; -import org.apache.thrift.transport.TSocket; -import org.apache.thrift.transport.TTransport; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This class is responsible for setting the ipAddress for operations executed via HiveServer2. - * - * - IP address is only set for operations that calls listeners with hookContext - * - IP address is only set if the underlying transport mechanism is socket - * - * @see org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext - */ -public class TSetIpAddressProcessor extends TCLIService.Processor { - - private static final Logger LOGGER = LoggerFactory.getLogger(TSetIpAddressProcessor.class.getName()); - - public TSetIpAddressProcessor(Iface iface) { - super(iface); - } - - @Override - public boolean process(final TProtocol in, final TProtocol out) throws TException { - setIpAddress(in); - setUserName(in); - try { - return super.process(in, out); - } finally { - THREAD_LOCAL_USER_NAME.remove(); - THREAD_LOCAL_IP_ADDRESS.remove(); - } - } - - private void setUserName(final TProtocol in) { - TTransport transport = in.getTransport(); - if (transport instanceof TSaslServerTransport) { - String userName = ((TSaslServerTransport) transport).getSaslServer().getAuthorizationID(); - THREAD_LOCAL_USER_NAME.set(userName); - } - } - - protected void setIpAddress(final TProtocol in) { - TTransport transport = in.getTransport(); - TSocket tSocket = getUnderlyingSocketFromTransport(transport); - if (tSocket == null) { - LOGGER.warn("Unknown Transport, cannot determine ipAddress"); - } else { - THREAD_LOCAL_IP_ADDRESS.set(tSocket.getSocket().getInetAddress().getHostAddress()); - } - } - - private TSocket getUnderlyingSocketFromTransport(TTransport transport) { - while (transport != null) { - if (transport instanceof TSaslServerTransport) { - transport = ((TSaslServerTransport) transport).getUnderlyingTransport(); - } - if (transport instanceof TSaslClientTransport) { - transport = ((TSaslClientTransport) transport).getUnderlyingTransport(); - } - if (transport instanceof TSocket) { - return (TSocket) transport; - } - } - return null; - } - - private static final ThreadLocal THREAD_LOCAL_IP_ADDRESS = new ThreadLocal() { - @Override - protected synchronized String initialValue() { - return null; - } - }; - - private static final ThreadLocal THREAD_LOCAL_USER_NAME = new ThreadLocal() { - @Override - protected synchronized String initialValue() { - return null; - } - }; - - public static String getUserIpAddress() { - return THREAD_LOCAL_IP_ADDRESS.get(); - } - - public static String getUserName() { - return THREAD_LOCAL_USER_NAME.get(); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/CLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/CLIService.java deleted file mode 100644 index 791ddcbd2c5b6..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/CLIService.java +++ /dev/null @@ -1,507 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CancellationException; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - -import javax.security.auth.login.LoginException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.ql.exec.FunctionRegistry; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.hive.shims.Utils; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hive.service.CompositeService; -import org.apache.hive.service.ServiceException; -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.cli.operation.Operation; -import org.apache.hive.service.cli.session.SessionManager; -import org.apache.hive.service.cli.thrift.TProtocolVersion; -import org.apache.hive.service.server.HiveServer2; - -/** - * CLIService. - * - */ -public class CLIService extends CompositeService implements ICLIService { - - public static final TProtocolVersion SERVER_VERSION; - - static { - TProtocolVersion[] protocols = TProtocolVersion.values(); - SERVER_VERSION = protocols[protocols.length - 1]; - } - - private final Log LOG = LogFactory.getLog(CLIService.class.getName()); - - private HiveConf hiveConf; - private SessionManager sessionManager; - private UserGroupInformation serviceUGI; - private UserGroupInformation httpUGI; - // The HiveServer2 instance running this service - private final HiveServer2 hiveServer2; - - public CLIService(HiveServer2 hiveServer2) { - super(CLIService.class.getSimpleName()); - this.hiveServer2 = hiveServer2; - } - - @Override - public synchronized void init(HiveConf hiveConf) { - this.hiveConf = hiveConf; - sessionManager = new SessionManager(hiveServer2); - addService(sessionManager); - // If the hadoop cluster is secure, do a kerberos login for the service from the keytab - if (UserGroupInformation.isSecurityEnabled()) { - try { - HiveAuthFactory.loginFromKeytab(hiveConf); - this.serviceUGI = Utils.getUGI(); - } catch (IOException e) { - throw new ServiceException("Unable to login to kerberos with given principal/keytab", e); - } catch (LoginException e) { - throw new ServiceException("Unable to login to kerberos with given principal/keytab", e); - } - - // Also try creating a UGI object for the SPNego principal - String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_PRINCIPAL); - String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_KEYTAB); - if (principal.isEmpty() || keyTabFile.isEmpty()) { - LOG.info("SPNego httpUGI not created, spNegoPrincipal: " + principal + - ", ketabFile: " + keyTabFile); - } else { - try { - this.httpUGI = HiveAuthFactory.loginFromSpnegoKeytabAndReturnUGI(hiveConf); - LOG.info("SPNego httpUGI successfully created."); - } catch (IOException e) { - LOG.warn("SPNego httpUGI creation failed: ", e); - } - } - } - // creates connection to HMS and thus *must* occur after kerberos login above - try { - applyAuthorizationConfigPolicy(hiveConf); - } catch (Exception e) { - throw new RuntimeException("Error applying authorization policy on hive configuration: " - + e.getMessage(), e); - } - setupBlockedUdfs(); - super.init(hiveConf); - } - - private void applyAuthorizationConfigPolicy(HiveConf newHiveConf) throws HiveException, - MetaException { - // authorization setup using SessionState should be revisited eventually, as - // authorization and authentication are not session specific settings - SessionState ss = new SessionState(newHiveConf); - ss.setIsHiveServerQuery(true); - SessionState.start(ss); - ss.applyAuthorizationPolicy(); - } - - private void setupBlockedUdfs() { - FunctionRegistry.setupPermissionsForBuiltinUDFs( - hiveConf.getVar(ConfVars.HIVE_SERVER2_BUILTIN_UDF_WHITELIST), - hiveConf.getVar(ConfVars.HIVE_SERVER2_BUILTIN_UDF_BLACKLIST)); - } - - public UserGroupInformation getServiceUGI() { - return this.serviceUGI; - } - - public UserGroupInformation getHttpUGI() { - return this.httpUGI; - } - - @Override - public synchronized void start() { - super.start(); - // Initialize and test a connection to the metastore - IMetaStoreClient metastoreClient = null; - try { - metastoreClient = new HiveMetaStoreClient(hiveConf); - metastoreClient.getDatabases("default"); - } catch (Exception e) { - throw new ServiceException("Unable to connect to MetaStore!", e); - } - finally { - if (metastoreClient != null) { - metastoreClient.close(); - } - } - } - - @Override - public synchronized void stop() { - super.stop(); - } - - /** - * @deprecated Use {@link #openSession(TProtocolVersion, String, String, String, Map)} - */ - @Deprecated - public SessionHandle openSession(TProtocolVersion protocol, String username, String password, - Map configuration) throws HiveSQLException { - SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, null, configuration, false, null); - LOG.debug(sessionHandle + ": openSession()"); - return sessionHandle; - } - - /** - * @deprecated Use {@link #openSessionWithImpersonation(TProtocolVersion, String, String, String, Map, String)} - */ - @Deprecated - public SessionHandle openSessionWithImpersonation(TProtocolVersion protocol, String username, - String password, Map configuration, String delegationToken) - throws HiveSQLException { - SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, null, configuration, - true, delegationToken); - LOG.debug(sessionHandle + ": openSessionWithImpersonation()"); - return sessionHandle; - } - - public SessionHandle openSession(TProtocolVersion protocol, String username, String password, String ipAddress, - Map configuration) throws HiveSQLException { - SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, ipAddress, configuration, false, null); - LOG.debug(sessionHandle + ": openSession()"); - return sessionHandle; - } - - public SessionHandle openSessionWithImpersonation(TProtocolVersion protocol, String username, - String password, String ipAddress, Map configuration, String delegationToken) - throws HiveSQLException { - SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, ipAddress, configuration, - true, delegationToken); - LOG.debug(sessionHandle + ": openSession()"); - return sessionHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map) - */ - @Override - public SessionHandle openSession(String username, String password, Map configuration) - throws HiveSQLException { - SessionHandle sessionHandle = sessionManager.openSession(SERVER_VERSION, username, password, null, configuration, false, null); - LOG.debug(sessionHandle + ": openSession()"); - return sessionHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map) - */ - @Override - public SessionHandle openSessionWithImpersonation(String username, String password, Map configuration, - String delegationToken) throws HiveSQLException { - SessionHandle sessionHandle = sessionManager.openSession(SERVER_VERSION, username, password, null, configuration, - true, delegationToken); - LOG.debug(sessionHandle + ": openSession()"); - return sessionHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public void closeSession(SessionHandle sessionHandle) - throws HiveSQLException { - sessionManager.closeSession(sessionHandle); - LOG.debug(sessionHandle + ": closeSession()"); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List) - */ - @Override - public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType getInfoType) - throws HiveSQLException { - GetInfoValue infoValue = sessionManager.getSession(sessionHandle) - .getInfo(getInfoType); - LOG.debug(sessionHandle + ": getInfo()"); - return infoValue; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#executeStatement(org.apache.hive.service.cli.SessionHandle, - * java.lang.String, java.util.Map) - */ - @Override - public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, - Map confOverlay) - throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .executeStatement(statement, confOverlay); - LOG.debug(sessionHandle + ": executeStatement()"); - return opHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#executeStatementAsync(org.apache.hive.service.cli.SessionHandle, - * java.lang.String, java.util.Map) - */ - @Override - public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, - Map confOverlay) throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .executeStatementAsync(statement, confOverlay); - LOG.debug(sessionHandle + ": executeStatementAsync()"); - return opHandle; - } - - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getTypeInfo(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getTypeInfo(SessionHandle sessionHandle) - throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .getTypeInfo(); - LOG.debug(sessionHandle + ": getTypeInfo()"); - return opHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getCatalogs(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getCatalogs(SessionHandle sessionHandle) - throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .getCatalogs(); - LOG.debug(sessionHandle + ": getCatalogs()"); - return opHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String) - */ - @Override - public OperationHandle getSchemas(SessionHandle sessionHandle, - String catalogName, String schemaName) - throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .getSchemas(catalogName, schemaName); - LOG.debug(sessionHandle + ": getSchemas()"); - return opHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List) - */ - @Override - public OperationHandle getTables(SessionHandle sessionHandle, - String catalogName, String schemaName, String tableName, List tableTypes) - throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .getTables(catalogName, schemaName, tableName, tableTypes); - LOG.debug(sessionHandle + ": getTables()"); - return opHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getTableTypes(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getTableTypes(SessionHandle sessionHandle) - throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .getTableTypes(); - LOG.debug(sessionHandle + ": getTableTypes()"); - return opHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getColumns(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getColumns(SessionHandle sessionHandle, - String catalogName, String schemaName, String tableName, String columnName) - throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .getColumns(catalogName, schemaName, tableName, columnName); - LOG.debug(sessionHandle + ": getColumns()"); - return opHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getFunctions(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getFunctions(SessionHandle sessionHandle, - String catalogName, String schemaName, String functionName) - throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .getFunctions(catalogName, schemaName, functionName); - LOG.debug(sessionHandle + ": getFunctions()"); - return opHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getOperationStatus(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public OperationStatus getOperationStatus(OperationHandle opHandle) - throws HiveSQLException { - Operation operation = sessionManager.getOperationManager().getOperation(opHandle); - /** - * If this is a background operation run asynchronously, - * we block for a configured duration, before we return - * (duration: HIVE_SERVER2_LONG_POLLING_TIMEOUT). - * However, if the background operation is complete, we return immediately. - */ - if (operation.shouldRunAsync()) { - HiveConf conf = operation.getParentSession().getHiveConf(); - long timeout = HiveConf.getTimeVar(conf, - HiveConf.ConfVars.HIVE_SERVER2_LONG_POLLING_TIMEOUT, TimeUnit.MILLISECONDS); - try { - operation.getBackgroundHandle().get(timeout, TimeUnit.MILLISECONDS); - } catch (TimeoutException e) { - // No Op, return to the caller since long polling timeout has expired - LOG.trace(opHandle + ": Long polling timed out"); - } catch (CancellationException e) { - // The background operation thread was cancelled - LOG.trace(opHandle + ": The background operation was cancelled", e); - } catch (ExecutionException e) { - // The background operation thread was aborted - LOG.warn(opHandle + ": The background operation was aborted", e); - } catch (InterruptedException e) { - // No op, this thread was interrupted - // In this case, the call might return sooner than long polling timeout - } - } - OperationStatus opStatus = operation.getStatus(); - LOG.debug(opHandle + ": getOperationStatus()"); - return opStatus; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#cancelOperation(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public void cancelOperation(OperationHandle opHandle) - throws HiveSQLException { - sessionManager.getOperationManager().getOperation(opHandle) - .getParentSession().cancelOperation(opHandle); - LOG.debug(opHandle + ": cancelOperation()"); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#closeOperation(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public void closeOperation(OperationHandle opHandle) - throws HiveSQLException { - sessionManager.getOperationManager().getOperation(opHandle) - .getParentSession().closeOperation(opHandle); - LOG.debug(opHandle + ": closeOperation"); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public TableSchema getResultSetMetadata(OperationHandle opHandle) - throws HiveSQLException { - TableSchema tableSchema = sessionManager.getOperationManager() - .getOperation(opHandle).getParentSession().getResultSetMetadata(opHandle); - LOG.debug(opHandle + ": getResultSetMetadata()"); - return tableSchema; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#fetchResults(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public RowSet fetchResults(OperationHandle opHandle) - throws HiveSQLException { - return fetchResults(opHandle, Operation.DEFAULT_FETCH_ORIENTATION, - Operation.DEFAULT_FETCH_MAX_ROWS, FetchType.QUERY_OUTPUT); - } - - @Override - public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, - long maxRows, FetchType fetchType) throws HiveSQLException { - RowSet rowSet = sessionManager.getOperationManager().getOperation(opHandle) - .getParentSession().fetchResults(opHandle, orientation, maxRows, fetchType); - LOG.debug(opHandle + ": fetchResults()"); - return rowSet; - } - - // obtain delegation token for the give user from metastore - public synchronized String getDelegationTokenFromMetaStore(String owner) - throws HiveSQLException, UnsupportedOperationException, LoginException, IOException { - if (!hiveConf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL) || - !hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS)) { - throw new UnsupportedOperationException( - "delegation token is can only be obtained for a secure remote metastore"); - } - - try { - Hive.closeCurrent(); - return Hive.get(hiveConf).getDelegationToken(owner, owner); - } catch (HiveException e) { - if (e.getCause() instanceof UnsupportedOperationException) { - throw (UnsupportedOperationException)e.getCause(); - } else { - throw new HiveSQLException("Error connect metastore to setup impersonation", e); - } - } - } - - @Override - public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String owner, String renewer) throws HiveSQLException { - String delegationToken = sessionManager.getSession(sessionHandle) - .getDelegationToken(authFactory, owner, renewer); - LOG.info(sessionHandle + ": getDelegationToken()"); - return delegationToken; - } - - @Override - public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String tokenStr) throws HiveSQLException { - sessionManager.getSession(sessionHandle).cancelDelegationToken(authFactory, tokenStr); - LOG.info(sessionHandle + ": cancelDelegationToken()"); - } - - @Override - public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String tokenStr) throws HiveSQLException { - sessionManager.getSession(sessionHandle).renewDelegationToken(authFactory, tokenStr); - LOG.info(sessionHandle + ": renewDelegationToken()"); - } - - public SessionManager getSessionManager() { - return sessionManager; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Column.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Column.java deleted file mode 100644 index 26d0f718f383a..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Column.java +++ /dev/null @@ -1,423 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import java.nio.ByteBuffer; -import java.util.AbstractList; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.BitSet; -import java.util.List; - -import com.google.common.primitives.Booleans; -import com.google.common.primitives.Bytes; -import com.google.common.primitives.Doubles; -import com.google.common.primitives.Ints; -import com.google.common.primitives.Longs; -import com.google.common.primitives.Shorts; -import org.apache.hive.service.cli.thrift.TBinaryColumn; -import org.apache.hive.service.cli.thrift.TBoolColumn; -import org.apache.hive.service.cli.thrift.TByteColumn; -import org.apache.hive.service.cli.thrift.TColumn; -import org.apache.hive.service.cli.thrift.TDoubleColumn; -import org.apache.hive.service.cli.thrift.TI16Column; -import org.apache.hive.service.cli.thrift.TI32Column; -import org.apache.hive.service.cli.thrift.TI64Column; -import org.apache.hive.service.cli.thrift.TStringColumn; - -/** - * Column. - */ -public class Column extends AbstractList { - - private static final int DEFAULT_SIZE = 100; - - private final Type type; - - private BitSet nulls; - - private int size; - private boolean[] boolVars; - private byte[] byteVars; - private short[] shortVars; - private int[] intVars; - private long[] longVars; - private double[] doubleVars; - private List stringVars; - private List binaryVars; - - public Column(Type type, BitSet nulls, Object values) { - this.type = type; - this.nulls = nulls; - if (type == Type.BOOLEAN_TYPE) { - boolVars = (boolean[]) values; - size = boolVars.length; - } else if (type == Type.TINYINT_TYPE) { - byteVars = (byte[]) values; - size = byteVars.length; - } else if (type == Type.SMALLINT_TYPE) { - shortVars = (short[]) values; - size = shortVars.length; - } else if (type == Type.INT_TYPE) { - intVars = (int[]) values; - size = intVars.length; - } else if (type == Type.BIGINT_TYPE) { - longVars = (long[]) values; - size = longVars.length; - } else if (type == Type.DOUBLE_TYPE) { - doubleVars = (double[]) values; - size = doubleVars.length; - } else if (type == Type.BINARY_TYPE) { - binaryVars = (List) values; - size = binaryVars.size(); - } else if (type == Type.STRING_TYPE) { - stringVars = (List) values; - size = stringVars.size(); - } else { - throw new IllegalStateException("invalid union object"); - } - } - - public Column(Type type) { - nulls = new BitSet(); - switch (type) { - case BOOLEAN_TYPE: - boolVars = new boolean[DEFAULT_SIZE]; - break; - case TINYINT_TYPE: - byteVars = new byte[DEFAULT_SIZE]; - break; - case SMALLINT_TYPE: - shortVars = new short[DEFAULT_SIZE]; - break; - case INT_TYPE: - intVars = new int[DEFAULT_SIZE]; - break; - case BIGINT_TYPE: - longVars = new long[DEFAULT_SIZE]; - break; - case FLOAT_TYPE: - case DOUBLE_TYPE: - type = Type.DOUBLE_TYPE; - doubleVars = new double[DEFAULT_SIZE]; - break; - case BINARY_TYPE: - binaryVars = new ArrayList(); - break; - default: - type = Type.STRING_TYPE; - stringVars = new ArrayList(); - } - this.type = type; - } - - public Column(TColumn colValues) { - if (colValues.isSetBoolVal()) { - type = Type.BOOLEAN_TYPE; - nulls = toBitset(colValues.getBoolVal().getNulls()); - boolVars = Booleans.toArray(colValues.getBoolVal().getValues()); - size = boolVars.length; - } else if (colValues.isSetByteVal()) { - type = Type.TINYINT_TYPE; - nulls = toBitset(colValues.getByteVal().getNulls()); - byteVars = Bytes.toArray(colValues.getByteVal().getValues()); - size = byteVars.length; - } else if (colValues.isSetI16Val()) { - type = Type.SMALLINT_TYPE; - nulls = toBitset(colValues.getI16Val().getNulls()); - shortVars = Shorts.toArray(colValues.getI16Val().getValues()); - size = shortVars.length; - } else if (colValues.isSetI32Val()) { - type = Type.INT_TYPE; - nulls = toBitset(colValues.getI32Val().getNulls()); - intVars = Ints.toArray(colValues.getI32Val().getValues()); - size = intVars.length; - } else if (colValues.isSetI64Val()) { - type = Type.BIGINT_TYPE; - nulls = toBitset(colValues.getI64Val().getNulls()); - longVars = Longs.toArray(colValues.getI64Val().getValues()); - size = longVars.length; - } else if (colValues.isSetDoubleVal()) { - type = Type.DOUBLE_TYPE; - nulls = toBitset(colValues.getDoubleVal().getNulls()); - doubleVars = Doubles.toArray(colValues.getDoubleVal().getValues()); - size = doubleVars.length; - } else if (colValues.isSetBinaryVal()) { - type = Type.BINARY_TYPE; - nulls = toBitset(colValues.getBinaryVal().getNulls()); - binaryVars = colValues.getBinaryVal().getValues(); - size = binaryVars.size(); - } else if (colValues.isSetStringVal()) { - type = Type.STRING_TYPE; - nulls = toBitset(colValues.getStringVal().getNulls()); - stringVars = colValues.getStringVal().getValues(); - size = stringVars.size(); - } else { - throw new IllegalStateException("invalid union object"); - } - } - - public Column extractSubset(int start, int end) { - BitSet subNulls = nulls.get(start, end); - if (type == Type.BOOLEAN_TYPE) { - Column subset = new Column(type, subNulls, Arrays.copyOfRange(boolVars, start, end)); - boolVars = Arrays.copyOfRange(boolVars, end, size); - nulls = nulls.get(start, size); - size = boolVars.length; - return subset; - } - if (type == Type.TINYINT_TYPE) { - Column subset = new Column(type, subNulls, Arrays.copyOfRange(byteVars, start, end)); - byteVars = Arrays.copyOfRange(byteVars, end, size); - nulls = nulls.get(start, size); - size = byteVars.length; - return subset; - } - if (type == Type.SMALLINT_TYPE) { - Column subset = new Column(type, subNulls, Arrays.copyOfRange(shortVars, start, end)); - shortVars = Arrays.copyOfRange(shortVars, end, size); - nulls = nulls.get(start, size); - size = shortVars.length; - return subset; - } - if (type == Type.INT_TYPE) { - Column subset = new Column(type, subNulls, Arrays.copyOfRange(intVars, start, end)); - intVars = Arrays.copyOfRange(intVars, end, size); - nulls = nulls.get(start, size); - size = intVars.length; - return subset; - } - if (type == Type.BIGINT_TYPE) { - Column subset = new Column(type, subNulls, Arrays.copyOfRange(longVars, start, end)); - longVars = Arrays.copyOfRange(longVars, end, size); - nulls = nulls.get(start, size); - size = longVars.length; - return subset; - } - if (type == Type.DOUBLE_TYPE) { - Column subset = new Column(type, subNulls, Arrays.copyOfRange(doubleVars, start, end)); - doubleVars = Arrays.copyOfRange(doubleVars, end, size); - nulls = nulls.get(start, size); - size = doubleVars.length; - return subset; - } - if (type == Type.BINARY_TYPE) { - Column subset = new Column(type, subNulls, binaryVars.subList(start, end)); - binaryVars = binaryVars.subList(end, binaryVars.size()); - nulls = nulls.get(start, size); - size = binaryVars.size(); - return subset; - } - if (type == Type.STRING_TYPE) { - Column subset = new Column(type, subNulls, stringVars.subList(start, end)); - stringVars = stringVars.subList(end, stringVars.size()); - nulls = nulls.get(start, size); - size = stringVars.size(); - return subset; - } - throw new IllegalStateException("invalid union object"); - } - - private static final byte[] MASKS = new byte[] { - 0x01, 0x02, 0x04, 0x08, 0x10, 0x20, 0x40, (byte)0x80 - }; - - private static BitSet toBitset(byte[] nulls) { - BitSet bitset = new BitSet(); - int bits = nulls.length * 8; - for (int i = 0; i < bits; i++) { - bitset.set(i, (nulls[i / 8] & MASKS[i % 8]) != 0); - } - return bitset; - } - - private static byte[] toBinary(BitSet bitset) { - byte[] nulls = new byte[1 + (bitset.length() / 8)]; - for (int i = 0; i < bitset.length(); i++) { - nulls[i / 8] |= bitset.get(i) ? MASKS[i % 8] : 0; - } - return nulls; - } - - public Type getType() { - return type; - } - - @Override - public Object get(int index) { - if (nulls.get(index)) { - return null; - } - switch (type) { - case BOOLEAN_TYPE: - return boolVars[index]; - case TINYINT_TYPE: - return byteVars[index]; - case SMALLINT_TYPE: - return shortVars[index]; - case INT_TYPE: - return intVars[index]; - case BIGINT_TYPE: - return longVars[index]; - case DOUBLE_TYPE: - return doubleVars[index]; - case STRING_TYPE: - return stringVars.get(index); - case BINARY_TYPE: - return binaryVars.get(index).array(); - } - return null; - } - - @Override - public int size() { - return size; - } - - public TColumn toTColumn() { - TColumn value = new TColumn(); - ByteBuffer nullMasks = ByteBuffer.wrap(toBinary(nulls)); - switch (type) { - case BOOLEAN_TYPE: - value.setBoolVal(new TBoolColumn(Booleans.asList(Arrays.copyOfRange(boolVars, 0, size)), nullMasks)); - break; - case TINYINT_TYPE: - value.setByteVal(new TByteColumn(Bytes.asList(Arrays.copyOfRange(byteVars, 0, size)), nullMasks)); - break; - case SMALLINT_TYPE: - value.setI16Val(new TI16Column(Shorts.asList(Arrays.copyOfRange(shortVars, 0, size)), nullMasks)); - break; - case INT_TYPE: - value.setI32Val(new TI32Column(Ints.asList(Arrays.copyOfRange(intVars, 0, size)), nullMasks)); - break; - case BIGINT_TYPE: - value.setI64Val(new TI64Column(Longs.asList(Arrays.copyOfRange(longVars, 0, size)), nullMasks)); - break; - case DOUBLE_TYPE: - value.setDoubleVal(new TDoubleColumn(Doubles.asList(Arrays.copyOfRange(doubleVars, 0, size)), nullMasks)); - break; - case STRING_TYPE: - value.setStringVal(new TStringColumn(stringVars, nullMasks)); - break; - case BINARY_TYPE: - value.setBinaryVal(new TBinaryColumn(binaryVars, nullMasks)); - break; - } - return value; - } - - private static final ByteBuffer EMPTY_BINARY = ByteBuffer.allocate(0); - private static final String EMPTY_STRING = ""; - - public void addValue(Type type, Object field) { - switch (type) { - case BOOLEAN_TYPE: - nulls.set(size, field == null); - boolVars()[size] = field == null ? true : (Boolean)field; - break; - case TINYINT_TYPE: - nulls.set(size, field == null); - byteVars()[size] = field == null ? 0 : (Byte) field; - break; - case SMALLINT_TYPE: - nulls.set(size, field == null); - shortVars()[size] = field == null ? 0 : (Short)field; - break; - case INT_TYPE: - nulls.set(size, field == null); - intVars()[size] = field == null ? 0 : (Integer)field; - break; - case BIGINT_TYPE: - nulls.set(size, field == null); - longVars()[size] = field == null ? 0 : (Long)field; - break; - case FLOAT_TYPE: - nulls.set(size, field == null); - doubleVars()[size] = field == null ? 0 : Double.valueOf(field.toString()); - break; - case DOUBLE_TYPE: - nulls.set(size, field == null); - doubleVars()[size] = field == null ? 0 : (Double)field; - break; - case BINARY_TYPE: - nulls.set(binaryVars.size(), field == null); - binaryVars.add(field == null ? EMPTY_BINARY : ByteBuffer.wrap((byte[])field)); - break; - default: - nulls.set(stringVars.size(), field == null); - stringVars.add(field == null ? EMPTY_STRING : String.valueOf(field)); - break; - } - size++; - } - - private boolean[] boolVars() { - if (boolVars.length == size) { - boolean[] newVars = new boolean[size << 1]; - System.arraycopy(boolVars, 0, newVars, 0, size); - return boolVars = newVars; - } - return boolVars; - } - - private byte[] byteVars() { - if (byteVars.length == size) { - byte[] newVars = new byte[size << 1]; - System.arraycopy(byteVars, 0, newVars, 0, size); - return byteVars = newVars; - } - return byteVars; - } - - private short[] shortVars() { - if (shortVars.length == size) { - short[] newVars = new short[size << 1]; - System.arraycopy(shortVars, 0, newVars, 0, size); - return shortVars = newVars; - } - return shortVars; - } - - private int[] intVars() { - if (intVars.length == size) { - int[] newVars = new int[size << 1]; - System.arraycopy(intVars, 0, newVars, 0, size); - return intVars = newVars; - } - return intVars; - } - - private long[] longVars() { - if (longVars.length == size) { - long[] newVars = new long[size << 1]; - System.arraycopy(longVars, 0, newVars, 0, size); - return longVars = newVars; - } - return longVars; - } - - private double[] doubleVars() { - if (doubleVars.length == size) { - double[] newVars = new double[size << 1]; - System.arraycopy(doubleVars, 0, newVars, 0, size); - return doubleVars = newVars; - } - return doubleVars; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java deleted file mode 100644 index 47a582e2223e4..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java +++ /dev/null @@ -1,149 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - -import org.apache.hive.service.cli.thrift.TColumn; -import org.apache.hive.service.cli.thrift.TRow; -import org.apache.hive.service.cli.thrift.TRowSet; - -/** - * ColumnBasedSet. - */ -public class ColumnBasedSet implements RowSet { - - private long startOffset; - - private final Type[] types; // non-null only for writing (server-side) - private final List columns; - - public ColumnBasedSet(TableSchema schema) { - types = schema.toTypes(); - columns = new ArrayList(); - for (ColumnDescriptor colDesc : schema.getColumnDescriptors()) { - columns.add(new Column(colDesc.getType())); - } - } - - public ColumnBasedSet(TRowSet tRowSet) { - types = null; - columns = new ArrayList(); - for (TColumn tvalue : tRowSet.getColumns()) { - columns.add(new Column(tvalue)); - } - startOffset = tRowSet.getStartRowOffset(); - } - - private ColumnBasedSet(Type[] types, List columns, long startOffset) { - this.types = types; - this.columns = columns; - this.startOffset = startOffset; - } - - @Override - public ColumnBasedSet addRow(Object[] fields) { - for (int i = 0; i < fields.length; i++) { - columns.get(i).addValue(types[i], fields[i]); - } - return this; - } - - public List getColumns() { - return columns; - } - - @Override - public int numColumns() { - return columns.size(); - } - - @Override - public int numRows() { - return columns.isEmpty() ? 0 : columns.get(0).size(); - } - - @Override - public ColumnBasedSet extractSubset(int maxRows) { - int numRows = Math.min(numRows(), maxRows); - - List subset = new ArrayList(); - for (int i = 0; i < columns.size(); i++) { - subset.add(columns.get(i).extractSubset(0, numRows)); - } - ColumnBasedSet result = new ColumnBasedSet(types, subset, startOffset); - startOffset += numRows; - return result; - } - - @Override - public long getStartOffset() { - return startOffset; - } - - @Override - public void setStartOffset(long startOffset) { - this.startOffset = startOffset; - } - - public TRowSet toTRowSet() { - TRowSet tRowSet = new TRowSet(startOffset, new ArrayList()); - for (int i = 0; i < columns.size(); i++) { - tRowSet.addToColumns(columns.get(i).toTColumn()); - } - return tRowSet; - } - - @Override - public Iterator iterator() { - return new Iterator() { - - private int index; - private final Object[] convey = new Object[numColumns()]; - - @Override - public boolean hasNext() { - return index < numRows(); - } - - @Override - public Object[] next() { - for (int i = 0; i < columns.size(); i++) { - convey[i] = columns.get(i).get(index); - } - index++; - return convey; - } - - @Override - public void remove() { - throw new UnsupportedOperationException("remove"); - } - }; - } - - public Object[] fill(int index, Object[] convey) { - for (int i = 0; i < columns.size(); i++) { - convey[i] = columns.get(i).get(index); - } - return convey; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java deleted file mode 100644 index f0bbf14693160..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java +++ /dev/null @@ -1,99 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hive.service.cli.thrift.TColumnDesc; - - -/** - * ColumnDescriptor. - * - */ -public class ColumnDescriptor { - private final String name; - private final String comment; - private final TypeDescriptor type; - // ordinal position of this column in the schema - private final int position; - - public ColumnDescriptor(String name, String comment, TypeDescriptor type, int position) { - this.name = name; - this.comment = comment; - this.type = type; - this.position = position; - } - - public ColumnDescriptor(TColumnDesc tColumnDesc) { - name = tColumnDesc.getColumnName(); - comment = tColumnDesc.getComment(); - type = new TypeDescriptor(tColumnDesc.getTypeDesc()); - position = tColumnDesc.getPosition(); - } - - public ColumnDescriptor(FieldSchema column, int position) { - name = column.getName(); - comment = column.getComment(); - type = new TypeDescriptor(column.getType()); - this.position = position; - } - - public static ColumnDescriptor newPrimitiveColumnDescriptor(String name, String comment, Type type, int position) { - // Current usage looks like it's only for metadata columns, but if that changes then - // this method may need to require a type qualifiers aruments. - return new ColumnDescriptor(name, comment, new TypeDescriptor(type), position); - } - - public String getName() { - return name; - } - - public String getComment() { - return comment; - } - - public TypeDescriptor getTypeDescriptor() { - return type; - } - - public int getOrdinalPosition() { - return position; - } - - public TColumnDesc toTColumnDesc() { - TColumnDesc tColumnDesc = new TColumnDesc(); - tColumnDesc.setColumnName(name); - tColumnDesc.setComment(comment); - tColumnDesc.setTypeDesc(type.toTTypeDesc()); - tColumnDesc.setPosition(position); - return tColumnDesc; - } - - public Type getType() { - return type.getType(); - } - - public boolean isPrimitive() { - return type.getType().isPrimitiveType(); - } - - public String getTypeName() { - return type.getTypeName(); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnValue.java deleted file mode 100644 index 462b93a0f09fe..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnValue.java +++ /dev/null @@ -1,288 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import java.math.BigDecimal; -import java.sql.Date; -import java.sql.Timestamp; - -import org.apache.hadoop.hive.common.type.HiveChar; -import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; -import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; -import org.apache.hadoop.hive.common.type.HiveVarchar; -import org.apache.hive.service.cli.thrift.TBoolValue; -import org.apache.hive.service.cli.thrift.TByteValue; -import org.apache.hive.service.cli.thrift.TColumnValue; -import org.apache.hive.service.cli.thrift.TDoubleValue; -import org.apache.hive.service.cli.thrift.TI16Value; -import org.apache.hive.service.cli.thrift.TI32Value; -import org.apache.hive.service.cli.thrift.TI64Value; -import org.apache.hive.service.cli.thrift.TStringValue; - -import org.apache.spark.unsafe.types.UTF8String; - -/** - * Protocols before HIVE_CLI_SERVICE_PROTOCOL_V6 (used by RowBasedSet) - * - */ -public class ColumnValue { - - private static TColumnValue booleanValue(Boolean value) { - TBoolValue tBoolValue = new TBoolValue(); - if (value != null) { - tBoolValue.setValue(value); - } - return TColumnValue.boolVal(tBoolValue); - } - - private static TColumnValue byteValue(Byte value) { - TByteValue tByteValue = new TByteValue(); - if (value != null) { - tByteValue.setValue(value); - } - return TColumnValue.byteVal(tByteValue); - } - - private static TColumnValue shortValue(Short value) { - TI16Value tI16Value = new TI16Value(); - if (value != null) { - tI16Value.setValue(value); - } - return TColumnValue.i16Val(tI16Value); - } - - private static TColumnValue intValue(Integer value) { - TI32Value tI32Value = new TI32Value(); - if (value != null) { - tI32Value.setValue(value); - } - return TColumnValue.i32Val(tI32Value); - } - - private static TColumnValue longValue(Long value) { - TI64Value tI64Value = new TI64Value(); - if (value != null) { - tI64Value.setValue(value); - } - return TColumnValue.i64Val(tI64Value); - } - - private static TColumnValue floatValue(Float value) { - TDoubleValue tDoubleValue = new TDoubleValue(); - if (value != null) { - tDoubleValue.setValue(value); - } - return TColumnValue.doubleVal(tDoubleValue); - } - - private static TColumnValue doubleValue(Double value) { - TDoubleValue tDoubleValue = new TDoubleValue(); - if (value != null) { - tDoubleValue.setValue(value); - } - return TColumnValue.doubleVal(tDoubleValue); - } - - private static TColumnValue stringValue(String value) { - TStringValue tStringValue = new TStringValue(); - if (value != null) { - tStringValue.setValue(value); - } - return TColumnValue.stringVal(tStringValue); - } - - private static TColumnValue stringValue(HiveChar value) { - TStringValue tStringValue = new TStringValue(); - if (value != null) { - tStringValue.setValue(value.toString()); - } - return TColumnValue.stringVal(tStringValue); - } - - private static TColumnValue stringValue(HiveVarchar value) { - TStringValue tStringValue = new TStringValue(); - if (value != null) { - tStringValue.setValue(value.toString()); - } - return TColumnValue.stringVal(tStringValue); - } - - private static TColumnValue stringValue(HiveIntervalYearMonth value) { - TStringValue tStrValue = new TStringValue(); - if (value != null) { - tStrValue.setValue(value.toString()); - } - return TColumnValue.stringVal(tStrValue); - } - - private static TColumnValue stringValue(HiveIntervalDayTime value) { - TStringValue tStrValue = new TStringValue(); - if (value != null) { - tStrValue.setValue(value.toString()); - } - return TColumnValue.stringVal(tStrValue); - } - - public static TColumnValue toTColumnValue(Type type, Object value) { - switch (type) { - case BOOLEAN_TYPE: - return booleanValue((Boolean)value); - case TINYINT_TYPE: - return byteValue((Byte)value); - case SMALLINT_TYPE: - return shortValue((Short)value); - case INT_TYPE: - return intValue((Integer)value); - case BIGINT_TYPE: - return longValue((Long)value); - case FLOAT_TYPE: - return floatValue((Float)value); - case DOUBLE_TYPE: - return doubleValue((Double)value); - case STRING_TYPE: - return stringValue((String)value); - case CHAR_TYPE: - return stringValue((HiveChar)value); - case VARCHAR_TYPE: - return stringValue((HiveVarchar)value); - case DATE_TYPE: - case TIMESTAMP_TYPE: - // SPARK-31859, SPARK-31861: converted to string already in SparkExecuteStatementOperation - return stringValue((String)value); - case INTERVAL_YEAR_MONTH_TYPE: - return stringValue((HiveIntervalYearMonth) value); - case INTERVAL_DAY_TIME_TYPE: - return stringValue((HiveIntervalDayTime) value); - case DECIMAL_TYPE: - String plainStr = value == null ? null : ((BigDecimal)value).toPlainString(); - return stringValue(plainStr); - case BINARY_TYPE: - String strVal = value == null ? null : UTF8String.fromBytes((byte[])value).toString(); - return stringValue(strVal); - case ARRAY_TYPE: - case MAP_TYPE: - case STRUCT_TYPE: - case UNION_TYPE: - case USER_DEFINED_TYPE: - return stringValue((String)value); - case NULL_TYPE: - return stringValue((String)value); - default: - return null; - } - } - - private static Boolean getBooleanValue(TBoolValue tBoolValue) { - if (tBoolValue.isSetValue()) { - return tBoolValue.isValue(); - } - return null; - } - - private static Byte getByteValue(TByteValue tByteValue) { - if (tByteValue.isSetValue()) { - return tByteValue.getValue(); - } - return null; - } - - private static Short getShortValue(TI16Value tI16Value) { - if (tI16Value.isSetValue()) { - return tI16Value.getValue(); - } - return null; - } - - private static Integer getIntegerValue(TI32Value tI32Value) { - if (tI32Value.isSetValue()) { - return tI32Value.getValue(); - } - return null; - } - - private static Long getLongValue(TI64Value tI64Value) { - if (tI64Value.isSetValue()) { - return tI64Value.getValue(); - } - return null; - } - - private static Double getDoubleValue(TDoubleValue tDoubleValue) { - if (tDoubleValue.isSetValue()) { - return tDoubleValue.getValue(); - } - return null; - } - - private static String getStringValue(TStringValue tStringValue) { - if (tStringValue.isSetValue()) { - return tStringValue.getValue(); - } - return null; - } - - private static Date getDateValue(TStringValue tStringValue) { - if (tStringValue.isSetValue()) { - return Date.valueOf(tStringValue.getValue()); - } - return null; - } - - private static Timestamp getTimestampValue(TStringValue tStringValue) { - if (tStringValue.isSetValue()) { - return Timestamp.valueOf(tStringValue.getValue()); - } - return null; - } - - private static byte[] getBinaryValue(TStringValue tString) { - if (tString.isSetValue()) { - return tString.getValue().getBytes(); - } - return null; - } - - private static BigDecimal getBigDecimalValue(TStringValue tStringValue) { - if (tStringValue.isSetValue()) { - return new BigDecimal(tStringValue.getValue()); - } - return null; - } - - public static Object toColumnValue(TColumnValue value) { - TColumnValue._Fields field = value.getSetField(); - switch (field) { - case BOOL_VAL: - return getBooleanValue(value.getBoolVal()); - case BYTE_VAL: - return getByteValue(value.getByteVal()); - case I16_VAL: - return getShortValue(value.getI16Val()); - case I32_VAL: - return getIntegerValue(value.getI32Val()); - case I64_VAL: - return getLongValue(value.getI64Val()); - case DOUBLE_VAL: - return getDoubleValue(value.getDoubleVal()); - case STRING_VAL: - return getStringValue(value.getStringVal()); - } - throw new IllegalArgumentException("never"); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java deleted file mode 100644 index 9cad5be198c06..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java +++ /dev/null @@ -1,208 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import java.util.List; -import java.util.Map; - -import org.apache.hive.service.auth.HiveAuthFactory; - - -/** - * EmbeddedCLIServiceClient. - * - */ -public class EmbeddedCLIServiceClient extends CLIServiceClient { - private final ICLIService cliService; - - public EmbeddedCLIServiceClient(ICLIService cliService) { - this.cliService = cliService; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#openSession(java.lang.String, java.lang.String, java.util.Map) - */ - @Override - public SessionHandle openSession(String username, String password, - Map configuration) throws HiveSQLException { - return cliService.openSession(username, password, configuration); - } - - @Override - public SessionHandle openSessionWithImpersonation(String username, String password, - Map configuration, String delegationToken) throws HiveSQLException { - throw new HiveSQLException("Impersonated session is not supported in the embedded mode"); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#closeSession(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public void closeSession(SessionHandle sessionHandle) throws HiveSQLException { - cliService.closeSession(sessionHandle); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List) - */ - @Override - public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType getInfoType) - throws HiveSQLException { - return cliService.getInfo(sessionHandle, getInfoType); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#executeStatement(org.apache.hive.service.cli.SessionHandle, - * java.lang.String, java.util.Map) - */ - @Override - public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, - Map confOverlay) throws HiveSQLException { - return cliService.executeStatement(sessionHandle, statement, confOverlay); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#executeStatementAsync(org.apache.hive.service.cli.SessionHandle, - * java.lang.String, java.util.Map) - */ - @Override - public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, - Map confOverlay) throws HiveSQLException { - return cliService.executeStatementAsync(sessionHandle, statement, confOverlay); - } - - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getTypeInfo(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getTypeInfo(SessionHandle sessionHandle) throws HiveSQLException { - return cliService.getTypeInfo(sessionHandle); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getCatalogs(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getCatalogs(SessionHandle sessionHandle) throws HiveSQLException { - return cliService.getCatalogs(sessionHandle); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String) - */ - @Override - public OperationHandle getSchemas(SessionHandle sessionHandle, String catalogName, - String schemaName) throws HiveSQLException { - return cliService.getSchemas(sessionHandle, catalogName, schemaName); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List) - */ - @Override - public OperationHandle getTables(SessionHandle sessionHandle, String catalogName, - String schemaName, String tableName, List tableTypes) throws HiveSQLException { - return cliService.getTables(sessionHandle, catalogName, schemaName, tableName, tableTypes); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getTableTypes(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getTableTypes(SessionHandle sessionHandle) throws HiveSQLException { - return cliService.getTableTypes(sessionHandle); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getColumns(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.lang.String) - */ - @Override - public OperationHandle getColumns(SessionHandle sessionHandle, String catalogName, - String schemaName, String tableName, String columnName) throws HiveSQLException { - return cliService.getColumns(sessionHandle, catalogName, schemaName, tableName, columnName); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getFunctions(org.apache.hive.service.cli.SessionHandle, java.lang.String) - */ - @Override - public OperationHandle getFunctions(SessionHandle sessionHandle, - String catalogName, String schemaName, String functionName) - throws HiveSQLException { - return cliService.getFunctions(sessionHandle, catalogName, schemaName, functionName); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getOperationStatus(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public OperationStatus getOperationStatus(OperationHandle opHandle) throws HiveSQLException { - return cliService.getOperationStatus(opHandle); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#cancelOperation(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { - cliService.cancelOperation(opHandle); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#closeOperation(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public void closeOperation(OperationHandle opHandle) throws HiveSQLException { - cliService.closeOperation(opHandle); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public TableSchema getResultSetMetadata(OperationHandle opHandle) throws HiveSQLException { - return cliService.getResultSetMetadata(opHandle); - } - - @Override - public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, - long maxRows, FetchType fetchType) throws HiveSQLException { - return cliService.fetchResults(opHandle, orientation, maxRows, fetchType); - } - - - @Override - public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String owner, String renewer) throws HiveSQLException { - return cliService.getDelegationToken(sessionHandle, authFactory, owner, renewer); - } - - @Override - public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String tokenStr) throws HiveSQLException { - cliService.cancelDelegationToken(sessionHandle, authFactory, tokenStr); - } - - @Override - public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String tokenStr) throws HiveSQLException { - cliService.renewDelegationToken(sessionHandle, authFactory, tokenStr); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/FetchOrientation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/FetchOrientation.java deleted file mode 100644 index ffa6f2e1f3743..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/FetchOrientation.java +++ /dev/null @@ -1,54 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import org.apache.hive.service.cli.thrift.TFetchOrientation; - -/** - * FetchOrientation. - * - */ -public enum FetchOrientation { - FETCH_NEXT(TFetchOrientation.FETCH_NEXT), - FETCH_PRIOR(TFetchOrientation.FETCH_PRIOR), - FETCH_RELATIVE(TFetchOrientation.FETCH_RELATIVE), - FETCH_ABSOLUTE(TFetchOrientation.FETCH_ABSOLUTE), - FETCH_FIRST(TFetchOrientation.FETCH_FIRST), - FETCH_LAST(TFetchOrientation.FETCH_LAST); - - private TFetchOrientation tFetchOrientation; - - FetchOrientation(TFetchOrientation tFetchOrientation) { - this.tFetchOrientation = tFetchOrientation; - } - - public static FetchOrientation getFetchOrientation(TFetchOrientation tFetchOrientation) { - for (FetchOrientation fetchOrientation : values()) { - if (tFetchOrientation.equals(fetchOrientation.toTFetchOrientation())) { - return fetchOrientation; - } - } - // TODO: Should this really default to FETCH_NEXT? - return FETCH_NEXT; - } - - public TFetchOrientation toTFetchOrientation() { - return tFetchOrientation; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoType.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoType.java deleted file mode 100644 index 8dd33a88fdeb2..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoType.java +++ /dev/null @@ -1,96 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import org.apache.hive.service.cli.thrift.TGetInfoType; - -/** - * GetInfoType. - * - */ -public enum GetInfoType { - CLI_MAX_DRIVER_CONNECTIONS(TGetInfoType.CLI_MAX_DRIVER_CONNECTIONS), - CLI_MAX_CONCURRENT_ACTIVITIES(TGetInfoType.CLI_MAX_CONCURRENT_ACTIVITIES), - CLI_DATA_SOURCE_NAME(TGetInfoType.CLI_DATA_SOURCE_NAME), - CLI_FETCH_DIRECTION(TGetInfoType.CLI_FETCH_DIRECTION), - CLI_SERVER_NAME(TGetInfoType.CLI_SERVER_NAME), - CLI_SEARCH_PATTERN_ESCAPE(TGetInfoType.CLI_SEARCH_PATTERN_ESCAPE), - CLI_DBMS_NAME(TGetInfoType.CLI_DBMS_NAME), - CLI_DBMS_VER(TGetInfoType.CLI_DBMS_VER), - CLI_ACCESSIBLE_TABLES(TGetInfoType.CLI_ACCESSIBLE_TABLES), - CLI_ACCESSIBLE_PROCEDURES(TGetInfoType.CLI_ACCESSIBLE_PROCEDURES), - CLI_CURSOR_COMMIT_BEHAVIOR(TGetInfoType.CLI_CURSOR_COMMIT_BEHAVIOR), - CLI_DATA_SOURCE_READ_ONLY(TGetInfoType.CLI_DATA_SOURCE_READ_ONLY), - CLI_DEFAULT_TXN_ISOLATION(TGetInfoType.CLI_DEFAULT_TXN_ISOLATION), - CLI_IDENTIFIER_CASE(TGetInfoType.CLI_IDENTIFIER_CASE), - CLI_IDENTIFIER_QUOTE_CHAR(TGetInfoType.CLI_IDENTIFIER_QUOTE_CHAR), - CLI_MAX_COLUMN_NAME_LEN(TGetInfoType.CLI_MAX_COLUMN_NAME_LEN), - CLI_MAX_CURSOR_NAME_LEN(TGetInfoType.CLI_MAX_CURSOR_NAME_LEN), - CLI_MAX_SCHEMA_NAME_LEN(TGetInfoType.CLI_MAX_SCHEMA_NAME_LEN), - CLI_MAX_CATALOG_NAME_LEN(TGetInfoType.CLI_MAX_CATALOG_NAME_LEN), - CLI_MAX_TABLE_NAME_LEN(TGetInfoType.CLI_MAX_TABLE_NAME_LEN), - CLI_SCROLL_CONCURRENCY(TGetInfoType.CLI_SCROLL_CONCURRENCY), - CLI_TXN_CAPABLE(TGetInfoType.CLI_TXN_CAPABLE), - CLI_USER_NAME(TGetInfoType.CLI_USER_NAME), - CLI_TXN_ISOLATION_OPTION(TGetInfoType.CLI_TXN_ISOLATION_OPTION), - CLI_INTEGRITY(TGetInfoType.CLI_INTEGRITY), - CLI_GETDATA_EXTENSIONS(TGetInfoType.CLI_GETDATA_EXTENSIONS), - CLI_NULL_COLLATION(TGetInfoType.CLI_NULL_COLLATION), - CLI_ALTER_TABLE(TGetInfoType.CLI_ALTER_TABLE), - CLI_ORDER_BY_COLUMNS_IN_SELECT(TGetInfoType.CLI_ORDER_BY_COLUMNS_IN_SELECT), - CLI_SPECIAL_CHARACTERS(TGetInfoType.CLI_SPECIAL_CHARACTERS), - CLI_MAX_COLUMNS_IN_GROUP_BY(TGetInfoType.CLI_MAX_COLUMNS_IN_GROUP_BY), - CLI_MAX_COLUMNS_IN_INDEX(TGetInfoType.CLI_MAX_COLUMNS_IN_INDEX), - CLI_MAX_COLUMNS_IN_ORDER_BY(TGetInfoType.CLI_MAX_COLUMNS_IN_ORDER_BY), - CLI_MAX_COLUMNS_IN_SELECT(TGetInfoType.CLI_MAX_COLUMNS_IN_SELECT), - CLI_MAX_COLUMNS_IN_TABLE(TGetInfoType.CLI_MAX_COLUMNS_IN_TABLE), - CLI_MAX_INDEX_SIZE(TGetInfoType.CLI_MAX_INDEX_SIZE), - CLI_MAX_ROW_SIZE(TGetInfoType.CLI_MAX_ROW_SIZE), - CLI_MAX_STATEMENT_LEN(TGetInfoType.CLI_MAX_STATEMENT_LEN), - CLI_MAX_TABLES_IN_SELECT(TGetInfoType.CLI_MAX_TABLES_IN_SELECT), - CLI_MAX_USER_NAME_LEN(TGetInfoType.CLI_MAX_USER_NAME_LEN), - CLI_OJ_CAPABILITIES(TGetInfoType.CLI_OJ_CAPABILITIES), - - CLI_XOPEN_CLI_YEAR(TGetInfoType.CLI_XOPEN_CLI_YEAR), - CLI_CURSOR_SENSITIVITY(TGetInfoType.CLI_CURSOR_SENSITIVITY), - CLI_DESCRIBE_PARAMETER(TGetInfoType.CLI_DESCRIBE_PARAMETER), - CLI_CATALOG_NAME(TGetInfoType.CLI_CATALOG_NAME), - CLI_COLLATION_SEQ(TGetInfoType.CLI_COLLATION_SEQ), - CLI_MAX_IDENTIFIER_LEN(TGetInfoType.CLI_MAX_IDENTIFIER_LEN); - - private final TGetInfoType tInfoType; - - GetInfoType(TGetInfoType tInfoType) { - this.tInfoType = tInfoType; - } - - public static GetInfoType getGetInfoType(TGetInfoType tGetInfoType) { - for (GetInfoType infoType : values()) { - if (tGetInfoType.equals(infoType.tInfoType)) { - return infoType; - } - } - throw new IllegalArgumentException("Unrecognized Thrift TGetInfoType value: " + tGetInfoType); - } - - public TGetInfoType toTGetInfoType() { - return tInfoType; - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoValue.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoValue.java deleted file mode 100644 index ba92ff4ab5c11..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoValue.java +++ /dev/null @@ -1,82 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import org.apache.hive.service.cli.thrift.TGetInfoValue; - -/** - * GetInfoValue. - * - */ -public class GetInfoValue { - private String stringValue = null; - private short shortValue; - private int intValue; - private long longValue; - - public GetInfoValue(String stringValue) { - this.stringValue = stringValue; - } - - public GetInfoValue(short shortValue) { - this.shortValue = shortValue; - } - - public GetInfoValue(int intValue) { - this.intValue = intValue; - } - - public GetInfoValue(long longValue) { - this.longValue = longValue; - } - - public GetInfoValue(TGetInfoValue tGetInfoValue) { - switch (tGetInfoValue.getSetField()) { - case STRING_VALUE: - stringValue = tGetInfoValue.getStringValue(); - break; - default: - throw new IllegalArgumentException("Unreconigzed TGetInfoValue"); - } - } - - public TGetInfoValue toTGetInfoValue() { - TGetInfoValue tInfoValue = new TGetInfoValue(); - if (stringValue != null) { - tInfoValue.setStringValue(stringValue); - } - return tInfoValue; - } - - public String getStringValue() { - return stringValue; - } - - public short getShortValue() { - return shortValue; - } - - public int getIntValue() { - return intValue; - } - - public long getLongValue() { - return longValue; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Handle.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Handle.java deleted file mode 100644 index cf3427ae20f3c..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Handle.java +++ /dev/null @@ -1,78 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.service.cli; - -import org.apache.hive.service.cli.thrift.THandleIdentifier; - - - - -public abstract class Handle { - - private final HandleIdentifier handleId; - - public Handle() { - handleId = new HandleIdentifier(); - } - - public Handle(HandleIdentifier handleId) { - this.handleId = handleId; - } - - public Handle(THandleIdentifier tHandleIdentifier) { - this.handleId = new HandleIdentifier(tHandleIdentifier); - } - - public HandleIdentifier getHandleIdentifier() { - return handleId; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + ((handleId == null) ? 0 : handleId.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null) { - return false; - } - if (!(obj instanceof Handle)) { - return false; - } - Handle other = (Handle) obj; - if (handleId == null) { - if (other.handleId != null) { - return false; - } - } else if (!handleId.equals(other.handleId)) { - return false; - } - return true; - } - - @Override - public abstract String toString(); - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java deleted file mode 100644 index 4dc80da8dc500..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java +++ /dev/null @@ -1,113 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import java.nio.ByteBuffer; -import java.util.UUID; - -import org.apache.hive.service.cli.thrift.THandleIdentifier; - -/** - * HandleIdentifier. - * - */ -public class HandleIdentifier { - private final UUID publicId; - private final UUID secretId; - - public HandleIdentifier() { - publicId = UUID.randomUUID(); - secretId = UUID.randomUUID(); - } - - public HandleIdentifier(UUID publicId, UUID secretId) { - this.publicId = publicId; - this.secretId = secretId; - } - - public HandleIdentifier(THandleIdentifier tHandleId) { - ByteBuffer bb = ByteBuffer.wrap(tHandleId.getGuid()); - this.publicId = new UUID(bb.getLong(), bb.getLong()); - bb = ByteBuffer.wrap(tHandleId.getSecret()); - this.secretId = new UUID(bb.getLong(), bb.getLong()); - } - - public UUID getPublicId() { - return publicId; - } - - public UUID getSecretId() { - return secretId; - } - - public THandleIdentifier toTHandleIdentifier() { - byte[] guid = new byte[16]; - byte[] secret = new byte[16]; - ByteBuffer guidBB = ByteBuffer.wrap(guid); - ByteBuffer secretBB = ByteBuffer.wrap(secret); - guidBB.putLong(publicId.getMostSignificantBits()); - guidBB.putLong(publicId.getLeastSignificantBits()); - secretBB.putLong(secretId.getMostSignificantBits()); - secretBB.putLong(secretId.getLeastSignificantBits()); - return new THandleIdentifier(ByteBuffer.wrap(guid), ByteBuffer.wrap(secret)); - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + ((publicId == null) ? 0 : publicId.hashCode()); - result = prime * result + ((secretId == null) ? 0 : secretId.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null) { - return false; - } - if (!(obj instanceof HandleIdentifier)) { - return false; - } - HandleIdentifier other = (HandleIdentifier) obj; - if (publicId == null) { - if (other.publicId != null) { - return false; - } - } else if (!publicId.equals(other.publicId)) { - return false; - } - if (secretId == null) { - if (other.secretId != null) { - return false; - } - } else if (!secretId.equals(other.secretId)) { - return false; - } - return true; - } - - @Override - public String toString() { - return publicId.toString(); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HiveSQLException.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HiveSQLException.java deleted file mode 100644 index 86e57fbf31fe0..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HiveSQLException.java +++ /dev/null @@ -1,249 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.List; - -import org.apache.hive.service.cli.thrift.TStatus; -import org.apache.hive.service.cli.thrift.TStatusCode; - -/** - * HiveSQLException. - * - */ -public class HiveSQLException extends SQLException { - - /** - * - */ - private static final long serialVersionUID = -6095254671958748094L; - - /** - * - */ - public HiveSQLException() { - super(); - } - - /** - * @param reason - */ - public HiveSQLException(String reason) { - super(reason); - } - - /** - * @param cause - */ - public HiveSQLException(Throwable cause) { - super(cause); - } - - /** - * @param reason - * @param sqlState - */ - public HiveSQLException(String reason, String sqlState) { - super(reason, sqlState); - } - - /** - * @param reason - * @param cause - */ - public HiveSQLException(String reason, Throwable cause) { - super(reason, cause); - } - - /** - * @param reason - * @param sqlState - * @param vendorCode - */ - public HiveSQLException(String reason, String sqlState, int vendorCode) { - super(reason, sqlState, vendorCode); - } - - /** - * @param reason - * @param sqlState - * @param cause - */ - public HiveSQLException(String reason, String sqlState, Throwable cause) { - super(reason, sqlState, cause); - } - - /** - * @param reason - * @param sqlState - * @param vendorCode - * @param cause - */ - public HiveSQLException(String reason, String sqlState, int vendorCode, Throwable cause) { - super(reason, sqlState, vendorCode, cause); - } - - public HiveSQLException(TStatus status) { - // TODO: set correct vendorCode field - super(status.getErrorMessage(), status.getSqlState(), status.getErrorCode()); - if (status.getInfoMessages() != null) { - initCause(toCause(status.getInfoMessages())); - } - } - - /** - * Converts current object to a {@link TStatus} object - * @return a {@link TStatus} object - */ - public TStatus toTStatus() { - // TODO: convert sqlState, etc. - TStatus tStatus = new TStatus(TStatusCode.ERROR_STATUS); - tStatus.setSqlState(getSQLState()); - tStatus.setErrorCode(getErrorCode()); - tStatus.setErrorMessage(getMessage()); - tStatus.setInfoMessages(toString(this)); - return tStatus; - } - - /** - * Converts the specified {@link Exception} object into a {@link TStatus} object - * @param e a {@link Exception} object - * @return a {@link TStatus} object - */ - public static TStatus toTStatus(Exception e) { - if (e instanceof HiveSQLException) { - return ((HiveSQLException)e).toTStatus(); - } - TStatus tStatus = new TStatus(TStatusCode.ERROR_STATUS); - tStatus.setErrorMessage(e.getMessage()); - tStatus.setInfoMessages(toString(e)); - return tStatus; - } - - /** - * Converts a {@link Throwable} object into a flattened list of texts including its stack trace - * and the stack traces of the nested causes. - * @param ex a {@link Throwable} object - * @return a flattened list of texts including the {@link Throwable} object's stack trace - * and the stack traces of the nested causes. - */ - public static List toString(Throwable ex) { - return toString(ex, null); - } - - private static List toString(Throwable cause, StackTraceElement[] parent) { - StackTraceElement[] trace = cause.getStackTrace(); - int m = trace.length - 1; - if (parent != null) { - int n = parent.length - 1; - while (m >= 0 && n >= 0 && trace[m].equals(parent[n])) { - m--; - n--; - } - } - List detail = enroll(cause, trace, m); - cause = cause.getCause(); - if (cause != null) { - detail.addAll(toString(cause, trace)); - } - return detail; - } - - private static List enroll(Throwable ex, StackTraceElement[] trace, int max) { - List details = new ArrayList(); - StringBuilder builder = new StringBuilder(); - builder.append('*').append(ex.getClass().getName()).append(':'); - builder.append(ex.getMessage()).append(':'); - builder.append(trace.length).append(':').append(max); - details.add(builder.toString()); - for (int i = 0; i <= max; i++) { - builder.setLength(0); - builder.append(trace[i].getClassName()).append(':'); - builder.append(trace[i].getMethodName()).append(':'); - String fileName = trace[i].getFileName(); - builder.append(fileName == null ? "" : fileName).append(':'); - builder.append(trace[i].getLineNumber()); - details.add(builder.toString()); - } - return details; - } - - /** - * Converts a flattened list of texts including the stack trace and the stack - * traces of the nested causes into a {@link Throwable} object. - * @param details a flattened list of texts including the stack trace and the stack - * traces of the nested causes - * @return a {@link Throwable} object - */ - public static Throwable toCause(List details) { - return toStackTrace(details, null, 0); - } - - private static Throwable toStackTrace(List details, StackTraceElement[] parent, int index) { - String detail = details.get(index++); - if (!detail.startsWith("*")) { - return null; // should not be happened. ignore remaining - } - int i1 = detail.indexOf(':'); - int i3 = detail.lastIndexOf(':'); - int i2 = detail.substring(0, i3).lastIndexOf(':'); - String exceptionClass = detail.substring(1, i1); - String exceptionMessage = detail.substring(i1 + 1, i2); - Throwable ex = newInstance(exceptionClass, exceptionMessage); - - Integer length = Integer.valueOf(detail.substring(i2 + 1, i3)); - Integer unique = Integer.valueOf(detail.substring(i3 + 1)); - - int i = 0; - StackTraceElement[] trace = new StackTraceElement[length]; - for (; i <= unique; i++) { - detail = details.get(index++); - int j1 = detail.indexOf(':'); - int j3 = detail.lastIndexOf(':'); - int j2 = detail.substring(0, j3).lastIndexOf(':'); - String className = detail.substring(0, j1); - String methodName = detail.substring(j1 + 1, j2); - String fileName = detail.substring(j2 + 1, j3); - if (fileName.isEmpty()) { - fileName = null; - } - int lineNumber = Integer.valueOf(detail.substring(j3 + 1)); - trace[i] = new StackTraceElement(className, methodName, fileName, lineNumber); - } - int common = trace.length - i; - if (common > 0) { - System.arraycopy(parent, parent.length - common, trace, trace.length - common, common); - } - if (details.size() > index) { - ex.initCause(toStackTrace(details, trace, index)); - } - ex.setStackTrace(trace); - return ex; - } - - private static Throwable newInstance(String className, String message) { - try { - return (Throwable)Class.forName(className).getConstructor(String.class).newInstance(message); - } catch (Exception e) { - return new RuntimeException(className + ":" + message); - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ICLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ICLIService.java deleted file mode 100644 index c9cc1f4da56f1..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ICLIService.java +++ /dev/null @@ -1,105 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.service.cli; - -import java.util.List; -import java.util.Map; - - - - -import org.apache.hive.service.auth.HiveAuthFactory; - -public interface ICLIService { - - SessionHandle openSession(String username, String password, - Map configuration) - throws HiveSQLException; - - SessionHandle openSessionWithImpersonation(String username, String password, - Map configuration, String delegationToken) - throws HiveSQLException; - - void closeSession(SessionHandle sessionHandle) - throws HiveSQLException; - - GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType infoType) - throws HiveSQLException; - - OperationHandle executeStatement(SessionHandle sessionHandle, String statement, - Map confOverlay) - throws HiveSQLException; - - OperationHandle executeStatementAsync(SessionHandle sessionHandle, - String statement, Map confOverlay) - throws HiveSQLException; - - OperationHandle getTypeInfo(SessionHandle sessionHandle) - throws HiveSQLException; - - OperationHandle getCatalogs(SessionHandle sessionHandle) - throws HiveSQLException; - - OperationHandle getSchemas(SessionHandle sessionHandle, - String catalogName, String schemaName) - throws HiveSQLException; - - OperationHandle getTables(SessionHandle sessionHandle, - String catalogName, String schemaName, String tableName, List tableTypes) - throws HiveSQLException; - - OperationHandle getTableTypes(SessionHandle sessionHandle) - throws HiveSQLException; - - OperationHandle getColumns(SessionHandle sessionHandle, - String catalogName, String schemaName, String tableName, String columnName) - throws HiveSQLException; - - OperationHandle getFunctions(SessionHandle sessionHandle, - String catalogName, String schemaName, String functionName) - throws HiveSQLException; - - OperationStatus getOperationStatus(OperationHandle opHandle) - throws HiveSQLException; - - void cancelOperation(OperationHandle opHandle) - throws HiveSQLException; - - void closeOperation(OperationHandle opHandle) - throws HiveSQLException; - - TableSchema getResultSetMetadata(OperationHandle opHandle) - throws HiveSQLException; - - RowSet fetchResults(OperationHandle opHandle) - throws HiveSQLException; - - RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, - long maxRows, FetchType fetchType) throws HiveSQLException; - - String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String owner, String renewer) throws HiveSQLException; - - void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String tokenStr) throws HiveSQLException; - - void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String tokenStr) throws HiveSQLException; - - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationHandle.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationHandle.java deleted file mode 100644 index 5426e28471239..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationHandle.java +++ /dev/null @@ -1,102 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.service.cli; - -import org.apache.hive.service.cli.thrift.TOperationHandle; -import org.apache.hive.service.cli.thrift.TProtocolVersion; - -public class OperationHandle extends Handle { - - private final OperationType opType; - private final TProtocolVersion protocol; - private boolean hasResultSet = false; - - public OperationHandle(OperationType opType, TProtocolVersion protocol) { - super(); - this.opType = opType; - this.protocol = protocol; - } - - // dummy handle for ThriftCLIService - public OperationHandle(TOperationHandle tOperationHandle) { - this(tOperationHandle, TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V1); - } - - public OperationHandle(TOperationHandle tOperationHandle, TProtocolVersion protocol) { - super(tOperationHandle.getOperationId()); - this.opType = OperationType.getOperationType(tOperationHandle.getOperationType()); - this.hasResultSet = tOperationHandle.isHasResultSet(); - this.protocol = protocol; - } - - public OperationType getOperationType() { - return opType; - } - - public void setHasResultSet(boolean hasResultSet) { - this.hasResultSet = hasResultSet; - } - - public boolean hasResultSet() { - return hasResultSet; - } - - public TOperationHandle toTOperationHandle() { - TOperationHandle tOperationHandle = new TOperationHandle(); - tOperationHandle.setOperationId(getHandleIdentifier().toTHandleIdentifier()); - tOperationHandle.setOperationType(opType.toTOperationType()); - tOperationHandle.setHasResultSet(hasResultSet); - return tOperationHandle; - } - - public TProtocolVersion getProtocolVersion() { - return protocol; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = super.hashCode(); - result = prime * result + ((opType == null) ? 0 : opType.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (!super.equals(obj)) { - return false; - } - if (!(obj instanceof OperationHandle)) { - return false; - } - OperationHandle other = (OperationHandle) obj; - if (opType != other.opType) { - return false; - } - return true; - } - - @Override - public String toString() { - return "OperationHandle [opType=" + opType + ", getHandleIdentifier()=" + getHandleIdentifier() - + "]"; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationState.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationState.java deleted file mode 100644 index 1165180118413..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationState.java +++ /dev/null @@ -1,108 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import org.apache.hive.service.cli.thrift.TOperationState; - -/** - * OperationState. - * - */ -public enum OperationState { - INITIALIZED(TOperationState.INITIALIZED_STATE, false), - RUNNING(TOperationState.RUNNING_STATE, false), - FINISHED(TOperationState.FINISHED_STATE, true), - CANCELED(TOperationState.CANCELED_STATE, true), - CLOSED(TOperationState.CLOSED_STATE, true), - ERROR(TOperationState.ERROR_STATE, true), - UNKNOWN(TOperationState.UKNOWN_STATE, false), - PENDING(TOperationState.PENDING_STATE, false); - - private final TOperationState tOperationState; - private final boolean terminal; - - OperationState(TOperationState tOperationState, boolean terminal) { - this.tOperationState = tOperationState; - this.terminal = terminal; - } - - // must be sync with TOperationState in order - public static OperationState getOperationState(TOperationState tOperationState) { - return OperationState.values()[tOperationState.getValue()]; - } - - public static void validateTransition(OperationState oldState, - OperationState newState) - throws HiveSQLException { - switch (oldState) { - case INITIALIZED: - switch (newState) { - case PENDING: - case RUNNING: - case CANCELED: - case CLOSED: - return; - } - break; - case PENDING: - switch (newState) { - case RUNNING: - case FINISHED: - case CANCELED: - case ERROR: - case CLOSED: - return; - } - break; - case RUNNING: - switch (newState) { - case FINISHED: - case CANCELED: - case ERROR: - case CLOSED: - return; - } - break; - case FINISHED: - case CANCELED: - case ERROR: - if (OperationState.CLOSED.equals(newState)) { - return; - } - break; - default: - // fall-through - } - throw new HiveSQLException("Illegal Operation state transition " + - "from " + oldState + " to " + newState); - } - - public void validateTransition(OperationState newState) - throws HiveSQLException { - validateTransition(this, newState); - } - - public TOperationState toTOperationState() { - return tOperationState; - } - - public boolean isTerminal() { - return terminal; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationType.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationType.java deleted file mode 100644 index 429d9a4c25688..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationType.java +++ /dev/null @@ -1,58 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import org.apache.hive.service.cli.thrift.TOperationType; - -/** - * OperationType. - * - */ -public enum OperationType { - - UNKNOWN_OPERATION(TOperationType.UNKNOWN), - EXECUTE_STATEMENT(TOperationType.EXECUTE_STATEMENT), - GET_TYPE_INFO(TOperationType.GET_TYPE_INFO), - GET_CATALOGS(TOperationType.GET_CATALOGS), - GET_SCHEMAS(TOperationType.GET_SCHEMAS), - GET_TABLES(TOperationType.GET_TABLES), - GET_TABLE_TYPES(TOperationType.GET_TABLE_TYPES), - GET_COLUMNS(TOperationType.GET_COLUMNS), - GET_FUNCTIONS(TOperationType.GET_FUNCTIONS); - - private TOperationType tOperationType; - - OperationType(TOperationType tOpType) { - this.tOperationType = tOpType; - } - - public static OperationType getOperationType(TOperationType tOperationType) { - // TODO: replace this with a Map? - for (OperationType opType : values()) { - if (tOperationType.equals(opType.tOperationType)) { - return opType; - } - } - return OperationType.UNKNOWN_OPERATION; - } - - public TOperationType toTOperationType() { - return tOperationType; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java deleted file mode 100644 index 6e4d43fd5df63..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java +++ /dev/null @@ -1,47 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -/** - * PatternOrIdentifier. - * - */ -public class PatternOrIdentifier { - - boolean isPattern = false; - String text; - - public PatternOrIdentifier(String tpoi) { - text = tpoi; - isPattern = false; - } - - public boolean isPattern() { - return isPattern; - } - - public boolean isIdentifier() { - return !isPattern; - } - - @Override - public String toString() { - return text; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowBasedSet.java deleted file mode 100644 index 7452137f077db..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowBasedSet.java +++ /dev/null @@ -1,140 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - -import org.apache.hive.service.cli.thrift.TColumnValue; -import org.apache.hive.service.cli.thrift.TRow; -import org.apache.hive.service.cli.thrift.TRowSet; - -/** - * RowBasedSet - */ -public class RowBasedSet implements RowSet { - - private long startOffset; - - private final Type[] types; // non-null only for writing (server-side) - private final RemovableList rows; - - public RowBasedSet(TableSchema schema) { - types = schema.toTypes(); - rows = new RemovableList(); - } - - public RowBasedSet(TRowSet tRowSet) { - types = null; - rows = new RemovableList(tRowSet.getRows()); - startOffset = tRowSet.getStartRowOffset(); - } - - private RowBasedSet(Type[] types, List rows, long startOffset) { - this.types = types; - this.rows = new RemovableList(rows); - this.startOffset = startOffset; - } - - @Override - public RowBasedSet addRow(Object[] fields) { - TRow tRow = new TRow(); - for (int i = 0; i < fields.length; i++) { - tRow.addToColVals(ColumnValue.toTColumnValue(types[i], fields[i])); - } - rows.add(tRow); - return this; - } - - @Override - public int numColumns() { - return rows.isEmpty() ? 0 : rows.get(0).getColVals().size(); - } - - @Override - public int numRows() { - return rows.size(); - } - - public RowBasedSet extractSubset(int maxRows) { - int numRows = Math.min(numRows(), maxRows); - RowBasedSet result = new RowBasedSet(types, rows.subList(0, numRows), startOffset); - rows.removeRange(0, numRows); - startOffset += numRows; - return result; - } - - public long getStartOffset() { - return startOffset; - } - - public void setStartOffset(long startOffset) { - this.startOffset = startOffset; - } - - public int getSize() { - return rows.size(); - } - - public TRowSet toTRowSet() { - TRowSet tRowSet = new TRowSet(); - tRowSet.setStartRowOffset(startOffset); - tRowSet.setRows(new ArrayList(rows)); - return tRowSet; - } - - @Override - public Iterator iterator() { - return new Iterator() { - - final Iterator iterator = rows.iterator(); - final Object[] convey = new Object[numColumns()]; - - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public Object[] next() { - TRow row = iterator.next(); - List values = row.getColVals(); - for (int i = 0; i < values.size(); i++) { - convey[i] = ColumnValue.toColumnValue(values.get(i)); - } - return convey; - } - - @Override - public void remove() { - throw new UnsupportedOperationException("remove"); - } - }; - } - - private static class RemovableList extends ArrayList { - RemovableList() { super(); } - RemovableList(List rows) { super(rows); } - @Override - public void removeRange(int fromIndex, int toIndex) { - super.removeRange(fromIndex, toIndex); - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSet.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSet.java deleted file mode 100644 index ab0787e1d389e..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSet.java +++ /dev/null @@ -1,38 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import org.apache.hive.service.cli.thrift.TRowSet; - -public interface RowSet extends Iterable { - - RowSet addRow(Object[] fields); - - RowSet extractSubset(int maxRows); - - int numColumns(); - - int numRows(); - - long getStartOffset(); - - void setStartOffset(long startOffset); - - TRowSet toTRowSet(); -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSetFactory.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSetFactory.java deleted file mode 100644 index e8f68eaaf9063..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSetFactory.java +++ /dev/null @@ -1,41 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import org.apache.hive.service.cli.thrift.TProtocolVersion; -import org.apache.hive.service.cli.thrift.TRowSet; - -import static org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V6; - -public class RowSetFactory { - - public static RowSet create(TableSchema schema, TProtocolVersion version) { - if (version.getValue() >= HIVE_CLI_SERVICE_PROTOCOL_V6.getValue()) { - return new ColumnBasedSet(schema); - } - return new RowBasedSet(schema); - } - - public static RowSet create(TRowSet results, TProtocolVersion version) { - if (version.getValue() >= HIVE_CLI_SERVICE_PROTOCOL_V6.getValue()) { - return new ColumnBasedSet(results); - } - return new RowBasedSet(results); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/SessionHandle.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/SessionHandle.java deleted file mode 100644 index 52e0ad4834d8b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/SessionHandle.java +++ /dev/null @@ -1,67 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import java.util.UUID; - -import org.apache.hive.service.cli.thrift.TProtocolVersion; -import org.apache.hive.service.cli.thrift.TSessionHandle; - - -/** - * SessionHandle. - * - */ -public class SessionHandle extends Handle { - - private final TProtocolVersion protocol; - - public SessionHandle(TProtocolVersion protocol) { - this.protocol = protocol; - } - - // dummy handle for ThriftCLIService - public SessionHandle(TSessionHandle tSessionHandle) { - this(tSessionHandle, TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V1); - } - - public SessionHandle(TSessionHandle tSessionHandle, TProtocolVersion protocol) { - super(tSessionHandle.getSessionId()); - this.protocol = protocol; - } - - public UUID getSessionId() { - return getHandleIdentifier().getPublicId(); - } - - public TSessionHandle toTSessionHandle() { - TSessionHandle tSessionHandle = new TSessionHandle(); - tSessionHandle.setSessionId(getHandleIdentifier().toTHandleIdentifier()); - return tSessionHandle; - } - - public TProtocolVersion getProtocolVersion() { - return protocol; - } - - @Override - public String toString() { - return "SessionHandle [" + getHandleIdentifier() + "]"; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TableSchema.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TableSchema.java deleted file mode 100644 index ee019bc737101..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TableSchema.java +++ /dev/null @@ -1,102 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.Schema; -import org.apache.hive.service.cli.thrift.TColumnDesc; -import org.apache.hive.service.cli.thrift.TTableSchema; - -/** - * TableSchema. - * - */ -public class TableSchema { - private final List columns = new ArrayList(); - - public TableSchema() { - } - - public TableSchema(int numColumns) { - // TODO: remove this constructor - } - - public TableSchema(TTableSchema tTableSchema) { - for (TColumnDesc tColumnDesc : tTableSchema.getColumns()) { - columns.add(new ColumnDescriptor(tColumnDesc)); - } - } - - public TableSchema(List fieldSchemas) { - int pos = 1; - for (FieldSchema field : fieldSchemas) { - columns.add(new ColumnDescriptor(field, pos++)); - } - } - - public TableSchema(Schema schema) { - this(schema.getFieldSchemas()); - } - - public List getColumnDescriptors() { - return new ArrayList(columns); - } - - public ColumnDescriptor getColumnDescriptorAt(int pos) { - return columns.get(pos); - } - - public int getSize() { - return columns.size(); - } - - public void clear() { - columns.clear(); - } - - - public TTableSchema toTTableSchema() { - TTableSchema tTableSchema = new TTableSchema(); - for (ColumnDescriptor col : columns) { - tTableSchema.addToColumns(col.toTColumnDesc()); - } - return tTableSchema; - } - - public Type[] toTypes() { - Type[] types = new Type[columns.size()]; - for (int i = 0; i < types.length; i++) { - types[i] = columns.get(i).getType(); - } - return types; - } - - public TableSchema addPrimitiveColumn(String columnName, Type columnType, String columnComment) { - columns.add(ColumnDescriptor.newPrimitiveColumnDescriptor(columnName, columnComment, columnType, columns.size() + 1)); - return this; - } - - public TableSchema addStringColumn(String columnName, String columnComment) { - columns.add(ColumnDescriptor.newPrimitiveColumnDescriptor(columnName, columnComment, Type.STRING_TYPE, columns.size() + 1)); - return this; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Type.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Type.java deleted file mode 100644 index 7752ec03a29b7..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Type.java +++ /dev/null @@ -1,349 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import java.sql.DatabaseMetaData; -import java.util.Locale; - -import org.apache.hadoop.hive.common.type.HiveDecimal; -import org.apache.hive.service.cli.thrift.TTypeId; - -/** - * Type. - * - */ -public enum Type { - NULL_TYPE("VOID", - java.sql.Types.NULL, - TTypeId.NULL_TYPE), - BOOLEAN_TYPE("BOOLEAN", - java.sql.Types.BOOLEAN, - TTypeId.BOOLEAN_TYPE), - TINYINT_TYPE("TINYINT", - java.sql.Types.TINYINT, - TTypeId.TINYINT_TYPE), - SMALLINT_TYPE("SMALLINT", - java.sql.Types.SMALLINT, - TTypeId.SMALLINT_TYPE), - INT_TYPE("INT", - java.sql.Types.INTEGER, - TTypeId.INT_TYPE), - BIGINT_TYPE("BIGINT", - java.sql.Types.BIGINT, - TTypeId.BIGINT_TYPE), - FLOAT_TYPE("FLOAT", - java.sql.Types.FLOAT, - TTypeId.FLOAT_TYPE), - DOUBLE_TYPE("DOUBLE", - java.sql.Types.DOUBLE, - TTypeId.DOUBLE_TYPE), - STRING_TYPE("STRING", - java.sql.Types.VARCHAR, - TTypeId.STRING_TYPE), - CHAR_TYPE("CHAR", - java.sql.Types.CHAR, - TTypeId.CHAR_TYPE, - true, false, false), - VARCHAR_TYPE("VARCHAR", - java.sql.Types.VARCHAR, - TTypeId.VARCHAR_TYPE, - true, false, false), - DATE_TYPE("DATE", - java.sql.Types.DATE, - TTypeId.DATE_TYPE), - TIMESTAMP_TYPE("TIMESTAMP", - java.sql.Types.TIMESTAMP, - TTypeId.TIMESTAMP_TYPE), - INTERVAL_YEAR_MONTH_TYPE("INTERVAL_YEAR_MONTH", - java.sql.Types.OTHER, - TTypeId.INTERVAL_YEAR_MONTH_TYPE), - INTERVAL_DAY_TIME_TYPE("INTERVAL_DAY_TIME", - java.sql.Types.OTHER, - TTypeId.INTERVAL_DAY_TIME_TYPE), - BINARY_TYPE("BINARY", - java.sql.Types.BINARY, - TTypeId.BINARY_TYPE), - DECIMAL_TYPE("DECIMAL", - java.sql.Types.DECIMAL, - TTypeId.DECIMAL_TYPE, - true, false, false), - ARRAY_TYPE("ARRAY", - java.sql.Types.ARRAY, - TTypeId.ARRAY_TYPE, - true, true), - MAP_TYPE("MAP", - java.sql.Types.JAVA_OBJECT, - TTypeId.MAP_TYPE, - true, true), - STRUCT_TYPE("STRUCT", - java.sql.Types.STRUCT, - TTypeId.STRUCT_TYPE, - true, false), - UNION_TYPE("UNIONTYPE", - java.sql.Types.OTHER, - TTypeId.UNION_TYPE, - true, false), - USER_DEFINED_TYPE("USER_DEFINED", - java.sql.Types.OTHER, - TTypeId.USER_DEFINED_TYPE, - true, false); - - private final String name; - private final TTypeId tType; - private final int javaSQLType; - private final boolean isQualified; - private final boolean isComplex; - private final boolean isCollection; - - Type(String name, int javaSQLType, TTypeId tType, boolean isQualified, boolean isComplex, boolean isCollection) { - this.name = name; - this.javaSQLType = javaSQLType; - this.tType = tType; - this.isQualified = isQualified; - this.isComplex = isComplex; - this.isCollection = isCollection; - } - - Type(String name, int javaSQLType, TTypeId tType, boolean isComplex, boolean isCollection) { - this(name, javaSQLType, tType, false, isComplex, isCollection); - } - - Type(String name, int javaSqlType, TTypeId tType) { - this(name, javaSqlType, tType, false, false, false); - } - - public boolean isPrimitiveType() { - return !isComplex; - } - - public boolean isQualifiedType() { - return isQualified; - } - - public boolean isComplexType() { - return isComplex; - } - - public boolean isCollectionType() { - return isCollection; - } - - public static Type getType(TTypeId tType) { - for (Type type : values()) { - if (tType.equals(type.tType)) { - return type; - } - } - throw new IllegalArgumentException("Unregonized Thrift TTypeId value: " + tType); - } - - public static Type getType(String name) { - if (name == null) { - throw new IllegalArgumentException("Invalid type name: null"); - } - for (Type type : values()) { - if (name.equalsIgnoreCase(type.name)) { - return type; - } else if (type.isQualifiedType() || type.isComplexType()) { - if (name.toUpperCase(Locale.ROOT).startsWith(type.name)) { - return type; - } - } - } - throw new IllegalArgumentException("Unrecognized type name: " + name); - } - - /** - * Radix for this type (typically either 2 or 10) - * Null is returned for data types where this is not applicable. - */ - public Integer getNumPrecRadix() { - if (this.isNumericType()) { - return 10; - } - return null; - } - - /** - * Maximum precision for numeric types. - * Returns null for non-numeric types. - * @return - */ - public Integer getMaxPrecision() { - switch (this) { - case TINYINT_TYPE: - return 3; - case SMALLINT_TYPE: - return 5; - case INT_TYPE: - return 10; - case BIGINT_TYPE: - return 19; - case FLOAT_TYPE: - return 7; - case DOUBLE_TYPE: - return 15; - case DECIMAL_TYPE: - return HiveDecimal.MAX_PRECISION; - default: - return null; - } - } - - public boolean isNumericType() { - switch (this) { - case TINYINT_TYPE: - case SMALLINT_TYPE: - case INT_TYPE: - case BIGINT_TYPE: - case FLOAT_TYPE: - case DOUBLE_TYPE: - case DECIMAL_TYPE: - return true; - default: - return false; - } - } - - /** - * Prefix used to quote a literal of this type (may be null) - */ - public String getLiteralPrefix() { - return null; - } - - /** - * Suffix used to quote a literal of this type (may be null) - * @return - */ - public String getLiteralSuffix() { - return null; - } - - /** - * Can you use NULL for this type? - * @return - * DatabaseMetaData.typeNoNulls - does not allow NULL values - * DatabaseMetaData.typeNullable - allows NULL values - * DatabaseMetaData.typeNullableUnknown - nullability unknown - */ - public Short getNullable() { - // All Hive types are nullable - return DatabaseMetaData.typeNullable; - } - - /** - * Is the type case sensitive? - * @return - */ - public Boolean isCaseSensitive() { - switch (this) { - case STRING_TYPE: - return true; - default: - return false; - } - } - - /** - * Parameters used in creating the type (may be null) - * @return - */ - public String getCreateParams() { - return null; - } - - /** - * Can you use WHERE based on this type? - * @return - * DatabaseMetaData.typePredNone - No support - * DatabaseMetaData.typePredChar - Only support with WHERE .. LIKE - * DatabaseMetaData.typePredBasic - Supported except for WHERE .. LIKE - * DatabaseMetaData.typeSearchable - Supported for all WHERE .. - */ - public Short getSearchable() { - if (isPrimitiveType()) { - return DatabaseMetaData.typeSearchable; - } - return DatabaseMetaData.typePredNone; - } - - /** - * Is this type unsigned? - * @return - */ - public Boolean isUnsignedAttribute() { - if (isNumericType()) { - return false; - } - return true; - } - - /** - * Can this type represent money? - * @return - */ - public Boolean isFixedPrecScale() { - return false; - } - - /** - * Can this type be used for an auto-increment value? - * @return - */ - public Boolean isAutoIncrement() { - return false; - } - - /** - * Localized version of type name (may be null). - * @return - */ - public String getLocalizedName() { - return null; - } - - /** - * Minimum scale supported for this type - * @return - */ - public Short getMinimumScale() { - return 0; - } - - /** - * Maximum scale supported for this type - * @return - */ - public Short getMaximumScale() { - return 0; - } - - public TTypeId toTType() { - return tType; - } - - public int toJavaSQLType() { - return javaSQLType; - } - - public String getName() { - return name; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java deleted file mode 100644 index b80fd67884add..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java +++ /dev/null @@ -1,159 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import java.util.List; - -import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.apache.hive.service.cli.thrift.TPrimitiveTypeEntry; -import org.apache.hive.service.cli.thrift.TTypeDesc; -import org.apache.hive.service.cli.thrift.TTypeEntry; - -/** - * TypeDescriptor. - * - */ -public class TypeDescriptor { - - private final Type type; - private String typeName = null; - private TypeQualifiers typeQualifiers = null; - - public TypeDescriptor(Type type) { - this.type = type; - } - - public TypeDescriptor(TTypeDesc tTypeDesc) { - List tTypeEntries = tTypeDesc.getTypes(); - TPrimitiveTypeEntry top = tTypeEntries.get(0).getPrimitiveEntry(); - this.type = Type.getType(top.getType()); - if (top.isSetTypeQualifiers()) { - setTypeQualifiers(TypeQualifiers.fromTTypeQualifiers(top.getTypeQualifiers())); - } - } - - public TypeDescriptor(String typeName) { - this.type = Type.getType(typeName); - if (this.type.isComplexType()) { - this.typeName = typeName; - } else if (this.type.isQualifiedType()) { - PrimitiveTypeInfo pti = TypeInfoFactory.getPrimitiveTypeInfo(typeName); - setTypeQualifiers(TypeQualifiers.fromTypeInfo(pti)); - } - } - - public Type getType() { - return type; - } - - public TTypeDesc toTTypeDesc() { - TPrimitiveTypeEntry primitiveEntry = new TPrimitiveTypeEntry(type.toTType()); - if (getTypeQualifiers() != null) { - primitiveEntry.setTypeQualifiers(getTypeQualifiers().toTTypeQualifiers()); - } - TTypeEntry entry = TTypeEntry.primitiveEntry(primitiveEntry); - - TTypeDesc desc = new TTypeDesc(); - desc.addToTypes(entry); - return desc; - } - - public String getTypeName() { - if (typeName != null) { - return typeName; - } else { - return type.getName(); - } - } - - public TypeQualifiers getTypeQualifiers() { - return typeQualifiers; - } - - public void setTypeQualifiers(TypeQualifiers typeQualifiers) { - this.typeQualifiers = typeQualifiers; - } - - /** - * The column size for this type. - * For numeric data this is the maximum precision. - * For character data this is the length in characters. - * For datetime types this is the length in characters of the String representation - * (assuming the maximum allowed precision of the fractional seconds component). - * For binary data this is the length in bytes. - * Null is returned for data types where the column size is not applicable. - */ - public Integer getColumnSize() { - if (type.isNumericType()) { - return getPrecision(); - } - switch (type) { - case STRING_TYPE: - case BINARY_TYPE: - return Integer.MAX_VALUE; - case CHAR_TYPE: - case VARCHAR_TYPE: - return typeQualifiers.getCharacterMaximumLength(); - case DATE_TYPE: - return 10; - case TIMESTAMP_TYPE: - return 29; - default: - return null; - } - } - - /** - * Maximum precision for numeric types. - * Returns null for non-numeric types. - * @return - */ - public Integer getPrecision() { - if (this.type == Type.DECIMAL_TYPE) { - return typeQualifiers.getPrecision(); - } - return this.type.getMaxPrecision(); - } - - /** - * The number of fractional digits for this type. - * Null is returned for data types where this is not applicable. - */ - public Integer getDecimalDigits() { - switch (this.type) { - case BOOLEAN_TYPE: - case TINYINT_TYPE: - case SMALLINT_TYPE: - case INT_TYPE: - case BIGINT_TYPE: - return 0; - case FLOAT_TYPE: - return 7; - case DOUBLE_TYPE: - return 15; - case DECIMAL_TYPE: - return typeQualifiers.getScale(); - case TIMESTAMP_TYPE: - return 9; - default: - return null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java deleted file mode 100644 index c6da52c15a2b5..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java +++ /dev/null @@ -1,133 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli; - -import java.util.HashMap; -import java.util.Map; - -import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; -import org.apache.hive.service.cli.thrift.TCLIServiceConstants; -import org.apache.hive.service.cli.thrift.TTypeQualifierValue; -import org.apache.hive.service.cli.thrift.TTypeQualifiers; - -/** - * This class holds type qualifier information for a primitive type, - * such as char/varchar length or decimal precision/scale. - */ -public class TypeQualifiers { - private Integer characterMaximumLength; - private Integer precision; - private Integer scale; - - public TypeQualifiers() {} - - public Integer getCharacterMaximumLength() { - return characterMaximumLength; - } - public void setCharacterMaximumLength(int characterMaximumLength) { - this.characterMaximumLength = characterMaximumLength; - } - - public TTypeQualifiers toTTypeQualifiers() { - TTypeQualifiers ret = null; - - Map qMap = new HashMap(); - if (getCharacterMaximumLength() != null) { - TTypeQualifierValue val = new TTypeQualifierValue(); - val.setI32Value(getCharacterMaximumLength().intValue()); - qMap.put(TCLIServiceConstants.CHARACTER_MAXIMUM_LENGTH, val); - } - - if (precision != null) { - TTypeQualifierValue val = new TTypeQualifierValue(); - val.setI32Value(precision.intValue()); - qMap.put(TCLIServiceConstants.PRECISION, val); - } - - if (scale != null) { - TTypeQualifierValue val = new TTypeQualifierValue(); - val.setI32Value(scale.intValue()); - qMap.put(TCLIServiceConstants.SCALE, val); - } - - if (qMap.size() > 0) { - ret = new TTypeQualifiers(qMap); - } - - return ret; - } - - public static TypeQualifiers fromTTypeQualifiers(TTypeQualifiers ttq) { - TypeQualifiers ret = null; - if (ttq != null) { - ret = new TypeQualifiers(); - Map tqMap = ttq.getQualifiers(); - - if (tqMap.containsKey(TCLIServiceConstants.CHARACTER_MAXIMUM_LENGTH)) { - ret.setCharacterMaximumLength( - tqMap.get(TCLIServiceConstants.CHARACTER_MAXIMUM_LENGTH).getI32Value()); - } - - if (tqMap.containsKey(TCLIServiceConstants.PRECISION)) { - ret.setPrecision(tqMap.get(TCLIServiceConstants.PRECISION).getI32Value()); - } - - if (tqMap.containsKey(TCLIServiceConstants.SCALE)) { - ret.setScale(tqMap.get(TCLIServiceConstants.SCALE).getI32Value()); - } - } - return ret; - } - - public static TypeQualifiers fromTypeInfo(PrimitiveTypeInfo pti) { - TypeQualifiers result = null; - if (pti instanceof VarcharTypeInfo) { - result = new TypeQualifiers(); - result.setCharacterMaximumLength(((VarcharTypeInfo)pti).getLength()); - } else if (pti instanceof CharTypeInfo) { - result = new TypeQualifiers(); - result.setCharacterMaximumLength(((CharTypeInfo)pti).getLength()); - } else if (pti instanceof DecimalTypeInfo) { - result = new TypeQualifiers(); - result.setPrecision(((DecimalTypeInfo)pti).precision()); - result.setScale(((DecimalTypeInfo)pti).scale()); - } - return result; - } - - public Integer getPrecision() { - return precision; - } - - public void setPrecision(Integer precision) { - this.precision = precision; - } - - public Integer getScale() { - return scale; - } - - public void setScale(Integer scale) { - this.scale = scale; - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java deleted file mode 100644 index af36057bdaeca..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java +++ /dev/null @@ -1,86 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.operation; - -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -import org.apache.hadoop.hive.metastore.TableType; - -/** - * ClassicTableTypeMapping. - * Classic table type mapping : - * Managed Table to Table - * External Table to Table - * Virtual View to View - */ -public class ClassicTableTypeMapping implements TableTypeMapping { - - public enum ClassicTableTypes { - TABLE, - VIEW, - } - - private final Map hiveToClientMap = new HashMap(); - private final Map clientToHiveMap = new HashMap(); - - public ClassicTableTypeMapping() { - hiveToClientMap.put(TableType.MANAGED_TABLE.toString(), - ClassicTableTypes.TABLE.toString()); - hiveToClientMap.put(TableType.EXTERNAL_TABLE.toString(), - ClassicTableTypes.TABLE.toString()); - hiveToClientMap.put(TableType.VIRTUAL_VIEW.toString(), - ClassicTableTypes.VIEW.toString()); - - clientToHiveMap.put(ClassicTableTypes.TABLE.toString(), - TableType.MANAGED_TABLE.toString()); - clientToHiveMap.put(ClassicTableTypes.VIEW.toString(), - TableType.VIRTUAL_VIEW.toString()); - } - - @Override - public String mapToHiveType(String clientTypeName) { - if (clientToHiveMap.containsKey(clientTypeName)) { - return clientToHiveMap.get(clientTypeName); - } else { - return clientTypeName; - } - } - - @Override - public String mapToClientType(String hiveTypeName) { - if (hiveToClientMap.containsKey(hiveTypeName)) { - return hiveToClientMap.get(hiveTypeName); - } else { - return hiveTypeName; - } - } - - @Override - public Set getTableTypeNames() { - Set typeNameSet = new HashSet(); - for (ClassicTableTypes typeNames : ClassicTableTypes.values()) { - typeNameSet.add(typeNames.toString()); - } - return typeNameSet; - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java deleted file mode 100644 index 6740d3bb59dc3..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java +++ /dev/null @@ -1,83 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.service.cli.operation; - -import java.sql.SQLException; -import java.util.HashMap; -import java.util.Map; - -import org.apache.hadoop.hive.ql.processors.CommandProcessor; -import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory; -import org.apache.hadoop.hive.ql.session.OperationLog; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.session.HiveSession; - -public abstract class ExecuteStatementOperation extends Operation { - protected String statement = null; - protected Map confOverlay = new HashMap(); - - public ExecuteStatementOperation(HiveSession parentSession, String statement, - Map confOverlay, boolean runInBackground) { - super(parentSession, OperationType.EXECUTE_STATEMENT, runInBackground); - this.statement = statement; - setConfOverlay(confOverlay); - } - - public String getStatement() { - return statement; - } - - public static ExecuteStatementOperation newExecuteStatementOperation( - HiveSession parentSession, String statement, Map confOverlay, boolean runAsync) - throws HiveSQLException { - String[] tokens = statement.trim().split("\\s+"); - CommandProcessor processor = null; - try { - processor = CommandProcessorFactory.getForHiveCommand(tokens, parentSession.getHiveConf()); - } catch (SQLException e) { - throw new HiveSQLException(e.getMessage(), e.getSQLState(), e); - } - if (processor == null) { - return new SQLOperation(parentSession, statement, confOverlay, runAsync); - } - return new HiveCommandOperation(parentSession, statement, processor, confOverlay); - } - - protected Map getConfOverlay() { - return confOverlay; - } - - protected void setConfOverlay(Map confOverlay) { - if (confOverlay != null) { - this.confOverlay = confOverlay; - } - } - - protected void registerCurrentOperationLog() { - if (isOperationLogEnabled) { - if (operationLog == null) { - LOG.warn("Failed to get current OperationLog object of Operation: " + - getHandle().getHandleIdentifier()); - isOperationLogEnabled = false; - return; - } - OperationLog.setCurrentOperationLog(operationLog); - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java deleted file mode 100644 index 581d975344060..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java +++ /dev/null @@ -1,81 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.operation; - -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; - -/** - * GetCatalogsOperation. - * - */ -public class GetCatalogsOperation extends MetadataOperation { - private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() - .addStringColumn("TABLE_CAT", "Catalog name. NULL if not applicable."); - - protected final RowSet rowSet; - - protected GetCatalogsOperation(HiveSession parentSession) { - super(parentSession, OperationType.GET_CATALOGS); - rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.RUNNING); - try { - if (isAuthV2Enabled()) { - authorizeMetaGets(HiveOperationType.GET_CATALOGS, null); - } - setState(OperationState.FINISHED); - } catch (HiveSQLException e) { - setState(OperationState.ERROR); - throw e; - } - - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getResultSetSchema() - */ - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - return RESULT_SET_SCHEMA; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) - */ - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - assertState(OperationState.FINISHED); - validateDefaultFetchOrientation(orientation); - if (orientation.equals(FetchOrientation.FETCH_FIRST)) { - rowSet.setStartOffset(0); - } - return rowSet.extractSubset((int)maxRows); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java deleted file mode 100644 index 96ba4890075ac..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java +++ /dev/null @@ -1,234 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.operation; - -import java.sql.DatabaseMetaData; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.regex.Pattern; - -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType; -import org.apache.hive.service.cli.ColumnDescriptor; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.Type; -import org.apache.hive.service.cli.session.HiveSession; - -/** - * GetColumnsOperation. - * - */ -public class GetColumnsOperation extends MetadataOperation { - - protected static final TableSchema RESULT_SET_SCHEMA = new TableSchema() - .addPrimitiveColumn("TABLE_CAT", Type.STRING_TYPE, - "Catalog name. NULL if not applicable") - .addPrimitiveColumn("TABLE_SCHEM", Type.STRING_TYPE, - "Schema name") - .addPrimitiveColumn("TABLE_NAME", Type.STRING_TYPE, - "Table name") - .addPrimitiveColumn("COLUMN_NAME", Type.STRING_TYPE, - "Column name") - .addPrimitiveColumn("DATA_TYPE", Type.INT_TYPE, - "SQL type from java.sql.Types") - .addPrimitiveColumn("TYPE_NAME", Type.STRING_TYPE, - "Data source dependent type name, for a UDT the type name is fully qualified") - .addPrimitiveColumn("COLUMN_SIZE", Type.INT_TYPE, - "Column size. For char or date types this is the maximum number of characters," - + " for numeric or decimal types this is precision.") - .addPrimitiveColumn("BUFFER_LENGTH", Type.TINYINT_TYPE, - "Unused") - .addPrimitiveColumn("DECIMAL_DIGITS", Type.INT_TYPE, - "The number of fractional digits") - .addPrimitiveColumn("NUM_PREC_RADIX", Type.INT_TYPE, - "Radix (typically either 10 or 2)") - .addPrimitiveColumn("NULLABLE", Type.INT_TYPE, - "Is NULL allowed") - .addPrimitiveColumn("REMARKS", Type.STRING_TYPE, - "Comment describing column (may be null)") - .addPrimitiveColumn("COLUMN_DEF", Type.STRING_TYPE, - "Default value (may be null)") - .addPrimitiveColumn("SQL_DATA_TYPE", Type.INT_TYPE, - "Unused") - .addPrimitiveColumn("SQL_DATETIME_SUB", Type.INT_TYPE, - "Unused") - .addPrimitiveColumn("CHAR_OCTET_LENGTH", Type.INT_TYPE, - "For char types the maximum number of bytes in the column") - .addPrimitiveColumn("ORDINAL_POSITION", Type.INT_TYPE, - "Index of column in table (starting at 1)") - .addPrimitiveColumn("IS_NULLABLE", Type.STRING_TYPE, - "\"NO\" means column definitely does not allow NULL values; " - + "\"YES\" means the column might allow NULL values. An empty " - + "string means nobody knows.") - .addPrimitiveColumn("SCOPE_CATALOG", Type.STRING_TYPE, - "Catalog of table that is the scope of a reference attribute " - + "(null if DATA_TYPE isn't REF)") - .addPrimitiveColumn("SCOPE_SCHEMA", Type.STRING_TYPE, - "Schema of table that is the scope of a reference attribute " - + "(null if the DATA_TYPE isn't REF)") - .addPrimitiveColumn("SCOPE_TABLE", Type.STRING_TYPE, - "Table name that this the scope of a reference attribure " - + "(null if the DATA_TYPE isn't REF)") - .addPrimitiveColumn("SOURCE_DATA_TYPE", Type.SMALLINT_TYPE, - "Source type of a distinct type or user-generated Ref type, " - + "SQL type from java.sql.Types (null if DATA_TYPE isn't DISTINCT or user-generated REF)") - .addPrimitiveColumn("IS_AUTO_INCREMENT", Type.STRING_TYPE, - "Indicates whether this column is auto incremented."); - - private final String catalogName; - private final String schemaName; - private final String tableName; - private final String columnName; - - protected final RowSet rowSet; - - protected GetColumnsOperation(HiveSession parentSession, String catalogName, String schemaName, - String tableName, String columnName) { - super(parentSession, OperationType.GET_COLUMNS); - this.catalogName = catalogName; - this.schemaName = schemaName; - this.tableName = tableName; - this.columnName = columnName; - this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.RUNNING); - try { - IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); - String schemaPattern = convertSchemaPattern(schemaName); - String tablePattern = convertIdentifierPattern(tableName, true); - - Pattern columnPattern = null; - if (columnName != null) { - columnPattern = Pattern.compile(convertIdentifierPattern(columnName, false)); - } - - List dbNames = metastoreClient.getDatabases(schemaPattern); - Collections.sort(dbNames); - Map> db2Tabs = new HashMap<>(); - - for (String dbName : dbNames) { - List tableNames = metastoreClient.getTables(dbName, tablePattern); - Collections.sort(tableNames); - db2Tabs.put(dbName, tableNames); - } - - if (isAuthV2Enabled()) { - List privObjs = getPrivObjs(db2Tabs); - String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName - + ", tablePattern : " + tableName; - authorizeMetaGets(HiveOperationType.GET_COLUMNS, privObjs, cmdStr); - } - - for (Entry> dbTabs : db2Tabs.entrySet()) { - String dbName = dbTabs.getKey(); - List tableNames = dbTabs.getValue(); - for (Table table : metastoreClient.getTableObjectsByName(dbName, tableNames)) { - TableSchema schema = new TableSchema(metastoreClient.getSchema(dbName, table.getTableName())); - for (ColumnDescriptor column : schema.getColumnDescriptors()) { - if (columnPattern != null && !columnPattern.matcher(column.getName()).matches()) { - continue; - } - Object[] rowData = new Object[] { - null, // TABLE_CAT - table.getDbName(), // TABLE_SCHEM - table.getTableName(), // TABLE_NAME - column.getName(), // COLUMN_NAME - column.getType().toJavaSQLType(), // DATA_TYPE - column.getTypeName(), // TYPE_NAME - column.getTypeDescriptor().getColumnSize(), // COLUMN_SIZE - null, // BUFFER_LENGTH, unused - column.getTypeDescriptor().getDecimalDigits(), // DECIMAL_DIGITS - column.getType().getNumPrecRadix(), // NUM_PREC_RADIX - DatabaseMetaData.columnNullable, // NULLABLE - column.getComment(), // REMARKS - null, // COLUMN_DEF - null, // SQL_DATA_TYPE - null, // SQL_DATETIME_SUB - null, // CHAR_OCTET_LENGTH - column.getOrdinalPosition(), // ORDINAL_POSITION - "YES", // IS_NULLABLE - null, // SCOPE_CATALOG - null, // SCOPE_SCHEMA - null, // SCOPE_TABLE - null, // SOURCE_DATA_TYPE - "NO", // IS_AUTO_INCREMENT - }; - rowSet.addRow(rowData); - } - } - } - setState(OperationState.FINISHED); - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException(e); - } - - } - - - private List getPrivObjs(Map> db2Tabs) { - List privObjs = new ArrayList<>(); - for (Entry> dbTabs : db2Tabs.entrySet()) { - for (String tabName : dbTabs.getValue()) { - privObjs.add(new HivePrivilegeObject(HivePrivilegeObjectType.TABLE_OR_VIEW, dbTabs.getKey(), - tabName)); - } - } - return privObjs; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getResultSetSchema() - */ - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - assertState(OperationState.FINISHED); - return RESULT_SET_SCHEMA; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) - */ - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - assertState(OperationState.FINISHED); - validateDefaultFetchOrientation(orientation); - if (orientation.equals(FetchOrientation.FETCH_FIRST)) { - rowSet.setStartOffset(0); - } - return rowSet.extractSubset((int)maxRows); - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java deleted file mode 100644 index 5dec8bdbf45de..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java +++ /dev/null @@ -1,147 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.operation; - -import java.sql.DatabaseMetaData; -import java.util.List; -import java.util.Set; - -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.ql.exec.FunctionInfo; -import org.apache.hadoop.hive.ql.exec.FunctionRegistry; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils; -import org.apache.hive.service.cli.CLIServiceUtils; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.Type; -import org.apache.hive.service.cli.session.HiveSession; -import org.apache.thrift.TException; - -/** - * GetFunctionsOperation. - * - */ -public class GetFunctionsOperation extends MetadataOperation { - private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() - .addPrimitiveColumn("FUNCTION_CAT", Type.STRING_TYPE, - "Function catalog (may be null)") - .addPrimitiveColumn("FUNCTION_SCHEM", Type.STRING_TYPE, - "Function schema (may be null)") - .addPrimitiveColumn("FUNCTION_NAME", Type.STRING_TYPE, - "Function name. This is the name used to invoke the function") - .addPrimitiveColumn("REMARKS", Type.STRING_TYPE, - "Explanatory comment on the function") - .addPrimitiveColumn("FUNCTION_TYPE", Type.INT_TYPE, - "Kind of function.") - .addPrimitiveColumn("SPECIFIC_NAME", Type.STRING_TYPE, - "The name which uniquely identifies this function within its schema"); - - private final String catalogName; - private final String schemaName; - private final String functionName; - - protected final RowSet rowSet; - - public GetFunctionsOperation(HiveSession parentSession, - String catalogName, String schemaName, String functionName) { - super(parentSession, OperationType.GET_FUNCTIONS); - this.catalogName = catalogName; - this.schemaName = schemaName; - this.functionName = functionName; - this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.RUNNING); - if (isAuthV2Enabled()) { - // get databases for schema pattern - IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); - String schemaPattern = convertSchemaPattern(schemaName); - List matchingDbs; - try { - matchingDbs = metastoreClient.getDatabases(schemaPattern); - } catch (TException e) { - setState(OperationState.ERROR); - throw new HiveSQLException(e); - } - // authorize this call on the schema objects - List privObjs = HivePrivilegeObjectUtils - .getHivePrivDbObjects(matchingDbs); - String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName; - authorizeMetaGets(HiveOperationType.GET_FUNCTIONS, privObjs, cmdStr); - } - - try { - if ((null == catalogName || "".equals(catalogName)) - && (null == schemaName || "".equals(schemaName))) { - Set functionNames = FunctionRegistry - .getFunctionNames(CLIServiceUtils.patternToRegex(functionName)); - for (String functionName : functionNames) { - FunctionInfo functionInfo = FunctionRegistry.getFunctionInfo(functionName); - Object[] rowData = new Object[] { - null, // FUNCTION_CAT - null, // FUNCTION_SCHEM - functionInfo.getDisplayName(), // FUNCTION_NAME - "", // REMARKS - (functionInfo.isGenericUDTF() ? - DatabaseMetaData.functionReturnsTable - : DatabaseMetaData.functionNoTable), // FUNCTION_TYPE - functionInfo.getClass().getCanonicalName() - }; - rowSet.addRow(rowData); - } - } - setState(OperationState.FINISHED); - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException(e); - } - } - - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getResultSetSchema() - */ - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - assertState(OperationState.FINISHED); - return RESULT_SET_SCHEMA; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) - */ - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - assertState(OperationState.FINISHED); - validateDefaultFetchOrientation(orientation); - if (orientation.equals(FetchOrientation.FETCH_FIRST)) { - rowSet.setStartOffset(0); - } - return rowSet.extractSubset((int)maxRows); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java deleted file mode 100644 index 3516bc2ba242c..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java +++ /dev/null @@ -1,96 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.operation; - -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; - -/** - * GetSchemasOperation. - * - */ -public class GetSchemasOperation extends MetadataOperation { - private final String catalogName; - private final String schemaName; - - private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() - .addStringColumn("TABLE_SCHEM", "Schema name.") - .addStringColumn("TABLE_CATALOG", "Catalog name."); - - protected RowSet rowSet; - - protected GetSchemasOperation(HiveSession parentSession, - String catalogName, String schemaName) { - super(parentSession, OperationType.GET_SCHEMAS); - this.catalogName = catalogName; - this.schemaName = schemaName; - this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.RUNNING); - if (isAuthV2Enabled()) { - String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName; - authorizeMetaGets(HiveOperationType.GET_SCHEMAS, null, cmdStr); - } - try { - IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); - String schemaPattern = convertSchemaPattern(schemaName); - for (String dbName : metastoreClient.getDatabases(schemaPattern)) { - rowSet.addRow(new Object[] {dbName, DEFAULT_HIVE_CATALOG}); - } - setState(OperationState.FINISHED); - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException(e); - } - } - - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getResultSetSchema() - */ - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - assertState(OperationState.FINISHED); - return RESULT_SET_SCHEMA; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) - */ - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - assertState(OperationState.FINISHED); - validateDefaultFetchOrientation(orientation); - if (orientation.equals(FetchOrientation.FETCH_FIRST)) { - rowSet.setStartOffset(0); - } - return rowSet.extractSubset((int)maxRows); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java deleted file mode 100644 index b372f55cedd1c..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java +++ /dev/null @@ -1,93 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.operation; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.TableType; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; - -/** - * GetTableTypesOperation. - * - */ -public class GetTableTypesOperation extends MetadataOperation { - - protected static TableSchema RESULT_SET_SCHEMA = new TableSchema() - .addStringColumn("TABLE_TYPE", "Table type name."); - - protected final RowSet rowSet; - private final TableTypeMapping tableTypeMapping; - - protected GetTableTypesOperation(HiveSession parentSession) { - super(parentSession, OperationType.GET_TABLE_TYPES); - String tableMappingStr = getParentSession().getHiveConf() - .getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); - tableTypeMapping = - TableTypeMappingFactory.getTableTypeMapping(tableMappingStr); - rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.RUNNING); - if (isAuthV2Enabled()) { - authorizeMetaGets(HiveOperationType.GET_TABLETYPES, null); - } - try { - for (TableType type : TableType.values()) { - rowSet.addRow(new String[] {tableTypeMapping.mapToClientType(type.toString())}); - } - setState(OperationState.FINISHED); - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getResultSetSchema() - */ - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - assertState(OperationState.FINISHED); - return RESULT_SET_SCHEMA; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) - */ - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - assertState(OperationState.FINISHED); - validateDefaultFetchOrientation(orientation); - if (orientation.equals(FetchOrientation.FETCH_FIRST)) { - rowSet.setStartOffset(0); - } - return rowSet.extractSubset((int)maxRows); - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java deleted file mode 100644 index 2af17a662a296..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java +++ /dev/null @@ -1,135 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.operation; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; - -/** - * GetTablesOperation. - * - */ -public class GetTablesOperation extends MetadataOperation { - - private final String catalogName; - private final String schemaName; - private final String tableName; - private final List tableTypes = new ArrayList(); - protected final RowSet rowSet; - private final TableTypeMapping tableTypeMapping; - - - private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() - .addStringColumn("TABLE_CAT", "Catalog name. NULL if not applicable.") - .addStringColumn("TABLE_SCHEM", "Schema name.") - .addStringColumn("TABLE_NAME", "Table name.") - .addStringColumn("TABLE_TYPE", "The table type, e.g. \"TABLE\", \"VIEW\", etc.") - .addStringColumn("REMARKS", "Comments about the table."); - - protected GetTablesOperation(HiveSession parentSession, - String catalogName, String schemaName, String tableName, - List tableTypes) { - super(parentSession, OperationType.GET_TABLES); - this.catalogName = catalogName; - this.schemaName = schemaName; - this.tableName = tableName; - String tableMappingStr = getParentSession().getHiveConf() - .getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); - tableTypeMapping = - TableTypeMappingFactory.getTableTypeMapping(tableMappingStr); - if (tableTypes != null) { - this.tableTypes.addAll(tableTypes); - } - this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.RUNNING); - try { - IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); - String schemaPattern = convertSchemaPattern(schemaName); - List matchingDbs = metastoreClient.getDatabases(schemaPattern); - if(isAuthV2Enabled()){ - List privObjs = HivePrivilegeObjectUtils.getHivePrivDbObjects(matchingDbs); - String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName; - authorizeMetaGets(HiveOperationType.GET_TABLES, privObjs, cmdStr); - } - - String tablePattern = convertIdentifierPattern(tableName, true); - for (String dbName : metastoreClient.getDatabases(schemaPattern)) { - List tableNames = metastoreClient.getTables(dbName, tablePattern); - for (Table table : metastoreClient.getTableObjectsByName(dbName, tableNames)) { - Object[] rowData = new Object[] { - DEFAULT_HIVE_CATALOG, - table.getDbName(), - table.getTableName(), - tableTypeMapping.mapToClientType(table.getTableType()), - table.getParameters().get("comment") - }; - if (tableTypes.isEmpty() || tableTypes.contains( - tableTypeMapping.mapToClientType(table.getTableType()))) { - rowSet.addRow(rowData); - } - } - } - setState(OperationState.FINISHED); - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getResultSetSchema() - */ - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - assertState(OperationState.FINISHED); - return RESULT_SET_SCHEMA; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) - */ - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - assertState(OperationState.FINISHED); - validateDefaultFetchOrientation(orientation); - if (orientation.equals(FetchOrientation.FETCH_FIRST)) { - rowSet.setStartOffset(0); - } - return rowSet.extractSubset((int)maxRows); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java deleted file mode 100644 index 3e81f8afbd85f..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java +++ /dev/null @@ -1,142 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.operation; - -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.Type; -import org.apache.hive.service.cli.session.HiveSession; - -/** - * GetTypeInfoOperation. - * - */ -public class GetTypeInfoOperation extends MetadataOperation { - - private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() - .addPrimitiveColumn("TYPE_NAME", Type.STRING_TYPE, - "Type name") - .addPrimitiveColumn("DATA_TYPE", Type.INT_TYPE, - "SQL data type from java.sql.Types") - .addPrimitiveColumn("PRECISION", Type.INT_TYPE, - "Maximum precision") - .addPrimitiveColumn("LITERAL_PREFIX", Type.STRING_TYPE, - "Prefix used to quote a literal (may be null)") - .addPrimitiveColumn("LITERAL_SUFFIX", Type.STRING_TYPE, - "Suffix used to quote a literal (may be null)") - .addPrimitiveColumn("CREATE_PARAMS", Type.STRING_TYPE, - "Parameters used in creating the type (may be null)") - .addPrimitiveColumn("NULLABLE", Type.SMALLINT_TYPE, - "Can you use NULL for this type") - .addPrimitiveColumn("CASE_SENSITIVE", Type.BOOLEAN_TYPE, - "Is it case sensitive") - .addPrimitiveColumn("SEARCHABLE", Type.SMALLINT_TYPE, - "Can you use \"WHERE\" based on this type") - .addPrimitiveColumn("UNSIGNED_ATTRIBUTE", Type.BOOLEAN_TYPE, - "Is it unsigned") - .addPrimitiveColumn("FIXED_PREC_SCALE", Type.BOOLEAN_TYPE, - "Can it be a money value") - .addPrimitiveColumn("AUTO_INCREMENT", Type.BOOLEAN_TYPE, - "Can it be used for an auto-increment value") - .addPrimitiveColumn("LOCAL_TYPE_NAME", Type.STRING_TYPE, - "Localized version of type name (may be null)") - .addPrimitiveColumn("MINIMUM_SCALE", Type.SMALLINT_TYPE, - "Minimum scale supported") - .addPrimitiveColumn("MAXIMUM_SCALE", Type.SMALLINT_TYPE, - "Maximum scale supported") - .addPrimitiveColumn("SQL_DATA_TYPE", Type.INT_TYPE, - "Unused") - .addPrimitiveColumn("SQL_DATETIME_SUB", Type.INT_TYPE, - "Unused") - .addPrimitiveColumn("NUM_PREC_RADIX", Type.INT_TYPE, - "Usually 2 or 10"); - - protected final RowSet rowSet; - - protected GetTypeInfoOperation(HiveSession parentSession) { - super(parentSession, OperationType.GET_TYPE_INFO); - rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.RUNNING); - if (isAuthV2Enabled()) { - authorizeMetaGets(HiveOperationType.GET_TYPEINFO, null); - } - try { - for (Type type : Type.values()) { - Object[] rowData = new Object[] { - type.getName(), // TYPE_NAME - type.toJavaSQLType(), // DATA_TYPE - type.getMaxPrecision(), // PRECISION - type.getLiteralPrefix(), // LITERAL_PREFIX - type.getLiteralSuffix(), // LITERAL_SUFFIX - type.getCreateParams(), // CREATE_PARAMS - type.getNullable(), // NULLABLE - type.isCaseSensitive(), // CASE_SENSITIVE - type.getSearchable(), // SEARCHABLE - type.isUnsignedAttribute(), // UNSIGNED_ATTRIBUTE - type.isFixedPrecScale(), // FIXED_PREC_SCALE - type.isAutoIncrement(), // AUTO_INCREMENT - type.getLocalizedName(), // LOCAL_TYPE_NAME - type.getMinimumScale(), // MINIMUM_SCALE - type.getMaximumScale(), // MAXIMUM_SCALE - null, // SQL_DATA_TYPE, unused - null, // SQL_DATETIME_SUB, unused - type.getNumPrecRadix() //NUM_PREC_RADIX - }; - rowSet.addRow(rowData); - } - setState(OperationState.FINISHED); - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException(e); - } - } - - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getResultSetSchema() - */ - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - assertState(OperationState.FINISHED); - return RESULT_SET_SCHEMA; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) - */ - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - assertState(OperationState.FINISHED); - validateDefaultFetchOrientation(orientation); - if (orientation.equals(FetchOrientation.FETCH_FIRST)) { - rowSet.setStartOffset(0); - } - return rowSet.extractSubset((int)maxRows); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java deleted file mode 100644 index 5b6e6ad042412..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java +++ /dev/null @@ -1,215 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.operation; - -import java.io.BufferedReader; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.FileReader; -import java.io.IOException; -import java.io.PrintStream; -import java.io.UnsupportedEncodingException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -import static java.nio.charset.StandardCharsets.UTF_8; - -import org.apache.hadoop.hive.metastore.api.Schema; -import org.apache.hadoop.hive.ql.processors.CommandProcessor; -import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.io.IOUtils; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; - -/** - * Executes a HiveCommand - */ -public class HiveCommandOperation extends ExecuteStatementOperation { - private CommandProcessor commandProcessor; - private TableSchema resultSchema = null; - - /** - * For processors other than Hive queries (Driver), they output to session.out (a temp file) - * first and the fetchOne/fetchN/fetchAll functions get the output from pipeIn. - */ - private BufferedReader resultReader; - - - protected HiveCommandOperation(HiveSession parentSession, String statement, - CommandProcessor commandProcessor, Map confOverlay) { - super(parentSession, statement, confOverlay, false); - this.commandProcessor = commandProcessor; - setupSessionIO(parentSession.getSessionState()); - } - - private void setupSessionIO(SessionState sessionState) { - try { - LOG.info("Putting temp output to file " + sessionState.getTmpOutputFile().toString()); - sessionState.in = null; // hive server's session input stream is not used - // open a per-session file in auto-flush mode for writing temp results - sessionState.out = new PrintStream(new FileOutputStream(sessionState.getTmpOutputFile()), true, UTF_8.name()); - // TODO: for hadoop jobs, progress is printed out to session.err, - // we should find a way to feed back job progress to client - sessionState.err = new PrintStream(System.err, true, UTF_8.name()); - } catch (IOException e) { - LOG.error("Error in creating temp output file ", e); - try { - sessionState.in = null; - sessionState.out = new PrintStream(System.out, true, UTF_8.name()); - sessionState.err = new PrintStream(System.err, true, UTF_8.name()); - } catch (UnsupportedEncodingException ee) { - LOG.error("Error creating PrintStream", e); - ee.printStackTrace(); - sessionState.out = null; - sessionState.err = null; - } - } - } - - - private void tearDownSessionIO() { - IOUtils.cleanup(LOG, parentSession.getSessionState().out); - IOUtils.cleanup(LOG, parentSession.getSessionState().err); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.RUNNING); - try { - String command = getStatement().trim(); - String[] tokens = statement.split("\\s"); - String commandArgs = command.substring(tokens[0].length()).trim(); - - CommandProcessorResponse response = commandProcessor.run(commandArgs); - int returnCode = response.getResponseCode(); - if (returnCode != 0) { - throw toSQLException("Error while processing statement", response); - } - Schema schema = response.getSchema(); - if (schema != null) { - setHasResultSet(true); - resultSchema = new TableSchema(schema); - } else { - setHasResultSet(false); - resultSchema = new TableSchema(); - } - } catch (HiveSQLException e) { - setState(OperationState.ERROR); - throw e; - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException("Error running query: " + e.toString(), e); - } - setState(OperationState.FINISHED); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.operation.Operation#close() - */ - @Override - public void close() throws HiveSQLException { - setState(OperationState.CLOSED); - tearDownSessionIO(); - cleanTmpFile(); - cleanupOperationLog(); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.operation.Operation#getResultSetSchema() - */ - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - return resultSchema; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.operation.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) - */ - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - validateDefaultFetchOrientation(orientation); - if (orientation.equals(FetchOrientation.FETCH_FIRST)) { - resetResultReader(); - } - List rows = readResults((int) maxRows); - RowSet rowSet = RowSetFactory.create(resultSchema, getProtocolVersion()); - - for (String row : rows) { - rowSet.addRow(new String[] {row}); - } - return rowSet; - } - - /** - * Reads the temporary results for non-Hive (non-Driver) commands to the - * resulting List of strings. - * @param nLines number of lines read at once. If it is <= 0, then read all lines. - */ - private List readResults(int nLines) throws HiveSQLException { - if (resultReader == null) { - SessionState sessionState = getParentSession().getSessionState(); - File tmp = sessionState.getTmpOutputFile(); - try { - resultReader = new BufferedReader(new FileReader(tmp)); - } catch (FileNotFoundException e) { - LOG.error("File " + tmp + " not found. ", e); - throw new HiveSQLException(e); - } - } - List results = new ArrayList(); - - for (int i = 0; i < nLines || nLines <= 0; ++i) { - try { - String line = resultReader.readLine(); - if (line == null) { - // reached the end of the result file - break; - } else { - results.add(line); - } - } catch (IOException e) { - LOG.error("Reading temp results encountered an exception: ", e); - throw new HiveSQLException(e); - } - } - return results; - } - - private void cleanTmpFile() { - resetResultReader(); - SessionState sessionState = getParentSession().getSessionState(); - File tmp = sessionState.getTmpOutputFile(); - tmp.delete(); - } - - private void resetResultReader() { - if (resultReader != null) { - IOUtils.cleanup(LOG, resultReader); - resultReader = null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java deleted file mode 100644 index b530f217125b8..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java +++ /dev/null @@ -1,51 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.operation; - -import java.util.HashSet; -import java.util.Set; - -import org.apache.hadoop.hive.metastore.TableType; - -/** - * HiveTableTypeMapping. - * Default table type mapping - * - */ -public class HiveTableTypeMapping implements TableTypeMapping { - - @Override - public String mapToHiveType(String clientTypeName) { - return clientTypeName; - } - - @Override - public String mapToClientType(String hiveTypeName) { - return hiveTypeName; - } - - @Override - public Set getTableTypeNames() { - Set typeNameSet = new HashSet(); - for (TableType typeNames : TableType.values()) { - typeNameSet.add(typeNames.toString()); - } - return typeNameSet; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java deleted file mode 100644 index 6c819876a556d..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java +++ /dev/null @@ -1,134 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.operation; - -import java.util.List; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; - -/** - * MetadataOperation. - * - */ -public abstract class MetadataOperation extends Operation { - - protected static final String DEFAULT_HIVE_CATALOG = ""; - protected static TableSchema RESULT_SET_SCHEMA; - private static final char SEARCH_STRING_ESCAPE = '\\'; - - protected MetadataOperation(HiveSession parentSession, OperationType opType) { - super(parentSession, opType, false); - setHasResultSet(true); - } - - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#close() - */ - @Override - public void close() throws HiveSQLException { - setState(OperationState.CLOSED); - cleanupOperationLog(); - } - - /** - * Convert wildchars and escape sequence from JDBC format to datanucleous/regex - */ - protected String convertIdentifierPattern(final String pattern, boolean datanucleusFormat) { - if (pattern == null) { - return convertPattern("%", true); - } else { - return convertPattern(pattern, datanucleusFormat); - } - } - - /** - * Convert wildchars and escape sequence of schema pattern from JDBC format to datanucleous/regex - * The schema pattern treats empty string also as wildchar - */ - protected String convertSchemaPattern(final String pattern) { - if ((pattern == null) || pattern.isEmpty()) { - return convertPattern("%", true); - } else { - return convertPattern(pattern, true); - } - } - - /** - * Convert a pattern containing JDBC catalog search wildcards into - * Java regex patterns. - * - * @param pattern input which may contain '%' or '_' wildcard characters, or - * these characters escaped using {@link #getSearchStringEscape()}. - * @return replace %/_ with regex search characters, also handle escaped - * characters. - * - * The datanucleus module expects the wildchar as '*'. The columns search on the - * other hand is done locally inside the hive code and that requires the regex wildchar - * format '.*' This is driven by the datanucleusFormat flag. - */ - private String convertPattern(final String pattern, boolean datanucleusFormat) { - String wStr; - if (datanucleusFormat) { - wStr = "*"; - } else { - wStr = ".*"; - } - return pattern - .replaceAll("([^\\\\])%", "$1" + wStr).replaceAll("\\\\%", "%").replaceAll("^%", wStr) - .replaceAll("([^\\\\])_", "$1.").replaceAll("\\\\_", "_").replaceAll("^_", "."); - } - - protected boolean isAuthV2Enabled(){ - SessionState ss = SessionState.get(); - return (ss.isAuthorizationModeV2() && - HiveConf.getBoolVar(ss.getConf(), HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)); - } - - protected void authorizeMetaGets(HiveOperationType opType, List inpObjs) - throws HiveSQLException { - authorizeMetaGets(opType, inpObjs, null); - } - - protected void authorizeMetaGets(HiveOperationType opType, List inpObjs, - String cmdString) throws HiveSQLException { - SessionState ss = SessionState.get(); - HiveAuthzContext.Builder ctxBuilder = new HiveAuthzContext.Builder(); - ctxBuilder.setUserIpAddress(ss.getUserIpAddress()); - ctxBuilder.setCommandString(cmdString); - try { - ss.getAuthorizerV2().checkPrivileges(opType, inpObjs, null, - ctxBuilder.build()); - } catch (HiveAuthzPluginException | HiveAccessControlException e) { - throw new HiveSQLException(e.getMessage(), e); - } - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/Operation.java deleted file mode 100644 index 4b331423948fa..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/Operation.java +++ /dev/null @@ -1,328 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.service.cli.operation; - -import java.io.File; -import java.io.FileNotFoundException; -import java.util.EnumSet; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; -import org.apache.hadoop.hive.ql.session.OperationLog; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationHandle; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationStatus; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; -import org.apache.hive.service.cli.thrift.TProtocolVersion; - -public abstract class Operation { - protected final HiveSession parentSession; - private OperationState state = OperationState.INITIALIZED; - private final OperationHandle opHandle; - private HiveConf configuration; - public static final Log LOG = LogFactory.getLog(Operation.class.getName()); - public static final FetchOrientation DEFAULT_FETCH_ORIENTATION = FetchOrientation.FETCH_NEXT; - public static final long DEFAULT_FETCH_MAX_ROWS = 100; - protected boolean hasResultSet; - protected volatile HiveSQLException operationException; - protected final boolean runAsync; - protected volatile Future backgroundHandle; - protected OperationLog operationLog; - protected boolean isOperationLogEnabled; - - private long operationTimeout; - private long lastAccessTime; - - protected static final EnumSet DEFAULT_FETCH_ORIENTATION_SET = - EnumSet.of( - FetchOrientation.FETCH_NEXT, - FetchOrientation.FETCH_FIRST, - FetchOrientation.FETCH_PRIOR); - - protected Operation(HiveSession parentSession, OperationType opType, boolean runInBackground) { - this.parentSession = parentSession; - this.runAsync = runInBackground; - this.opHandle = new OperationHandle(opType, parentSession.getProtocolVersion()); - lastAccessTime = System.currentTimeMillis(); - operationTimeout = HiveConf.getTimeVar(parentSession.getHiveConf(), - HiveConf.ConfVars.HIVE_SERVER2_IDLE_OPERATION_TIMEOUT, TimeUnit.MILLISECONDS); - } - - public Future getBackgroundHandle() { - return backgroundHandle; - } - - protected void setBackgroundHandle(Future backgroundHandle) { - this.backgroundHandle = backgroundHandle; - } - - public boolean shouldRunAsync() { - return runAsync; - } - - public void setConfiguration(HiveConf configuration) { - this.configuration = new HiveConf(configuration); - } - - public HiveConf getConfiguration() { - return new HiveConf(configuration); - } - - public HiveSession getParentSession() { - return parentSession; - } - - public OperationHandle getHandle() { - return opHandle; - } - - public TProtocolVersion getProtocolVersion() { - return opHandle.getProtocolVersion(); - } - - public OperationType getType() { - return opHandle.getOperationType(); - } - - public OperationStatus getStatus() { - return new OperationStatus(state, operationException); - } - - public boolean hasResultSet() { - return hasResultSet; - } - - protected void setHasResultSet(boolean hasResultSet) { - this.hasResultSet = hasResultSet; - opHandle.setHasResultSet(hasResultSet); - } - - public OperationLog getOperationLog() { - return operationLog; - } - - protected final OperationState setState(OperationState newState) throws HiveSQLException { - state.validateTransition(newState); - this.state = newState; - this.lastAccessTime = System.currentTimeMillis(); - return this.state; - } - - public boolean isTimedOut(long current) { - if (operationTimeout == 0) { - return false; - } - if (operationTimeout > 0) { - // check only when it's in terminal state - return state.isTerminal() && lastAccessTime + operationTimeout <= current; - } - return lastAccessTime + -operationTimeout <= current; - } - - public long getLastAccessTime() { - return lastAccessTime; - } - - public long getOperationTimeout() { - return operationTimeout; - } - - public void setOperationTimeout(long operationTimeout) { - this.operationTimeout = operationTimeout; - } - - protected void setOperationException(HiveSQLException operationException) { - this.operationException = operationException; - } - - protected final void assertState(OperationState state) throws HiveSQLException { - if (this.state != state) { - throw new HiveSQLException("Expected state " + state + ", but found " + this.state); - } - this.lastAccessTime = System.currentTimeMillis(); - } - - public boolean isRunning() { - return OperationState.RUNNING.equals(state); - } - - public boolean isFinished() { - return OperationState.FINISHED.equals(state); - } - - public boolean isCanceled() { - return OperationState.CANCELED.equals(state); - } - - public boolean isFailed() { - return OperationState.ERROR.equals(state); - } - - protected void createOperationLog() { - if (parentSession.isOperationLogEnabled()) { - File operationLogFile = new File(parentSession.getOperationLogSessionDir(), - opHandle.getHandleIdentifier().toString()); - isOperationLogEnabled = true; - - // create log file - try { - if (operationLogFile.exists()) { - LOG.warn("The operation log file should not exist, but it is already there: " + - operationLogFile.getAbsolutePath()); - operationLogFile.delete(); - } - if (!operationLogFile.createNewFile()) { - // the log file already exists and cannot be deleted. - // If it can be read/written, keep its contents and use it. - if (!operationLogFile.canRead() || !operationLogFile.canWrite()) { - LOG.warn("The already existed operation log file cannot be recreated, " + - "and it cannot be read or written: " + operationLogFile.getAbsolutePath()); - isOperationLogEnabled = false; - return; - } - } - } catch (Exception e) { - LOG.warn("Unable to create operation log file: " + operationLogFile.getAbsolutePath(), e); - isOperationLogEnabled = false; - return; - } - - // create OperationLog object with above log file - try { - operationLog = new OperationLog(opHandle.toString(), operationLogFile, parentSession.getHiveConf()); - } catch (FileNotFoundException e) { - LOG.warn("Unable to instantiate OperationLog object for operation: " + - opHandle, e); - isOperationLogEnabled = false; - return; - } - - // register this operationLog to current thread - OperationLog.setCurrentOperationLog(operationLog); - } - } - - protected void unregisterOperationLog() { - if (isOperationLogEnabled) { - OperationLog.removeCurrentOperationLog(); - } - } - - /** - * Invoked before runInternal(). - * Set up some preconditions, or configurations. - */ - protected void beforeRun() { - createOperationLog(); - } - - /** - * Invoked after runInternal(), even if an exception is thrown in runInternal(). - * Clean up resources, which was set up in beforeRun(). - */ - protected void afterRun() { - unregisterOperationLog(); - } - - /** - * Implemented by subclass of Operation class to execute specific behaviors. - * @throws HiveSQLException - */ - protected abstract void runInternal() throws HiveSQLException; - - public void run() throws HiveSQLException { - beforeRun(); - try { - runInternal(); - } finally { - afterRun(); - } - } - - protected void cleanupOperationLog() { - if (isOperationLogEnabled) { - if (operationLog == null) { - LOG.error("Operation [ " + opHandle.getHandleIdentifier() + " ] " - + "logging is enabled, but its OperationLog object cannot be found."); - } else { - operationLog.close(); - } - } - } - - // TODO: make this abstract and implement in subclasses. - public void cancel() throws HiveSQLException { - setState(OperationState.CANCELED); - throw new UnsupportedOperationException("SQLOperation.cancel()"); - } - - public void close() throws HiveSQLException { - setState(OperationState.CLOSED); - cleanupOperationLog(); - } - - public abstract TableSchema getResultSetSchema() throws HiveSQLException; - - public abstract RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException; - - public RowSet getNextRowSet() throws HiveSQLException { - return getNextRowSet(FetchOrientation.FETCH_NEXT, DEFAULT_FETCH_MAX_ROWS); - } - - /** - * Verify if the given fetch orientation is part of the default orientation types. - * @param orientation - * @throws HiveSQLException - */ - protected void validateDefaultFetchOrientation(FetchOrientation orientation) - throws HiveSQLException { - validateFetchOrientation(orientation, DEFAULT_FETCH_ORIENTATION_SET); - } - - /** - * Verify if the given fetch orientation is part of the supported orientation types. - * @param orientation - * @param supportedOrientations - * @throws HiveSQLException - */ - protected void validateFetchOrientation(FetchOrientation orientation, - EnumSet supportedOrientations) throws HiveSQLException { - if (!supportedOrientations.contains(orientation)) { - throw new HiveSQLException("The fetch type " + orientation.toString() + - " is not supported for this resultset", "HY106"); - } - } - - protected HiveSQLException toSQLException(String prefix, CommandProcessorResponse response) { - HiveSQLException ex = new HiveSQLException(prefix + ": " + response.getErrorMessage(), - response.getSQLState(), response.getResponseCode()); - if (response.getException() != null) { - ex.initCause(response.getException()); - } - return ex; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java deleted file mode 100644 index 92c340a29c107..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java +++ /dev/null @@ -1,284 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.operation; - -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.Schema; -import org.apache.hadoop.hive.ql.session.OperationLog; -import org.apache.hive.service.AbstractService; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationHandle; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationStatus; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; -import org.apache.log4j.Appender; -import org.apache.log4j.Logger; - -/** - * OperationManager. - * - */ -public class OperationManager extends AbstractService { - private final Log LOG = LogFactory.getLog(OperationManager.class.getName()); - - private final Map handleToOperation = - new HashMap(); - - public OperationManager() { - super(OperationManager.class.getSimpleName()); - } - - @Override - public synchronized void init(HiveConf hiveConf) { - if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED)) { - initOperationLogCapture(hiveConf.getVar( - HiveConf.ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LEVEL)); - } else { - LOG.debug("Operation level logging is turned off"); - } - super.init(hiveConf); - } - - @Override - public synchronized void start() { - super.start(); - // TODO - } - - @Override - public synchronized void stop() { - // TODO - super.stop(); - } - - private void initOperationLogCapture(String loggingMode) { - // Register another Appender (with the same layout) that talks to us. - Appender ap = new LogDivertAppender(this, OperationLog.getLoggingLevel(loggingMode)); - Logger.getRootLogger().addAppender(ap); - } - - public ExecuteStatementOperation newExecuteStatementOperation(HiveSession parentSession, - String statement, Map confOverlay, boolean runAsync) - throws HiveSQLException { - ExecuteStatementOperation executeStatementOperation = ExecuteStatementOperation - .newExecuteStatementOperation(parentSession, statement, confOverlay, runAsync); - addOperation(executeStatementOperation); - return executeStatementOperation; - } - - public GetTypeInfoOperation newGetTypeInfoOperation(HiveSession parentSession) { - GetTypeInfoOperation operation = new GetTypeInfoOperation(parentSession); - addOperation(operation); - return operation; - } - - public GetCatalogsOperation newGetCatalogsOperation(HiveSession parentSession) { - GetCatalogsOperation operation = new GetCatalogsOperation(parentSession); - addOperation(operation); - return operation; - } - - public GetSchemasOperation newGetSchemasOperation(HiveSession parentSession, - String catalogName, String schemaName) { - GetSchemasOperation operation = new GetSchemasOperation(parentSession, catalogName, schemaName); - addOperation(operation); - return operation; - } - - public MetadataOperation newGetTablesOperation(HiveSession parentSession, - String catalogName, String schemaName, String tableName, - List tableTypes) { - MetadataOperation operation = - new GetTablesOperation(parentSession, catalogName, schemaName, tableName, tableTypes); - addOperation(operation); - return operation; - } - - public GetTableTypesOperation newGetTableTypesOperation(HiveSession parentSession) { - GetTableTypesOperation operation = new GetTableTypesOperation(parentSession); - addOperation(operation); - return operation; - } - - public GetColumnsOperation newGetColumnsOperation(HiveSession parentSession, - String catalogName, String schemaName, String tableName, String columnName) { - GetColumnsOperation operation = new GetColumnsOperation(parentSession, - catalogName, schemaName, tableName, columnName); - addOperation(operation); - return operation; - } - - public GetFunctionsOperation newGetFunctionsOperation(HiveSession parentSession, - String catalogName, String schemaName, String functionName) { - GetFunctionsOperation operation = new GetFunctionsOperation(parentSession, - catalogName, schemaName, functionName); - addOperation(operation); - return operation; - } - - public Operation getOperation(OperationHandle operationHandle) throws HiveSQLException { - Operation operation = getOperationInternal(operationHandle); - if (operation == null) { - throw new HiveSQLException("Invalid OperationHandle: " + operationHandle); - } - return operation; - } - - private synchronized Operation getOperationInternal(OperationHandle operationHandle) { - return handleToOperation.get(operationHandle); - } - - private synchronized Operation removeTimedOutOperation(OperationHandle operationHandle) { - Operation operation = handleToOperation.get(operationHandle); - if (operation != null && operation.isTimedOut(System.currentTimeMillis())) { - handleToOperation.remove(operationHandle); - return operation; - } - return null; - } - - private synchronized void addOperation(Operation operation) { - handleToOperation.put(operation.getHandle(), operation); - } - - private synchronized Operation removeOperation(OperationHandle opHandle) { - return handleToOperation.remove(opHandle); - } - - public OperationStatus getOperationStatus(OperationHandle opHandle) - throws HiveSQLException { - return getOperation(opHandle).getStatus(); - } - - public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { - Operation operation = getOperation(opHandle); - OperationState opState = operation.getStatus().getState(); - if (opState == OperationState.CANCELED || - opState == OperationState.CLOSED || - opState == OperationState.FINISHED || - opState == OperationState.ERROR || - opState == OperationState.UNKNOWN) { - // Cancel should be a no-op in either cases - LOG.debug(opHandle + ": Operation is already aborted in state - " + opState); - } - else { - LOG.debug(opHandle + ": Attempting to cancel from state - " + opState); - operation.cancel(); - } - } - - public void closeOperation(OperationHandle opHandle) throws HiveSQLException { - Operation operation = removeOperation(opHandle); - if (operation == null) { - throw new HiveSQLException("Operation does not exist!"); - } - operation.close(); - } - - public TableSchema getOperationResultSetSchema(OperationHandle opHandle) - throws HiveSQLException { - return getOperation(opHandle).getResultSetSchema(); - } - - public RowSet getOperationNextRowSet(OperationHandle opHandle) - throws HiveSQLException { - return getOperation(opHandle).getNextRowSet(); - } - - public RowSet getOperationNextRowSet(OperationHandle opHandle, - FetchOrientation orientation, long maxRows) - throws HiveSQLException { - return getOperation(opHandle).getNextRowSet(orientation, maxRows); - } - - public RowSet getOperationLogRowSet(OperationHandle opHandle, - FetchOrientation orientation, long maxRows) - throws HiveSQLException { - // get the OperationLog object from the operation - OperationLog operationLog = getOperation(opHandle).getOperationLog(); - if (operationLog == null) { - throw new HiveSQLException("Couldn't find log associated with operation handle: " + opHandle); - } - - // read logs - List logs; - try { - logs = operationLog.readOperationLog(isFetchFirst(orientation), maxRows); - } catch (SQLException e) { - throw new HiveSQLException(e.getMessage(), e.getCause()); - } - - - // convert logs to RowSet - TableSchema tableSchema = new TableSchema(getLogSchema()); - RowSet rowSet = RowSetFactory.create(tableSchema, getOperation(opHandle).getProtocolVersion()); - for (String log : logs) { - rowSet.addRow(new String[] {log}); - } - - return rowSet; - } - - private boolean isFetchFirst(FetchOrientation fetchOrientation) { - //TODO: Since OperationLog is moved to package o.a.h.h.ql.session, - // we may add a Enum there and map FetchOrientation to it. - if (fetchOrientation.equals(FetchOrientation.FETCH_FIRST)) { - return true; - } - return false; - } - - private Schema getLogSchema() { - Schema schema = new Schema(); - FieldSchema fieldSchema = new FieldSchema(); - fieldSchema.setName("operation_log"); - fieldSchema.setType("string"); - schema.addToFieldSchemas(fieldSchema); - return schema; - } - - public OperationLog getOperationLogByThread() { - return OperationLog.getCurrentOperationLog(); - } - - public List removeExpiredOperations(OperationHandle[] handles) { - List removed = new ArrayList(); - for (OperationHandle handle : handles) { - Operation operation = removeTimedOutOperation(handle); - if (operation != null) { - LOG.warn("Operation " + handle + " is timed-out and will be closed"); - removed.add(operation); - } - } - return removed; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java deleted file mode 100644 index c7726f1fac07a..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java +++ /dev/null @@ -1,456 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.operation; - -import java.io.IOException; -import java.io.Serializable; -import java.nio.charset.StandardCharsets; -import java.security.PrivilegedExceptionAction; -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.concurrent.Future; -import java.util.concurrent.RejectedExecutionException; - -import org.apache.commons.codec.binary.Base64; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.Schema; -import org.apache.hadoop.hive.ql.CommandNeedRetryException; -import org.apache.hadoop.hive.ql.Driver; -import org.apache.hadoop.hive.ql.exec.ExplainTask; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.parse.VariableSubstitution; -import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.hive.serde.serdeConstants; -import org.apache.hadoop.hive.serde2.SerDe; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.SerDeUtils; -import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.StructField; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.hive.shims.Utils; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; -import org.apache.hive.service.server.ThreadWithGarbageCleanup; - -/** - * SQLOperation. - * - */ -public class SQLOperation extends ExecuteStatementOperation { - - private Driver driver = null; - private CommandProcessorResponse response; - private TableSchema resultSchema = null; - private Schema mResultSchema = null; - private SerDe serde = null; - private boolean fetchStarted = false; - - public SQLOperation(HiveSession parentSession, String statement, Map confOverlay, boolean runInBackground) { - // TODO: call setRemoteUser in ExecuteStatementOperation or higher. - super(parentSession, statement, confOverlay, runInBackground); - } - - /*** - * Compile the query and extract metadata - * @param sqlOperationConf - * @throws HiveSQLException - */ - public void prepare(HiveConf sqlOperationConf) throws HiveSQLException { - setState(OperationState.RUNNING); - - try { - driver = new Driver(sqlOperationConf, getParentSession().getUserName()); - - // set the operation handle information in Driver, so that thrift API users - // can use the operation handle they receive, to lookup query information in - // Yarn ATS - String guid64 = Base64.encodeBase64URLSafeString(getHandle().getHandleIdentifier() - .toTHandleIdentifier().getGuid()).trim(); - driver.setOperationId(guid64); - - // In Hive server mode, we are not able to retry in the FetchTask - // case, when calling fetch queries since execute() has returned. - // For now, we disable the test attempts. - driver.setTryCount(Integer.MAX_VALUE); - - String subStatement = new VariableSubstitution().substitute(sqlOperationConf, statement); - response = driver.compileAndRespond(subStatement); - if (0 != response.getResponseCode()) { - throw toSQLException("Error while compiling statement", response); - } - - mResultSchema = driver.getSchema(); - - // hasResultSet should be true only if the query has a FetchTask - // "explain" is an exception for now - if(driver.getPlan().getFetchTask() != null) { - //Schema has to be set - if (mResultSchema == null || !mResultSchema.isSetFieldSchemas()) { - throw new HiveSQLException("Error compiling query: Schema and FieldSchema " + - "should be set when query plan has a FetchTask"); - } - resultSchema = new TableSchema(mResultSchema); - setHasResultSet(true); - } else { - setHasResultSet(false); - } - // Set hasResultSet true if the plan has ExplainTask - // TODO explain should use a FetchTask for reading - for (Task task: driver.getPlan().getRootTasks()) { - if (task.getClass() == ExplainTask.class) { - resultSchema = new TableSchema(mResultSchema); - setHasResultSet(true); - break; - } - } - } catch (HiveSQLException e) { - setState(OperationState.ERROR); - throw e; - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException("Error running query: " + e.toString(), e); - } - } - - private void runQuery(HiveConf sqlOperationConf) throws HiveSQLException { - try { - // In Hive server mode, we are not able to retry in the FetchTask - // case, when calling fetch queries since execute() has returned. - // For now, we disable the test attempts. - driver.setTryCount(Integer.MAX_VALUE); - response = driver.run(); - if (0 != response.getResponseCode()) { - throw toSQLException("Error while processing statement", response); - } - } catch (HiveSQLException e) { - // If the operation was cancelled by another thread, - // Driver#run will return a non-zero response code. - // We will simply return if the operation state is CANCELED, - // otherwise throw an exception - if (getStatus().getState() == OperationState.CANCELED) { - return; - } - else { - setState(OperationState.ERROR); - throw e; - } - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException("Error running query: " + e.toString(), e); - } - setState(OperationState.FINISHED); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.PENDING); - final HiveConf opConfig = getConfigForOperation(); - prepare(opConfig); - if (!shouldRunAsync()) { - runQuery(opConfig); - } else { - // We'll pass ThreadLocals in the background thread from the foreground (handler) thread - final SessionState parentSessionState = SessionState.get(); - // ThreadLocal Hive object needs to be set in background thread. - // The metastore client in Hive is associated with right user. - final Hive parentHive = getSessionHive(); - // Current UGI will get used by metastore when metsatore is in embedded mode - // So this needs to get passed to the new background thread - final UserGroupInformation currentUGI = getCurrentUGI(opConfig); - // Runnable impl to call runInternal asynchronously, - // from a different thread - Runnable backgroundOperation = new Runnable() { - @Override - public void run() { - PrivilegedExceptionAction doAsAction = new PrivilegedExceptionAction() { - @Override - public Object run() throws HiveSQLException { - Hive.set(parentHive); - SessionState.setCurrentSessionState(parentSessionState); - // Set current OperationLog in this async thread for keeping on saving query log. - registerCurrentOperationLog(); - try { - runQuery(opConfig); - } catch (HiveSQLException e) { - setOperationException(e); - LOG.error("Error running hive query: ", e); - } finally { - unregisterOperationLog(); - } - return null; - } - }; - - try { - currentUGI.doAs(doAsAction); - } catch (Exception e) { - setOperationException(new HiveSQLException(e)); - LOG.error("Error running hive query as user : " + currentUGI.getShortUserName(), e); - } - finally { - /** - * We'll cache the ThreadLocal RawStore object for this background thread for an orderly cleanup - * when this thread is garbage collected later. - * @see org.apache.hive.service.server.ThreadWithGarbageCleanup#finalize() - */ - if (ThreadWithGarbageCleanup.currentThread() instanceof ThreadWithGarbageCleanup) { - ThreadWithGarbageCleanup currentThread = - (ThreadWithGarbageCleanup) ThreadWithGarbageCleanup.currentThread(); - currentThread.cacheThreadLocalRawStore(); - } - } - } - }; - try { - // This submit blocks if no background threads are available to run this operation - Future backgroundHandle = - getParentSession().getSessionManager().submitBackgroundOperation(backgroundOperation); - setBackgroundHandle(backgroundHandle); - } catch (RejectedExecutionException rejected) { - setState(OperationState.ERROR); - throw new HiveSQLException("The background threadpool cannot accept" + - " new task for execution, please retry the operation", rejected); - } - } - } - - /** - * Returns the current UGI on the stack - * @param opConfig - * @return UserGroupInformation - * @throws HiveSQLException - */ - private UserGroupInformation getCurrentUGI(HiveConf opConfig) throws HiveSQLException { - try { - return Utils.getUGI(); - } catch (Exception e) { - throw new HiveSQLException("Unable to get current user", e); - } - } - - /** - * Returns the ThreadLocal Hive for the current thread - * @return Hive - * @throws HiveSQLException - */ - private Hive getSessionHive() throws HiveSQLException { - try { - return Hive.get(); - } catch (HiveException e) { - throw new HiveSQLException("Failed to get ThreadLocal Hive object", e); - } - } - - private void cleanup(OperationState state) throws HiveSQLException { - setState(state); - if (shouldRunAsync()) { - Future backgroundHandle = getBackgroundHandle(); - if (backgroundHandle != null) { - backgroundHandle.cancel(true); - } - } - if (driver != null) { - driver.close(); - driver.destroy(); - } - driver = null; - - SessionState ss = SessionState.get(); - if (ss.getTmpOutputFile() != null) { - ss.getTmpOutputFile().delete(); - } - } - - @Override - public void cancel() throws HiveSQLException { - cleanup(OperationState.CANCELED); - } - - @Override - public void close() throws HiveSQLException { - cleanup(OperationState.CLOSED); - cleanupOperationLog(); - } - - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - assertState(OperationState.FINISHED); - if (resultSchema == null) { - resultSchema = new TableSchema(driver.getSchema()); - } - return resultSchema; - } - - private final transient List convey = new ArrayList(); - - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - validateDefaultFetchOrientation(orientation); - assertState(OperationState.FINISHED); - - RowSet rowSet = RowSetFactory.create(resultSchema, getProtocolVersion()); - - try { - /* if client is requesting fetch-from-start and its not the first time reading from this operation - * then reset the fetch position to beginning - */ - if (orientation.equals(FetchOrientation.FETCH_FIRST) && fetchStarted) { - driver.resetFetch(); - } - fetchStarted = true; - driver.setMaxRows((int) maxRows); - if (driver.getResults(convey)) { - return decode(convey, rowSet); - } - return rowSet; - } catch (IOException e) { - throw new HiveSQLException(e); - } catch (CommandNeedRetryException e) { - throw new HiveSQLException(e); - } catch (Exception e) { - throw new HiveSQLException(e); - } finally { - convey.clear(); - } - } - - private RowSet decode(List rows, RowSet rowSet) throws Exception { - if (driver.isFetchingTable()) { - return prepareFromRow(rows, rowSet); - } - return decodeFromString(rows, rowSet); - } - - // already encoded to thrift-able object in ThriftFormatter - private RowSet prepareFromRow(List rows, RowSet rowSet) throws Exception { - for (Object row : rows) { - rowSet.addRow((Object[]) row); - } - return rowSet; - } - - private RowSet decodeFromString(List rows, RowSet rowSet) - throws SQLException, SerDeException { - getSerDe(); - StructObjectInspector soi = (StructObjectInspector) serde.getObjectInspector(); - List fieldRefs = soi.getAllStructFieldRefs(); - - Object[] deserializedFields = new Object[fieldRefs.size()]; - Object rowObj; - ObjectInspector fieldOI; - - int protocol = getProtocolVersion().getValue(); - for (Object rowString : rows) { - rowObj = serde.deserialize(new BytesWritable(((String)rowString).getBytes(StandardCharsets.UTF_8))); - for (int i = 0; i < fieldRefs.size(); i++) { - StructField fieldRef = fieldRefs.get(i); - fieldOI = fieldRef.getFieldObjectInspector(); - Object fieldData = soi.getStructFieldData(rowObj, fieldRef); - deserializedFields[i] = SerDeUtils.toThriftPayload(fieldData, fieldOI, protocol); - } - rowSet.addRow(deserializedFields); - } - return rowSet; - } - - private SerDe getSerDe() throws SQLException { - if (serde != null) { - return serde; - } - try { - List fieldSchemas = mResultSchema.getFieldSchemas(); - StringBuilder namesSb = new StringBuilder(); - StringBuilder typesSb = new StringBuilder(); - - if (fieldSchemas != null && !fieldSchemas.isEmpty()) { - for (int pos = 0; pos < fieldSchemas.size(); pos++) { - if (pos != 0) { - namesSb.append(","); - typesSb.append(","); - } - namesSb.append(fieldSchemas.get(pos).getName()); - typesSb.append(fieldSchemas.get(pos).getType()); - } - } - String names = namesSb.toString(); - String types = typesSb.toString(); - - serde = new LazySimpleSerDe(); - Properties props = new Properties(); - if (names.length() > 0) { - LOG.debug("Column names: " + names); - props.setProperty(serdeConstants.LIST_COLUMNS, names); - } - if (types.length() > 0) { - LOG.debug("Column types: " + types); - props.setProperty(serdeConstants.LIST_COLUMN_TYPES, types); - } - SerDeUtils.initializeSerDe(serde, new HiveConf(), props, null); - - } catch (Exception ex) { - ex.printStackTrace(); - throw new SQLException("Could not create ResultSet: " + ex.getMessage(), ex); - } - return serde; - } - - /** - * If there are query specific settings to overlay, then create a copy of config - * There are two cases we need to clone the session config that's being passed to hive driver - * 1. Async query - - * If the client changes a config setting, that shouldn't reflect in the execution already underway - * 2. confOverlay - - * The query specific settings should only be applied to the query config and not session - * @return new configuration - * @throws HiveSQLException - */ - private HiveConf getConfigForOperation() throws HiveSQLException { - HiveConf sqlOperationConf = getParentSession().getHiveConf(); - if (!getConfOverlay().isEmpty() || shouldRunAsync()) { - // clone the parent session config for this query - sqlOperationConf = new HiveConf(sqlOperationConf); - - // apply overlay query specific settings, if any - for (Map.Entry confEntry : getConfOverlay().entrySet()) { - try { - sqlOperationConf.verifyAndSet(confEntry.getKey(), confEntry.getValue()); - } catch (IllegalArgumentException e) { - throw new HiveSQLException("Error applying statement specific settings", e); - } - } - } - return sqlOperationConf; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java deleted file mode 100644 index e59d19ea6be42..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java +++ /dev/null @@ -1,44 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.operation; - -import java.util.Set; - - -public interface TableTypeMapping { - /** - * Map client's table type name to hive's table type - * @param clientTypeName - * @return - */ - String mapToHiveType(String clientTypeName); - - /** - * Map hive's table type name to client's table type - * @param hiveTypeName - * @return - */ - String mapToClientType(String hiveTypeName); - - /** - * Get all the table types of this mapping - * @return - */ - Set getTableTypeNames(); -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSession.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSession.java deleted file mode 100644 index 65f9b298bf4f6..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSession.java +++ /dev/null @@ -1,156 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.session; - -import java.util.List; -import java.util.Map; - -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.cli.*; - -public interface HiveSession extends HiveSessionBase { - - void open(Map sessionConfMap) throws Exception; - - IMetaStoreClient getMetaStoreClient() throws HiveSQLException; - - /** - * getInfo operation handler - * @param getInfoType - * @return - * @throws HiveSQLException - */ - GetInfoValue getInfo(GetInfoType getInfoType) throws HiveSQLException; - - /** - * execute operation handler - * @param statement - * @param confOverlay - * @return - * @throws HiveSQLException - */ - OperationHandle executeStatement(String statement, - Map confOverlay) throws HiveSQLException; - - /** - * execute operation handler - * @param statement - * @param confOverlay - * @return - * @throws HiveSQLException - */ - OperationHandle executeStatementAsync(String statement, - Map confOverlay) throws HiveSQLException; - - /** - * getTypeInfo operation handler - * @return - * @throws HiveSQLException - */ - OperationHandle getTypeInfo() throws HiveSQLException; - - /** - * getCatalogs operation handler - * @return - * @throws HiveSQLException - */ - OperationHandle getCatalogs() throws HiveSQLException; - - /** - * getSchemas operation handler - * @param catalogName - * @param schemaName - * @return - * @throws HiveSQLException - */ - OperationHandle getSchemas(String catalogName, String schemaName) - throws HiveSQLException; - - /** - * getTables operation handler - * @param catalogName - * @param schemaName - * @param tableName - * @param tableTypes - * @return - * @throws HiveSQLException - */ - OperationHandle getTables(String catalogName, String schemaName, - String tableName, List tableTypes) throws HiveSQLException; - - /** - * getTableTypes operation handler - * @return - * @throws HiveSQLException - */ - OperationHandle getTableTypes() throws HiveSQLException ; - - /** - * getColumns operation handler - * @param catalogName - * @param schemaName - * @param tableName - * @param columnName - * @return - * @throws HiveSQLException - */ - OperationHandle getColumns(String catalogName, String schemaName, - String tableName, String columnName) throws HiveSQLException; - - /** - * getFunctions operation handler - * @param catalogName - * @param schemaName - * @param functionName - * @return - * @throws HiveSQLException - */ - OperationHandle getFunctions(String catalogName, String schemaName, - String functionName) throws HiveSQLException; - - /** - * close the session - * @throws HiveSQLException - */ - void close() throws HiveSQLException; - - void cancelOperation(OperationHandle opHandle) throws HiveSQLException; - - void closeOperation(OperationHandle opHandle) throws HiveSQLException; - - TableSchema getResultSetMetadata(OperationHandle opHandle) - throws HiveSQLException; - - RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, - long maxRows, FetchType fetchType) throws HiveSQLException; - - String getDelegationToken(HiveAuthFactory authFactory, String owner, - String renewer) throws HiveSQLException; - - void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr) - throws HiveSQLException; - - void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr) - throws HiveSQLException; - - void closeExpiredOperations(); - - long getNoOperationTime(); -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java deleted file mode 100644 index b72c18b2b2135..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java +++ /dev/null @@ -1,90 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.session; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hive.service.cli.SessionHandle; -import org.apache.hive.service.cli.operation.OperationManager; -import org.apache.hive.service.cli.thrift.TProtocolVersion; - -import java.io.File; - -/** - * Methods that don't need to be executed under a doAs - * context are here. Rest of them in HiveSession interface - */ -public interface HiveSessionBase { - - TProtocolVersion getProtocolVersion(); - - /** - * Set the session manager for the session - * @param sessionManager - */ - void setSessionManager(SessionManager sessionManager); - - /** - * Get the session manager for the session - */ - SessionManager getSessionManager(); - - /** - * Set operation manager for the session - * @param operationManager - */ - void setOperationManager(OperationManager operationManager); - - /** - * Check whether operation logging is enabled and session dir is created successfully - */ - boolean isOperationLogEnabled(); - - /** - * Get the session dir, which is the parent dir of operation logs - * @return a file representing the parent directory of operation logs - */ - File getOperationLogSessionDir(); - - /** - * Set the session dir, which is the parent dir of operation logs - * @param operationLogRootDir the parent dir of the session dir - */ - void setOperationLogSessionDir(File operationLogRootDir); - - SessionHandle getSessionHandle(); - - String getUsername(); - - String getPassword(); - - HiveConf getHiveConf(); - - SessionState getSessionState(); - - String getUserName(); - - void setUserName(String userName); - - String getIpAddress(); - - void setIpAddress(String ipAddress); - - long getLastAccessTime(); -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java deleted file mode 100644 index e3fb54d9f47e9..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java +++ /dev/null @@ -1,842 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.session; - -import java.io.BufferedReader; -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStreamReader; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.commons.io.FileUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.common.cli.HiveFileProcessor; -import org.apache.hadoop.hive.common.cli.IHiveFileProcessor; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.ql.exec.FetchFormatter; -import org.apache.hadoop.hive.ql.exec.ListSinkOperator; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.history.HiveHistory; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.parse.VariableSubstitution; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hive.common.util.HiveVersionInfo; -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.FetchType; -import org.apache.hive.service.cli.GetInfoType; -import org.apache.hive.service.cli.GetInfoValue; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationHandle; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.SessionHandle; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.operation.ExecuteStatementOperation; -import org.apache.hive.service.cli.operation.GetCatalogsOperation; -import org.apache.hive.service.cli.operation.GetColumnsOperation; -import org.apache.hive.service.cli.operation.GetFunctionsOperation; -import org.apache.hive.service.cli.operation.GetSchemasOperation; -import org.apache.hive.service.cli.operation.GetTableTypesOperation; -import org.apache.hive.service.cli.operation.GetTypeInfoOperation; -import org.apache.hive.service.cli.operation.MetadataOperation; -import org.apache.hive.service.cli.operation.Operation; -import org.apache.hive.service.cli.operation.OperationManager; -import org.apache.hive.service.cli.thrift.TProtocolVersion; -import org.apache.hive.service.server.ThreadWithGarbageCleanup; - -import static org.apache.hadoop.hive.conf.SystemVariables.ENV_PREFIX; -import static org.apache.hadoop.hive.conf.SystemVariables.HIVECONF_PREFIX; -import static org.apache.hadoop.hive.conf.SystemVariables.HIVEVAR_PREFIX; -import static org.apache.hadoop.hive.conf.SystemVariables.METACONF_PREFIX; -import static org.apache.hadoop.hive.conf.SystemVariables.SYSTEM_PREFIX; - -/** - * HiveSession - * - */ -public class HiveSessionImpl implements HiveSession { - private final SessionHandle sessionHandle; - private String username; - private final String password; - private HiveConf hiveConf; - private SessionState sessionState; - private String ipAddress; - private static final String FETCH_WORK_SERDE_CLASS = - "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"; - private static final Log LOG = LogFactory.getLog(HiveSessionImpl.class); - private SessionManager sessionManager; - private OperationManager operationManager; - private final Set opHandleSet = new HashSet(); - private boolean isOperationLogEnabled; - private File sessionLogDir; - private volatile long lastAccessTime; - private volatile long lastIdleTime; - - public HiveSessionImpl(TProtocolVersion protocol, String username, String password, - HiveConf serverhiveConf, String ipAddress) { - this.username = username; - this.password = password; - this.sessionHandle = new SessionHandle(protocol); - this.hiveConf = new HiveConf(serverhiveConf); - this.ipAddress = ipAddress; - - try { - // In non-impersonation mode, map scheduler queue to current user - // if fair scheduler is configured. - if (! hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && - hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_MAP_FAIR_SCHEDULER_QUEUE)) { - ShimLoader.getHadoopShims().refreshDefaultQueue(hiveConf, username); - } - } catch (IOException e) { - LOG.warn("Error setting scheduler queue: " + e, e); - } - // Set an explicit session name to control the download directory name - hiveConf.set(ConfVars.HIVESESSIONID.varname, - sessionHandle.getHandleIdentifier().toString()); - // Use thrift transportable formatter - hiveConf.set(ListSinkOperator.OUTPUT_FORMATTER, - FetchFormatter.ThriftFormatter.class.getName()); - hiveConf.setInt(ListSinkOperator.OUTPUT_PROTOCOL, protocol.getValue()); - } - - @Override - /** - * Opens a new HiveServer2 session for the client connection. - * Creates a new SessionState object that will be associated with this HiveServer2 session. - * When the server executes multiple queries in the same session, - * this SessionState object is reused across multiple queries. - * Note that if doAs is true, this call goes through a proxy object, - * which wraps the method logic in a UserGroupInformation#doAs. - * That's why it is important to create SessionState here rather than in the constructor. - */ - public void open(Map sessionConfMap) throws HiveSQLException { - sessionState = new SessionState(hiveConf, username); - sessionState.setUserIpAddress(ipAddress); - sessionState.setIsHiveServerQuery(true); - SessionState.start(sessionState); - try { - sessionState.reloadAuxJars(); - } catch (IOException e) { - String msg = "Failed to load reloadable jar file path: " + e; - LOG.error(msg, e); - throw new HiveSQLException(msg, e); - } - // Process global init file: .hiverc - processGlobalInitFile(); - if (sessionConfMap != null) { - configureSession(sessionConfMap); - } - lastAccessTime = System.currentTimeMillis(); - lastIdleTime = lastAccessTime; - } - - /** - * It is used for processing hiverc file from HiveServer2 side. - */ - private class GlobalHivercFileProcessor extends HiveFileProcessor { - @Override - protected BufferedReader loadFile(String fileName) throws IOException { - FileInputStream initStream = null; - BufferedReader bufferedReader = null; - initStream = new FileInputStream(fileName); - bufferedReader = new BufferedReader(new InputStreamReader(initStream)); - return bufferedReader; - } - - @Override - protected int processCmd(String cmd) { - int rc = 0; - String cmd_trimed = cmd.trim(); - try { - executeStatementInternal(cmd_trimed, null, false); - } catch (HiveSQLException e) { - rc = -1; - LOG.warn("Failed to execute HQL command in global .hiverc file.", e); - } - return rc; - } - } - - private void processGlobalInitFile() { - IHiveFileProcessor processor = new GlobalHivercFileProcessor(); - - try { - String hiverc = hiveConf.getVar(ConfVars.HIVE_SERVER2_GLOBAL_INIT_FILE_LOCATION); - if (hiverc != null) { - File hivercFile = new File(hiverc); - if (hivercFile.isDirectory()) { - hivercFile = new File(hivercFile, SessionManager.HIVERCFILE); - } - if (hivercFile.isFile()) { - LOG.info("Running global init file: " + hivercFile); - int rc = processor.processFile(hivercFile.getAbsolutePath()); - if (rc != 0) { - LOG.error("Failed on initializing global .hiverc file"); - } - } else { - LOG.debug("Global init file " + hivercFile + " does not exist"); - } - } - } catch (IOException e) { - LOG.warn("Failed on initializing global .hiverc file", e); - } - } - - private void configureSession(Map sessionConfMap) throws HiveSQLException { - SessionState.setCurrentSessionState(sessionState); - for (Map.Entry entry : sessionConfMap.entrySet()) { - String key = entry.getKey(); - if (key.startsWith("set:")) { - try { - setVariable(key.substring(4), entry.getValue()); - } catch (Exception e) { - throw new HiveSQLException(e); - } - } else if (key.startsWith("use:")) { - SessionState.get().setCurrentDatabase(entry.getValue()); - } else { - hiveConf.verifyAndSet(key, entry.getValue()); - } - } - } - - // Copy from org.apache.hadoop.hive.ql.processors.SetProcessor, only change: - // setConf(varname, propName, varvalue, true) when varname.startsWith(HIVECONF_PREFIX) - public static int setVariable(String varname, String varvalue) throws Exception { - SessionState ss = SessionState.get(); - if (varvalue.contains("\n")){ - ss.err.println("Warning: Value had a \\n character in it."); - } - varname = varname.trim(); - if (varname.startsWith(ENV_PREFIX)){ - ss.err.println("env:* variables can not be set."); - return 1; - } else if (varname.startsWith(SYSTEM_PREFIX)){ - String propName = varname.substring(SYSTEM_PREFIX.length()); - System.getProperties().setProperty(propName, - new VariableSubstitution().substitute(ss.getConf(),varvalue)); - } else if (varname.startsWith(HIVECONF_PREFIX)){ - String propName = varname.substring(HIVECONF_PREFIX.length()); - setConf(varname, propName, varvalue, true); - } else if (varname.startsWith(HIVEVAR_PREFIX)) { - String propName = varname.substring(HIVEVAR_PREFIX.length()); - ss.getHiveVariables().put(propName, - new VariableSubstitution().substitute(ss.getConf(),varvalue)); - } else if (varname.startsWith(METACONF_PREFIX)) { - String propName = varname.substring(METACONF_PREFIX.length()); - Hive hive = Hive.get(ss.getConf()); - hive.setMetaConf(propName, new VariableSubstitution().substitute(ss.getConf(), varvalue)); - } else { - setConf(varname, varname, varvalue, true); - } - return 0; - } - - // returns non-null string for validation fail - private static void setConf(String varname, String key, String varvalue, boolean register) - throws IllegalArgumentException { - HiveConf conf = SessionState.get().getConf(); - String value = new VariableSubstitution().substitute(conf, varvalue); - if (conf.getBoolVar(HiveConf.ConfVars.HIVECONFVALIDATION)) { - HiveConf.ConfVars confVars = HiveConf.getConfVars(key); - if (confVars != null) { - if (!confVars.isType(value)) { - StringBuilder message = new StringBuilder(); - message.append("'SET ").append(varname).append('=').append(varvalue); - message.append("' FAILED because ").append(key).append(" expects "); - message.append(confVars.typeString()).append(" type value."); - throw new IllegalArgumentException(message.toString()); - } - String fail = confVars.validate(value); - if (fail != null) { - StringBuilder message = new StringBuilder(); - message.append("'SET ").append(varname).append('=').append(varvalue); - message.append("' FAILED in validation : ").append(fail).append('.'); - throw new IllegalArgumentException(message.toString()); - } - } else if (key.startsWith("hive.")) { - throw new IllegalArgumentException("hive configuration " + key + " does not exists."); - } - } - conf.verifyAndSet(key, value); - if (register) { - SessionState.get().getOverriddenConfigurations().put(key, value); - } - } - - @Override - public void setOperationLogSessionDir(File operationLogRootDir) { - if (!operationLogRootDir.exists()) { - LOG.warn("The operation log root directory is removed, recreating: " + - operationLogRootDir.getAbsolutePath()); - if (!operationLogRootDir.mkdirs()) { - LOG.warn("Unable to create operation log root directory: " + - operationLogRootDir.getAbsolutePath()); - } - } - if (!operationLogRootDir.canWrite()) { - LOG.warn("The operation log root directory is not writable: " + - operationLogRootDir.getAbsolutePath()); - } - sessionLogDir = new File(operationLogRootDir, sessionHandle.getHandleIdentifier().toString()); - isOperationLogEnabled = true; - if (!sessionLogDir.exists()) { - if (!sessionLogDir.mkdir()) { - LOG.warn("Unable to create operation log session directory: " + - sessionLogDir.getAbsolutePath()); - isOperationLogEnabled = false; - } - } - if (isOperationLogEnabled) { - LOG.info("Operation log session directory is created: " + sessionLogDir.getAbsolutePath()); - } - } - - @Override - public boolean isOperationLogEnabled() { - return isOperationLogEnabled; - } - - @Override - public File getOperationLogSessionDir() { - return sessionLogDir; - } - - @Override - public TProtocolVersion getProtocolVersion() { - return sessionHandle.getProtocolVersion(); - } - - @Override - public SessionManager getSessionManager() { - return sessionManager; - } - - @Override - public void setSessionManager(SessionManager sessionManager) { - this.sessionManager = sessionManager; - } - - private OperationManager getOperationManager() { - return operationManager; - } - - @Override - public void setOperationManager(OperationManager operationManager) { - this.operationManager = operationManager; - } - - protected synchronized void acquire(boolean userAccess) { - // Need to make sure that the this HiveServer2's session's SessionState is - // stored in the thread local for the handler thread. - SessionState.setCurrentSessionState(sessionState); - if (userAccess) { - lastAccessTime = System.currentTimeMillis(); - } - } - - /** - * 1. We'll remove the ThreadLocal SessionState as this thread might now serve - * other requests. - * 2. We'll cache the ThreadLocal RawStore object for this background thread for an orderly cleanup - * when this thread is garbage collected later. - * @see org.apache.hive.service.server.ThreadWithGarbageCleanup#finalize() - */ - protected synchronized void release(boolean userAccess) { - SessionState.detachSession(); - if (ThreadWithGarbageCleanup.currentThread() instanceof ThreadWithGarbageCleanup) { - ThreadWithGarbageCleanup currentThread = - (ThreadWithGarbageCleanup) ThreadWithGarbageCleanup.currentThread(); - currentThread.cacheThreadLocalRawStore(); - } - if (userAccess) { - lastAccessTime = System.currentTimeMillis(); - } - if (opHandleSet.isEmpty()) { - lastIdleTime = System.currentTimeMillis(); - } else { - lastIdleTime = 0; - } - } - - @Override - public SessionHandle getSessionHandle() { - return sessionHandle; - } - - @Override - public String getUsername() { - return username; - } - - @Override - public String getPassword() { - return password; - } - - @Override - public HiveConf getHiveConf() { - hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHOUTPUTSERDE, FETCH_WORK_SERDE_CLASS); - return hiveConf; - } - - @Override - public IMetaStoreClient getMetaStoreClient() throws HiveSQLException { - try { - return Hive.get(getHiveConf()).getMSC(); - } catch (HiveException e) { - throw new HiveSQLException("Failed to get metastore connection", e); - } catch (MetaException e) { - throw new HiveSQLException("Failed to get metastore connection", e); - } - } - - @Override - public GetInfoValue getInfo(GetInfoType getInfoType) - throws HiveSQLException { - acquire(true); - try { - switch (getInfoType) { - case CLI_SERVER_NAME: - return new GetInfoValue("Hive"); - case CLI_DBMS_NAME: - return new GetInfoValue("Apache Hive"); - case CLI_DBMS_VER: - return new GetInfoValue(HiveVersionInfo.getVersion()); - case CLI_MAX_COLUMN_NAME_LEN: - return new GetInfoValue(128); - case CLI_MAX_SCHEMA_NAME_LEN: - return new GetInfoValue(128); - case CLI_MAX_TABLE_NAME_LEN: - return new GetInfoValue(128); - case CLI_TXN_CAPABLE: - default: - throw new HiveSQLException("Unrecognized GetInfoType value: " + getInfoType.toString()); - } - } finally { - release(true); - } - } - - @Override - public OperationHandle executeStatement(String statement, Map confOverlay) - throws HiveSQLException { - return executeStatementInternal(statement, confOverlay, false); - } - - @Override - public OperationHandle executeStatementAsync(String statement, Map confOverlay) - throws HiveSQLException { - return executeStatementInternal(statement, confOverlay, true); - } - - private OperationHandle executeStatementInternal(String statement, Map confOverlay, - boolean runAsync) - throws HiveSQLException { - acquire(true); - - OperationManager operationManager = getOperationManager(); - ExecuteStatementOperation operation = operationManager - .newExecuteStatementOperation(getSession(), statement, confOverlay, runAsync); - OperationHandle opHandle = operation.getHandle(); - try { - operation.run(); - opHandleSet.add(opHandle); - return opHandle; - } catch (HiveSQLException e) { - // Referring to SQLOperation.java, there is no chance that a HiveSQLException throws and the asyn - // background operation submits to thread pool successfully at the same time. So, Cleanup - // opHandle directly when got HiveSQLException - operationManager.closeOperation(opHandle); - throw e; - } finally { - release(true); - } - } - - @Override - public OperationHandle getTypeInfo() - throws HiveSQLException { - acquire(true); - - OperationManager operationManager = getOperationManager(); - GetTypeInfoOperation operation = operationManager.newGetTypeInfoOperation(getSession()); - OperationHandle opHandle = operation.getHandle(); - try { - operation.run(); - opHandleSet.add(opHandle); - return opHandle; - } catch (HiveSQLException e) { - operationManager.closeOperation(opHandle); - throw e; - } finally { - release(true); - } - } - - @Override - public OperationHandle getCatalogs() - throws HiveSQLException { - acquire(true); - - OperationManager operationManager = getOperationManager(); - GetCatalogsOperation operation = operationManager.newGetCatalogsOperation(getSession()); - OperationHandle opHandle = operation.getHandle(); - try { - operation.run(); - opHandleSet.add(opHandle); - return opHandle; - } catch (HiveSQLException e) { - operationManager.closeOperation(opHandle); - throw e; - } finally { - release(true); - } - } - - @Override - public OperationHandle getSchemas(String catalogName, String schemaName) - throws HiveSQLException { - acquire(true); - - OperationManager operationManager = getOperationManager(); - GetSchemasOperation operation = - operationManager.newGetSchemasOperation(getSession(), catalogName, schemaName); - OperationHandle opHandle = operation.getHandle(); - try { - operation.run(); - opHandleSet.add(opHandle); - return opHandle; - } catch (HiveSQLException e) { - operationManager.closeOperation(opHandle); - throw e; - } finally { - release(true); - } - } - - @Override - public OperationHandle getTables(String catalogName, String schemaName, String tableName, - List tableTypes) - throws HiveSQLException { - acquire(true); - - OperationManager operationManager = getOperationManager(); - MetadataOperation operation = - operationManager.newGetTablesOperation(getSession(), catalogName, schemaName, tableName, tableTypes); - OperationHandle opHandle = operation.getHandle(); - try { - operation.run(); - opHandleSet.add(opHandle); - return opHandle; - } catch (HiveSQLException e) { - operationManager.closeOperation(opHandle); - throw e; - } finally { - release(true); - } - } - - @Override - public OperationHandle getTableTypes() - throws HiveSQLException { - acquire(true); - - OperationManager operationManager = getOperationManager(); - GetTableTypesOperation operation = operationManager.newGetTableTypesOperation(getSession()); - OperationHandle opHandle = operation.getHandle(); - try { - operation.run(); - opHandleSet.add(opHandle); - return opHandle; - } catch (HiveSQLException e) { - operationManager.closeOperation(opHandle); - throw e; - } finally { - release(true); - } - } - - @Override - public OperationHandle getColumns(String catalogName, String schemaName, - String tableName, String columnName) throws HiveSQLException { - acquire(true); - String addedJars = Utilities.getResourceFiles(hiveConf, SessionState.ResourceType.JAR); - if (StringUtils.isNotBlank(addedJars)) { - IMetaStoreClient metastoreClient = getSession().getMetaStoreClient(); - metastoreClient.setHiveAddedJars(addedJars); - } - OperationManager operationManager = getOperationManager(); - GetColumnsOperation operation = operationManager.newGetColumnsOperation(getSession(), - catalogName, schemaName, tableName, columnName); - OperationHandle opHandle = operation.getHandle(); - try { - operation.run(); - opHandleSet.add(opHandle); - return opHandle; - } catch (HiveSQLException e) { - operationManager.closeOperation(opHandle); - throw e; - } finally { - release(true); - } - } - - @Override - public OperationHandle getFunctions(String catalogName, String schemaName, String functionName) - throws HiveSQLException { - acquire(true); - - OperationManager operationManager = getOperationManager(); - GetFunctionsOperation operation = operationManager - .newGetFunctionsOperation(getSession(), catalogName, schemaName, functionName); - OperationHandle opHandle = operation.getHandle(); - try { - operation.run(); - opHandleSet.add(opHandle); - return opHandle; - } catch (HiveSQLException e) { - operationManager.closeOperation(opHandle); - throw e; - } finally { - release(true); - } - } - - @Override - public void close() throws HiveSQLException { - try { - acquire(true); - // Iterate through the opHandles and close their operations - for (OperationHandle opHandle : opHandleSet) { - try { - operationManager.closeOperation(opHandle); - } catch (Exception e) { - LOG.warn("Exception is thrown closing operation " + opHandle, e); - } - } - opHandleSet.clear(); - // Cleanup session log directory. - cleanupSessionLogDir(); - // Cleanup pipeout file. - cleanupPipeoutFile(); - HiveHistory hiveHist = sessionState.getHiveHistory(); - if (null != hiveHist) { - hiveHist.closeStream(); - } - try { - sessionState.close(); - } finally { - sessionState = null; - } - } catch (IOException ioe) { - throw new HiveSQLException("Failure to close", ioe); - } finally { - if (sessionState != null) { - try { - sessionState.close(); - } catch (Throwable t) { - LOG.warn("Error closing session", t); - } - sessionState = null; - } - release(true); - } - } - - private void cleanupPipeoutFile() { - String lScratchDir = hiveConf.getVar(ConfVars.LOCALSCRATCHDIR); - String sessionID = hiveConf.getVar(ConfVars.HIVESESSIONID); - - File[] fileAry = new File(lScratchDir).listFiles( - (dir, name) -> name.startsWith(sessionID) && name.endsWith(".pipeout")); - - if (fileAry == null) { - LOG.error("Unable to access pipeout files in " + lScratchDir); - } else { - for (File file : fileAry) { - try { - FileUtils.forceDelete(file); - } catch (Exception e) { - LOG.error("Failed to cleanup pipeout file: " + file, e); - } - } - } - } - - private void cleanupSessionLogDir() { - if (isOperationLogEnabled) { - try { - FileUtils.forceDelete(sessionLogDir); - } catch (Exception e) { - LOG.error("Failed to cleanup session log dir: " + sessionHandle, e); - } - } - } - - @Override - public SessionState getSessionState() { - return sessionState; - } - - @Override - public String getUserName() { - return username; - } - - @Override - public void setUserName(String userName) { - this.username = userName; - } - - @Override - public long getLastAccessTime() { - return lastAccessTime; - } - - @Override - public void closeExpiredOperations() { - OperationHandle[] handles = opHandleSet.toArray(new OperationHandle[opHandleSet.size()]); - if (handles.length > 0) { - List operations = operationManager.removeExpiredOperations(handles); - if (!operations.isEmpty()) { - closeTimedOutOperations(operations); - } - } - } - - @Override - public long getNoOperationTime() { - return lastIdleTime > 0 ? System.currentTimeMillis() - lastIdleTime : 0; - } - - private void closeTimedOutOperations(List operations) { - acquire(false); - try { - for (Operation operation : operations) { - opHandleSet.remove(operation.getHandle()); - try { - operation.close(); - } catch (Exception e) { - LOG.warn("Exception is thrown closing timed-out operation " + operation.getHandle(), e); - } - } - } finally { - release(false); - } - } - - @Override - public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { - acquire(true); - try { - sessionManager.getOperationManager().cancelOperation(opHandle); - } finally { - release(true); - } - } - - @Override - public void closeOperation(OperationHandle opHandle) throws HiveSQLException { - acquire(true); - try { - operationManager.closeOperation(opHandle); - opHandleSet.remove(opHandle); - } finally { - release(true); - } - } - - @Override - public TableSchema getResultSetMetadata(OperationHandle opHandle) throws HiveSQLException { - acquire(true); - try { - return sessionManager.getOperationManager().getOperationResultSetSchema(opHandle); - } finally { - release(true); - } - } - - @Override - public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, - long maxRows, FetchType fetchType) throws HiveSQLException { - acquire(true); - try { - if (fetchType == FetchType.QUERY_OUTPUT) { - return operationManager.getOperationNextRowSet(opHandle, orientation, maxRows); - } - return operationManager.getOperationLogRowSet(opHandle, orientation, maxRows); - } finally { - release(true); - } - } - - protected HiveSession getSession() { - return this; - } - - @Override - public String getIpAddress() { - return ipAddress; - } - - @Override - public void setIpAddress(String ipAddress) { - this.ipAddress = ipAddress; - } - - @Override - public String getDelegationToken(HiveAuthFactory authFactory, String owner, String renewer) - throws HiveSQLException { - HiveAuthFactory.verifyProxyAccess(getUsername(), owner, getIpAddress(), getHiveConf()); - return authFactory.getDelegationToken(owner, renewer); - } - - @Override - public void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr) - throws HiveSQLException { - HiveAuthFactory.verifyProxyAccess(getUsername(), getUserFromToken(authFactory, tokenStr), - getIpAddress(), getHiveConf()); - authFactory.cancelDelegationToken(tokenStr); - } - - @Override - public void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr) - throws HiveSQLException { - HiveAuthFactory.verifyProxyAccess(getUsername(), getUserFromToken(authFactory, tokenStr), - getIpAddress(), getHiveConf()); - authFactory.renewDelegationToken(tokenStr); - } - - // extract the real user from the given token string - private String getUserFromToken(HiveAuthFactory authFactory, String tokenStr) throws HiveSQLException { - return authFactory.getUserFromToken(tokenStr); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java deleted file mode 100644 index 762dbb2faadec..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java +++ /dev/null @@ -1,182 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.session; - -import java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.shims.Utils; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.thrift.TProtocolVersion; - -/** - * - * HiveSessionImplwithUGI. - * HiveSession with connecting user's UGI and delegation token if required - */ -public class HiveSessionImplwithUGI extends HiveSessionImpl { - public static final String HS2TOKEN = "HiveServer2ImpersonationToken"; - - private UserGroupInformation sessionUgi = null; - private String delegationTokenStr = null; - private Hive sessionHive = null; - private HiveSession proxySession = null; - static final Log LOG = LogFactory.getLog(HiveSessionImplwithUGI.class); - - public HiveSessionImplwithUGI(TProtocolVersion protocol, String username, String password, - HiveConf hiveConf, String ipAddress, String delegationToken) throws HiveSQLException { - super(protocol, username, password, hiveConf, ipAddress); - setSessionUGI(username); - setDelegationToken(delegationToken); - - // create a new metastore connection for this particular user session - Hive.set(null); - try { - sessionHive = Hive.get(getHiveConf()); - } catch (HiveException e) { - throw new HiveSQLException("Failed to setup metastore connection", e); - } - } - - // setup appropriate UGI for the session - public void setSessionUGI(String owner) throws HiveSQLException { - if (owner == null) { - throw new HiveSQLException("No username provided for impersonation"); - } - if (UserGroupInformation.isSecurityEnabled()) { - try { - sessionUgi = UserGroupInformation.createProxyUser( - owner, UserGroupInformation.getLoginUser()); - } catch (IOException e) { - throw new HiveSQLException("Couldn't setup proxy user", e); - } - } else { - sessionUgi = UserGroupInformation.createRemoteUser(owner); - } - } - - public UserGroupInformation getSessionUgi() { - return this.sessionUgi; - } - - public String getDelegationToken() { - return this.delegationTokenStr; - } - - @Override - protected synchronized void acquire(boolean userAccess) { - super.acquire(userAccess); - // if we have a metastore connection with impersonation, then set it first - if (sessionHive != null) { - Hive.set(sessionHive); - } - } - - /** - * Close the file systems for the session and remove it from the FileSystem cache. - * Cancel the session's delegation token and close the metastore connection - */ - @Override - public void close() throws HiveSQLException { - try { - acquire(true); - cancelDelegationToken(); - } finally { - try { - super.close(); - } finally { - try { - FileSystem.closeAllForUGI(sessionUgi); - } catch (IOException ioe) { - throw new HiveSQLException("Could not clean up file-system handles for UGI: " - + sessionUgi, ioe); - } - } - } - } - - /** - * Enable delegation token for the session - * save the token string and set the token.signature in hive conf. The metastore client uses - * this token.signature to determine where to use kerberos or delegation token - * @throws HiveException - * @throws IOException - */ - private void setDelegationToken(String delegationTokenStr) throws HiveSQLException { - this.delegationTokenStr = delegationTokenStr; - if (delegationTokenStr != null) { - getHiveConf().set("hive.metastore.token.signature", HS2TOKEN); - try { - Utils.setTokenStr(sessionUgi, delegationTokenStr, HS2TOKEN); - } catch (IOException e) { - throw new HiveSQLException("Couldn't setup delegation token in the ugi", e); - } - } - } - - // If the session has a delegation token obtained from the metastore, then cancel it - private void cancelDelegationToken() throws HiveSQLException { - if (delegationTokenStr != null) { - try { - Hive.get(getHiveConf()).cancelDelegationToken(delegationTokenStr); - } catch (HiveException e) { - throw new HiveSQLException("Couldn't cancel delegation token", e); - } - // close the metastore connection created with this delegation token - Hive.closeCurrent(); - } - } - - @Override - protected HiveSession getSession() { - assert proxySession != null; - - return proxySession; - } - - public void setProxySession(HiveSession proxySession) { - this.proxySession = proxySession; - } - - @Override - public String getDelegationToken(HiveAuthFactory authFactory, String owner, - String renewer) throws HiveSQLException { - return authFactory.getDelegationToken(owner, renewer); - } - - @Override - public void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr) - throws HiveSQLException { - authFactory.cancelDelegationToken(tokenStr); - } - - @Override - public void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr) - throws HiveSQLException { - authFactory.renewDelegationToken(tokenStr); - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/SessionManager.java deleted file mode 100644 index ad6fb3ba37a0e..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/SessionManager.java +++ /dev/null @@ -1,377 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.session; - -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Date; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.Future; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -import org.apache.commons.io.FileUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hive.service.CompositeService; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.SessionHandle; -import org.apache.hive.service.cli.operation.OperationManager; -import org.apache.hive.service.cli.thrift.TProtocolVersion; -import org.apache.hive.service.server.HiveServer2; -import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; - -/** - * SessionManager. - * - */ -public class SessionManager extends CompositeService { - - private static final Log LOG = LogFactory.getLog(SessionManager.class); - public static final String HIVERCFILE = ".hiverc"; - private HiveConf hiveConf; - private final Map handleToSession = - new ConcurrentHashMap(); - private final OperationManager operationManager = new OperationManager(); - private ThreadPoolExecutor backgroundOperationPool; - private boolean isOperationLogEnabled; - private File operationLogRootDir; - - private long checkInterval; - private long sessionTimeout; - private boolean checkOperation; - - private volatile boolean shutdown; - // The HiveServer2 instance running this service - private final HiveServer2 hiveServer2; - - public SessionManager(HiveServer2 hiveServer2) { - super(SessionManager.class.getSimpleName()); - this.hiveServer2 = hiveServer2; - } - - @Override - public synchronized void init(HiveConf hiveConf) { - this.hiveConf = hiveConf; - //Create operation log root directory, if operation logging is enabled - if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED)) { - initOperationLogRootDir(); - } - createBackgroundOperationPool(); - addService(operationManager); - super.init(hiveConf); - } - - private void createBackgroundOperationPool() { - int poolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS); - LOG.info("HiveServer2: Background operation thread pool size: " + poolSize); - int poolQueueSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_WAIT_QUEUE_SIZE); - LOG.info("HiveServer2: Background operation thread wait queue size: " + poolQueueSize); - long keepAliveTime = HiveConf.getTimeVar( - hiveConf, ConfVars.HIVE_SERVER2_ASYNC_EXEC_KEEPALIVE_TIME, TimeUnit.SECONDS); - LOG.info( - "HiveServer2: Background operation thread keepalive time: " + keepAliveTime + " seconds"); - - // Create a thread pool with #poolSize threads - // Threads terminate when they are idle for more than the keepAliveTime - // A bounded blocking queue is used to queue incoming operations, if #operations > poolSize - String threadPoolName = "HiveServer2-Background-Pool"; - backgroundOperationPool = new ThreadPoolExecutor(poolSize, poolSize, - keepAliveTime, TimeUnit.SECONDS, new LinkedBlockingQueue(poolQueueSize), - new ThreadFactoryWithGarbageCleanup(threadPoolName)); - backgroundOperationPool.allowCoreThreadTimeOut(true); - - checkInterval = HiveConf.getTimeVar( - hiveConf, ConfVars.HIVE_SERVER2_SESSION_CHECK_INTERVAL, TimeUnit.MILLISECONDS); - sessionTimeout = HiveConf.getTimeVar( - hiveConf, ConfVars.HIVE_SERVER2_IDLE_SESSION_TIMEOUT, TimeUnit.MILLISECONDS); - checkOperation = HiveConf.getBoolVar(hiveConf, - ConfVars.HIVE_SERVER2_IDLE_SESSION_CHECK_OPERATION); - } - - private void initOperationLogRootDir() { - operationLogRootDir = new File( - hiveConf.getVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LOG_LOCATION)); - isOperationLogEnabled = true; - - if (operationLogRootDir.exists() && !operationLogRootDir.isDirectory()) { - LOG.warn("The operation log root directory exists, but it is not a directory: " + - operationLogRootDir.getAbsolutePath()); - isOperationLogEnabled = false; - } - - if (!operationLogRootDir.exists()) { - if (!operationLogRootDir.mkdirs()) { - LOG.warn("Unable to create operation log root directory: " + - operationLogRootDir.getAbsolutePath()); - isOperationLogEnabled = false; - } - } - - if (isOperationLogEnabled) { - LOG.info("Operation log root directory is created: " + operationLogRootDir.getAbsolutePath()); - try { - FileUtils.forceDeleteOnExit(operationLogRootDir); - } catch (IOException e) { - LOG.warn("Failed to schedule cleanup HS2 operation logging root dir: " + - operationLogRootDir.getAbsolutePath(), e); - } - } - } - - @Override - public synchronized void start() { - super.start(); - if (checkInterval > 0) { - startTimeoutChecker(); - } - } - - private final Object timeoutCheckerLock = new Object(); - - private void startTimeoutChecker() { - final long interval = Math.max(checkInterval, 3000L); // minimum 3 seconds - final Runnable timeoutChecker = new Runnable() { - @Override - public void run() { - sleepFor(interval); - while (!shutdown) { - long current = System.currentTimeMillis(); - for (HiveSession session : new ArrayList(handleToSession.values())) { - if (shutdown) { - break; - } - if (sessionTimeout > 0 && session.getLastAccessTime() + sessionTimeout <= current - && (!checkOperation || session.getNoOperationTime() > sessionTimeout)) { - SessionHandle handle = session.getSessionHandle(); - LOG.warn("Session " + handle + " is Timed-out (last access : " + - new Date(session.getLastAccessTime()) + ") and will be closed"); - try { - closeSession(handle); - } catch (HiveSQLException e) { - LOG.warn("Exception is thrown closing session " + handle, e); - } - } else { - session.closeExpiredOperations(); - } - } - sleepFor(interval); - } - } - - private void sleepFor(long interval) { - synchronized (timeoutCheckerLock) { - try { - timeoutCheckerLock.wait(interval); - } catch (InterruptedException e) { - // Ignore, and break. - } - } - } - }; - backgroundOperationPool.execute(timeoutChecker); - } - - private void shutdownTimeoutChecker() { - shutdown = true; - synchronized (timeoutCheckerLock) { - timeoutCheckerLock.notify(); - } - } - - @Override - public synchronized void stop() { - super.stop(); - shutdownTimeoutChecker(); - if (backgroundOperationPool != null) { - backgroundOperationPool.shutdown(); - long timeout = hiveConf.getTimeVar( - ConfVars.HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT, TimeUnit.SECONDS); - try { - backgroundOperationPool.awaitTermination(timeout, TimeUnit.SECONDS); - } catch (InterruptedException e) { - LOG.warn("HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT = " + timeout + - " seconds has been exceeded. RUNNING background operations will be shut down", e); - } - backgroundOperationPool = null; - } - cleanupLoggingRootDir(); - } - - private void cleanupLoggingRootDir() { - if (isOperationLogEnabled) { - try { - FileUtils.forceDelete(operationLogRootDir); - } catch (Exception e) { - LOG.warn("Failed to cleanup root dir of HS2 logging: " + operationLogRootDir - .getAbsolutePath(), e); - } - } - } - - public SessionHandle openSession(TProtocolVersion protocol, String username, String password, String ipAddress, - Map sessionConf) throws HiveSQLException { - return openSession(protocol, username, password, ipAddress, sessionConf, false, null); - } - - /** - * Opens a new session and creates a session handle. - * The username passed to this method is the effective username. - * If withImpersonation is true (==doAs true) we wrap all the calls in HiveSession - * within a UGI.doAs, where UGI corresponds to the effective user. - * - * Please see {@code org.apache.hive.service.cli.thrift.ThriftCLIService.getUserName()} for - * more details. - * - * @param protocol - * @param username - * @param password - * @param ipAddress - * @param sessionConf - * @param withImpersonation - * @param delegationToken - * @return - * @throws HiveSQLException - */ - public SessionHandle openSession(TProtocolVersion protocol, String username, String password, String ipAddress, - Map sessionConf, boolean withImpersonation, String delegationToken) - throws HiveSQLException { - HiveSession session; - // If doAs is set to true for HiveServer2, we will create a proxy object for the session impl. - // Within the proxy object, we wrap the method call in a UserGroupInformation#doAs - if (withImpersonation) { - HiveSessionImplwithUGI sessionWithUGI = new HiveSessionImplwithUGI(protocol, username, password, - hiveConf, ipAddress, delegationToken); - session = HiveSessionProxy.getProxy(sessionWithUGI, sessionWithUGI.getSessionUgi()); - sessionWithUGI.setProxySession(session); - } else { - session = new HiveSessionImpl(protocol, username, password, hiveConf, ipAddress); - } - session.setSessionManager(this); - session.setOperationManager(operationManager); - try { - session.open(sessionConf); - } catch (Exception e) { - try { - session.close(); - } catch (Throwable t) { - LOG.warn("Error closing session", t); - } - session = null; - throw new HiveSQLException("Failed to open new session: " + e, e); - } - if (isOperationLogEnabled) { - session.setOperationLogSessionDir(operationLogRootDir); - } - handleToSession.put(session.getSessionHandle(), session); - return session.getSessionHandle(); - } - - public void closeSession(SessionHandle sessionHandle) throws HiveSQLException { - HiveSession session = handleToSession.remove(sessionHandle); - if (session == null) { - throw new HiveSQLException("Session does not exist!"); - } - session.close(); - } - - public HiveSession getSession(SessionHandle sessionHandle) throws HiveSQLException { - HiveSession session = handleToSession.get(sessionHandle); - if (session == null) { - throw new HiveSQLException("Invalid SessionHandle: " + sessionHandle); - } - return session; - } - - public OperationManager getOperationManager() { - return operationManager; - } - - private static ThreadLocal threadLocalIpAddress = new ThreadLocal() { - @Override - protected synchronized String initialValue() { - return null; - } - }; - - public static void setIpAddress(String ipAddress) { - threadLocalIpAddress.set(ipAddress); - } - - public static void clearIpAddress() { - threadLocalIpAddress.remove(); - } - - public static String getIpAddress() { - return threadLocalIpAddress.get(); - } - - private static ThreadLocal threadLocalUserName = new ThreadLocal(){ - @Override - protected synchronized String initialValue() { - return null; - } - }; - - public static void setUserName(String userName) { - threadLocalUserName.set(userName); - } - - public static void clearUserName() { - threadLocalUserName.remove(); - } - - public static String getUserName() { - return threadLocalUserName.get(); - } - - private static ThreadLocal threadLocalProxyUserName = new ThreadLocal(){ - @Override - protected synchronized String initialValue() { - return null; - } - }; - - public static void setProxyUserName(String userName) { - LOG.debug("setting proxy user name based on query param to: " + userName); - threadLocalProxyUserName.set(userName); - } - - public static String getProxyUserName() { - return threadLocalProxyUserName.get(); - } - - public static void clearProxyUserName() { - threadLocalProxyUserName.remove(); - } - - public Future submitBackgroundOperation(Runnable r) { - return backgroundOperationPool.submit(r); - } - - public int getOpenSessionCount() { - return handleToSession.size(); - } -} - diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java deleted file mode 100644 index 00bdf7e19126e..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ /dev/null @@ -1,121 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.thrift; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hive.service.ServiceException; -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.cli.CLIService; -import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; -import org.apache.thrift.TProcessorFactory; -import org.apache.thrift.protocol.TBinaryProtocol; -import org.apache.thrift.server.TThreadPoolServer; -import org.apache.thrift.transport.TServerSocket; -import org.apache.thrift.transport.TTransportFactory; - - -public class ThriftBinaryCLIService extends ThriftCLIService { - - public ThriftBinaryCLIService(CLIService cliService) { - super(cliService, ThriftBinaryCLIService.class.getSimpleName()); - } - - @Override - protected void initializeServer() { - try { - // Server thread pool - String threadPoolName = "HiveServer2-Handler-Pool"; - ExecutorService executorService = new ThreadPoolExecutor(minWorkerThreads, maxWorkerThreads, - workerKeepAliveTime, TimeUnit.SECONDS, new SynchronousQueue(), - new ThreadFactoryWithGarbageCleanup(threadPoolName)); - - // Thrift configs - hiveAuthFactory = new HiveAuthFactory(hiveConf); - TTransportFactory transportFactory = hiveAuthFactory.getAuthTransFactory(); - TProcessorFactory processorFactory = hiveAuthFactory.getAuthProcFactory(this); - TServerSocket serverSocket = null; - List sslVersionBlacklist = new ArrayList(); - for (String sslVersion : hiveConf.getVar(ConfVars.HIVE_SSL_PROTOCOL_BLACKLIST).split(",")) { - sslVersionBlacklist.add(sslVersion); - } - if (!hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL)) { - serverSocket = HiveAuthFactory.getServerSocket(hiveHost, portNum); - } else { - String keyStorePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH).trim(); - if (keyStorePath.isEmpty()) { - throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname - + " Not configured for SSL connection"); - } - String keyStorePassword = ShimLoader.getHadoopShims().getPassword(hiveConf, - HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname); - serverSocket = HiveAuthFactory.getServerSSLSocket(hiveHost, portNum, keyStorePath, - keyStorePassword, sslVersionBlacklist); - } - - // In case HIVE_SERVER2_THRIFT_PORT or hive.server2.thrift.port is configured with 0 which - // represents any free port, we should set it to the actual one - portNum = serverSocket.getServerSocket().getLocalPort(); - - // Server args - int maxMessageSize = hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_MAX_MESSAGE_SIZE); - int requestTimeout = (int) hiveConf.getTimeVar( - HiveConf.ConfVars.HIVE_SERVER2_THRIFT_LOGIN_TIMEOUT, TimeUnit.SECONDS); - int beBackoffSlotLength = (int) hiveConf.getTimeVar( - HiveConf.ConfVars.HIVE_SERVER2_THRIFT_LOGIN_BEBACKOFF_SLOT_LENGTH, TimeUnit.MILLISECONDS); - TThreadPoolServer.Args sargs = new TThreadPoolServer.Args(serverSocket) - .processorFactory(processorFactory).transportFactory(transportFactory) - .protocolFactory(new TBinaryProtocol.Factory()) - .inputProtocolFactory(new TBinaryProtocol.Factory(true, true, maxMessageSize, maxMessageSize)) - .requestTimeout(requestTimeout).requestTimeoutUnit(TimeUnit.SECONDS) - .beBackoffSlotLength(beBackoffSlotLength).beBackoffSlotLengthUnit(TimeUnit.MILLISECONDS) - .executorService(executorService); - - // TCP Server - server = new TThreadPoolServer(sargs); - server.setServerEventHandler(serverEventHandler); - String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " - + serverSocket.getServerSocket().getLocalPort() + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; - LOG.info(msg); - } catch (Exception t) { - throw new ServiceException("Error initializing " + getName(), t); - } - } - - @Override - public void run() { - try { - server.serve(); - } catch (Throwable t) { - LOG.fatal( - "Error starting HiveServer2: could not start " - + ThriftBinaryCLIService.class.getSimpleName(), t); - System.exit(-1); - } - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java deleted file mode 100644 index ff533769b5b84..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ /dev/null @@ -1,693 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.thrift; - -import javax.security.auth.login.LoginException; -import java.io.IOException; -import java.net.InetAddress; -import java.net.UnknownHostException; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hive.service.AbstractService; -import org.apache.hive.service.ServiceException; -import org.apache.hive.service.ServiceUtils; -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.auth.TSetIpAddressProcessor; -import org.apache.hive.service.cli.*; -import org.apache.hive.service.cli.session.SessionManager; -import org.apache.hive.service.server.HiveServer2; -import org.apache.thrift.TException; -import org.apache.thrift.protocol.TProtocol; -import org.apache.thrift.server.ServerContext; -import org.apache.thrift.server.TServer; -import org.apache.thrift.server.TServerEventHandler; -import org.apache.thrift.transport.TTransport; - -/** - * ThriftCLIService. - * - */ -public abstract class ThriftCLIService extends AbstractService implements TCLIService.Iface, Runnable { - - public static final Log LOG = LogFactory.getLog(ThriftCLIService.class.getName()); - - protected CLIService cliService; - private static final TStatus OK_STATUS = new TStatus(TStatusCode.SUCCESS_STATUS); - protected static HiveAuthFactory hiveAuthFactory; - - protected int portNum; - protected InetAddress serverIPAddress; - protected String hiveHost; - protected TServer server; - protected org.eclipse.jetty.server.Server httpServer; - - private boolean isStarted = false; - protected boolean isEmbedded = false; - - protected HiveConf hiveConf; - - protected int minWorkerThreads; - protected int maxWorkerThreads; - protected long workerKeepAliveTime; - - protected TServerEventHandler serverEventHandler; - protected ThreadLocal currentServerContext; - - static class ThriftCLIServerContext implements ServerContext { - private SessionHandle sessionHandle = null; - - public void setSessionHandle(SessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public SessionHandle getSessionHandle() { - return sessionHandle; - } - } - - public ThriftCLIService(CLIService service, String serviceName) { - super(serviceName); - this.cliService = service; - currentServerContext = new ThreadLocal(); - serverEventHandler = new TServerEventHandler() { - @Override - public ServerContext createContext( - TProtocol input, TProtocol output) { - return new ThriftCLIServerContext(); - } - - @Override - public void deleteContext(ServerContext serverContext, - TProtocol input, TProtocol output) { - ThriftCLIServerContext context = (ThriftCLIServerContext)serverContext; - SessionHandle sessionHandle = context.getSessionHandle(); - if (sessionHandle != null) { - LOG.info("Session disconnected without closing properly, close it now"); - try { - cliService.closeSession(sessionHandle); - } catch (HiveSQLException e) { - LOG.warn("Failed to close session: " + e, e); - } - } - } - - @Override - public void preServe() { - } - - @Override - public void processContext(ServerContext serverContext, - TTransport input, TTransport output) { - currentServerContext.set(serverContext); - } - }; - } - - @Override - public synchronized void init(HiveConf hiveConf) { - this.hiveConf = hiveConf; - // Initialize common server configs needed in both binary & http modes - String portString; - hiveHost = System.getenv("HIVE_SERVER2_THRIFT_BIND_HOST"); - if (hiveHost == null) { - hiveHost = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST); - } - try { - if (hiveHost != null && !hiveHost.isEmpty()) { - serverIPAddress = InetAddress.getByName(hiveHost); - } else { - serverIPAddress = InetAddress.getLocalHost(); - } - } catch (UnknownHostException e) { - throw new ServiceException(e); - } - // HTTP mode - if (HiveServer2.isHTTPTransportMode(hiveConf)) { - workerKeepAliveTime = - hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_WORKER_KEEPALIVE_TIME, - TimeUnit.SECONDS); - portString = System.getenv("HIVE_SERVER2_THRIFT_HTTP_PORT"); - if (portString != null) { - portNum = Integer.valueOf(portString); - } else { - portNum = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT); - } - } - // Binary mode - else { - workerKeepAliveTime = - hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_WORKER_KEEPALIVE_TIME, TimeUnit.SECONDS); - portString = System.getenv("HIVE_SERVER2_THRIFT_PORT"); - if (portString != null) { - portNum = Integer.valueOf(portString); - } else { - portNum = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT); - } - } - minWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_MIN_WORKER_THREADS); - maxWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_MAX_WORKER_THREADS); - super.init(hiveConf); - } - - @Override - public synchronized void start() { - super.start(); - if (!isStarted && !isEmbedded) { - initializeServer(); - new Thread(this).start(); - isStarted = true; - } - } - - @Override - public synchronized void stop() { - if (isStarted && !isEmbedded) { - if(server != null) { - server.stop(); - LOG.info("Thrift server has stopped"); - } - if((httpServer != null) && httpServer.isStarted()) { - try { - httpServer.stop(); - LOG.info("Http server has stopped"); - } catch (Exception e) { - LOG.error("Error stopping Http server: ", e); - } - } - isStarted = false; - } - super.stop(); - } - - public int getPortNumber() { - return portNum; - } - - public InetAddress getServerIPAddress() { - return serverIPAddress; - } - - @Override - public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) - throws TException { - TGetDelegationTokenResp resp = new TGetDelegationTokenResp(); - resp.setStatus(notSupportTokenErrorStatus()); - return resp; - } - - @Override - public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) - throws TException { - TCancelDelegationTokenResp resp = new TCancelDelegationTokenResp(); - resp.setStatus(notSupportTokenErrorStatus()); - return resp; - } - - @Override - public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) - throws TException { - TRenewDelegationTokenResp resp = new TRenewDelegationTokenResp(); - resp.setStatus(notSupportTokenErrorStatus()); - return resp; - } - - private TStatus notSupportTokenErrorStatus() { - TStatus errorStatus = new TStatus(TStatusCode.ERROR_STATUS); - errorStatus.setErrorMessage("Delegation token is not supported"); - return errorStatus; - } - - @Override - public TOpenSessionResp OpenSession(TOpenSessionReq req) throws TException { - LOG.info("Client protocol version: " + req.getClient_protocol()); - TOpenSessionResp resp = new TOpenSessionResp(); - try { - SessionHandle sessionHandle = getSessionHandle(req, resp); - resp.setSessionHandle(sessionHandle.toTSessionHandle()); - // TODO: set real configuration map - resp.setConfiguration(new HashMap()); - resp.setStatus(OK_STATUS); - ThriftCLIServerContext context = - (ThriftCLIServerContext)currentServerContext.get(); - if (context != null) { - context.setSessionHandle(sessionHandle); - } - } catch (Exception e) { - LOG.warn("Error opening session: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - private String getIpAddress() { - String clientIpAddress; - // Http transport mode. - // We set the thread local ip address, in ThriftHttpServlet. - if (cliService.getHiveConf().getVar( - ConfVars.HIVE_SERVER2_TRANSPORT_MODE).equalsIgnoreCase("http")) { - clientIpAddress = SessionManager.getIpAddress(); - } - else { - // Kerberos - if (isKerberosAuthMode()) { - clientIpAddress = hiveAuthFactory.getIpAddress(); - } - // Except kerberos, NOSASL - else { - clientIpAddress = TSetIpAddressProcessor.getUserIpAddress(); - } - } - LOG.debug("Client's IP Address: " + clientIpAddress); - return clientIpAddress; - } - - /** - * Returns the effective username. - * 1. If hive.server2.allow.user.substitution = false: the username of the connecting user - * 2. If hive.server2.allow.user.substitution = true: the username of the end user, - * that the connecting user is trying to proxy for. - * This includes a check whether the connecting user is allowed to proxy for the end user. - * @param req - * @return - * @throws HiveSQLException - */ - private String getUserName(TOpenSessionReq req) throws HiveSQLException { - String userName = null; - // Kerberos - if (isKerberosAuthMode()) { - userName = hiveAuthFactory.getRemoteUser(); - } - // Except kerberos, NOSASL - if (userName == null) { - userName = TSetIpAddressProcessor.getUserName(); - } - // Http transport mode. - // We set the thread local username, in ThriftHttpServlet. - if (cliService.getHiveConf().getVar( - ConfVars.HIVE_SERVER2_TRANSPORT_MODE).equalsIgnoreCase("http")) { - userName = SessionManager.getUserName(); - } - if (userName == null) { - userName = req.getUsername(); - } - - userName = getShortName(userName); - String effectiveClientUser = getProxyUser(userName, req.getConfiguration(), getIpAddress()); - LOG.debug("Client's username: " + effectiveClientUser); - return effectiveClientUser; - } - - private String getShortName(String userName) { - String ret = null; - if (userName != null) { - int indexOfDomainMatch = ServiceUtils.indexOfDomainMatch(userName); - ret = (indexOfDomainMatch <= 0) ? userName : - userName.substring(0, indexOfDomainMatch); - } - - return ret; - } - - /** - * Create a session handle - * @param req - * @param res - * @return - * @throws HiveSQLException - * @throws LoginException - * @throws IOException - */ - SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res) - throws HiveSQLException, LoginException, IOException { - String userName = getUserName(req); - String ipAddress = getIpAddress(); - TProtocolVersion protocol = getMinVersion(CLIService.SERVER_VERSION, - req.getClient_protocol()); - res.setServerProtocolVersion(protocol); - SessionHandle sessionHandle; - if (cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && - (userName != null)) { - String delegationTokenStr = getDelegationToken(userName); - sessionHandle = cliService.openSessionWithImpersonation(protocol, userName, - req.getPassword(), ipAddress, req.getConfiguration(), delegationTokenStr); - } else { - sessionHandle = cliService.openSession(protocol, userName, req.getPassword(), - ipAddress, req.getConfiguration()); - } - return sessionHandle; - } - - - private String getDelegationToken(String userName) - throws HiveSQLException, LoginException, IOException { - if (userName == null || !cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION) - .equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString())) { - return null; - } - try { - return cliService.getDelegationTokenFromMetaStore(userName); - } catch (UnsupportedOperationException e) { - // The delegation token is not applicable in the given deployment mode - } - return null; - } - - private TProtocolVersion getMinVersion(TProtocolVersion... versions) { - TProtocolVersion[] values = TProtocolVersion.values(); - int current = values[values.length - 1].getValue(); - for (TProtocolVersion version : versions) { - if (current > version.getValue()) { - current = version.getValue(); - } - } - for (TProtocolVersion version : values) { - if (version.getValue() == current) { - return version; - } - } - throw new IllegalArgumentException("never"); - } - - @Override - public TCloseSessionResp CloseSession(TCloseSessionReq req) throws TException { - TCloseSessionResp resp = new TCloseSessionResp(); - try { - SessionHandle sessionHandle = new SessionHandle(req.getSessionHandle()); - cliService.closeSession(sessionHandle); - resp.setStatus(OK_STATUS); - ThriftCLIServerContext context = - (ThriftCLIServerContext)currentServerContext.get(); - if (context != null) { - context.setSessionHandle(null); - } - } catch (Exception e) { - LOG.warn("Error closing session: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetInfoResp GetInfo(TGetInfoReq req) throws TException { - TGetInfoResp resp = new TGetInfoResp(); - try { - GetInfoValue getInfoValue = - cliService.getInfo(new SessionHandle(req.getSessionHandle()), - GetInfoType.getGetInfoType(req.getInfoType())); - resp.setInfoValue(getInfoValue.toTGetInfoValue()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting info: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TExecuteStatementResp ExecuteStatement(TExecuteStatementReq req) throws TException { - TExecuteStatementResp resp = new TExecuteStatementResp(); - try { - SessionHandle sessionHandle = new SessionHandle(req.getSessionHandle()); - String statement = req.getStatement(); - Map confOverlay = req.getConfOverlay(); - Boolean runAsync = req.isRunAsync(); - OperationHandle operationHandle = runAsync ? - cliService.executeStatementAsync(sessionHandle, statement, confOverlay) - : cliService.executeStatement(sessionHandle, statement, confOverlay); - resp.setOperationHandle(operationHandle.toTOperationHandle()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error executing statement: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetTypeInfoResp GetTypeInfo(TGetTypeInfoReq req) throws TException { - TGetTypeInfoResp resp = new TGetTypeInfoResp(); - try { - OperationHandle operationHandle = cliService.getTypeInfo(new SessionHandle(req.getSessionHandle())); - resp.setOperationHandle(operationHandle.toTOperationHandle()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting type info: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetCatalogsResp GetCatalogs(TGetCatalogsReq req) throws TException { - TGetCatalogsResp resp = new TGetCatalogsResp(); - try { - OperationHandle opHandle = cliService.getCatalogs(new SessionHandle(req.getSessionHandle())); - resp.setOperationHandle(opHandle.toTOperationHandle()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting catalogs: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetSchemasResp GetSchemas(TGetSchemasReq req) throws TException { - TGetSchemasResp resp = new TGetSchemasResp(); - try { - OperationHandle opHandle = cliService.getSchemas( - new SessionHandle(req.getSessionHandle()), req.getCatalogName(), req.getSchemaName()); - resp.setOperationHandle(opHandle.toTOperationHandle()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting schemas: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetTablesResp GetTables(TGetTablesReq req) throws TException { - TGetTablesResp resp = new TGetTablesResp(); - try { - OperationHandle opHandle = cliService - .getTables(new SessionHandle(req.getSessionHandle()), req.getCatalogName(), - req.getSchemaName(), req.getTableName(), req.getTableTypes()); - resp.setOperationHandle(opHandle.toTOperationHandle()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting tables: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetTableTypesResp GetTableTypes(TGetTableTypesReq req) throws TException { - TGetTableTypesResp resp = new TGetTableTypesResp(); - try { - OperationHandle opHandle = cliService.getTableTypes(new SessionHandle(req.getSessionHandle())); - resp.setOperationHandle(opHandle.toTOperationHandle()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting table types: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetColumnsResp GetColumns(TGetColumnsReq req) throws TException { - TGetColumnsResp resp = new TGetColumnsResp(); - try { - OperationHandle opHandle = cliService.getColumns( - new SessionHandle(req.getSessionHandle()), - req.getCatalogName(), - req.getSchemaName(), - req.getTableName(), - req.getColumnName()); - resp.setOperationHandle(opHandle.toTOperationHandle()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting columns: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetFunctionsResp GetFunctions(TGetFunctionsReq req) throws TException { - TGetFunctionsResp resp = new TGetFunctionsResp(); - try { - OperationHandle opHandle = cliService.getFunctions( - new SessionHandle(req.getSessionHandle()), req.getCatalogName(), - req.getSchemaName(), req.getFunctionName()); - resp.setOperationHandle(opHandle.toTOperationHandle()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting functions: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) throws TException { - TGetOperationStatusResp resp = new TGetOperationStatusResp(); - try { - OperationStatus operationStatus = cliService.getOperationStatus( - new OperationHandle(req.getOperationHandle())); - resp.setOperationState(operationStatus.getState().toTOperationState()); - HiveSQLException opException = operationStatus.getOperationException(); - if (opException != null) { - resp.setSqlState(opException.getSQLState()); - resp.setErrorCode(opException.getErrorCode()); - resp.setErrorMessage(org.apache.hadoop.util.StringUtils - .stringifyException(opException)); - } - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting operation status: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TCancelOperationResp CancelOperation(TCancelOperationReq req) throws TException { - TCancelOperationResp resp = new TCancelOperationResp(); - try { - cliService.cancelOperation(new OperationHandle(req.getOperationHandle())); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error cancelling operation: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TCloseOperationResp CloseOperation(TCloseOperationReq req) throws TException { - TCloseOperationResp resp = new TCloseOperationResp(); - try { - cliService.closeOperation(new OperationHandle(req.getOperationHandle())); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error closing operation: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetResultSetMetadataResp GetResultSetMetadata(TGetResultSetMetadataReq req) - throws TException { - TGetResultSetMetadataResp resp = new TGetResultSetMetadataResp(); - try { - TableSchema schema = cliService.getResultSetMetadata(new OperationHandle(req.getOperationHandle())); - resp.setSchema(schema.toTTableSchema()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting result set metadata: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TFetchResultsResp FetchResults(TFetchResultsReq req) throws TException { - TFetchResultsResp resp = new TFetchResultsResp(); - try { - RowSet rowSet = cliService.fetchResults( - new OperationHandle(req.getOperationHandle()), - FetchOrientation.getFetchOrientation(req.getOrientation()), - req.getMaxRows(), - FetchType.getFetchType(req.getFetchType())); - resp.setResults(rowSet.toTRowSet()); - resp.setHasMoreRows(false); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error fetching results: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - protected abstract void initializeServer(); - - @Override - public abstract void run(); - - /** - * If the proxy user name is provided then check privileges to substitute the user. - * @param realUser - * @param sessionConf - * @param ipAddress - * @return - * @throws HiveSQLException - */ - private String getProxyUser(String realUser, Map sessionConf, - String ipAddress) throws HiveSQLException { - String proxyUser = null; - // Http transport mode. - // We set the thread local proxy username, in ThriftHttpServlet. - if (cliService.getHiveConf().getVar( - ConfVars.HIVE_SERVER2_TRANSPORT_MODE).equalsIgnoreCase("http")) { - proxyUser = SessionManager.getProxyUserName(); - LOG.debug("Proxy user from query string: " + proxyUser); - } - - if (proxyUser == null && sessionConf != null && sessionConf.containsKey(HiveAuthFactory.HS2_PROXY_USER)) { - String proxyUserFromThriftBody = sessionConf.get(HiveAuthFactory.HS2_PROXY_USER); - LOG.debug("Proxy user from thrift body: " + proxyUserFromThriftBody); - proxyUser = proxyUserFromThriftBody; - } - - if (proxyUser == null) { - return realUser; - } - - // check whether substitution is allowed - if (!hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ALLOW_USER_SUBSTITUTION)) { - throw new HiveSQLException("Proxy user substitution is not allowed"); - } - - // If there's no authentication, then directly substitute the user - if (HiveAuthFactory.AuthTypes.NONE.toString() - .equalsIgnoreCase(hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION))) { - return proxyUser; - } - - // Verify proxy user privilege of the realUser for the proxyUser - HiveAuthFactory.verifyProxyAccess(realUser, proxyUser, ipAddress, hiveConf); - LOG.debug("Verified proxy user: " + proxyUser); - return proxyUser; - } - - private boolean isKerberosAuthMode() { - return cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION) - .equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString()); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java deleted file mode 100644 index 1af45398b895c..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java +++ /dev/null @@ -1,440 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.thrift; - -import java.util.List; -import java.util.Map; - -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.cli.*; -import org.apache.thrift.TException; - -/** - * ThriftCLIServiceClient. - * - */ -public class ThriftCLIServiceClient extends CLIServiceClient { - private final TCLIService.Iface cliService; - - public ThriftCLIServiceClient(TCLIService.Iface cliService) { - this.cliService = cliService; - } - - public void checkStatus(TStatus status) throws HiveSQLException { - if (TStatusCode.ERROR_STATUS.equals(status.getStatusCode())) { - throw new HiveSQLException(status); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map) - */ - @Override - public SessionHandle openSession(String username, String password, - Map configuration) - throws HiveSQLException { - try { - TOpenSessionReq req = new TOpenSessionReq(); - req.setUsername(username); - req.setPassword(password); - req.setConfiguration(configuration); - TOpenSessionResp resp = cliService.OpenSession(req); - checkStatus(resp.getStatus()); - return new SessionHandle(resp.getSessionHandle(), resp.getServerProtocolVersion()); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public SessionHandle openSessionWithImpersonation(String username, String password, - Map configuration, String delegationToken) throws HiveSQLException { - throw new HiveSQLException("open with impersonation operation is not supported in the client"); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public void closeSession(SessionHandle sessionHandle) throws HiveSQLException { - try { - TCloseSessionReq req = new TCloseSessionReq(sessionHandle.toTSessionHandle()); - TCloseSessionResp resp = cliService.CloseSession(req); - checkStatus(resp.getStatus()); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List) - */ - @Override - public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType infoType) - throws HiveSQLException { - try { - // FIXME extract the right info type - TGetInfoReq req = new TGetInfoReq(sessionHandle.toTSessionHandle(), infoType.toTGetInfoType()); - TGetInfoResp resp = cliService.GetInfo(req); - checkStatus(resp.getStatus()); - return new GetInfoValue(resp.getInfoValue()); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#executeStatement(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.util.Map) - */ - @Override - public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, - Map confOverlay) - throws HiveSQLException { - return executeStatementInternal(sessionHandle, statement, confOverlay, false); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#executeStatementAsync(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.util.Map) - */ - @Override - public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, - Map confOverlay) - throws HiveSQLException { - return executeStatementInternal(sessionHandle, statement, confOverlay, true); - } - - private OperationHandle executeStatementInternal(SessionHandle sessionHandle, String statement, - Map confOverlay, boolean isAsync) - throws HiveSQLException { - try { - TExecuteStatementReq req = - new TExecuteStatementReq(sessionHandle.toTSessionHandle(), statement); - req.setConfOverlay(confOverlay); - req.setRunAsync(isAsync); - TExecuteStatementResp resp = cliService.ExecuteStatement(req); - checkStatus(resp.getStatus()); - TProtocolVersion protocol = sessionHandle.getProtocolVersion(); - return new OperationHandle(resp.getOperationHandle(), protocol); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getTypeInfo(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getTypeInfo(SessionHandle sessionHandle) throws HiveSQLException { - try { - TGetTypeInfoReq req = new TGetTypeInfoReq(sessionHandle.toTSessionHandle()); - TGetTypeInfoResp resp = cliService.GetTypeInfo(req); - checkStatus(resp.getStatus()); - TProtocolVersion protocol = sessionHandle.getProtocolVersion(); - return new OperationHandle(resp.getOperationHandle(), protocol); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getCatalogs(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getCatalogs(SessionHandle sessionHandle) throws HiveSQLException { - try { - TGetCatalogsReq req = new TGetCatalogsReq(sessionHandle.toTSessionHandle()); - TGetCatalogsResp resp = cliService.GetCatalogs(req); - checkStatus(resp.getStatus()); - TProtocolVersion protocol = sessionHandle.getProtocolVersion(); - return new OperationHandle(resp.getOperationHandle(), protocol); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String) - */ - @Override - public OperationHandle getSchemas(SessionHandle sessionHandle, String catalogName, - String schemaName) - throws HiveSQLException { - try { - TGetSchemasReq req = new TGetSchemasReq(sessionHandle.toTSessionHandle()); - req.setCatalogName(catalogName); - req.setSchemaName(schemaName); - TGetSchemasResp resp = cliService.GetSchemas(req); - checkStatus(resp.getStatus()); - TProtocolVersion protocol = sessionHandle.getProtocolVersion(); - return new OperationHandle(resp.getOperationHandle(), protocol); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List) - */ - @Override - public OperationHandle getTables(SessionHandle sessionHandle, String catalogName, - String schemaName, String tableName, List tableTypes) - throws HiveSQLException { - try { - TGetTablesReq req = new TGetTablesReq(sessionHandle.toTSessionHandle()); - req.setTableName(tableName); - req.setTableTypes(tableTypes); - req.setSchemaName(schemaName); - TGetTablesResp resp = cliService.GetTables(req); - checkStatus(resp.getStatus()); - TProtocolVersion protocol = sessionHandle.getProtocolVersion(); - return new OperationHandle(resp.getOperationHandle(), protocol); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getTableTypes(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getTableTypes(SessionHandle sessionHandle) throws HiveSQLException { - try { - TGetTableTypesReq req = new TGetTableTypesReq(sessionHandle.toTSessionHandle()); - TGetTableTypesResp resp = cliService.GetTableTypes(req); - checkStatus(resp.getStatus()); - TProtocolVersion protocol = sessionHandle.getProtocolVersion(); - return new OperationHandle(resp.getOperationHandle(), protocol); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getColumns(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getColumns(SessionHandle sessionHandle, - String catalogName, String schemaName, String tableName, String columnName) - throws HiveSQLException { - try { - TGetColumnsReq req = new TGetColumnsReq(); - req.setSessionHandle(sessionHandle.toTSessionHandle()); - req.setCatalogName(catalogName); - req.setSchemaName(schemaName); - req.setTableName(tableName); - req.setColumnName(columnName); - TGetColumnsResp resp = cliService.GetColumns(req); - checkStatus(resp.getStatus()); - TProtocolVersion protocol = sessionHandle.getProtocolVersion(); - return new OperationHandle(resp.getOperationHandle(), protocol); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getFunctions(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getFunctions(SessionHandle sessionHandle, - String catalogName, String schemaName, String functionName) throws HiveSQLException { - try { - TGetFunctionsReq req = new TGetFunctionsReq(sessionHandle.toTSessionHandle(), functionName); - req.setCatalogName(catalogName); - req.setSchemaName(schemaName); - TGetFunctionsResp resp = cliService.GetFunctions(req); - checkStatus(resp.getStatus()); - TProtocolVersion protocol = sessionHandle.getProtocolVersion(); - return new OperationHandle(resp.getOperationHandle(), protocol); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getOperationStatus(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public OperationStatus getOperationStatus(OperationHandle opHandle) throws HiveSQLException { - try { - TGetOperationStatusReq req = new TGetOperationStatusReq(opHandle.toTOperationHandle()); - TGetOperationStatusResp resp = cliService.GetOperationStatus(req); - // Checks the status of the RPC call, throws an exception in case of error - checkStatus(resp.getStatus()); - OperationState opState = OperationState.getOperationState(resp.getOperationState()); - HiveSQLException opException = null; - if (opState == OperationState.ERROR) { - opException = new HiveSQLException(resp.getErrorMessage(), resp.getSqlState(), resp.getErrorCode()); - } - return new OperationStatus(opState, opException); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#cancelOperation(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { - try { - TCancelOperationReq req = new TCancelOperationReq(opHandle.toTOperationHandle()); - TCancelOperationResp resp = cliService.CancelOperation(req); - checkStatus(resp.getStatus()); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#closeOperation(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public void closeOperation(OperationHandle opHandle) - throws HiveSQLException { - try { - TCloseOperationReq req = new TCloseOperationReq(opHandle.toTOperationHandle()); - TCloseOperationResp resp = cliService.CloseOperation(req); - checkStatus(resp.getStatus()); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public TableSchema getResultSetMetadata(OperationHandle opHandle) - throws HiveSQLException { - try { - TGetResultSetMetadataReq req = new TGetResultSetMetadataReq(opHandle.toTOperationHandle()); - TGetResultSetMetadataResp resp = cliService.GetResultSetMetadata(req); - checkStatus(resp.getStatus()); - return new TableSchema(resp.getSchema()); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - @Override - public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, long maxRows, - FetchType fetchType) throws HiveSQLException { - try { - TFetchResultsReq req = new TFetchResultsReq(); - req.setOperationHandle(opHandle.toTOperationHandle()); - req.setOrientation(orientation.toTFetchOrientation()); - req.setMaxRows(maxRows); - req.setFetchType(fetchType.toTFetchType()); - TFetchResultsResp resp = cliService.FetchResults(req); - checkStatus(resp.getStatus()); - return RowSetFactory.create(resp.getResults(), opHandle.getProtocolVersion()); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#fetchResults(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public RowSet fetchResults(OperationHandle opHandle) throws HiveSQLException { - // TODO: set the correct default fetch size - return fetchResults(opHandle, FetchOrientation.FETCH_NEXT, 10000, FetchType.QUERY_OUTPUT); - } - - @Override - public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String owner, String renewer) throws HiveSQLException { - TGetDelegationTokenReq req = new TGetDelegationTokenReq( - sessionHandle.toTSessionHandle(), owner, renewer); - try { - TGetDelegationTokenResp tokenResp = cliService.GetDelegationToken(req); - checkStatus(tokenResp.getStatus()); - return tokenResp.getDelegationToken(); - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - @Override - public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String tokenStr) throws HiveSQLException { - TCancelDelegationTokenReq cancelReq = new TCancelDelegationTokenReq( - sessionHandle.toTSessionHandle(), tokenStr); - try { - TCancelDelegationTokenResp cancelResp = - cliService.CancelDelegationToken(cancelReq); - checkStatus(cancelResp.getStatus()); - return; - } catch (TException e) { - throw new HiveSQLException(e); - } - } - - @Override - public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String tokenStr) throws HiveSQLException { - TRenewDelegationTokenReq cancelReq = new TRenewDelegationTokenReq( - sessionHandle.toTSessionHandle(), tokenStr); - try { - TRenewDelegationTokenResp renewResp = - cliService.RenewDelegationToken(cancelReq); - checkStatus(renewResp.getStatus()); - return; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java deleted file mode 100644 index bd64c777c1d76..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ /dev/null @@ -1,194 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.thrift; - -import java.util.Arrays; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.util.Shell; -import org.apache.hive.service.ServiceException; -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.cli.CLIService; -import org.apache.hive.service.cli.thrift.TCLIService.Iface; -import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; -import org.apache.thrift.TProcessor; -import org.apache.thrift.protocol.TBinaryProtocol; -import org.apache.thrift.protocol.TProtocolFactory; -import org.apache.thrift.server.TServlet; -import org.eclipse.jetty.server.AbstractConnectionFactory; -import org.eclipse.jetty.server.ConnectionFactory; -import org.eclipse.jetty.server.HttpConnectionFactory; -import org.eclipse.jetty.server.ServerConnector; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.util.ssl.SslContextFactory; -import org.eclipse.jetty.util.thread.ExecutorThreadPool; -import org.eclipse.jetty.util.thread.ScheduledExecutorScheduler; - - -public class ThriftHttpCLIService extends ThriftCLIService { - - public ThriftHttpCLIService(CLIService cliService) { - super(cliService, ThriftHttpCLIService.class.getSimpleName()); - } - - @Override - protected void initializeServer() { - try { - // Server thread pool - // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests - String threadPoolName = "HiveServer2-HttpHandler-Pool"; - ThreadPoolExecutor executorService = new ThreadPoolExecutor(minWorkerThreads, maxWorkerThreads, - workerKeepAliveTime, TimeUnit.SECONDS, new SynchronousQueue(), - new ThreadFactoryWithGarbageCleanup(threadPoolName)); - ExecutorThreadPool threadPool = new ExecutorThreadPool(executorService); - - // HTTP Server - httpServer = new org.eclipse.jetty.server.Server(threadPool); - - // Connector configs - - ConnectionFactory[] connectionFactories; - boolean useSsl = hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL); - String schemeName = useSsl ? "https" : "http"; - // Change connector if SSL is used - if (useSsl) { - String keyStorePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH).trim(); - String keyStorePassword = ShimLoader.getHadoopShims().getPassword(hiveConf, - HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname); - if (keyStorePath.isEmpty()) { - throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname - + " Not configured for SSL connection"); - } - SslContextFactory sslContextFactory = new SslContextFactory.Server(); - String[] excludedProtocols = hiveConf.getVar(ConfVars.HIVE_SSL_PROTOCOL_BLACKLIST).split(","); - LOG.info("HTTP Server SSL: adding excluded protocols: " + Arrays.toString(excludedProtocols)); - sslContextFactory.addExcludeProtocols(excludedProtocols); - LOG.info("HTTP Server SSL: SslContextFactory.getExcludeProtocols = " + - Arrays.toString(sslContextFactory.getExcludeProtocols())); - sslContextFactory.setKeyStorePath(keyStorePath); - sslContextFactory.setKeyStorePassword(keyStorePassword); - connectionFactories = AbstractConnectionFactory.getFactories( - sslContextFactory, new HttpConnectionFactory()); - } else { - connectionFactories = new ConnectionFactory[] { new HttpConnectionFactory() }; - } - ServerConnector connector = new ServerConnector( - httpServer, - null, - // Call this full constructor to set this, which forces daemon threads: - new ScheduledExecutorScheduler("HiveServer2-HttpHandler-JettyScheduler", true), - null, - -1, - -1, - connectionFactories); - - connector.setPort(portNum); - // Linux:yes, Windows:no - connector.setReuseAddress(!Shell.WINDOWS); - int maxIdleTime = (int) hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME, - TimeUnit.MILLISECONDS); - connector.setIdleTimeout(maxIdleTime); - - httpServer.addConnector(connector); - - // Thrift configs - hiveAuthFactory = new HiveAuthFactory(hiveConf); - TProcessor processor = new TCLIService.Processor(this); - TProtocolFactory protocolFactory = new TBinaryProtocol.Factory(); - // Set during the init phase of HiveServer2 if auth mode is kerberos - // UGI for the hive/_HOST (kerberos) principal - UserGroupInformation serviceUGI = cliService.getServiceUGI(); - // UGI for the http/_HOST (SPNego) principal - UserGroupInformation httpUGI = cliService.getHttpUGI(); - String authType = hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION); - TServlet thriftHttpServlet = new ThriftHttpServlet(processor, protocolFactory, authType, - serviceUGI, httpUGI); - - // Context handler - final ServletContextHandler context = new ServletContextHandler( - ServletContextHandler.SESSIONS); - context.setContextPath("/"); - String httpPath = getHttpPath(hiveConf - .getVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH)); - httpServer.setHandler(context); - context.addServlet(new ServletHolder(thriftHttpServlet), httpPath); - - // TODO: check defaults: maxTimeout, keepalive, maxBodySize, bodyRecieveDuration, etc. - // Finally, start the server - httpServer.start(); - // In case HIVE_SERVER2_THRIFT_HTTP_PORT or hive.server2.thrift.http.port is configured with - // 0 which represents any free port, we should set it to the actual one - portNum = connector.getLocalPort(); - String msg = "Started " + ThriftHttpCLIService.class.getSimpleName() + " in " + schemeName - + " mode on port " + connector.getLocalPort()+ " path=" + httpPath + " with " + minWorkerThreads + "..." - + maxWorkerThreads + " worker threads"; - LOG.info(msg); - } catch (Exception t) { - throw new ServiceException("Error initializing " + getName(), t); - } - } - - /** - * Configure Jetty to serve http requests. Example of a client connection URL: - * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, - * e.g. http://gateway:port/hive2/servlets/thrifths2/ - */ - @Override - public void run() { - try { - httpServer.join(); - } catch (Throwable t) { - LOG.fatal( - "Error starting HiveServer2: could not start " - + ThriftHttpCLIService.class.getSimpleName(), t); - System.exit(-1); - } - } - - /** - * The config parameter can be like "path", "/path", "/path/", "path/*", "/path1/path2/*" and so on. - * httpPath should end up as "/*", "/path/*" or "/path1/../pathN/*" - * @param httpPath - * @return - */ - private String getHttpPath(String httpPath) { - if(httpPath == null || httpPath.equals("")) { - httpPath = "/*"; - } - else { - if(!httpPath.startsWith("/")) { - httpPath = "/" + httpPath; - } - if(httpPath.endsWith("/")) { - httpPath = httpPath + "*"; - } - if(!httpPath.endsWith("/*")) { - httpPath = httpPath + "/*"; - } - } - return httpPath; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java deleted file mode 100644 index e15d2d0566d2b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java +++ /dev/null @@ -1,545 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.cli.thrift; - -import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.security.PrivilegedExceptionAction; -import java.util.Map; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.TimeUnit; - -import javax.servlet.ServletException; -import javax.servlet.http.Cookie; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; -import javax.ws.rs.core.NewCookie; - -import org.apache.commons.codec.binary.Base64; -import org.apache.commons.codec.binary.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.shims.HadoopShims.KerberosNameShim; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hive.service.auth.AuthenticationProviderFactory; -import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods; -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.auth.HttpAuthUtils; -import org.apache.hive.service.auth.HttpAuthenticationException; -import org.apache.hive.service.auth.PasswdAuthenticationProvider; -import org.apache.hive.service.cli.session.SessionManager; -import org.apache.hive.service.CookieSigner; -import org.apache.thrift.TProcessor; -import org.apache.thrift.protocol.TProtocolFactory; -import org.apache.thrift.server.TServlet; -import org.ietf.jgss.GSSContext; -import org.ietf.jgss.GSSCredential; -import org.ietf.jgss.GSSException; -import org.ietf.jgss.GSSManager; -import org.ietf.jgss.GSSName; -import org.ietf.jgss.Oid; - -/** - * - * ThriftHttpServlet - * - */ -public class ThriftHttpServlet extends TServlet { - - private static final long serialVersionUID = 1L; - public static final Log LOG = LogFactory.getLog(ThriftHttpServlet.class.getName()); - private final String authType; - private final UserGroupInformation serviceUGI; - private final UserGroupInformation httpUGI; - private HiveConf hiveConf = new HiveConf(); - - // Class members for cookie based authentication. - private CookieSigner signer; - public static final String AUTH_COOKIE = "hive.server2.auth"; - private static final Random RAN = new Random(); - private boolean isCookieAuthEnabled; - private String cookieDomain; - private String cookiePath; - private int cookieMaxAge; - private boolean isCookieSecure; - private boolean isHttpOnlyCookie; - - public ThriftHttpServlet(TProcessor processor, TProtocolFactory protocolFactory, - String authType, UserGroupInformation serviceUGI, UserGroupInformation httpUGI) { - super(processor, protocolFactory); - this.authType = authType; - this.serviceUGI = serviceUGI; - this.httpUGI = httpUGI; - this.isCookieAuthEnabled = hiveConf.getBoolVar( - ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_AUTH_ENABLED); - // Initialize the cookie based authentication related variables. - if (isCookieAuthEnabled) { - // Generate the signer with secret. - String secret = Long.toString(RAN.nextLong()); - LOG.debug("Using the random number as the secret for cookie generation " + secret); - this.signer = new CookieSigner(secret.getBytes()); - this.cookieMaxAge = (int) hiveConf.getTimeVar( - ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_MAX_AGE, TimeUnit.SECONDS); - this.cookieDomain = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_DOMAIN); - this.cookiePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_PATH); - this.isCookieSecure = hiveConf.getBoolVar( - ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_IS_SECURE); - this.isHttpOnlyCookie = hiveConf.getBoolVar( - ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_IS_HTTPONLY); - } - } - - @Override - protected void doPost(HttpServletRequest request, HttpServletResponse response) - throws ServletException, IOException { - String clientUserName = null; - String clientIpAddress; - boolean requireNewCookie = false; - - try { - // If the cookie based authentication is already enabled, parse the - // request and validate the request cookies. - if (isCookieAuthEnabled) { - clientUserName = validateCookie(request); - requireNewCookie = (clientUserName == null); - if (requireNewCookie) { - LOG.info("Could not validate cookie sent, will try to generate a new cookie"); - } - } - // If the cookie based authentication is not enabled or the request does - // not have a valid cookie, use the kerberos or password based authentication - // depending on the server setup. - if (clientUserName == null) { - // For a kerberos setup - if (isKerberosAuthMode(authType)) { - clientUserName = doKerberosAuth(request); - } - // For password based authentication - else { - clientUserName = doPasswdAuth(request, authType); - } - } - LOG.debug("Client username: " + clientUserName); - - // Set the thread local username to be used for doAs if true - SessionManager.setUserName(clientUserName); - - // find proxy user if any from query param - String doAsQueryParam = getDoAsQueryParam(request.getQueryString()); - if (doAsQueryParam != null) { - SessionManager.setProxyUserName(doAsQueryParam); - } - - clientIpAddress = request.getRemoteAddr(); - LOG.debug("Client IP Address: " + clientIpAddress); - // Set the thread local ip address - SessionManager.setIpAddress(clientIpAddress); - // Generate new cookie and add it to the response - if (requireNewCookie && - !authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.NOSASL.toString())) { - String cookieToken = HttpAuthUtils.createCookieToken(clientUserName); - Cookie hs2Cookie = createCookie(signer.signCookie(cookieToken)); - - if (isHttpOnlyCookie) { - response.setHeader("SET-COOKIE", getHttpOnlyCookieHeader(hs2Cookie)); - } else { - response.addCookie(hs2Cookie); - } - LOG.info("Cookie added for clientUserName " + clientUserName); - } - super.doPost(request, response); - } - catch (HttpAuthenticationException e) { - LOG.error("Error: ", e); - // Send a 401 to the client - response.setStatus(HttpServletResponse.SC_UNAUTHORIZED); - if(isKerberosAuthMode(authType)) { - response.addHeader(HttpAuthUtils.WWW_AUTHENTICATE, HttpAuthUtils.NEGOTIATE); - } - response.getWriter().println("Authentication Error: " + e.getMessage()); - } - finally { - // Clear the thread locals - SessionManager.clearUserName(); - SessionManager.clearIpAddress(); - SessionManager.clearProxyUserName(); - } - } - - /** - * Retrieves the client name from cookieString. If the cookie does not - * correspond to a valid client, the function returns null. - * @param cookies HTTP Request cookies. - * @return Client Username if cookieString has a HS2 Generated cookie that is currently valid. - * Else, returns null. - */ - private String getClientNameFromCookie(Cookie[] cookies) { - // Current Cookie Name, Current Cookie Value - String currName, currValue; - - // Following is the main loop which iterates through all the cookies send by the client. - // The HS2 generated cookies are of the format hive.server2.auth= - // A cookie which is identified as a hiveserver2 generated cookie is validated - // by calling signer.verifyAndExtract(). If the validation passes, send the - // username for which the cookie is validated to the caller. If no client side - // cookie passes the validation, return null to the caller. - for (Cookie currCookie : cookies) { - // Get the cookie name - currName = currCookie.getName(); - if (!currName.equals(AUTH_COOKIE)) { - // Not a HS2 generated cookie, continue. - continue; - } - // If we reached here, we have match for HS2 generated cookie - currValue = currCookie.getValue(); - // Validate the value. - currValue = signer.verifyAndExtract(currValue); - // Retrieve the user name, do the final validation step. - if (currValue != null) { - String userName = HttpAuthUtils.getUserNameFromCookieToken(currValue); - - if (userName == null) { - LOG.warn("Invalid cookie token " + currValue); - continue; - } - //We have found a valid cookie in the client request. - if (LOG.isDebugEnabled()) { - LOG.debug("Validated the cookie for user " + userName); - } - return userName; - } - } - // No valid HS2 generated cookies found, return null - return null; - } - - /** - * Convert cookie array to human readable cookie string - * @param cookies Cookie Array - * @return String containing all the cookies separated by a newline character. - * Each cookie is of the format [key]=[value] - */ - private String toCookieStr(Cookie[] cookies) { - String cookieStr = ""; - - for (Cookie c : cookies) { - cookieStr += c.getName() + "=" + c.getValue() + " ;\n"; - } - return cookieStr; - } - - /** - * Validate the request cookie. This function iterates over the request cookie headers - * and finds a cookie that represents a valid client/server session. If it finds one, it - * returns the client name associated with the session. Else, it returns null. - * @param request The HTTP Servlet Request send by the client - * @return Client Username if the request has valid HS2 cookie, else returns null - * @throws UnsupportedEncodingException - */ - private String validateCookie(HttpServletRequest request) throws UnsupportedEncodingException { - // Find all the valid cookies associated with the request. - Cookie[] cookies = request.getCookies(); - - if (cookies == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("No valid cookies associated with the request " + request); - } - return null; - } - if (LOG.isDebugEnabled()) { - LOG.debug("Received cookies: " + toCookieStr(cookies)); - } - return getClientNameFromCookie(cookies); - } - - /** - * Generate a server side cookie given the cookie value as the input. - * @param str Input string token. - * @return The generated cookie. - * @throws UnsupportedEncodingException - */ - private Cookie createCookie(String str) throws UnsupportedEncodingException { - if (LOG.isDebugEnabled()) { - LOG.debug("Cookie name = " + AUTH_COOKIE + " value = " + str); - } - Cookie cookie = new Cookie(AUTH_COOKIE, str); - - cookie.setMaxAge(cookieMaxAge); - if (cookieDomain != null) { - cookie.setDomain(cookieDomain); - } - if (cookiePath != null) { - cookie.setPath(cookiePath); - } - cookie.setSecure(isCookieSecure); - return cookie; - } - - /** - * Generate httponly cookie from HS2 cookie - * @param cookie HS2 generated cookie - * @return The httponly cookie - */ - private static String getHttpOnlyCookieHeader(Cookie cookie) { - NewCookie newCookie = new NewCookie(cookie.getName(), cookie.getValue(), - cookie.getPath(), cookie.getDomain(), cookie.getVersion(), - cookie.getComment(), cookie.getMaxAge(), cookie.getSecure()); - return newCookie + "; HttpOnly"; - } - - /** - * Do the LDAP/PAM authentication - * @param request - * @param authType - * @throws HttpAuthenticationException - */ - private String doPasswdAuth(HttpServletRequest request, String authType) - throws HttpAuthenticationException { - String userName = getUsername(request, authType); - // No-op when authType is NOSASL - if (!authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.NOSASL.toString())) { - try { - AuthMethods authMethod = AuthMethods.getValidAuthMethod(authType); - PasswdAuthenticationProvider provider = - AuthenticationProviderFactory.getAuthenticationProvider(authMethod); - provider.Authenticate(userName, getPassword(request, authType)); - - } catch (Exception e) { - throw new HttpAuthenticationException(e); - } - } - return userName; - } - - /** - * Do the GSS-API kerberos authentication. - * We already have a logged in subject in the form of serviceUGI, - * which GSS-API will extract information from. - * In case of a SPNego request we use the httpUGI, - * for the authenticating service tickets. - * @param request - * @return - * @throws HttpAuthenticationException - */ - private String doKerberosAuth(HttpServletRequest request) - throws HttpAuthenticationException { - // Try authenticating with the http/_HOST principal - if (httpUGI != null) { - try { - return httpUGI.doAs(new HttpKerberosServerAction(request, httpUGI)); - } catch (Exception e) { - LOG.info("Failed to authenticate with http/_HOST kerberos principal, " + - "trying with hive/_HOST kerberos principal"); - } - } - // Now try with hive/_HOST principal - try { - return serviceUGI.doAs(new HttpKerberosServerAction(request, serviceUGI)); - } catch (Exception e) { - LOG.error("Failed to authenticate with hive/_HOST kerberos principal"); - throw new HttpAuthenticationException(e); - } - - } - - class HttpKerberosServerAction implements PrivilegedExceptionAction { - HttpServletRequest request; - UserGroupInformation serviceUGI; - - HttpKerberosServerAction(HttpServletRequest request, - UserGroupInformation serviceUGI) { - this.request = request; - this.serviceUGI = serviceUGI; - } - - @Override - public String run() throws HttpAuthenticationException { - // Get own Kerberos credentials for accepting connection - GSSManager manager = GSSManager.getInstance(); - GSSContext gssContext = null; - String serverPrincipal = getPrincipalWithoutRealm( - serviceUGI.getUserName()); - try { - // This Oid for Kerberos GSS-API mechanism. - Oid kerberosMechOid = new Oid("1.2.840.113554.1.2.2"); - // Oid for SPNego GSS-API mechanism. - Oid spnegoMechOid = new Oid("1.3.6.1.5.5.2"); - // Oid for kerberos principal name - Oid krb5PrincipalOid = new Oid("1.2.840.113554.1.2.2.1"); - - // GSS name for server - GSSName serverName = manager.createName(serverPrincipal, krb5PrincipalOid); - - // GSS credentials for server - GSSCredential serverCreds = manager.createCredential(serverName, - GSSCredential.DEFAULT_LIFETIME, - new Oid[]{kerberosMechOid, spnegoMechOid}, - GSSCredential.ACCEPT_ONLY); - - // Create a GSS context - gssContext = manager.createContext(serverCreds); - // Get service ticket from the authorization header - String serviceTicketBase64 = getAuthHeader(request, authType); - byte[] inToken = Base64.decodeBase64(serviceTicketBase64.getBytes()); - gssContext.acceptSecContext(inToken, 0, inToken.length); - // Authenticate or deny based on its context completion - if (!gssContext.isEstablished()) { - throw new HttpAuthenticationException("Kerberos authentication failed: " + - "unable to establish context with the service ticket " + - "provided by the client."); - } - else { - return getPrincipalWithoutRealmAndHost(gssContext.getSrcName().toString()); - } - } - catch (GSSException e) { - throw new HttpAuthenticationException("Kerberos authentication failed: ", e); - } - finally { - if (gssContext != null) { - try { - gssContext.dispose(); - } catch (GSSException e) { - // No-op - } - } - } - } - - private String getPrincipalWithoutRealm(String fullPrincipal) - throws HttpAuthenticationException { - KerberosNameShim fullKerberosName; - try { - fullKerberosName = ShimLoader.getHadoopShims().getKerberosNameShim(fullPrincipal); - } catch (IOException e) { - throw new HttpAuthenticationException(e); - } - String serviceName = fullKerberosName.getServiceName(); - String hostName = fullKerberosName.getHostName(); - String principalWithoutRealm = serviceName; - if (hostName != null) { - principalWithoutRealm = serviceName + "/" + hostName; - } - return principalWithoutRealm; - } - - private String getPrincipalWithoutRealmAndHost(String fullPrincipal) - throws HttpAuthenticationException { - KerberosNameShim fullKerberosName; - try { - fullKerberosName = ShimLoader.getHadoopShims().getKerberosNameShim(fullPrincipal); - return fullKerberosName.getShortName(); - } catch (IOException e) { - throw new HttpAuthenticationException(e); - } - } - } - - private String getUsername(HttpServletRequest request, String authType) - throws HttpAuthenticationException { - String[] creds = getAuthHeaderTokens(request, authType); - // Username must be present - if (creds[0] == null || creds[0].isEmpty()) { - throw new HttpAuthenticationException("Authorization header received " + - "from the client does not contain username."); - } - return creds[0]; - } - - private String getPassword(HttpServletRequest request, String authType) - throws HttpAuthenticationException { - String[] creds = getAuthHeaderTokens(request, authType); - // Password must be present - if (creds[1] == null || creds[1].isEmpty()) { - throw new HttpAuthenticationException("Authorization header received " + - "from the client does not contain username."); - } - return creds[1]; - } - - private String[] getAuthHeaderTokens(HttpServletRequest request, - String authType) throws HttpAuthenticationException { - String authHeaderBase64 = getAuthHeader(request, authType); - String authHeaderString = StringUtils.newStringUtf8( - Base64.decodeBase64(authHeaderBase64.getBytes())); - String[] creds = authHeaderString.split(":"); - return creds; - } - - /** - * Returns the base64 encoded auth header payload - * @param request - * @param authType - * @return - * @throws HttpAuthenticationException - */ - private String getAuthHeader(HttpServletRequest request, String authType) - throws HttpAuthenticationException { - String authHeader = request.getHeader(HttpAuthUtils.AUTHORIZATION); - // Each http request must have an Authorization header - if (authHeader == null || authHeader.isEmpty()) { - throw new HttpAuthenticationException("Authorization header received " + - "from the client is empty."); - } - - String authHeaderBase64String; - int beginIndex; - if (isKerberosAuthMode(authType)) { - beginIndex = (HttpAuthUtils.NEGOTIATE + " ").length(); - } - else { - beginIndex = (HttpAuthUtils.BASIC + " ").length(); - } - authHeaderBase64String = authHeader.substring(beginIndex); - // Authorization header must have a payload - if (authHeaderBase64String == null || authHeaderBase64String.isEmpty()) { - throw new HttpAuthenticationException("Authorization header received " + - "from the client does not contain any data."); - } - return authHeaderBase64String; - } - - private boolean isKerberosAuthMode(String authType) { - return authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString()); - } - - private static String getDoAsQueryParam(String queryString) { - if (LOG.isDebugEnabled()) { - LOG.debug("URL query string:" + queryString); - } - if (queryString == null) { - return null; - } - Map params = javax.servlet.http.HttpUtils.parseQueryString( queryString ); - Set keySet = params.keySet(); - for (String key: keySet) { - if (key.equalsIgnoreCase("doAs")) { - return params.get(key)[0]; - } - } - return null; - } - -} - - diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/HiveServer2.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/HiveServer2.java deleted file mode 100644 index 95233996cbbcb..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/HiveServer2.java +++ /dev/null @@ -1,277 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.service.server; - -import java.util.Properties; - -import scala.runtime.AbstractFunction0; -import scala.runtime.BoxedUnit; - -import org.apache.commons.cli.GnuParser; -import org.apache.commons.cli.HelpFormatter; -import org.apache.commons.cli.Option; -import org.apache.commons.cli.OptionBuilder; -import org.apache.commons.cli.Options; -import org.apache.commons.cli.ParseException; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hive.common.util.HiveStringUtils; -import org.apache.hive.service.CompositeService; -import org.apache.hive.service.cli.CLIService; -import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService; -import org.apache.hive.service.cli.thrift.ThriftCLIService; -import org.apache.hive.service.cli.thrift.ThriftHttpCLIService; - -import org.apache.spark.util.ShutdownHookManager; - -/** - * HiveServer2. - * - */ -public class HiveServer2 extends CompositeService { - private static final Log LOG = LogFactory.getLog(HiveServer2.class); - - private CLIService cliService; - private ThriftCLIService thriftCLIService; - - public HiveServer2() { - super(HiveServer2.class.getSimpleName()); - HiveConf.setLoadHiveServer2Config(true); - } - - @Override - public synchronized void init(HiveConf hiveConf) { - cliService = new CLIService(this); - addService(cliService); - if (isHTTPTransportMode(hiveConf)) { - thriftCLIService = new ThriftHttpCLIService(cliService); - } else { - thriftCLIService = new ThriftBinaryCLIService(cliService); - } - addService(thriftCLIService); - super.init(hiveConf); - - // Add a shutdown hook for catching SIGTERM & SIGINT - // this must be higher than the Hadoop Filesystem priority of 10, - // which the default priority is. - // The signature of the callback must match that of a scala () -> Unit - // function - ShutdownHookManager.addShutdownHook( - new AbstractFunction0() { - public BoxedUnit apply() { - try { - LOG.info("Hive Server Shutdown hook invoked"); - stop(); - } catch (Throwable e) { - LOG.warn("Ignoring Exception while stopping Hive Server from shutdown hook", - e); - } - return BoxedUnit.UNIT; - } - }); - } - - public static boolean isHTTPTransportMode(HiveConf hiveConf) { - String transportMode = System.getenv("HIVE_SERVER2_TRANSPORT_MODE"); - if (transportMode == null) { - transportMode = hiveConf.getVar(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE); - } - if (transportMode != null && (transportMode.equalsIgnoreCase("http"))) { - return true; - } - return false; - } - - @Override - public synchronized void start() { - super.start(); - } - - @Override - public synchronized void stop() { - LOG.info("Shutting down HiveServer2"); - super.stop(); - } - - private static void startHiveServer2() throws Throwable { - long attempts = 0, maxAttempts = 1; - while (true) { - LOG.info("Starting HiveServer2"); - HiveConf hiveConf = new HiveConf(); - maxAttempts = hiveConf.getLongVar(HiveConf.ConfVars.HIVE_SERVER2_MAX_START_ATTEMPTS); - HiveServer2 server = null; - try { - server = new HiveServer2(); - server.init(hiveConf); - server.start(); - ShimLoader.getHadoopShims().startPauseMonitor(hiveConf); - break; - } catch (Throwable throwable) { - if (server != null) { - try { - server.stop(); - } catch (Throwable t) { - LOG.info("Exception caught when calling stop of HiveServer2 before retrying start", t); - } finally { - server = null; - } - } - if (++attempts >= maxAttempts) { - throw new Error("Max start attempts " + maxAttempts + " exhausted", throwable); - } else { - LOG.warn("Error starting HiveServer2 on attempt " + attempts - + ", will retry in 60 seconds", throwable); - try { - Thread.sleep(60L * 1000L); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - } - } - } - - public static void main(String[] args) { - HiveConf.setLoadHiveServer2Config(true); - ServerOptionsProcessor oproc = new ServerOptionsProcessor("hiveserver2"); - ServerOptionsProcessorResponse oprocResponse = oproc.parse(args); - - HiveStringUtils.startupShutdownMessage(HiveServer2.class, args, LOG); - - // Call the executor which will execute the appropriate command based on the parsed options - oprocResponse.getServerOptionsExecutor().execute(); - } - - /** - * ServerOptionsProcessor. - * Process arguments given to HiveServer2 (-hiveconf property=value) - * Set properties in System properties - * Create an appropriate response object, - * which has executor to execute the appropriate command based on the parsed options. - */ - public static class ServerOptionsProcessor { - private final Options options = new Options(); - private org.apache.commons.cli.CommandLine commandLine; - private final String serverName; - private final StringBuilder debugMessage = new StringBuilder(); - - @SuppressWarnings("static-access") - public ServerOptionsProcessor(String serverName) { - this.serverName = serverName; - // -hiveconf x=y - options.addOption(OptionBuilder - .withValueSeparator() - .hasArgs(2) - .withArgName("property=value") - .withLongOpt("hiveconf") - .withDescription("Use value for given property") - .create()); - options.addOption(new Option("H", "help", false, "Print help information")); - } - - public ServerOptionsProcessorResponse parse(String[] argv) { - try { - commandLine = new GnuParser().parse(options, argv); - // Process --hiveconf - // Get hiveconf param values and set the System property values - Properties confProps = commandLine.getOptionProperties("hiveconf"); - for (String propKey : confProps.stringPropertyNames()) { - // save logging message for log4j output latter after log4j initialize properly - debugMessage.append("Setting " + propKey + "=" + confProps.getProperty(propKey) + ";\n"); - System.setProperty(propKey, confProps.getProperty(propKey)); - } - - // Process --help - if (commandLine.hasOption('H')) { - return new ServerOptionsProcessorResponse(new HelpOptionExecutor(serverName, options)); - } - } catch (ParseException e) { - // Error out & exit - we were not able to parse the args successfully - System.err.println("Error starting HiveServer2 with given arguments: "); - System.err.println(e.getMessage()); - System.exit(-1); - } - // Default executor, when no option is specified - return new ServerOptionsProcessorResponse(new StartOptionExecutor()); - } - - StringBuilder getDebugMessage() { - return debugMessage; - } - } - - /** - * The response sent back from {@link ServerOptionsProcessor#parse(String[])} - */ - static class ServerOptionsProcessorResponse { - private final ServerOptionsExecutor serverOptionsExecutor; - - ServerOptionsProcessorResponse(ServerOptionsExecutor serverOptionsExecutor) { - this.serverOptionsExecutor = serverOptionsExecutor; - } - - ServerOptionsExecutor getServerOptionsExecutor() { - return serverOptionsExecutor; - } - } - - /** - * The executor interface for running the appropriate HiveServer2 command based on parsed options - */ - interface ServerOptionsExecutor { - void execute(); - } - - /** - * HelpOptionExecutor: executes the --help option by printing out the usage - */ - static class HelpOptionExecutor implements ServerOptionsExecutor { - private final Options options; - private final String serverName; - - HelpOptionExecutor(String serverName, Options options) { - this.options = options; - this.serverName = serverName; - } - - @Override - public void execute() { - new HelpFormatter().printHelp(serverName, options); - System.exit(0); - } - } - - /** - * StartOptionExecutor: starts HiveServer2. - * This is the default executor, when no option is specified. - */ - static class StartOptionExecutor implements ServerOptionsExecutor { - @Override - public void execute() { - try { - startHiveServer2(); - } catch (Throwable t) { - LOG.fatal("Error starting HiveServer2", t); - System.exit(-1); - } - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java deleted file mode 100644 index 8ee98103f7ef7..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java +++ /dev/null @@ -1,77 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.hive.service.server; - -import java.util.Map; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.metastore.HiveMetaStore; -import org.apache.hadoop.hive.metastore.RawStore; - -/** - * A HiveServer2 thread used to construct new server threads. - * In particular, this thread ensures an orderly cleanup, - * when killed by its corresponding ExecutorService. - */ -public class ThreadWithGarbageCleanup extends Thread { - private static final Log LOG = LogFactory.getLog(ThreadWithGarbageCleanup.class); - - Map threadRawStoreMap = - ThreadFactoryWithGarbageCleanup.getThreadRawStoreMap(); - - public ThreadWithGarbageCleanup(Runnable runnable) { - super(runnable); - } - - /** - * Add any Thread specific garbage cleanup code here. - * Currently, it shuts down the RawStore object for this thread if it is not null. - */ - @Override - public void finalize() throws Throwable { - cleanRawStore(); - super.finalize(); - } - - private void cleanRawStore() { - Long threadId = this.getId(); - RawStore threadLocalRawStore = threadRawStoreMap.get(threadId); - if (threadLocalRawStore != null) { - LOG.debug("RawStore: " + threadLocalRawStore + ", for the thread: " + - this.getName() + " will be closed now."); - threadLocalRawStore.shutdown(); - threadRawStoreMap.remove(threadId); - } - } - - /** - * Cache the ThreadLocal RawStore object. Called from the corresponding thread. - */ - public void cacheThreadLocalRawStore() { - Long threadId = this.getId(); - RawStore threadLocalRawStore = HiveMetaStore.HMSHandler.getRawStore(); - if (threadLocalRawStore != null && !threadRawStoreMap.containsKey(threadId)) { - LOG.debug("Adding RawStore: " + threadLocalRawStore + ", for the thread: " + - this.getName() + " to threadRawStoreMap for future cleanup."); - threadRawStoreMap.put(threadId, threadLocalRawStore); - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala b/sql/hive-thriftserver/v1.2/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala deleted file mode 100644 index 9a28dd6a31e6e..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import org.apache.commons.logging.LogFactory -import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hive.service.cli.{RowSet, RowSetFactory, TableSchema, Type} -import org.apache.hive.service.cli.Type._ -import org.apache.hive.service.cli.thrift.TProtocolVersion._ - -/** - * Various utilities for hive-thriftserver used to upgrade the built-in Hive. - */ -private[thriftserver] object ThriftserverShimUtils { - - private[thriftserver] object TOperationType { - val GET_TYPE_INFO = org.apache.hive.service.cli.thrift.TOperationType.GET_TYPE_INFO - } - - private[thriftserver] type TProtocolVersion = org.apache.hive.service.cli.thrift.TProtocolVersion - private[thriftserver] type Client = org.apache.hive.service.cli.thrift.TCLIService.Client - private[thriftserver] type TOpenSessionReq = org.apache.hive.service.cli.thrift.TOpenSessionReq - private[thriftserver] type TGetSchemasReq = org.apache.hive.service.cli.thrift.TGetSchemasReq - private[thriftserver] type TGetTablesReq = org.apache.hive.service.cli.thrift.TGetTablesReq - private[thriftserver] type TGetColumnsReq = org.apache.hive.service.cli.thrift.TGetColumnsReq - private[thriftserver] type TGetInfoReq = org.apache.hive.service.cli.thrift.TGetInfoReq - private[thriftserver] type TExecuteStatementReq = - org.apache.hive.service.cli.thrift.TExecuteStatementReq - private[thriftserver] type THandleIdentifier = - org.apache.hive.service.cli.thrift.THandleIdentifier - private[thriftserver] type TOperationType = org.apache.hive.service.cli.thrift.TOperationType - private[thriftserver] type TOperationHandle = org.apache.hive.service.cli.thrift.TOperationHandle - - private[thriftserver] def getConsole: SessionState.LogHelper = { - val LOG = LogFactory.getLog(classOf[SparkSQLCLIDriver]) - new SessionState.LogHelper(LOG) - } - - private[thriftserver] def resultRowSet( - getResultSetSchema: TableSchema, - getProtocolVersion: TProtocolVersion): RowSet = { - RowSetFactory.create(getResultSetSchema, getProtocolVersion) - } - - private[thriftserver] def supportedType(): Seq[Type] = { - Seq(NULL_TYPE, BOOLEAN_TYPE, STRING_TYPE, BINARY_TYPE, - TINYINT_TYPE, SMALLINT_TYPE, INT_TYPE, BIGINT_TYPE, - FLOAT_TYPE, DOUBLE_TYPE, DECIMAL_TYPE, - DATE_TYPE, TIMESTAMP_TYPE, - ARRAY_TYPE, MAP_TYPE, STRUCT_TYPE) - } - - private[thriftserver] val testedProtocolVersions = Seq( - HIVE_CLI_SERVICE_PROTOCOL_V1, - HIVE_CLI_SERVICE_PROTOCOL_V2, - HIVE_CLI_SERVICE_PROTOCOL_V3, - HIVE_CLI_SERVICE_PROTOCOL_V4, - HIVE_CLI_SERVICE_PROTOCOL_V5, - HIVE_CLI_SERVICE_PROTOCOL_V6, - HIVE_CLI_SERVICE_PROTOCOL_V7, - HIVE_CLI_SERVICE_PROTOCOL_V8) -} diff --git a/sql/hive-thriftserver/v2.3/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala b/sql/hive-thriftserver/v2.3/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala deleted file mode 100644 index c8ac5226b296e..0000000000000 --- a/sql/hive-thriftserver/v2.3/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.hive.serde2.thrift.Type -import org.apache.hadoop.hive.serde2.thrift.Type._ -import org.apache.hive.service.cli.{RowSet, RowSetFactory, TableSchema} -import org.apache.hive.service.rpc.thrift.TProtocolVersion._ -import org.slf4j.LoggerFactory - -/** - * Various utilities for hive-thriftserver used to upgrade the built-in Hive. - */ -private[thriftserver] object ThriftserverShimUtils { - - private[thriftserver] object TOperationType { - val GET_TYPE_INFO = org.apache.hive.service.rpc.thrift.TOperationType.GET_TYPE_INFO - } - - private[thriftserver] type TProtocolVersion = org.apache.hive.service.rpc.thrift.TProtocolVersion - private[thriftserver] type Client = org.apache.hive.service.rpc.thrift.TCLIService.Client - private[thriftserver] type TOpenSessionReq = org.apache.hive.service.rpc.thrift.TOpenSessionReq - private[thriftserver] type TGetSchemasReq = org.apache.hive.service.rpc.thrift.TGetSchemasReq - private[thriftserver] type TGetTablesReq = org.apache.hive.service.rpc.thrift.TGetTablesReq - private[thriftserver] type TGetColumnsReq = org.apache.hive.service.rpc.thrift.TGetColumnsReq - private[thriftserver] type TGetInfoReq = org.apache.hive.service.rpc.thrift.TGetInfoReq - private[thriftserver] type TExecuteStatementReq = - org.apache.hive.service.rpc.thrift.TExecuteStatementReq - private[thriftserver] type THandleIdentifier = - org.apache.hive.service.rpc.thrift.THandleIdentifier - private[thriftserver] type TOperationType = org.apache.hive.service.rpc.thrift.TOperationType - private[thriftserver] type TOperationHandle = org.apache.hive.service.rpc.thrift.TOperationHandle - - private[thriftserver] def getConsole: SessionState.LogHelper = { - val LOG = LoggerFactory.getLogger(classOf[SparkSQLCLIDriver]) - new SessionState.LogHelper(LOG) - } - - private[thriftserver] def resultRowSet( - getResultSetSchema: TableSchema, - getProtocolVersion: TProtocolVersion): RowSet = { - RowSetFactory.create(getResultSetSchema, getProtocolVersion, false) - } - - private[thriftserver] def supportedType(): Seq[Type] = { - Seq(NULL_TYPE, BOOLEAN_TYPE, STRING_TYPE, BINARY_TYPE, - TINYINT_TYPE, SMALLINT_TYPE, INT_TYPE, BIGINT_TYPE, - FLOAT_TYPE, DOUBLE_TYPE, DECIMAL_TYPE, - DATE_TYPE, TIMESTAMP_TYPE, - ARRAY_TYPE, MAP_TYPE, STRUCT_TYPE) - } - - private[thriftserver] val testedProtocolVersions = Seq( - HIVE_CLI_SERVICE_PROTOCOL_V1, - HIVE_CLI_SERVICE_PROTOCOL_V2, - HIVE_CLI_SERVICE_PROTOCOL_V3, - HIVE_CLI_SERVICE_PROTOCOL_V4, - HIVE_CLI_SERVICE_PROTOCOL_V5, - HIVE_CLI_SERVICE_PROTOCOL_V6, - HIVE_CLI_SERVICE_PROTOCOL_V7, - HIVE_CLI_SERVICE_PROTOCOL_V8, - HIVE_CLI_SERVICE_PROTOCOL_V9, - HIVE_CLI_SERVICE_PROTOCOL_V10) -} diff --git a/sql/hive/benchmarks/InsertIntoHiveTableBenchmark-hive1.2-results.txt b/sql/hive/benchmarks/InsertIntoHiveTableBenchmark-hive1.2-results.txt deleted file mode 100644 index 85884a1aaf739..0000000000000 --- a/sql/hive/benchmarks/InsertIntoHiveTableBenchmark-hive1.2-results.txt +++ /dev/null @@ -1,11 +0,0 @@ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz -insert hive table benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -INSERT INTO DYNAMIC 6812 7043 328 0.0 665204.8 1.0X -INSERT INTO HYBRID 817 852 32 0.0 79783.6 8.3X -INSERT INTO STATIC 231 246 21 0.0 22568.2 29.5X -INSERT OVERWRITE DYNAMIC 25947 26671 1024 0.0 2533910.2 0.3X -INSERT OVERWRITE HYBRID 2846 2884 54 0.0 277908.7 2.4X -INSERT OVERWRITE STATIC 232 247 26 0.0 22659.9 29.4X - diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index b7ea0630dd85f..a685549290f0e 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -1145,11 +1145,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { * The set of tests that are believed to be working in catalyst. Tests not on includeList or * excludeList are implicitly marked as ignored. */ - override def includeList: Seq[String] = if (HiveUtils.isHive23) { + override def includeList: Seq[String] = commonIncludeList ++ Seq( "decimal_1_1" ) - } else { - commonIncludeList - } } diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 474c6066ed040..0453094cf8b7b 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -35,11 +35,6 @@ - - - ${hive.parquet.group} - parquet-hadoop-bundle - org.apache.spark spark-core_${scala.binary.version} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index bc7760c982aab..8a248a251820f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -30,7 +30,7 @@ import org.apache.hadoop.hive.ql.udf.generic.{AbstractGenericUDAFResolver, Gener import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.FunctionRegistry -import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, ExternalCatalog, FunctionResourceLoader, GlobalTempViewManager, SessionCatalog} +import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{Cast, Expression} import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper @@ -57,6 +57,56 @@ private[sql] class HiveSessionCatalog( parser, functionResourceLoader) { + private def makeHiveFunctionExpression( + name: String, + clazz: Class[_], + input: Seq[Expression]): Expression = { + var udfExpr: Option[Expression] = None + try { + // When we instantiate hive UDF wrapper class, we may throw exception if the input + // expressions don't satisfy the hive UDF, such as type mismatch, input number + // mismatch, etc. Here we catch the exception and throw AnalysisException instead. + if (classOf[UDF].isAssignableFrom(clazz)) { + udfExpr = Some(HiveSimpleUDF(name, new HiveFunctionWrapper(clazz.getName), input)) + udfExpr.get.dataType // Force it to check input data types. + } else if (classOf[GenericUDF].isAssignableFrom(clazz)) { + udfExpr = Some(HiveGenericUDF(name, new HiveFunctionWrapper(clazz.getName), input)) + udfExpr.get.dataType // Force it to check input data types. + } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(clazz)) { + udfExpr = Some(HiveUDAFFunction(name, new HiveFunctionWrapper(clazz.getName), input)) + udfExpr.get.dataType // Force it to check input data types. + } else if (classOf[UDAF].isAssignableFrom(clazz)) { + udfExpr = Some(HiveUDAFFunction( + name, + new HiveFunctionWrapper(clazz.getName), + input, + isUDAFBridgeRequired = true)) + udfExpr.get.dataType // Force it to check input data types. + } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) { + udfExpr = Some(HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), input)) + // Force it to check data types. + udfExpr.get.asInstanceOf[HiveGenericUDTF].elementSchema + } + } catch { + case NonFatal(e) => + val noHandlerMsg = s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}': $e" + val errorMsg = + if (classOf[GenericUDTF].isAssignableFrom(clazz)) { + s"$noHandlerMsg\nPlease make sure your function overrides " + + "`public StructObjectInspector initialize(ObjectInspector[] args)`." + } else { + noHandlerMsg + } + val analysisException = new AnalysisException(errorMsg) + analysisException.setStackTrace(e.getStackTrace) + throw analysisException + } + udfExpr.getOrElse { + throw new InvalidUDFClassException( + s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}'") + } + } + /** * Constructs a [[Expression]] based on the provided class that represents a function. * @@ -69,49 +119,14 @@ private[sql] class HiveSessionCatalog( // Current thread context classloader may not be the one loaded the class. Need to switch // context classloader to initialize instance properly. Utils.withContextClassLoader(clazz.getClassLoader) { - Try(super.makeFunctionExpression(name, clazz, input)).getOrElse { - var udfExpr: Option[Expression] = None - try { - // When we instantiate hive UDF wrapper class, we may throw exception if the input - // expressions don't satisfy the hive UDF, such as type mismatch, input number - // mismatch, etc. Here we catch the exception and throw AnalysisException instead. - if (classOf[UDF].isAssignableFrom(clazz)) { - udfExpr = Some(HiveSimpleUDF(name, new HiveFunctionWrapper(clazz.getName), input)) - udfExpr.get.dataType // Force it to check input data types. - } else if (classOf[GenericUDF].isAssignableFrom(clazz)) { - udfExpr = Some(HiveGenericUDF(name, new HiveFunctionWrapper(clazz.getName), input)) - udfExpr.get.dataType // Force it to check input data types. - } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(clazz)) { - udfExpr = Some(HiveUDAFFunction(name, new HiveFunctionWrapper(clazz.getName), input)) - udfExpr.get.dataType // Force it to check input data types. - } else if (classOf[UDAF].isAssignableFrom(clazz)) { - udfExpr = Some(HiveUDAFFunction( - name, - new HiveFunctionWrapper(clazz.getName), - input, - isUDAFBridgeRequired = true)) - udfExpr.get.dataType // Force it to check input data types. - } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) { - udfExpr = Some(HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), input)) - udfExpr.get.asInstanceOf[HiveGenericUDTF].elementSchema // Force it to check data types. - } - } catch { - case NonFatal(e) => - val noHandlerMsg = s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}': $e" - val errorMsg = - if (classOf[GenericUDTF].isAssignableFrom(clazz)) { - s"$noHandlerMsg\nPlease make sure your function overrides " + - "`public StructObjectInspector initialize(ObjectInspector[] args)`." - } else { - noHandlerMsg - } - val analysisException = new AnalysisException(errorMsg) - analysisException.setStackTrace(e.getStackTrace) - throw analysisException - } - udfExpr.getOrElse { - throw new AnalysisException(s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}'") - } + try { + super.makeFunctionExpression(name, clazz, input) + } catch { + // If `super.makeFunctionExpression` throw `InvalidUDFClassException`, we construct + // Hive UDF/UDAF/UDTF with function definition. Otherwise, we just throw it earlier. + case _: InvalidUDFClassException => + makeHiveFunctionExpression(name, clazz, input) + case NonFatal(e) => throw e } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 78ec2b8e2047e..b9135733856a5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -21,10 +21,9 @@ import org.apache.spark.annotation.Unstable import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.{Analyzer, ResolveSessionCatalog} import org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener -import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{SparkOptimizer, SparkPlanner} +import org.apache.spark.sql.execution.SparkPlanner import org.apache.spark.sql.execution.aggregate.ResolveEncodersInScalaAgg import org.apache.spark.sql.execution.analysis.DetectAmbiguousSelfJoin import org.apache.spark.sql.execution.command.CommandCheck @@ -38,8 +37,11 @@ import org.apache.spark.sql.internal.{BaseSessionStateBuilder, SessionResourceLo * Builder that produces a Hive-aware `SessionState`. */ @Unstable -class HiveSessionStateBuilder(session: SparkSession, parentState: Option[SessionState] = None) - extends BaseSessionStateBuilder(session, parentState) { +class HiveSessionStateBuilder( + session: SparkSession, + parentState: Option[SessionState], + options: Map[String, String]) + extends BaseSessionStateBuilder(session, parentState, options) { private def externalCatalog: ExternalCatalogWithListener = session.sharedState.externalCatalog @@ -116,7 +118,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session } } - override protected def newBuilder: NewBuilder = new HiveSessionStateBuilder(_, _) + override protected def newBuilder: NewBuilder = new HiveSessionStateBuilder(_, _, Map.empty) } class HiveSessionResourceLoader( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala index 04a6a8f8aa9a5..3a53a2a8dadd8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala @@ -17,18 +17,16 @@ package org.apache.spark.sql.hive -import java.io.{InputStream, OutputStream} -import java.lang.reflect.Method import java.rmi.server.UID import scala.collection.JavaConverters._ import scala.language.implicitConversions -import scala.reflect.ClassTag import com.google.common.base.Objects import org.apache.avro.Schema import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.ql.exec.SerializationUtilities import org.apache.hadoop.hive.ql.exec.UDF import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro @@ -148,60 +146,12 @@ private[hive] object HiveShim { case _ => false } - private lazy val serUtilClass = - Utils.classForName("org.apache.hadoop.hive.ql.exec.SerializationUtilities") - private lazy val utilClass = Utils.classForName("org.apache.hadoop.hive.ql.exec.Utilities") - private val deserializeMethodName = "deserializeObjectByKryo" - private val serializeMethodName = "serializeObjectByKryo" - - private def findMethod(klass: Class[_], name: String, args: Class[_]*): Method = { - val method = klass.getDeclaredMethod(name, args: _*) - method.setAccessible(true) - method - } - def deserializePlan[UDFType](is: java.io.InputStream, clazz: Class[_]): UDFType = { - if (HiveUtils.isHive23) { - val borrowKryo = serUtilClass.getMethod("borrowKryo") - val kryo = borrowKryo.invoke(serUtilClass) - val deserializeObjectByKryo = findMethod(serUtilClass, deserializeMethodName, - kryo.getClass.getSuperclass, classOf[InputStream], classOf[Class[_]]) - try { - deserializeObjectByKryo.invoke(null, kryo, is, clazz).asInstanceOf[UDFType] - } finally { - serUtilClass.getMethod("releaseKryo", kryo.getClass.getSuperclass).invoke(null, kryo) - } - } else { - val runtimeSerializationKryo = utilClass.getField("runtimeSerializationKryo") - val threadLocalValue = runtimeSerializationKryo.get(utilClass) - val getMethod = threadLocalValue.getClass.getMethod("get") - val kryo = getMethod.invoke(threadLocalValue) - val deserializeObjectByKryo = findMethod(utilClass, deserializeMethodName, - kryo.getClass, classOf[InputStream], classOf[Class[_]]) - deserializeObjectByKryo.invoke(null, kryo, is, clazz).asInstanceOf[UDFType] - } + SerializationUtilities.deserializePlan(is, clazz).asInstanceOf[UDFType] } def serializePlan(function: AnyRef, out: java.io.OutputStream): Unit = { - if (HiveUtils.isHive23) { - val borrowKryo = serUtilClass.getMethod("borrowKryo") - val kryo = borrowKryo.invoke(serUtilClass) - val serializeObjectByKryo = findMethod(serUtilClass, serializeMethodName, - kryo.getClass.getSuperclass, classOf[Object], classOf[OutputStream]) - try { - serializeObjectByKryo.invoke(null, kryo, function, out) - } finally { - serUtilClass.getMethod("releaseKryo", kryo.getClass.getSuperclass).invoke(null, kryo) - } - } else { - val runtimeSerializationKryo = utilClass.getField("runtimeSerializationKryo") - val threadLocalValue = runtimeSerializationKryo.get(utilClass) - val getMethod = threadLocalValue.getClass.getMethod("get") - val kryo = getMethod.invoke(threadLocalValue) - val serializeObjectByKryo = findMethod(utilClass, serializeMethodName, - kryo.getClass, classOf[Object], classOf[OutputStream]) - serializeObjectByKryo.invoke(null, kryo, function, out) - } + SerializationUtilities.serializePlan(function, out) } def writeExternal(out: java.io.ObjectOutput): Unit = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index 62ff2db2ecb3c..96c207913d49a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -55,11 +55,8 @@ private[spark] object HiveUtils extends Logging { sc } - private val hiveVersion = HiveVersionInfo.getVersion - val isHive23: Boolean = hiveVersion.startsWith("2.3") - /** The version of hive used internally by Spark SQL. */ - val builtinHiveVersion: String = if (isHive23) hiveVersion else "1.2.1" + val builtinHiveVersion: String = HiveVersionInfo.getVersion val HIVE_METASTORE_VERSION = buildStaticConf("spark.sql.hive.metastore.version") .doc("Version of the Hive metastore. Available options are " + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 805bcb2bc3a60..a78e1cebc588c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -169,9 +169,7 @@ private[hive] class HiveClientImpl( // since HIVE-11878, and ADDJarCommand will add jars to clientLoader.classLoader. // For this reason we cannot load the jars added by ADDJarCommand because of class loader // got changed. We reset it to clientLoader.ClassLoader here. - if (HiveUtils.isHive23) { - state.getConf.setClassLoader(clientLoader.classLoader) - } + state.getConf.setClassLoader(clientLoader.classLoader) SessionState.start(state) state.out = new PrintStream(outputBuffer, true, UTF_8.name()) state.err = new PrintStream(outputBuffer, true, UTF_8.name()) @@ -179,9 +177,7 @@ private[hive] class HiveClientImpl( } /** Returns the configuration for the current session. */ - def conf: HiveConf = if (!HiveUtils.isHive23) { - state.getConf - } else { + def conf: HiveConf = { val hiveConf = state.getConf // Hive changed the default of datanucleus.schema.autoCreateAll from true to false // and hive.metastore.schema.verification from false to true since Hive 2.0. @@ -293,11 +289,9 @@ private[hive] class HiveClientImpl( val ret = try { f } catch { - case e: NoClassDefFoundError - if HiveUtils.isHive23 && e.getMessage.contains("org/apache/hadoop/hive/serde2/SerDe") => + case e: NoClassDefFoundError if e.getMessage.contains("apache/hadoop/hive/serde2/SerDe") => throw new ClassNotFoundException("The SerDe interface removed since Hive 2.3(HIVE-15167)." + - " Please migrate your custom SerDes to Hive 2.3 or build your own Spark with" + - " hive-1.2 profile. See HIVE-15167 for more details.", e) + " Please migrate your custom SerDes to Hive 2.3. See HIVE-15167 for more details.", e) } finally { state.getConf.setClassLoader(originalConfLoader) Thread.currentThread().setContextClassLoader(original) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 99c9199e466f9..4ab0599e4477b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -1327,8 +1327,6 @@ private[client] class Shim_v3_0 extends Shim_v2_3 { inheritTableSpecs: Boolean, isSkewedStoreAsSubdir: Boolean, isSrcLocal: Boolean): Unit = { - val session = SparkSession.getActiveSession - assert(session.nonEmpty) val table = hive.getTable(tableName) val loadFileType = if (replace) { clazzLoadFileType.getEnumConstants.find(_.toString.equalsIgnoreCase("REPLACE_ALL")) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala index 8ad5cb70d248b..c7002853bed54 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.hive -import java.lang.{Boolean => JBoolean} import java.nio.ByteBuffer import scala.collection.JavaConverters._ @@ -39,8 +38,12 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.hive.HiveShim._ import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils +/** + * Here we cannot extends `ImplicitTypeCasts` to compatible with UDF input data type, the reason is: + * we use children data type to reflect UDF method first and will get exception if it fails so that + * we can never go into `ImplicitTypeCasts`. + */ private[hive] case class HiveSimpleUDF( name: String, funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) extends Expression @@ -344,20 +347,8 @@ private[hive] case class HiveUDAFFunction( funcWrapper.createFunction[AbstractGenericUDAFResolver]() } - val clazz = Utils.classForName(classOf[SimpleGenericUDAFParameterInfo].getName) - if (HiveUtils.isHive23) { - val ctor = clazz.getDeclaredConstructor( - classOf[Array[ObjectInspector]], JBoolean.TYPE, JBoolean.TYPE, JBoolean.TYPE) - val args = Array[AnyRef](inputInspectors, JBoolean.FALSE, JBoolean.FALSE, JBoolean.FALSE) - val parameterInfo = ctor.newInstance(args: _*).asInstanceOf[SimpleGenericUDAFParameterInfo] - resolver.getEvaluator(parameterInfo) - } else { - val ctor = clazz.getDeclaredConstructor( - classOf[Array[ObjectInspector]], JBoolean.TYPE, JBoolean.TYPE) - val args = Array[AnyRef](inputInspectors, JBoolean.FALSE, JBoolean.FALSE) - val parameterInfo = ctor.newInstance(args: _*).asInstanceOf[SimpleGenericUDAFParameterInfo] - resolver.getEvaluator(parameterInfo) - } + val parameterInfo = new SimpleGenericUDAFParameterInfo(inputInspectors, false, false, false) + resolver.getEvaluator(parameterInfo) } private case class HiveEvaluator( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index 356b92b4652b3..2868bb4ba85d3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -47,7 +47,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.orc.OrcOptions +import org.apache.spark.sql.execution.datasources.orc.{OrcFilters, OrcOptions} import org.apache.spark.sql.hive.{HiveInspectors, HiveShim} import org.apache.spark.sql.sources.{Filter, _} import org.apache.spark.sql.types._ @@ -75,7 +75,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles OrcFileOperator.readSchema( files.map(_.getPath.toString), - Some(sparkSession.sessionState.newHadoopConf()), + Some(sparkSession.sessionState.newHadoopConfWithOptions(options)), ignoreCorruptFiles ) } @@ -139,7 +139,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable if (sparkSession.sessionState.conf.orcFilterPushDown) { // Sets pushed predicates - OrcFilters.createFilter(requiredSchema, filters.toArray).foreach { f => + OrcFilters.createFilter(requiredSchema, filters).foreach { f => hadoopConf.set(OrcFileFormat.SARG_PUSHDOWN, toKryo(f)) hadoopConf.setBoolean(ConfVars.HIVEOPTINDEXFILTER.varname, true) } @@ -296,7 +296,7 @@ private[orc] class OrcOutputWriter( override def close(): Unit = { if (recordWriterInstantiated) { - // Hive 1.2.1 ORC initializes its private `writer` field at the first write. + // Hive ORC initializes its private `writer` field at the first write. OrcFileFormat.addSparkVersionMetadata(recordWriter) recordWriter.close(Reporter.NULL) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala deleted file mode 100644 index f9c514567c639..0000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala +++ /dev/null @@ -1,262 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.orc - -import java.lang.reflect.Method - -import org.apache.hadoop.hive.ql.io.sarg.SearchArgument -import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.Builder -import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory.newBuilder - -import org.apache.spark.SparkException -import org.apache.spark.internal.Logging -import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.quoteIfNeeded -import org.apache.spark.sql.execution.datasources.orc.{OrcFilters => DatasourceOrcFilters} -import org.apache.spark.sql.execution.datasources.orc.OrcFilters.buildTree -import org.apache.spark.sql.hive.HiveUtils -import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types._ - -/** - * Helper object for building ORC `SearchArgument`s, which are used for ORC predicate push-down. - * - * Due to limitation of ORC `SearchArgument` builder, we had to end up with a pretty weird double- - * checking pattern when converting `And`/`Or`/`Not` filters. - * - * An ORC `SearchArgument` must be built in one pass using a single builder. For example, you can't - * build `a = 1` and `b = 2` first, and then combine them into `a = 1 AND b = 2`. This is quite - * different from the cases in Spark SQL or Parquet, where complex filters can be easily built using - * existing simpler ones. - * - * The annoying part is that, `SearchArgument` builder methods like `startAnd()`, `startOr()`, and - * `startNot()` mutate internal state of the builder instance. This forces us to translate all - * convertible filters with a single builder instance. However, before actually converting a filter, - * we've no idea whether it can be recognized by ORC or not. Thus, when an inconvertible filter is - * found, we may already end up with a builder whose internal state is inconsistent. - * - * For example, to convert an `And` filter with builder `b`, we call `b.startAnd()` first, and then - * try to convert its children. Say we convert `left` child successfully, but find that `right` - * child is inconvertible. Alas, `b.startAnd()` call can't be rolled back, and `b` is inconsistent - * now. - * - * The workaround employed here is that, for `And`/`Or`/`Not`, we first try to convert their - * children with brand new builders, and only do the actual conversion with the right builder - * instance when the children are proven to be convertible. - * - * P.S.: Hive seems to use `SearchArgument` together with `ExprNodeGenericFuncDesc` only. Usage of - * builder methods mentioned above can only be found in test code, where all tested filters are - * known to be convertible. - */ -private[orc] object OrcFilters extends Logging { - - private def findMethod(klass: Class[_], name: String, args: Class[_]*): Method = { - val method = klass.getMethod(name, args: _*) - method.setAccessible(true) - method - } - - def createFilter(schema: StructType, filters: Array[Filter]): Option[SearchArgument] = { - if (HiveUtils.isHive23) { - DatasourceOrcFilters.createFilter(schema, filters).asInstanceOf[Option[SearchArgument]] - } else { - val dataTypeMap = schema.map(f => quoteIfNeeded(f.name) -> f.dataType).toMap - // TODO (SPARK-25557): ORC doesn't support nested predicate pushdown, so they are removed. - val newFilters = filters.filter(!_.containsNestedColumn) - // Combines all convertible filters using `And` to produce a single conjunction - val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, newFilters)) - conjunctionOptional.map { conjunction => - // Then tries to build a single ORC `SearchArgument` for the conjunction predicate. - // The input predicate is fully convertible. There should not be any empty result in the - // following recursive method call `buildSearchArgument`. - buildSearchArgument(dataTypeMap, conjunction, newBuilder).build() - } - } - } - - def convertibleFilters( - schema: StructType, - dataTypeMap: Map[String, DataType], - filters: Seq[Filter]): Seq[Filter] = { - import org.apache.spark.sql.sources._ - - def convertibleFiltersHelper( - filter: Filter, - canPartialPushDown: Boolean): Option[Filter] = filter match { - // At here, it is not safe to just convert one side and remove the other side - // if we do not understand what the parent filters are. - // - // Here is an example used to explain the reason. - // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to - // convert b in ('1'). If we only convert a = 2, we will end up with a filter - // NOT(a = 2), which will generate wrong results. - // - // Pushing one side of AND down is only safe to do at the top level or in the child - // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate - // can be safely removed. - case And(left, right) => - val leftResultOptional = convertibleFiltersHelper(left, canPartialPushDown) - val rightResultOptional = convertibleFiltersHelper(right, canPartialPushDown) - (leftResultOptional, rightResultOptional) match { - case (Some(leftResult), Some(rightResult)) => Some(And(leftResult, rightResult)) - case (Some(leftResult), None) if canPartialPushDown => Some(leftResult) - case (None, Some(rightResult)) if canPartialPushDown => Some(rightResult) - case _ => None - } - - // The Or predicate is convertible when both of its children can be pushed down. - // That is to say, if one/both of the children can be partially pushed down, the Or - // predicate can be partially pushed down as well. - // - // Here is an example used to explain the reason. - // Let's say we have - // (a1 AND a2) OR (b1 AND b2), - // a1 and b1 is convertible, while a2 and b2 is not. - // The predicate can be converted as - // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) - // As per the logical in And predicate, we can push down (a1 OR b1). - case Or(left, right) => - for { - lhs <- convertibleFiltersHelper(left, canPartialPushDown) - rhs <- convertibleFiltersHelper(right, canPartialPushDown) - } yield Or(lhs, rhs) - case Not(pred) => - val childResultOptional = convertibleFiltersHelper(pred, canPartialPushDown = false) - childResultOptional.map(Not) - case other => - for (_ <- buildLeafSearchArgument(dataTypeMap, other, newBuilder())) yield other - } - filters.flatMap { filter => - convertibleFiltersHelper(filter, true) - } - } - - /** - * Build a SearchArgument and return the builder so far. - * - * @param dataTypeMap a map from the attribute name to its data type. - * @param expression the input predicates, which should be fully convertible to SearchArgument. - * @param builder the input SearchArgument.Builder. - * @return the builder so far. - */ - private def buildSearchArgument( - dataTypeMap: Map[String, DataType], - expression: Filter, - builder: Builder): Builder = { - expression match { - case And(left, right) => - val lhs = buildSearchArgument(dataTypeMap, left, builder.startAnd()) - val rhs = buildSearchArgument(dataTypeMap, right, lhs) - rhs.end() - - case Or(left, right) => - val lhs = buildSearchArgument(dataTypeMap, left, builder.startOr()) - val rhs = buildSearchArgument(dataTypeMap, right, lhs) - rhs.end() - - case Not(child) => - buildSearchArgument(dataTypeMap, child, builder.startNot()).end() - - case other => - buildLeafSearchArgument(dataTypeMap, other, builder).getOrElse { - throw new SparkException( - "The input filter of OrcFilters.buildSearchArgument should be fully convertible.") - } - } - } - - /** - * Build a SearchArgument for a leaf predicate and return the builder so far. - * - * @param dataTypeMap a map from the attribute name to its data type. - * @param expression the input filter predicates. - * @param builder the input SearchArgument.Builder. - * @return the builder so far. - */ - private def buildLeafSearchArgument( - dataTypeMap: Map[String, DataType], - expression: Filter, - builder: Builder): Option[Builder] = { - def isSearchableType(dataType: DataType): Boolean = dataType match { - // Only the values in the Spark types below can be recognized by - // the `SearchArgumentImpl.BuilderImpl.boxLiteral()` method. - case ByteType | ShortType | FloatType | DoubleType => true - case IntegerType | LongType | StringType | BooleanType => true - case TimestampType | _: DecimalType => true - case _ => false - } - - import org.apache.spark.sql.sources._ - - // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` - // call is mandatory. ORC `SearchArgument` builder requires that all leaf predicates must be - // wrapped by a "parent" predicate (`And`, `Or`, or `Not`). - expression match { - // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` - // call is mandatory. ORC `SearchArgument` builder requires that all leaf predicates must be - // wrapped by a "parent" predicate (`And`, `Or`, or `Not`). - - case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startAnd() - val method = findMethod(bd.getClass, "equals", classOf[String], classOf[Object]) - Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) - - case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startAnd() - val method = findMethod(bd.getClass, "nullSafeEquals", classOf[String], classOf[Object]) - Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) - - case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startAnd() - val method = findMethod(bd.getClass, "lessThan", classOf[String], classOf[Object]) - Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) - - case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startAnd() - val method = findMethod(bd.getClass, "lessThanEquals", classOf[String], classOf[Object]) - Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) - - case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startNot() - val method = findMethod(bd.getClass, "lessThanEquals", classOf[String], classOf[Object]) - Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) - - case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startNot() - val method = findMethod(bd.getClass, "lessThan", classOf[String], classOf[Object]) - Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) - - case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startAnd() - val method = findMethod(bd.getClass, "isNull", classOf[String]) - Some(method.invoke(bd, attribute).asInstanceOf[Builder].end()) - - case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startNot() - val method = findMethod(bd.getClass, "isNull", classOf[String]) - Some(method.invoke(bd, attribute).asInstanceOf[Builder].end()) - - case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startAnd() - val method = findMethod(bd.getClass, "in", classOf[String], classOf[Array[Object]]) - Some(method.invoke(bd, attribute, values.map(_.asInstanceOf[AnyRef])) - .asInstanceOf[Builder].end()) - - case _ => None - } - } -} diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar new file mode 100644 index 0000000000000..0d10f7ff03b35 Binary files /dev/null and b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar differ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/InsertIntoHiveTableBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/InsertIntoHiveTableBenchmark.scala index 81eb5e2591f13..da34c54cb36a2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/InsertIntoHiveTableBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/InsertIntoHiveTableBenchmark.scala @@ -28,14 +28,11 @@ import org.apache.spark.sql.hive.test.TestHive * {{{ * 1. without sbt: bin/spark-submit --class * --jars ,, - * --packages org.spark-project.hive:hive-exec:1.2.1.spark2 * - * 2. build/sbt "hive/test:runMain " -Phive-1.2 or - * build/sbt "hive/test:runMain " -Phive-2.3 + * 2. build/sbt "hive/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "hive/test:runMain " * Results will be written to "benchmarks/InsertIntoHiveTableBenchmark-hive2.3-results.txt". - * 4. -Phive-1.2 does not work for JDK 11 * }}} */ object InsertIntoHiveTableBenchmark extends SqlBasedBenchmark { @@ -136,5 +133,5 @@ object InsertIntoHiveTableBenchmark extends SqlBasedBenchmark { } } - override def suffix: String = if (HiveUtils.isHive23) "-hive2.3" else "-hive1.2" + override def suffix: String = "-hive2.3" } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala index a696d6aaff27b..c136c4c9790fd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala @@ -57,20 +57,12 @@ class ClasspathDependenciesSuite extends SparkFunSuite { } } - test("shaded Protobuf") { - if (HiveUtils.isHive23) { - assertLoads("com.google.protobuf.ServiceException") - } else { - assertLoads("org.apache.hive.com.google.protobuf.ServiceException") - } + test("protobuf") { + assertLoads("com.google.protobuf.ServiceException") } - test("shaded Kryo") { - if (HiveUtils.isHive23) { - assertLoads("com.esotericsoftware.kryo.Kryo") - } else { - assertLoads("org.apache.hive.com.esotericsoftware.kryo.Kryo") - } + test("kryo") { + assertLoads("com.esotericsoftware.kryo.Kryo") } test("hive-common") { @@ -89,12 +81,7 @@ class ClasspathDependenciesSuite extends SparkFunSuite { } test("parquet-hadoop-bundle") { - if (HiveUtils.isHive23) { - assertLoads("org.apache.parquet.hadoop.ParquetOutputFormat") - assertLoads("org.apache.parquet.hadoop.ParquetInputFormat") - } else { - assertLoads("parquet.hadoop.ParquetOutputFormat") - assertLoads("parquet.hadoop.ParquetInputFormat") - } + assertLoads("org.apache.parquet.hadoop.ParquetOutputFormat") + assertLoads("org.apache.parquet.hadoop.ParquetInputFormat") } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala index aa96fa035c4f0..b81b7e8ec0c0f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala @@ -234,7 +234,7 @@ object PROCESS_TABLES extends QueryTest with SQLTestUtils { // Tests the latest version of every release line. val testingVersions: Seq[String] = { import scala.io.Source - try { + val versions: Seq[String] = try { Source.fromURL(s"${releaseMirror}/spark").mkString .split("\n") .filter(_.contains("""
  • Seq("2.3.4", "2.4.5") // A temporary fallback to use a specific version + case NonFatal(_) => Seq("3.0.1", "2.4.7") // A temporary fallback to use a specific version } + versions.filter(v => v.startsWith("3") || !TestUtils.isPythonVersionAtLeast38()) } protected var spark: SparkSession = _ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index 95e99c653d6f6..8f71ba3337aa2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -206,13 +206,8 @@ class DataSourceWithHiveMetastoreCatalogSuite assert(columns.map(_.dataType) === Seq(DecimalType(10, 3), StringType)) checkAnswer(table("t"), testDF) - if (HiveUtils.isHive23) { - assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === - Seq("1.100\t1", "2.100\t2")) - } else { - assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === - Seq("1.1\t1", "2.1\t2")) - } + assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === + Seq("1.100\t1", "2.100\t2")) } } @@ -244,13 +239,8 @@ class DataSourceWithHiveMetastoreCatalogSuite assert(columns.map(_.dataType) === Seq(DecimalType(10, 3), StringType)) checkAnswer(table("t"), testDF) - if (HiveUtils.isHive23) { - assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === - Seq("1.100\t1", "2.100\t2")) - } else { - assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === - Seq("1.1\t1", "2.1\t2")) - } + assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === + Seq("1.100\t1", "2.100\t2")) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala index 590ef949ffbd7..ce82756428849 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala @@ -129,7 +129,7 @@ class HiveSchemaInferenceSuite // properties out). assert(!externalCatalog.getTable(DATABASE, TEST_TABLE_NAME).schemaPreservesCase) val rawTable = client.getTable(DATABASE, TEST_TABLE_NAME) - assert(rawTable.properties.filterKeys(_.startsWith(DATASOURCE_SCHEMA_PREFIX)) == Map.empty) + assert(rawTable.properties.filterKeys(_.startsWith(DATASOURCE_SCHEMA_PREFIX)).isEmpty) schema } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShimSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShimSuite.scala index 14d07cdf8db08..54c64a4eeb190 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShimSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShimSuite.scala @@ -35,18 +35,10 @@ class HiveShimSuite extends SparkFunSuite { // test when READ_COLUMN_NAMES_CONF_STR is empty HiveShim.appendReadColumns(conf, ids, names) - if (HiveUtils.isHive23) { - assert(names === ColumnProjectionUtils.getReadColumnNames(conf)) - } else { - assert(names.asJava === ColumnProjectionUtils.getReadColumnNames(conf)) - } + assert(names === ColumnProjectionUtils.getReadColumnNames(conf)) // test when READ_COLUMN_NAMES_CONF_STR is non-empty HiveShim.appendReadColumns(conf, moreIds, moreNames) - if (HiveUtils.isHive23) { - assert((names ++ moreNames) === ColumnProjectionUtils.getReadColumnNames(conf)) - } else { - assert((names ++ moreNames).asJava === ColumnProjectionUtils.getReadColumnNames(conf)) - } + assert((names ++ moreNames) === ColumnProjectionUtils.getReadColumnNames(conf)) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 6feaaea3dfb89..501a877e8b7fb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -153,7 +153,7 @@ class HiveSparkSubmitSuite // For more detail, see sql/hive/src/test/resources/regression-test-SPARK-8489/*scala. // TODO: revisit for Scala 2.13 support val version = Properties.versionNumberString match { - case v if v.startsWith("2.12") => v.substring(0, 4) + case v if v.startsWith("2.12") || v.startsWith("2.13") => v.substring(0, 4) case x => throw new Exception(s"Unsupported Scala Version: $x") } val jarDir = getTestResourcePath("regression-test-SPARK-8489") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 421dcb499bd6a..ebc6cfb77d355 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -847,4 +847,26 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } } } + + test("SPARK-32508 " + + "Disallow empty part col values in partition spec before static partition writing") { + withTable("t1") { + spark.sql( + """ + |CREATE TABLE t1 (c1 int) + |PARTITIONED BY (d string) + """.stripMargin) + + val e = intercept[AnalysisException] { + spark.sql( + """ + |INSERT OVERWRITE TABLE t1 PARTITION(d='') + |SELECT 1 + """.stripMargin) + }.getMessage + + assert(!e.contains("get partition: Value for key d is null or empty")) + assert(e.contains("Partition spec is invalid")) + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index be6d023302293..52dd2b34a0e95 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -101,14 +101,9 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto .asInstanceOf[HiveTableRelation] val properties = relation.tableMeta.ignoredProperties - if (HiveUtils.isHive23) { - // Since HIVE-6727, Hive fixes table-level stats for external tables are incorrect. - assert(properties("totalSize").toLong == 6) - assert(properties.get("rawDataSize").isEmpty) - } else { - assert(properties("totalSize").toLong <= 0, "external table totalSize must be <= 0") - assert(properties("rawDataSize").toLong <= 0, "external table rawDataSize must be <= 0") - } + // Since HIVE-6727, Hive fixes table-level stats for external tables are incorrect. + assert(properties("totalSize").toLong == 6) + assert(properties.get("rawDataSize").isEmpty) val sizeInBytes = relation.stats.sizeInBytes assert(sizeInBytes === BigInt(file1.length() + file2.length())) @@ -872,25 +867,10 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto assert(totalSize.isDefined && totalSize.get > 0, "totalSize is lost") val numRows = extractStatsPropValues(describeResult, "numRows") - if (HiveUtils.isHive23) { - // Since HIVE-15653(Hive 2.3.0), Hive fixs some ALTER TABLE commands drop table stats. - assert(numRows.isDefined && numRows.get == 500) - val rawDataSize = extractStatsPropValues(describeResult, "rawDataSize") - assert(rawDataSize.isDefined && rawDataSize.get == 5312) - checkTableStats(tabName, hasSizeInBytes = true, expectedRowCounts = Some(500)) - } else { - // ALTER TABLE SET/UNSET TBLPROPERTIES invalidates some Hive specific statistics, but not - // Spark specific statistics. This is triggered by the Hive alterTable API. - assert(numRows.isDefined && numRows.get == -1, "numRows is lost") - val rawDataSize = extractStatsPropValues(describeResult, "rawDataSize") - assert(rawDataSize.isDefined && rawDataSize.get == -1, "rawDataSize is lost") - - if (analyzedBySpark) { - checkTableStats(tabName, hasSizeInBytes = true, expectedRowCounts = Some(500)) - } else { - checkTableStats(tabName, hasSizeInBytes = true, expectedRowCounts = None) - } - } + assert(numRows.isDefined && numRows.get == 500) + val rawDataSize = extractStatsPropValues(describeResult, "rawDataSize") + assert(rawDataSize.isDefined && rawDataSize.get == 5312) + checkTableStats(tabName, hasSizeInBytes = true, expectedRowCounts = Some(500)) } } } @@ -1128,7 +1108,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto def checkColStatsProps(expected: Map[String, String]): Unit = { sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS " + stats.keys.mkString(", ")) val table = hiveClient.getTable("default", tableName) - val props = table.properties.filterKeys(_.startsWith("spark.sql.statistics.colStats")) + val props = table.properties.filterKeys(_.startsWith("spark.sql.statistics.colStats")).toMap assert(props == expected) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index 87771eed17b1b..70dcfb05c2ba9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -825,7 +825,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te """ |SELECT corr(b, c) FROM covar_tab WHERE a = 3 """.stripMargin), - Row(Double.NaN) :: Nil) + Row(null) :: Nil) checkAnswer( spark.sql( @@ -834,10 +834,10 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te """.stripMargin), Row(1, null) :: Row(2, null) :: - Row(3, Double.NaN) :: - Row(4, Double.NaN) :: - Row(5, Double.NaN) :: - Row(6, Double.NaN) :: Nil) + Row(3, null) :: + Row(4, null) :: + Row(5, null) :: + Row(6, null) :: Nil) val corr7 = spark.sql("SELECT corr(b, c) FROM covar_tab").collect()(0).getDouble(0) assert(math.abs(corr7 - 0.6633880657639323) < 1e-12) @@ -869,7 +869,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te // one row test val df3 = Seq.tabulate(1)(x => (1 * x, x * x * x - 2)).toDF("a", "b") - checkAnswer(df3.groupBy().agg(covar_samp("a", "b")), Row(Double.NaN)) + checkAnswer(df3.groupBy().agg(covar_samp("a", "b")), Row(null)) checkAnswer(df3.groupBy().agg(covar_pop("a", "b")), Row(0.0)) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 7f198632a1cd6..01cf214574eeb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -375,7 +375,7 @@ abstract class HiveComparisonTest extends SparkFunSuite with BeforeAndAfterAll { (!hiveQuery.logical.isInstanceOf[DescribeFunction]) && (!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) && (!hiveQuery.logical.isInstanceOf[DescribeRelation]) && - (!hiveQuery.logical.isInstanceOf[DescribeColumnStatement]) && + (!hiveQuery.logical.isInstanceOf[DescribeColumn]) && preparedHive != catalyst) { val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index fbd1fc1ea98df..44c551cf4a4c1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -994,7 +994,7 @@ class HiveDDLSuite |""".stripMargin) val newPart = catalog.getPartition(TableIdentifier("boxes"), Map("width" -> "4")) assert(newPart.storage.serde == Some(expectedSerde)) - assert(newPart.storage.properties.filterKeys(expectedSerdeProps.contains) == + assert(newPart.storage.properties.filterKeys(expectedSerdeProps.contains).toMap == expectedSerdeProps) } @@ -1192,7 +1192,7 @@ class HiveDDLSuite expectedDBUri, Map.empty)) // the database directory was created - assert(fs.exists(dbPath) && fs.isDirectory(dbPath)) + assert(fs.exists(dbPath) && fs.getFileStatus(dbPath).isDirectory) sql(s"USE $dbName") val tabName = "tab1" diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala index bb87246acf4ca..d247f37130776 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala @@ -38,8 +38,6 @@ class HiveScriptTransformationSuite extends BaseScriptTransformationSuite with T import ScriptTransformationIOSchema._ - override def isHive23OrSpark: Boolean = HiveUtils.isHive23 - override def createScriptTransformationExec( input: Seq[Expression], script: String, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index 67d7ed0841abb..bdccfccbc5bdb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.Row +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.{TestHive, TestHiveSingleton} import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ @@ -187,6 +189,65 @@ class HiveTableScanSuite extends HiveComparisonTest with SQLTestUtils with TestH } } + test("SPARK-32867: When explain, HiveTableRelation show limited message") { + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + withTable("df") { + spark.range(30) + .select(col("id"), col("id").as("k")) + .write + .partitionBy("k") + .format("hive") + .mode("overwrite") + .saveAsTable("df") + + val scan1 = getHiveTableScanExec("SELECT * FROM df WHERE df.k < 3") + assert(scan1.simpleString(100).replaceAll("#\\d+L", "") == + "Scan hive default.df [id, k]," + + " HiveTableRelation [" + + "`default`.`df`," + + " org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe," + + " Data Cols: [id]," + + " Partition Cols: [k]," + + " Pruned Partitions: [(k=0), (k=1), (k=2)]" + + "]," + + " [isnotnull(k), (k < 3)]") + + val scan2 = getHiveTableScanExec("SELECT * FROM df WHERE df.k < 30") + assert(scan2.simpleString(100).replaceAll("#\\d+L", "") == + "Scan hive default.df [id, k]," + + " HiveTableRelation [" + + "`default`.`df`," + + " org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe," + + " Data Cols: [id]," + + " Partition Cols: [k]," + + " Pruned Partitions: [(k=0), (k=1), (k=10), (k=11), (k=12), (k=13), (k=14), (k=15)," + + " (k=16), (k=17), (k=18), (k=19), (k..." + + "]," + + " [isnotnull(k), (k < 30)]") + + sql( + """ + |ALTER TABLE df PARTITION (k=10) SET SERDE + |'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe'; + """.stripMargin) + + val scan3 = getHiveTableScanExec("SELECT * FROM df WHERE df.k < 30") + assert(scan3.simpleString(100).replaceAll("#\\d+L", "") == + "Scan hive default.df [id, k]," + + " HiveTableRelation [" + + "`default`.`df`," + + " org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe," + + " Data Cols: [id]," + + " Partition Cols: [k]," + + " Pruned Partitions: [(k=0), (k=1)," + + " (k=10, org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe)," + + " (k=11), (k=12), (k=1..." + + "]," + + " [isnotnull(k), (k < 30)]") + } + } + } + private def getHiveTableScanExec(query: String): HiveTableScanExec = { sql(query).queryExecution.sparkPlan.collectFirst { case p: HiveTableScanExec => p diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala index 9e33a8ee4cc5c..ed44dcd8d7a29 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala @@ -161,6 +161,20 @@ class HiveUDAFSuite extends QueryTest checkAnswer(sql("select histogram_numeric(a,2) from abc where a=3"), Row(null)) } } + + test("SPARK-32243: Spark UDAF Invalid arguments number error should throw earlier") { + // func need two arguments + val functionName = "longProductSum" + val functionClass = "org.apache.spark.sql.hive.execution.LongProductSum" + withUserDefinedFunction(functionName -> true) { + sql(s"CREATE TEMPORARY FUNCTION $functionName AS '$functionClass'") + val e = intercept[AnalysisException] { + sql(s"SELECT $functionName(100)") + }.getMessage + assert(e.contains( + s"Invalid number of arguments for function $functionName. Expected: 2; Found: 1;")) + } + } } /** diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala index 057f2f4ce01be..dd797b39e0939 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.command.FunctionsCommand import org.apache.spark.sql.functions.max +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -658,6 +659,24 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } } + test("SPARK-32877: add test for Hive UDF complex decimal type") { + withUserDefinedFunction("testArraySum" -> false) { + sql(s"CREATE FUNCTION testArraySum AS '${classOf[ArraySumUDF].getName}'") + checkAnswer( + sql("SELECT testArraySum(array(1, 1.1, 1.2))"), + Seq(Row(3.3))) + + val msg = intercept[AnalysisException] { + sql("SELECT testArraySum(1)") + }.getMessage + assert(msg.contains(s"No handler for UDF/UDAF/UDTF '${classOf[ArraySumUDF].getName}'")) + + val msg2 = intercept[AnalysisException] { + sql("SELECT testArraySum(1, 2)") + }.getMessage + assert(msg2.contains(s"No handler for UDF/UDAF/UDTF '${classOf[ArraySumUDF].getName}'")) + } + } } class TestPair(x: Int, y: Int) extends Writable with Serializable { @@ -741,3 +760,14 @@ class StatelessUDF extends UDF { result } } + +class ArraySumUDF extends UDF { + import scala.collection.JavaConverters._ + def evaluate(values: java.util.List[java.lang.Double]): java.lang.Double = { + var r = 0d + for (v <- values.asScala) { + r += v + } + r + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 431790e1fbb6d..a69a949e3a3a2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -2206,39 +2206,63 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi } } - test("SPARK-21912 ORC/Parquet table should not create invalid column names") { + test("SPARK-21912 Parquet table should not create invalid column names") { Seq(" ", ",", ";", "{", "}", "(", ")", "\n", "\t", "=").foreach { name => - Seq("ORC", "PARQUET").foreach { source => - withTable("t21912") { - val m = intercept[AnalysisException] { - sql(s"CREATE TABLE t21912(`col$name` INT) USING $source") - }.getMessage - assert(m.contains(s"contains invalid character(s)")) + val source = "PARQUET" + withTable("t21912") { + val m = intercept[AnalysisException] { + sql(s"CREATE TABLE t21912(`col$name` INT) USING $source") + }.getMessage + assert(m.contains(s"contains invalid character(s)")) - val m1 = intercept[AnalysisException] { - sql(s"CREATE TABLE t21912 STORED AS $source AS SELECT 1 `col$name`") - }.getMessage - assert(m1.contains(s"contains invalid character(s)")) + val m1 = intercept[AnalysisException] { + sql(s"CREATE TABLE t21912 STORED AS $source AS SELECT 1 `col$name`") + }.getMessage + assert(m1.contains(s"contains invalid character(s)")) + + val m2 = intercept[AnalysisException] { + sql(s"CREATE TABLE t21912 USING $source AS SELECT 1 `col$name`") + }.getMessage + assert(m2.contains(s"contains invalid character(s)")) - val m2 = intercept[AnalysisException] { - sql(s"CREATE TABLE t21912 USING $source AS SELECT 1 `col$name`") + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false") { + val m3 = intercept[AnalysisException] { + sql(s"CREATE TABLE t21912(`col$name` INT) USING hive OPTIONS (fileFormat '$source')") }.getMessage - assert(m2.contains(s"contains invalid character(s)")) + assert(m3.contains(s"contains invalid character(s)")) + } - withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false") { - val m3 = intercept[AnalysisException] { - sql(s"CREATE TABLE t21912(`col$name` INT) USING hive OPTIONS (fileFormat '$source')") - }.getMessage - assert(m3.contains(s"contains invalid character(s)")) - } + sql(s"CREATE TABLE t21912(`col` INT) USING $source") + val m4 = intercept[AnalysisException] { + sql(s"ALTER TABLE t21912 ADD COLUMNS(`col$name` INT)") + }.getMessage + assert(m4.contains(s"contains invalid character(s)")) + } + } + } - sql(s"CREATE TABLE t21912(`col` INT) USING $source") - val m4 = intercept[AnalysisException] { - sql(s"ALTER TABLE t21912 ADD COLUMNS(`col$name` INT)") - }.getMessage - assert(m4.contains(s"contains invalid character(s)")) + test("SPARK-32889: ORC table column name supports special characters") { + // " " "," is not allowed. + Seq("$", ";", "{", "}", "(", ")", "\n", "\t", "=").foreach { name => + val source = "ORC" + Seq(s"CREATE TABLE t32889(`$name` INT) USING $source", + s"CREATE TABLE t32889 STORED AS $source AS SELECT 1 `$name`", + s"CREATE TABLE t32889 USING $source AS SELECT 1 `$name`", + s"CREATE TABLE t32889(`$name` INT) USING hive OPTIONS (fileFormat '$source')") + .foreach { command => + withTable("t32889") { + sql(command) + assertResult(name)( + sessionState.catalog.getTableMetadata(TableIdentifier("t32889")).schema.fields(0).name) } } + + withTable("t32889") { + sql(s"CREATE TABLE t32889(`col` INT) USING $source") + sql(s"ALTER TABLE t32889 ADD COLUMNS(`$name` INT)") + assertResult(name)( + sessionState.catalog.getTableMetadata(TableIdentifier("t32889")).schema.fields(1).name) + } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala index 15712a18ce751..6bf7bd6cbb90e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala @@ -62,7 +62,6 @@ class WindowQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleto // Moved because: // - Spark uses a different default stddev (sample instead of pop) // - Tiny numerical differences in stddev results. - // - Different StdDev behavior when n=1 (NaN instead of 0) checkAnswer(sql(s""" |select p_mfgr,p_name, p_size, |rank() over(distribute by p_mfgr sort by p_name) as r, @@ -88,22 +87,22 @@ class WindowQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleto Row("Manufacturer#1", "almond antique salmon chartreuse burlywood", 6, 4, 3, 0.6666666666666666, 0.6, 2, 4, 11.0, 15.448840301675292, 2, 6, 2), Row("Manufacturer#1", "almond aquamarine burnished black steel", 28, 5, 4, 0.8333333333333334, 0.8, 3, 5, 14.4, 15.388307249337076, 2, 28, 34), Row("Manufacturer#1", "almond aquamarine pink moccasin thistle", 42, 6, 5, 1.0, 1.0, 3, 6, 19.0, 17.787636155487327, 2, 42, 6), - Row("Manufacturer#2", "almond antique violet chocolate turquoise", 14, 1, 1, 0.2, 0.0, 1, 1, 14.0, Double.NaN, 4, 14, 14), + Row("Manufacturer#2", "almond antique violet chocolate turquoise", 14, 1, 1, 0.2, 0.0, 1, 1, 14.0, null, 4, 14, 14), Row("Manufacturer#2", "almond antique violet turquoise frosted", 40, 2, 2, 0.4, 0.25, 1, 2, 27.0, 18.384776310850235, 4, 40, 14), Row("Manufacturer#2", "almond aquamarine midnight light salmon", 2, 3, 3, 0.6, 0.5, 2, 3, 18.666666666666668, 19.42506971244462, 4, 2, 14), Row("Manufacturer#2", "almond aquamarine rose maroon antique", 25, 4, 4, 0.8, 0.75, 2, 4, 20.25, 16.17353805861084, 4, 25, 40), Row("Manufacturer#2", "almond aquamarine sandy cyan gainsboro", 18, 5, 5, 1.0, 1.0, 3, 5, 19.8, 14.042791745233567, 4, 18, 2), - Row("Manufacturer#3", "almond antique chartreuse khaki white", 17, 1, 1, 0.2, 0.0, 1, 1, 17.0,Double.NaN, 2, 17, 17), + Row("Manufacturer#3", "almond antique chartreuse khaki white", 17, 1, 1, 0.2, 0.0, 1, 1, 17.0, null, 2, 17, 17), Row("Manufacturer#3", "almond antique forest lavender goldenrod", 14, 2, 2, 0.4, 0.25, 1, 2, 15.5, 2.1213203435596424, 2, 14, 17), Row("Manufacturer#3", "almond antique metallic orange dim", 19, 3, 3, 0.6, 0.5, 2, 3, 16.666666666666668, 2.516611478423583, 2, 19, 17), Row("Manufacturer#3", "almond antique misty red olive", 1, 4, 4, 0.8, 0.75, 2, 4, 12.75, 8.098353742170895, 2, 1, 14), Row("Manufacturer#3", "almond antique olive coral navajo", 45, 5, 5, 1.0, 1.0, 3, 5, 19.2, 16.037456157383566, 2, 45, 19), - Row("Manufacturer#4", "almond antique gainsboro frosted violet", 10, 1, 1, 0.2, 0.0, 1, 1, 10.0, Double.NaN, 0, 10, 10), + Row("Manufacturer#4", "almond antique gainsboro frosted violet", 10, 1, 1, 0.2, 0.0, 1, 1, 10.0, null, 0, 10, 10), Row("Manufacturer#4", "almond antique violet mint lemon", 39, 2, 2, 0.4, 0.25, 1, 2, 24.5, 20.506096654409877, 0, 39, 10), Row("Manufacturer#4", "almond aquamarine floral ivory bisque", 27, 3, 3, 0.6, 0.5, 2, 3, 25.333333333333332, 14.571661996262929, 0, 27, 10), Row("Manufacturer#4", "almond aquamarine yellow dodger mint", 7, 4, 4, 0.8, 0.75, 2, 4, 20.75, 15.01943185787443, 0, 7, 39), Row("Manufacturer#4", "almond azure aquamarine papaya violet", 12, 5, 5, 1.0, 1.0, 3, 5, 19.0, 13.583077707206124, 0, 12, 27), - Row("Manufacturer#5", "almond antique blue firebrick mint", 31, 1, 1, 0.2, 0.0, 1, 1, 31.0, Double.NaN, 1, 31, 31), + Row("Manufacturer#5", "almond antique blue firebrick mint", 31, 1, 1, 0.2, 0.0, 1, 1, 31.0, null, 1, 31, 31), Row("Manufacturer#5", "almond antique medium spring khaki", 6, 2, 2, 0.4, 0.25, 1, 2, 18.5, 17.67766952966369, 1, 6, 31), Row("Manufacturer#5", "almond antique sky peru orange", 2, 3, 3, 0.6, 0.5, 2, 3, 13.0, 15.716233645501712, 1, 2, 31), Row("Manufacturer#5", "almond aquamarine dodger light gainsboro", 46, 4, 4, 0.8, 0.75, 2, 4, 21.25, 20.902551678363736, 1, 46, 6), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala deleted file mode 100644 index 5fc41067f661d..0000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala +++ /dev/null @@ -1,484 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.orc - -import java.nio.charset.StandardCharsets -import java.sql.{Date, Timestamp} - -import scala.collection.JavaConverters._ - -import org.apache.hadoop.hive.ql.io.sarg.{PredicateLeaf, SearchArgument} - -import org.apache.spark.sql.{Column, DataFrame} -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} -import org.apache.spark.sql.execution.datasources.orc.OrcTest -import org.apache.spark.sql.hive.HiveUtils -import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.types._ - -/** - * A test suite that tests Hive ORC filter API based filter pushdown optimization. - */ -class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { - - override val orcImp: String = "hive" - - private def checkFilterPredicate( - df: DataFrame, - predicate: Predicate, - checker: (SearchArgument) => Unit): Unit = { - val output = predicate.collect { case a: Attribute => a }.distinct - val query = df - .select(output.map(e => Column(e)): _*) - .where(Column(predicate)) - - var maybeRelation: Option[HadoopFsRelation] = None - val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => - maybeRelation = Some(orcRelation) - filters - }.flatten.reduceLeftOption(_ && _) - assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") - - val (_, selectedFilters, _) = - DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) - assert(selectedFilters.nonEmpty, "No filter is pushed down") - - val maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters.toArray) - assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $selectedFilters") - checker(maybeFilter.get) - } - - private def checkFilterPredicate - (predicate: Predicate, filterOperator: PredicateLeaf.Operator) - (implicit df: DataFrame): Unit = { - def checkComparisonOperator(filter: SearchArgument) = { - val operator = filter.getLeaves.asScala - assert(operator.map(_.getOperator).contains(filterOperator)) - } - checkFilterPredicate(df, predicate, checkComparisonOperator) - } - - private def checkFilterPredicateWithDiffHiveVersion - (predicate: Predicate, stringExpr: String) - (implicit df: DataFrame): Unit = { - def checkLogicalOperator(filter: SearchArgument) = { - if (HiveUtils.isHive23) { - assert(filter.toString == stringExpr.replace("\n", ", ")) - } else { - assert(filter.toString == stringExpr) - } - } - checkFilterPredicate(df, predicate, checkLogicalOperator) - } - - private def assertResultWithDiffHiveVersion(expected : String)(c : scala.Any) = { - if (HiveUtils.isHive23) { - assertResult(expected.replace("\n", ", "))(c) - } else { - assertResult(expected)(c) - } - } - - private def checkNoFilterPredicate - (predicate: Predicate) - (implicit df: DataFrame): Unit = { - val output = predicate.collect { case a: Attribute => a }.distinct - val query = df - .select(output.map(e => Column(e)): _*) - .where(Column(predicate)) - - var maybeRelation: Option[HadoopFsRelation] = None - val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => - maybeRelation = Some(orcRelation) - filters - }.flatten.reduceLeftOption(_ && _) - assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") - - val (_, selectedFilters, _) = - DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) - assert(selectedFilters.nonEmpty, "No filter is pushed down") - - val maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters.toArray) - assert(maybeFilter.isEmpty, s"Could generate filter predicate for $selectedFilters") - } - - test("filter pushdown - integer") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - long") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - float") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - double") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - string") { - withOrcDataFrame((1 to 4).map(i => Tuple1(i.toString))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === "1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < "2", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= "4", PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal("1") === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal("1") <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal("2") > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal("3") < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("1") >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("4") <= $"_1", PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - boolean") { - withOrcDataFrame((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === true, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < true, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= false, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(false) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(false) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(false) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(true) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) <= $"_1", PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - decimal") { - withOrcDataFrame((1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> BigDecimal.valueOf(1), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > BigDecimal.valueOf(3), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= BigDecimal.valueOf(1), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(2)) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate( - Literal(BigDecimal.valueOf(3)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(4)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - timestamp") { - val timeString = "2015-08-20 14:57:00" - val timestamps = (1 to 4).map { i => - val milliseconds = Timestamp.valueOf(timeString).getTime + i * 3600 - new Timestamp(milliseconds) - } - withOrcDataFrame(timestamps.map(Tuple1(_))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === timestamps(0), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < timestamps(1), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(timestamps(0)) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(timestamps(0)) <=> $"_1", - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(timestamps(1)) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(timestamps(2)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(0)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(3)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - combinations with logical operators") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => - // Because `ExpressionTree` is not accessible at Hive 1.2.x, this should be checked - // in string form in order to check filter creation including logical operators - // such as `and`, `or` or `not`. So, this function uses `SearchArgument.toString()` - // to produce string expression and then compare it to given string expression below. - // This might have to be changed after Hive version is upgraded. - checkFilterPredicateWithDiffHiveVersion( - $"_1".isNotNull, - """leaf-0 = (IS_NULL _1) - |expr = (not leaf-0)""".stripMargin.trim - ) - checkFilterPredicateWithDiffHiveVersion( - $"_1" =!= 1, - """leaf-0 = (IS_NULL _1) - |leaf-1 = (EQUALS _1 1) - |expr = (and (not leaf-0) (not leaf-1))""".stripMargin.trim - ) - checkFilterPredicateWithDiffHiveVersion( - !($"_1" < 4), - """leaf-0 = (IS_NULL _1) - |leaf-1 = (LESS_THAN _1 4) - |expr = (and (not leaf-0) (not leaf-1))""".stripMargin.trim - ) - checkFilterPredicateWithDiffHiveVersion( - $"_1" < 2 || $"_1" > 3, - """leaf-0 = (LESS_THAN _1 2) - |leaf-1 = (LESS_THAN_EQUALS _1 3) - |expr = (or leaf-0 (not leaf-1))""".stripMargin.trim - ) - checkFilterPredicateWithDiffHiveVersion( - $"_1" < 2 && $"_1" > 3, - """leaf-0 = (IS_NULL _1) - |leaf-1 = (LESS_THAN _1 2) - |leaf-2 = (LESS_THAN_EQUALS _1 3) - |expr = (and (not leaf-0) leaf-1 (not leaf-2))""".stripMargin.trim - ) - } - } - - test("no filter pushdown - non-supported types") { - implicit class IntToBinary(int: Int) { - def b: Array[Byte] = int.toString.getBytes(StandardCharsets.UTF_8) - } - // ArrayType - withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => - checkNoFilterPredicate($"_1".isNull) - } - // BinaryType - withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => - checkNoFilterPredicate($"_1" <=> 1.b) - } - // DateType - if (!HiveUtils.isHive23) { - val stringDate = "2015-01-01" - withOrcDataFrame(Seq(Tuple1(Date.valueOf(stringDate)))) { implicit df => - checkNoFilterPredicate($"_1" === Date.valueOf(stringDate)) - } - } - // MapType - withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => - checkNoFilterPredicate($"_1".isNotNull) - } - } - - test("SPARK-12218 and SPARK-25699 Converting conjunctions into ORC SearchArguments") { - import org.apache.spark.sql.sources._ - // The `LessThan` should be converted while the `StringContains` shouldn't - val schema = new StructType( - Array( - StructField("a", IntegerType, nullable = true), - StructField("b", StringType, nullable = true))) - assertResultWithDiffHiveVersion( - """leaf-0 = (LESS_THAN a 10) - |expr = leaf-0 - """.stripMargin.trim - ) { - OrcFilters.createFilter(schema, Array( - LessThan("a", 10), - StringContains("b", "prefix") - )).get.toString - } - - // The `LessThan` should be converted while the whole inner `And` shouldn't - assertResultWithDiffHiveVersion( - """leaf-0 = (LESS_THAN a 10) - |expr = leaf-0 - """.stripMargin.trim - ) { - OrcFilters.createFilter(schema, Array( - LessThan("a", 10), - Not(And( - GreaterThan("a", 1), - StringContains("b", "prefix") - )) - )).get.toString - } - - // Safely remove unsupported `StringContains` predicate and push down `LessThan` - assertResultWithDiffHiveVersion( - """leaf-0 = (LESS_THAN a 10) - |expr = leaf-0 - """.stripMargin.trim - ) { - OrcFilters.createFilter(schema, Array( - And( - LessThan("a", 10), - StringContains("b", "prefix") - ) - )).get.toString - } - - // Safely remove unsupported `StringContains` predicate, push down `LessThan` and `GreaterThan`. - assertResultWithDiffHiveVersion( - """leaf-0 = (LESS_THAN a 10) - |leaf-1 = (LESS_THAN_EQUALS a 1) - |expr = (and leaf-0 (not leaf-1)) - """.stripMargin.trim - ) { - OrcFilters.createFilter(schema, Array( - And( - And( - LessThan("a", 10), - StringContains("b", "prefix") - ), - GreaterThan("a", 1) - ) - )).get.toString - } - } - - test("SPARK-27699 Converting disjunctions into ORC SearchArguments") { - import org.apache.spark.sql.sources._ - // The `LessThan` should be converted while the `StringContains` shouldn't - val schema = new StructType( - Array( - StructField("a", IntegerType, nullable = true), - StructField("b", StringType, nullable = true))) - - // The predicate `StringContains` predicate is not able to be pushed down. - assertResultWithDiffHiveVersion("leaf-0 = (LESS_THAN_EQUALS a 10)\nleaf-1 = (LESS_THAN a 1)\n" + - "expr = (or (not leaf-0) leaf-1)") { - OrcFilters.createFilter(schema, Array( - Or( - GreaterThan("a", 10), - And( - StringContains("b", "prefix"), - LessThan("a", 1) - ) - ) - )).get.toString - } - - assertResultWithDiffHiveVersion("leaf-0 = (LESS_THAN_EQUALS a 10)\nleaf-1 = (LESS_THAN a 1)\n" + - "expr = (or (not leaf-0) leaf-1)") { - OrcFilters.createFilter(schema, Array( - Or( - And( - GreaterThan("a", 10), - StringContains("b", "foobar") - ), - And( - StringContains("b", "prefix"), - LessThan("a", 1) - ) - ) - )).get.toString - } - - assert(OrcFilters.createFilter(schema, Array( - Or( - StringContains("b", "foobar"), - And( - StringContains("b", "prefix"), - LessThan("a", 1) - ) - ) - )).isEmpty) - } -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala index 12ee5bea7c2f9..fcf7febe33121 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala @@ -168,9 +168,6 @@ class HiveOrcQuerySuite extends OrcQueryTest with TestHiveSingleton { } } - // Since Hive 1.2.1 library code path still has this problem, users may hit this - // when spark.sql.hive.convertMetastoreOrc=false. However, after SPARK-22279, - // Apache Spark with the default configuration doesn't hit this bug. test("SPARK-22267 Spark SQL incorrectly reads ORC files when column order is different") { Seq("native", "hive").foreach { orcImpl => withSQLConf(SQLConf.ORC_IMPLEMENTATION.key -> orcImpl) { @@ -179,10 +176,12 @@ class HiveOrcQuerySuite extends OrcQueryTest with TestHiveSingleton { Seq(1 -> 2).toDF("c1", "c2").write.orc(path) checkAnswer(spark.read.orc(path), Row(1, 2)) - withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "true") { // default since 2.3.0 - withTable("t") { - sql(s"CREATE EXTERNAL TABLE t(c2 INT, c1 INT) STORED AS ORC LOCATION '$path'") - checkAnswer(spark.table("t"), Row(2, 1)) + Seq(true, false).foreach { convertMetastoreOrc => + withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> convertMetastoreOrc.toString) { + withTable("t") { + sql(s"CREATE EXTERNAL TABLE t(c2 INT, c1 INT) STORED AS ORC LOCATION '$path'") + checkAnswer(spark.table("t"), Row(2, 1)) + } } } } @@ -190,9 +189,6 @@ class HiveOrcQuerySuite extends OrcQueryTest with TestHiveSingleton { } } - // Since Hive 1.2.1 library code path still has this problem, users may hit this - // when spark.sql.hive.convertMetastoreOrc=false. However, after SPARK-22279, - // Apache Spark with the default configuration doesn't hit this bug. test("SPARK-19809 NullPointerException on zero-size ORC file") { Seq("native", "hive").foreach { orcImpl => withSQLConf(SQLConf.ORC_IMPLEMENTATION.key -> orcImpl) { @@ -201,8 +197,10 @@ class HiveOrcQuerySuite extends OrcQueryTest with TestHiveSingleton { sql(s"CREATE TABLE spark_19809(a int) STORED AS ORC LOCATION '$dir'") Files.touch(new File(s"${dir.getCanonicalPath}", "zero.orc")) - withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "true") { // default since 2.3.0 - checkAnswer(spark.table("spark_19809"), Seq.empty) + Seq(true, false).foreach { convertMetastoreOrc => + withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> convertMetastoreOrc.toString) { + checkAnswer(spark.table("spark_19809"), Seq.empty) + } } } } @@ -224,7 +222,6 @@ class HiveOrcQuerySuite extends OrcQueryTest with TestHiveSingleton { } test("SPARK-26437 Can not query decimal type when value is 0") { - assume(HiveUtils.isHive23, "bad test: This bug fixed by HIVE-13083(Hive 2.0.1)") withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "false") { withTable("spark_26437") { sql("CREATE TABLE spark_26437 STORED AS ORCFILE AS SELECT 0.00 AS c1") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala index 91fd8a47339fc..e94e0b39c859c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala @@ -149,12 +149,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { test("Check BloomFilter creation") { Seq(true, false).foreach { convertMetastore => withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> s"$convertMetastore") { - if (HiveUtils.isHive23) { - testBloomFilterCreation(org.apache.orc.OrcProto.Stream.Kind.BLOOM_FILTER_UTF8) - } else { - // Before ORC-101 - testBloomFilterCreation(org.apache.orc.OrcProto.Stream.Kind.BLOOM_FILTER) - } + testBloomFilterCreation(org.apache.orc.OrcProto.Stream.Kind.BLOOM_FILTER_UTF8) } } } @@ -162,7 +157,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { test("Enforce direct encoding column-wise selectively") { Seq(true, false).foreach { convertMetastore => withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> s"$convertMetastore") { - testSelectiveDictionaryEncoding(isSelective = false, isHive23 = HiveUtils.isHive23) + testSelectiveDictionaryEncoding(isSelective = false, isHiveOrc = true) } } } @@ -322,7 +317,6 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { } test("SPARK-31580: Read a file written before ORC-569") { - assume(HiveUtils.isHive23) // Hive 1.2 doesn't use Apache ORC // Test ORC file came from ORC-621 val df = readResourceOrcFile("test-data/TestStringDictionary.testRowIndex.orc") assert(df.where("str < 'row 001000'").count() === 1000) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala index a2518e70a013b..accfcb8d9deff 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -38,6 +38,7 @@ import org.apache.spark.internal.config.UI._ import org.apache.spark.sql.{DataFrame, Dataset, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -58,6 +59,7 @@ object TestHive new SparkConf() .set("spark.sql.test", "") .set(SQLConf.CODEGEN_FALLBACK.key, "false") + .set(SQLConf.CODEGEN_FACTORY_MODE.key, CodegenObjectFactoryMode.CODEGEN_ONLY.toString) .set(HiveUtils.HIVE_METASTORE_BARRIER_PREFIXES.key, "org.apache.spark.sql.hive.execution.PairSerDe") .set(WAREHOUSE_PATH.key, TestHiveContext.makeWarehouseDir().toURI.getPath) @@ -222,7 +224,7 @@ private[hive] class TestHiveSparkSession( @transient override lazy val sessionState: SessionState = { - new TestHiveSessionStateBuilder(this, parentSessionState).build() + new TestHiveSessionStateBuilder(this, parentSessionState, Map.empty).build() } lazy val metadataHive: HiveClient = { @@ -599,7 +601,7 @@ private[hive] class TestHiveQueryExecution( // Make sure any test tables referenced are loaded. val referencedTables = describedTables ++ - logical.collect { case UnresolvedRelation(ident, _) => ident.asTableIdentifier } + logical.collect { case UnresolvedRelation(ident, _, _) => ident.asTableIdentifier } val resolver = sparkSession.sessionState.conf.resolver val referencedTestTables = referencedTables.flatMap { tbl => val testTableOpt = sparkSession.testTables.keys.find(resolver(_, tbl.table)) @@ -648,8 +650,9 @@ private[hive] object TestHiveContext { private[sql] class TestHiveSessionStateBuilder( session: SparkSession, - state: Option[SessionState]) - extends HiveSessionStateBuilder(session, state) + state: Option[SessionState], + options: Map[String, String]) + extends HiveSessionStateBuilder(session, state, options) with WithTestConf { override def overrideConfs: Map[String, String] = TestHiveContext.overrideConfs @@ -658,7 +661,7 @@ private[sql] class TestHiveSessionStateBuilder( new TestHiveQueryExecution(session.asInstanceOf[TestHiveSparkSession], plan) } - override protected def newBuilder: NewBuilder = new TestHiveSessionStateBuilder(_, _) + override protected def newBuilder: NewBuilder = new TestHiveSessionStateBuilder(_, _, Map.empty) } private[hive] object HiveTestJars { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanWithHiveSupportSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanWithHiveSupportSuite.scala new file mode 100644 index 0000000000000..30eb93cb5c3e8 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanWithHiveSupportSuite.scala @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources + +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION + +class DisableUnnecessaryBucketedScanWithHiveSupportSuite + extends DisableUnnecessaryBucketedScanSuite + with TestHiveSingleton { + + protected override def beforeAll(): Unit = { + super.beforeAll() + assert(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "hive") + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index 37cc1b8a6d2ab..43aaa7e1eeaec 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -19,7 +19,7 @@ package org.apache.spark.streaming import java.io.{IOException, ObjectInputStream, ObjectOutputStream} -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable import scala.collection.parallel.immutable.ParVector import org.apache.spark.internal.Logging @@ -29,8 +29,8 @@ import org.apache.spark.util.Utils final private[streaming] class DStreamGraph extends Serializable with Logging { - private val inputStreams = new ArrayBuffer[InputDStream[_]]() - private val outputStreams = new ArrayBuffer[DStream[_]]() + private var inputStreams = mutable.ArraySeq.empty[InputDStream[_]] + private var outputStreams = mutable.ArraySeq.empty[DStream[_]] @volatile private var inputStreamNameAndID: Seq[(String, Int)] = Nil @@ -91,14 +91,14 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { def addInputStream(inputStream: InputDStream[_]): Unit = { this.synchronized { inputStream.setGraph(this) - inputStreams += inputStream + inputStreams = inputStreams :+ inputStream } } def addOutputStream(outputStream: DStream[_]): Unit = { this.synchronized { outputStream.setGraph(this) - outputStreams += outputStream + outputStreams = outputStreams :+ outputStream } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala index f677c492d561f..6494e512713f8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala @@ -96,7 +96,7 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( @transient private val hadoopConfig = sc.hadoopConfiguration private val broadcastedHadoopConf = new SerializableConfiguration(hadoopConfig) - override def isValid(): Boolean = true + override def isValid: Boolean = true override def getPartitions: Array[Partition] = { assertValid() diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index 12ed8015117e5..7a561ecb4990f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -234,7 +234,7 @@ private[streaming] class CountingIterator[T](iterator: Iterator[T]) extends Iter private def isFullyConsumed: Boolean = !iterator.hasNext - def hasNext(): Boolean = iterator.hasNext + def hasNext: Boolean = iterator.hasNext def count(): Option[Long] = { if (isFullyConsumed) Some(_count) else None diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala index 146577214de17..006bcad5d68c2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala @@ -58,7 +58,7 @@ private[streaming] object HdfsUtils { // If we are really unlucky, the file may be deleted as we're opening the stream. // This can happen as clean up is performed by daemon threads that may be left over from // previous runs. - if (!dfs.isFile(dfsPath)) null else throw e + if (!dfs.getFileStatus(dfsPath).isFile) null else throw e } } @@ -92,6 +92,10 @@ private[streaming] object HdfsUtils { def checkFileExists(path: String, conf: Configuration): Boolean = { val hdpPath = new Path(path) val fs = getFileSystemForPath(hdpPath, conf) - fs.isFile(hdpPath) + try { + fs.getFileStatus(hdpPath).isFile + } catch { + case _: FileNotFoundException => false + } } } diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala index f9bc499961ad7..a6fee8616df11 100644 --- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala +++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala @@ -24,6 +24,7 @@ import scala.reflect.runtime.universe.runtimeMirror import scala.util.Try import org.clapper.classutil.ClassFinder +import org.objectweb.asm.Opcodes /** * A tool for generating classes to be excluded during binary checking with MIMA. It is expected @@ -146,7 +147,7 @@ object GenerateMIMAIgnore { * and subpackages both from directories and jars present on the classpath. */ private def getClasses(packageName: String): Set[String] = { - val finder = ClassFinder() + val finder = ClassFinder(maybeOverrideAsmVersion = Some(Opcodes.ASM7)) finder .getClasses .map(_.name)