diff --git a/.gitignore b/.gitignore index d54d21b802be8..3624d12269612 100644 --- a/.gitignore +++ b/.gitignore @@ -65,6 +65,7 @@ scalastyle.txt scalastyle-output.xml R-unit-tests.log R/unit-tests.out +python/lib/pyspark.zip # For Hive metastore_db/ diff --git a/.rat-excludes b/.rat-excludes index ac652ed29fc09..c0f81b57fe09d 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -36,7 +36,6 @@ graphlib-dot.min.js sorttable.js vis.min.js vis.min.css -vis.map .*avsc .*txt .*json @@ -80,6 +79,6 @@ local-1422981780767/* local-1425081759269/* local-1426533911241/* local-1426633911242/* -local-1427397477963/* +local-1430917381534/* DESCRIPTION NAMESPACE diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 1c1779a763c7e..efc85bbc4b316 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -15,11 +15,11 @@ Suggests: Description: R frontend for Spark License: Apache License (== 2.0) Collate: + 'schema.R' 'generics.R' 'jobj.R' 'RDD.R' 'pairRDD.R' - 'schema.R' 'column.R' 'group.R' 'DataFrame.R' diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 1fb3311b7f955..7611f479a628b 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -13,6 +13,7 @@ exportMethods("cache", "collect", "columns", "count", + "describe", "distinct", "dtypes", "except", @@ -25,7 +26,6 @@ exportMethods("cache", "intersect", "isLocal", "join", - "length", "limit", "orderBy", "names", @@ -44,8 +44,6 @@ exportMethods("cache", "showDF", "sortDF", "take", - "toJSON", - "toRDD", "unionAll", "unpersist", "where", @@ -94,19 +92,14 @@ export("cacheTable", "createExternalTable", "dropTempTable", "jsonFile", - "jsonRDD", "loadDF", "parquetFile", "sql", "table", "tableNames", "tables", - "toDF", "uncacheTable") -export("sparkRSQL.init", - "sparkRHive.init") - export("structField", "structField.jobj", "structField.character", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 841e77e55e0d8..354642e7bc307 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -45,6 +45,9 @@ setMethod("initialize", "DataFrame", function(.Object, sdf, isCached) { #' @rdname DataFrame #' @export +#' +#' @param sdf A Java object reference to the backing Scala DataFrame +#' @param isCached TRUE if the dataFrame is cached dataFrame <- function(sdf, isCached = FALSE) { new("DataFrame", sdf, isCached) } @@ -244,7 +247,7 @@ setMethod("columns", }) #' @rdname columns -#' @export +#' @aliases names,DataFrame,function-method setMethod("names", signature(x = "DataFrame"), function(x) { @@ -272,7 +275,7 @@ setMethod("names", setMethod("registerTempTable", signature(x = "DataFrame", tableName = "character"), function(x, tableName) { - callJMethod(x@sdf, "registerTempTable", tableName) + invisible(callJMethod(x@sdf, "registerTempTable", tableName)) }) #' insertInto @@ -399,23 +402,23 @@ setMethod("repartition", dataFrame(sdf) }) -#' toJSON -#' -#' Convert the rows of a DataFrame into JSON objects and return an RDD where -#' each element contains a JSON string. -#' -#' @param x A SparkSQL DataFrame -#' @return A StringRRDD of JSON objects -#' @rdname tojson -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) -#' path <- "path/to/file.json" -#' df <- jsonFile(sqlCtx, path) -#' newRDD <- toJSON(df) -#'} +# toJSON +# +# Convert the rows of a DataFrame into JSON objects and return an RDD where +# each element contains a JSON string. +# +#@param x A SparkSQL DataFrame +# @return A StringRRDD of JSON objects +# @rdname tojson +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# sqlCtx <- sparkRSQL.init(sc) +# path <- "path/to/file.json" +# df <- jsonFile(sqlCtx, path) +# newRDD <- toJSON(df) +#} setMethod("toJSON", signature(x = "DataFrame"), function(x) { @@ -578,8 +581,8 @@ setMethod("limit", dataFrame(res) }) -# Take the first NUM rows of a DataFrame and return a the results as a data.frame - +#' Take the first NUM rows of a DataFrame and return a the results as a data.frame +#' #' @rdname take #' @export #' @examples @@ -644,22 +647,22 @@ setMethod("first", take(x, 1) }) -#' toRDD() -#' -#' Converts a Spark DataFrame to an RDD while preserving column names. -#' -#' @param x A Spark DataFrame -#' -#' @rdname DataFrame -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) -#' path <- "path/to/file.json" -#' df <- jsonFile(sqlCtx, path) -#' rdd <- toRDD(df) -#' } +# toRDD() +# +# Converts a Spark DataFrame to an RDD while preserving column names. +# +# @param x A Spark DataFrame +# +# @rdname DataFrame +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# sqlCtx <- sparkRSQL.init(sc) +# path <- "path/to/file.json" +# df <- jsonFile(sqlCtx, path) +# rdd <- toRDD(df) +# } setMethod("toRDD", signature(x = "DataFrame"), function(x) { @@ -706,6 +709,7 @@ setMethod("groupBy", #' #' Compute aggregates by specifying a list of columns #' +#' @param x a DataFrame #' @rdname DataFrame #' @export setMethod("agg", @@ -721,7 +725,7 @@ setMethod("agg", # the requested map function. # ################################################################################### -#' @rdname lapply +# @rdname lapply setMethod("lapply", signature(X = "DataFrame", FUN = "function"), function(X, FUN) { @@ -729,14 +733,14 @@ setMethod("lapply", lapply(rdd, FUN) }) -#' @rdname lapply +# @rdname lapply setMethod("map", signature(X = "DataFrame", FUN = "function"), function(X, FUN) { lapply(X, FUN) }) -#' @rdname flatMap +# @rdname flatMap setMethod("flatMap", signature(X = "DataFrame", FUN = "function"), function(X, FUN) { @@ -744,7 +748,7 @@ setMethod("flatMap", flatMap(rdd, FUN) }) -#' @rdname lapplyPartition +# @rdname lapplyPartition setMethod("lapplyPartition", signature(X = "DataFrame", FUN = "function"), function(X, FUN) { @@ -752,14 +756,14 @@ setMethod("lapplyPartition", lapplyPartition(rdd, FUN) }) -#' @rdname lapplyPartition +# @rdname lapplyPartition setMethod("mapPartitions", signature(X = "DataFrame", FUN = "function"), function(X, FUN) { lapplyPartition(X, FUN) }) -#' @rdname foreach +# @rdname foreach setMethod("foreach", signature(x = "DataFrame", func = "function"), function(x, func) { @@ -767,7 +771,7 @@ setMethod("foreach", foreach(rdd, func) }) -#' @rdname foreach +# @rdname foreach setMethod("foreachPartition", signature(x = "DataFrame", func = "function"), function(x, func) { @@ -788,6 +792,7 @@ setMethod("$", signature(x = "DataFrame"), getColumn(x, name) }) +#' @rdname select setMethod("$<-", signature(x = "DataFrame"), function(x, name, value) { stopifnot(class(value) == "Column" || is.null(value)) @@ -1009,7 +1014,7 @@ setMethod("sortDF", }) #' @rdname sortDF -#' @export +#' @aliases orderBy,DataFrame,function-method setMethod("orderBy", signature(x = "DataFrame", col = "characterOrColumn"), function(x, col) { @@ -1046,7 +1051,7 @@ setMethod("filter", }) #' @rdname filter -#' @export +#' @aliases where,DataFrame,function-method setMethod("where", signature(x = "DataFrame", condition = "characterOrColumn"), function(x, condition) { @@ -1276,3 +1281,40 @@ setMethod("saveAsTable", callJMethod(df@sdf, "saveAsTable", tableName, source, jmode, options) }) +#' describe +#' +#' Computes statistics for numeric columns. +#' If no columns are given, this function computes statistics for all numerical columns. +#' +#' @param x A DataFrame to be computed. +#' @param col A string of name +#' @param ... Additional expressions +#' @return A DataFrame +#' @rdname describe +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' describe(df) +#' describe(df, "col1") +#' describe(df, "col1", "col2") +#' } +setMethod("describe", + signature(x = "DataFrame", col = "character"), + function(x, col, ...) { + colList <- list(col, ...) + sdf <- callJMethod(x@sdf, "describe", listToSeq(colList)) + dataFrame(sdf) + }) + +#' @rdname describe +setMethod("describe", + signature(x = "DataFrame"), + function(x) { + colList <- as.list(c(columns(x))) + sdf <- callJMethod(x@sdf, "describe", listToSeq(colList)) + dataFrame(sdf) + }) diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index d1018c2361670..73999a6737032 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -19,16 +19,16 @@ setOldClass("jobj") -#' @title S4 class that represents an RDD -#' @description RDD can be created using functions like -#' \code{parallelize}, \code{textFile} etc. -#' @rdname RDD -#' @seealso parallelize, textFile -#' -#' @slot env An R environment that stores bookkeeping states of the RDD -#' @slot jrdd Java object reference to the backing JavaRDD -#' to an RDD -#' @export +# @title S4 class that represents an RDD +# @description RDD can be created using functions like +# \code{parallelize}, \code{textFile} etc. +# @rdname RDD +# @seealso parallelize, textFile +# +# @slot env An R environment that stores bookkeeping states of the RDD +# @slot jrdd Java object reference to the backing JavaRDD +# to an RDD +# @export setClass("RDD", slots = list(env = "environment", jrdd = "jobj")) @@ -108,14 +108,14 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) .Object }) -#' @rdname RDD -#' @export -#' -#' @param jrdd Java object reference to the backing JavaRDD -#' @param serializedMode Use "byte" if the RDD stores data serialized in R, "string" if the RDD -#' stores strings, and "row" if the RDD stores the rows of a DataFrame -#' @param isCached TRUE if the RDD is cached -#' @param isCheckpointed TRUE if the RDD has been checkpointed +# @rdname RDD +# @export +# +# @param jrdd Java object reference to the backing JavaRDD +# @param serializedMode Use "byte" if the RDD stores data serialized in R, "string" if the RDD +# stores strings, and "row" if the RDD stores the rows of a DataFrame +# @param isCached TRUE if the RDD is cached +# @param isCheckpointed TRUE if the RDD has been checkpointed RDD <- function(jrdd, serializedMode = "byte", isCached = FALSE, isCheckpointed = FALSE) { new("RDD", jrdd, serializedMode, isCached, isCheckpointed) @@ -200,19 +200,19 @@ setValidity("RDD", ############ Actions and Transformations ############ -#' Persist an RDD -#' -#' Persist this RDD with the default storage level (MEMORY_ONLY). -#' -#' @param x The RDD to cache -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' cache(rdd) -#'} -#' @rdname cache-methods -#' @aliases cache,RDD-method +# Persist an RDD +# +# Persist this RDD with the default storage level (MEMORY_ONLY). +# +# @param x The RDD to cache +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10, 2L) +# cache(rdd) +#} +# @rdname cache-methods +# @aliases cache,RDD-method setMethod("cache", signature(x = "RDD"), function(x) { @@ -221,22 +221,22 @@ setMethod("cache", x }) -#' Persist an RDD -#' -#' Persist this RDD with the specified storage level. For details of the -#' supported storage levels, refer to -#' http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. -#' -#' @param x The RDD to persist -#' @param newLevel The new storage level to be assigned -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' persist(rdd, "MEMORY_AND_DISK") -#'} -#' @rdname persist -#' @aliases persist,RDD-method +# Persist an RDD +# +# Persist this RDD with the specified storage level. For details of the +# supported storage levels, refer to +# http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. +# +# @param x The RDD to persist +# @param newLevel The new storage level to be assigned +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10, 2L) +# persist(rdd, "MEMORY_AND_DISK") +#} +# @rdname persist +# @aliases persist,RDD-method setMethod("persist", signature(x = "RDD", newLevel = "character"), function(x, newLevel) { @@ -245,21 +245,21 @@ setMethod("persist", x }) -#' Unpersist an RDD -#' -#' Mark the RDD as non-persistent, and remove all blocks for it from memory and -#' disk. -#' -#' @param x The RDD to unpersist -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' cache(rdd) # rdd@@env$isCached == TRUE -#' unpersist(rdd) # rdd@@env$isCached == FALSE -#'} -#' @rdname unpersist-methods -#' @aliases unpersist,RDD-method +# Unpersist an RDD +# +# Mark the RDD as non-persistent, and remove all blocks for it from memory and +# disk. +# +# @param x The RDD to unpersist +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10, 2L) +# cache(rdd) # rdd@@env$isCached == TRUE +# unpersist(rdd) # rdd@@env$isCached == FALSE +#} +# @rdname unpersist-methods +# @aliases unpersist,RDD-method setMethod("unpersist", signature(x = "RDD"), function(x) { @@ -268,24 +268,24 @@ setMethod("unpersist", x }) -#' Checkpoint an RDD -#' -#' Mark this RDD for checkpointing. It will be saved to a file inside the -#' checkpoint directory set with setCheckpointDir() and all references to its -#' parent RDDs will be removed. This function must be called before any job has -#' been executed on this RDD. It is strongly recommended that this RDD is -#' persisted in memory, otherwise saving it on a file will require recomputation. -#' -#' @param x The RDD to checkpoint -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' setCheckpointDir(sc, "checkpoint") -#' rdd <- parallelize(sc, 1:10, 2L) -#' checkpoint(rdd) -#'} -#' @rdname checkpoint-methods -#' @aliases checkpoint,RDD-method +# Checkpoint an RDD +# +# Mark this RDD for checkpointing. It will be saved to a file inside the +# checkpoint directory set with setCheckpointDir() and all references to its +# parent RDDs will be removed. This function must be called before any job has +# been executed on this RDD. It is strongly recommended that this RDD is +# persisted in memory, otherwise saving it on a file will require recomputation. +# +# @param x The RDD to checkpoint +# @examples +#\dontrun{ +# sc <- sparkR.init() +# setCheckpointDir(sc, "checkpoint") +# rdd <- parallelize(sc, 1:10, 2L) +# checkpoint(rdd) +#} +# @rdname checkpoint-methods +# @aliases checkpoint,RDD-method setMethod("checkpoint", signature(x = "RDD"), function(x) { @@ -295,18 +295,18 @@ setMethod("checkpoint", x }) -#' Gets the number of partitions of an RDD -#' -#' @param x A RDD. -#' @return the number of partitions of rdd as an integer. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' numPartitions(rdd) # 2L -#'} -#' @rdname numPartitions -#' @aliases numPartitions,RDD-method +# Gets the number of partitions of an RDD +# +# @param x A RDD. +# @return the number of partitions of rdd as an integer. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10, 2L) +# numPartitions(rdd) # 2L +#} +# @rdname numPartitions +# @aliases numPartitions,RDD-method setMethod("numPartitions", signature(x = "RDD"), function(x) { @@ -315,24 +315,24 @@ setMethod("numPartitions", callJMethod(partitions, "size") }) -#' Collect elements of an RDD -#' -#' @description -#' \code{collect} returns a list that contains all of the elements in this RDD. -#' -#' @param x The RDD to collect -#' @param ... Other optional arguments to collect -#' @param flatten FALSE if the list should not flattened -#' @return a list containing elements in the RDD -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' collect(rdd) # list from 1 to 10 -#' collectPartition(rdd, 0L) # list from 1 to 5 -#'} -#' @rdname collect-methods -#' @aliases collect,RDD-method +# Collect elements of an RDD +# +# @description +# \code{collect} returns a list that contains all of the elements in this RDD. +# +# @param x The RDD to collect +# @param ... Other optional arguments to collect +# @param flatten FALSE if the list should not flattened +# @return a list containing elements in the RDD +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10, 2L) +# collect(rdd) # list from 1 to 10 +# collectPartition(rdd, 0L) # list from 1 to 5 +#} +# @rdname collect-methods +# @aliases collect,RDD-method setMethod("collect", signature(x = "RDD"), function(x, flatten = TRUE) { @@ -343,12 +343,12 @@ setMethod("collect", }) -#' @description -#' \code{collectPartition} returns a list that contains all of the elements -#' in the specified partition of the RDD. -#' @param partitionId the partition to collect (starts from 0) -#' @rdname collect-methods -#' @aliases collectPartition,integer,RDD-method +# @description +# \code{collectPartition} returns a list that contains all of the elements +# in the specified partition of the RDD. +# @param partitionId the partition to collect (starts from 0) +# @rdname collect-methods +# @aliases collectPartition,integer,RDD-method setMethod("collectPartition", signature(x = "RDD", partitionId = "integer"), function(x, partitionId) { @@ -361,17 +361,17 @@ setMethod("collectPartition", serializedMode = getSerializedMode(x)) }) -#' @description -#' \code{collectAsMap} returns a named list as a map that contains all of the elements -#' in a key-value pair RDD. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) -#' collectAsMap(rdd) # list(`1` = 2, `3` = 4) -#'} -#' @rdname collect-methods -#' @aliases collectAsMap,RDD-method +# @description +# \code{collectAsMap} returns a named list as a map that contains all of the elements +# in a key-value pair RDD. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) +# collectAsMap(rdd) # list(`1` = 2, `3` = 4) +#} +# @rdname collect-methods +# @aliases collectAsMap,RDD-method setMethod("collectAsMap", signature(x = "RDD"), function(x) { @@ -381,19 +381,19 @@ setMethod("collectAsMap", as.list(map) }) -#' Return the number of elements in the RDD. -#' -#' @param x The RDD to count -#' @return number of elements in the RDD. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' count(rdd) # 10 -#' length(rdd) # Same as count -#'} -#' @rdname count -#' @aliases count,RDD-method +# Return the number of elements in the RDD. +# +# @param x The RDD to count +# @return number of elements in the RDD. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# count(rdd) # 10 +# length(rdd) # Same as count +#} +# @rdname count +# @aliases count,RDD-method setMethod("count", signature(x = "RDD"), function(x) { @@ -405,31 +405,31 @@ setMethod("count", sum(as.integer(vals)) }) -#' Return the number of elements in the RDD -#' @export -#' @rdname count +# Return the number of elements in the RDD +# @export +# @rdname count setMethod("length", signature(x = "RDD"), function(x) { count(x) }) -#' Return the count of each unique value in this RDD as a list of -#' (value, count) pairs. -#' -#' Same as countByValue in Spark. -#' -#' @param x The RDD to count -#' @return list of (value, count) pairs, where count is number of each unique -#' value in rdd. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, c(1,2,3,2,1)) -#' countByValue(rdd) # (1,2L), (2,2L), (3,1L) -#'} -#' @rdname countByValue -#' @aliases countByValue,RDD-method +# Return the count of each unique value in this RDD as a list of +# (value, count) pairs. +# +# Same as countByValue in Spark. +# +# @param x The RDD to count +# @return list of (value, count) pairs, where count is number of each unique +# value in rdd. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, c(1,2,3,2,1)) +# countByValue(rdd) # (1,2L), (2,2L), (3,1L) +#} +# @rdname countByValue +# @aliases countByValue,RDD-method setMethod("countByValue", signature(x = "RDD"), function(x) { @@ -437,23 +437,23 @@ setMethod("countByValue", collect(reduceByKey(ones, `+`, numPartitions(x))) }) -#' Apply a function to all elements -#' -#' This function creates a new RDD by applying the given transformation to all -#' elements of the given RDD -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on each element -#' @return a new RDD created by the transformation. -#' @rdname lapply -#' @aliases lapply -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' multiplyByTwo <- lapply(rdd, function(x) { x * 2 }) -#' collect(multiplyByTwo) # 2,4,6... -#'} +# Apply a function to all elements +# +# This function creates a new RDD by applying the given transformation to all +# elements of the given RDD +# +# @param X The RDD to apply the transformation. +# @param FUN the transformation to apply on each element +# @return a new RDD created by the transformation. +# @rdname lapply +# @aliases lapply +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# multiplyByTwo <- lapply(rdd, function(x) { x * 2 }) +# collect(multiplyByTwo) # 2,4,6... +#} setMethod("lapply", signature(X = "RDD", FUN = "function"), function(X, FUN) { @@ -463,31 +463,31 @@ setMethod("lapply", lapplyPartitionsWithIndex(X, func) }) -#' @rdname lapply -#' @aliases map,RDD,function-method +# @rdname lapply +# @aliases map,RDD,function-method setMethod("map", signature(X = "RDD", FUN = "function"), function(X, FUN) { lapply(X, FUN) }) -#' Flatten results after apply a function to all elements -#' -#' This function return a new RDD by first applying a function to all -#' elements of this RDD, and then flattening the results. -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on each element -#' @return a new RDD created by the transformation. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' multiplyByTwo <- flatMap(rdd, function(x) { list(x*2, x*10) }) -#' collect(multiplyByTwo) # 2,20,4,40,6,60... -#'} -#' @rdname flatMap -#' @aliases flatMap,RDD,function-method +# Flatten results after apply a function to all elements +# +# This function return a new RDD by first applying a function to all +# elements of this RDD, and then flattening the results. +# +# @param X The RDD to apply the transformation. +# @param FUN the transformation to apply on each element +# @return a new RDD created by the transformation. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# multiplyByTwo <- flatMap(rdd, function(x) { list(x*2, x*10) }) +# collect(multiplyByTwo) # 2,20,4,40,6,60... +#} +# @rdname flatMap +# @aliases flatMap,RDD,function-method setMethod("flatMap", signature(X = "RDD", FUN = "function"), function(X, FUN) { @@ -500,83 +500,83 @@ setMethod("flatMap", lapplyPartition(X, partitionFunc) }) -#' Apply a function to each partition of an RDD -#' -#' Return a new RDD by applying a function to each partition of this RDD. -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on each partition. -#' @return a new RDD created by the transformation. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' partitionSum <- lapplyPartition(rdd, function(part) { Reduce("+", part) }) -#' collect(partitionSum) # 15, 40 -#'} -#' @rdname lapplyPartition -#' @aliases lapplyPartition,RDD,function-method +# Apply a function to each partition of an RDD +# +# Return a new RDD by applying a function to each partition of this RDD. +# +# @param X The RDD to apply the transformation. +# @param FUN the transformation to apply on each partition. +# @return a new RDD created by the transformation. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# partitionSum <- lapplyPartition(rdd, function(part) { Reduce("+", part) }) +# collect(partitionSum) # 15, 40 +#} +# @rdname lapplyPartition +# @aliases lapplyPartition,RDD,function-method setMethod("lapplyPartition", signature(X = "RDD", FUN = "function"), function(X, FUN) { lapplyPartitionsWithIndex(X, function(s, part) { FUN(part) }) }) -#' mapPartitions is the same as lapplyPartition. -#' -#' @rdname lapplyPartition -#' @aliases mapPartitions,RDD,function-method +# mapPartitions is the same as lapplyPartition. +# +# @rdname lapplyPartition +# @aliases mapPartitions,RDD,function-method setMethod("mapPartitions", signature(X = "RDD", FUN = "function"), function(X, FUN) { lapplyPartition(X, FUN) }) -#' Return a new RDD by applying a function to each partition of this RDD, while -#' tracking the index of the original partition. -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on each partition; takes the partition -#' index and a list of elements in the particular partition. -#' @return a new RDD created by the transformation. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 5L) -#' prod <- lapplyPartitionsWithIndex(rdd, function(partIndex, part) { -#' partIndex * Reduce("+", part) }) -#' collect(prod, flatten = FALSE) # 0, 7, 22, 45, 76 -#'} -#' @rdname lapplyPartitionsWithIndex -#' @aliases lapplyPartitionsWithIndex,RDD,function-method +# Return a new RDD by applying a function to each partition of this RDD, while +# tracking the index of the original partition. +# +# @param X The RDD to apply the transformation. +# @param FUN the transformation to apply on each partition; takes the partition +# index and a list of elements in the particular partition. +# @return a new RDD created by the transformation. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10, 5L) +# prod <- lapplyPartitionsWithIndex(rdd, function(partIndex, part) { +# partIndex * Reduce("+", part) }) +# collect(prod, flatten = FALSE) # 0, 7, 22, 45, 76 +#} +# @rdname lapplyPartitionsWithIndex +# @aliases lapplyPartitionsWithIndex,RDD,function-method setMethod("lapplyPartitionsWithIndex", signature(X = "RDD", FUN = "function"), function(X, FUN) { PipelinedRDD(X, FUN) }) -#' @rdname lapplyPartitionsWithIndex -#' @aliases mapPartitionsWithIndex,RDD,function-method +# @rdname lapplyPartitionsWithIndex +# @aliases mapPartitionsWithIndex,RDD,function-method setMethod("mapPartitionsWithIndex", signature(X = "RDD", FUN = "function"), function(X, FUN) { lapplyPartitionsWithIndex(X, FUN) }) -#' This function returns a new RDD containing only the elements that satisfy -#' a predicate (i.e. returning TRUE in a given logical function). -#' The same as `filter()' in Spark. -#' -#' @param x The RDD to be filtered. -#' @param f A unary predicate function. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' unlist(collect(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) -#'} -#' @rdname filterRDD -#' @aliases filterRDD,RDD,function-method +# This function returns a new RDD containing only the elements that satisfy +# a predicate (i.e. returning TRUE in a given logical function). +# The same as `filter()' in Spark. +# +# @param x The RDD to be filtered. +# @param f A unary predicate function. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# unlist(collect(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) +#} +# @rdname filterRDD +# @aliases filterRDD,RDD,function-method setMethod("filterRDD", signature(x = "RDD", f = "function"), function(x, f) { @@ -586,30 +586,30 @@ setMethod("filterRDD", lapplyPartition(x, filter.func) }) -#' @rdname filterRDD -#' @aliases Filter +# @rdname filterRDD +# @aliases Filter setMethod("Filter", signature(f = "function", x = "RDD"), function(f, x) { filterRDD(x, f) }) -#' Reduce across elements of an RDD. -#' -#' This function reduces the elements of this RDD using the -#' specified commutative and associative binary operator. -#' -#' @param x The RDD to reduce -#' @param func Commutative and associative function to apply on elements -#' of the RDD. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' reduce(rdd, "+") # 55 -#'} -#' @rdname reduce -#' @aliases reduce,RDD,ANY-method +# Reduce across elements of an RDD. +# +# This function reduces the elements of this RDD using the +# specified commutative and associative binary operator. +# +# @param x The RDD to reduce +# @param func Commutative and associative function to apply on elements +# of the RDD. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# reduce(rdd, "+") # 55 +#} +# @rdname reduce +# @aliases reduce,RDD,ANY-method setMethod("reduce", signature(x = "RDD", func = "ANY"), function(x, func) { @@ -623,70 +623,70 @@ setMethod("reduce", Reduce(func, partitionList) }) -#' Get the maximum element of an RDD. -#' -#' @param x The RDD to get the maximum element from -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' maximum(rdd) # 10 -#'} -#' @rdname maximum -#' @aliases maximum,RDD +# Get the maximum element of an RDD. +# +# @param x The RDD to get the maximum element from +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# maximum(rdd) # 10 +#} +# @rdname maximum +# @aliases maximum,RDD setMethod("maximum", signature(x = "RDD"), function(x) { reduce(x, max) }) -#' Get the minimum element of an RDD. -#' -#' @param x The RDD to get the minimum element from -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' minimum(rdd) # 1 -#'} -#' @rdname minimum -#' @aliases minimum,RDD +# Get the minimum element of an RDD. +# +# @param x The RDD to get the minimum element from +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# minimum(rdd) # 1 +#} +# @rdname minimum +# @aliases minimum,RDD setMethod("minimum", signature(x = "RDD"), function(x) { reduce(x, min) }) -#' Add up the elements in an RDD. -#' -#' @param x The RDD to add up the elements in -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' sumRDD(rdd) # 55 -#'} -#' @rdname sumRDD -#' @aliases sumRDD,RDD +# Add up the elements in an RDD. +# +# @param x The RDD to add up the elements in +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# sumRDD(rdd) # 55 +#} +# @rdname sumRDD +# @aliases sumRDD,RDD setMethod("sumRDD", signature(x = "RDD"), function(x) { reduce(x, "+") }) -#' Applies a function to all elements in an RDD, and force evaluation. -#' -#' @param x The RDD to apply the function -#' @param func The function to be applied. -#' @return invisible NULL. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' foreach(rdd, function(x) { save(x, file=...) }) -#'} -#' @rdname foreach -#' @aliases foreach,RDD,function-method +# Applies a function to all elements in an RDD, and force evaluation. +# +# @param x The RDD to apply the function +# @param func The function to be applied. +# @return invisible NULL. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# foreach(rdd, function(x) { save(x, file=...) }) +#} +# @rdname foreach +# @aliases foreach,RDD,function-method setMethod("foreach", signature(x = "RDD", func = "function"), function(x, func) { @@ -697,37 +697,37 @@ setMethod("foreach", invisible(collect(mapPartitions(x, partition.func))) }) -#' Applies a function to each partition in an RDD, and force evaluation. -#' -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' foreachPartition(rdd, function(part) { save(part, file=...); NULL }) -#'} -#' @rdname foreach -#' @aliases foreachPartition,RDD,function-method +# Applies a function to each partition in an RDD, and force evaluation. +# +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# foreachPartition(rdd, function(part) { save(part, file=...); NULL }) +#} +# @rdname foreach +# @aliases foreachPartition,RDD,function-method setMethod("foreachPartition", signature(x = "RDD", func = "function"), function(x, func) { invisible(collect(mapPartitions(x, func))) }) -#' Take elements from an RDD. -#' -#' This function takes the first NUM elements in the RDD and -#' returns them in a list. -#' -#' @param x The RDD to take elements from -#' @param num Number of elements to take -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' take(rdd, 2L) # list(1, 2) -#'} -#' @rdname take -#' @aliases take,RDD,numeric-method +# Take elements from an RDD. +# +# This function takes the first NUM elements in the RDD and +# returns them in a list. +# +# @param x The RDD to take elements from +# @param num Number of elements to take +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# take(rdd, 2L) # list(1, 2) +#} +# @rdname take +# @aliases take,RDD,numeric-method setMethod("take", signature(x = "RDD", num = "numeric"), function(x, num) { @@ -762,39 +762,39 @@ setMethod("take", }) -#' First -#' -#' Return the first element of an RDD -#' -#' @rdname first -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' first(rdd) -#' } +# First +# +# Return the first element of an RDD +# +# @rdname first +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# first(rdd) +# } setMethod("first", signature(x = "RDD"), function(x) { take(x, 1)[[1]] }) -#' Removes the duplicates from RDD. -#' -#' This function returns a new RDD containing the distinct elements in the -#' given RDD. The same as `distinct()' in Spark. -#' -#' @param x The RDD to remove duplicates from. -#' @param numPartitions Number of partitions to create. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, c(1,2,2,3,3,3)) -#' sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) -#'} -#' @rdname distinct -#' @aliases distinct,RDD-method +# Removes the duplicates from RDD. +# +# This function returns a new RDD containing the distinct elements in the +# given RDD. The same as `distinct()' in Spark. +# +# @param x The RDD to remove duplicates from. +# @param numPartitions Number of partitions to create. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, c(1,2,2,3,3,3)) +# sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) +#} +# @rdname distinct +# @aliases distinct,RDD-method setMethod("distinct", signature(x = "RDD"), function(x, numPartitions = SparkR:::numPartitions(x)) { @@ -806,24 +806,24 @@ setMethod("distinct", resRDD }) -#' Return an RDD that is a sampled subset of the given RDD. -#' -#' The same as `sample()' in Spark. (We rename it due to signature -#' inconsistencies with the `sample()' function in R's base package.) -#' -#' @param x The RDD to sample elements from -#' @param withReplacement Sampling with replacement or not -#' @param fraction The (rough) sample target fraction -#' @param seed Randomness seed value -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' collect(sampleRDD(rdd, FALSE, 0.5, 1618L)) # ~5 distinct elements -#' collect(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates -#'} -#' @rdname sampleRDD -#' @aliases sampleRDD,RDD +# Return an RDD that is a sampled subset of the given RDD. +# +# The same as `sample()' in Spark. (We rename it due to signature +# inconsistencies with the `sample()' function in R's base package.) +# +# @param x The RDD to sample elements from +# @param withReplacement Sampling with replacement or not +# @param fraction The (rough) sample target fraction +# @param seed Randomness seed value +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# collect(sampleRDD(rdd, FALSE, 0.5, 1618L)) # ~5 distinct elements +# collect(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates +#} +# @rdname sampleRDD +# @aliases sampleRDD,RDD setMethod("sampleRDD", signature(x = "RDD", withReplacement = "logical", fraction = "numeric", seed = "integer"), @@ -867,23 +867,23 @@ setMethod("sampleRDD", lapplyPartitionsWithIndex(x, samplingFunc) }) -#' Return a list of the elements that are a sampled subset of the given RDD. -#' -#' @param x The RDD to sample elements from -#' @param withReplacement Sampling with replacement or not -#' @param num Number of elements to return -#' @param seed Randomness seed value -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:100) -#' # exactly 5 elements sampled, which may not be distinct -#' takeSample(rdd, TRUE, 5L, 1618L) -#' # exactly 5 distinct elements sampled -#' takeSample(rdd, FALSE, 5L, 16181618L) -#'} -#' @rdname takeSample -#' @aliases takeSample,RDD +# Return a list of the elements that are a sampled subset of the given RDD. +# +# @param x The RDD to sample elements from +# @param withReplacement Sampling with replacement or not +# @param num Number of elements to return +# @param seed Randomness seed value +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:100) +# # exactly 5 elements sampled, which may not be distinct +# takeSample(rdd, TRUE, 5L, 1618L) +# # exactly 5 distinct elements sampled +# takeSample(rdd, FALSE, 5L, 16181618L) +#} +# @rdname takeSample +# @aliases takeSample,RDD setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", num = "integer", seed = "integer"), function(x, withReplacement, num, seed) { @@ -930,18 +930,18 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", sample(samples)[1:total] }) -#' Creates tuples of the elements in this RDD by applying a function. -#' -#' @param x The RDD. -#' @param func The function to be applied. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3)) -#' collect(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) -#'} -#' @rdname keyBy -#' @aliases keyBy,RDD +# Creates tuples of the elements in this RDD by applying a function. +# +# @param x The RDD. +# @param func The function to be applied. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(1, 2, 3)) +# collect(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) +#} +# @rdname keyBy +# @aliases keyBy,RDD setMethod("keyBy", signature(x = "RDD", func = "function"), function(x, func) { @@ -951,44 +951,44 @@ setMethod("keyBy", lapply(x, apply.func) }) -#' Return a new RDD that has exactly numPartitions partitions. -#' Can increase or decrease the level of parallelism in this RDD. Internally, -#' this uses a shuffle to redistribute data. -#' If you are decreasing the number of partitions in this RDD, consider using -#' coalesce, which can avoid performing a shuffle. -#' -#' @param x The RDD. -#' @param numPartitions Number of partitions to create. -#' @seealso coalesce -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5, 6, 7), 4L) -#' numPartitions(rdd) # 4 -#' numPartitions(repartition(rdd, 2L)) # 2 -#'} -#' @rdname repartition -#' @aliases repartition,RDD +# Return a new RDD that has exactly numPartitions partitions. +# Can increase or decrease the level of parallelism in this RDD. Internally, +# this uses a shuffle to redistribute data. +# If you are decreasing the number of partitions in this RDD, consider using +# coalesce, which can avoid performing a shuffle. +# +# @param x The RDD. +# @param numPartitions Number of partitions to create. +# @seealso coalesce +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(1, 2, 3, 4, 5, 6, 7), 4L) +# numPartitions(rdd) # 4 +# numPartitions(repartition(rdd, 2L)) # 2 +#} +# @rdname repartition +# @aliases repartition,RDD setMethod("repartition", signature(x = "RDD", numPartitions = "numeric"), function(x, numPartitions) { coalesce(x, numPartitions, TRUE) }) -#' Return a new RDD that is reduced into numPartitions partitions. -#' -#' @param x The RDD. -#' @param numPartitions Number of partitions to create. -#' @seealso repartition -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5), 3L) -#' numPartitions(rdd) # 3 -#' numPartitions(coalesce(rdd, 1L)) # 1 -#'} -#' @rdname coalesce -#' @aliases coalesce,RDD +# Return a new RDD that is reduced into numPartitions partitions. +# +# @param x The RDD. +# @param numPartitions Number of partitions to create. +# @seealso repartition +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(1, 2, 3, 4, 5), 3L) +# numPartitions(rdd) # 3 +# numPartitions(coalesce(rdd, 1L)) # 1 +#} +# @rdname coalesce +# @aliases coalesce,RDD setMethod("coalesce", signature(x = "RDD", numPartitions = "numeric"), function(x, numPartitions, shuffle = FALSE) { @@ -1012,19 +1012,19 @@ setMethod("coalesce", } }) -#' Save this RDD as a SequenceFile of serialized objects. -#' -#' @param x The RDD to save -#' @param path The directory where the file is saved -#' @seealso objectFile -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:3) -#' saveAsObjectFile(rdd, "/tmp/sparkR-tmp") -#'} -#' @rdname saveAsObjectFile -#' @aliases saveAsObjectFile,RDD +# Save this RDD as a SequenceFile of serialized objects. +# +# @param x The RDD to save +# @param path The directory where the file is saved +# @seealso objectFile +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:3) +# saveAsObjectFile(rdd, "/tmp/sparkR-tmp") +#} +# @rdname saveAsObjectFile +# @aliases saveAsObjectFile,RDD setMethod("saveAsObjectFile", signature(x = "RDD", path = "character"), function(x, path) { @@ -1037,18 +1037,18 @@ setMethod("saveAsObjectFile", invisible(callJMethod(getJRDD(x), "saveAsObjectFile", path)) }) -#' Save this RDD as a text file, using string representations of elements. -#' -#' @param x The RDD to save -#' @param path The directory where the partitions of the text file are saved -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:3) -#' saveAsTextFile(rdd, "/tmp/sparkR-tmp") -#'} -#' @rdname saveAsTextFile -#' @aliases saveAsTextFile,RDD +# Save this RDD as a text file, using string representations of elements. +# +# @param x The RDD to save +# @param path The directory where the partitions of the text file are saved +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:3) +# saveAsTextFile(rdd, "/tmp/sparkR-tmp") +#} +# @rdname saveAsTextFile +# @aliases saveAsTextFile,RDD setMethod("saveAsTextFile", signature(x = "RDD", path = "character"), function(x, path) { @@ -1061,21 +1061,21 @@ setMethod("saveAsTextFile", callJMethod(getJRDD(stringRdd, serializedMode = "string"), "saveAsTextFile", path)) }) -#' Sort an RDD by the given key function. -#' -#' @param x An RDD to be sorted. -#' @param func A function used to compute the sort key for each element. -#' @param ascending A flag to indicate whether the sorting is ascending or descending. -#' @param numPartitions Number of partitions to create. -#' @return An RDD where all elements are sorted. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(3, 2, 1)) -#' collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) -#'} -#' @rdname sortBy -#' @aliases sortBy,RDD,RDD-method +# Sort an RDD by the given key function. +# +# @param x An RDD to be sorted. +# @param func A function used to compute the sort key for each element. +# @param ascending A flag to indicate whether the sorting is ascending or descending. +# @param numPartitions Number of partitions to create. +# @return An RDD where all elements are sorted. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(3, 2, 1)) +# collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) +#} +# @rdname sortBy +# @aliases sortBy,RDD,RDD-method setMethod("sortBy", signature(x = "RDD", func = "function"), function(x, func, ascending = TRUE, numPartitions = SparkR:::numPartitions(x)) { @@ -1137,97 +1137,97 @@ takeOrderedElem <- function(x, num, ascending = TRUE) { resList } -#' Returns the first N elements from an RDD in ascending order. -#' -#' @param x An RDD. -#' @param num Number of elements to return. -#' @return The first N elements from the RDD in ascending order. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) -#' takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) -#'} -#' @rdname takeOrdered -#' @aliases takeOrdered,RDD,RDD-method +# Returns the first N elements from an RDD in ascending order. +# +# @param x An RDD. +# @param num Number of elements to return. +# @return The first N elements from the RDD in ascending order. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +# takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) +#} +# @rdname takeOrdered +# @aliases takeOrdered,RDD,RDD-method setMethod("takeOrdered", signature(x = "RDD", num = "integer"), function(x, num) { takeOrderedElem(x, num) }) -#' Returns the top N elements from an RDD. -#' -#' @param x An RDD. -#' @param num Number of elements to return. -#' @return The top N elements from the RDD. -#' @rdname top -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) -#' top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) -#'} -#' @rdname top -#' @aliases top,RDD,RDD-method +# Returns the top N elements from an RDD. +# +# @param x An RDD. +# @param num Number of elements to return. +# @return The top N elements from the RDD. +# @rdname top +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +# top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) +#} +# @rdname top +# @aliases top,RDD,RDD-method setMethod("top", signature(x = "RDD", num = "integer"), function(x, num) { takeOrderedElem(x, num, FALSE) }) -#' Fold an RDD using a given associative function and a neutral "zero value". -#' -#' Aggregate the elements of each partition, and then the results for all the -#' partitions, using a given associative function and a neutral "zero value". -#' -#' @param x An RDD. -#' @param zeroValue A neutral "zero value". -#' @param op An associative function for the folding operation. -#' @return The folding result. -#' @rdname fold -#' @seealso reduce -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) -#' fold(rdd, 0, "+") # 15 -#'} -#' @rdname fold -#' @aliases fold,RDD,RDD-method +# Fold an RDD using a given associative function and a neutral "zero value". +# +# Aggregate the elements of each partition, and then the results for all the +# partitions, using a given associative function and a neutral "zero value". +# +# @param x An RDD. +# @param zeroValue A neutral "zero value". +# @param op An associative function for the folding operation. +# @return The folding result. +# @rdname fold +# @seealso reduce +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) +# fold(rdd, 0, "+") # 15 +#} +# @rdname fold +# @aliases fold,RDD,RDD-method setMethod("fold", signature(x = "RDD", zeroValue = "ANY", op = "ANY"), function(x, zeroValue, op) { aggregateRDD(x, zeroValue, op, op) }) -#' Aggregate an RDD using the given combine functions and a neutral "zero value". -#' -#' Aggregate the elements of each partition, and then the results for all the -#' partitions, using given combine functions and a neutral "zero value". -#' -#' @param x An RDD. -#' @param zeroValue A neutral "zero value". -#' @param seqOp A function to aggregate the RDD elements. It may return a different -#' result type from the type of the RDD elements. -#' @param combOp A function to aggregate results of seqOp. -#' @return The aggregation result. -#' @rdname aggregateRDD -#' @seealso reduce -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3, 4)) -#' zeroValue <- list(0, 0) -#' seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } -#' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } -#' aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) -#'} -#' @rdname aggregateRDD -#' @aliases aggregateRDD,RDD,RDD-method +# Aggregate an RDD using the given combine functions and a neutral "zero value". +# +# Aggregate the elements of each partition, and then the results for all the +# partitions, using given combine functions and a neutral "zero value". +# +# @param x An RDD. +# @param zeroValue A neutral "zero value". +# @param seqOp A function to aggregate the RDD elements. It may return a different +# result type from the type of the RDD elements. +# @param combOp A function to aggregate results of seqOp. +# @return The aggregation result. +# @rdname aggregateRDD +# @seealso reduce +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(1, 2, 3, 4)) +# zeroValue <- list(0, 0) +# seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } +# combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } +# aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) +#} +# @rdname aggregateRDD +# @aliases aggregateRDD,RDD,RDD-method setMethod("aggregateRDD", signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", combOp = "ANY"), function(x, zeroValue, seqOp, combOp) { @@ -1240,25 +1240,25 @@ setMethod("aggregateRDD", Reduce(combOp, partitionList, zeroValue) }) -#' Pipes elements to a forked external process. -#' -#' The same as 'pipe()' in Spark. -#' -#' @param x The RDD whose elements are piped to the forked external process. -#' @param command The command to fork an external process. -#' @param env A named list to set environment variables of the external process. -#' @return A new RDD created by piping all elements to a forked external process. -#' @rdname pipeRDD -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' collect(pipeRDD(rdd, "more") -#' Output: c("1", "2", ..., "10") -#'} -#' @rdname pipeRDD -#' @aliases pipeRDD,RDD,character-method +# Pipes elements to a forked external process. +# +# The same as 'pipe()' in Spark. +# +# @param x The RDD whose elements are piped to the forked external process. +# @param command The command to fork an external process. +# @param env A named list to set environment variables of the external process. +# @return A new RDD created by piping all elements to a forked external process. +# @rdname pipeRDD +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# collect(pipeRDD(rdd, "more") +# Output: c("1", "2", ..., "10") +#} +# @rdname pipeRDD +# @aliases pipeRDD,RDD,character-method setMethod("pipeRDD", signature(x = "RDD", command = "character"), function(x, command, env = list()) { @@ -1274,41 +1274,41 @@ setMethod("pipeRDD", }) # TODO: Consider caching the name in the RDD's environment -#' Return an RDD's name. -#' -#' @param x The RDD whose name is returned. -#' @rdname name -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1,2,3)) -#' name(rdd) # NULL (if not set before) -#'} -#' @rdname name -#' @aliases name,RDD +# Return an RDD's name. +# +# @param x The RDD whose name is returned. +# @rdname name +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(1,2,3)) +# name(rdd) # NULL (if not set before) +#} +# @rdname name +# @aliases name,RDD setMethod("name", signature(x = "RDD"), function(x) { callJMethod(getJRDD(x), "name") }) -#' Set an RDD's name. -#' -#' @param x The RDD whose name is to be set. -#' @param name The RDD name to be set. -#' @return a new RDD renamed. -#' @rdname setName -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1,2,3)) -#' setName(rdd, "myRDD") -#' name(rdd) # "myRDD" -#'} -#' @rdname setName -#' @aliases setName,RDD +# Set an RDD's name. +# +# @param x The RDD whose name is to be set. +# @param name The RDD name to be set. +# @return a new RDD renamed. +# @rdname setName +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(1,2,3)) +# setName(rdd, "myRDD") +# name(rdd) # "myRDD" +#} +# @rdname setName +# @aliases setName,RDD setMethod("setName", signature(x = "RDD", name = "character"), function(x, name) { @@ -1316,25 +1316,25 @@ setMethod("setName", x }) -#' Zip an RDD with generated unique Long IDs. -#' -#' Items in the kth partition will get ids k, n+k, 2*n+k, ..., where -#' n is the number of partitions. So there may exist gaps, but this -#' method won't trigger a spark job, which is different from -#' zipWithIndex. -#' -#' @param x An RDD to be zipped. -#' @return An RDD with zipped items. -#' @seealso zipWithIndex -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) -#' collect(zipWithUniqueId(rdd)) -#' # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) -#'} -#' @rdname zipWithUniqueId -#' @aliases zipWithUniqueId,RDD +# Zip an RDD with generated unique Long IDs. +# +# Items in the kth partition will get ids k, n+k, 2*n+k, ..., where +# n is the number of partitions. So there may exist gaps, but this +# method won't trigger a spark job, which is different from +# zipWithIndex. +# +# @param x An RDD to be zipped. +# @return An RDD with zipped items. +# @seealso zipWithIndex +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +# collect(zipWithUniqueId(rdd)) +# # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) +#} +# @rdname zipWithUniqueId +# @aliases zipWithUniqueId,RDD setMethod("zipWithUniqueId", signature(x = "RDD"), function(x) { @@ -1353,28 +1353,28 @@ setMethod("zipWithUniqueId", lapplyPartitionsWithIndex(x, partitionFunc) }) -#' Zip an RDD with its element indices. -#' -#' The ordering is first based on the partition index and then the -#' ordering of items within each partition. So the first item in -#' the first partition gets index 0, and the last item in the last -#' partition receives the largest index. -#' -#' This method needs to trigger a Spark job when this RDD contains -#' more than one partition. -#' -#' @param x An RDD to be zipped. -#' @return An RDD with zipped items. -#' @seealso zipWithUniqueId -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) -#' collect(zipWithIndex(rdd)) -#' # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) -#'} -#' @rdname zipWithIndex -#' @aliases zipWithIndex,RDD +# Zip an RDD with its element indices. +# +# The ordering is first based on the partition index and then the +# ordering of items within each partition. So the first item in +# the first partition gets index 0, and the last item in the last +# partition receives the largest index. +# +# This method needs to trigger a Spark job when this RDD contains +# more than one partition. +# +# @param x An RDD to be zipped. +# @return An RDD with zipped items. +# @seealso zipWithUniqueId +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +# collect(zipWithIndex(rdd)) +# # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) +#} +# @rdname zipWithIndex +# @aliases zipWithIndex,RDD setMethod("zipWithIndex", signature(x = "RDD"), function(x) { @@ -1406,20 +1406,20 @@ setMethod("zipWithIndex", lapplyPartitionsWithIndex(x, partitionFunc) }) -#' Coalesce all elements within each partition of an RDD into a list. -#' -#' @param x An RDD. -#' @return An RDD created by coalescing all elements within -#' each partition into a list. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, as.list(1:4), 2L) -#' collect(glom(rdd)) -#' # list(list(1, 2), list(3, 4)) -#'} -#' @rdname glom -#' @aliases glom,RDD +# Coalesce all elements within each partition of an RDD into a list. +# +# @param x An RDD. +# @return An RDD created by coalescing all elements within +# each partition into a list. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, as.list(1:4), 2L) +# collect(glom(rdd)) +# # list(list(1, 2), list(3, 4)) +#} +# @rdname glom +# @aliases glom,RDD setMethod("glom", signature(x = "RDD"), function(x) { @@ -1432,21 +1432,21 @@ setMethod("glom", ############ Binary Functions ############# -#' Return the union RDD of two RDDs. -#' The same as union() in Spark. -#' -#' @param x An RDD. -#' @param y An RDD. -#' @return a new RDD created by performing the simple union (witout removing -#' duplicates) of two input RDDs. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:3) -#' unionRDD(rdd, rdd) # 1, 2, 3, 1, 2, 3 -#'} -#' @rdname unionRDD -#' @aliases unionRDD,RDD,RDD-method +# Return the union RDD of two RDDs. +# The same as union() in Spark. +# +# @param x An RDD. +# @param y An RDD. +# @return a new RDD created by performing the simple union (witout removing +# duplicates) of two input RDDs. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:3) +# unionRDD(rdd, rdd) # 1, 2, 3, 1, 2, 3 +#} +# @rdname unionRDD +# @aliases unionRDD,RDD,RDD-method setMethod("unionRDD", signature(x = "RDD", y = "RDD"), function(x, y) { @@ -1463,27 +1463,27 @@ setMethod("unionRDD", union.rdd }) -#' Zip an RDD with another RDD. -#' -#' Zips this RDD with another one, returning key-value pairs with the -#' first element in each RDD second element in each RDD, etc. Assumes -#' that the two RDDs have the same number of partitions and the same -#' number of elements in each partition (e.g. one was made through -#' a map on the other). -#' -#' @param x An RDD to be zipped. -#' @param other Another RDD to be zipped. -#' @return An RDD zipped from the two RDDs. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, 0:4) -#' rdd2 <- parallelize(sc, 1000:1004) -#' collect(zipRDD(rdd1, rdd2)) -#' # list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004)) -#'} -#' @rdname zipRDD -#' @aliases zipRDD,RDD +# Zip an RDD with another RDD. +# +# Zips this RDD with another one, returning key-value pairs with the +# first element in each RDD second element in each RDD, etc. Assumes +# that the two RDDs have the same number of partitions and the same +# number of elements in each partition (e.g. one was made through +# a map on the other). +# +# @param x An RDD to be zipped. +# @param other Another RDD to be zipped. +# @return An RDD zipped from the two RDDs. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, 0:4) +# rdd2 <- parallelize(sc, 1000:1004) +# collect(zipRDD(rdd1, rdd2)) +# # list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004)) +#} +# @rdname zipRDD +# @aliases zipRDD,RDD setMethod("zipRDD", signature(x = "RDD", other = "RDD"), function(x, other) { @@ -1502,24 +1502,24 @@ setMethod("zipRDD", mergePartitions(rdd, TRUE) }) -#' Cartesian product of this RDD and another one. -#' -#' Return the Cartesian product of this RDD and another one, -#' that is, the RDD of all pairs of elements (a, b) where a -#' is in this and b is in other. -#' -#' @param x An RDD. -#' @param other An RDD. -#' @return A new RDD which is the Cartesian product of these two RDDs. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:2) -#' sortByKey(cartesian(rdd, rdd)) -#' # list(list(1, 1), list(1, 2), list(2, 1), list(2, 2)) -#'} -#' @rdname cartesian -#' @aliases cartesian,RDD,RDD-method +# Cartesian product of this RDD and another one. +# +# Return the Cartesian product of this RDD and another one, +# that is, the RDD of all pairs of elements (a, b) where a +# is in this and b is in other. +# +# @param x An RDD. +# @param other An RDD. +# @return A new RDD which is the Cartesian product of these two RDDs. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:2) +# sortByKey(cartesian(rdd, rdd)) +# # list(list(1, 1), list(1, 2), list(2, 1), list(2, 2)) +#} +# @rdname cartesian +# @aliases cartesian,RDD,RDD-method setMethod("cartesian", signature(x = "RDD", other = "RDD"), function(x, other) { @@ -1532,24 +1532,24 @@ setMethod("cartesian", mergePartitions(rdd, FALSE) }) -#' Subtract an RDD with another RDD. -#' -#' Return an RDD with the elements from this that are not in other. -#' -#' @param x An RDD. -#' @param other An RDD. -#' @param numPartitions Number of the partitions in the result RDD. -#' @return An RDD with the elements from this that are not in other. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, list(1, 1, 2, 2, 3, 4)) -#' rdd2 <- parallelize(sc, list(2, 4)) -#' collect(subtract(rdd1, rdd2)) -#' # list(1, 1, 3) -#'} -#' @rdname subtract -#' @aliases subtract,RDD +# Subtract an RDD with another RDD. +# +# Return an RDD with the elements from this that are not in other. +# +# @param x An RDD. +# @param other An RDD. +# @param numPartitions Number of the partitions in the result RDD. +# @return An RDD with the elements from this that are not in other. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, list(1, 1, 2, 2, 3, 4)) +# rdd2 <- parallelize(sc, list(2, 4)) +# collect(subtract(rdd1, rdd2)) +# # list(1, 1, 3) +#} +# @rdname subtract +# @aliases subtract,RDD setMethod("subtract", signature(x = "RDD", other = "RDD"), function(x, other, numPartitions = SparkR:::numPartitions(x)) { @@ -1559,28 +1559,28 @@ setMethod("subtract", keys(subtractByKey(rdd1, rdd2, numPartitions)) }) -#' Intersection of this RDD and another one. -#' -#' Return the intersection of this RDD and another one. -#' The output will not contain any duplicate elements, -#' even if the input RDDs did. Performs a hash partition -#' across the cluster. -#' Note that this method performs a shuffle internally. -#' -#' @param x An RDD. -#' @param other An RDD. -#' @param numPartitions The number of partitions in the result RDD. -#' @return An RDD which is the intersection of these two RDDs. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) -#' rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) -#' collect(sortBy(intersection(rdd1, rdd2), function(x) { x })) -#' # list(1, 2, 3) -#'} -#' @rdname intersection -#' @aliases intersection,RDD +# Intersection of this RDD and another one. +# +# Return the intersection of this RDD and another one. +# The output will not contain any duplicate elements, +# even if the input RDDs did. Performs a hash partition +# across the cluster. +# Note that this method performs a shuffle internally. +# +# @param x An RDD. +# @param other An RDD. +# @param numPartitions The number of partitions in the result RDD. +# @return An RDD which is the intersection of these two RDDs. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) +# rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) +# collect(sortBy(intersection(rdd1, rdd2), function(x) { x })) +# # list(1, 2, 3) +#} +# @rdname intersection +# @aliases intersection,RDD setMethod("intersection", signature(x = "RDD", other = "RDD"), function(x, other, numPartitions = SparkR:::numPartitions(x)) { @@ -1596,26 +1596,26 @@ setMethod("intersection", keys(filterRDD(cogroup(rdd1, rdd2, numPartitions = numPartitions), filterFunction)) }) -#' Zips an RDD's partitions with one (or more) RDD(s). -#' Same as zipPartitions in Spark. -#' -#' @param ... RDDs to be zipped. -#' @param func A function to transform zipped partitions. -#' @return A new RDD by applying a function to the zipped partitions. -#' Assumes that all the RDDs have the *same number of partitions*, but -#' does *not* require them to have the same number of elements in each partition. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, 1:2, 2L) # 1, 2 -#' rdd2 <- parallelize(sc, 1:4, 2L) # 1:2, 3:4 -#' rdd3 <- parallelize(sc, 1:6, 2L) # 1:3, 4:6 -#' collect(zipPartitions(rdd1, rdd2, rdd3, -#' func = function(x, y, z) { list(list(x, y, z))} )) -#' # list(list(1, c(1,2), c(1,2,3)), list(2, c(3,4), c(4,5,6))) -#'} -#' @rdname zipRDD -#' @aliases zipPartitions,RDD +# Zips an RDD's partitions with one (or more) RDD(s). +# Same as zipPartitions in Spark. +# +# @param ... RDDs to be zipped. +# @param func A function to transform zipped partitions. +# @return A new RDD by applying a function to the zipped partitions. +# Assumes that all the RDDs have the *same number of partitions*, but +# does *not* require them to have the same number of elements in each partition. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, 1:2, 2L) # 1, 2 +# rdd2 <- parallelize(sc, 1:4, 2L) # 1:2, 3:4 +# rdd3 <- parallelize(sc, 1:6, 2L) # 1:3, 4:6 +# collect(zipPartitions(rdd1, rdd2, rdd3, +# func = function(x, y, z) { list(list(x, y, z))} )) +# # list(list(1, c(1,2), c(1,2,3)), list(2, c(3,4), c(4,5,6))) +#} +# @rdname zipRDD +# @aliases zipPartitions,RDD setMethod("zipPartitions", "RDD", function(..., func) { diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 4f05ba524a01a..cae06e6af2bff 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -150,21 +150,21 @@ createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) { dataFrame(sdf) } -#' toDF -#' -#' Converts an RDD to a DataFrame by infer the types. -#' -#' @param x An RDD -#' -#' @rdname DataFrame -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) -#' rdd <- lapply(parallelize(sc, 1:10), function(x) list(a=x, b=as.character(x))) -#' df <- toDF(rdd) -#' } +# toDF +# +# Converts an RDD to a DataFrame by infer the types. +# +# @param x An RDD +# +# @rdname DataFrame +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# sqlCtx <- sparkRSQL.init(sc) +# rdd <- lapply(parallelize(sc, 1:10), function(x) list(a=x, b=as.character(x))) +# df <- toDF(rdd) +# } setGeneric("toDF", function(x, ...) { standardGeneric("toDF") }) @@ -207,23 +207,23 @@ jsonFile <- function(sqlCtx, path) { } -#' JSON RDD -#' -#' Loads an RDD storing one JSON object per string as a DataFrame. -#' -#' @param sqlCtx SQLContext to use -#' @param rdd An RDD of JSON string -#' @param schema A StructType object to use as schema -#' @param samplingRatio The ratio of simpling used to infer the schema -#' @return A DataFrame -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) -#' rdd <- texFile(sc, "path/to/json") -#' df <- jsonRDD(sqlCtx, rdd) -#' } +# JSON RDD +# +# Loads an RDD storing one JSON object per string as a DataFrame. +# +# @param sqlCtx SQLContext to use +# @param rdd An RDD of JSON string +# @param schema A StructType object to use as schema +# @param samplingRatio The ratio of simpling used to infer the schema +# @return A DataFrame +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# sqlCtx <- sparkRSQL.init(sc) +# rdd <- texFile(sc, "path/to/json") +# df <- jsonRDD(sqlCtx, rdd) +# } # TODO: support schema jsonRDD <- function(sqlCtx, rdd, schema = NULL, samplingRatio = 1.0) { diff --git a/R/pkg/R/broadcast.R b/R/pkg/R/broadcast.R index 583fa2e7fdcfd..23dc38780716e 100644 --- a/R/pkg/R/broadcast.R +++ b/R/pkg/R/broadcast.R @@ -23,21 +23,21 @@ .broadcastValues <- new.env() .broadcastIdToName <- new.env() -#' @title S4 class that represents a Broadcast variable -#' @description Broadcast variables can be created using the broadcast -#' function from a \code{SparkContext}. -#' @rdname broadcast-class -#' @seealso broadcast -#' -#' @param id Id of the backing Spark broadcast variable -#' @export +# @title S4 class that represents a Broadcast variable +# @description Broadcast variables can be created using the broadcast +# function from a \code{SparkContext}. +# @rdname broadcast-class +# @seealso broadcast +# +# @param id Id of the backing Spark broadcast variable +# @export setClass("Broadcast", slots = list(id = "character")) -#' @rdname broadcast-class -#' @param value Value of the broadcast variable -#' @param jBroadcastRef reference to the backing Java broadcast object -#' @param objName name of broadcasted object -#' @export +# @rdname broadcast-class +# @param value Value of the broadcast variable +# @param jBroadcastRef reference to the backing Java broadcast object +# @param objName name of broadcasted object +# @export Broadcast <- function(id, value, jBroadcastRef, objName) { .broadcastValues[[id]] <- value .broadcastNames[[as.character(objName)]] <- jBroadcastRef @@ -45,13 +45,13 @@ Broadcast <- function(id, value, jBroadcastRef, objName) { new("Broadcast", id = id) } -#' @description -#' \code{value} can be used to get the value of a broadcast variable inside -#' a distributed function. -#' -#' @param bcast The broadcast variable to get -#' @rdname broadcast -#' @aliases value,Broadcast-method +# @description +# \code{value} can be used to get the value of a broadcast variable inside +# a distributed function. +# +# @param bcast The broadcast variable to get +# @rdname broadcast +# @aliases value,Broadcast-method setMethod("value", signature(bcast = "Broadcast"), function(bcast) { @@ -62,24 +62,24 @@ setMethod("value", } }) -#' Internal function to set values of a broadcast variable. -#' -#' This function is used internally by Spark to set the value of a broadcast -#' variable on workers. Not intended for use outside the package. -#' -#' @rdname broadcast-internal -#' @seealso broadcast, value +# Internal function to set values of a broadcast variable. +# +# This function is used internally by Spark to set the value of a broadcast +# variable on workers. Not intended for use outside the package. +# +# @rdname broadcast-internal +# @seealso broadcast, value -#' @param bcastId The id of broadcast variable to set -#' @param value The value to be set -#' @export +# @param bcastId The id of broadcast variable to set +# @param value The value to be set +# @export setBroadcastValue <- function(bcastId, value) { bcastIdStr <- as.character(bcastId) .broadcastValues[[bcastIdStr]] <- value } -#' Helper function to clear the list of broadcast variables we know about -#' Should be called when the SparkR JVM backend is shutdown +# Helper function to clear the list of broadcast variables we know about +# Should be called when the SparkR JVM backend is shutdown clearBroadcastVariables <- function() { bcasts <- ls(.broadcastNames) rm(list = bcasts, envir = .broadcastNames) diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index b4845b6948997..43be9c904fdf6 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -25,27 +25,27 @@ getMinPartitions <- function(sc, minPartitions) { as.integer(minPartitions) } -#' Create an RDD from a text file. -#' -#' This function reads a text file from HDFS, a local file system (available on all -#' nodes), or any Hadoop-supported file system URI, and creates an -#' RDD of strings from it. -#' -#' @param sc SparkContext to use -#' @param path Path of file to read. A vector of multiple paths is allowed. -#' @param minPartitions Minimum number of partitions to be created. If NULL, the default -#' value is chosen based on available parallelism. -#' @return RDD where each item is of type \code{character} -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' lines <- textFile(sc, "myfile.txt") -#'} +# Create an RDD from a text file. +# +# This function reads a text file from HDFS, a local file system (available on all +# nodes), or any Hadoop-supported file system URI, and creates an +# RDD of strings from it. +# +# @param sc SparkContext to use +# @param path Path of file to read. A vector of multiple paths is allowed. +# @param minPartitions Minimum number of partitions to be created. If NULL, the default +# value is chosen based on available parallelism. +# @return RDD where each item is of type \code{character} +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# lines <- textFile(sc, "myfile.txt") +#} textFile <- function(sc, path, minPartitions = NULL) { # Allow the user to have a more flexible definiton of the text file path path <- suppressWarnings(normalizePath(path)) - #' Convert a string vector of paths to a string containing comma separated paths + # Convert a string vector of paths to a string containing comma separated paths path <- paste(path, collapse = ",") jrdd <- callJMethod(sc, "textFile", path, getMinPartitions(sc, minPartitions)) @@ -53,27 +53,27 @@ textFile <- function(sc, path, minPartitions = NULL) { RDD(jrdd, "string") } -#' Load an RDD saved as a SequenceFile containing serialized objects. -#' -#' The file to be loaded should be one that was previously generated by calling -#' saveAsObjectFile() of the RDD class. -#' -#' @param sc SparkContext to use -#' @param path Path of file to read. A vector of multiple paths is allowed. -#' @param minPartitions Minimum number of partitions to be created. If NULL, the default -#' value is chosen based on available parallelism. -#' @return RDD containing serialized R objects. -#' @seealso saveAsObjectFile -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- objectFile(sc, "myfile") -#'} +# Load an RDD saved as a SequenceFile containing serialized objects. +# +# The file to be loaded should be one that was previously generated by calling +# saveAsObjectFile() of the RDD class. +# +# @param sc SparkContext to use +# @param path Path of file to read. A vector of multiple paths is allowed. +# @param minPartitions Minimum number of partitions to be created. If NULL, the default +# value is chosen based on available parallelism. +# @return RDD containing serialized R objects. +# @seealso saveAsObjectFile +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- objectFile(sc, "myfile") +#} objectFile <- function(sc, path, minPartitions = NULL) { # Allow the user to have a more flexible definiton of the text file path path <- suppressWarnings(normalizePath(path)) - #' Convert a string vector of paths to a string containing comma separated paths + # Convert a string vector of paths to a string containing comma separated paths path <- paste(path, collapse = ",") jrdd <- callJMethod(sc, "objectFile", path, getMinPartitions(sc, minPartitions)) @@ -81,24 +81,24 @@ objectFile <- function(sc, path, minPartitions = NULL) { RDD(jrdd, "byte") } -#' Create an RDD from a homogeneous list or vector. -#' -#' This function creates an RDD from a local homogeneous list in R. The elements -#' in the list are split into \code{numSlices} slices and distributed to nodes -#' in the cluster. -#' -#' @param sc SparkContext to use -#' @param coll collection to parallelize -#' @param numSlices number of partitions to create in the RDD -#' @return an RDD created from this collection -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2) -#' # The RDD should contain 10 elements -#' length(rdd) -#'} +# Create an RDD from a homogeneous list or vector. +# +# This function creates an RDD from a local homogeneous list in R. The elements +# in the list are split into \code{numSlices} slices and distributed to nodes +# in the cluster. +# +# @param sc SparkContext to use +# @param coll collection to parallelize +# @param numSlices number of partitions to create in the RDD +# @return an RDD created from this collection +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10, 2) +# # The RDD should contain 10 elements +# length(rdd) +#} parallelize <- function(sc, coll, numSlices = 1) { # TODO: bound/safeguard numSlices # TODO: unit tests for if the split works for all primitives @@ -133,33 +133,33 @@ parallelize <- function(sc, coll, numSlices = 1) { RDD(jrdd, "byte") } -#' Include this specified package on all workers -#' -#' This function can be used to include a package on all workers before the -#' user's code is executed. This is useful in scenarios where other R package -#' functions are used in a function passed to functions like \code{lapply}. -#' NOTE: The package is assumed to be installed on every node in the Spark -#' cluster. -#' -#' @param sc SparkContext to use -#' @param pkg Package name -#' -#' @export -#' @examples -#'\dontrun{ -#' library(Matrix) -#' -#' sc <- sparkR.init() -#' # Include the matrix library we will be using -#' includePackage(sc, Matrix) -#' -#' generateSparse <- function(x) { -#' sparseMatrix(i=c(1, 2, 3), j=c(1, 2, 3), x=c(1, 2, 3)) -#' } -#' -#' rdd <- lapplyPartition(parallelize(sc, 1:2, 2L), generateSparse) -#' collect(rdd) -#'} +# Include this specified package on all workers +# +# This function can be used to include a package on all workers before the +# user's code is executed. This is useful in scenarios where other R package +# functions are used in a function passed to functions like \code{lapply}. +# NOTE: The package is assumed to be installed on every node in the Spark +# cluster. +# +# @param sc SparkContext to use +# @param pkg Package name +# +# @export +# @examples +#\dontrun{ +# library(Matrix) +# +# sc <- sparkR.init() +# # Include the matrix library we will be using +# includePackage(sc, Matrix) +# +# generateSparse <- function(x) { +# sparseMatrix(i=c(1, 2, 3), j=c(1, 2, 3), x=c(1, 2, 3)) +# } +# +# rdd <- lapplyPartition(parallelize(sc, 1:2, 2L), generateSparse) +# collect(rdd) +#} includePackage <- function(sc, pkg) { pkg <- as.character(substitute(pkg)) if (exists(".packages", .sparkREnv)) { @@ -171,30 +171,30 @@ includePackage <- function(sc, pkg) { .sparkREnv$.packages <- packages } -#' @title Broadcast a variable to all workers -#' -#' @description -#' Broadcast a read-only variable to the cluster, returning a \code{Broadcast} -#' object for reading it in distributed functions. -#' -#' @param sc Spark Context to use -#' @param object Object to be broadcast -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:2, 2L) -#' -#' # Large Matrix object that we want to broadcast -#' randomMat <- matrix(nrow=100, ncol=10, data=rnorm(1000)) -#' randomMatBr <- broadcast(sc, randomMat) -#' -#' # Use the broadcast variable inside the function -#' useBroadcast <- function(x) { -#' sum(value(randomMatBr) * x) -#' } -#' sumRDD <- lapply(rdd, useBroadcast) -#'} +# @title Broadcast a variable to all workers +# +# @description +# Broadcast a read-only variable to the cluster, returning a \code{Broadcast} +# object for reading it in distributed functions. +# +# @param sc Spark Context to use +# @param object Object to be broadcast +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:2, 2L) +# +# # Large Matrix object that we want to broadcast +# randomMat <- matrix(nrow=100, ncol=10, data=rnorm(1000)) +# randomMatBr <- broadcast(sc, randomMat) +# +# # Use the broadcast variable inside the function +# useBroadcast <- function(x) { +# sum(value(randomMatBr) * x) +# } +# sumRDD <- lapply(rdd, useBroadcast) +#} broadcast <- function(sc, object) { objName <- as.character(substitute(object)) serializedObj <- serialize(object, connection = NULL) @@ -205,21 +205,21 @@ broadcast <- function(sc, object) { Broadcast(id, object, jBroadcast, objName) } -#' @title Set the checkpoint directory -#' -#' Set the directory under which RDDs are going to be checkpointed. The -#' directory must be a HDFS path if running on a cluster. -#' -#' @param sc Spark Context to use -#' @param dirName Directory path -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' setCheckpointDir(sc, "~/checkpoint") -#' rdd <- parallelize(sc, 1:2, 2L) -#' checkpoint(rdd) -#'} +# @title Set the checkpoint directory +# +# Set the directory under which RDDs are going to be checkpointed. The +# directory must be a HDFS path if running on a cluster. +# +# @param sc Spark Context to use +# @param dirName Directory path +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# setCheckpointDir(sc, "~/checkpoint") +# rdd <- parallelize(sc, 1:2, 2L) +# checkpoint(rdd) +#} setCheckpointDir <- function(sc, dirName) { invisible(callJMethod(sc, "setCheckpointDir", suppressWarnings(normalizePath(dirName)))) } diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index e88729387ef95..380e8ebe8c8f4 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -17,353 +17,353 @@ ############ RDD Actions and Transformations ############ -#' @rdname aggregateRDD -#' @seealso reduce -#' @export +# @rdname aggregateRDD +# @seealso reduce +# @export setGeneric("aggregateRDD", function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) -#' @rdname cache-methods -#' @export +# @rdname cache-methods +# @export setGeneric("cache", function(x) { standardGeneric("cache") }) -#' @rdname coalesce -#' @seealso repartition -#' @export +# @rdname coalesce +# @seealso repartition +# @export setGeneric("coalesce", function(x, numPartitions, ...) { standardGeneric("coalesce") }) -#' @rdname checkpoint-methods -#' @export +# @rdname checkpoint-methods +# @export setGeneric("checkpoint", function(x) { standardGeneric("checkpoint") }) -#' @rdname collect-methods -#' @export +# @rdname collect-methods +# @export setGeneric("collect", function(x, ...) { standardGeneric("collect") }) -#' @rdname collect-methods -#' @export +# @rdname collect-methods +# @export setGeneric("collectAsMap", function(x) { standardGeneric("collectAsMap") }) -#' @rdname collect-methods -#' @export +# @rdname collect-methods +# @export setGeneric("collectPartition", function(x, partitionId) { standardGeneric("collectPartition") }) -#' @rdname count -#' @export +# @rdname count +# @export setGeneric("count", function(x) { standardGeneric("count") }) -#' @rdname countByValue -#' @export +# @rdname countByValue +# @export setGeneric("countByValue", function(x) { standardGeneric("countByValue") }) -#' @rdname distinct -#' @export +# @rdname distinct +# @export setGeneric("distinct", function(x, numPartitions = 1) { standardGeneric("distinct") }) -#' @rdname filterRDD -#' @export +# @rdname filterRDD +# @export setGeneric("filterRDD", function(x, f) { standardGeneric("filterRDD") }) -#' @rdname first -#' @export +# @rdname first +# @export setGeneric("first", function(x) { standardGeneric("first") }) -#' @rdname flatMap -#' @export +# @rdname flatMap +# @export setGeneric("flatMap", function(X, FUN) { standardGeneric("flatMap") }) -#' @rdname fold -#' @seealso reduce -#' @export +# @rdname fold +# @seealso reduce +# @export setGeneric("fold", function(x, zeroValue, op) { standardGeneric("fold") }) -#' @rdname foreach -#' @export +# @rdname foreach +# @export setGeneric("foreach", function(x, func) { standardGeneric("foreach") }) -#' @rdname foreach -#' @export +# @rdname foreach +# @export setGeneric("foreachPartition", function(x, func) { standardGeneric("foreachPartition") }) # The jrdd accessor function. setGeneric("getJRDD", function(rdd, ...) { standardGeneric("getJRDD") }) -#' @rdname glom -#' @export +# @rdname glom +# @export setGeneric("glom", function(x) { standardGeneric("glom") }) -#' @rdname keyBy -#' @export +# @rdname keyBy +# @export setGeneric("keyBy", function(x, func) { standardGeneric("keyBy") }) -#' @rdname lapplyPartition -#' @export +# @rdname lapplyPartition +# @export setGeneric("lapplyPartition", function(X, FUN) { standardGeneric("lapplyPartition") }) -#' @rdname lapplyPartitionsWithIndex -#' @export +# @rdname lapplyPartitionsWithIndex +# @export setGeneric("lapplyPartitionsWithIndex", function(X, FUN) { standardGeneric("lapplyPartitionsWithIndex") }) -#' @rdname lapply -#' @export +# @rdname lapply +# @export setGeneric("map", function(X, FUN) { standardGeneric("map") }) -#' @rdname lapplyPartition -#' @export +# @rdname lapplyPartition +# @export setGeneric("mapPartitions", function(X, FUN) { standardGeneric("mapPartitions") }) -#' @rdname lapplyPartitionsWithIndex -#' @export +# @rdname lapplyPartitionsWithIndex +# @export setGeneric("mapPartitionsWithIndex", function(X, FUN) { standardGeneric("mapPartitionsWithIndex") }) -#' @rdname maximum -#' @export +# @rdname maximum +# @export setGeneric("maximum", function(x) { standardGeneric("maximum") }) -#' @rdname minimum -#' @export +# @rdname minimum +# @export setGeneric("minimum", function(x) { standardGeneric("minimum") }) -#' @rdname sumRDD -#' @export +# @rdname sumRDD +# @export setGeneric("sumRDD", function(x) { standardGeneric("sumRDD") }) -#' @rdname name -#' @export +# @rdname name +# @export setGeneric("name", function(x) { standardGeneric("name") }) -#' @rdname numPartitions -#' @export +# @rdname numPartitions +# @export setGeneric("numPartitions", function(x) { standardGeneric("numPartitions") }) -#' @rdname persist -#' @export +# @rdname persist +# @export setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) -#' @rdname pipeRDD -#' @export +# @rdname pipeRDD +# @export setGeneric("pipeRDD", function(x, command, env = list()) { standardGeneric("pipeRDD")}) -#' @rdname reduce -#' @export +# @rdname reduce +# @export setGeneric("reduce", function(x, func) { standardGeneric("reduce") }) -#' @rdname repartition -#' @seealso coalesce -#' @export +# @rdname repartition +# @seealso coalesce +# @export setGeneric("repartition", function(x, numPartitions) { standardGeneric("repartition") }) -#' @rdname sampleRDD -#' @export +# @rdname sampleRDD +# @export setGeneric("sampleRDD", function(x, withReplacement, fraction, seed) { standardGeneric("sampleRDD") }) -#' @rdname saveAsObjectFile -#' @seealso objectFile -#' @export +# @rdname saveAsObjectFile +# @seealso objectFile +# @export setGeneric("saveAsObjectFile", function(x, path) { standardGeneric("saveAsObjectFile") }) -#' @rdname saveAsTextFile -#' @export +# @rdname saveAsTextFile +# @export setGeneric("saveAsTextFile", function(x, path) { standardGeneric("saveAsTextFile") }) -#' @rdname setName -#' @export +# @rdname setName +# @export setGeneric("setName", function(x, name) { standardGeneric("setName") }) -#' @rdname sortBy -#' @export +# @rdname sortBy +# @export setGeneric("sortBy", function(x, func, ascending = TRUE, numPartitions = 1) { standardGeneric("sortBy") }) -#' @rdname take -#' @export +# @rdname take +# @export setGeneric("take", function(x, num) { standardGeneric("take") }) -#' @rdname takeOrdered -#' @export +# @rdname takeOrdered +# @export setGeneric("takeOrdered", function(x, num) { standardGeneric("takeOrdered") }) -#' @rdname takeSample -#' @export +# @rdname takeSample +# @export setGeneric("takeSample", function(x, withReplacement, num, seed) { standardGeneric("takeSample") }) -#' @rdname top -#' @export +# @rdname top +# @export setGeneric("top", function(x, num) { standardGeneric("top") }) -#' @rdname unionRDD -#' @export +# @rdname unionRDD +# @export setGeneric("unionRDD", function(x, y) { standardGeneric("unionRDD") }) -#' @rdname unpersist-methods -#' @export +# @rdname unpersist-methods +# @export setGeneric("unpersist", function(x, ...) { standardGeneric("unpersist") }) -#' @rdname zipRDD -#' @export +# @rdname zipRDD +# @export setGeneric("zipRDD", function(x, other) { standardGeneric("zipRDD") }) -#' @rdname zipRDD -#' @export +# @rdname zipRDD +# @export setGeneric("zipPartitions", function(..., func) { standardGeneric("zipPartitions") }, signature = "...") -#' @rdname zipWithIndex -#' @seealso zipWithUniqueId -#' @export +# @rdname zipWithIndex +# @seealso zipWithUniqueId +# @export setGeneric("zipWithIndex", function(x) { standardGeneric("zipWithIndex") }) -#' @rdname zipWithUniqueId -#' @seealso zipWithIndex -#' @export +# @rdname zipWithUniqueId +# @seealso zipWithIndex +# @export setGeneric("zipWithUniqueId", function(x) { standardGeneric("zipWithUniqueId") }) ############ Binary Functions ############# -#' @rdname cartesian -#' @export +# @rdname cartesian +# @export setGeneric("cartesian", function(x, other) { standardGeneric("cartesian") }) -#' @rdname countByKey -#' @export +# @rdname countByKey +# @export setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) -#' @rdname flatMapValues -#' @export +# @rdname flatMapValues +# @export setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") }) -#' @rdname intersection -#' @export +# @rdname intersection +# @export setGeneric("intersection", function(x, other, numPartitions = 1) { standardGeneric("intersection") }) -#' @rdname keys -#' @export +# @rdname keys +# @export setGeneric("keys", function(x) { standardGeneric("keys") }) -#' @rdname lookup -#' @export +# @rdname lookup +# @export setGeneric("lookup", function(x, key) { standardGeneric("lookup") }) -#' @rdname mapValues -#' @export +# @rdname mapValues +# @export setGeneric("mapValues", function(X, FUN) { standardGeneric("mapValues") }) -#' @rdname sampleByKey -#' @export +# @rdname sampleByKey +# @export setGeneric("sampleByKey", function(x, withReplacement, fractions, seed) { standardGeneric("sampleByKey") }) -#' @rdname values -#' @export +# @rdname values +# @export setGeneric("values", function(x) { standardGeneric("values") }) ############ Shuffle Functions ############ -#' @rdname aggregateByKey -#' @seealso foldByKey, combineByKey -#' @export +# @rdname aggregateByKey +# @seealso foldByKey, combineByKey +# @export setGeneric("aggregateByKey", function(x, zeroValue, seqOp, combOp, numPartitions) { standardGeneric("aggregateByKey") }) -#' @rdname cogroup -#' @export +# @rdname cogroup +# @export setGeneric("cogroup", function(..., numPartitions) { standardGeneric("cogroup") }, signature = "...") -#' @rdname combineByKey -#' @seealso groupByKey, reduceByKey -#' @export +# @rdname combineByKey +# @seealso groupByKey, reduceByKey +# @export setGeneric("combineByKey", function(x, createCombiner, mergeValue, mergeCombiners, numPartitions) { standardGeneric("combineByKey") }) -#' @rdname foldByKey -#' @seealso aggregateByKey, combineByKey -#' @export +# @rdname foldByKey +# @seealso aggregateByKey, combineByKey +# @export setGeneric("foldByKey", function(x, zeroValue, func, numPartitions) { standardGeneric("foldByKey") }) -#' @rdname join-methods -#' @export +# @rdname join-methods +# @export setGeneric("fullOuterJoin", function(x, y, numPartitions) { standardGeneric("fullOuterJoin") }) -#' @rdname groupByKey -#' @seealso reduceByKey -#' @export +# @rdname groupByKey +# @seealso reduceByKey +# @export setGeneric("groupByKey", function(x, numPartitions) { standardGeneric("groupByKey") }) -#' @rdname join-methods -#' @export +# @rdname join-methods +# @export setGeneric("join", function(x, y, ...) { standardGeneric("join") }) -#' @rdname join-methods -#' @export +# @rdname join-methods +# @export setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") }) -#' @rdname partitionBy -#' @export +# @rdname partitionBy +# @export setGeneric("partitionBy", function(x, numPartitions, ...) { standardGeneric("partitionBy") }) -#' @rdname reduceByKey -#' @seealso groupByKey -#' @export +# @rdname reduceByKey +# @seealso groupByKey +# @export setGeneric("reduceByKey", function(x, combineFunc, numPartitions) { standardGeneric("reduceByKey")}) -#' @rdname reduceByKeyLocally -#' @seealso reduceByKey -#' @export +# @rdname reduceByKeyLocally +# @seealso reduceByKey +# @export setGeneric("reduceByKeyLocally", function(x, combineFunc) { standardGeneric("reduceByKeyLocally") }) -#' @rdname join-methods -#' @export +# @rdname join-methods +# @export setGeneric("rightOuterJoin", function(x, y, numPartitions) { standardGeneric("rightOuterJoin") }) -#' @rdname sortByKey -#' @export +# @rdname sortByKey +# @export setGeneric("sortByKey", function(x, ascending = TRUE, numPartitions = 1) { standardGeneric("sortByKey") }) -#' @rdname subtract -#' @export +# @rdname subtract +# @export setGeneric("subtract", function(x, other, numPartitions = 1) { standardGeneric("subtract") }) -#' @rdname subtractByKey -#' @export +# @rdname subtractByKey +# @export setGeneric("subtractByKey", function(x, other, numPartitions = 1) { standardGeneric("subtractByKey") @@ -372,8 +372,8 @@ setGeneric("subtractByKey", ################### Broadcast Variable Methods ################# -#' @rdname broadcast -#' @export +# @rdname broadcast +# @export setGeneric("value", function(bcast) { standardGeneric("value") }) @@ -384,6 +384,10 @@ setGeneric("value", function(bcast) { standardGeneric("value") }) #' @export setGeneric("columns", function(x) {standardGeneric("columns") }) +#' @rdname describe +#' @export +setGeneric("describe", function(x, col, ...) { standardGeneric("describe") }) + #' @rdname schema #' @export setGeneric("dtypes", function(x) { standardGeneric("dtypes") }) @@ -473,8 +477,8 @@ setGeneric("showDF", function(x,...) { standardGeneric("showDF") }) #' @export setGeneric("sortDF", function(x, col, ...) { standardGeneric("sortDF") }) -#' @rdname tojson -#' @export +# @rdname tojson +# @export setGeneric("toJSON", function(x) { standardGeneric("toJSON") }) #' @rdname DataFrame diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index edeb8d9f75f7b..7694652856da5 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -21,23 +21,23 @@ NULL ############ Actions and Transformations ############ -#' Look up elements of a key in an RDD -#' -#' @description -#' \code{lookup} returns a list of values in this RDD for key key. -#' -#' @param x The RDD to collect -#' @param key The key to look up for -#' @return a list of values in this RDD for key key -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' pairs <- list(c(1, 1), c(2, 2), c(1, 3)) -#' rdd <- parallelize(sc, pairs) -#' lookup(rdd, 1) # list(1, 3) -#'} -#' @rdname lookup -#' @aliases lookup,RDD-method +# Look up elements of a key in an RDD +# +# @description +# \code{lookup} returns a list of values in this RDD for key key. +# +# @param x The RDD to collect +# @param key The key to look up for +# @return a list of values in this RDD for key key +# @examples +#\dontrun{ +# sc <- sparkR.init() +# pairs <- list(c(1, 1), c(2, 2), c(1, 3)) +# rdd <- parallelize(sc, pairs) +# lookup(rdd, 1) # list(1, 3) +#} +# @rdname lookup +# @aliases lookup,RDD-method setMethod("lookup", signature(x = "RDD", key = "ANY"), function(x, key) { @@ -49,21 +49,21 @@ setMethod("lookup", collect(valsRDD) }) -#' Count the number of elements for each key, and return the result to the -#' master as lists of (key, count) pairs. -#' -#' Same as countByKey in Spark. -#' -#' @param x The RDD to count keys. -#' @return list of (key, count) pairs, where count is number of each key in rdd. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) -#' countByKey(rdd) # ("a", 2L), ("b", 1L) -#'} -#' @rdname countByKey -#' @aliases countByKey,RDD-method +# Count the number of elements for each key, and return the result to the +# master as lists of (key, count) pairs. +# +# Same as countByKey in Spark. +# +# @param x The RDD to count keys. +# @return list of (key, count) pairs, where count is number of each key in rdd. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) +# countByKey(rdd) # ("a", 2L), ("b", 1L) +#} +# @rdname countByKey +# @aliases countByKey,RDD-method setMethod("countByKey", signature(x = "RDD"), function(x) { @@ -71,17 +71,17 @@ setMethod("countByKey", countByValue(keys) }) -#' Return an RDD with the keys of each tuple. -#' -#' @param x The RDD from which the keys of each tuple is returned. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) -#' collect(keys(rdd)) # list(1, 3) -#'} -#' @rdname keys -#' @aliases keys,RDD +# Return an RDD with the keys of each tuple. +# +# @param x The RDD from which the keys of each tuple is returned. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +# collect(keys(rdd)) # list(1, 3) +#} +# @rdname keys +# @aliases keys,RDD setMethod("keys", signature(x = "RDD"), function(x) { @@ -91,17 +91,17 @@ setMethod("keys", lapply(x, func) }) -#' Return an RDD with the values of each tuple. -#' -#' @param x The RDD from which the values of each tuple is returned. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) -#' collect(values(rdd)) # list(2, 4) -#'} -#' @rdname values -#' @aliases values,RDD +# Return an RDD with the values of each tuple. +# +# @param x The RDD from which the values of each tuple is returned. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +# collect(values(rdd)) # list(2, 4) +#} +# @rdname values +# @aliases values,RDD setMethod("values", signature(x = "RDD"), function(x) { @@ -111,23 +111,23 @@ setMethod("values", lapply(x, func) }) -#' Applies a function to all values of the elements, without modifying the keys. -#' -#' The same as `mapValues()' in Spark. -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on the value of each element. -#' @return a new RDD created by the transformation. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' makePairs <- lapply(rdd, function(x) { list(x, x) }) -#' collect(mapValues(makePairs, function(x) { x * 2) }) -#' Output: list(list(1,2), list(2,4), list(3,6), ...) -#'} -#' @rdname mapValues -#' @aliases mapValues,RDD,function-method +# Applies a function to all values of the elements, without modifying the keys. +# +# The same as `mapValues()' in Spark. +# +# @param X The RDD to apply the transformation. +# @param FUN the transformation to apply on the value of each element. +# @return a new RDD created by the transformation. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# makePairs <- lapply(rdd, function(x) { list(x, x) }) +# collect(mapValues(makePairs, function(x) { x * 2) }) +# Output: list(list(1,2), list(2,4), list(3,6), ...) +#} +# @rdname mapValues +# @aliases mapValues,RDD,function-method setMethod("mapValues", signature(X = "RDD", FUN = "function"), function(X, FUN) { @@ -137,23 +137,23 @@ setMethod("mapValues", lapply(X, func) }) -#' Pass each value in the key-value pair RDD through a flatMap function without -#' changing the keys; this also retains the original RDD's partitioning. -#' -#' The same as 'flatMapValues()' in Spark. -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on the value of each element. -#' @return a new RDD created by the transformation. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) -#' collect(flatMapValues(rdd, function(x) { x })) -#' Output: list(list(1,1), list(1,2), list(2,3), list(2,4)) -#'} -#' @rdname flatMapValues -#' @aliases flatMapValues,RDD,function-method +# Pass each value in the key-value pair RDD through a flatMap function without +# changing the keys; this also retains the original RDD's partitioning. +# +# The same as 'flatMapValues()' in Spark. +# +# @param X The RDD to apply the transformation. +# @param FUN the transformation to apply on the value of each element. +# @return a new RDD created by the transformation. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) +# collect(flatMapValues(rdd, function(x) { x })) +# Output: list(list(1,1), list(1,2), list(2,3), list(2,4)) +#} +# @rdname flatMapValues +# @aliases flatMapValues,RDD,function-method setMethod("flatMapValues", signature(X = "RDD", FUN = "function"), function(X, FUN) { @@ -165,30 +165,30 @@ setMethod("flatMapValues", ############ Shuffle Functions ############ -#' Partition an RDD by key -#' -#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). -#' For each element of this RDD, the partitioner is used to compute a hash -#' function and the RDD is partitioned using this hash value. -#' -#' @param x The RDD to partition. Should be an RDD where each element is -#' list(K, V) or c(K, V). -#' @param numPartitions Number of partitions to create. -#' @param ... Other optional arguments to partitionBy. -#' -#' @param partitionFunc The partition function to use. Uses a default hashCode -#' function if not provided -#' @return An RDD partitioned using the specified partitioner. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) -#' rdd <- parallelize(sc, pairs) -#' parts <- partitionBy(rdd, 2L) -#' collectPartition(parts, 0L) # First partition should contain list(1, 2) and list(1, 4) -#'} -#' @rdname partitionBy -#' @aliases partitionBy,RDD,integer-method +# Partition an RDD by key +# +# This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +# For each element of this RDD, the partitioner is used to compute a hash +# function and the RDD is partitioned using this hash value. +# +# @param x The RDD to partition. Should be an RDD where each element is +# list(K, V) or c(K, V). +# @param numPartitions Number of partitions to create. +# @param ... Other optional arguments to partitionBy. +# +# @param partitionFunc The partition function to use. Uses a default hashCode +# function if not provided +# @return An RDD partitioned using the specified partitioner. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +# rdd <- parallelize(sc, pairs) +# parts <- partitionBy(rdd, 2L) +# collectPartition(parts, 0L) # First partition should contain list(1, 2) and list(1, 4) +#} +# @rdname partitionBy +# @aliases partitionBy,RDD,integer-method setMethod("partitionBy", signature(x = "RDD", numPartitions = "numeric"), function(x, numPartitions, partitionFunc = hashCode) { @@ -234,27 +234,27 @@ setMethod("partitionBy", RDD(r, serializedMode = "byte") }) -#' Group values by key -#' -#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). -#' and group values for each key in the RDD into a single sequence. -#' -#' @param x The RDD to group. Should be an RDD where each element is -#' list(K, V) or c(K, V). -#' @param numPartitions Number of partitions to create. -#' @return An RDD where each element is list(K, list(V)) -#' @seealso reduceByKey -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) -#' rdd <- parallelize(sc, pairs) -#' parts <- groupByKey(rdd, 2L) -#' grouped <- collect(parts) -#' grouped[[1]] # Should be a list(1, list(2, 4)) -#'} -#' @rdname groupByKey -#' @aliases groupByKey,RDD,integer-method +# Group values by key +# +# This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +# and group values for each key in the RDD into a single sequence. +# +# @param x The RDD to group. Should be an RDD where each element is +# list(K, V) or c(K, V). +# @param numPartitions Number of partitions to create. +# @return An RDD where each element is list(K, list(V)) +# @seealso reduceByKey +# @examples +#\dontrun{ +# sc <- sparkR.init() +# pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +# rdd <- parallelize(sc, pairs) +# parts <- groupByKey(rdd, 2L) +# grouped <- collect(parts) +# grouped[[1]] # Should be a list(1, list(2, 4)) +#} +# @rdname groupByKey +# @aliases groupByKey,RDD,integer-method setMethod("groupByKey", signature(x = "RDD", numPartitions = "numeric"), function(x, numPartitions) { @@ -292,28 +292,28 @@ setMethod("groupByKey", lapplyPartition(shuffled, groupVals) }) -#' Merge values by key -#' -#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). -#' and merges the values for each key using an associative reduce function. -#' -#' @param x The RDD to reduce by key. Should be an RDD where each element is -#' list(K, V) or c(K, V). -#' @param combineFunc The associative reduce function to use. -#' @param numPartitions Number of partitions to create. -#' @return An RDD where each element is list(K, V') where V' is the merged -#' value -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) -#' rdd <- parallelize(sc, pairs) -#' parts <- reduceByKey(rdd, "+", 2L) -#' reduced <- collect(parts) -#' reduced[[1]] # Should be a list(1, 6) -#'} -#' @rdname reduceByKey -#' @aliases reduceByKey,RDD,integer-method +# Merge values by key +# +# This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +# and merges the values for each key using an associative reduce function. +# +# @param x The RDD to reduce by key. Should be an RDD where each element is +# list(K, V) or c(K, V). +# @param combineFunc The associative reduce function to use. +# @param numPartitions Number of partitions to create. +# @return An RDD where each element is list(K, V') where V' is the merged +# value +# @examples +#\dontrun{ +# sc <- sparkR.init() +# pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +# rdd <- parallelize(sc, pairs) +# parts <- reduceByKey(rdd, "+", 2L) +# reduced <- collect(parts) +# reduced[[1]] # Should be a list(1, 6) +#} +# @rdname reduceByKey +# @aliases reduceByKey,RDD,integer-method setMethod("reduceByKey", signature(x = "RDD", combineFunc = "ANY", numPartitions = "numeric"), function(x, combineFunc, numPartitions) { @@ -333,27 +333,27 @@ setMethod("reduceByKey", lapplyPartition(shuffled, reduceVals) }) -#' Merge values by key locally -#' -#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). -#' and merges the values for each key using an associative reduce function, but return the -#' results immediately to the driver as an R list. -#' -#' @param x The RDD to reduce by key. Should be an RDD where each element is -#' list(K, V) or c(K, V). -#' @param combineFunc The associative reduce function to use. -#' @return A list of elements of type list(K, V') where V' is the merged value for each key -#' @seealso reduceByKey -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) -#' rdd <- parallelize(sc, pairs) -#' reduced <- reduceByKeyLocally(rdd, "+") -#' reduced # list(list(1, 6), list(1.1, 3)) -#'} -#' @rdname reduceByKeyLocally -#' @aliases reduceByKeyLocally,RDD,integer-method +# Merge values by key locally +# +# This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +# and merges the values for each key using an associative reduce function, but return the +# results immediately to the driver as an R list. +# +# @param x The RDD to reduce by key. Should be an RDD where each element is +# list(K, V) or c(K, V). +# @param combineFunc The associative reduce function to use. +# @return A list of elements of type list(K, V') where V' is the merged value for each key +# @seealso reduceByKey +# @examples +#\dontrun{ +# sc <- sparkR.init() +# pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +# rdd <- parallelize(sc, pairs) +# reduced <- reduceByKeyLocally(rdd, "+") +# reduced # list(list(1, 6), list(1.1, 3)) +#} +# @rdname reduceByKeyLocally +# @aliases reduceByKeyLocally,RDD,integer-method setMethod("reduceByKeyLocally", signature(x = "RDD", combineFunc = "ANY"), function(x, combineFunc) { @@ -385,41 +385,41 @@ setMethod("reduceByKeyLocally", convertEnvsToList(merged[[1]], merged[[2]]) }) -#' Combine values by key -#' -#' Generic function to combine the elements for each key using a custom set of -#' aggregation functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], -#' for a "combined type" C. Note that V and C can be different -- for example, one -#' might group an RDD of type (Int, Int) into an RDD of type (Int, Seq[Int]). - -#' Users provide three functions: -#' \itemize{ -#' \item createCombiner, which turns a V into a C (e.g., creates a one-element list) -#' \item mergeValue, to merge a V into a C (e.g., adds it to the end of a list) - -#' \item mergeCombiners, to combine two C's into a single one (e.g., concatentates -#' two lists). -#' } -#' -#' @param x The RDD to combine. Should be an RDD where each element is -#' list(K, V) or c(K, V). -#' @param createCombiner Create a combiner (C) given a value (V) -#' @param mergeValue Merge the given value (V) with an existing combiner (C) -#' @param mergeCombiners Merge two combiners and return a new combiner -#' @param numPartitions Number of partitions to create. -#' @return An RDD where each element is list(K, C) where C is the combined type -#' -#' @seealso groupByKey, reduceByKey -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) -#' rdd <- parallelize(sc, pairs) -#' parts <- combineByKey(rdd, function(x) { x }, "+", "+", 2L) -#' combined <- collect(parts) -#' combined[[1]] # Should be a list(1, 6) -#'} -#' @rdname combineByKey -#' @aliases combineByKey,RDD,ANY,ANY,ANY,integer-method +# Combine values by key +# +# Generic function to combine the elements for each key using a custom set of +# aggregation functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], +# for a "combined type" C. Note that V and C can be different -- for example, one +# might group an RDD of type (Int, Int) into an RDD of type (Int, Seq[Int]). + +# Users provide three functions: +# \itemize{ +# \item createCombiner, which turns a V into a C (e.g., creates a one-element list) +# \item mergeValue, to merge a V into a C (e.g., adds it to the end of a list) - +# \item mergeCombiners, to combine two C's into a single one (e.g., concatentates +# two lists). +# } +# +# @param x The RDD to combine. Should be an RDD where each element is +# list(K, V) or c(K, V). +# @param createCombiner Create a combiner (C) given a value (V) +# @param mergeValue Merge the given value (V) with an existing combiner (C) +# @param mergeCombiners Merge two combiners and return a new combiner +# @param numPartitions Number of partitions to create. +# @return An RDD where each element is list(K, C) where C is the combined type +# +# @seealso groupByKey, reduceByKey +# @examples +#\dontrun{ +# sc <- sparkR.init() +# pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +# rdd <- parallelize(sc, pairs) +# parts <- combineByKey(rdd, function(x) { x }, "+", "+", 2L) +# combined <- collect(parts) +# combined[[1]] # Should be a list(1, 6) +#} +# @rdname combineByKey +# @aliases combineByKey,RDD,ANY,ANY,ANY,integer-method setMethod("combineByKey", signature(x = "RDD", createCombiner = "ANY", mergeValue = "ANY", mergeCombiners = "ANY", numPartitions = "numeric"), @@ -451,36 +451,36 @@ setMethod("combineByKey", lapplyPartition(shuffled, mergeAfterShuffle) }) -#' Aggregate a pair RDD by each key. -#' -#' Aggregate the values of each key in an RDD, using given combine functions -#' and a neutral "zero value". This function can return a different result type, -#' U, than the type of the values in this RDD, V. Thus, we need one operation -#' for merging a V into a U and one operation for merging two U's, The former -#' operation is used for merging values within a partition, and the latter is -#' used for merging values between partitions. To avoid memory allocation, both -#' of these functions are allowed to modify and return their first argument -#' instead of creating a new U. -#' -#' @param x An RDD. -#' @param zeroValue A neutral "zero value". -#' @param seqOp A function to aggregate the values of each key. It may return -#' a different result type from the type of the values. -#' @param combOp A function to aggregate results of seqOp. -#' @return An RDD containing the aggregation result. -#' @seealso foldByKey, combineByKey -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) -#' zeroValue <- list(0, 0) -#' seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } -#' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } -#' aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) -#' # list(list(1, list(3, 2)), list(2, list(7, 2))) -#'} -#' @rdname aggregateByKey -#' @aliases aggregateByKey,RDD,ANY,ANY,ANY,integer-method +# Aggregate a pair RDD by each key. +# +# Aggregate the values of each key in an RDD, using given combine functions +# and a neutral "zero value". This function can return a different result type, +# U, than the type of the values in this RDD, V. Thus, we need one operation +# for merging a V into a U and one operation for merging two U's, The former +# operation is used for merging values within a partition, and the latter is +# used for merging values between partitions. To avoid memory allocation, both +# of these functions are allowed to modify and return their first argument +# instead of creating a new U. +# +# @param x An RDD. +# @param zeroValue A neutral "zero value". +# @param seqOp A function to aggregate the values of each key. It may return +# a different result type from the type of the values. +# @param combOp A function to aggregate results of seqOp. +# @return An RDD containing the aggregation result. +# @seealso foldByKey, combineByKey +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) +# zeroValue <- list(0, 0) +# seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } +# combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } +# aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) +# # list(list(1, list(3, 2)), list(2, list(7, 2))) +#} +# @rdname aggregateByKey +# @aliases aggregateByKey,RDD,ANY,ANY,ANY,integer-method setMethod("aggregateByKey", signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", combOp = "ANY", numPartitions = "numeric"), @@ -492,26 +492,26 @@ setMethod("aggregateByKey", combineByKey(x, createCombiner, seqOp, combOp, numPartitions) }) -#' Fold a pair RDD by each key. -#' -#' Aggregate the values of each key in an RDD, using an associative function "func" -#' and a neutral "zero value" which may be added to the result an arbitrary -#' number of times, and must not change the result (e.g., 0 for addition, or -#' 1 for multiplication.). -#' -#' @param x An RDD. -#' @param zeroValue A neutral "zero value". -#' @param func An associative function for folding values of each key. -#' @return An RDD containing the aggregation result. -#' @seealso aggregateByKey, combineByKey -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) -#' foldByKey(rdd, 0, "+", 2L) # list(list(1, 3), list(2, 7)) -#'} -#' @rdname foldByKey -#' @aliases foldByKey,RDD,ANY,ANY,integer-method +# Fold a pair RDD by each key. +# +# Aggregate the values of each key in an RDD, using an associative function "func" +# and a neutral "zero value" which may be added to the result an arbitrary +# number of times, and must not change the result (e.g., 0 for addition, or +# 1 for multiplication.). +# +# @param x An RDD. +# @param zeroValue A neutral "zero value". +# @param func An associative function for folding values of each key. +# @return An RDD containing the aggregation result. +# @seealso aggregateByKey, combineByKey +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) +# foldByKey(rdd, 0, "+", 2L) # list(list(1, 3), list(2, 7)) +#} +# @rdname foldByKey +# @aliases foldByKey,RDD,ANY,ANY,integer-method setMethod("foldByKey", signature(x = "RDD", zeroValue = "ANY", func = "ANY", numPartitions = "numeric"), @@ -521,28 +521,28 @@ setMethod("foldByKey", ############ Binary Functions ############# -#' Join two RDDs -#' -#' @description -#' \code{join} This function joins two RDDs where every element is of the form list(K, V). -#' The key types of the two RDDs should be the same. -#' -#' @param x An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param y An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param numPartitions Number of partitions to create. -#' @return a new RDD containing all pairs of elements with matching keys in -#' two input RDDs. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) -#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) -#' join(rdd1, rdd2, 2L) # list(list(1, list(1, 2)), list(1, list(1, 3)) -#'} -#' @rdname join-methods -#' @aliases join,RDD,RDD-method +# Join two RDDs +# +# @description +# \code{join} This function joins two RDDs where every element is of the form list(K, V). +# The key types of the two RDDs should be the same. +# +# @param x An RDD to be joined. Should be an RDD where each element is +# list(K, V). +# @param y An RDD to be joined. Should be an RDD where each element is +# list(K, V). +# @param numPartitions Number of partitions to create. +# @return a new RDD containing all pairs of elements with matching keys in +# two input RDDs. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +# rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +# join(rdd1, rdd2, 2L) # list(list(1, list(1, 2)), list(1, list(1, 3)) +#} +# @rdname join-methods +# @aliases join,RDD,RDD-method setMethod("join", signature(x = "RDD", y = "RDD"), function(x, y, numPartitions) { @@ -557,30 +557,30 @@ setMethod("join", doJoin) }) -#' Left outer join two RDDs -#' -#' @description -#' \code{leftouterjoin} This function left-outer-joins two RDDs where every element is of the form list(K, V). -#' The key types of the two RDDs should be the same. -#' -#' @param x An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param y An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param numPartitions Number of partitions to create. -#' @return For each element (k, v) in x, the resulting RDD will either contain -#' all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL)) -#' if no elements in rdd2 have key k. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) -#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) -#' leftOuterJoin(rdd1, rdd2, 2L) -#' # list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) -#'} -#' @rdname join-methods -#' @aliases leftOuterJoin,RDD,RDD-method +# Left outer join two RDDs +# +# @description +# \code{leftouterjoin} This function left-outer-joins two RDDs where every element is of the form list(K, V). +# The key types of the two RDDs should be the same. +# +# @param x An RDD to be joined. Should be an RDD where each element is +# list(K, V). +# @param y An RDD to be joined. Should be an RDD where each element is +# list(K, V). +# @param numPartitions Number of partitions to create. +# @return For each element (k, v) in x, the resulting RDD will either contain +# all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL)) +# if no elements in rdd2 have key k. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +# rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +# leftOuterJoin(rdd1, rdd2, 2L) +# # list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) +#} +# @rdname join-methods +# @aliases leftOuterJoin,RDD,RDD-method setMethod("leftOuterJoin", signature(x = "RDD", y = "RDD", numPartitions = "numeric"), function(x, y, numPartitions) { @@ -594,30 +594,30 @@ setMethod("leftOuterJoin", joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) -#' Right outer join two RDDs -#' -#' @description -#' \code{rightouterjoin} This function right-outer-joins two RDDs where every element is of the form list(K, V). -#' The key types of the two RDDs should be the same. -#' -#' @param x An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param y An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param numPartitions Number of partitions to create. -#' @return For each element (k, w) in y, the resulting RDD will either contain -#' all pairs (k, (v, w)) for (k, v) in x, or the pair (k, (NULL, w)) -#' if no elements in x have key k. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3))) -#' rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) -#' rightOuterJoin(rdd1, rdd2, 2L) -#' # list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) -#'} -#' @rdname join-methods -#' @aliases rightOuterJoin,RDD,RDD-method +# Right outer join two RDDs +# +# @description +# \code{rightouterjoin} This function right-outer-joins two RDDs where every element is of the form list(K, V). +# The key types of the two RDDs should be the same. +# +# @param x An RDD to be joined. Should be an RDD where each element is +# list(K, V). +# @param y An RDD to be joined. Should be an RDD where each element is +# list(K, V). +# @param numPartitions Number of partitions to create. +# @return For each element (k, w) in y, the resulting RDD will either contain +# all pairs (k, (v, w)) for (k, v) in x, or the pair (k, (NULL, w)) +# if no elements in x have key k. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3))) +# rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) +# rightOuterJoin(rdd1, rdd2, 2L) +# # list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) +#} +# @rdname join-methods +# @aliases rightOuterJoin,RDD,RDD-method setMethod("rightOuterJoin", signature(x = "RDD", y = "RDD", numPartitions = "numeric"), function(x, y, numPartitions) { @@ -631,33 +631,33 @@ setMethod("rightOuterJoin", joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) -#' Full outer join two RDDs -#' -#' @description -#' \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V). -#' The key types of the two RDDs should be the same. -#' -#' @param x An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param y An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param numPartitions Number of partitions to create. -#' @return For each element (k, v) in x and (k, w) in y, the resulting RDD -#' will contain all pairs (k, (v, w)) for both (k, v) in x and -#' (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements -#' in x/y have key k. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3), list(3, 3))) -#' rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) -#' fullOuterJoin(rdd1, rdd2, 2L) # list(list(1, list(2, 1)), -#' # list(1, list(3, 1)), -#' # list(2, list(NULL, 4))) -#' # list(3, list(3, NULL)), -#'} -#' @rdname join-methods -#' @aliases fullOuterJoin,RDD,RDD-method +# Full outer join two RDDs +# +# @description +# \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V). +# The key types of the two RDDs should be the same. +# +# @param x An RDD to be joined. Should be an RDD where each element is +# list(K, V). +# @param y An RDD to be joined. Should be an RDD where each element is +# list(K, V). +# @param numPartitions Number of partitions to create. +# @return For each element (k, v) in x and (k, w) in y, the resulting RDD +# will contain all pairs (k, (v, w)) for both (k, v) in x and +# (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements +# in x/y have key k. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3), list(3, 3))) +# rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) +# fullOuterJoin(rdd1, rdd2, 2L) # list(list(1, list(2, 1)), +# # list(1, list(3, 1)), +# # list(2, list(NULL, 4))) +# # list(3, list(3, NULL)), +#} +# @rdname join-methods +# @aliases fullOuterJoin,RDD,RDD-method setMethod("fullOuterJoin", signature(x = "RDD", y = "RDD", numPartitions = "numeric"), function(x, y, numPartitions) { @@ -671,23 +671,23 @@ setMethod("fullOuterJoin", joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) -#' For each key k in several RDDs, return a resulting RDD that -#' whose values are a list of values for the key in all RDDs. -#' -#' @param ... Several RDDs. -#' @param numPartitions Number of partitions to create. -#' @return a new RDD containing all pairs of elements with values in a list -#' in all RDDs. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) -#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) -#' cogroup(rdd1, rdd2, numPartitions = 2L) -#' # list(list(1, list(1, list(2, 3))), list(2, list(list(4), list())) -#'} -#' @rdname cogroup -#' @aliases cogroup,RDD-method +# For each key k in several RDDs, return a resulting RDD that +# whose values are a list of values for the key in all RDDs. +# +# @param ... Several RDDs. +# @param numPartitions Number of partitions to create. +# @return a new RDD containing all pairs of elements with values in a list +# in all RDDs. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +# rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +# cogroup(rdd1, rdd2, numPartitions = 2L) +# # list(list(1, list(1, list(2, 3))), list(2, list(list(4), list())) +#} +# @rdname cogroup +# @aliases cogroup,RDD-method setMethod("cogroup", "RDD", function(..., numPartitions) { @@ -723,20 +723,20 @@ setMethod("cogroup", group.func) }) -#' Sort a (k, v) pair RDD by k. -#' -#' @param x A (k, v) pair RDD to be sorted. -#' @param ascending A flag to indicate whether the sorting is ascending or descending. -#' @param numPartitions Number of partitions to create. -#' @return An RDD where all (k, v) pair elements are sorted. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(3, 1), list(2, 2), list(1, 3))) -#' collect(sortByKey(rdd)) # list (list(1, 3), list(2, 2), list(3, 1)) -#'} -#' @rdname sortByKey -#' @aliases sortByKey,RDD,RDD-method +# Sort a (k, v) pair RDD by k. +# +# @param x A (k, v) pair RDD to be sorted. +# @param ascending A flag to indicate whether the sorting is ascending or descending. +# @param numPartitions Number of partitions to create. +# @return An RDD where all (k, v) pair elements are sorted. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(list(3, 1), list(2, 2), list(1, 3))) +# collect(sortByKey(rdd)) # list (list(1, 3), list(2, 2), list(3, 1)) +#} +# @rdname sortByKey +# @aliases sortByKey,RDD,RDD-method setMethod("sortByKey", signature(x = "RDD"), function(x, ascending = TRUE, numPartitions = SparkR:::numPartitions(x)) { @@ -785,25 +785,25 @@ setMethod("sortByKey", lapplyPartition(newRDD, partitionFunc) }) -#' Subtract a pair RDD with another pair RDD. -#' -#' Return an RDD with the pairs from x whose keys are not in other. -#' -#' @param x An RDD. -#' @param other An RDD. -#' @param numPartitions Number of the partitions in the result RDD. -#' @return An RDD with the pairs from x whose keys are not in other. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4), -#' list("b", 5), list("a", 2))) -#' rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) -#' collect(subtractByKey(rdd1, rdd2)) -#' # list(list("b", 4), list("b", 5)) -#'} -#' @rdname subtractByKey -#' @aliases subtractByKey,RDD +# Subtract a pair RDD with another pair RDD. +# +# Return an RDD with the pairs from x whose keys are not in other. +# +# @param x An RDD. +# @param other An RDD. +# @param numPartitions Number of the partitions in the result RDD. +# @return An RDD with the pairs from x whose keys are not in other. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4), +# list("b", 5), list("a", 2))) +# rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) +# collect(subtractByKey(rdd1, rdd2)) +# # list(list("b", 4), list("b", 5)) +#} +# @rdname subtractByKey +# @aliases subtractByKey,RDD setMethod("subtractByKey", signature(x = "RDD", other = "RDD"), function(x, other, numPartitions = SparkR:::numPartitions(x)) { @@ -819,41 +819,41 @@ setMethod("subtractByKey", function (v) { v[[1]] }) }) -#' Return a subset of this RDD sampled by key. -#' -#' @description -#' \code{sampleByKey} Create a sample of this RDD using variable sampling rates -#' for different keys as specified by fractions, a key to sampling rate map. -#' -#' @param x The RDD to sample elements by key, where each element is -#' list(K, V) or c(K, V). -#' @param withReplacement Sampling with replacement or not -#' @param fraction The (rough) sample target fraction -#' @param seed Randomness seed value -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:3000) -#' pairs <- lapply(rdd, function(x) { if (x %% 3 == 0) list("a", x) -#' else { if (x %% 3 == 1) list("b", x) else list("c", x) }}) -#' fractions <- list(a = 0.2, b = 0.1, c = 0.3) -#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) -#' 100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")) # TRUE -#' 50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")) # TRUE -#' 200 < length(lookup(sample, "c")) && 400 > length(lookup(sample, "c")) # TRUE -#' lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0 # TRUE -#' lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000 # TRUE -#' lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0 # TRUE -#' lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000 # TRUE -#' lookup(sample, "c")[which.min(lookup(sample, "c"))] >= 0 # TRUE -#' lookup(sample, "c")[which.max(lookup(sample, "c"))] <= 2000 # TRUE -#' fractions <- list(a = 0.2, b = 0.1, c = 0.3, d = 0.4) -#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # Key "d" will be ignored -#' fractions <- list(a = 0.2, b = 0.1) -#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # KeyError: "c" -#'} -#' @rdname sampleByKey -#' @aliases sampleByKey,RDD-method +# Return a subset of this RDD sampled by key. +# +# @description +# \code{sampleByKey} Create a sample of this RDD using variable sampling rates +# for different keys as specified by fractions, a key to sampling rate map. +# +# @param x The RDD to sample elements by key, where each element is +# list(K, V) or c(K, V). +# @param withReplacement Sampling with replacement or not +# @param fraction The (rough) sample target fraction +# @param seed Randomness seed value +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:3000) +# pairs <- lapply(rdd, function(x) { if (x %% 3 == 0) list("a", x) +# else { if (x %% 3 == 1) list("b", x) else list("c", x) }}) +# fractions <- list(a = 0.2, b = 0.1, c = 0.3) +# sample <- sampleByKey(pairs, FALSE, fractions, 1618L) +# 100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")) # TRUE +# 50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")) # TRUE +# 200 < length(lookup(sample, "c")) && 400 > length(lookup(sample, "c")) # TRUE +# lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0 # TRUE +# lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000 # TRUE +# lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0 # TRUE +# lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000 # TRUE +# lookup(sample, "c")[which.min(lookup(sample, "c"))] >= 0 # TRUE +# lookup(sample, "c")[which.max(lookup(sample, "c"))] <= 2000 # TRUE +# fractions <- list(a = 0.2, b = 0.1, c = 0.3, d = 0.4) +# sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # Key "d" will be ignored +# fractions <- list(a = 0.2, b = 0.1) +# sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # KeyError: "c" +#} +# @rdname sampleByKey +# @aliases sampleByKey,RDD-method setMethod("sampleByKey", signature(x = "RDD", withReplacement = "logical", fractions = "vector", seed = "integer"), diff --git a/R/pkg/inst/profile/shell.R b/R/pkg/inst/profile/shell.R index 7a7f2031152a0..33478d9e29995 100644 --- a/R/pkg/inst/profile/shell.R +++ b/R/pkg/inst/profile/shell.R @@ -20,11 +20,13 @@ .libPaths(c(file.path(home, "R", "lib"), .libPaths())) Sys.setenv(NOAWT=1) - library(utils) - library(SparkR) - sc <- sparkR.init(Sys.getenv("MASTER", unset = "")) + # Make sure SparkR package is the last loaded one + old <- getOption("defaultPackages") + options(defaultPackages = c(old, "SparkR")) + + sc <- SparkR::sparkR.init(Sys.getenv("MASTER", unset = "")) assign("sc", sc, envir=.GlobalEnv) - sqlCtx <- sparkRSQL.init(sc) + sqlCtx <- SparkR::sparkRSQL.init(sc) assign("sqlCtx", sqlCtx, envir=.GlobalEnv) cat("\n Welcome to SparkR!") cat("\n Spark context is available as sc, SQL context is available as sqlCtx\n") diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index f82e56fdd8278..7a42e289fcd9e 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -705,5 +705,16 @@ test_that("parquetFile works with multiple input paths", { expect_true(count(parquetDF) == count(df)*2) }) +test_that("describe() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + stats <- describe(df, "age") + expect_true(collect(stats)[1, "summary"] == "count") + expect_true(collect(stats)[2, "age"] == 24.5) + expect_true(collect(stats)[3, "age"] == 5.5) + stats <- describe(df) + expect_true(collect(stats)[4, "name"] == "Andy") + expect_true(collect(stats)[5, "age"] == 30.0) +}) + unlink(parquetPath) unlink(jsonPath) diff --git a/assembly/pom.xml b/assembly/pom.xml index 2b4d0a990bf22..626c8577e31fe 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -92,6 +92,27 @@ true + + + org.apache.maven.plugins + maven-antrun-plugin + + + package + + run + + + + + + + + + + + + org.apache.maven.plugins diff --git a/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js b/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js index 6d2da25024a83..6beb7db855ca0 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js +++ b/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js @@ -1,4 +1,4 @@ -/*v0.4.3 with 1 additional commit (see http://github.com/andrewor14/dagre-d3)*/(function(f){if(typeof exports==="object"&&typeof module!=="undefined"){module.exports=f()}else if(typeof define==="function"&&define.amd){define([],f)}else{var g;if(typeof window!=="undefined"){g=window}else if(typeof global!=="undefined"){g=global}else if(typeof self!=="undefined"){g=self}else{g=this}g.dagreD3=f()}})(function(){var define,module,exports;return function e(t,n,r){function s(o,u){if(!n[o]){if(!t[o]){var a=typeof require=="function"&&require;if(!u&&a)return a(o,!0);if(i)return i(o,!0);var f=new Error("Cannot find module '"+o+"'");throw f.code="MODULE_NOT_FOUND",f}var l=n[o]={exports:{}};t[o][0].call(l.exports,function(e){var n=t[o][1][e];return s(n?n:e)},l,l.exports,e,t,n,r)}return n[o].exports}var i=typeof require=="function"&&require;for(var o=0;o0}},{}],14:[function(require,module,exports){module.exports=intersectNode;function intersectNode(node,point){return node.intersect(point)}},{}],15:[function(require,module,exports){var intersectLine=require("./intersect-line");module.exports=intersectPolygon;function intersectPolygon(node,polyPoints,point){var x1=node.x;var y1=node.y;var intersections=[];var minX=Number.POSITIVE_INFINITY,minY=Number.POSITIVE_INFINITY;polyPoints.forEach(function(entry){minX=Math.min(minX,entry.x);minY=Math.min(minY,entry.y)});var left=x1-node.width/2-minX;var top=y1-node.height/2-minY;for(var i=0;i1){intersections.sort(function(p,q){var pdx=p.x-point.x,pdy=p.y-point.y,distp=Math.sqrt(pdx*pdx+pdy*pdy),qdx=q.x-point.x,qdy=q.y-point.y,distq=Math.sqrt(qdx*qdx+qdy*qdy);return distpMath.abs(dx)*h){if(dy<0){h=-h}sx=dy===0?0:h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=dx===0?0:w*dy/dx}return{x:x+sx,y:y+sy}}},{}],17:[function(require,module,exports){var util=require("../util");module.exports=addHtmlLabel;function addHtmlLabel(root,node){var fo=root.append("foreignObject").attr("width","100000");var div=fo.append("xhtml:div");var label=node.label;switch(typeof label){case"function":div.insert(label);break;case"object":div.insert(function(){return label});break;default:div.html(label)}util.applyStyle(div,node.labelStyle);div.style("display","inline-block");div.style("white-space","nowrap");var w,h;div.each(function(){w=this.clientWidth;h=this.clientHeight});fo.attr("width",w).attr("height",h);return fo}},{"../util":25}],18:[function(require,module,exports){var addTextLabel=require("./add-text-label"),addHtmlLabel=require("./add-html-label");module.exports=addLabel;function addLabel(root,node){var label=node.label;var labelSvg=root.append("g");if(typeof label!=="string"||node.labelType==="html"){addHtmlLabel(labelSvg,node)}else{addTextLabel(labelSvg,node)}var labelBBox=labelSvg.node().getBBox();labelSvg.attr("transform","translate("+-labelBBox.width/2+","+-labelBBox.height/2+")");return labelSvg}},{"./add-html-label":17,"./add-text-label":19}],19:[function(require,module,exports){var util=require("../util");module.exports=addTextLabel;function addTextLabel(root,node){var domNode=root.append("text");var lines=processEscapeSequences(node.label).split("\n");for(var i=0;i0;--i){entry=buckets[i].dequeue();if(entry){results=results.concat(removeNode(g,buckets,zeroIdx,entry,true));break}}}}return results}function removeNode(g,buckets,zeroIdx,entry,collectPredecessors){var results=collectPredecessors?[]:undefined;_.each(g.inEdges(entry.v),function(edge){var weight=g.edge(edge),uEntry=g.node(edge.v);if(collectPredecessors){results.push({v:edge.v,w:edge.w})}uEntry.out-=weight;assignBucket(buckets,zeroIdx,uEntry)});_.each(g.outEdges(entry.v),function(edge){var weight=g.edge(edge),w=edge.w,wEntry=g.node(w);wEntry["in"]-=weight;assignBucket(buckets,zeroIdx,wEntry)});g.removeNode(entry.v);return results}function buildState(g,weightFn){var fasGraph=new Graph,maxIn=0,maxOut=0;_.each(g.nodes(),function(v){fasGraph.setNode(v,{v:v,"in":0,out:0})});_.each(g.edges(),function(e){var prevWeight=fasGraph.edge(e.v,e.w)||0,weight=weightFn(e),edgeWeight=prevWeight+weight;fasGraph.setEdge(e.v,e.w,edgeWeight);maxOut=Math.max(maxOut,fasGraph.node(e.v).out+=weight);maxIn=Math.max(maxIn,fasGraph.node(e.w)["in"]+=weight)});var buckets=_.range(maxOut+maxIn+3).map(function(){return new List});var zeroIdx=maxIn+1;_.each(fasGraph.nodes(),function(v){assignBucket(buckets,zeroIdx,fasGraph.node(v))});return{graph:fasGraph,buckets:buckets,zeroIdx:zeroIdx}}function assignBucket(buckets,zeroIdx,entry){if(!entry.out){buckets[0].enqueue(entry)}else if(!entry["in"]){buckets[buckets.length-1].enqueue(entry)}else{buckets[entry.out-entry["in"]+zeroIdx].enqueue(entry)}}},{"./data/list":31,"./graphlib":33,"./lodash":36}],35:[function(require,module,exports){"use strict";var _=require("./lodash"),acyclic=require("./acyclic"),normalize=require("./normalize"),rank=require("./rank"),normalizeRanks=require("./util").normalizeRanks,parentDummyChains=require("./parent-dummy-chains"),removeEmptyRanks=require("./util").removeEmptyRanks,nestingGraph=require("./nesting-graph"),addBorderSegments=require("./add-border-segments"),coordinateSystem=require("./coordinate-system"),order=require("./order"),position=require("./position"),util=require("./util"),Graph=require("./graphlib").Graph;module.exports=layout;function layout(g,opts){var time=opts&&opts.debugTiming?util.time:util.notime;time("layout",function(){var layoutGraph=time(" buildLayoutGraph",function(){return buildLayoutGraph(g)});time(" runLayout",function(){runLayout(layoutGraph,time)});time(" updateInputGraph",function(){updateInputGraph(g,layoutGraph)})})}function runLayout(g,time){time(" makeSpaceForEdgeLabels",function(){makeSpaceForEdgeLabels(g)});time(" removeSelfEdges",function(){removeSelfEdges(g)});time(" acyclic",function(){acyclic.run(g)});time(" nestingGraph.run",function(){nestingGraph.run(g)});time(" rank",function(){rank(util.asNonCompoundGraph(g))});time(" injectEdgeLabelProxies",function(){injectEdgeLabelProxies(g)});time(" removeEmptyRanks",function(){removeEmptyRanks(g)});time(" nestingGraph.cleanup",function(){nestingGraph.cleanup(g)});time(" normalizeRanks",function(){normalizeRanks(g)});time(" assignRankMinMax",function(){assignRankMinMax(g)});time(" removeEdgeLabelProxies",function(){removeEdgeLabelProxies(g)});time(" normalize.run",function(){normalize.run(g)});time(" parentDummyChains",function(){parentDummyChains(g)});time(" addBorderSegments",function(){addBorderSegments(g)});time(" order",function(){order(g)});time(" insertSelfEdges",function(){insertSelfEdges(g)});time(" adjustCoordinateSystem",function(){coordinateSystem.adjust(g)});time(" position",function(){position(g)});time(" positionSelfEdges",function(){positionSelfEdges(g)});time(" removeBorderNodes",function(){removeBorderNodes(g)});time(" normalize.undo",function(){normalize.undo(g)});time(" fixupEdgeLabelCoords",function(){fixupEdgeLabelCoords(g)});time(" undoCoordinateSystem",function(){coordinateSystem.undo(g)});time(" translateGraph",function(){translateGraph(g)});time(" assignNodeIntersects",function(){assignNodeIntersects(g)});time(" reversePoints",function(){reversePointsForReversedEdges(g)});time(" acyclic.undo",function(){acyclic.undo(g)})}function updateInputGraph(inputGraph,layoutGraph){_.each(inputGraph.nodes(),function(v){var inputLabel=inputGraph.node(v),layoutLabel=layoutGraph.node(v);if(inputLabel){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y;if(layoutGraph.children(v).length){inputLabel.width=layoutLabel.width;inputLabel.height=layoutLabel.height}}});_.each(inputGraph.edges(),function(e){var inputLabel=inputGraph.edge(e),layoutLabel=layoutGraph.edge(e);inputLabel.points=layoutLabel.points;if(_.has(layoutLabel,"x")){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y}});inputGraph.graph().width=layoutGraph.graph().width;inputGraph.graph().height=layoutGraph.graph().height}var graphNumAttrs=["nodesep","edgesep","ranksep","marginx","marginy"],graphDefaults={ranksep:50,edgesep:20,nodesep:50,rankdir:"tb"},graphAttrs=["acyclicer","ranker","rankdir","align"],nodeNumAttrs=["width","height"],nodeDefaults={width:0,height:0},edgeNumAttrs=["minlen","weight","width","height","labeloffset"],edgeDefaults={minlen:1,weight:1,width:0,height:0,labeloffset:10,labelpos:"r"},edgeAttrs=["labelpos"];function buildLayoutGraph(inputGraph){var g=new Graph({multigraph:true,compound:true}),graph=canonicalize(inputGraph.graph());g.setGraph(_.merge({},graphDefaults,selectNumberAttrs(graph,graphNumAttrs),_.pick(graph,graphAttrs)));_.each(inputGraph.nodes(),function(v){var node=canonicalize(inputGraph.node(v));g.setNode(v,_.defaults(selectNumberAttrs(node,nodeNumAttrs),nodeDefaults));g.setParent(v,inputGraph.parent(v))});_.each(inputGraph.edges(),function(e){var edge=canonicalize(inputGraph.edge(e));g.setEdge(e,_.merge({},edgeDefaults,selectNumberAttrs(edge,edgeNumAttrs),_.pick(edge,edgeAttrs)))});return g}function makeSpaceForEdgeLabels(g){var graph=g.graph(); +module.exports={graphlib:require("./lib/graphlib"),dagre:require("./lib/dagre"),intersect:require("./lib/intersect"),render:require("./lib/render"),util:require("./lib/util"),version:require("./lib/version")}},{"./lib/dagre":8,"./lib/graphlib":9,"./lib/intersect":10,"./lib/render":23,"./lib/util":25,"./lib/version":26}],2:[function(require,module,exports){var util=require("./util");module.exports={"default":normal,normal:normal,vee:vee,undirected:undirected};function normal(parent,id,edge,type){var marker=parent.append("marker").attr("id",id).attr("viewBox","0 0 10 10").attr("refX",9).attr("refY",5).attr("markerUnits","strokeWidth").attr("markerWidth",8).attr("markerHeight",6).attr("orient","auto");var path=marker.append("path").attr("d","M 0 0 L 10 5 L 0 10 z").style("stroke-width",1).style("stroke-dasharray","1,0");util.applyStyle(path,edge[type+"Style"])}function vee(parent,id,edge,type){var marker=parent.append("marker").attr("id",id).attr("viewBox","0 0 10 10").attr("refX",9).attr("refY",5).attr("markerUnits","strokeWidth").attr("markerWidth",8).attr("markerHeight",6).attr("orient","auto");var path=marker.append("path").attr("d","M 0 0 L 10 5 L 0 10 L 4 5 z").style("stroke-width",1).style("stroke-dasharray","1,0");util.applyStyle(path,edge[type+"Style"])}function undirected(parent,id,edge,type){var marker=parent.append("marker").attr("id",id).attr("viewBox","0 0 10 10").attr("refX",9).attr("refY",5).attr("markerUnits","strokeWidth").attr("markerWidth",8).attr("markerHeight",6).attr("orient","auto");var path=marker.append("path").attr("d","M 0 5 L 10 5").style("stroke-width",1).style("stroke-dasharray","1,0");util.applyStyle(path,edge[type+"Style"])}},{"./util":25}],3:[function(require,module,exports){var util=require("./util");module.exports=createClusters;function createClusters(selection,g){var clusters=g.nodes().filter(function(v){return util.isSubgraph(g,v)}),svgClusters=selection.selectAll("g.cluster").data(clusters,function(v){return v});svgClusters.enter().append("g").attr("id",function(v){return"cluster_"+v.replace(/^cluster/,"")}).attr("name",function(v){return g.node(v).label}).attr("class","cluster").style("opacity",0).append("rect");util.applyTransition(svgClusters.exit(),g).style("opacity",0).remove();util.applyTransition(svgClusters,g).style("opacity",1);util.applyTransition(svgClusters.selectAll("rect"),g).attr("width",function(v){var node=g.node(v);return node.width+node.paddingLeft+node.paddingRight}).attr("height",function(v){var node=g.node(v);return node.height+node.paddingTop+node.paddingBottom}).attr("x",function(v){var node=g.node(v);return node.x-node.width/2-node.paddingLeft}).attr("y",function(v){var node=g.node(v);return node.y-node.height/2-node.paddingTop})}},{"./util":25}],4:[function(require,module,exports){"use strict";var _=require("./lodash"),addLabel=require("./label/add-label"),util=require("./util"),d3=require("./d3");module.exports=createEdgeLabels;function createEdgeLabels(selection,g){var svgEdgeLabels=selection.selectAll("g.edgeLabel").data(g.edges(),function(e){return util.edgeToId(e)}).classed("update",true);svgEdgeLabels.selectAll("*").remove();svgEdgeLabels.enter().append("g").classed("edgeLabel",true).style("opacity",0);svgEdgeLabels.each(function(e){var edge=g.edge(e),label=addLabel(d3.select(this),g.edge(e),0,0).classed("label",true),bbox=label.node().getBBox();if(edge.labelId){label.attr("id",edge.labelId)}if(!_.has(edge,"width")){edge.width=bbox.width}if(!_.has(edge,"height")){edge.height=bbox.height}});util.applyTransition(svgEdgeLabels.exit(),g).style("opacity",0).remove();return svgEdgeLabels}},{"./d3":7,"./label/add-label":18,"./lodash":20,"./util":25}],5:[function(require,module,exports){"use strict";var _=require("./lodash"),intersectNode=require("./intersect/intersect-node"),util=require("./util"),d3=require("./d3");module.exports=createEdgePaths;function createEdgePaths(selection,g,arrows){var svgPaths=selection.selectAll("g.edgePath").data(g.edges(),function(e){return util.edgeToId(e)}).classed("update",true);enter(svgPaths,g);exit(svgPaths,g);util.applyTransition(svgPaths,g).style("opacity",1);svgPaths.each(function(e){var domEdge=d3.select(this);var edge=g.edge(e);edge.elem=this;if(edge.id){domEdge.attr("id",edge.id)}util.applyClass(domEdge,edge["class"],(domEdge.classed("update")?"update ":"")+"edgePath")});svgPaths.selectAll("path.path").each(function(e){var edge=g.edge(e);edge.arrowheadId=_.uniqueId("arrowhead");var domEdge=d3.select(this).attr("marker-end",function(){return"url(#"+edge.arrowheadId+")"}).style("fill","none");util.applyTransition(domEdge,g).attr("d",function(e){return calcPoints(g,e)});util.applyStyle(domEdge,edge.style)});svgPaths.selectAll("defs *").remove();svgPaths.selectAll("defs").each(function(e){var edge=g.edge(e),arrowhead=arrows[edge.arrowhead];arrowhead(d3.select(this),edge.arrowheadId,edge,"arrowhead")});return svgPaths}function calcPoints(g,e){var edge=g.edge(e),tail=g.node(e.v),head=g.node(e.w),points=edge.points.slice(1,edge.points.length-1);points.unshift(intersectNode(tail,points[0]));points.push(intersectNode(head,points[points.length-1]));return createLine(edge,points)}function createLine(edge,points){var line=d3.svg.line().x(function(d){return d.x}).y(function(d){return d.y});if(_.has(edge,"lineInterpolate")){line.interpolate(edge.lineInterpolate)}if(_.has(edge,"lineTension")){line.tension(Number(edge.lineTension))}return line(points)}function getCoords(elem){var bbox=elem.getBBox(),matrix=elem.getTransformToElement(elem.ownerSVGElement).translate(bbox.width/2,bbox.height/2);return{x:matrix.e,y:matrix.f}}function enter(svgPaths,g){var svgPathsEnter=svgPaths.enter().append("g").attr("class","edgePath").style("opacity",0);svgPathsEnter.append("path").attr("class","path").attr("d",function(e){var edge=g.edge(e),sourceElem=g.node(e.v).elem,points=_.range(edge.points.length).map(function(){return getCoords(sourceElem)});return createLine(edge,points)});svgPathsEnter.append("defs")}function exit(svgPaths,g){var svgPathExit=svgPaths.exit();util.applyTransition(svgPathExit,g).style("opacity",0).remove();util.applyTransition(svgPathExit.select("path.path"),g).attr("d",function(e){var source=g.node(e.v);if(source){var points=_.range(this.pathSegList.length).map(function(){return source});return createLine({},points)}else{return d3.select(this).attr("d")}})}},{"./d3":7,"./intersect/intersect-node":14,"./lodash":20,"./util":25}],6:[function(require,module,exports){"use strict";var _=require("./lodash"),addLabel=require("./label/add-label"),util=require("./util"),d3=require("./d3");module.exports=createNodes;function createNodes(selection,g,shapes){var simpleNodes=g.nodes().filter(function(v){return!util.isSubgraph(g,v)});var svgNodes=selection.selectAll("g.node").data(simpleNodes,function(v){return v}).classed("update",true);svgNodes.selectAll("*").remove();svgNodes.enter().append("g").attr("id",function(v){return"node_"+v}).attr("name",function(v){return g.node(v).label}).attr("class","node").style("opacity",0);svgNodes.each(function(v){var node=g.node(v),thisGroup=d3.select(this),labelGroup=thisGroup.append("g").attr("class","label"),labelDom=addLabel(labelGroup,node),shape=shapes[node.shape],bbox=_.pick(labelDom.node().getBBox(),"width","height");node.elem=this;if(node.id){thisGroup.attr("id",node.id)}if(node.labelId){labelGroup.attr("id",node.labelId)}util.applyClass(thisGroup,node["class"],(thisGroup.classed("update")?"update ":"")+"node");if(_.has(node,"width")){bbox.width=node.width}if(_.has(node,"height")){bbox.height=node.height}bbox.width+=node.paddingLeft+node.paddingRight;bbox.height+=node.paddingTop+node.paddingBottom;labelGroup.attr("transform","translate("+(node.paddingLeft-node.paddingRight)/2+","+(node.paddingTop-node.paddingBottom)/2+")");var shapeSvg=shape(d3.select(this),bbox,node);util.applyStyle(shapeSvg,node.style);var shapeBBox=shapeSvg.node().getBBox();node.width=shapeBBox.width;node.height=shapeBBox.height});util.applyTransition(svgNodes.exit(),g).style("opacity",0).remove();return svgNodes}},{"./d3":7,"./label/add-label":18,"./lodash":20,"./util":25}],7:[function(require,module,exports){module.exports=window.d3},{}],8:[function(require,module,exports){var dagre;if(require){try{dagre=require("dagre")}catch(e){}}if(!dagre){dagre=window.dagre}module.exports=dagre},{dagre:27}],9:[function(require,module,exports){var graphlib;if(require){try{graphlib=require("graphlib")}catch(e){}}if(!graphlib){graphlib=window.graphlib}module.exports=graphlib},{graphlib:57}],10:[function(require,module,exports){module.exports={node:require("./intersect-node"),circle:require("./intersect-circle"),ellipse:require("./intersect-ellipse"),polygon:require("./intersect-polygon"),rect:require("./intersect-rect")}},{"./intersect-circle":11,"./intersect-ellipse":12,"./intersect-node":14,"./intersect-polygon":15,"./intersect-rect":16}],11:[function(require,module,exports){var intersectEllipse=require("./intersect-ellipse");module.exports=intersectCircle;function intersectCircle(node,rx,point){return intersectEllipse(node,rx,rx,point)}},{"./intersect-ellipse":12}],12:[function(require,module,exports){module.exports=intersectEllipse;function intersectEllipse(node,rx,ry,point){var cx=node.x;var cy=node.y;var px=cx-point.x;var py=cy-point.y;var det=Math.sqrt(rx*rx*py*py+ry*ry*px*px);var dx=Math.abs(rx*ry*px/det);if(point.x0}},{}],14:[function(require,module,exports){module.exports=intersectNode;function intersectNode(node,point){return node.intersect(point)}},{}],15:[function(require,module,exports){var intersectLine=require("./intersect-line");module.exports=intersectPolygon;function intersectPolygon(node,polyPoints,point){var x1=node.x;var y1=node.y;var intersections=[];var minX=Number.POSITIVE_INFINITY,minY=Number.POSITIVE_INFINITY;polyPoints.forEach(function(entry){minX=Math.min(minX,entry.x);minY=Math.min(minY,entry.y)});var left=x1-node.width/2-minX;var top=y1-node.height/2-minY;for(var i=0;i1){intersections.sort(function(p,q){var pdx=p.x-point.x,pdy=p.y-point.y,distp=Math.sqrt(pdx*pdx+pdy*pdy),qdx=q.x-point.x,qdy=q.y-point.y,distq=Math.sqrt(qdx*qdx+qdy*qdy);return distpMath.abs(dx)*h){if(dy<0){h=-h}sx=dy===0?0:h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=dx===0?0:w*dy/dx}return{x:x+sx,y:y+sy}}},{}],17:[function(require,module,exports){var util=require("../util");module.exports=addHtmlLabel;function addHtmlLabel(root,node){var fo=root.append("foreignObject").attr("width","100000");var div=fo.append("xhtml:div");var label=node.label;switch(typeof label){case"function":div.insert(label);break;case"object":div.insert(function(){return label});break;default:div.html(label)}util.applyStyle(div,node.labelStyle);div.style("display","inline-block");div.style("white-space","nowrap");var w,h;div.each(function(){w=this.clientWidth;h=this.clientHeight});fo.attr("width",w).attr("height",h);return fo}},{"../util":25}],18:[function(require,module,exports){var addTextLabel=require("./add-text-label"),addHtmlLabel=require("./add-html-label");module.exports=addLabel;function addLabel(root,node){var label=node.label;var labelSvg=root.append("g");if(typeof label!=="string"||node.labelType==="html"){addHtmlLabel(labelSvg,node)}else{addTextLabel(labelSvg,node)}var labelBBox=labelSvg.node().getBBox();labelSvg.attr("transform","translate("+-labelBBox.width/2+","+-labelBBox.height/2+")");return labelSvg}},{"./add-html-label":17,"./add-text-label":19}],19:[function(require,module,exports){var util=require("../util");module.exports=addTextLabel;function addTextLabel(root,node){var domNode=root.append("text");var lines=processEscapeSequences(node.label).split("\n");for(var i=0;i0;--i){entry=buckets[i].dequeue();if(entry){results=results.concat(removeNode(g,buckets,zeroIdx,entry,true));break}}}}return results}function removeNode(g,buckets,zeroIdx,entry,collectPredecessors){var results=collectPredecessors?[]:undefined;_.each(g.inEdges(entry.v),function(edge){var weight=g.edge(edge),uEntry=g.node(edge.v);if(collectPredecessors){results.push({v:edge.v,w:edge.w})}uEntry.out-=weight;assignBucket(buckets,zeroIdx,uEntry)});_.each(g.outEdges(entry.v),function(edge){var weight=g.edge(edge),w=edge.w,wEntry=g.node(w);wEntry["in"]-=weight;assignBucket(buckets,zeroIdx,wEntry)});g.removeNode(entry.v);return results}function buildState(g,weightFn){var fasGraph=new Graph,maxIn=0,maxOut=0;_.each(g.nodes(),function(v){fasGraph.setNode(v,{v:v,"in":0,out:0})});_.each(g.edges(),function(e){var prevWeight=fasGraph.edge(e.v,e.w)||0,weight=weightFn(e),edgeWeight=prevWeight+weight;fasGraph.setEdge(e.v,e.w,edgeWeight);maxOut=Math.max(maxOut,fasGraph.node(e.v).out+=weight);maxIn=Math.max(maxIn,fasGraph.node(e.w)["in"]+=weight)});var buckets=_.range(maxOut+maxIn+3).map(function(){return new List});var zeroIdx=maxIn+1;_.each(fasGraph.nodes(),function(v){assignBucket(buckets,zeroIdx,fasGraph.node(v))});return{graph:fasGraph,buckets:buckets,zeroIdx:zeroIdx}}function assignBucket(buckets,zeroIdx,entry){if(!entry.out){buckets[0].enqueue(entry)}else if(!entry["in"]){buckets[buckets.length-1].enqueue(entry)}else{buckets[entry.out-entry["in"]+zeroIdx].enqueue(entry)}}},{"./data/list":31,"./graphlib":33,"./lodash":36}],35:[function(require,module,exports){"use strict";var _=require("./lodash"),acyclic=require("./acyclic"),normalize=require("./normalize"),rank=require("./rank"),normalizeRanks=require("./util").normalizeRanks,parentDummyChains=require("./parent-dummy-chains"),removeEmptyRanks=require("./util").removeEmptyRanks,nestingGraph=require("./nesting-graph"),addBorderSegments=require("./add-border-segments"),coordinateSystem=require("./coordinate-system"),order=require("./order"),position=require("./position"),util=require("./util"),Graph=require("./graphlib").Graph;module.exports=layout;function layout(g,opts){var time=opts&&opts.debugTiming?util.time:util.notime;time("layout",function(){var layoutGraph=time(" buildLayoutGraph",function(){return buildLayoutGraph(g)});time(" runLayout",function(){runLayout(layoutGraph,time)});time(" updateInputGraph",function(){updateInputGraph(g,layoutGraph)})})}function runLayout(g,time){time(" makeSpaceForEdgeLabels",function(){makeSpaceForEdgeLabels(g)});time(" removeSelfEdges",function(){removeSelfEdges(g)});time(" acyclic",function(){acyclic.run(g)});time(" nestingGraph.run",function(){nestingGraph.run(g)});time(" rank",function(){rank(util.asNonCompoundGraph(g))});time(" injectEdgeLabelProxies",function(){injectEdgeLabelProxies(g)});time(" removeEmptyRanks",function(){removeEmptyRanks(g)});time(" nestingGraph.cleanup",function(){nestingGraph.cleanup(g)});time(" normalizeRanks",function(){normalizeRanks(g)});time(" assignRankMinMax",function(){assignRankMinMax(g)});time(" removeEdgeLabelProxies",function(){removeEdgeLabelProxies(g)});time(" normalize.run",function(){normalize.run(g)});time(" parentDummyChains",function(){parentDummyChains(g)});time(" addBorderSegments",function(){addBorderSegments(g)});time(" order",function(){order(g)});time(" insertSelfEdges",function(){insertSelfEdges(g)});time(" adjustCoordinateSystem",function(){coordinateSystem.adjust(g)});time(" position",function(){position(g)});time(" positionSelfEdges",function(){positionSelfEdges(g)});time(" removeBorderNodes",function(){removeBorderNodes(g)});time(" normalize.undo",function(){normalize.undo(g)});time(" fixupEdgeLabelCoords",function(){fixupEdgeLabelCoords(g)});time(" undoCoordinateSystem",function(){coordinateSystem.undo(g)});time(" translateGraph",function(){translateGraph(g)});time(" assignNodeIntersects",function(){assignNodeIntersects(g)});time(" reversePoints",function(){reversePointsForReversedEdges(g)});time(" acyclic.undo",function(){acyclic.undo(g)})}function updateInputGraph(inputGraph,layoutGraph){_.each(inputGraph.nodes(),function(v){var inputLabel=inputGraph.node(v),layoutLabel=layoutGraph.node(v);if(inputLabel){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y;if(layoutGraph.children(v).length){inputLabel.width=layoutLabel.width;inputLabel.height=layoutLabel.height}}});_.each(inputGraph.edges(),function(e){var inputLabel=inputGraph.edge(e),layoutLabel=layoutGraph.edge(e);inputLabel.points=layoutLabel.points;if(_.has(layoutLabel,"x")){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y}});inputGraph.graph().width=layoutGraph.graph().width;inputGraph.graph().height=layoutGraph.graph().height}var graphNumAttrs=["nodesep","edgesep","ranksep","marginx","marginy"],graphDefaults={ranksep:50,edgesep:20,nodesep:50,rankdir:"tb"},graphAttrs=["acyclicer","ranker","rankdir","align"],nodeNumAttrs=["width","height"],nodeDefaults={width:0,height:0},edgeNumAttrs=["minlen","weight","width","height","labeloffset"],edgeDefaults={minlen:1,weight:1,width:0,height:0,labeloffset:10,labelpos:"r"},edgeAttrs=["labelpos"];function buildLayoutGraph(inputGraph){var g=new Graph({multigraph:true,compound:true}),graph=canonicalize(inputGraph.graph());g.setGraph(_.merge({},graphDefaults,selectNumberAttrs(graph,graphNumAttrs),_.pick(graph,graphAttrs))); -graph.ranksep/=2;_.each(g.edges(),function(e){var edge=g.edge(e);edge.minlen*=2;if(edge.labelpos.toLowerCase()!=="c"){if(graph.rankdir==="TB"||graph.rankdir==="BT"){edge.width+=edge.labeloffset}else{edge.height+=edge.labeloffset}}})}function injectEdgeLabelProxies(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.width&&edge.height){var v=g.node(e.v),w=g.node(e.w),label={rank:(w.rank-v.rank)/2+v.rank,e:e};util.addDummyNode(g,"edge-proxy",label,"_ep")}})}function assignRankMinMax(g){var maxRank=0;_.each(g.nodes(),function(v){var node=g.node(v);if(node.borderTop){node.minRank=g.node(node.borderTop).rank;node.maxRank=g.node(node.borderBottom).rank;maxRank=_.max(maxRank,node.maxRank)}});g.graph().maxRank=maxRank}function removeEdgeLabelProxies(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="edge-proxy"){g.edge(node.e).labelRank=node.rank;g.removeNode(v)}})}function translateGraph(g){var minX=Number.POSITIVE_INFINITY,maxX=0,minY=Number.POSITIVE_INFINITY,maxY=0,graphLabel=g.graph(),marginX=graphLabel.marginx||0,marginY=graphLabel.marginy||0;function getExtremes(attrs){var x=attrs.x,y=attrs.y,w=attrs.width,h=attrs.height;minX=Math.min(minX,x-w/2);maxX=Math.max(maxX,x+w/2);minY=Math.min(minY,y-h/2);maxY=Math.max(maxY,y+h/2)}_.each(g.nodes(),function(v){getExtremes(g.node(v))});_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){getExtremes(edge)}});minX-=marginX;minY-=marginY;_.each(g.nodes(),function(v){var node=g.node(v);node.x-=minX;node.y-=minY});_.each(g.edges(),function(e){var edge=g.edge(e);_.each(edge.points,function(p){p.x-=minX;p.y-=minY});if(_.has(edge,"x")){edge.x-=minX}if(_.has(edge,"y")){edge.y-=minY}});graphLabel.width=maxX-minX+marginX;graphLabel.height=maxY-minY+marginY}function assignNodeIntersects(g){_.each(g.edges(),function(e){var edge=g.edge(e),nodeV=g.node(e.v),nodeW=g.node(e.w),p1,p2;if(!edge.points){edge.points=[];p1=nodeW;p2=nodeV}else{p1=edge.points[0];p2=edge.points[edge.points.length-1]}edge.points.unshift(util.intersectRect(nodeV,p1));edge.points.push(util.intersectRect(nodeW,p2))})}function fixupEdgeLabelCoords(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){if(edge.labelpos==="l"||edge.labelpos==="r"){edge.width-=edge.labeloffset}switch(edge.labelpos){case"l":edge.x-=edge.width/2+edge.labeloffset;break;case"r":edge.x+=edge.width/2+edge.labeloffset;break}}})}function reversePointsForReversedEdges(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.reversed){edge.points.reverse()}})}function removeBorderNodes(g){_.each(g.nodes(),function(v){if(g.children(v).length){var node=g.node(v),t=g.node(node.borderTop),b=g.node(node.borderBottom),l=g.node(_.last(node.borderLeft)),r=g.node(_.last(node.borderRight));node.width=Math.abs(r.x-l.x);node.height=Math.abs(b.y-t.y);node.x=l.x+node.width/2;node.y=t.y+node.height/2}});_.each(g.nodes(),function(v){if(g.node(v).dummy==="border"){g.removeNode(v)}})}function removeSelfEdges(g){_.each(g.edges(),function(e){if(e.v===e.w){var node=g.node(e.v);if(!node.selfEdges){node.selfEdges=[]}node.selfEdges.push({e:e,label:g.edge(e)});g.removeEdge(e)}})}function insertSelfEdges(g){var layers=util.buildLayerMatrix(g);_.each(layers,function(layer){var orderShift=0;_.each(layer,function(v,i){var node=g.node(v);node.order=i+orderShift;_.each(node.selfEdges,function(selfEdge){util.addDummyNode(g,"selfedge",{width:selfEdge.label.width,height:selfEdge.label.height,rank:node.rank,order:i+ ++orderShift,e:selfEdge.e,label:selfEdge.label},"_se")});delete node.selfEdges})})}function positionSelfEdges(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="selfedge"){var selfNode=g.node(node.e.v),x=selfNode.x+selfNode.width/2,y=selfNode.y,dx=node.x-x,dy=selfNode.height/2;g.setEdge(node.e,node.label);g.removeNode(v);node.label.points=[{x:x+2*dx/3,y:y-dy},{x:x+5*dx/6,y:y-dy},{x:x+dx,y:y},{x:x+5*dx/6,y:y+dy},{x:x+2*dx/3,y:y+dy}];node.label.x=node.x;node.label.y=node.y}})}function selectNumberAttrs(obj,attrs){return _.mapValues(_.pick(obj,attrs),Number)}function canonicalize(attrs){var newAttrs={};_.each(attrs,function(v,k){newAttrs[k.toLowerCase()]=v});return newAttrs}},{"./acyclic":28,"./add-border-segments":29,"./coordinate-system":30,"./graphlib":33,"./lodash":36,"./nesting-graph":37,"./normalize":38,"./order":43,"./parent-dummy-chains":48,"./position":50,"./rank":52,"./util":55}],36:[function(require,module,exports){arguments[4][20][0].apply(exports,arguments)},{dup:20,lodash:77}],37:[function(require,module,exports){var _=require("./lodash"),util=require("./util");module.exports={run:run,cleanup:cleanup};function run(g){var root=util.addDummyNode(g,"root",{},"_root"),depths=treeDepths(g),height=_.max(depths)-1,nodeSep=2*height+1;g.graph().nestingRoot=root;_.each(g.edges(),function(e){g.edge(e).minlen*=nodeSep});var weight=sumWeights(g)+1;_.each(g.children(),function(child){dfs(g,root,nodeSep,weight,height,depths,child)});g.graph().nodeRankFactor=nodeSep}function dfs(g,root,nodeSep,weight,height,depths,v){var children=g.children(v);if(!children.length){if(v!==root){g.setEdge(root,v,{weight:0,minlen:nodeSep})}return}var top=util.addBorderNode(g,"_bt"),bottom=util.addBorderNode(g,"_bb"),label=g.node(v);g.setParent(top,v);label.borderTop=top;g.setParent(bottom,v);label.borderBottom=bottom;_.each(children,function(child){dfs(g,root,nodeSep,weight,height,depths,child);var childNode=g.node(child),childTop=childNode.borderTop?childNode.borderTop:child,childBottom=childNode.borderBottom?childNode.borderBottom:child,thisWeight=childNode.borderTop?weight:2*weight,minlen=childTop!==childBottom?1:height-depths[v]+1;g.setEdge(top,childTop,{weight:thisWeight,minlen:minlen,nestingEdge:true});g.setEdge(childBottom,bottom,{weight:thisWeight,minlen:minlen,nestingEdge:true})});if(!g.parent(v)){g.setEdge(root,top,{weight:0,minlen:height+depths[v]})}}function treeDepths(g){var depths={};function dfs(v,depth){var children=g.children(v);if(children&&children.length){_.each(children,function(child){dfs(child,depth+1)})}depths[v]=depth}_.each(g.children(),function(v){dfs(v,1)});return depths}function sumWeights(g){return _.reduce(g.edges(),function(acc,e){return acc+g.edge(e).weight},0)}function cleanup(g){var graphLabel=g.graph();g.removeNode(graphLabel.nestingRoot);delete graphLabel.nestingRoot;_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.nestingEdge){g.removeEdge(e)}})}},{"./lodash":36,"./util":55}],38:[function(require,module,exports){"use strict";var _=require("./lodash"),util=require("./util");module.exports={run:run,undo:undo};function run(g){g.graph().dummyChains=[];_.each(g.edges(),function(edge){normalizeEdge(g,edge)})}function normalizeEdge(g,e){var v=e.v,vRank=g.node(v).rank,w=e.w,wRank=g.node(w).rank,name=e.name,edgeLabel=g.edge(e),labelRank=edgeLabel.labelRank;if(wRank===vRank+1)return;g.removeEdge(e);var dummy,attrs,i;for(i=0,++vRank;vRank0){if(index%2){weightSum+=tree[index+1]}index=index-1>>1;tree[index]+=entry.weight}cc+=entry.weight*weightSum}));return cc}},{"../lodash":36}],43:[function(require,module,exports){"use strict";var _=require("../lodash"),initOrder=require("./init-order"),crossCount=require("./cross-count"),sortSubgraph=require("./sort-subgraph"),buildLayerGraph=require("./build-layer-graph"),addSubgraphConstraints=require("./add-subgraph-constraints"),Graph=require("../graphlib").Graph,util=require("../util");module.exports=order;function order(g){var maxRank=util.maxRank(g),downLayerGraphs=buildLayerGraphs(g,_.range(1,maxRank+1),"inEdges"),upLayerGraphs=buildLayerGraphs(g,_.range(maxRank-1,-1,-1),"outEdges");var layering=initOrder(g);assignOrder(g,layering);var bestCC=Number.POSITIVE_INFINITY,best;for(var i=0,lastBest=0;lastBest<4;++i,++lastBest){sweepLayerGraphs(i%2?downLayerGraphs:upLayerGraphs,i%4>=2);layering=util.buildLayerMatrix(g);var cc=crossCount(g,layering);if(cc=vEntry.barycenter){mergeEntries(vEntry,uEntry)}}}function handleOut(vEntry){return function(wEntry){wEntry["in"].push(vEntry);if(--wEntry.indegree===0){sourceSet.push(wEntry)}}}while(sourceSet.length){var entry=sourceSet.pop();entries.push(entry);_.each(entry["in"].reverse(),handleIn(entry));_.each(entry.out,handleOut(entry))}return _.chain(entries).filter(function(entry){return!entry.merged}).map(function(entry){return _.pick(entry,["vs","i","barycenter","weight"])}).value()}function mergeEntries(target,source){var sum=0,weight=0;if(target.weight){sum+=target.barycenter*target.weight;weight+=target.weight}if(source.weight){sum+=source.barycenter*source.weight;weight+=source.weight}target.vs=source.vs.concat(target.vs);target.barycenter=sum/weight;target.weight=weight;target.i=Math.min(source.i,target.i);source.merged=true}},{"../lodash":36}],46:[function(require,module,exports){var _=require("../lodash"),barycenter=require("./barycenter"),resolveConflicts=require("./resolve-conflicts"),sort=require("./sort");module.exports=sortSubgraph;function sortSubgraph(g,v,cg,biasRight){var movable=g.children(v),node=g.node(v),bl=node?node.borderLeft:undefined,br=node?node.borderRight:undefined,subgraphs={};if(bl){movable=_.filter(movable,function(w){return w!==bl&&w!==br})}var barycenters=barycenter(g,movable);_.each(barycenters,function(entry){if(g.children(entry.v).length){var subgraphResult=sortSubgraph(g,entry.v,cg,biasRight);subgraphs[entry.v]=subgraphResult;if(_.has(subgraphResult,"barycenter")){mergeBarycenters(entry,subgraphResult)}}});var entries=resolveConflicts(barycenters,cg);expandSubgraphs(entries,subgraphs);var result=sort(entries,biasRight);if(bl){result.vs=_.flatten([bl,result.vs,br],true);if(g.predecessors(bl).length){var blPred=g.node(g.predecessors(bl)[0]),brPred=g.node(g.predecessors(br)[0]);if(!_.has(result,"barycenter")){result.barycenter=0;result.weight=0}result.barycenter=(result.barycenter*result.weight+blPred.order+brPred.order)/(result.weight+2);result.weight+=2}}return result}function expandSubgraphs(entries,subgraphs){_.each(entries,function(entry){entry.vs=_.flatten(entry.vs.map(function(v){if(subgraphs[v]){return subgraphs[v].vs}return v}),true)})}function mergeBarycenters(target,other){if(!_.isUndefined(target.barycenter)){target.barycenter=(target.barycenter*target.weight+other.barycenter*other.weight)/(target.weight+other.weight);target.weight+=other.weight}else{target.barycenter=other.barycenter;target.weight=other.weight}}},{"../lodash":36,"./barycenter":40,"./resolve-conflicts":45,"./sort":47}],47:[function(require,module,exports){var _=require("../lodash"),util=require("../util");module.exports=sort;function sort(entries,biasRight){var parts=util.partition(entries,function(entry){return _.has(entry,"barycenter")});var sortable=parts.lhs,unsortable=_.sortBy(parts.rhs,function(entry){return-entry.i}),vs=[],sum=0,weight=0,vsIndex=0;sortable.sort(compareWithBias(!!biasRight));vsIndex=consumeUnsortable(vs,unsortable,vsIndex);_.each(sortable,function(entry){vsIndex+=entry.vs.length;vs.push(entry.vs);sum+=entry.barycenter*entry.weight;weight+=entry.weight;vsIndex=consumeUnsortable(vs,unsortable,vsIndex)});var result={vs:_.flatten(vs,true)};if(weight){result.barycenter=sum/weight;result.weight=weight}return result}function consumeUnsortable(vs,unsortable,index){var last;while(unsortable.length&&(last=_.last(unsortable)).i<=index){unsortable.pop();vs.push(last.vs);index++}return index}function compareWithBias(bias){return function(entryV,entryW){if(entryV.barycenterentryW.barycenter){return 1}return!bias?entryV.i-entryW.i:entryW.i-entryV.i}}},{"../lodash":36,"../util":55}],48:[function(require,module,exports){var _=require("./lodash");module.exports=parentDummyChains;function parentDummyChains(g){var postorderNums=postorder(g);_.each(g.graph().dummyChains,function(v){var node=g.node(v),edgeObj=node.edgeObj,pathData=findPath(g,postorderNums,edgeObj.v,edgeObj.w),path=pathData.path,lca=pathData.lca,pathIdx=0,pathV=path[pathIdx],ascending=true;while(v!==edgeObj.w){node=g.node(v);if(ascending){while((pathV=path[pathIdx])!==lca&&g.node(pathV).maxRanklow||lim>postorderNums[parent].lim));lca=parent;parent=w;while((parent=g.parent(parent))!==lca){wPath.push(parent)}return{path:vPath.concat(wPath.reverse()),lca:lca}}function postorder(g){var result={},lim=0;function dfs(v){var low=lim;_.each(g.children(v),dfs);result[v]={low:low,lim:lim++}}_.each(g.children(),dfs);return result}},{"./lodash":36}],49:[function(require,module,exports){"use strict";var _=require("../lodash"),Graph=require("../graphlib").Graph,util=require("../util");module.exports={positionX:positionX,findType1Conflicts:findType1Conflicts,findType2Conflicts:findType2Conflicts,addConflict:addConflict,hasConflict:hasConflict,verticalAlignment:verticalAlignment,horizontalCompaction:horizontalCompaction,alignCoordinates:alignCoordinates,findSmallestWidthAlignment:findSmallestWidthAlignment,balance:balance};function findType1Conflicts(g,layering){var conflicts={};function visitLayer(prevLayer,layer){var k0=0,scanPos=0,prevLayerLength=prevLayer.length,lastNode=_.last(layer);_.each(layer,function(v,i){var w=findOtherInnerSegmentNode(g,v),k1=w?g.node(w).order:prevLayerLength;if(w||v===lastNode){_.each(layer.slice(scanPos,i+1),function(scanNode){_.each(g.predecessors(scanNode),function(u){var uLabel=g.node(u),uPos=uLabel.order;if((uPosnextNorthBorder)){addConflict(conflicts,u,v)}})}})}function visitLayer(north,south){var prevNorthPos=-1,nextNorthPos,southPos=0;_.each(south,function(v,southLookahead){if(g.node(v).dummy==="border"){var predecessors=g.predecessors(v);if(predecessors.length){nextNorthPos=g.node(predecessors[0]).order;scan(south,southPos,southLookahead,prevNorthPos,nextNorthPos);southPos=southLookahead;prevNorthPos=nextNorthPos}}scan(south,southPos,south.length,nextNorthPos,north.length)});return south}_.reduce(layering,visitLayer);return conflicts}function findOtherInnerSegmentNode(g,v){if(g.node(v).dummy){return _.find(g.predecessors(v),function(u){return g.node(u).dummy})}}function addConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}var conflictsV=conflicts[v];if(!conflictsV){conflicts[v]=conflictsV={}}conflictsV[w]=true}function hasConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}return _.has(conflicts[v],w)}function verticalAlignment(g,layering,conflicts,neighborFn){var root={},align={},pos={};_.each(layering,function(layer){_.each(layer,function(v,order){root[v]=v;align[v]=v;pos[v]=order})});_.each(layering,function(layer){var prevIdx=-1;_.each(layer,function(v){var ws=neighborFn(v);if(ws.length){ws=_.sortBy(ws,function(w){return pos[w]});var mp=(ws.length-1)/2;for(var i=Math.floor(mp),il=Math.ceil(mp);i<=il;++i){var w=ws[i];if(align[v]===v&&prevIdxwLabel.lim){tailLabel=wLabel;flip=true}var candidates=_.filter(g.edges(),function(edge){return flip===isDescendant(t,t.node(edge.v),tailLabel)&&flip!==isDescendant(t,t.node(edge.w),tailLabel)});return _.min(candidates,function(edge){return slack(g,edge)})}function exchangeEdges(t,g,e,f){var v=e.v,w=e.w;t.removeEdge(v,w);t.setEdge(f.v,f.w,{});initLowLimValues(t);initCutValues(t,g);updateRanks(t,g)}function updateRanks(t,g){var root=_.find(t.nodes(),function(v){return!g.node(v).parent}),vs=preorder(t,root);vs=vs.slice(1);_.each(vs,function(v){var parent=t.node(v).parent,edge=g.edge(v,parent),flipped=false;if(!edge){edge=g.edge(parent,v);flipped=true}g.node(v).rank=g.node(parent).rank+(flipped?edge.minlen:-edge.minlen)})}function isTreeEdge(tree,u,v){return tree.hasEdge(u,v)}function isDescendant(tree,vLabel,rootLabel){return rootLabel.low<=vLabel.lim&&vLabel.lim<=rootLabel.lim}},{"../graphlib":33,"../lodash":36,"../util":55,"./feasible-tree":51,"./util":54}],54:[function(require,module,exports){"use strict";var _=require("../lodash");module.exports={longestPath:longestPath,slack:slack};function longestPath(g){var visited={};function dfs(v){var label=g.node(v);if(_.has(visited,v)){return label.rank}visited[v]=true;var rank=_.min(_.map(g.outEdges(v),function(e){return dfs(e.w)-g.edge(e).minlen}));if(rank===Number.POSITIVE_INFINITY){rank=0}return label.rank=rank}_.each(g.sources(),dfs)}function slack(g,e){return g.node(e.w).rank-g.node(e.v).rank-g.edge(e).minlen}},{"../lodash":36}],55:[function(require,module,exports){"use strict";var _=require("./lodash"),Graph=require("./graphlib").Graph;module.exports={addDummyNode:addDummyNode,simplify:simplify,asNonCompoundGraph:asNonCompoundGraph,successorWeights:successorWeights,predecessorWeights:predecessorWeights,intersectRect:intersectRect,buildLayerMatrix:buildLayerMatrix,normalizeRanks:normalizeRanks,removeEmptyRanks:removeEmptyRanks,addBorderNode:addBorderNode,maxRank:maxRank,partition:partition,time:time,notime:notime};function addDummyNode(g,type,attrs,name){var v;do{v=_.uniqueId(name)}while(g.hasNode(v));attrs.dummy=type;g.setNode(v,attrs);return v}function simplify(g){var simplified=(new Graph).setGraph(g.graph());_.each(g.nodes(),function(v){simplified.setNode(v,g.node(v))});_.each(g.edges(),function(e){var simpleLabel=simplified.edge(e.v,e.w)||{weight:0,minlen:1},label=g.edge(e);simplified.setEdge(e.v,e.w,{weight:simpleLabel.weight+label.weight,minlen:Math.max(simpleLabel.minlen,label.minlen)})});return simplified}function asNonCompoundGraph(g){ -var simplified=new Graph({multigraph:g.isMultigraph()}).setGraph(g.graph());_.each(g.nodes(),function(v){if(!g.children(v).length){simplified.setNode(v,g.node(v))}});_.each(g.edges(),function(e){simplified.setEdge(e,g.edge(e))});return simplified}function successorWeights(g){var weightMap=_.map(g.nodes(),function(v){var sucs={};_.each(g.outEdges(v),function(e){sucs[e.w]=(sucs[e.w]||0)+g.edge(e).weight});return sucs});return _.zipObject(g.nodes(),weightMap)}function predecessorWeights(g){var weightMap=_.map(g.nodes(),function(v){var preds={};_.each(g.inEdges(v),function(e){preds[e.v]=(preds[e.v]||0)+g.edge(e).weight});return preds});return _.zipObject(g.nodes(),weightMap)}function intersectRect(rect,point){var x=rect.x;var y=rect.y;var dx=point.x-x;var dy=point.y-y;var w=rect.width/2;var h=rect.height/2;if(!dx&&!dy){throw new Error("Not possible to find intersection inside of the rectangle")}var sx,sy;if(Math.abs(dy)*w>Math.abs(dx)*h){if(dy<0){h=-h}sx=h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=w*dy/dx}return{x:x+sx,y:y+sy}}function buildLayerMatrix(g){var layering=_.map(_.range(maxRank(g)+1),function(){return[]});_.each(g.nodes(),function(v){var node=g.node(v),rank=node.rank;if(!_.isUndefined(rank)){layering[rank][node.order]=v}});return layering}function normalizeRanks(g){var min=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));_.each(g.nodes(),function(v){var node=g.node(v);if(_.has(node,"rank")){node.rank-=min}})}function removeEmptyRanks(g){var offset=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));var layers=[];_.each(g.nodes(),function(v){var rank=g.node(v).rank-offset;if(!_.has(layers,rank)){layers[rank]=[]}layers[rank].push(v)});var delta=0,nodeRankFactor=g.graph().nodeRankFactor;_.each(layers,function(vs,i){if(_.isUndefined(vs)&&i%nodeRankFactor!==0){--delta}else if(delta){_.each(vs,function(v){g.node(v).rank+=delta})}})}function addBorderNode(g,prefix,rank,order){var node={width:0,height:0};if(arguments.length>=4){node.rank=rank;node.order=order}return addDummyNode(g,"border",node,prefix)}function maxRank(g){return _.max(_.map(g.nodes(),function(v){var rank=g.node(v).rank;if(!_.isUndefined(rank)){return rank}}))}function partition(collection,fn){var result={lhs:[],rhs:[]};_.each(collection,function(value){if(fn(value)){result.lhs.push(value)}else{result.rhs.push(value)}});return result}function time(name,fn){var start=_.now();try{return fn()}finally{console.log(name+" time: "+(_.now()-start)+"ms")}}function notime(name,fn){return fn()}},{"./graphlib":33,"./lodash":36}],56:[function(require,module,exports){module.exports="0.7.1"},{}],57:[function(require,module,exports){var lib=require("./lib");module.exports={Graph:lib.Graph,json:require("./lib/json"),alg:require("./lib/alg"),version:lib.version}},{"./lib":73,"./lib/alg":64,"./lib/json":74}],58:[function(require,module,exports){var _=require("../lodash");module.exports=components;function components(g){var visited={},cmpts=[],cmpt;function dfs(v){if(_.has(visited,v))return;visited[v]=true;cmpt.push(v);_.each(g.successors(v),dfs);_.each(g.predecessors(v),dfs)}_.each(g.nodes(),function(v){cmpt=[];dfs(v);if(cmpt.length){cmpts.push(cmpt)}});return cmpts}},{"../lodash":75}],59:[function(require,module,exports){var _=require("../lodash");module.exports=dfs;function dfs(g,vs,order){if(!_.isArray(vs)){vs=[vs]}var acc=[],visited={};_.each(vs,function(v){if(!g.hasNode(v)){throw new Error("Graph does not have node: "+v)}doDfs(g,v,order==="post",visited,acc)});return acc}function doDfs(g,v,postorder,visited,acc){if(!_.has(visited,v)){visited[v]=true;if(!postorder){acc.push(v)}_.each(g.neighbors(v),function(w){doDfs(g,w,postorder,visited,acc)});if(postorder){acc.push(v)}}}},{"../lodash":75}],60:[function(require,module,exports){var dijkstra=require("./dijkstra"),_=require("../lodash");module.exports=dijkstraAll;function dijkstraAll(g,weightFunc,edgeFunc){return _.transform(g.nodes(),function(acc,v){acc[v]=dijkstra(g,v,weightFunc,edgeFunc)},{})}},{"../lodash":75,"./dijkstra":61}],61:[function(require,module,exports){var _=require("../lodash"),PriorityQueue=require("../data/priority-queue");module.exports=dijkstra;var DEFAULT_WEIGHT_FUNC=_.constant(1);function dijkstra(g,source,weightFn,edgeFn){return runDijkstra(g,String(source),weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runDijkstra(g,source,weightFn,edgeFn){var results={},pq=new PriorityQueue,v,vEntry;var updateNeighbors=function(edge){var w=edge.v!==v?edge.v:edge.w,wEntry=results[w],weight=weightFn(edge),distance=vEntry.distance+weight;if(weight<0){throw new Error("dijkstra does not allow negative edge weights. "+"Bad edge: "+edge+" Weight: "+weight)}if(distance0){v=pq.removeMin();vEntry=results[v];if(vEntry.distance===Number.POSITIVE_INFINITY){break}edgeFn(v).forEach(updateNeighbors)}return results}},{"../data/priority-queue":71,"../lodash":75}],62:[function(require,module,exports){var _=require("../lodash"),tarjan=require("./tarjan");module.exports=findCycles;function findCycles(g){return _.filter(tarjan(g),function(cmpt){return cmpt.length>1})}},{"../lodash":75,"./tarjan":69}],63:[function(require,module,exports){var _=require("../lodash");module.exports=floydWarshall;var DEFAULT_WEIGHT_FUNC=_.constant(1);function floydWarshall(g,weightFn,edgeFn){return runFloydWarshall(g,weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runFloydWarshall(g,weightFn,edgeFn){var results={},nodes=g.nodes();nodes.forEach(function(v){results[v]={};results[v][v]={distance:0};nodes.forEach(function(w){if(v!==w){results[v][w]={distance:Number.POSITIVE_INFINITY}}});edgeFn(v).forEach(function(edge){var w=edge.v===v?edge.w:edge.v,d=weightFn(edge);results[v][w]={distance:d,predecessor:v}})});nodes.forEach(function(k){var rowK=results[k];nodes.forEach(function(i){var rowI=results[i];nodes.forEach(function(j){var ik=rowI[k];var kj=rowK[j];var ij=rowI[j];var altDistance=ik.distance+kj.distance;if(altDistance0){v=pq.removeMin();if(_.has(parents,v)){result.setEdge(v,parents[v])}else if(init){throw new Error("Input graph is not connected: "+g)}else{init=true}g.nodeEdges(v).forEach(updateNeighbors)}return result}},{"../data/priority-queue":71,"../graph":72,"../lodash":75}],69:[function(require,module,exports){var _=require("../lodash");module.exports=tarjan;function tarjan(g){var index=0,stack=[],visited={},results=[];function dfs(v){var entry=visited[v]={onStack:true,lowlink:index,index:index++};stack.push(v);g.successors(v).forEach(function(w){if(!_.has(visited,w)){dfs(w);entry.lowlink=Math.min(entry.lowlink,visited[w].lowlink)}else if(visited[w].onStack){entry.lowlink=Math.min(entry.lowlink,visited[w].index)}});if(entry.lowlink===entry.index){var cmpt=[],w;do{w=stack.pop();visited[w].onStack=false;cmpt.push(w)}while(v!==w);results.push(cmpt)}}g.nodes().forEach(function(v){if(!_.has(visited,v)){dfs(v)}});return results}},{"../lodash":75}],70:[function(require,module,exports){var _=require("../lodash");module.exports=topsort;topsort.CycleException=CycleException;function topsort(g){var visited={},stack={},results=[];function visit(node){if(_.has(stack,node)){throw new CycleException}if(!_.has(visited,node)){stack[node]=true;visited[node]=true;_.each(g.predecessors(node),visit);delete stack[node];results.push(node)}}_.each(g.sinks(),visit);if(_.size(visited)!==g.nodeCount()){throw new CycleException}return results}function CycleException(){}},{"../lodash":75}],71:[function(require,module,exports){var _=require("../lodash");module.exports=PriorityQueue;function PriorityQueue(){this._arr=[];this._keyIndices={}}PriorityQueue.prototype.size=function(){return this._arr.length};PriorityQueue.prototype.keys=function(){return this._arr.map(function(x){return x.key})};PriorityQueue.prototype.has=function(key){return _.has(this._keyIndices,key)};PriorityQueue.prototype.priority=function(key){var index=this._keyIndices[key];if(index!==undefined){return this._arr[index].priority}};PriorityQueue.prototype.min=function(){if(this.size()===0){throw new Error("Queue underflow")}return this._arr[0].key};PriorityQueue.prototype.add=function(key,priority){var keyIndices=this._keyIndices;key=String(key);if(!_.has(keyIndices,key)){var arr=this._arr;var index=arr.length;keyIndices[key]=index;arr.push({key:key,priority:priority});this._decrease(index);return true}return false};PriorityQueue.prototype.removeMin=function(){this._swap(0,this._arr.length-1);var min=this._arr.pop();delete this._keyIndices[min.key];this._heapify(0);return min.key};PriorityQueue.prototype.decrease=function(key,priority){var index=this._keyIndices[key];if(priority>this._arr[index].priority){throw new Error("New priority is greater than current priority. "+"Key: "+key+" Old: "+this._arr[index].priority+" New: "+priority)}this._arr[index].priority=priority;this._decrease(index)};PriorityQueue.prototype._heapify=function(i){var arr=this._arr;var l=2*i,r=l+1,largest=i;if(l>1;if(arr[parent].priority1){this.setNode(v,value)}else{this.setNode(v)}},this);return this};Graph.prototype.setNode=function(v,value){if(_.has(this._nodes,v)){if(arguments.length>1){this._nodes[v]=value}return this}this._nodes[v]=arguments.length>1?value:this._defaultNodeLabelFn(v);if(this._isCompound){this._parent[v]=GRAPH_NODE;this._children[v]={};this._children[GRAPH_NODE][v]=true}this._in[v]={};this._preds[v]={};this._out[v]={};this._sucs[v]={};++this._nodeCount;return this};Graph.prototype.node=function(v){return this._nodes[v]};Graph.prototype.hasNode=function(v){return _.has(this._nodes,v)};Graph.prototype.removeNode=function(v){var self=this;if(_.has(this._nodes,v)){var removeEdge=function(e){self.removeEdge(self._edgeObjs[e])};delete this._nodes[v];if(this._isCompound){this._removeFromParentsChildList(v);delete this._parent[v];_.each(this.children(v),function(child){this.setParent(child)},this);delete this._children[v]}_.each(_.keys(this._in[v]),removeEdge);delete this._in[v];delete this._preds[v];_.each(_.keys(this._out[v]),removeEdge);delete this._out[v];delete this._sucs[v];--this._nodeCount}return this};Graph.prototype.setParent=function(v,parent){if(!this._isCompound){throw new Error("Cannot set parent in a non-compound graph")}if(_.isUndefined(parent)){parent=GRAPH_NODE}else{for(var ancestor=parent;!_.isUndefined(ancestor);ancestor=this.parent(ancestor)){if(ancestor===v){throw new Error("Setting "+parent+" as parent of "+v+" would create create a cycle")}}this.setNode(parent)}this.setNode(v);this._removeFromParentsChildList(v);this._parent[v]=parent;this._children[parent][v]=true;return this};Graph.prototype._removeFromParentsChildList=function(v){delete this._children[this._parent[v]][v]};Graph.prototype.parent=function(v){if(this._isCompound){var parent=this._parent[v];if(parent!==GRAPH_NODE){return parent}}};Graph.prototype.children=function(v){if(_.isUndefined(v)){v=GRAPH_NODE}if(this._isCompound){var children=this._children[v];if(children){return _.keys(children)}}else if(v===GRAPH_NODE){return this.nodes()}else if(this.hasNode(v)){return[]}};Graph.prototype.predecessors=function(v){var predsV=this._preds[v];if(predsV){return _.keys(predsV)}};Graph.prototype.successors=function(v){var sucsV=this._sucs[v];if(sucsV){return _.keys(sucsV)}};Graph.prototype.neighbors=function(v){var preds=this.predecessors(v);if(preds){return _.union(preds,this.successors(v))}};Graph.prototype.setDefaultEdgeLabel=function(newDefault){if(!_.isFunction(newDefault)){newDefault=_.constant(newDefault)}this._defaultEdgeLabelFn=newDefault;return this};Graph.prototype.edgeCount=function(){return this._edgeCount};Graph.prototype.edges=function(){return _.values(this._edgeObjs)};Graph.prototype.setPath=function(vs,value){var self=this,args=arguments;_.reduce(vs,function(v,w){if(args.length>1){self.setEdge(v,w,value)}else{self.setEdge(v,w)}return w});return this};Graph.prototype.setEdge=function(){var v,w,name,value,valueSpecified=false;if(_.isPlainObject(arguments[0])){v=arguments[0].v;w=arguments[0].w;name=arguments[0].name;if(arguments.length===2){value=arguments[1];valueSpecified=true}}else{v=arguments[0];w=arguments[1];name=arguments[3];if(arguments.length>2){value=arguments[2];valueSpecified=true}}v=""+v;w=""+w;if(!_.isUndefined(name)){name=""+name}var e=edgeArgsToId(this._isDirected,v,w,name);if(_.has(this._edgeLabels,e)){if(valueSpecified){this._edgeLabels[e]=value}return this}if(!_.isUndefined(name)&&!this._isMultigraph){throw new Error("Cannot set a named edge when isMultigraph = false")}this.setNode(v);this.setNode(w);this._edgeLabels[e]=valueSpecified?value:this._defaultEdgeLabelFn(v,w,name);var edgeObj=edgeArgsToObj(this._isDirected,v,w,name);v=edgeObj.v;w=edgeObj.w;Object.freeze(edgeObj);this._edgeObjs[e]=edgeObj;incrementOrInitEntry(this._preds[w],v);incrementOrInitEntry(this._sucs[v],w);this._in[w][e]=edgeObj;this._out[v][e]=edgeObj;this._edgeCount++;return this};Graph.prototype.edge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return this._edgeLabels[e]};Graph.prototype.hasEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return _.has(this._edgeLabels,e)};Graph.prototype.removeEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name),edge=this._edgeObjs[e];if(edge){v=edge.v;w=edge.w;delete this._edgeLabels[e];delete this._edgeObjs[e];decrementOrRemoveEntry(this._preds[w],v);decrementOrRemoveEntry(this._sucs[v],w);delete this._in[w][e];delete this._out[v][e];this._edgeCount--}return this};Graph.prototype.inEdges=function(v,u){var inV=this._in[v];if(inV){var edges=_.values(inV);if(!u){return edges}return _.filter(edges,function(edge){return edge.v===u})}};Graph.prototype.outEdges=function(v,w){var outV=this._out[v];if(outV){var edges=_.values(outV);if(!w){return edges}return _.filter(edges,function(edge){return edge.w===w})}};Graph.prototype.nodeEdges=function(v,w){var inEdges=this.inEdges(v,w);if(inEdges){return inEdges.concat(this.outEdges(v,w))}};function incrementOrInitEntry(map,k){if(_.has(map,k)){map[k]++}else{map[k]=1}}function decrementOrRemoveEntry(map,k){if(!--map[k]){delete map[k]}}function edgeArgsToId(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}return v+EDGE_KEY_DELIM+w+EDGE_KEY_DELIM+(_.isUndefined(name)?DEFAULT_EDGE_NAME:name)}function edgeArgsToObj(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}var edgeObj={v:v,w:w};if(name){edgeObj.name=name}return edgeObj}function edgeObjToId(isDirected,edgeObj){return edgeArgsToId(isDirected,edgeObj.v,edgeObj.w,edgeObj.name)}},{"./lodash":75}],73:[function(require,module,exports){module.exports={Graph:require("./graph"),version:require("./version")}},{"./graph":72,"./version":76}],74:[function(require,module,exports){var _=require("./lodash"),Graph=require("./graph");module.exports={write:write,read:read};function write(g){var json={options:{directed:g.isDirected(),multigraph:g.isMultigraph(),compound:g.isCompound()},nodes:writeNodes(g),edges:writeEdges(g)};if(!_.isUndefined(g.graph())){json.value=_.clone(g.graph())}return json}function writeNodes(g){return _.map(g.nodes(),function(v){var nodeValue=g.node(v),parent=g.parent(v),node={v:v};if(!_.isUndefined(nodeValue)){node.value=nodeValue}if(!_.isUndefined(parent)){node.parent=parent}return node})}function writeEdges(g){return _.map(g.edges(),function(e){var edgeValue=g.edge(e),edge={v:e.v,w:e.w};if(!_.isUndefined(e.name)){edge.name=e.name}if(!_.isUndefined(edgeValue)){edge.value=edgeValue}return edge})}function read(json){var g=new Graph(json.options).setGraph(json.value);_.each(json.nodes,function(entry){g.setNode(entry.v,entry.value);if(entry.parent){g.setParent(entry.v,entry.parent)}});_.each(json.edges,function(entry){g.setEdge({v:entry.v,w:entry.w,name:entry.name},entry.value)});return g}},{"./graph":72,"./lodash":75}],75:[function(require,module,exports){arguments[4][20][0].apply(exports,arguments)},{dup:20,lodash:77}],76:[function(require,module,exports){module.exports="1.0.1"},{}],77:[function(require,module,exports){(function(global){(function(){var undefined;var arrayPool=[],objectPool=[];var idCounter=0;var keyPrefix=+new Date+"";var largeArraySize=75;var maxPoolSize=40;var whitespace=" \f \ufeff"+"\n\r\u2028\u2029"+" ᠎              ";var reEmptyStringLeading=/\b__p \+= '';/g,reEmptyStringMiddle=/\b(__p \+=) '' \+/g,reEmptyStringTrailing=/(__e\(.*?\)|\b__t\)) \+\n'';/g;var reEsTemplate=/\$\{([^\\}]*(?:\\.[^\\}]*)*)\}/g;var reFlags=/\w*$/;var reFuncName=/^\s*function[ \n\r\t]+\w/;var reInterpolate=/<%=([\s\S]+?)%>/g;var reLeadingSpacesAndZeros=RegExp("^["+whitespace+"]*0+(?=.$)");var reNoMatch=/($^)/;var reThis=/\bthis\b/;var reUnescapedString=/['\n\r\t\u2028\u2029\\]/g;var contextProps=["Array","Boolean","Date","Function","Math","Number","Object","RegExp","String","_","attachEvent","clearTimeout","isFinite","isNaN","parseInt","setTimeout"];var templateCounter=0;var argsClass="[object Arguments]",arrayClass="[object Array]",boolClass="[object Boolean]",dateClass="[object Date]",funcClass="[object Function]",numberClass="[object Number]",objectClass="[object Object]",regexpClass="[object RegExp]",stringClass="[object String]";var cloneableClasses={};cloneableClasses[funcClass]=false;cloneableClasses[argsClass]=cloneableClasses[arrayClass]=cloneableClasses[boolClass]=cloneableClasses[dateClass]=cloneableClasses[numberClass]=cloneableClasses[objectClass]=cloneableClasses[regexpClass]=cloneableClasses[stringClass]=true;var debounceOptions={leading:false,maxWait:0,trailing:false};var descriptor={configurable:false,enumerable:false,value:null,writable:false};var objectTypes={"boolean":false,"function":true,object:true,number:false,string:false,undefined:false};var stringEscapes={"\\":"\\","'":"'","\n":"n","\r":"r"," ":"t","\u2028":"u2028","\u2029":"u2029"};var root=objectTypes[typeof window]&&window||this;var freeExports=objectTypes[typeof exports]&&exports&&!exports.nodeType&&exports;var freeModule=objectTypes[typeof module]&&module&&!module.nodeType&&module;var moduleExports=freeModule&&freeModule.exports===freeExports&&freeExports;var freeGlobal=objectTypes[typeof global]&&global;if(freeGlobal&&(freeGlobal.global===freeGlobal||freeGlobal.window===freeGlobal)){root=freeGlobal}function baseIndexOf(array,value,fromIndex){var index=(fromIndex||0)-1,length=array?array.length:0;while(++index-1?0:-1:cache?0:-1}function cachePush(value){var cache=this.cache,type=typeof value;if(type=="boolean"||value==null){cache[value]=true}else{if(type!="number"&&type!="string"){type="object"}var key=type=="number"?value:keyPrefix+value,typeCache=cache[type]||(cache[type]={});if(type=="object"){(typeCache[key]||(typeCache[key]=[])).push(value)}else{typeCache[key]=true}}}function charAtCallback(value){return value.charCodeAt(0)}function compareAscending(a,b){var ac=a.criteria,bc=b.criteria,index=-1,length=ac.length;while(++indexother||typeof value=="undefined"){return 1}if(value/g,evaluate:/<%([\s\S]+?)%>/g,interpolate:reInterpolate,variable:"",imports:{_:lodash}};function baseBind(bindData){var func=bindData[0],partialArgs=bindData[2],thisArg=bindData[4];function bound(){if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(this instanceof bound){var thisBinding=baseCreate(func.prototype),result=func.apply(thisBinding,args||arguments);return isObject(result)?result:thisBinding}return func.apply(thisArg,args||arguments)}setBindData(bound,bindData);return bound}function baseClone(value,isDeep,callback,stackA,stackB){if(callback){var result=callback(value);if(typeof result!="undefined"){return result}}var isObj=isObject(value);if(isObj){var className=toString.call(value);if(!cloneableClasses[className]){return value}var ctor=ctorByClass[className];switch(className){case boolClass:case dateClass:return new ctor(+value);case numberClass:case stringClass:return new ctor(value);case regexpClass:result=ctor(value.source,reFlags.exec(value));result.lastIndex=value.lastIndex;return result}}else{return value}var isArr=isArray(value);if(isDeep){var initedStack=!stackA;stackA||(stackA=getArray());stackB||(stackB=getArray());var length=stackA.length;while(length--){if(stackA[length]==value){return stackB[length]}}result=isArr?ctor(value.length):{}}else{result=isArr?slice(value):assign({},value)}if(isArr){if(hasOwnProperty.call(value,"index")){result.index=value.index}if(hasOwnProperty.call(value,"input")){result.input=value.input}}if(!isDeep){return result}stackA.push(value);stackB.push(result);(isArr?forEach:forOwn)(value,function(objValue,key){result[key]=baseClone(objValue,isDeep,callback,stackA,stackB)});if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseCreate(prototype,properties){return isObject(prototype)?nativeCreate(prototype):{}}if(!nativeCreate){baseCreate=function(){function Object(){}return function(prototype){if(isObject(prototype)){Object.prototype=prototype;var result=new Object;Object.prototype=null}return result||context.Object()}}()}function baseCreateCallback(func,thisArg,argCount){if(typeof func!="function"){return identity}if(typeof thisArg=="undefined"||!("prototype"in func)){return func}var bindData=func.__bindData__;if(typeof bindData=="undefined"){if(support.funcNames){bindData=!func.name}bindData=bindData||!support.funcDecomp;if(!bindData){var source=fnToString.call(func);if(!support.funcNames){bindData=!reFuncName.test(source)}if(!bindData){bindData=reThis.test(source);setBindData(func,bindData)}}}if(bindData===false||bindData!==true&&bindData[1]&1){return func}switch(argCount){case 1:return function(value){return func.call(thisArg,value)};case 2:return function(a,b){return func.call(thisArg,a,b)};case 3:return function(value,index,collection){return func.call(thisArg,value,index,collection)};case 4:return function(accumulator,value,index,collection){return func.call(thisArg,accumulator,value,index,collection)}}return bind(func,thisArg)}function baseCreateWrapper(bindData){var func=bindData[0],bitmask=bindData[1],partialArgs=bindData[2],partialRightArgs=bindData[3],thisArg=bindData[4],arity=bindData[5];var isBind=bitmask&1,isBindKey=bitmask&2,isCurry=bitmask&4,isCurryBound=bitmask&8,key=func;function bound(){var thisBinding=isBind?thisArg:this;if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(partialRightArgs||isCurry){args||(args=slice(arguments));if(partialRightArgs){push.apply(args,partialRightArgs)}if(isCurry&&args.length=largeArraySize&&indexOf===baseIndexOf,result=[];if(isLarge){var cache=createCache(values);if(cache){indexOf=cacheIndexOf;values=cache}else{isLarge=false}}while(++index-1}})}}stackA.pop();stackB.pop();if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseMerge(object,source,callback,stackA,stackB){(isArray(source)?forEach:forOwn)(source,function(source,key){var found,isArr,result=source,value=object[key];if(source&&((isArr=isArray(source))||isPlainObject(source))){var stackLength=stackA.length;while(stackLength--){if(found=stackA[stackLength]==source){value=stackB[stackLength];break}}if(!found){var isShallow;if(callback){result=callback(value,source);if(isShallow=typeof result!="undefined"){value=result}}if(!isShallow){value=isArr?isArray(value)?value:[]:isPlainObject(value)?value:{}}stackA.push(source);stackB.push(value);if(!isShallow){baseMerge(value,source,callback,stackA,stackB)}}}else{if(callback){result=callback(value,source);if(typeof result=="undefined"){result=source}}if(typeof result!="undefined"){value=result}}object[key]=value})}function baseRandom(min,max){return min+floor(nativeRandom()*(max-min+1))}function baseUniq(array,isSorted,callback){var index=-1,indexOf=getIndexOf(),length=array?array.length:0,result=[];var isLarge=!isSorted&&length>=largeArraySize&&indexOf===baseIndexOf,seen=callback||isLarge?getArray():result;if(isLarge){var cache=createCache(seen);indexOf=cacheIndexOf;seen=cache}while(++index":">",'"':""","'":"'"};var htmlUnescapes=invert(htmlEscapes);var reEscapedHtml=RegExp("("+keys(htmlUnescapes).join("|")+")","g"),reUnescapedHtml=RegExp("["+keys(htmlEscapes).join("")+"]","g");var assign=function(object,source,guard){var index,iterable=object,result=iterable;if(!iterable)return result;var args=arguments,argsIndex=0,argsLength=typeof guard=="number"?2:args.length;if(argsLength>3&&typeof args[argsLength-2]=="function"){var callback=baseCreateCallback(args[--argsLength-1],args[argsLength--],2)}else if(argsLength>2&&typeof args[argsLength-1]=="function"){callback=args[--argsLength]}while(++argsIndex3&&typeof args[length-2]=="function"){var callback=baseCreateCallback(args[--length-1],args[length--],2)}else if(length>2&&typeof args[length-1]=="function"){callback=args[--length]}var sources=slice(arguments,1,length),index=-1,stackA=getArray(),stackB=getArray();while(++index-1}else if(typeof length=="number"){result=(isString(collection)?collection.indexOf(target,fromIndex):indexOf(collection,target,fromIndex))>-1}else{forOwn(collection,function(value){if(++index>=fromIndex){return!(result=value===target)}})}return result}var countBy=createAggregator(function(result,value,key){hasOwnProperty.call(result,key)?result[key]++:result[key]=1});function every(collection,callback,thisArg){var result=true;callback=lodash.createCallback(callback,thisArg,3);var index=-1,length=collection?collection.length:0;if(typeof length=="number"){while(++indexresult){result=value}}}else{callback=callback==null&&isString(collection)?charAtCallback:lodash.createCallback(callback,thisArg,3);forEach(collection,function(value,index,collection){var current=callback(value,index,collection);if(current>computed){computed=current;result=value}})}return result}function min(collection,callback,thisArg){var computed=Infinity,result=computed;if(typeof callback!="function"&&thisArg&&thisArg[callback]===collection){callback=null}if(callback==null&&isArray(collection)){var index=-1,length=collection.length;while(++index=largeArraySize&&createCache(argsIndex?args[argsIndex]:seen))}}var array=args[0],index=-1,length=array?array.length:0,result=[];outer:while(++index>>1;callback(array[mid])1?arguments:arguments[0],index=-1,length=array?max(pluck(array,"length")):0,result=Array(length<0?0:length);while(++index2?createWrapper(func,17,slice(arguments,2),null,thisArg):createWrapper(func,1,null,null,thisArg)}function bindAll(object){var funcs=arguments.length>1?baseFlatten(arguments,true,false,1):functions(object),index=-1,length=funcs.length;while(++index2?createWrapper(key,19,slice(arguments,2),null,object):createWrapper(key,3,null,null,object)}function compose(){var funcs=arguments,length=funcs.length;while(length--){if(!isFunction(funcs[length])){throw new TypeError}}return function(){var args=arguments,length=funcs.length;while(length--){args=[funcs[length].apply(this,args)]}return args[0]}}function curry(func,arity){arity=typeof arity=="number"?arity:+arity||func.length;return createWrapper(func,4,null,null,null,arity)}function debounce(func,wait,options){var args,maxTimeoutId,result,stamp,thisArg,timeoutId,trailingCall,lastCalled=0,maxWait=false,trailing=true;if(!isFunction(func)){throw new TypeError}wait=nativeMax(0,wait)||0;if(options===true){var leading=true;trailing=false}else if(isObject(options)){leading=options.leading;maxWait="maxWait"in options&&(nativeMax(wait,options.maxWait)||0);trailing="trailing"in options?options.trailing:trailing}var delayed=function(){var remaining=wait-(now()-stamp);if(remaining<=0){if(maxTimeoutId){clearTimeout(maxTimeoutId)}var isCalled=trailingCall;maxTimeoutId=timeoutId=trailingCall=undefined;if(isCalled){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}}else{timeoutId=setTimeout(delayed,remaining)}};var maxDelayed=function(){if(timeoutId){clearTimeout(timeoutId)}maxTimeoutId=timeoutId=trailingCall=undefined;if(trailing||maxWait!==wait){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}};return function(){args=arguments; +_.each(inputGraph.nodes(),function(v){var node=canonicalize(inputGraph.node(v));g.setNode(v,_.defaults(selectNumberAttrs(node,nodeNumAttrs),nodeDefaults));g.setParent(v,inputGraph.parent(v))});_.each(inputGraph.edges(),function(e){var edge=canonicalize(inputGraph.edge(e));g.setEdge(e,_.merge({},edgeDefaults,selectNumberAttrs(edge,edgeNumAttrs),_.pick(edge,edgeAttrs)))});return g}function makeSpaceForEdgeLabels(g){var graph=g.graph();graph.ranksep/=2;_.each(g.edges(),function(e){var edge=g.edge(e);edge.minlen*=2;if(edge.labelpos.toLowerCase()!=="c"){if(graph.rankdir==="TB"||graph.rankdir==="BT"){edge.width+=edge.labeloffset}else{edge.height+=edge.labeloffset}}})}function injectEdgeLabelProxies(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.width&&edge.height){var v=g.node(e.v),w=g.node(e.w),label={rank:(w.rank-v.rank)/2+v.rank,e:e};util.addDummyNode(g,"edge-proxy",label,"_ep")}})}function assignRankMinMax(g){var maxRank=0;_.each(g.nodes(),function(v){var node=g.node(v);if(node.borderTop){node.minRank=g.node(node.borderTop).rank;node.maxRank=g.node(node.borderBottom).rank;maxRank=_.max(maxRank,node.maxRank)}});g.graph().maxRank=maxRank}function removeEdgeLabelProxies(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="edge-proxy"){g.edge(node.e).labelRank=node.rank;g.removeNode(v)}})}function translateGraph(g){var minX=Number.POSITIVE_INFINITY,maxX=0,minY=Number.POSITIVE_INFINITY,maxY=0,graphLabel=g.graph(),marginX=graphLabel.marginx||0,marginY=graphLabel.marginy||0;function getExtremes(attrs){var x=attrs.x,y=attrs.y,w=attrs.width,h=attrs.height;minX=Math.min(minX,x-w/2);maxX=Math.max(maxX,x+w/2);minY=Math.min(minY,y-h/2);maxY=Math.max(maxY,y+h/2)}_.each(g.nodes(),function(v){getExtremes(g.node(v))});_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){getExtremes(edge)}});minX-=marginX;minY-=marginY;_.each(g.nodes(),function(v){var node=g.node(v);node.x-=minX;node.y-=minY});_.each(g.edges(),function(e){var edge=g.edge(e);_.each(edge.points,function(p){p.x-=minX;p.y-=minY});if(_.has(edge,"x")){edge.x-=minX}if(_.has(edge,"y")){edge.y-=minY}});graphLabel.width=maxX-minX+marginX;graphLabel.height=maxY-minY+marginY}function assignNodeIntersects(g){_.each(g.edges(),function(e){var edge=g.edge(e),nodeV=g.node(e.v),nodeW=g.node(e.w),p1,p2;if(!edge.points){edge.points=[];p1=nodeW;p2=nodeV}else{p1=edge.points[0];p2=edge.points[edge.points.length-1]}edge.points.unshift(util.intersectRect(nodeV,p1));edge.points.push(util.intersectRect(nodeW,p2))})}function fixupEdgeLabelCoords(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){if(edge.labelpos==="l"||edge.labelpos==="r"){edge.width-=edge.labeloffset}switch(edge.labelpos){case"l":edge.x-=edge.width/2+edge.labeloffset;break;case"r":edge.x+=edge.width/2+edge.labeloffset;break}}})}function reversePointsForReversedEdges(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.reversed){edge.points.reverse()}})}function removeBorderNodes(g){_.each(g.nodes(),function(v){if(g.children(v).length){var node=g.node(v),t=g.node(node.borderTop),b=g.node(node.borderBottom),l=g.node(_.last(node.borderLeft)),r=g.node(_.last(node.borderRight));node.width=Math.abs(r.x-l.x);node.height=Math.abs(b.y-t.y);node.x=l.x+node.width/2;node.y=t.y+node.height/2}});_.each(g.nodes(),function(v){if(g.node(v).dummy==="border"){g.removeNode(v)}})}function removeSelfEdges(g){_.each(g.edges(),function(e){if(e.v===e.w){var node=g.node(e.v);if(!node.selfEdges){node.selfEdges=[]}node.selfEdges.push({e:e,label:g.edge(e)});g.removeEdge(e)}})}function insertSelfEdges(g){var layers=util.buildLayerMatrix(g);_.each(layers,function(layer){var orderShift=0;_.each(layer,function(v,i){var node=g.node(v);node.order=i+orderShift;_.each(node.selfEdges,function(selfEdge){util.addDummyNode(g,"selfedge",{width:selfEdge.label.width,height:selfEdge.label.height,rank:node.rank,order:i+ ++orderShift,e:selfEdge.e,label:selfEdge.label},"_se")});delete node.selfEdges})})}function positionSelfEdges(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="selfedge"){var selfNode=g.node(node.e.v),x=selfNode.x+selfNode.width/2,y=selfNode.y,dx=node.x-x,dy=selfNode.height/2;g.setEdge(node.e,node.label);g.removeNode(v);node.label.points=[{x:x+2*dx/3,y:y-dy},{x:x+5*dx/6,y:y-dy},{x:x+dx,y:y},{x:x+5*dx/6,y:y+dy},{x:x+2*dx/3,y:y+dy}];node.label.x=node.x;node.label.y=node.y}})}function selectNumberAttrs(obj,attrs){return _.mapValues(_.pick(obj,attrs),Number)}function canonicalize(attrs){var newAttrs={};_.each(attrs,function(v,k){newAttrs[k.toLowerCase()]=v});return newAttrs}},{"./acyclic":28,"./add-border-segments":29,"./coordinate-system":30,"./graphlib":33,"./lodash":36,"./nesting-graph":37,"./normalize":38,"./order":43,"./parent-dummy-chains":48,"./position":50,"./rank":52,"./util":55}],36:[function(require,module,exports){module.exports=require(20)},{"/Users/andrew/Documents/dev/dagre-d3/lib/lodash.js":20,lodash:77}],37:[function(require,module,exports){var _=require("./lodash"),util=require("./util");module.exports={run:run,cleanup:cleanup};function run(g){var root=util.addDummyNode(g,"root",{},"_root"),depths=treeDepths(g),height=_.max(depths)-1,nodeSep=2*height+1;g.graph().nestingRoot=root;_.each(g.edges(),function(e){g.edge(e).minlen*=nodeSep});var weight=sumWeights(g)+1;_.each(g.children(),function(child){dfs(g,root,nodeSep,weight,height,depths,child)});g.graph().nodeRankFactor=nodeSep}function dfs(g,root,nodeSep,weight,height,depths,v){var children=g.children(v);if(!children.length){if(v!==root){g.setEdge(root,v,{weight:0,minlen:nodeSep})}return}var top=util.addBorderNode(g,"_bt"),bottom=util.addBorderNode(g,"_bb"),label=g.node(v);g.setParent(top,v);label.borderTop=top;g.setParent(bottom,v);label.borderBottom=bottom;_.each(children,function(child){dfs(g,root,nodeSep,weight,height,depths,child);var childNode=g.node(child),childTop=childNode.borderTop?childNode.borderTop:child,childBottom=childNode.borderBottom?childNode.borderBottom:child,thisWeight=childNode.borderTop?weight:2*weight,minlen=childTop!==childBottom?1:height-depths[v]+1;g.setEdge(top,childTop,{weight:thisWeight,minlen:minlen,nestingEdge:true});g.setEdge(childBottom,bottom,{weight:thisWeight,minlen:minlen,nestingEdge:true})});if(!g.parent(v)){g.setEdge(root,top,{weight:0,minlen:height+depths[v]})}}function treeDepths(g){var depths={};function dfs(v,depth){var children=g.children(v);if(children&&children.length){_.each(children,function(child){dfs(child,depth+1)})}depths[v]=depth}_.each(g.children(),function(v){dfs(v,1)});return depths}function sumWeights(g){return _.reduce(g.edges(),function(acc,e){return acc+g.edge(e).weight},0)}function cleanup(g){var graphLabel=g.graph();g.removeNode(graphLabel.nestingRoot);delete graphLabel.nestingRoot;_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.nestingEdge){g.removeEdge(e)}})}},{"./lodash":36,"./util":55}],38:[function(require,module,exports){"use strict";var _=require("./lodash"),util=require("./util");module.exports={run:run,undo:undo};function run(g){g.graph().dummyChains=[];_.each(g.edges(),function(edge){normalizeEdge(g,edge)})}function normalizeEdge(g,e){var v=e.v,vRank=g.node(v).rank,w=e.w,wRank=g.node(w).rank,name=e.name,edgeLabel=g.edge(e),labelRank=edgeLabel.labelRank;if(wRank===vRank+1)return;g.removeEdge(e);var dummy,attrs,i;for(i=0,++vRank;vRank0){if(index%2){weightSum+=tree[index+1]}index=index-1>>1;tree[index]+=entry.weight}cc+=entry.weight*weightSum}));return cc}},{"../lodash":36}],43:[function(require,module,exports){"use strict";var _=require("../lodash"),initOrder=require("./init-order"),crossCount=require("./cross-count"),sortSubgraph=require("./sort-subgraph"),buildLayerGraph=require("./build-layer-graph"),addSubgraphConstraints=require("./add-subgraph-constraints"),Graph=require("../graphlib").Graph,util=require("../util");module.exports=order;function order(g){var maxRank=util.maxRank(g),downLayerGraphs=buildLayerGraphs(g,_.range(1,maxRank+1),"inEdges"),upLayerGraphs=buildLayerGraphs(g,_.range(maxRank-1,-1,-1),"outEdges");var layering=initOrder(g);assignOrder(g,layering);var bestCC=Number.POSITIVE_INFINITY,best;for(var i=0,lastBest=0;lastBest<4;++i,++lastBest){sweepLayerGraphs(i%2?downLayerGraphs:upLayerGraphs,i%4>=2);layering=util.buildLayerMatrix(g);var cc=crossCount(g,layering);if(cc=vEntry.barycenter){mergeEntries(vEntry,uEntry)}}}function handleOut(vEntry){return function(wEntry){wEntry["in"].push(vEntry);if(--wEntry.indegree===0){sourceSet.push(wEntry)}}}while(sourceSet.length){var entry=sourceSet.pop();entries.push(entry);_.each(entry["in"].reverse(),handleIn(entry));_.each(entry.out,handleOut(entry))}return _.chain(entries).filter(function(entry){return!entry.merged}).map(function(entry){return _.pick(entry,["vs","i","barycenter","weight"])}).value()}function mergeEntries(target,source){var sum=0,weight=0;if(target.weight){sum+=target.barycenter*target.weight;weight+=target.weight}if(source.weight){sum+=source.barycenter*source.weight;weight+=source.weight}target.vs=source.vs.concat(target.vs);target.barycenter=sum/weight;target.weight=weight;target.i=Math.min(source.i,target.i);source.merged=true}},{"../lodash":36}],46:[function(require,module,exports){var _=require("../lodash"),barycenter=require("./barycenter"),resolveConflicts=require("./resolve-conflicts"),sort=require("./sort");module.exports=sortSubgraph;function sortSubgraph(g,v,cg,biasRight){var movable=g.children(v),node=g.node(v),bl=node?node.borderLeft:undefined,br=node?node.borderRight:undefined,subgraphs={};if(bl){movable=_.filter(movable,function(w){return w!==bl&&w!==br})}var barycenters=barycenter(g,movable);_.each(barycenters,function(entry){if(g.children(entry.v).length){var subgraphResult=sortSubgraph(g,entry.v,cg,biasRight);subgraphs[entry.v]=subgraphResult;if(_.has(subgraphResult,"barycenter")){mergeBarycenters(entry,subgraphResult)}}});var entries=resolveConflicts(barycenters,cg);expandSubgraphs(entries,subgraphs);var result=sort(entries,biasRight);if(bl){result.vs=_.flatten([bl,result.vs,br],true);if(g.predecessors(bl).length){var blPred=g.node(g.predecessors(bl)[0]),brPred=g.node(g.predecessors(br)[0]);if(!_.has(result,"barycenter")){result.barycenter=0;result.weight=0}result.barycenter=(result.barycenter*result.weight+blPred.order+brPred.order)/(result.weight+2);result.weight+=2}}return result}function expandSubgraphs(entries,subgraphs){_.each(entries,function(entry){entry.vs=_.flatten(entry.vs.map(function(v){if(subgraphs[v]){return subgraphs[v].vs}return v}),true)})}function mergeBarycenters(target,other){if(!_.isUndefined(target.barycenter)){target.barycenter=(target.barycenter*target.weight+other.barycenter*other.weight)/(target.weight+other.weight);target.weight+=other.weight}else{target.barycenter=other.barycenter;target.weight=other.weight}}},{"../lodash":36,"./barycenter":40,"./resolve-conflicts":45,"./sort":47}],47:[function(require,module,exports){var _=require("../lodash"),util=require("../util");module.exports=sort;function sort(entries,biasRight){var parts=util.partition(entries,function(entry){return _.has(entry,"barycenter")});var sortable=parts.lhs,unsortable=_.sortBy(parts.rhs,function(entry){return-entry.i}),vs=[],sum=0,weight=0,vsIndex=0;sortable.sort(compareWithBias(!!biasRight));vsIndex=consumeUnsortable(vs,unsortable,vsIndex);_.each(sortable,function(entry){vsIndex+=entry.vs.length;vs.push(entry.vs);sum+=entry.barycenter*entry.weight;weight+=entry.weight;vsIndex=consumeUnsortable(vs,unsortable,vsIndex)});var result={vs:_.flatten(vs,true)};if(weight){result.barycenter=sum/weight;result.weight=weight}return result}function consumeUnsortable(vs,unsortable,index){var last;while(unsortable.length&&(last=_.last(unsortable)).i<=index){unsortable.pop();vs.push(last.vs);index++}return index}function compareWithBias(bias){return function(entryV,entryW){if(entryV.barycenterentryW.barycenter){return 1}return!bias?entryV.i-entryW.i:entryW.i-entryV.i}}},{"../lodash":36,"../util":55}],48:[function(require,module,exports){var _=require("./lodash");module.exports=parentDummyChains;function parentDummyChains(g){var postorderNums=postorder(g);_.each(g.graph().dummyChains,function(v){var node=g.node(v),edgeObj=node.edgeObj,pathData=findPath(g,postorderNums,edgeObj.v,edgeObj.w),path=pathData.path,lca=pathData.lca,pathIdx=0,pathV=path[pathIdx],ascending=true;while(v!==edgeObj.w){node=g.node(v);if(ascending){while((pathV=path[pathIdx])!==lca&&g.node(pathV).maxRanklow||lim>postorderNums[parent].lim));lca=parent;parent=w;while((parent=g.parent(parent))!==lca){wPath.push(parent)}return{path:vPath.concat(wPath.reverse()),lca:lca}}function postorder(g){var result={},lim=0;function dfs(v){var low=lim;_.each(g.children(v),dfs);result[v]={low:low,lim:lim++}}_.each(g.children(),dfs);return result}},{"./lodash":36}],49:[function(require,module,exports){"use strict";var _=require("../lodash"),Graph=require("../graphlib").Graph,util=require("../util");module.exports={positionX:positionX,findType1Conflicts:findType1Conflicts,findType2Conflicts:findType2Conflicts,addConflict:addConflict,hasConflict:hasConflict,verticalAlignment:verticalAlignment,horizontalCompaction:horizontalCompaction,alignCoordinates:alignCoordinates,findSmallestWidthAlignment:findSmallestWidthAlignment,balance:balance};function findType1Conflicts(g,layering){var conflicts={};function visitLayer(prevLayer,layer){var k0=0,scanPos=0,prevLayerLength=prevLayer.length,lastNode=_.last(layer);_.each(layer,function(v,i){var w=findOtherInnerSegmentNode(g,v),k1=w?g.node(w).order:prevLayerLength;if(w||v===lastNode){_.each(layer.slice(scanPos,i+1),function(scanNode){_.each(g.predecessors(scanNode),function(u){var uLabel=g.node(u),uPos=uLabel.order;if((uPosnextNorthBorder)){addConflict(conflicts,u,v)}})}})}function visitLayer(north,south){var prevNorthPos=-1,nextNorthPos,southPos=0;_.each(south,function(v,southLookahead){if(g.node(v).dummy==="border"){var predecessors=g.predecessors(v);if(predecessors.length){nextNorthPos=g.node(predecessors[0]).order;scan(south,southPos,southLookahead,prevNorthPos,nextNorthPos);southPos=southLookahead;prevNorthPos=nextNorthPos}}scan(south,southPos,south.length,nextNorthPos,north.length)});return south}_.reduce(layering,visitLayer);return conflicts}function findOtherInnerSegmentNode(g,v){if(g.node(v).dummy){return _.find(g.predecessors(v),function(u){return g.node(u).dummy})}}function addConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}var conflictsV=conflicts[v];if(!conflictsV){conflicts[v]=conflictsV={}}conflictsV[w]=true}function hasConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}return _.has(conflicts[v],w)}function verticalAlignment(g,layering,conflicts,neighborFn){var root={},align={},pos={};_.each(layering,function(layer){_.each(layer,function(v,order){root[v]=v;align[v]=v;pos[v]=order})});_.each(layering,function(layer){var prevIdx=-1;_.each(layer,function(v){var ws=neighborFn(v);if(ws.length){ws=_.sortBy(ws,function(w){return pos[w]});var mp=(ws.length-1)/2;for(var i=Math.floor(mp),il=Math.ceil(mp);i<=il;++i){var w=ws[i];if(align[v]===v&&prevIdxwLabel.lim){tailLabel=wLabel;flip=true}var candidates=_.filter(g.edges(),function(edge){return flip===isDescendant(t,t.node(edge.v),tailLabel)&&flip!==isDescendant(t,t.node(edge.w),tailLabel)});return _.min(candidates,function(edge){return slack(g,edge)})}function exchangeEdges(t,g,e,f){var v=e.v,w=e.w;t.removeEdge(v,w);t.setEdge(f.v,f.w,{});initLowLimValues(t);initCutValues(t,g);updateRanks(t,g)}function updateRanks(t,g){var root=_.find(t.nodes(),function(v){return!g.node(v).parent}),vs=preorder(t,root);vs=vs.slice(1);_.each(vs,function(v){var parent=t.node(v).parent,edge=g.edge(v,parent),flipped=false;if(!edge){edge=g.edge(parent,v);flipped=true}g.node(v).rank=g.node(parent).rank+(flipped?edge.minlen:-edge.minlen)})}function isTreeEdge(tree,u,v){return tree.hasEdge(u,v)}function isDescendant(tree,vLabel,rootLabel){return rootLabel.low<=vLabel.lim&&vLabel.lim<=rootLabel.lim}},{"../graphlib":33,"../lodash":36,"../util":55,"./feasible-tree":51,"./util":54}],54:[function(require,module,exports){"use strict";var _=require("../lodash");module.exports={longestPath:longestPath,slack:slack};function longestPath(g){var visited={};function dfs(v){var label=g.node(v);if(_.has(visited,v)){return label.rank}visited[v]=true;var rank=_.min(_.map(g.outEdges(v),function(e){return dfs(e.w)-g.edge(e).minlen}));if(rank===Number.POSITIVE_INFINITY){rank=0}return label.rank=rank}_.each(g.sources(),dfs)}function slack(g,e){return g.node(e.w).rank-g.node(e.v).rank-g.edge(e).minlen}},{"../lodash":36}],55:[function(require,module,exports){"use strict";var _=require("./lodash"),Graph=require("./graphlib").Graph;module.exports={addDummyNode:addDummyNode,simplify:simplify,asNonCompoundGraph:asNonCompoundGraph,successorWeights:successorWeights,predecessorWeights:predecessorWeights,intersectRect:intersectRect,buildLayerMatrix:buildLayerMatrix,normalizeRanks:normalizeRanks,removeEmptyRanks:removeEmptyRanks,addBorderNode:addBorderNode,maxRank:maxRank,partition:partition,time:time,notime:notime};function addDummyNode(g,type,attrs,name){var v; -stamp=now();thisArg=this;trailingCall=trailing&&(timeoutId||!leading);if(maxWait===false){var leadingCall=leading&&!timeoutId}else{if(!maxTimeoutId&&!leading){lastCalled=stamp}var remaining=maxWait-(stamp-lastCalled),isCalled=remaining<=0;if(isCalled){if(maxTimeoutId){maxTimeoutId=clearTimeout(maxTimeoutId)}lastCalled=stamp;result=func.apply(thisArg,args)}else if(!maxTimeoutId){maxTimeoutId=setTimeout(maxDelayed,remaining)}}if(isCalled&&timeoutId){timeoutId=clearTimeout(timeoutId)}else if(!timeoutId&&wait!==maxWait){timeoutId=setTimeout(delayed,wait)}if(leadingCall){isCalled=true;result=func.apply(thisArg,args)}if(isCalled&&!timeoutId&&!maxTimeoutId){args=thisArg=null}return result}}function defer(func){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,1);return setTimeout(function(){func.apply(undefined,args)},1)}function delay(func,wait){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,2);return setTimeout(function(){func.apply(undefined,args)},wait)}function memoize(func,resolver){if(!isFunction(func)){throw new TypeError}var memoized=function(){var cache=memoized.cache,key=resolver?resolver.apply(this,arguments):keyPrefix+arguments[0];return hasOwnProperty.call(cache,key)?cache[key]:cache[key]=func.apply(this,arguments)};memoized.cache={};return memoized}function once(func){var ran,result;if(!isFunction(func)){throw new TypeError}return function(){if(ran){return result}ran=true;result=func.apply(this,arguments);func=null;return result}}function partial(func){return createWrapper(func,16,slice(arguments,1))}function partialRight(func){return createWrapper(func,32,null,slice(arguments,1))}function throttle(func,wait,options){var leading=true,trailing=true;if(!isFunction(func)){throw new TypeError}if(options===false){leading=false}else if(isObject(options)){leading="leading"in options?options.leading:leading;trailing="trailing"in options?options.trailing:trailing}debounceOptions.leading=leading;debounceOptions.maxWait=wait;debounceOptions.trailing=trailing;return debounce(func,wait,debounceOptions)}function wrap(value,wrapper){return createWrapper(wrapper,16,[value])}function constant(value){return function(){return value}}function createCallback(func,thisArg,argCount){var type=typeof func;if(func==null||type=="function"){return baseCreateCallback(func,thisArg,argCount)}if(type!="object"){return property(func)}var props=keys(func),key=props[0],a=func[key];if(props.length==1&&a===a&&!isObject(a)){return function(object){var b=object[key];return a===b&&(a!==0||1/a==1/b)}}return function(object){var length=props.length,result=false;while(length--){if(!(result=baseIsEqual(object[props[length]],func[props[length]],null,true))){break}}return result}}function escape(string){return string==null?"":String(string).replace(reUnescapedHtml,escapeHtmlChar)}function identity(value){return value}function mixin(object,source,options){var chain=true,methodNames=source&&functions(source);if(!source||!options&&!methodNames.length){if(options==null){options=source}ctor=lodashWrapper;source=object;object=lodash;methodNames=functions(source)}if(options===false){chain=false}else if(isObject(options)&&"chain"in options){chain=options.chain}var ctor=object,isFunc=isFunction(ctor);forEach(methodNames,function(methodName){var func=object[methodName]=source[methodName];if(isFunc){ctor.prototype[methodName]=function(){var chainAll=this.__chain__,value=this.__wrapped__,args=[value];push.apply(args,arguments);var result=func.apply(object,args);if(chain||chainAll){if(value===result&&isObject(result)){return this}result=new ctor(result);result.__chain__=chainAll}return result}}})}function noConflict(){context._=oldDash;return this}function noop(){}var now=isNative(now=Date.now)&&now||function(){return(new Date).getTime()};var parseInt=nativeParseInt(whitespace+"08")==8?nativeParseInt:function(value,radix){return nativeParseInt(isString(value)?value.replace(reLeadingSpacesAndZeros,""):value,radix||0)};function property(key){return function(object){return object[key]}}function random(min,max,floating){var noMin=min==null,noMax=max==null;if(floating==null){if(typeof min=="boolean"&&noMax){floating=min;min=1}else if(!noMax&&typeof max=="boolean"){floating=max;noMax=true}}if(noMin&&noMax){max=1}min=+min||0;if(noMax){max=min;min=0}else{max=+max||0}if(floating||min%1||max%1){var rand=nativeRandom();return nativeMin(min+rand*(max-min+parseFloat("1e-"+((rand+"").length-1))),max)}return baseRandom(min,max)}function result(object,key){if(object){var value=object[key];return isFunction(value)?object[key]():value}}function template(text,data,options){var settings=lodash.templateSettings;text=String(text||"");options=defaults({},options,settings);var imports=defaults({},options.imports,settings.imports),importsKeys=keys(imports),importsValues=values(imports);var isEvaluating,index=0,interpolate=options.interpolate||reNoMatch,source="__p += '";var reDelimiters=RegExp((options.escape||reNoMatch).source+"|"+interpolate.source+"|"+(interpolate===reInterpolate?reEsTemplate:reNoMatch).source+"|"+(options.evaluate||reNoMatch).source+"|$","g");text.replace(reDelimiters,function(match,escapeValue,interpolateValue,esTemplateValue,evaluateValue,offset){interpolateValue||(interpolateValue=esTemplateValue);source+=text.slice(index,offset).replace(reUnescapedString,escapeStringChar);if(escapeValue){source+="' +\n__e("+escapeValue+") +\n'"}if(evaluateValue){isEvaluating=true;source+="';\n"+evaluateValue+";\n__p += '"}if(interpolateValue){source+="' +\n((__t = ("+interpolateValue+")) == null ? '' : __t) +\n'"}index=offset+match.length;return match});source+="';\n";var variable=options.variable,hasVariable=variable;if(!hasVariable){variable="obj";source="with ("+variable+") {\n"+source+"\n}\n"}source=(isEvaluating?source.replace(reEmptyStringLeading,""):source).replace(reEmptyStringMiddle,"$1").replace(reEmptyStringTrailing,"$1;");source="function("+variable+") {\n"+(hasVariable?"":variable+" || ("+variable+" = {});\n")+"var __t, __p = '', __e = _.escape"+(isEvaluating?", __j = Array.prototype.join;\n"+"function print() { __p += __j.call(arguments, '') }\n":";\n")+source+"return __p\n}";var sourceURL="\n/*\n//# sourceURL="+(options.sourceURL||"/lodash/template/source["+templateCounter++ +"]")+"\n*/";try{var result=Function(importsKeys,"return "+source+sourceURL).apply(undefined,importsValues)}catch(e){e.source=source;throw e}if(data){return result(data)}result.source=source;return result}function times(n,callback,thisArg){n=(n=+n)>-1?n:0;var index=-1,result=Array(n);callback=baseCreateCallback(callback,thisArg,1);while(++indexMath.abs(dx)*h){if(dy<0){h=-h}sx=h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=w*dy/dx}return{x:x+sx,y:y+sy}}function buildLayerMatrix(g){var layering=_.map(_.range(maxRank(g)+1),function(){return[]});_.each(g.nodes(),function(v){var node=g.node(v),rank=node.rank;if(!_.isUndefined(rank)){layering[rank][node.order]=v}});return layering}function normalizeRanks(g){var min=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));_.each(g.nodes(),function(v){var node=g.node(v);if(_.has(node,"rank")){node.rank-=min}})}function removeEmptyRanks(g){var offset=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));var layers=[];_.each(g.nodes(),function(v){var rank=g.node(v).rank-offset;if(!_.has(layers,rank)){layers[rank]=[]}layers[rank].push(v)});var delta=0,nodeRankFactor=g.graph().nodeRankFactor;_.each(layers,function(vs,i){if(_.isUndefined(vs)&&i%nodeRankFactor!==0){--delta}else if(delta){_.each(vs,function(v){g.node(v).rank+=delta})}})}function addBorderNode(g,prefix,rank,order){var node={width:0,height:0};if(arguments.length>=4){node.rank=rank;node.order=order}return addDummyNode(g,"border",node,prefix)}function maxRank(g){return _.max(_.map(g.nodes(),function(v){var rank=g.node(v).rank;if(!_.isUndefined(rank)){return rank}}))}function partition(collection,fn){var result={lhs:[],rhs:[]};_.each(collection,function(value){if(fn(value)){result.lhs.push(value)}else{result.rhs.push(value)}});return result}function time(name,fn){var start=_.now();try{return fn()}finally{console.log(name+" time: "+(_.now()-start)+"ms")}}function notime(name,fn){return fn()}},{"./graphlib":33,"./lodash":36}],56:[function(require,module,exports){module.exports="0.7.1"},{}],57:[function(require,module,exports){var lib=require("./lib");module.exports={Graph:lib.Graph,json:require("./lib/json"),alg:require("./lib/alg"),version:lib.version}},{"./lib":73,"./lib/alg":64,"./lib/json":74}],58:[function(require,module,exports){var _=require("../lodash");module.exports=components;function components(g){var visited={},cmpts=[],cmpt;function dfs(v){if(_.has(visited,v))return;visited[v]=true;cmpt.push(v);_.each(g.successors(v),dfs);_.each(g.predecessors(v),dfs)}_.each(g.nodes(),function(v){cmpt=[];dfs(v);if(cmpt.length){cmpts.push(cmpt)}});return cmpts}},{"../lodash":75}],59:[function(require,module,exports){var _=require("../lodash");module.exports=dfs;function dfs(g,vs,order){if(!_.isArray(vs)){vs=[vs]}var acc=[],visited={};_.each(vs,function(v){if(!g.hasNode(v)){throw new Error("Graph does not have node: "+v)}doDfs(g,v,order==="post",visited,acc)});return acc}function doDfs(g,v,postorder,visited,acc){if(!_.has(visited,v)){visited[v]=true;if(!postorder){acc.push(v)}_.each(g.neighbors(v),function(w){doDfs(g,w,postorder,visited,acc)});if(postorder){acc.push(v)}}}},{"../lodash":75}],60:[function(require,module,exports){var dijkstra=require("./dijkstra"),_=require("../lodash");module.exports=dijkstraAll;function dijkstraAll(g,weightFunc,edgeFunc){return _.transform(g.nodes(),function(acc,v){acc[v]=dijkstra(g,v,weightFunc,edgeFunc)},{})}},{"../lodash":75,"./dijkstra":61}],61:[function(require,module,exports){var _=require("../lodash"),PriorityQueue=require("../data/priority-queue");module.exports=dijkstra;var DEFAULT_WEIGHT_FUNC=_.constant(1);function dijkstra(g,source,weightFn,edgeFn){return runDijkstra(g,String(source),weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runDijkstra(g,source,weightFn,edgeFn){var results={},pq=new PriorityQueue,v,vEntry;var updateNeighbors=function(edge){var w=edge.v!==v?edge.v:edge.w,wEntry=results[w],weight=weightFn(edge),distance=vEntry.distance+weight;if(weight<0){throw new Error("dijkstra does not allow negative edge weights. "+"Bad edge: "+edge+" Weight: "+weight)}if(distance0){v=pq.removeMin();vEntry=results[v];if(vEntry.distance===Number.POSITIVE_INFINITY){break}edgeFn(v).forEach(updateNeighbors)}return results}},{"../data/priority-queue":71,"../lodash":75}],62:[function(require,module,exports){var _=require("../lodash"),tarjan=require("./tarjan");module.exports=findCycles;function findCycles(g){return _.filter(tarjan(g),function(cmpt){return cmpt.length>1})}},{"../lodash":75,"./tarjan":69}],63:[function(require,module,exports){var _=require("../lodash");module.exports=floydWarshall;var DEFAULT_WEIGHT_FUNC=_.constant(1);function floydWarshall(g,weightFn,edgeFn){return runFloydWarshall(g,weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runFloydWarshall(g,weightFn,edgeFn){var results={},nodes=g.nodes();nodes.forEach(function(v){results[v]={};results[v][v]={distance:0};nodes.forEach(function(w){if(v!==w){results[v][w]={distance:Number.POSITIVE_INFINITY}}});edgeFn(v).forEach(function(edge){var w=edge.v===v?edge.w:edge.v,d=weightFn(edge);results[v][w]={distance:d,predecessor:v}})});nodes.forEach(function(k){var rowK=results[k];nodes.forEach(function(i){var rowI=results[i];nodes.forEach(function(j){var ik=rowI[k];var kj=rowK[j];var ij=rowI[j];var altDistance=ik.distance+kj.distance;if(altDistance0){v=pq.removeMin();if(_.has(parents,v)){result.setEdge(v,parents[v])}else if(init){throw new Error("Input graph is not connected: "+g)}else{init=true}g.nodeEdges(v).forEach(updateNeighbors)}return result}},{"../data/priority-queue":71,"../graph":72,"../lodash":75}],69:[function(require,module,exports){var _=require("../lodash");module.exports=tarjan;function tarjan(g){var index=0,stack=[],visited={},results=[];function dfs(v){var entry=visited[v]={onStack:true,lowlink:index,index:index++};stack.push(v);g.successors(v).forEach(function(w){if(!_.has(visited,w)){dfs(w);entry.lowlink=Math.min(entry.lowlink,visited[w].lowlink)}else if(visited[w].onStack){entry.lowlink=Math.min(entry.lowlink,visited[w].index)}});if(entry.lowlink===entry.index){var cmpt=[],w;do{w=stack.pop();visited[w].onStack=false;cmpt.push(w)}while(v!==w);results.push(cmpt)}}g.nodes().forEach(function(v){if(!_.has(visited,v)){dfs(v)}});return results}},{"../lodash":75}],70:[function(require,module,exports){var _=require("../lodash");module.exports=topsort;topsort.CycleException=CycleException;function topsort(g){var visited={},stack={},results=[];function visit(node){if(_.has(stack,node)){throw new CycleException}if(!_.has(visited,node)){stack[node]=true;visited[node]=true;_.each(g.predecessors(node),visit);delete stack[node];results.push(node)}}_.each(g.sinks(),visit);if(_.size(visited)!==g.nodeCount()){throw new CycleException}return results}function CycleException(){}},{"../lodash":75}],71:[function(require,module,exports){var _=require("../lodash");module.exports=PriorityQueue;function PriorityQueue(){this._arr=[];this._keyIndices={}}PriorityQueue.prototype.size=function(){return this._arr.length};PriorityQueue.prototype.keys=function(){return this._arr.map(function(x){return x.key})};PriorityQueue.prototype.has=function(key){return _.has(this._keyIndices,key)};PriorityQueue.prototype.priority=function(key){var index=this._keyIndices[key];if(index!==undefined){return this._arr[index].priority}};PriorityQueue.prototype.min=function(){if(this.size()===0){throw new Error("Queue underflow")}return this._arr[0].key};PriorityQueue.prototype.add=function(key,priority){var keyIndices=this._keyIndices;key=String(key);if(!_.has(keyIndices,key)){var arr=this._arr;var index=arr.length;keyIndices[key]=index;arr.push({key:key,priority:priority});this._decrease(index);return true}return false};PriorityQueue.prototype.removeMin=function(){this._swap(0,this._arr.length-1);var min=this._arr.pop();delete this._keyIndices[min.key];this._heapify(0);return min.key};PriorityQueue.prototype.decrease=function(key,priority){var index=this._keyIndices[key];if(priority>this._arr[index].priority){throw new Error("New priority is greater than current priority. "+"Key: "+key+" Old: "+this._arr[index].priority+" New: "+priority)}this._arr[index].priority=priority;this._decrease(index)};PriorityQueue.prototype._heapify=function(i){var arr=this._arr;var l=2*i,r=l+1,largest=i;if(l>1;if(arr[parent].priority1){this.setNode(v,value)}else{this.setNode(v)}},this);return this};Graph.prototype.setNode=function(v,value){if(_.has(this._nodes,v)){if(arguments.length>1){this._nodes[v]=value}return this}this._nodes[v]=arguments.length>1?value:this._defaultNodeLabelFn(v);if(this._isCompound){this._parent[v]=GRAPH_NODE;this._children[v]={};this._children[GRAPH_NODE][v]=true}this._in[v]={};this._preds[v]={};this._out[v]={};this._sucs[v]={};++this._nodeCount;return this};Graph.prototype.node=function(v){return this._nodes[v]};Graph.prototype.hasNode=function(v){return _.has(this._nodes,v)};Graph.prototype.removeNode=function(v){var self=this;if(_.has(this._nodes,v)){var removeEdge=function(e){self.removeEdge(self._edgeObjs[e])};delete this._nodes[v];if(this._isCompound){this._removeFromParentsChildList(v);delete this._parent[v];_.each(this.children(v),function(child){this.setParent(child)},this);delete this._children[v]}_.each(_.keys(this._in[v]),removeEdge);delete this._in[v];delete this._preds[v];_.each(_.keys(this._out[v]),removeEdge);delete this._out[v];delete this._sucs[v];--this._nodeCount}return this};Graph.prototype.setParent=function(v,parent){if(!this._isCompound){throw new Error("Cannot set parent in a non-compound graph")}if(_.isUndefined(parent)){parent=GRAPH_NODE}else{for(var ancestor=parent;!_.isUndefined(ancestor);ancestor=this.parent(ancestor)){if(ancestor===v){throw new Error("Setting "+parent+" as parent of "+v+" would create create a cycle")}}this.setNode(parent)}this.setNode(v);this._removeFromParentsChildList(v);this._parent[v]=parent;this._children[parent][v]=true;return this};Graph.prototype._removeFromParentsChildList=function(v){delete this._children[this._parent[v]][v]};Graph.prototype.parent=function(v){if(this._isCompound){var parent=this._parent[v];if(parent!==GRAPH_NODE){return parent}}};Graph.prototype.children=function(v){if(_.isUndefined(v)){v=GRAPH_NODE}if(this._isCompound){var children=this._children[v];if(children){return _.keys(children)}}else if(v===GRAPH_NODE){return this.nodes()}else if(this.hasNode(v)){return[]}};Graph.prototype.predecessors=function(v){var predsV=this._preds[v];if(predsV){return _.keys(predsV)}};Graph.prototype.successors=function(v){var sucsV=this._sucs[v];if(sucsV){return _.keys(sucsV)}};Graph.prototype.neighbors=function(v){var preds=this.predecessors(v);if(preds){return _.union(preds,this.successors(v))}};Graph.prototype.setDefaultEdgeLabel=function(newDefault){if(!_.isFunction(newDefault)){newDefault=_.constant(newDefault)}this._defaultEdgeLabelFn=newDefault;return this};Graph.prototype.edgeCount=function(){return this._edgeCount};Graph.prototype.edges=function(){return _.values(this._edgeObjs)};Graph.prototype.setPath=function(vs,value){var self=this,args=arguments;_.reduce(vs,function(v,w){if(args.length>1){self.setEdge(v,w,value)}else{self.setEdge(v,w)}return w});return this};Graph.prototype.setEdge=function(){var v,w,name,value,valueSpecified=false;if(_.isPlainObject(arguments[0])){v=arguments[0].v;w=arguments[0].w;name=arguments[0].name;if(arguments.length===2){value=arguments[1];valueSpecified=true}}else{v=arguments[0];w=arguments[1];name=arguments[3];if(arguments.length>2){value=arguments[2];valueSpecified=true}}v=""+v;w=""+w;if(!_.isUndefined(name)){name=""+name}var e=edgeArgsToId(this._isDirected,v,w,name);if(_.has(this._edgeLabels,e)){if(valueSpecified){this._edgeLabels[e]=value}return this}if(!_.isUndefined(name)&&!this._isMultigraph){throw new Error("Cannot set a named edge when isMultigraph = false")}this.setNode(v);this.setNode(w);this._edgeLabels[e]=valueSpecified?value:this._defaultEdgeLabelFn(v,w,name);var edgeObj=edgeArgsToObj(this._isDirected,v,w,name);v=edgeObj.v;w=edgeObj.w;Object.freeze(edgeObj);this._edgeObjs[e]=edgeObj;incrementOrInitEntry(this._preds[w],v);incrementOrInitEntry(this._sucs[v],w);this._in[w][e]=edgeObj;this._out[v][e]=edgeObj;this._edgeCount++;return this};Graph.prototype.edge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return this._edgeLabels[e]};Graph.prototype.hasEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return _.has(this._edgeLabels,e)};Graph.prototype.removeEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name),edge=this._edgeObjs[e];if(edge){v=edge.v;w=edge.w;delete this._edgeLabels[e];delete this._edgeObjs[e];decrementOrRemoveEntry(this._preds[w],v);decrementOrRemoveEntry(this._sucs[v],w);delete this._in[w][e];delete this._out[v][e];this._edgeCount--}return this};Graph.prototype.inEdges=function(v,u){var inV=this._in[v];if(inV){var edges=_.values(inV);if(!u){return edges}return _.filter(edges,function(edge){return edge.v===u})}};Graph.prototype.outEdges=function(v,w){var outV=this._out[v];if(outV){var edges=_.values(outV);if(!w){return edges}return _.filter(edges,function(edge){return edge.w===w})}};Graph.prototype.nodeEdges=function(v,w){var inEdges=this.inEdges(v,w);if(inEdges){return inEdges.concat(this.outEdges(v,w))}};function incrementOrInitEntry(map,k){if(_.has(map,k)){map[k]++}else{map[k]=1}}function decrementOrRemoveEntry(map,k){if(!--map[k]){delete map[k]}}function edgeArgsToId(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}return v+EDGE_KEY_DELIM+w+EDGE_KEY_DELIM+(_.isUndefined(name)?DEFAULT_EDGE_NAME:name)}function edgeArgsToObj(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}var edgeObj={v:v,w:w};if(name){edgeObj.name=name}return edgeObj}function edgeObjToId(isDirected,edgeObj){return edgeArgsToId(isDirected,edgeObj.v,edgeObj.w,edgeObj.name)}},{"./lodash":75}],73:[function(require,module,exports){module.exports={Graph:require("./graph"),version:require("./version")}},{"./graph":72,"./version":76}],74:[function(require,module,exports){var _=require("./lodash"),Graph=require("./graph");module.exports={write:write,read:read};function write(g){var json={options:{directed:g.isDirected(),multigraph:g.isMultigraph(),compound:g.isCompound()},nodes:writeNodes(g),edges:writeEdges(g)};if(!_.isUndefined(g.graph())){json.value=_.clone(g.graph())}return json}function writeNodes(g){return _.map(g.nodes(),function(v){var nodeValue=g.node(v),parent=g.parent(v),node={v:v};if(!_.isUndefined(nodeValue)){node.value=nodeValue}if(!_.isUndefined(parent)){node.parent=parent}return node})}function writeEdges(g){return _.map(g.edges(),function(e){var edgeValue=g.edge(e),edge={v:e.v,w:e.w};if(!_.isUndefined(e.name)){edge.name=e.name}if(!_.isUndefined(edgeValue)){edge.value=edgeValue}return edge})}function read(json){var g=new Graph(json.options).setGraph(json.value);_.each(json.nodes,function(entry){g.setNode(entry.v,entry.value);if(entry.parent){g.setParent(entry.v,entry.parent)}});_.each(json.edges,function(entry){g.setEdge({v:entry.v,w:entry.w,name:entry.name},entry.value)});return g}},{"./graph":72,"./lodash":75}],75:[function(require,module,exports){module.exports=require(20)},{"/Users/andrew/Documents/dev/dagre-d3/lib/lodash.js":20,lodash:77}],76:[function(require,module,exports){module.exports="1.0.1"},{}],77:[function(require,module,exports){(function(global){(function(){var undefined;var arrayPool=[],objectPool=[];var idCounter=0;var keyPrefix=+new Date+"";var largeArraySize=75;var maxPoolSize=40;var whitespace=" \f \ufeff"+"\n\r\u2028\u2029"+" ᠎              ";var reEmptyStringLeading=/\b__p \+= '';/g,reEmptyStringMiddle=/\b(__p \+=) '' \+/g,reEmptyStringTrailing=/(__e\(.*?\)|\b__t\)) \+\n'';/g;var reEsTemplate=/\$\{([^\\}]*(?:\\.[^\\}]*)*)\}/g;var reFlags=/\w*$/;var reFuncName=/^\s*function[ \n\r\t]+\w/;var reInterpolate=/<%=([\s\S]+?)%>/g;var reLeadingSpacesAndZeros=RegExp("^["+whitespace+"]*0+(?=.$)");var reNoMatch=/($^)/;var reThis=/\bthis\b/;var reUnescapedString=/['\n\r\t\u2028\u2029\\]/g;var contextProps=["Array","Boolean","Date","Function","Math","Number","Object","RegExp","String","_","attachEvent","clearTimeout","isFinite","isNaN","parseInt","setTimeout"];var templateCounter=0;var argsClass="[object Arguments]",arrayClass="[object Array]",boolClass="[object Boolean]",dateClass="[object Date]",funcClass="[object Function]",numberClass="[object Number]",objectClass="[object Object]",regexpClass="[object RegExp]",stringClass="[object String]";var cloneableClasses={};cloneableClasses[funcClass]=false;cloneableClasses[argsClass]=cloneableClasses[arrayClass]=cloneableClasses[boolClass]=cloneableClasses[dateClass]=cloneableClasses[numberClass]=cloneableClasses[objectClass]=cloneableClasses[regexpClass]=cloneableClasses[stringClass]=true;var debounceOptions={leading:false,maxWait:0,trailing:false};var descriptor={configurable:false,enumerable:false,value:null,writable:false};var objectTypes={"boolean":false,"function":true,object:true,number:false,string:false,undefined:false};var stringEscapes={"\\":"\\","'":"'","\n":"n","\r":"r"," ":"t","\u2028":"u2028","\u2029":"u2029"};var root=objectTypes[typeof window]&&window||this;var freeExports=objectTypes[typeof exports]&&exports&&!exports.nodeType&&exports;var freeModule=objectTypes[typeof module]&&module&&!module.nodeType&&module;var moduleExports=freeModule&&freeModule.exports===freeExports&&freeExports;var freeGlobal=objectTypes[typeof global]&&global;if(freeGlobal&&(freeGlobal.global===freeGlobal||freeGlobal.window===freeGlobal)){root=freeGlobal}function baseIndexOf(array,value,fromIndex){var index=(fromIndex||0)-1,length=array?array.length:0;while(++index-1?0:-1:cache?0:-1}function cachePush(value){var cache=this.cache,type=typeof value;if(type=="boolean"||value==null){cache[value]=true}else{if(type!="number"&&type!="string"){type="object"}var key=type=="number"?value:keyPrefix+value,typeCache=cache[type]||(cache[type]={});if(type=="object"){(typeCache[key]||(typeCache[key]=[])).push(value)}else{typeCache[key]=true}}}function charAtCallback(value){return value.charCodeAt(0)}function compareAscending(a,b){var ac=a.criteria,bc=b.criteria,index=-1,length=ac.length;while(++indexother||typeof value=="undefined"){return 1}if(value/g,evaluate:/<%([\s\S]+?)%>/g,interpolate:reInterpolate,variable:"",imports:{_:lodash}};function baseBind(bindData){var func=bindData[0],partialArgs=bindData[2],thisArg=bindData[4];function bound(){if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(this instanceof bound){var thisBinding=baseCreate(func.prototype),result=func.apply(thisBinding,args||arguments);return isObject(result)?result:thisBinding}return func.apply(thisArg,args||arguments)}setBindData(bound,bindData);return bound}function baseClone(value,isDeep,callback,stackA,stackB){if(callback){var result=callback(value);if(typeof result!="undefined"){return result}}var isObj=isObject(value);if(isObj){var className=toString.call(value);if(!cloneableClasses[className]){return value}var ctor=ctorByClass[className];switch(className){case boolClass:case dateClass:return new ctor(+value);case numberClass:case stringClass:return new ctor(value);case regexpClass:result=ctor(value.source,reFlags.exec(value));result.lastIndex=value.lastIndex;return result}}else{return value}var isArr=isArray(value);if(isDeep){var initedStack=!stackA;stackA||(stackA=getArray());stackB||(stackB=getArray());var length=stackA.length;while(length--){if(stackA[length]==value){return stackB[length]}}result=isArr?ctor(value.length):{}}else{result=isArr?slice(value):assign({},value)}if(isArr){if(hasOwnProperty.call(value,"index")){result.index=value.index}if(hasOwnProperty.call(value,"input")){result.input=value.input}}if(!isDeep){return result}stackA.push(value);stackB.push(result);(isArr?forEach:forOwn)(value,function(objValue,key){result[key]=baseClone(objValue,isDeep,callback,stackA,stackB)});if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseCreate(prototype,properties){return isObject(prototype)?nativeCreate(prototype):{}}if(!nativeCreate){baseCreate=function(){function Object(){}return function(prototype){if(isObject(prototype)){Object.prototype=prototype;var result=new Object;Object.prototype=null}return result||context.Object()}}()}function baseCreateCallback(func,thisArg,argCount){if(typeof func!="function"){return identity}if(typeof thisArg=="undefined"||!("prototype"in func)){return func}var bindData=func.__bindData__;if(typeof bindData=="undefined"){if(support.funcNames){bindData=!func.name}bindData=bindData||!support.funcDecomp;if(!bindData){var source=fnToString.call(func);if(!support.funcNames){bindData=!reFuncName.test(source)}if(!bindData){bindData=reThis.test(source);setBindData(func,bindData)}}}if(bindData===false||bindData!==true&&bindData[1]&1){return func}switch(argCount){case 1:return function(value){return func.call(thisArg,value)};case 2:return function(a,b){return func.call(thisArg,a,b)};case 3:return function(value,index,collection){return func.call(thisArg,value,index,collection)};case 4:return function(accumulator,value,index,collection){return func.call(thisArg,accumulator,value,index,collection)}}return bind(func,thisArg)}function baseCreateWrapper(bindData){var func=bindData[0],bitmask=bindData[1],partialArgs=bindData[2],partialRightArgs=bindData[3],thisArg=bindData[4],arity=bindData[5];var isBind=bitmask&1,isBindKey=bitmask&2,isCurry=bitmask&4,isCurryBound=bitmask&8,key=func;function bound(){var thisBinding=isBind?thisArg:this;if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(partialRightArgs||isCurry){args||(args=slice(arguments));if(partialRightArgs){push.apply(args,partialRightArgs)}if(isCurry&&args.length=largeArraySize&&indexOf===baseIndexOf,result=[]; + +if(isLarge){var cache=createCache(values);if(cache){indexOf=cacheIndexOf;values=cache}else{isLarge=false}}while(++index-1}})}}stackA.pop();stackB.pop();if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseMerge(object,source,callback,stackA,stackB){(isArray(source)?forEach:forOwn)(source,function(source,key){var found,isArr,result=source,value=object[key];if(source&&((isArr=isArray(source))||isPlainObject(source))){var stackLength=stackA.length;while(stackLength--){if(found=stackA[stackLength]==source){value=stackB[stackLength];break}}if(!found){var isShallow;if(callback){result=callback(value,source);if(isShallow=typeof result!="undefined"){value=result}}if(!isShallow){value=isArr?isArray(value)?value:[]:isPlainObject(value)?value:{}}stackA.push(source);stackB.push(value);if(!isShallow){baseMerge(value,source,callback,stackA,stackB)}}}else{if(callback){result=callback(value,source);if(typeof result=="undefined"){result=source}}if(typeof result!="undefined"){value=result}}object[key]=value})}function baseRandom(min,max){return min+floor(nativeRandom()*(max-min+1))}function baseUniq(array,isSorted,callback){var index=-1,indexOf=getIndexOf(),length=array?array.length:0,result=[];var isLarge=!isSorted&&length>=largeArraySize&&indexOf===baseIndexOf,seen=callback||isLarge?getArray():result;if(isLarge){var cache=createCache(seen);indexOf=cacheIndexOf;seen=cache}while(++index":">",'"':""","'":"'"};var htmlUnescapes=invert(htmlEscapes);var reEscapedHtml=RegExp("("+keys(htmlUnescapes).join("|")+")","g"),reUnescapedHtml=RegExp("["+keys(htmlEscapes).join("")+"]","g");var assign=function(object,source,guard){var index,iterable=object,result=iterable;if(!iterable)return result;var args=arguments,argsIndex=0,argsLength=typeof guard=="number"?2:args.length;if(argsLength>3&&typeof args[argsLength-2]=="function"){var callback=baseCreateCallback(args[--argsLength-1],args[argsLength--],2)}else if(argsLength>2&&typeof args[argsLength-1]=="function"){callback=args[--argsLength]}while(++argsIndex3&&typeof args[length-2]=="function"){var callback=baseCreateCallback(args[--length-1],args[length--],2)}else if(length>2&&typeof args[length-1]=="function"){callback=args[--length]}var sources=slice(arguments,1,length),index=-1,stackA=getArray(),stackB=getArray();while(++index-1}else if(typeof length=="number"){result=(isString(collection)?collection.indexOf(target,fromIndex):indexOf(collection,target,fromIndex))>-1}else{forOwn(collection,function(value){if(++index>=fromIndex){return!(result=value===target)}})}return result}var countBy=createAggregator(function(result,value,key){hasOwnProperty.call(result,key)?result[key]++:result[key]=1});function every(collection,callback,thisArg){var result=true;callback=lodash.createCallback(callback,thisArg,3);var index=-1,length=collection?collection.length:0;if(typeof length=="number"){while(++indexresult){result=value}}}else{callback=callback==null&&isString(collection)?charAtCallback:lodash.createCallback(callback,thisArg,3);forEach(collection,function(value,index,collection){var current=callback(value,index,collection);if(current>computed){computed=current;result=value}})}return result}function min(collection,callback,thisArg){var computed=Infinity,result=computed;if(typeof callback!="function"&&thisArg&&thisArg[callback]===collection){callback=null}if(callback==null&&isArray(collection)){var index=-1,length=collection.length;while(++index=largeArraySize&&createCache(argsIndex?args[argsIndex]:seen))}}var array=args[0],index=-1,length=array?array.length:0,result=[];outer:while(++index>>1;callback(array[mid])1?arguments:arguments[0],index=-1,length=array?max(pluck(array,"length")):0,result=Array(length<0?0:length);while(++index2?createWrapper(func,17,slice(arguments,2),null,thisArg):createWrapper(func,1,null,null,thisArg)}function bindAll(object){var funcs=arguments.length>1?baseFlatten(arguments,true,false,1):functions(object),index=-1,length=funcs.length;while(++index2?createWrapper(key,19,slice(arguments,2),null,object):createWrapper(key,3,null,null,object)}function compose(){var funcs=arguments,length=funcs.length;while(length--){if(!isFunction(funcs[length])){throw new TypeError}}return function(){var args=arguments,length=funcs.length;while(length--){args=[funcs[length].apply(this,args)]}return args[0]}}function curry(func,arity){arity=typeof arity=="number"?arity:+arity||func.length;return createWrapper(func,4,null,null,null,arity)}function debounce(func,wait,options){var args,maxTimeoutId,result,stamp,thisArg,timeoutId,trailingCall,lastCalled=0,maxWait=false,trailing=true;if(!isFunction(func)){throw new TypeError}wait=nativeMax(0,wait)||0;if(options===true){var leading=true;trailing=false}else if(isObject(options)){leading=options.leading;maxWait="maxWait"in options&&(nativeMax(wait,options.maxWait)||0);trailing="trailing"in options?options.trailing:trailing}var delayed=function(){var remaining=wait-(now()-stamp);if(remaining<=0){if(maxTimeoutId){clearTimeout(maxTimeoutId); + +}var isCalled=trailingCall;maxTimeoutId=timeoutId=trailingCall=undefined;if(isCalled){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}}else{timeoutId=setTimeout(delayed,remaining)}};var maxDelayed=function(){if(timeoutId){clearTimeout(timeoutId)}maxTimeoutId=timeoutId=trailingCall=undefined;if(trailing||maxWait!==wait){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}};return function(){args=arguments;stamp=now();thisArg=this;trailingCall=trailing&&(timeoutId||!leading);if(maxWait===false){var leadingCall=leading&&!timeoutId}else{if(!maxTimeoutId&&!leading){lastCalled=stamp}var remaining=maxWait-(stamp-lastCalled),isCalled=remaining<=0;if(isCalled){if(maxTimeoutId){maxTimeoutId=clearTimeout(maxTimeoutId)}lastCalled=stamp;result=func.apply(thisArg,args)}else if(!maxTimeoutId){maxTimeoutId=setTimeout(maxDelayed,remaining)}}if(isCalled&&timeoutId){timeoutId=clearTimeout(timeoutId)}else if(!timeoutId&&wait!==maxWait){timeoutId=setTimeout(delayed,wait)}if(leadingCall){isCalled=true;result=func.apply(thisArg,args)}if(isCalled&&!timeoutId&&!maxTimeoutId){args=thisArg=null}return result}}function defer(func){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,1);return setTimeout(function(){func.apply(undefined,args)},1)}function delay(func,wait){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,2);return setTimeout(function(){func.apply(undefined,args)},wait)}function memoize(func,resolver){if(!isFunction(func)){throw new TypeError}var memoized=function(){var cache=memoized.cache,key=resolver?resolver.apply(this,arguments):keyPrefix+arguments[0];return hasOwnProperty.call(cache,key)?cache[key]:cache[key]=func.apply(this,arguments)};memoized.cache={};return memoized}function once(func){var ran,result;if(!isFunction(func)){throw new TypeError}return function(){if(ran){return result}ran=true;result=func.apply(this,arguments);func=null;return result}}function partial(func){return createWrapper(func,16,slice(arguments,1))}function partialRight(func){return createWrapper(func,32,null,slice(arguments,1))}function throttle(func,wait,options){var leading=true,trailing=true;if(!isFunction(func)){throw new TypeError}if(options===false){leading=false}else if(isObject(options)){leading="leading"in options?options.leading:leading;trailing="trailing"in options?options.trailing:trailing}debounceOptions.leading=leading;debounceOptions.maxWait=wait;debounceOptions.trailing=trailing;return debounce(func,wait,debounceOptions)}function wrap(value,wrapper){return createWrapper(wrapper,16,[value])}function constant(value){return function(){return value}}function createCallback(func,thisArg,argCount){var type=typeof func;if(func==null||type=="function"){return baseCreateCallback(func,thisArg,argCount)}if(type!="object"){return property(func)}var props=keys(func),key=props[0],a=func[key];if(props.length==1&&a===a&&!isObject(a)){return function(object){var b=object[key];return a===b&&(a!==0||1/a==1/b)}}return function(object){var length=props.length,result=false;while(length--){if(!(result=baseIsEqual(object[props[length]],func[props[length]],null,true))){break}}return result}}function escape(string){return string==null?"":String(string).replace(reUnescapedHtml,escapeHtmlChar)}function identity(value){return value}function mixin(object,source,options){var chain=true,methodNames=source&&functions(source);if(!source||!options&&!methodNames.length){if(options==null){options=source}ctor=lodashWrapper;source=object;object=lodash;methodNames=functions(source)}if(options===false){chain=false}else if(isObject(options)&&"chain"in options){chain=options.chain}var ctor=object,isFunc=isFunction(ctor);forEach(methodNames,function(methodName){var func=object[methodName]=source[methodName];if(isFunc){ctor.prototype[methodName]=function(){var chainAll=this.__chain__,value=this.__wrapped__,args=[value];push.apply(args,arguments);var result=func.apply(object,args);if(chain||chainAll){if(value===result&&isObject(result)){return this}result=new ctor(result);result.__chain__=chainAll}return result}}})}function noConflict(){context._=oldDash;return this}function noop(){}var now=isNative(now=Date.now)&&now||function(){return(new Date).getTime()};var parseInt=nativeParseInt(whitespace+"08")==8?nativeParseInt:function(value,radix){return nativeParseInt(isString(value)?value.replace(reLeadingSpacesAndZeros,""):value,radix||0)};function property(key){return function(object){return object[key]}}function random(min,max,floating){var noMin=min==null,noMax=max==null;if(floating==null){if(typeof min=="boolean"&&noMax){floating=min;min=1}else if(!noMax&&typeof max=="boolean"){floating=max;noMax=true}}if(noMin&&noMax){max=1}min=+min||0;if(noMax){max=min;min=0}else{max=+max||0}if(floating||min%1||max%1){var rand=nativeRandom();return nativeMin(min+rand*(max-min+parseFloat("1e-"+((rand+"").length-1))),max)}return baseRandom(min,max)}function result(object,key){if(object){var value=object[key];return isFunction(value)?object[key]():value}}function template(text,data,options){var settings=lodash.templateSettings;text=String(text||"");options=defaults({},options,settings);var imports=defaults({},options.imports,settings.imports),importsKeys=keys(imports),importsValues=values(imports);var isEvaluating,index=0,interpolate=options.interpolate||reNoMatch,source="__p += '";var reDelimiters=RegExp((options.escape||reNoMatch).source+"|"+interpolate.source+"|"+(interpolate===reInterpolate?reEsTemplate:reNoMatch).source+"|"+(options.evaluate||reNoMatch).source+"|$","g");text.replace(reDelimiters,function(match,escapeValue,interpolateValue,esTemplateValue,evaluateValue,offset){interpolateValue||(interpolateValue=esTemplateValue);source+=text.slice(index,offset).replace(reUnescapedString,escapeStringChar);if(escapeValue){source+="' +\n__e("+escapeValue+") +\n'"}if(evaluateValue){isEvaluating=true;source+="';\n"+evaluateValue+";\n__p += '"}if(interpolateValue){source+="' +\n((__t = ("+interpolateValue+")) == null ? '' : __t) +\n'"}index=offset+match.length;return match});source+="';\n";var variable=options.variable,hasVariable=variable;if(!hasVariable){variable="obj";source="with ("+variable+") {\n"+source+"\n}\n"}source=(isEvaluating?source.replace(reEmptyStringLeading,""):source).replace(reEmptyStringMiddle,"$1").replace(reEmptyStringTrailing,"$1;");source="function("+variable+") {\n"+(hasVariable?"":variable+" || ("+variable+" = {});\n")+"var __t, __p = '', __e = _.escape"+(isEvaluating?", __j = Array.prototype.join;\n"+"function print() { __p += __j.call(arguments, '') }\n":";\n")+source+"return __p\n}";var sourceURL="\n/*\n//# sourceURL="+(options.sourceURL||"/lodash/template/source["+templateCounter++ +"]")+"\n*/";try{var result=Function(importsKeys,"return "+source+sourceURL).apply(undefined,importsValues)}catch(e){e.source=source;throw e}if(data){return result(data)}result.source=source;return result}function times(n,callback,thisArg){n=(n=+n)>-1?n:0;var index=-1,result=Array(n);callback=baseCreateCallback(callback,thisArg,1);while(++index 0) { - // Take into account the position and width of the last stage's container - var existingStages = stageClusters(); + var existingStages = svgContainer + .selectAll("g.cluster") + .filter("[id*=\"" + VizConstants.stageClusterPrefix + "\"]"); if (!existingStages.empty()) { - var lastStage = existingStages[0].pop(); - var lastStageId = d3.select(lastStage).attr("id"); - var lastStageWidth = toFloat(d3.select("#" + lastStageId + " rect").attr("width")); - var lastStagePosition = getAbsolutePosition(lastStageId); + var lastStage = d3.select(existingStages[0].pop()); + var lastStageId = lastStage.attr("id"); + var lastStageWidth = toFloat(svgContainer + .select("#" + lastStageId) + .select("rect") + .attr("width")); + var lastStagePosition = getAbsolutePosition(lastStage); var offset = lastStagePosition.x + lastStageWidth + VizConstants.stageSep; container.attr("transform", "translate(" + offset + ", 0)"); } } - renderDot(dot, container); + + // Actually render the stage + renderDot(dot, container, JobPageVizConstants.rankSep); + + // Round corners on rectangles + container + .selectAll("rect") + .attr("rx", "4") + .attr("ry", "4"); + // If there are any incoming edges into this graph, keep track of them to render // them separately later. Note that we cannot draw them now because we need to // put these edges in a separate container that is on top of all stage graphs. @@ -221,121 +226,173 @@ function renderDagVizForJob(svgContainer) { }); }); - // Draw edges that cross stages - if (crossStageEdges.length > 0) { - var container = svgContainer.append("g").attr("id", "cross-stage-edges"); - for (var i = 0; i < crossStageEdges.length; i++) { - var fromRDDId = crossStageEdges[i][0]; - var toRDDId = crossStageEdges[i][1]; - connectRDDs(fromRDDId, toRDDId, container); - } - } + addTooltipsForRDDs(svgContainer); + drawCrossStageEdges(crossStageEdges, svgContainer); } /* Render the dot file as an SVG in the given container. */ -function renderDot(dot, container) { +function renderDot(dot, container, rankSep) { var escaped_dot = dot .replace(/</g, "<") .replace(/>/g, ">") .replace(/"/g, "\""); var g = graphlibDot.read(escaped_dot); + g.graph().rankSep = rankSep; var renderer = new dagreD3.render(); renderer(container, g); } -/* Style the visualization we just rendered. */ -function styleDagViz(forJob) { - graphContainer().selectAll("svg g.cluster rect") - .style("fill", "white") - .style("stroke", VizConstants.rddOperationColor) - .style("stroke-width", "4px") - .style("stroke-opacity", "0.5"); - graphContainer().selectAll("svg g.cluster text") - .attr("fill", VizConstants.clusterLabelColor) - .attr("font-size", "11px"); - graphContainer().selectAll("svg path") - .style("stroke", VizConstants.edgeColor) - .style("stroke-width", VizConstants.edgeWidth); - stageClusters() - .select("rect") - .style("stroke", VizConstants.stageColor) - .style("strokeWidth", "6px"); - - // Put an arrow at the end of every edge - // We need to do this because we manually render some edges ourselves - // For these edges, we borrow the arrow marker generated by dagre-d3 - var dagreD3Marker = graphContainer().select("svg g.edgePaths marker").node(); - graphContainer().select("svg") - .append(function() { return dagreD3Marker.cloneNode(true); }) - .attr("id", "marker-arrow") - .select("path") - .attr("fill", VizConstants.edgeColor) - .attr("strokeWidth", "0px"); - graphContainer().selectAll("svg g > path").attr("marker-end", "url(#marker-arrow)"); - graphContainer().selectAll("svg g.edgePaths def").remove(); // We no longer need these - - // Apply any job or stage specific styles +/* -------------------- * + * | Helper functions | * + * -------------------- */ + +// Helper d3 accessors +function graphContainer() { return d3.select("#dag-viz-graph"); } +function metadataContainer() { return d3.select("#dag-viz-metadata"); } + +/* + * Helper function to create draw a label for each cluster. + * + * We need to do this manually because dagre-d3 does not support labeling clusters. + * In general, the clustering support for dagre-d3 is quite limited at this point. + */ +function drawClusterLabels(svgContainer, forJob) { + var clusterLabelSize, stageClusterLabelSize; if (forJob) { - styleDagVizForJob(); + clusterLabelSize = JobPageVizConstants.clusterLabelSize; + stageClusterLabelSize = JobPageVizConstants.stageClusterLabelSize; } else { - styleDagVizForStage(); + clusterLabelSize = StagePageVizConstants.clusterLabelSize; + stageClusterLabelSize = StagePageVizConstants.stageClusterLabelSize; } + svgContainer.selectAll("g.cluster").each(function() { + var cluster = d3.select(this); + var isStage = cluster.attr("id").indexOf(VizConstants.stageClusterPrefix) > -1; + var labelSize = isStage ? stageClusterLabelSize : clusterLabelSize; + drawClusterLabel(cluster, labelSize); + }); } -/* Apply job-page-specific style to the visualization. */ -function styleDagVizForJob() { - graphContainer().selectAll("svg g.node circle") - .style("fill", VizConstants.rddColor); - // TODO: add a legend to explain what a highlighted dot means - graphContainer().selectAll("svg g.cached circle") - .style("fill", VizConstants.rddCachedColor); - graphContainer().selectAll("svg g#cross-stage-edges path") - .style("fill", "none"); +/* + * Helper function to draw a label for the given cluster element based on its name. + * + * In the process, we need to expand the bounding box to make room for the label. + * We need to do this because dagre-d3 did not take this into account when it first + * rendered the bounding boxes. Note that this means we need to adjust the view box + * of the SVG afterwards since we shifted a few boxes around. + */ +function drawClusterLabel(d3cluster, fontSize) { + var cluster = d3cluster; + var rect = d3cluster.select("rect"); + rect.attr("y", toFloat(rect.attr("y")) - fontSize); + rect.attr("height", toFloat(rect.attr("height")) + fontSize); + var labelX = toFloat(rect.attr("x")) + toFloat(rect.attr("width")) - fontSize / 2; + var labelY = toFloat(rect.attr("y")) + fontSize * 1.5; + var labelText = cluster.attr("name").replace(VizConstants.clusterPrefix, ""); + cluster.append("text") + .attr("x", labelX) + .attr("y", labelY) + .attr("text-anchor", "end") + .style("font-size", fontSize + "px") + .text(labelText); } -/* Apply stage-page-specific style to the visualization. */ -function styleDagVizForStage() { - graphContainer().selectAll("svg g.node rect") - .style("fill", "none") - .style("stroke", VizConstants.rddColor) - .style("stroke-width", "2px") - .attr("rx", "5") // round corners - .attr("ry", "5"); - // TODO: add a legend to explain what a highlighted RDD means - graphContainer().selectAll("svg g.cached rect") - .style("stroke", VizConstants.rddCachedColor); - graphContainer().selectAll("svg g.node g.label text tspan") - .style("fill", VizConstants.rddColor); +/* + * Helper function to size the SVG appropriately such that all elements are displyed. + * This assumes that all outermost elements are clusters (rectangles). + */ +function resizeSvg(svg) { + var allClusters = svg.selectAll("g.cluster rect")[0]; + var startX = -VizConstants.svgMarginX + + toFloat(d3.min(allClusters, function(e) { + return getAbsolutePosition(d3.select(e)).x; + })); + var startY = -VizConstants.svgMarginY + + toFloat(d3.min(allClusters, function(e) { + return getAbsolutePosition(d3.select(e)).y; + })); + var endX = VizConstants.svgMarginX + + toFloat(d3.max(allClusters, function(e) { + var t = d3.select(e); + return getAbsolutePosition(t).x + toFloat(t.attr("width")); + })); + var endY = VizConstants.svgMarginY + + toFloat(d3.max(allClusters, function(e) { + var t = d3.select(e); + return getAbsolutePosition(t).y + toFloat(t.attr("height")); + })); + var width = endX - startX; + var height = endY - startY; + svg.attr("viewBox", startX + " " + startY + " " + width + " " + height) + .attr("width", width) + .attr("height", height); +} + +/* + * (Job page only) Helper function to draw edges that cross stage boundaries. + * We need to do this manually because we render each stage separately in dagre-d3. + */ +function drawCrossStageEdges(edges, svgContainer) { + if (edges.length == 0) { + return; + } + // Draw the paths first + var edgesContainer = svgContainer.append("g").attr("id", "cross-stage-edges"); + for (var i = 0; i < edges.length; i++) { + var fromRDDId = edges[i][0]; + var toRDDId = edges[i][1]; + connectRDDs(fromRDDId, toRDDId, edgesContainer, svgContainer); + } + // Now draw the arrows by borrowing the arrow marker generated by dagre-d3 + var dagreD3Marker = svgContainer.select("g.edgePaths marker"); + if (!dagreD3Marker.empty()) { + svgContainer + .append(function() { return dagreD3Marker.node().cloneNode(true); }) + .attr("id", "marker-arrow"); + svgContainer.selectAll("g > path").attr("marker-end", "url(#marker-arrow)"); + svgContainer.selectAll("g.edgePaths def").remove(); // We no longer need these + } } /* - * (Job page only) Helper method to compute the absolute - * position of the group element identified by the given ID. + * (Job page only) Helper function to compute the absolute + * position of the specified element in our graph. */ -function getAbsolutePosition(groupId) { - var obj = d3.select("#" + groupId).filter("g"); - var _x = 0, _y = 0; +function getAbsolutePosition(d3selection) { + if (d3selection.empty()) { + throw "Attempted to get absolute position of an empty selection."; + } + var obj = d3selection; + var _x = toFloat(obj.attr("x")) || 0; + var _y = toFloat(obj.attr("y")) || 0; while (!obj.empty()) { var transformText = obj.attr("transform"); - var translate = d3.transform(transformText).translate - _x += translate[0]; - _y += translate[1]; - obj = d3.select(obj.node().parentNode).filter("g") + if (transformText) { + var translate = d3.transform(transformText).translate; + _x += toFloat(translate[0]); + _y += toFloat(translate[1]); + } + // Climb upwards to find how our parents are translated + obj = d3.select(obj.node().parentNode); + // Stop when we've reached the graph container itself + if (obj.node() == graphContainer().node()) { + break; + } } return { x: _x, y: _y }; } -/* (Job page only) Connect two RDD nodes with a curved edge. */ -function connectRDDs(fromRDDId, toRDDId, container) { +/* (Job page only) Helper function to connect two RDDs with a curved edge. */ +function connectRDDs(fromRDDId, toRDDId, edgesContainer, svgContainer) { var fromNodeId = VizConstants.nodePrefix + fromRDDId; - var toNodeId = VizConstants.nodePrefix + toRDDId - var fromPos = getAbsolutePosition(fromNodeId); - var toPos = getAbsolutePosition(toNodeId); + var toNodeId = VizConstants.nodePrefix + toRDDId; + var fromPos = getAbsolutePosition(svgContainer.select("#" + fromNodeId)); + var toPos = getAbsolutePosition(svgContainer.select("#" + toNodeId)); // On the job page, RDDs are rendered as dots (circles). When rendering the path, // we need to account for the radii of these circles. Otherwise the arrow heads // will bleed into the circle itself. - var delta = toFloat(graphContainer() + var delta = toFloat(svgContainer .select("g.node#" + toNodeId) .select("circle") .attr("r")); @@ -347,12 +404,13 @@ function connectRDDs(fromRDDId, toRDDId, container) { toPos.x += delta; } + var points; if (fromPos.y == toPos.y) { // If they are on the same rank, curve the middle part of the edge // upward a little to avoid interference with things in between // e.g. _______ // _____/ \_____ - var points = [ + points = [ [fromPos.x, fromPos.y], [fromPos.x + (toPos.x - fromPos.x) * 0.2, fromPos.y], [fromPos.x + (toPos.x - fromPos.x) * 0.3, fromPos.y - 20], @@ -366,7 +424,7 @@ function connectRDDs(fromRDDId, toRDDId, container) { // / // | // _____/ - var points = [ + points = [ [fromPos.x, fromPos.y], [fromPos.x + (toPos.x - fromPos.x) * 0.4, fromPos.y], [fromPos.x + (toPos.x - fromPos.x) * 0.6, toPos.y], @@ -375,18 +433,30 @@ function connectRDDs(fromRDDId, toRDDId, container) { } var line = d3.svg.line().interpolate("basis"); - container.append("path").datum(points).attr("d", line); + edgesContainer.append("path").datum(points).attr("d", line); } -/* Helper d3 accessor to clusters that represent stages. */ -function stageClusters() { - return graphContainer().selectAll("g.cluster").filter(function() { - return d3.select(this).attr("id").indexOf(VizConstants.stageClusterPrefix) > -1; +/* (Job page only) Helper function to add tooltips for RDDs. */ +function addTooltipsForRDDs(svgContainer) { + svgContainer.selectAll("g.node").each(function() { + var node = d3.select(this); + var tooltipText = node.attr("name"); + if (tooltipText) { + node.select("circle") + .attr("data-toggle", "tooltip") + .attr("data-placement", "right") + .attr("title", tooltipText) + } }); + $("[data-toggle=tooltip]").tooltip({container: "body"}); } -/* Helper method to convert attributes to numeric values. */ +/* Helper function to convert attributes to numeric values. */ function toFloat(f) { - return parseFloat(f.replace(/px$/, "")); + if (f) { + return parseFloat(f.toString().replace(/px$/, "")); + } else { + return f; + } } diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css index 35ef14e5aaf1a..d40de704229c3 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css @@ -23,6 +23,10 @@ div#application-timeline, div#job-timeline { margin-top: 5px; } +.vis.timeline { + line-height: 14px; +} + .vis.timeline div.content { width: 100%; } @@ -32,48 +36,55 @@ div#application-timeline, div#job-timeline { } .vis.timeline .item.stage.succeeded { - background-color: #D5DDF6; + background-color: #A0DFFF; + border-color: #3EC0FF; } .vis.timeline .item.stage.succeeded.selected { - background-color: #D5DDF6; - border-color: #97B0F8; - z-index: auto; + background-color: #A0DFFF; + border-color: #3EC0FF; + z-index: auto; } .legend-area rect.completed-stage-legend { - fill: #D5DDF6; - stroke: #97B0F8; + fill: #A0DFFF; + stroke: #3EC0FF; } .vis.timeline .item.stage.failed { - background-color: #FF5475; + background-color: #FFA1B0; + border-color: #FF4D6D; } .vis.timeline .item.stage.failed.selected { - background-color: #FF5475; - border-color: #97B0F8; - z-index: auto; + background-color: #FFA1B0; + border-color: #FF4D6D; + z-index: auto; } .legend-area rect.failed-stage-legend { - fill: #FF5475; - stroke: #97B0F8; + fill: #FFA1B0; + stroke: #FF4D6D; } .vis.timeline .item.stage.running { - background-color: #FDFFCA; + background-color: #A2FCC0; + border-color: #36F572; } .vis.timeline .item.stage.running.selected { - background-color: #FDFFCA; - border-color: #97B0F8; - z-index: auto; + background-color: #A2FCC0; + border-color: #36F572; + z-index: auto; } .legend-area rect.active-stage-legend { - fill: #FDFFCA; - stroke: #97B0F8; + fill: #A2FCC0; + stroke: #36F572; +} + +.vis.timeline .foreground { + cursor: move; } .vis.timeline .item.job { @@ -81,76 +92,81 @@ div#application-timeline, div#job-timeline { } .vis.timeline .item.job.succeeded { - background-color: #D5DDF6; + background-color: #A0DFFF; + border-color: #3EC0FF; } .vis.timeline .item.job.succeeded.selected { - background-color: #D5DDF6; - border-color: #97B0F8; - z-index: auto; + background-color: #A0DFFF; + border-color: #3EC0FF; + z-index: auto; } .legend-area rect.succeeded-job-legend { - fill: #D5DDF6; - stroke: #97B0F8; + fill: #A0DFFF; + stroke: #3EC0FF; } .vis.timeline .item.job.failed { - background-color: #FF5475; + background-color: #FFA1B0; + border-color: #FF4D6D; } .vis.timeline .item.job.failed.selected { - background-color: #FF5475; - border-color: #97B0F8; - z-index: auto; + background-color: #FFA1B0; + border-color: #FF4D6D; + z-index: auto; } .legend-area rect.failed-job-legend { - fill: #FF5475; - stroke: #97B0F8; + fill: #FFA1B0; + stroke: #FF4D6D; } .vis.timeline .item.job.running { - background-color: #FDFFCA; + background-color: #A2FCC0; + border-color: #36F572; } .vis.timeline .item.job.running.selected { - background-color: #FDFFCA; - border-color: #97B0F8; - z-index: auto; + background-color: #A2FCC0; + border-color: #36F572; + z-index: auto; } .legend-area rect.running-job-legend { - fill: #FDFFCA; - stroke: #97B0F8; + fill: #A2FCC0; + stroke: #36F572; } .vis.timeline .item.executor.added { - background-color: #D5DDF6; + background-color: #A0DFFF; + border-color: #3EC0FF; } .legend-area rect.executor-added-legend { - fill: #D5DDF6; - stroke: #97B0F8; + fill: #A0DFFF; + stroke: #3EC0FF; } .vis.timeline .item.executor.removed { - background-color: #EBCA59; + background-color: #FFA1B0; + border-color: #FF4D6D; } .legend-area rect.executor-removed-legend { - fill: #EBCA59; - stroke: #97B0F8; + fill: #FFA1B0; + stroke: #FF4D6D; } .vis.timeline .item.executor.selected { - border-color: #FFC200; - background-color: #FFF785; + background-color: #A2FCC0; + border-color: #36F572; z-index: 2; } -tr.corresponding-item-hover>td, tr.corresponding-item-hover>th { - background-color: #FFE1FA !important; +tr.corresponding-item-hover > td, tr.corresponding-item-hover > th { + background-color: #D6FFE4 !important; } #application-timeline.collapsed { @@ -165,11 +181,15 @@ tr.corresponding-item-hover>td, tr.corresponding-item-hover>th { margin-bottom: 5px; } +.control-panel input[type="checkbox"] { + margin: 0; +} + span.expand-application-timeline, span.expand-job-timeline { cursor: pointer; } -.control-panel input+span { +.control-panel input + span { cursor: pointer; } @@ -180,3 +200,17 @@ span.expand-application-timeline, span.expand-job-timeline { .vis.timeline .item .tooltip-inner { max-width: unset !important; } + +.vispanel.center { + font-size: 12px; + line-height: 12px; +} + +.legend-area text { + fill: #4D4D4D; +} + +.additional-metrics ul { + list-style: none; + margin-left: 15px; +} diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js index e4a891d47f035..48fbb33b1155b 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js @@ -156,9 +156,9 @@ function setupExecutorEventAction() { function setupZoomable(id, timeline) { $(id + '>input[type="checkbox"]').click(function() { if (this.checked) { - timeline.setOptions({zoomable: false}); - } else { timeline.setOptions({zoomable: true}); + } else { + timeline.setOptions({zoomable: false}); } }); diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis.map b/core/src/main/resources/org/apache/spark/ui/static/vis.map deleted file mode 100644 index 9be8b618989b7..0000000000000 --- a/core/src/main/resources/org/apache/spark/ui/static/vis.map +++ /dev/null @@ -1 +0,0 @@ -{"version":3,"file":"vis.map","sources":["./dist/vis.js"],"names":["root","factory","exports","module","define","amd","this","modules","__webpack_require__","moduleId","installedModules","id","loaded","call","m","c","p","util","DOMutil","DataSet","DataView","Queue","Graph3d","graph3d","Camera","Filter","Point2d","Point3d","Slider","StepNumber","Timeline","Graph2d","timeline","DateUtil","DataStep","Range","stack","TimeStep","components","items","Item","BackgroundItem","BoxItem","PointItem","RangeItem","Component","CurrentTime","CustomTime","DataAxis","GraphGroup","Group","BackgroundGroup","ItemSet","Legend","LineGraph","TimeAxis","Network","network","Edge","Groups","Images","Node","Popup","dotparser","gephiParser","Graph","Error","moment","hammer","Hammer","isNumber","object","Number","isString","String","isDate","Date","match","ASPDateRegex","exec","isNaN","parse","isDataTable","google","visualization","DataTable","randomUUID","S4","Math","floor","random","toString","extend","a","i","len","arguments","length","other","prop","hasOwnProperty","selectiveExtend","props","Array","isArray","selectiveDeepExtend","b","TypeError","constructor","Object","undefined","deepExtend","selectiveNotDeepExtend","indexOf","equalArray","convert","type","Boolean","valueOf","isMoment","toDate","getType","toISOString","value","getAbsoluteLeft","elem","getBoundingClientRect","left","getAbsoluteTop","top","addClassName","className","classes","split","push","join","removeClassName","index","splice","forEach","callback","toArray","array","updateProperty","key","addEventListener","element","action","listener","useCapture","navigator","userAgent","attachEvent","removeEventListener","detachEvent","preventDefault","event","window","returnValue","getTarget","target","srcElement","nodeType","parentNode","option","asBoolean","defaultValue","asNumber","asString","asSize","asElement","hexToRGB","hex","shorthandRegex","replace","r","g","result","parseInt","RGBToHex","red","green","blue","slice","parseColor","color","isValidRGB","rgb","substr","isValidHex","hsv","hexToHSV","lighterColorHSV","h","s","v","min","darkerColorHSV","darkerColorHex","HSVToHex","lighterColorHex","background","border","highlight","hover","RGBToHSV","minRGB","maxRGB","max","d","hue","saturation","cssUtil","cssText","styles","style","trim","parts","keys","map","addCssText","currentStyles","newStyles","removeCssText","removeStyles","HSVToRGB","f","q","t","isOk","test","selectiveBridgeObject","fields","referenceObject","objectTo","create","bridgeObject","mergeOptions","mergeTarget","options","enabled","binarySearchCustom","orderedItems","searchFunction","field","field2","maxIterations","iteration","low","high","middle","item","searchResult","binarySearchValue","sidePreference","prevValue","nextValue","easeInOutQuad","start","end","duration","change","easingFunctions","linear","easeInQuad","easeOutQuad","easeInCubic","easeOutCubic","easeInOutCubic","easeInQuart","easeOutQuart","easeInOutQuart","easeInQuint","easeOutQuint","easeInOutQuint","prepareElements","JSONcontainer","elementType","redundant","used","cleanupElements","removeChild","getSVGElement","svgContainer","shift","document","createElementNS","appendChild","getDOMElement","DOMContainer","insertBefore","createElement","drawPoint","x","y","group","point","drawPoints","setAttributeNS","size","drawBar","width","height","rect","data","_options","_data","_fieldId","fieldId","_type","_subscribers","add","setOptions","prototype","queue","_queue","destroy","on","subscribers","subscribe","off","filter","unsubscribe","_trigger","params","senderId","concat","subscriber","addedIds","me","_addItem","columns","_getColumnNames","row","rows","getNumberOfRows","col","cols","getValue","update","updatedIds","updatedData","addOrUpdate","_updateItem","get","ids","firstType","returnType","allowedValues","itemId","_getItem","order","_sort","_filterFields","_appendRow","getIds","getDataSet","mappedItems","filteredItem","name","sort","av","bv","remove","removedId","removedIds","_remove","clear","maxField","itemField","minField","distinct","values","fieldType","count","exists","types","raw","converted","JSON","stringify","dataTable","getNumberOfColumns","getColumnId","getColumnLabel","addRow","setValue","_ids","_onEvent","apply","setData","viewOptions","getArguments","defaultFilter","dataSet","added","updated","removed","delay","Infinity","_timeout","_extended","_flushIfNeeded","flush","methods","original","method","args","fn","context","entry","clearTimeout","setTimeout","container","SyntaxError","containerElement","margin","defaultXCenter","defaultYCenter","xLabel","yLabel","zLabel","passValueFn","xValueLabel","yValueLabel","zValueLabel","filterLabel","legendLabel","STYLE","DOT","showPerspective","showGrid","keepAspectRatio","showShadow","showGrayBottom","showTooltip","verticalRatio","animationInterval","animationPreload","camera","eye","dataPoints","colX","colY","colZ","colValue","colFilter","xMin","xStep","xMax","yMin","yStep","yMax","zMin","zStep","zMax","valueMin","valueMax","xBarWidth","yBarWidth","colorAxis","colorGrid","colorDot","colorDotBorder","getMouseX","clientX","targetTouches","getMouseY","clientY","Emitter","_setScale","scale","z","xCenter","yCenter","zCenter","setArmLocation","_convert3Dto2D","point3d","translation","_convertPointToTranslation","_convertTranslationToScreen","ax","ay","az","cx","getCameraLocation","cy","cz","sinTx","sin","getCameraRotation","cosTx","cos","sinTy","cosTy","sinTz","cosTz","dx","dy","dz","bx","by","ex","ey","ez","getArmLength","xcenter","frame","canvas","clientWidth","ycenter","_setBackgroundColor","backgroundColor","fill","stroke","strokeWidth","borderColor","borderWidth","borderStyle","BAR","BARCOLOR","BARSIZE","DOTLINE","DOTCOLOR","DOTSIZE","GRID","LINE","SURFACE","_getStyleNumber","styleName","_determineColumnIndexes","counter","column","getDistinctValues","distinctValues","getColumnRange","minMax","_dataInitialize","rawData","_onChange","dataFilter","setOnLoadCallback","redraw","withBars","defaultXBarWidth","dataX","defaultYBarWidth","dataY","xRange","defaultXMin","defaultXMax","defaultXStep","yRange","defaultYMin","defaultYMax","defaultYStep","zRange","defaultZMin","defaultZMax","defaultZStep","valueRange","defaultValueMin","defaultValueMax","_getDataPoints","obj","sortNumber","dataMatrix","xIndex","yIndex","trans","screen","bottom","pointRight","pointTop","pointCross","hasChildNodes","firstChild","position","overflow","noCanvas","fontWeight","padding","innerHTML","onmousedown","_onMouseDown","ontouchstart","_onTouchStart","onmousewheel","_onWheel","ontooltip","_onTooltip","onkeydown","setSize","_resizeCanvas","clientHeight","animationStart","slider","play","animationStop","stop","_resizeCenter","charAt","parseFloat","setCameraPosition","pos","horizontal","vertical","setArmRotation","distance","setArmLength","getCameraPosition","getArmRotation","_readData","_redrawFilter","animationAutoStart","cameraPosition","styleNumber","tooltip","showAnimationControls","_redrawSlider","_redrawClear","_redrawAxis","_redrawDataGrid","_redrawDataLine","_redrawDataBar","_redrawDataDot","_redrawInfo","_redrawLegend","ctx","getContext","clearRect","widthMin","widthMax","dotSize","right","lineWidth","font","ymin","ymax","_hsv2rgb","strokeStyle","beginPath","moveTo","lineTo","strokeRect","fillStyle","closePath","gridLineLen","step","getCurrent","next","textAlign","textBaseline","fillText","label","visible","setValues","setPlayInterval","onchange","getIndex","selectValue","setOnChangeCallback","lineStyle","getLabel","getSelectedValue","from","to","prettyStep","text","xText","yText","zText","offset","xOffset","yOffset","xMin2d","xMax2d","gridLenX","gridLenY","textMargin","armAngle","H","S","V","R","G","B","C","Hi","X","abs","cross","topSideVisible","zAvg","transBottom","dist","sortDepth","aDiff","subtract","bDiff","crossproduct","crossProduct","radius","arc","PI","j","surface","corners","xWidth","yWidth","surfaces","center","avg","transCenter","diff","leftButtonDown","_onMouseUp","which","button","touchDown","startMouseX","startMouseY","startStart","startEnd","startArmRotation","cursor","onmousemove","_onMouseMove","onmouseup","diffX","diffY","horizontalNew","verticalNew","snapAngle","snapValue","round","parameters","emit","boundingRect","mouseX","mouseY","tooltipTimeout","_hideTooltip","dataPoint","_dataPointFromXY","_showTooltip","ontouchmove","_onTouchMove","ontouchend","_onTouchEnd","delta","wheelDelta","detail","oldLength","newLength","_insideTriangle","triangle","sign","as","bs","cs","distMax","closestDataPoint","closestDist","triangle1","triangle2","distX","distY","sqrt","content","line","dot","dom","borderRadius","boxShadow","borderLeft","contentWidth","offsetWidth","contentHeight","offsetHeight","lineHeight","dotWidth","dotHeight","armLocation","armRotation","armLength","cameraLocation","cameraRotation","calculateCameraOrientation","rot","graph","onLoadCallback","loadInBackground","isLoaded","getLoadedProgress","getColumn","getValues","dataView","progress","sub","sum","prev","bar","MozBorderRadius","slide","onclick","togglePlay","onChangeCallback","playTimeout","playInterval","playLoop","setIndex","playNext","interval","clearInterval","getPlayInterval","setPlayLoop","doLoop","onChange","indexToLeft","startClientX","startSlideX","leftToIndex","_start","_end","_step","precision","_current","setRange","setStep","calculatePrettyStep","log10","log","LN10","step1","pow","step2","step5","toPrecision","getStep","groups","forthArgument","defaultOptions","autoResize","orientation","maxHeight","minHeight","_create","body","domProps","emitter","bind","hiddenDates","snap","toScreen","_toScreen","toGlobalScreen","_toGlobalScreen","toTime","_toTime","toGlobalTime","_toGlobalTime","range","timeAxis","currentTime","customTime","itemSet","itemsData","groupsData","setGroups","setItems","Core","newDataSet","initialLoad","dataRange","_getDataRange","setWindow","animate","fit","setSelection","focus","getSelection","itemData","e","getItemRange","dataset","minItem","maxStartItem","maxEndItem","linegraph","getLegend","groupId","isGroupVisible","visibility","convertHiddenOptions","repeat","dateItem","updateHiddenDates","centerContainer","totalRange","pixelTime","startDate","endDate","_d","runUntil","clone","day","dayOfYear","year","dayOffset","date","month","console","removeDuplicates","startHidden","isHidden","endHidden","rangeStart","rangeEnd","hidden","startToFront","endToFront","_applyRange","safeDates","printDates","dates","stepOverHiddenDates","timeStep","previousTime","stepInHidden","currentValue","current","newValue","switchedYear","switchedMonth","switchedDay","time","conversion","getHiddenDurationBetween","correctTimeForHidden","hiddenDuration","totalDuration","partialDuration","accumulatedHiddenDuration","getAccumulatedHiddenDuration","newTime","getHiddenDurationBefore","timeOffset","requiredDuration","previousPoint","snapAwayFromHidden","direction","correctionEnabled","minimumStep","containerHeight","customRange","alignZeros","autoScale","stepIndex","marginStart","marginEnd","deadSpace","majorSteps","minorSteps","setMinimumStep","setFirst","safeSize","minimumStepValue","orderOfMagnitude","minorStepIdx","magnitudefactor","solutionFound","stepSize","niceStart","niceEnd","roundToMinor","marginRange","rounded","hasNext","previous","decimals","exp","cnt","isMajor","now","hours","minutes","seconds","milliseconds","deltaDifference","scaleOffset","moveable","zoomable","zoomMin","zoomMax","touch","animateTimer","_onDragStart","_onDrag","_onDragEnd","_onHold","_onMouseWheel","_onTouch","_onPinch","validateDirection","getPointer","byUser","_cancelAnimation","initStart","initEnd","initTime","anyChanged","dragging","done","changed","newStart","newEnd","getRange","totalHidden","previousDelta","allowDragging","deltaX","deltaY","diffRange","safeStart","safeEnd","pointer","pageX","pageY","pointerDate","_pointerToDate","zoom","centerDate","hiddenDurationBefore","hiddenDurationAfter","move","EPSILON","orderByStart","orderByEnd","aTime","bTime","force","iMax","axis","collidingItem","jj","collision","nostack","subgroups","newTop","subgroup","format","FORMAT","minorLabels","millisecond","second","minute","hour","weekday","majorLabels","setFormat","defaultFormat","first","setFullYear","getFullYear","setMonth","setDate","setHours","setMinutes","setSeconds","setMilliseconds","getMilliseconds","getSeconds","getMinutes","getHours","getDate","getMonth","setScale","newScale","newStep","setAutoScale","enable","stepYear","stepMonth","stepDay","stepHour","stepMinute","stepSecond","stepMillisecond","getLabelMinor","getLabelMajor","getClassName","even","today","isSame","currentWeek","currentMonth","currentYear","locale","lang","toLowerCase","_isResized","resized","_previousWidth","_previousHeight","showCurrentTime","locales","parent","backgroundVertical","title","toUpperCase","substring","currentTimeTimer","setCurrentTime","getCurrentTime","showCustomTime","eventParams","drag","setCustomTime","getCustomTime","stopPropagation","svg","linegraphOptions","showMinorLabels","showMajorLabels","icons","majorLinesOffset","minorLinesOffset","labelOffsetX","labelOffsetY","iconWidth","linegraphSVG","DOMelements","lines","labels","conversionFactor","minWidth","stepPixels","stepPixelsForced","zeroCrossing","lineOffset","master","svgElements","iconsRemoved","amountOfGroups","lineContainer","scrollTop","addGroup","graphOptions","updateGroup","removeGroup","hide","show","display","_redrawGroupIcons","iconHeight","iconOffset","drawIcon","_cleanupIcons","backgroundHorizontal","activeGroups","_calculateCharSize","minorLabelHeight","minorCharHeight","majorLabelHeight","majorCharHeight","minorLineWidth","minorLineHeight","majorLineWidth","majorLineHeight","_redrawLabels","_redrawTitle","amountOfSteps","stepDifference","zeroStepDifference","valueAtZero","marginStartPos","maxLabelSize","_redrawLabel","_redrawLine","titleWidth","titleCharHeight","convertValue","invertedValue","convertedValue","characterHeight","largestWidth","majorCharWidth","minorCharWidth","textMinor","createTextNode","measureCharMinor","textMajor","measureCharMajor","textTitle","measureCharTitle","titleCharWidth","groupsUsingDefaultStyles","usingDefaultStyle","zeroPosition","Line","Bar","Points","setZeroPosition","catmullRom","parametrization","alpha","SVGcontainer","path","fillPath","fillHeight","outline","shaded","barWidth","bar1Height","bar2Height","icon","yAxisOrientation","getYRange","groupData","draw","framework","subgroupIndex","subgroupOrderer","subgroupOrder","visibleItems","byStart","byEnd","checkRangedItems","inner","foreground","marker","Element","getLabelWidth","restack","_updateVisibleItems","markerHeight","lastMarkerHeight","dirty","displayed","_calculateHeight","offsetTop","offsetLeft","ii","repositionY","resetSubgroups","labelSet","setParent","orderSubgroups","_checkIfVisible","sortArray","sortField","removeFromDataSet","removeItem","startArray","endArray","oldVisibleItems","visibleItemsLookup","lowerBound","upperBound","_checkIfVisibleWithReference","initialPosByStart","_traceVisible","initialPosByEnd","repositionX","initialPos","breakCondition","isVisible","align","groupOrder","selectable","editable","updateTime","onAdd","onUpdate","onMove","onRemove","onMoving","itemOptions","itemListeners","_onAdd","_onUpdate","_onRemove","groupListeners","_onAddGroups","_onUpdateGroups","_onRemoveGroups","groupIds","selection","stackDirty","touchParams","UNGROUPED","BACKGROUND","box","_updateUngrouped","backgroundGroup","isFirst","_onSelectItem","_onMultiSelectItem","_onAddItem","addCallback","Function","markDirty","unselect","select","getVisibleItems","rawVisibleItems","_deselect","_orderGroups","visibleInterval","zoomed","lastVisibleInterval","lastWidth","firstGroup","_firstGroup","firstMargin","nonFirstMargin","groupMargin","groupResized","firstGroupIndex","firstGroupId","ungrouped","_getGroupId","getLabelSet","oldItemsData","getItems","_order","getGroups","_getType","_removeItem","groupOptions","oldGroupId","oldGroup","_constructByEndArray","itemFromTarget","dragLeftItem","dragRightItem","itemProps","selected","initialX","newProps","initial","groupFromTarget","_updateItemProps","_moveToGroup","changes","ctrlKey","srcEvent","shiftKey","oldSelection","newSelection","xAbs","newItem","_getItemRange","_item","itemSetFromTarget","side","iconSize","iconSpacing","textArea","scrollableHeight","drawLegendIcons","getComputedStyle","paddingTop","defaultGroup","sampling","graphHeight","barChart","handleOverlap","dataAxis","legend","abortedGraphUpdate","updateSVGheight","updateSVGheightOnResize","lastStart","COUNTER","BarGraphFunctions","yAxisLeft","yAxisRight","legendLeft","legendRight","_updateAllGroupData","_updateGroup","groupsContent","ungroupedCounter","forceGraphUpdate","_updateGraph","rangePerPixelInv","preprocessedGroupData","processedGroupData","groupRanges","changeCalled","minDate","maxDate","_getRelevantData","_applySampling","_convertXcoordinates","_getYRanges","_updateYAxis","MAX_CYCLES","_convertYcoordinates","dataContainer","guess","increment","amountOfPoints","xDistance","pointsPerPixel","ceil","sampledData","barCombinedDataLeft","barCombinedDataRight","getStackedBarYRange","minVal","maxVal","yAxisLeftUsed","yAxisRightUsed","minLeft","minRight","maxLeft","maxRight","ignore","_toggleAxisVisiblity","drawIcons","axisUsed","datapoints","xValue","yValue","extractedData","svgHeight","majorTexts","minorTexts","lineTop","parentChanged","foregroundNextSibling","nextSibling","backgroundNextSibling","_repaintLabels","timeLabelsize","cur","prevLine","xPrev","xFirstMajorLabel","_repaintMinorText","_repaintMajorText","_repaintMajorLine","_repaintMinorLine","leftTime","leftText","widthText","arr","pop","childNodes","nodeValue","_repaintDeleteButton","anchor","deleteButton","_updateContents","template","_updateTitle","removeAttribute","_updateDataAttributes","dataAttributes","attributes","setAttribute","_updateStyle","emptyContent","baseClassName","onTop","itemSubgroup","itemSetHeight","marginLeft","maxWidth","_repaintDragLeft","_repaintDragRight","contentLeft","parentWidth","boxWidth","dragLeft","dragRight","_determineBrowserMethod","_initializeMixinLoaders","renderRefreshRate","renderTimestep","renderTime","physicsTime","runDoubleSpeed","physicsDiscreteStepsize","initializing","triggerFunctions","edit","editEdge","connect","del","nodes","mass","radiusMin","radiusMax","shape","image","fontColor","fontSize","fontFace","fontFill","fontStrokeWidth","fontStrokeColor","level","borderWidthSelected","edges","widthSelectionMultiplier","hoverWidth","labelAlignment","arrowScaleFactor","dash","gap","altLength","inheritColor","configurePhysics","physics","barnesHut","thetaInverted","gravitationalConstant","centralGravity","springLength","springConstant","damping","repulsion","nodeDistance","hierarchicalRepulsion","clustering","initialMaxNodes","clusterThreshold","reduceToNodes","chainThreshold","clusterEdgeThreshold","sectorThreshold","screenSizeThreshold","fontSizeMultiplier","maxFontSize","forceAmplification","distanceAmplification","edgeGrowth","nodeScaling","maxNodeSizeIncrements","activeAreaBoxSize","clusterLevelDifference","navigation","keyboard","speed","dataManipulation","initiallyVisible","hierarchicalLayout","levelSeparation","nodeSpacing","layout","freezeForStabilization","smoothCurves","dynamic","roundness","maxVelocity","minVelocity","stabilize","stabilizationIterations","zoomExtentOnStabilize","dragNetwork","dragNodes","hideEdgesOnDrag","hideNodesOnDrag","constants","pixelRatio","hoverObj","controlNodesActive","navigationHammers","existing","_new","animationSpeed","animationEasingFunction","easingTime","sourceScale","targetScale","sourceTranslation","targetTranslation","lockedOnNodeId","lockedOnNodeOffset","touchTime","images","setOnloadCallback","_redraw","xIncrement","yIncrement","zoomIncrement","_loadPhysicsSystem","_loadSectorSystem","_loadClusterSystem","_loadSelectionSystem","_loadHierarchySystem","_setTranslation","freezeSimulation","cachedFunctions","startedStabilization","stabilized","draggingNodes","calculationNodes","calculationNodeIndices","nodeIndices","canvasTopLeft","canvasBottomRight","pointerPosition","areaCenter","previousScale","nodesData","edgesData","nodesListeners","_addNodes","_updateNodes","_removeNodes","edgesListeners","_addEdges","_updateEdges","_removeEdges","moving","timer","_setupHierarchicalLayout","zoomExtent","startWithClustering","keycharm","hammerUtil","MixinLoader","Activator","browserType","requiresTimeout","_getScriptPath","scripts","getElementsByTagName","src","_getRange","node","minY","maxY","minX","maxX","nodeId","boundingBox","_findCenter","animationOptions","initialZoom","disableStart","zoomLevel","numberOfNodes","factor","yDistance","xZoomLevel","yZoomLevel","animation","_updateNodeIndexList","_clearNodeIndexList","idx","_createManipulatorBar","dotData","DOTToGraph","gephi","gephiData","parseGephi","_setNodes","_setEdges","_putDataInSector","_resetLevels","_stabilize","onEdit","onEditEdge","onConnect","onDelete","editMode","newColorObj","groupname","clickToUse","activator","_createKeyBinds","_loadNavigationControls","_loadManipulationSystem","_configureSmoothCurves","devicePixelRatio","webkitBackingStorePixelRatio","mozBackingStorePixelRatio","msBackingStorePixelRatio","oBackingStorePixelRatio","backingStorePixelRatio","setTransform","pinch","set","_onTap","_onDoubleTap","onTouch","_onMouseMoveTitle","reset","isActive","_moveUp","_yStopMoving","_moveDown","_moveLeft","_xStopMoving","_moveRight","_zoomIn","_stopZoom","_zoomOut","_deleteSelected","_cleanupPhysicsConfiguration","_recursiveDOMDelete","DOMobject","_getPointer","pinched","_getScale","_handleTouch","_handleDragStart","_getNodeAt","_getTranslation","isSelected","_selectObject","nodeIds","objectId","selectionObj","xFixed","yFixed","_handleOnDrag","releaseNode","_XconvertDOMtoCanvas","_XconvertCanvasToDOM","_YconvertDOMtoCanvas","_YconvertCanvasToDOM","_handleDragEnd","_handleTap","_handleDoubleTap","_handleOnHold","_onRelease","_handleOnRelease","_zoom","scaleOld","preScaleDragPointer","DOMtoCanvas","scaleFrac","tx","ty","updateClustersDefault","postScaleDragPointer","canvasToDOM","popupObj","_checkHidePopup","checkShow","_checkShowPopup","popupTimer","edgeId","_getEdgeAt","_hoverObject","_blurObject","lastPopupNode","nodeUnderCursor","overlappingNodes","isOverlappingWith","getTitle","overlappingEdges","edge","connected","popup","setPosition","setText","emitEvent","oldWidth","oldHeight","oldNodesData","_updateSelection","angle","_updateCalculationNodes","_reconnectEdges","_updateValueRange","updateLabels","changedData","setProperties","properties","oldEdgesData","oldEdge","disconnect","showInternalIds","_createBezierNodes","via","sectors","dynamicEdges","setValueRange","w","save","translate","_doInAllSectors","restore","offsetX","offsetY","_drawNodes","alwaysShow","setScaleAndPos","inArea","sMax","_drawEdges","_drawControlNodes","_freezeDefinedNodes","_physicsTick","_restoreFrozenNodes","fixedData","_isMoving","vmin","isMoving","_discreteStepNodes","nodesPresent","discreteStepLimited","discreteStep","vminCorrected","_revertPhysicsState","revertPosition","_revertPhysicsTick","_doInAllActiveSectors","_doInSupportSector","mainMovingStatus","supportMovingStatus","mainMoving","_animationStep","_handleNavigation","startTime","renderStartTime","requestAnimationFrame","mozRequestAnimationFrame","webkitRequestAnimationFrame","msRequestAnimationFrame","iterations","toggleFreeze","parentEdgeId","internalMultiplier","positionBezierNode","mixin","storePosition","storePositions","dataArray","allowedToMoveX","allowedToMoveY","getPositions","focusOnNode","nodePosition","lockedOnNode","easingFunction","animateView","locked","_transitionRedraw","viewCenter","distanceFromCenter","_classicRedraw","_lockedRedraw","active","getScale","getCenterCoordinates","getBoundingBox","networkConstants","fromId","toId","widthSelected","labelDimensions","yLine","dirtyLabel","fromBackup","toBackup","originalFromId","originalToId","widthFixed","lengthFixed","controlNodesEnabled","controlNodes","positions","connectedNode","_drawLine","_drawArrow","_drawArrowCenter","_drawDashLine","attachEdge","detachEdge","xFrom","yFrom","xTo","yTo","xObj","yObj","_getDistanceToEdge","_getColor","colorObj","_getLineWidth","_line","midpointX","midpointY","_pointOnLine","_label","resize","_circle","_pointOnCircle","networkScaleInv","_getViaCoordinates","xVia","yVia","quadraticCurveTo","lineCount","measureText","_rotateForLabelAlignment","_drawLabelRect","_drawLabelText","angleInDegrees","atan2","rotate","lineMargin","fillRect","lineJoin","strokeText","setLineDash","pattern","lineDashOffset","lineCap","dashedLine","percentage","arrow","_pointOnBezier","_findBorderPosition","distanceToBorder","distanceToNodes","difference","threshold","arrowPos","guidePos","edgeSegmentLength","toBorderDist","toBorderPoint","x1","y1","x2","y2","x3","y3","lastX","lastY","minDistance","_getDistanceToLine","px","py","something","u","nodeIdFrom","nodeIdTo","getControlNodeFromPosition","getControlNodeToPosition","_enableControlNodes","_disableControlNodes","_getSelectedControlNode","fromDistance","toDistance","_restoreControlNodes","controlnodeFromPos","fromBorderDist","fromBorderPoint","controlnodeToPos","defaultIndex","DEFAULT","imageBroken","load","url","brokenUrl","img","Image","onload","onerror","error","imagelist","grouplist","reroutedEdges","fontDrawThreshold","horizontalAlignLeft","verticalAlignTop","baseRadiusValue","radiusFixed","preassignedLevel","hierarchyEnumerated","fx","fy","vx","vy","previousState","resetCluster","dynamicEdgesLength","clusterSession","clusterSizeWidthFactor","clusterSizeHeightFactor","clusterSizeRadiusFactor","growthIndicator","networkScale","formationScale","clusterSize","containedNodes","containedEdges","clusterSessions","originalLabel","triggerFunction","groupObj","imageObj","brokenImage","_drawDatabase","_resizeDatabase","_drawBox","_resizeBox","_drawCircle","_resizeCircle","_drawEllipse","_resizeEllipse","_drawImage","_resizeImage","_drawCircularImage","_resizeCircularImage","_drawText","_resizeText","_drawDot","_resizeShape","_drawSquare","_drawTriangle","_drawTriangleDown","_drawStar","_reset","clearSizeCache","_setForce","_addForce","storeState","isFixed","velocity","getDistance","_drawImageAtPosition","globalAlpha","drawImage","_drawImageLabel","getTextSize","_swapToImageResizeWhenImageLoaded","diameter","centerX","centerY","_drawRawCircle","circle","clip","textSize","clusterLineWidth","selectionLineWidth","roundRect","database","defaultSize","ellipse","_drawShape","radiusMultiplier","baseline","labelUnderNode","inView","clearVelocity","updateVelocity","massBeforeClustering","energyBefore","styleAttr","fontFamily","WebkitBorderRadius","whiteSpace","parseDOT","parseGraph","nextPreview","isAlphaNumeric","regexAlphaNumeric","merge","o","addNode","graphs","attr","addEdge","createEdge","getToken","tokenType","TOKENTYPE","NULL","token","isComment","DELIMITER","c2","DELIMITERS","IDENTIFIER","newSyntaxError","UNKNOWN","chop","strict","parseStatements","parseStatement","subgraph","parseSubgraph","parseEdge","parseAttributeStatement","parseNodeStatement","subgraphs","parseAttributeList","message","maxLength","forEach2","array1","array2","elem1","elem2","graphData","dotNode","graphNode","convertEdge","dotEdge","graphEdge","subEdge","{","}","[","]",";","=",",","->","--","gephiJSON","allowedToMove","gEdges","gNodes","gEdge","source","gNode","propagating","onMouseWheel","leftContainer","rightContainer","shadowTop","shadowBottom","shadowTopLeft","shadowBottomLeft","shadowTopRight","shadowBottomRight","_redrawTimer","touchAction","listeners","events","scrollTopMin","redrawCount","_initAutoResize","component","_stopAutoResize","what","getWindow","borderRootHeight","borderRootWidth","autoHeight","centerWidth","_updateScrollTop","visibilityTop","visibilityBottom","MAX_REDRAWS","repaint","_startAutoResize","_onResize","lastHeight","watchTimer","setInterval","_setScrollTop","_getScrollTop","inputHandler","onRelease","isFinal","offTouch","offRelease","custom","back","editNode","addDescription","edgeDescription","editEdgeDescription","createEdgeError","deleteClusterError","CanvasRenderingContext2D","square","s2","ir","triangleDown","star","n","r2d","kappa","ox","oy","xe","ye","xm","ym","bezierCurveTo","wEllipse","hEllipse","ymb","yeb","xt","yt","xi","yi","xl","yl","xr","yr","dashArray","dashLength","dashCount","slope","distRemaining","dashIndex","_catmullRom","_linear","dFill","_catmullRomUniform","p0","p1","p2","p3","bp1","bp2","normalization","d1","d2","d3","A","N","M","d3powA","d2powA","d3pow2A","d2pow2A","d1pow2A","d1powA","Bargraph","barCombinedData","coreDistance","drawData","combinedData","intersections","barPoints","_getDataIntersections","heightOffset","_getSafeDrawData","nextKey","amount","resolved","prevKey","accumulated","groupLabel","_getStackedBarYRange","xpos","PhysicsMixin","ClusterMixin","SectorsMixin","SelectionMixin","ManipulationMixin","NavigationMixin","HierarchicalLayoutMixin","_loadMixin","sourceVariable","mixinFunction","_clearMixin","_loadSelectedForceSolver","_loadPhysicsConfiguration","hubThreshold","activeSector","drawingNode","blockConnectingEdgeSelection","forceAppendSelection","manipulationDiv","editModeDiv","closeDiv","_cleanNavigation","_loadNavigationElements","overlay","prevent_default","_onTapOverlay","windowHammer","_hasParent","deactivate","escListener","activate","unbind","_callbacks","once","self","removeListener","removeAllListeners","callbacks","cb","hasListeners","__WEBPACK_AMD_DEFINE_RESULT__","global","dfl","hasOwnProp","defaultParsingFlags","empty","unusedTokens","unusedInput","charsLeftOver","nullInput","invalidMonth","invalidFormat","userInvalidated","iso","printMsg","msg","suppressDeprecationWarnings","warn","deprecate","firstTime","deprecateSimple","deprecations","padToken","func","leftZeroFill","ordinalizeToken","period","localeData","ordinal","monthDiff","anchor2","adjust","wholeMonthDiff","meridiemFixWrap","meridiem","isPm","meridiemHour","isPM","Locale","Moment","config","skipOverflow","checkOverflow","copyConfig","updateInProgress","updateOffset","Duration","normalizedInput","normalizeObjectUnits","years","quarters","quarter","months","weeks","week","days","_milliseconds","_days","_months","_locale","_bubble","val","_isAMomentObject","_i","_f","_l","_strict","_tzm","_isUTC","_offset","_pf","momentProperties","absRound","number","targetLength","forceSign","output","positiveMomentsDifference","base","res","isAfter","momentsDifference","makeAs","isBefore","createAdder","dur","tmp","addOrSubtractDurationFromMoment","mom","isAdding","setTime","rawSetter","rawGetter","rawMonthSetter","input","compareArrays","dontConvert","lengthDiff","diffs","toInt","normalizeUnits","units","lowered","unitAliases","camelFunctions","inputObject","normalizedProp","makeList","setter","getter","results","utc","argumentForCoercion","coercedNumber","isFinite","daysInMonth","UTC","getUTCDate","weeksInYear","dow","doy","weekOfYear","daysInYear","isLeapYear","_a","MONTH","DATE","YEAR","HOUR","MINUTE","SECOND","MILLISECOND","_overflowDayOfYear","isValid","_isValid","getTime","bigHour","normalizeLocale","chooseLocale","names","loadLocale","oldLocale","hasModule","code","model","local","removeFormattingTokens","makeFormatFunction","formattingTokens","formatTokenFunctions","formatMoment","expandFormat","formatFunctions","invalidDate","replaceLongDateFormatTokens","longDateFormat","localFormattingTokens","lastIndex","getParseRegexForToken","parseTokenOneDigit","parseTokenThreeDigits","parseTokenFourDigits","parseTokenOneToFourDigits","parseTokenSignedNumber","parseTokenSixDigits","parseTokenOneToSixDigits","parseTokenTwoDigits","parseTokenOneToThreeDigits","parseTokenWord","_meridiemParse","parseTokenOffsetMs","parseTokenTimestampMs","parseTokenTimezone","parseTokenT","parseTokenDigits","parseTokenOneOrTwoDigits","_ordinalParse","_ordinalParseLenient","RegExp","regexpEscape","unescapeFormat","utcOffsetFromString","string","possibleTzMatches","tzChunk","parseTimezoneChunker","addTimeToArrayFromToken","datePartArray","monthsParse","_dayOfYear","parseTwoDigitYear","_meridiem","_useUTC","weekdaysParse","_w","invalidWeekday","dayOfYearFromWeekInfo","weekYear","temp","GG","W","E","_week","gg","dayOfYearFromWeeks","dateFromConfig","currentDate","yearToUse","currentDateArray","makeUTCDate","getUTCMonth","_nextDay","makeDate","setUTCMinutes","getUTCMinutes","dateFromObject","getUTCFullYear","makeDateFromStringAndFormat","ISO_8601","parseISO","parsedInput","tokens","skipped","stringLength","totalParsedInputLength","matched","p4","makeDateFromStringAndArray","tempConfig","bestMoment","scoreToBeat","currentScore","NaN","score","l","isoRegex","isoDates","isoTimes","makeDateFromString","createFromInputFallback","makeDateFromInput","aspNetJsonRegex","ms","setUTCFullYear","parseWeekday","substituteTimeAgo","withoutSuffix","isFuture","relativeTime","posNegDuration","relativeTimeThresholds","firstDayOfWeek","firstDayOfWeekOfYear","adjustedMoment","daysToDayOfWeek","daysToAdd","getUTCDay","makeMoment","invalid","preparse","pickBy","moments","dayOfMonth","unit","makeAccessor","keepTime","daysToYears","yearsToDays","makeDurationGetter","makeGlobal","shouldDeprecate","ender","oldGlobalMoment","globalScope","VERSION","aspNetTimeSpanJsonRegex","isoDurationRegex","isoFormat","unitMillisecondFactors","Milliseconds","Seconds","Minutes","Hours","Days","Months","Years","D","Q","DDD","dayofyear","isoweekday","isoweek","weekyear","isoweekyear","ordinalizeTokens","paddedTokens","MMM","monthsShort","MMMM","dd","weekdaysMin","ddd","weekdaysShort","dddd","weekdays","isoWeek","YY","YYYY","YYYYY","YYYYYY","gggg","ggggg","isoWeekYear","GGGG","GGGGG","isoWeekday","SS","SSS","SSSS","Z","utcOffset","ZZ","zoneAbbr","zz","zoneName","unix","lists","DDDD","_monthsShort","monthName","regex","_monthsParse","_longMonthsParse","_shortMonthsParse","_weekdays","_weekdaysShort","_weekdaysMin","weekdayName","_weekdaysParse","_longDateFormat","LTS","LT","L","LL","LLL","LLLL","isLower","_calendar","sameDay","nextDay","nextWeek","lastDay","lastWeek","sameElse","calendar","_relativeTime","future","past","mm","hh","MM","yy","pastFuture","_ordinal","postformat","firstDayOfYear","_invalidDate","ret","parseIso","diffRes","isDuration","inp","version","relativeTimeThreshold","limit","defineLocale","_abbr","abbr","langData","flags","parseZone","isDSTShifted","parsingFlags","invalidAt","keepLocalTime","_dateUtcOffset","inputString","asFloat","that","zoneDiff","humanize","fromNow","sod","startOf","isDST","getDay","endOf","inputMs","isBetween","zone","localAdjust","_changeInProgress","isLocal","isUtcOffset","isUtc","hasAlignedHourOffset","isoWeeksInYear","weekInfo","newLocaleData","getTimezoneOffset","isoWeeks","toJSON","isUTC","withSuffix","toIsoString","asSeconds","asMilliseconds","asMinutes","asHours","asDays","asWeeks","asMonths","asYears","ordinalParse","require","noGlobal","__WEBPACK_AMD_DEFINE_FACTORY__","__WEBPACK_AMD_DEFINE_ARRAY__","_exportFunctions","_bound","keydown","keyup","_keys","fromCharCode","down","handleEvent","up","keyCode","bound","bindAll","getKey","newBindings","_firstTarget","propagatedHandler","_handled","stopped","firstTarget","_handlers","Manager","PropagatingHammer","_on","_off","_destroy","handler","exportName","setTimeoutContext","timeout","bindFn","invokeArrayArg","arg","each","iterator","dest","inherit","child","childP","baseP","_super","boolOrFn","TYPE_FUNCTION","ifUndefined","val1","val2","addEventListeners","splitStr","removeEventListeners","hasParent","inStr","str","find","inArray","findByKey","uniqueArray","prefixed","property","prefix","camelProp","VENDOR_PREFIXES","uniqueId","_uniqueId","getWindowForElement","doc","ownerDocument","defaultView","parentWindow","Input","manager","inputTarget","domHandler","ev","init","createInputInstance","Type","inputClass","SUPPORT_POINTER_EVENTS","PointerEventInput","SUPPORT_ONLY_TOUCH","TouchInput","SUPPORT_TOUCH","TouchMouseInput","MouseInput","eventType","pointersLen","pointers","changedPointersLen","changedPointers","INPUT_START","INPUT_END","INPUT_CANCEL","session","computeInputData","recognize","prevInput","pointersLength","firstInput","simpleCloneInputData","firstMultiple","offsetCenter","getCenter","timeStamp","deltaTime","getAngle","computeDeltaXY","offsetDirection","getDirection","rotation","getRotation","computeIntervalInputData","offsetDelta","prevDelta","velocityX","velocityY","last","lastInterval","COMPUTE_INTERVAL","getVelocity","DIRECTION_NONE","DIRECTION_LEFT","DIRECTION_RIGHT","DIRECTION_UP","DIRECTION_DOWN","PROPS_XY","PROPS_CLIENT_XY","evEl","MOUSE_ELEMENT_EVENTS","evWin","MOUSE_WINDOW_EVENTS","allow","pressed","POINTER_ELEMENT_EVENTS","POINTER_WINDOW_EVENTS","store","pointerEvents","SingleTouchInput","evTarget","SINGLE_TOUCH_TARGET_EVENTS","SINGLE_TOUCH_WINDOW_EVENTS","started","normalizeSingleTouches","all","touches","changedTouches","TOUCH_TARGET_EVENTS","targetIds","getTouches","allTouches","INPUT_MOVE","identifier","changedTargetTouches","mouse","TouchAction","cleanTouchActions","actions","TOUCH_ACTION_NONE","hasPanX","TOUCH_ACTION_PAN_X","hasPanY","TOUCH_ACTION_PAN_Y","TOUCH_ACTION_MANIPULATION","TOUCH_ACTION_AUTO","Recognizer","defaults","state","STATE_POSSIBLE","simultaneous","requireFail","stateStr","STATE_CANCELLED","STATE_ENDED","STATE_CHANGED","STATE_BEGAN","directionStr","getRecognizerByNameIfManager","otherRecognizer","recognizer","AttrRecognizer","PanRecognizer","pX","pY","PinchRecognizer","PressRecognizer","_timer","_input","RotateRecognizer","SwipeRecognizer","TapRecognizer","pTime","pCenter","recognizers","preset","handlers","toggleCssProps","recognizeWith","requireFailure","cssProps","triggerDomEvent","gestureEvent","createEvent","initEvent","gesture","dispatchEvent","TEST_ELEMENT","MOBILE_REGEX","INPUT_TYPE_TOUCH","INPUT_TYPE_PEN","INPUT_TYPE_MOUSE","INPUT_TYPE_KINECT","DIRECTION_HORIZONTAL","DIRECTION_VERTICAL","DIRECTION_ALL","MOUSE_INPUT_MAP","mousedown","mousemove","mouseup","pointerType","POINTER_INPUT_MAP","pointerdown","pointermove","pointerup","pointercancel","pointerout","IE10_POINTER_TYPE_ENUM",2,3,4,5,"MSPointerEvent","removePointer","eventTypeNormalized","isTouch","storeIndex","pointerId","SINGLE_TOUCH_INPUT_MAP","touchstart","touchmove","touchend","touchcancel","TOUCH_INPUT_MAP","inputEvent","inputData","isMouse","PREFIXED_TOUCH_ACTION","NATIVE_TOUCH_ACTION","TOUCH_ACTION_COMPUTE","compute","getTouchAction","preventDefaults","prevented","hasNone","preventSrc","STATE_RECOGNIZED","STATE_FAILED","dropRecognizeWith","dropRequireFailure","hasRequireFailures","canRecognizeWith","withState","tryEmit","canEmit","inputDataClone","process","attrTest","optionPointers","isRecognized","directionTest","hasMoved","inOut","validPointers","validMovement","validTime","taps","posThreshold","validTouchTime","failTimeout","validInterval","validMultiTap","tapCount","domEvents","userSelect","touchSelect","touchCallout","contentZooming","userDrag","tapHighlightColor","STOP","FORCED_STOP","curRecognizer","Tap","Pan","Swipe","Pinch","Rotate","Press","clusterToFit","maxNumberOfNodes","reposition","maxLevels","forceAggregateHubs","normalizeClusterLevels","increaseClusterLevel","repositionNodes","openCluster","isMovingBeforeClustering","_nodeInActiveArea","_sector","_addSector","decreaseClusterLevel","_expandClusterNode","_updateDynamicEdges","updateClusters","zoomDirection","recursive","doNotStart","amountOfNodes","_collapseSector","_formClusters","_openClusters","_openClustersBySize","_aggregateHubs","handleChains","chainPercentage","_getChainFraction","_reduceAmountOfChains","_getHubSize","_formClustersByHub","openAll","containedNodeId","childNode","_expelChildFromParent","_unselectAll","_releaseContainedEdges","_connectEdgeBackToChild","_validateEdges","othersPresent","childNodeId","_repositionBezierNodes","_formClustersByZoom","_forceClustersByZoom","minLength","_addToCluster","_clusterToSmallestNeighbour","smallestNeighbour","smallestNeighbourNode","neighbour","onlyEqual","_formClusterFromHub","hubNode","absorptionSizeOffset","allowCluster","edgesIdarray","amountOfInitialEdges","_addToContainedEdges","_connectEdgeToCluster","_containCircularEdgesFromNode","massBefore","correction","edgeToId","edgeFromId","k","_addToReroutedEdges","maxLevel","minLevel","clusterLevel","targetLevel","average","averageSquared","hubCounter","largestHub","variance","standardDeviation","fraction","reduceAmount","chains","total","_switchToSector","sectorId","sectorType","_switchToActiveSector","_switchToFrozenSector","_switchToSupportSector","_loadLatestSector","_previousSector","_setActiveSector","newId","_forgetLastSector","_createNewSector","_deleteActiveSector","_deleteFrozenSector","_freezeSector","_activateSector","_mergeThisWithFrozen","_collapseThisToSingleCluster","sector","unqiueIdentifier","previousSector","runFunction","argument","returnValues","_doInAllFrozenSectors","_drawSectorNodes","_drawAllSectorNodes","_getNodesOverlappingWith","_getAllNodesOverlappingWith","_pointerToPositionObject","positionObject","_getEdgesOverlappingWith","_getAllEdgesOverlappingWith","_addToSelection","_addToHover","_removeFromSelection","doNotTrigger","_unselectClusters","_getSelectedNodeCount","_getSelectedNode","_getSelectedEdge","_getSelectedEdgeCount","_getSelectedObjectCount","_selectionIsEmpty","_clusterInSelection","_selectConnectedEdges","_hoverConnectedEdges","_unselectConnectedEdges","append","highlightEdges","overrideSelectable","DOM","_manipulationReleaseOverload","_navigationReleaseOverload","getSelectedNodes","edgeIds","getSelectedEdges","idArray","selectNodes","RangeError","selectEdges","_clearManipulatorBar","manipulationDOM","_restoreOverloadedFunctions","functionName","_toggleEditMode","toolbar","boundFunction","edgeBeingEdited","selectedControlNode","_createAddNodeToolbar","_createAddEdgeToolbar","_editNode","_createEditEdgeToolbar","_addNode","_handleConnect","_finishConnect","_selectControlNode","_controlNodeDrag","_releaseControlNode","newNode","_editEdge","alert","supportNodes","targetNode","connectionEdge","connectFromId","_createEdge","defaultData","finalizedData","sourceNodeId","targetNodeId","selectedNodes","selectedEdges","navigationDivs","navigationDivActions","_stopMovement","_zoomExtent","hubsize","definedLevel","undefinedLevel","_changeConstants","_determineLevels","_determineLevelsDirected","distribution","_getDistribution","_placeNodesByHierarchy","minPos","_placeBranchNodes","maxCount","_setLevel","firstNode","_setLevelDirected","parentId","parentLevel","nodeMoved","_restoreNodes","graphToggleSmoothCurves","graph_toggleSmooth","getElementById","graphRepositionNodes","showValueOfRange","graphGenerateOptions","optionsSpecific","radioButton1","radioButton2","checked","backupConstants","optionsDiv","switchConfigurations","radioButton","querySelector","tableId","table","constantsVariableName","valueId","rangeValue","_overWriteGraphConstants","RepulsionMixin","HierarchialRepulsionMixin","BarnesHutMixin","_toggleBarnesHut","barnesHutTree","_initializeForceCalculation","_calculateForces","_calculateGravitationalForces","_calculateNodeForces","_calculateSpringForcesWithSupport","_calculateHierarchicalSpringForces","_calculateSpringForces","supportNodeId","gravity","gravityForce","edgeLength","springForce","combinedClusterSize","node1","node2","node3","_calculateSpringForce","physicsConfiguration","hierarchicalLayoutDirections","parentElement","rangeElement","radioButton3","graph_repositionNodes","graph_generateOptions","dynamicSmoothCurves","nameArray","webpackContext","req","resolve","repulsingForce","a_base","minimumDistance","steepness","springFx","springFy","totalFx","totalFy","correctionFx","correctionFy","nodeCount","_formBarnesHutTree","_getForceContribution","children","NW","NE","SW","SE","parentBranch","childrenCount","centerOfMass","calcSize","MAX_VALUE","sizeDiff","minimumTreeSize","rootSize","halfRootSize","_splitBranch","_placeInTree","_updateBranchMass","totalMass","totalMassInv","biggestSize","skipMassUpdate","_placeInRegion","region","containedNode","_insertRegion","childSize","_drawTree","_drawBranch","branch","webpackPolyfill","paths","__webpack_amd_options__"],"mappings":";;;;;;;;;;;;;;;;;;;;;;;;AAyBA,cAEA,SAA2CA,EAAMC,GAC1B,gBAAZC,UAA0C,gBAAXC,QACxCA,OAAOD,QAAUD,IACQ,kBAAXG,SAAyBA,OAAOC,IAC9CD,OAAOH,GACmB,gBAAZC,SACdA,QAAa,IAAID,IAEjBD,EAAU,IAAIC,KACbK,KAAM,WACT,MAAgB,UAAUC,GAKhB,QAASC,GAAoBC,GAG5B,GAAGC,EAAiBD,GACnB,MAAOC,GAAiBD,GAAUP,OAGnC,IAAIC,GAASO,EAAiBD,IAC7BP,WACAS,GAAIF,EACJG,QAAQ,EAUT,OANAL,GAAQE,GAAUI,KAAKV,EAAOD,QAASC,EAAQA,EAAOD,QAASM,GAG/DL,EAAOS,QAAS,EAGTT,EAAOD,QAvBf,GAAIQ,KAqCJ,OATAF,GAAoBM,EAAIP,EAGxBC,EAAoBO,EAAIL,EAGxBF,EAAoBQ,EAAI,GAGjBR,EAAoB,KAK/B,SAASL,EAAQD,EAASM,GAG9BN,EAAQe,KAAOT,EAAoB,GACnCN,EAAQgB,QAAUV,EAAoB,GAGtCN,EAAQiB,QAAUX,EAAoB,GACtCN,EAAQkB,SAAWZ,EAAoB,GACvCN,EAAQmB,MAAQb,EAAoB,GAGpCN,EAAQoB,QAAUd,EAAoB,GACtCN,EAAQqB,SACNC,OAAQhB,EAAoB,GAC5BiB,OAAQjB,EAAoB,GAC5BkB,QAASlB,EAAoB,GAC7BmB,QAASnB,EAAoB,IAC7BoB,OAAQpB,EAAoB,IAC5BqB,WAAYrB,EAAoB,KAIlCN,EAAQ4B,SAAWtB,EAAoB,IACvCN,EAAQ6B,QAAUvB,EAAoB,IACtCN,EAAQ8B,UACNC,SAAUzB,EAAoB,IAC9B0B,SAAU1B,EAAoB,IAC9B2B,MAAO3B,EAAoB,IAC3B4B,MAAO5B,EAAoB,IAC3B6B,SAAU7B,EAAoB,IAE9B8B,YACEC,OACEC,KAAMhC,EAAoB,IAC1BiC,eAAgBjC,EAAoB,IACpCkC,QAASlC,EAAoB,IAC7BmC,UAAWnC,EAAoB,IAC/BoC,UAAWpC,EAAoB,KAGjCqC,UAAWrC,EAAoB,IAC/BsC,YAAatC,EAAoB,IACjCuC,WAAYvC,EAAoB,IAChCwC,SAAUxC,EAAoB,IAC9ByC,WAAYzC,EAAoB,IAChC0C,MAAO1C,EAAoB,IAC3B2C,gBAAiB3C,EAAoB,IACrC4C,QAAS5C,EAAoB,IAC7B6C,OAAQ7C,EAAoB,IAC5B8C,UAAW9C,EAAoB,IAC/B+C,SAAU/C,EAAoB,MAKlCN,EAAQsD,QAAUhD,EAAoB,IACtCN,EAAQuD,SACNC,KAAMlD,EAAoB,IAC1BmD,OAAQnD,EAAoB,IAC5BoD,OAAQpD,EAAoB,IAC5BqD,KAAMrD,EAAoB,IAC1BsD,MAAOtD,EAAoB,IAC3BuD,UAAWvD,EAAoB,IAC/BwD,YAAaxD,EAAoB,KAInCN,EAAQ+D,MAAQ,WACd,KAAM,IAAIC,OAAM,+EAIlBhE,EAAQiE,OAAS3D,EAAoB,IACrCN,EAAQkE,OAAS5D,EAAoB,IACrCN,EAAQmE,OAAS7D,EAAoB,KAKjC,SAASL,EAAQD,EAASM,GAM9B,GAAI2D,GAAS3D,EAAoB,GAOjCN,GAAQoE,SAAW,SAASC,GAC1B,MAAQA,aAAkBC,SAA2B,gBAAVD,IAQ7CrE,EAAQuE,SAAW,SAASF,GAC1B,MAAQA,aAAkBG,SAA2B,gBAAVH,IAQ7CrE,EAAQyE,OAAS,SAASJ,GACxB,GAAIA,YAAkBK,MACpB,OAAO,CAEJ,IAAI1E,EAAQuE,SAASF,GAAS,CAEjC,GAAIM,GAAQC,EAAaC,KAAKR,EAC9B,IAAIM,EACF,OAAO,CAEJ,KAAKG,MAAMJ,KAAKK,MAAMV,IACzB,OAAO,EAIX,OAAO,GAQTrE,EAAQgF,YAAc,SAASX,GAC7B,MAA4B,mBAAb,SACVY,OAAoB,eACpBA,OAAOC,cAAuB,WAC9Bb,YAAkBY,QAAOC,cAAcC,WAQ9CnF,EAAQoF,WAAa,WACnB,GAAIC,GAAK,WACP,MAAOC,MAAKC,MACQ,MAAhBD,KAAKE,UACPC,SAAS,IAGb,OACIJ,KAAOA,IAAO,IACVA,IAAO,IACPA,IAAO,IACPA,IAAO,IACPA,IAAOA,IAAOA,KAWxBrF,EAAQ0F,OAAS,SAAUC,GACzB,IAAK,GAAIC,GAAI,EAAGC,EAAMC,UAAUC,OAAYF,EAAJD,EAASA,IAAK,CACpD,GAAII,GAAQF,UAAUF,EACtB,KAAK,GAAIK,KAAQD,GACXA,EAAME,eAAeD,KACvBN,EAAEM,GAAQD,EAAMC,IAKtB,MAAON,IAWT3F,EAAQmG,gBAAkB,SAAUC,EAAOT,GACzC,IAAKU,MAAMC,QAAQF,GACjB,KAAM,IAAIpC,OAAM,uDAGlB,KAAK,GAAI4B,GAAI,EAAGA,EAAIE,UAAUC,OAAQH,IAGpC,IAAK,GAFDI,GAAQF,UAAUF,GAEb9E,EAAI,EAAGA,EAAIsF,EAAML,OAAQjF,IAAK,CACrC,GAAImF,GAAOG,EAAMtF,EACbkF,GAAME,eAAeD,KACvBN,EAAEM,GAAQD,EAAMC,IAItB,MAAON,IAWT3F,EAAQuG,oBAAsB,SAAUH,EAAOT,EAAGa,GAEhD,GAAIH,MAAMC,QAAQE,GAChB,KAAM,IAAIC,WAAU,yCAEtB,KAAK,GAAIb,GAAI,EAAGA,EAAIE,UAAUC,OAAQH,IAEpC,IAAK,GADDI,GAAQF,UAAUF,GACb9E,EAAI,EAAGA,EAAIsF,EAAML,OAAQjF,IAAK,CACrC,GAAImF,GAAOG,EAAMtF,EACjB,IAAIkF,EAAME,eAAeD,GACvB,GAAIO,EAAEP,IAASO,EAAEP,GAAMS,cAAgBC,OACrBC,SAAZjB,EAAEM,KACJN,EAAEM,OAEAN,EAAEM,GAAMS,cAAgBC,OAC1B3G,EAAQ6G,WAAWlB,EAAEM,GAAOO,EAAEP,IAG9BN,EAAEM,GAAQO,EAAEP,OAET,CAAA,GAAII,MAAMC,QAAQE,EAAEP,IACzB,KAAM,IAAIQ,WAAU,yCAEpBd,GAAEM,GAAQO,EAAEP,IAMpB,MAAON,IAWT3F,EAAQ8G,uBAAyB,SAAUV,EAAOT,EAAGa,GAEnD,GAAIH,MAAMC,QAAQE,GAChB,KAAM,IAAIC,WAAU,yCAEtB,KAAK,GAAIR,KAAQO,GACf,GAAIA,EAAEN,eAAeD,IACQ,IAAvBG,EAAMW,QAAQd,GAChB,GAAIO,EAAEP,IAASO,EAAEP,GAAMS,cAAgBC,OACrBC,SAAZjB,EAAEM,KACJN,EAAEM,OAEAN,EAAEM,GAAMS,cAAgBC,OAC1B3G,EAAQ6G,WAAWlB,EAAEM,GAAOO,EAAEP,IAG9BN,EAAEM,GAAQO,EAAEP,OAET,CAAA,GAAII,MAAMC,QAAQE,EAAEP,IACzB,KAAM,IAAIQ,WAAU,yCAEpBd,GAAEM,GAAQO,EAAEP,GAKpB,MAAON,IAST3F,EAAQ6G,WAAa,SAASlB,EAAGa,GAE/B,GAAIH,MAAMC,QAAQE,GAChB,KAAM,IAAIC,WAAU,yCAGtB,KAAK,GAAIR,KAAQO,GACf,GAAIA,EAAEN,eAAeD,GACnB,GAAIO,EAAEP,IAASO,EAAEP,GAAMS,cAAgBC,OACrBC,SAAZjB,EAAEM,KACJN,EAAEM,OAEAN,EAAEM,GAAMS,cAAgBC,OAC1B3G,EAAQ6G,WAAWlB,EAAEM,GAAOO,EAAEP,IAG9BN,EAAEM,GAAQO,EAAEP,OAET,CAAA,GAAII,MAAMC,QAAQE,EAAEP,IACzB,KAAM,IAAIQ,WAAU,yCAEpBd,GAAEM,GAAQO,EAAEP,GAIlB,MAAON,IAUT3F,EAAQgH,WAAa,SAAUrB,EAAGa,GAChC,GAAIb,EAAEI,QAAUS,EAAET,OAAQ,OAAO,CAEjC,KAAK,GAAIH,GAAI,EAAGC,EAAMF,EAAEI,OAAYF,EAAJD,EAASA,IACvC,GAAID,EAAEC,IAAMY,EAAEZ,GAAI,OAAO,CAG3B,QAAO,GAYT5F,EAAQiH,QAAU,SAAS5C,EAAQ6C,GACjC,GAAIvC,EAEJ,IAAeiC,SAAXvC,EACF,MAAOuC,OAET,IAAe,OAAXvC,EACF,MAAO,KAGT,KAAK6C,EACH,MAAO7C,EAET,IAAsB,gBAAT6C,MAAwBA,YAAgB1C,SACnD,KAAM,IAAIR,OAAM,wBAIlB,QAAQkD,GACN,IAAK,UACL,IAAK,UACH,MAAOC,SAAQ9C,EAEjB,KAAK,SACL,IAAK,SACH,MAAOC,QAAOD,EAAO+C,UAEvB,KAAK,SACL,IAAK,SACH,MAAO5C,QAAOH,EAEhB,KAAK,OACH,GAAIrE,EAAQoE,SAASC,GACnB,MAAO,IAAIK,MAAKL,EAElB,IAAIA,YAAkBK,MACpB,MAAO,IAAIA,MAAKL,EAAO+C,UAEpB,IAAInD,EAAOoD,SAAShD,GACvB,MAAO,IAAIK,MAAKL,EAAO+C,UAEzB,IAAIpH,EAAQuE,SAASF,GAEnB,MADAM,GAAQC,EAAaC,KAAKR,GACtBM,EAEK,GAAID,MAAKJ,OAAOK,EAAM,KAGtBV,EAAOI,GAAQiD,QAIxB,MAAM,IAAItD,OACN,iCAAmChE,EAAQuH,QAAQlD,GAC/C,gBAGZ,KAAK,SACH,GAAIrE,EAAQoE,SAASC,GACnB,MAAOJ,GAAOI,EAEhB,IAAIA,YAAkBK,MACpB,MAAOT,GAAOI,EAAO+C,UAElB,IAAInD,EAAOoD,SAAShD,GACvB,MAAOJ,GAAOI,EAEhB,IAAIrE,EAAQuE,SAASF,GAEnB,MADAM,GAAQC,EAAaC,KAAKR,GAGjBJ,EAFLU,EAEYL,OAAOK,EAAM,IAGbN,EAIhB,MAAM,IAAIL,OACN,iCAAmChE,EAAQuH,QAAQlD,GAC/C,gBAGZ,KAAK,UACH,GAAIrE,EAAQoE,SAASC,GACnB,MAAO,IAAIK,MAAKL,EAEb,IAAIA,YAAkBK,MACzB,MAAOL,GAAOmD,aAEX,IAAIvD,EAAOoD,SAAShD,GACvB,MAAOA,GAAOiD,SAASE,aAEpB,IAAIxH,EAAQuE,SAASF,GAExB,MADAM,GAAQC,EAAaC,KAAKR,GACtBM,EAEK,GAAID,MAAKJ,OAAOK,EAAM,KAAK6C,cAG3B,GAAI9C,MAAKL,GAAQmD,aAI1B,MAAM,IAAIxD,OACN,iCAAmChE,EAAQuH,QAAQlD,GAC/C,mBAGZ,KAAK,UACH,GAAIrE,EAAQoE,SAASC,GACnB,MAAO,SAAWA,EAAS,IAExB,IAAIA,YAAkBK,MACzB,MAAO,SAAWL,EAAO+C,UAAY,IAElC,IAAIpH,EAAQuE,SAASF,GAAS,CACjCM,EAAQC,EAAaC,KAAKR,EAC1B,IAAIoD,EAQJ,OALEA,GAFE9C,EAEM,GAAID,MAAKJ,OAAOK,EAAM,KAAKyC,UAG3B,GAAI1C,MAAKL,GAAQ+C,UAEpB,SAAWK,EAAQ,KAG1B,KAAM,IAAIzD,OACN,iCAAmChE,EAAQuH,QAAQlD,GAC/C,mBAGZ,SACE,KAAM,IAAIL,OAAM,iBAAmBkD,EAAO,MAOhD,IAAItC,GAAe,qBAOnB5E,GAAQuH,QAAU,SAASlD,GACzB,GAAI6C,SAAc7C,EAElB,OAAY,UAAR6C,EACY,MAAV7C,EACK,OAELA,YAAkB8C,SACb,UAEL9C,YAAkBC,QACb,SAELD,YAAkBG,QACb,SAEL6B,MAAMC,QAAQjC,GACT,QAELA,YAAkBK,MACb,OAEF,SAEQ,UAARwC,EACA,SAEQ,WAARA,EACA,UAEQ,UAARA,EACA,SAGFA,GASTlH,EAAQ0H,gBAAkB,SAASC,GACjC,MAAOA,GAAKC,wBAAwBC,MAStC7H,EAAQ8H,eAAiB,SAASH,GAChC,MAAOA,GAAKC,wBAAwBG,KAQtC/H,EAAQgI,aAAe,SAASL,EAAMM,GACpC,GAAIC,GAAUP,EAAKM,UAAUE,MAAM,IACD,KAA9BD,EAAQnB,QAAQkB,KAClBC,EAAQE,KAAKH,GACbN,EAAKM,UAAYC,EAAQG,KAAK,OASlCrI,EAAQsI,gBAAkB,SAASX,EAAMM,GACvC,GAAIC,GAAUP,EAAKM,UAAUE,MAAM,KAC/BI,EAAQL,EAAQnB,QAAQkB,EACf,KAATM,IACFL,EAAQM,OAAOD,EAAO,GACtBZ,EAAKM,UAAYC,EAAQG,KAAK,OAalCrI,EAAQyI,QAAU,SAASpE,EAAQqE,GACjC,GAAI9C,GACAC,CACJ,IAAIQ,MAAMC,QAAQjC,GAEhB,IAAKuB,EAAI,EAAGC,EAAMxB,EAAO0B,OAAYF,EAAJD,EAASA,IACxC8C,EAASrE,EAAOuB,GAAIA,EAAGvB,OAKzB,KAAKuB,IAAKvB,GACJA,EAAO6B,eAAeN,IACxB8C,EAASrE,EAAOuB,GAAIA,EAAGvB,IAY/BrE,EAAQ2I,QAAU,SAAStE,GACzB,GAAIuE,KAEJ,KAAK,GAAI3C,KAAQ5B,GACXA,EAAO6B,eAAeD,IAAO2C,EAAMR,KAAK/D,EAAO4B,GAGrD,OAAO2C,IAUT5I,EAAQ6I,eAAiB,SAASxE,EAAQyE,EAAKrB,GAC7C,MAAIpD,GAAOyE,KAASrB,GAClBpD,EAAOyE,GAAOrB,GACP,IAGA,GAYXzH,EAAQ+I,iBAAmB,SAASC,EAASC,EAAQC,EAAUC,GACzDH,EAAQD,kBACSnC,SAAfuC,IACFA,GAAa,GAEA,eAAXF,GAA2BG,UAAUC,UAAUtC,QAAQ,YAAc,IACvEkC,EAAS,kBAGXD,EAAQD,iBAAiBE,EAAQC,EAAUC,IAE3CH,EAAQM,YAAY,KAAOL,EAAQC,IAWvClJ,EAAQuJ,oBAAsB,SAASP,EAASC,EAAQC,EAAUC,GAC5DH,EAAQO,qBAES3C,SAAfuC,IACFA,GAAa,GAEA,eAAXF,GAA2BG,UAAUC,UAAUtC,QAAQ,YAAc,IACvEkC,EAAS,kBAGXD,EAAQO,oBAAoBN,EAAQC,EAAUC,IAG9CH,EAAQQ,YAAY,KAAOP,EAAQC,IAOvClJ,EAAQyJ,eAAiB,SAAUC,GAC5BA,IACHA,EAAQC,OAAOD,OAEbA,EAAMD,eACRC,EAAMD,iBAGNC,EAAME,aAAc,GASxB5J,EAAQ6J,UAAY,SAASH,GAEtBA,IACHA,EAAQC,OAAOD,MAGjB,IAAII,EAcJ,OAZIJ,GAAMI,OACRA,EAASJ,EAAMI,OAERJ,EAAMK,aACbD,EAASJ,EAAMK,YAGMnD,QAAnBkD,EAAOE,UAA4C,GAAnBF,EAAOE,WAEzCF,EAASA,EAAOG,YAGXH,GAGT9J,EAAQkK,UAQRlK,EAAQkK,OAAOC,UAAY,SAAU1C,EAAO2C,GAK1C,MAJoB,kBAAT3C,KACTA,EAAQA,KAGG,MAATA,EACe,GAATA,EAGH2C,GAAgB,MASzBpK,EAAQkK,OAAOG,SAAW,SAAU5C,EAAO2C,GAKzC,MAJoB,kBAAT3C,KACTA,EAAQA,KAGG,MAATA,EACKnD,OAAOmD,IAAU2C,GAAgB,KAGnCA,GAAgB,MASzBpK,EAAQkK,OAAOI,SAAW,SAAU7C,EAAO2C,GAKzC,MAJoB,kBAAT3C,KACTA,EAAQA,KAGG,MAATA,EACKjD,OAAOiD,GAGT2C,GAAgB,MASzBpK,EAAQkK,OAAOK,OAAS,SAAU9C,EAAO2C,GAKvC,MAJoB,kBAAT3C,KACTA,EAAQA,KAGNzH,EAAQuE,SAASkD,GACZA,EAEAzH,EAAQoE,SAASqD,GACjBA,EAAQ,KAGR2C,GAAgB,MAU3BpK,EAAQkK,OAAOM,UAAY,SAAU/C,EAAO2C,GAK1C,MAJoB,kBAAT3C,KACTA,EAAQA,KAGHA,GAAS2C,GAAgB,MASlCpK,EAAQyK,SAAW,SAASC,GAE1B,GAAIC,GAAiB,kCACrBD,GAAMA,EAAIE,QAAQD,EAAgB,SAAS/J,EAAGiK,EAAGC,EAAGtE,GAChD,MAAOqE,GAAIA,EAAIC,EAAIA,EAAItE,EAAIA,GAE/B,IAAIuE,GAAS,4CAA4ClG,KAAK6F,EAC9D,OAAOK,IACHF,EAAGG,SAASD,EAAO,GAAI,IACvBD,EAAGE,SAASD,EAAO,GAAI,IACvBvE,EAAGwE,SAASD,EAAO,GAAI,KACvB,MAWN/K,EAAQiL,SAAW,SAASC,EAAIC,EAAMC,GACpC,MAAO,MAAQ,GAAK,KAAOF,GAAO,KAAOC,GAAS,GAAKC,GAAM3F,SAAS,IAAI4F,MAAM,IASlFrL,EAAQsL,WAAa,SAASC,GAC5B,GAAI1K,EACJ,IAAIb,EAAQuE,SAASgH,GAAQ,CAC3B,GAAIvL,EAAQwL,WAAWD,GAAQ,CAC7B,GAAIE,GAAMF,EAAMG,OAAO,GAAGA,OAAO,EAAEH,EAAMxF,OAAO,GAAGoC,MAAM,IACzDoD,GAAQvL,EAAQiL,SAASQ,EAAI,GAAGA,EAAI,GAAGA,EAAI,IAE7C,GAAIzL,EAAQ2L,WAAWJ,GAAQ,CAC7B,GAAIK,GAAM5L,EAAQ6L,SAASN,GACvBO,GAAmBC,EAAEH,EAAIG,EAAEC,EAAU,IAARJ,EAAII,EAASC,EAAE3G,KAAK4G,IAAI,EAAU,KAARN,EAAIK,IAC3DE,GAAmBJ,EAAEH,EAAIG,EAAEC,EAAE1G,KAAK4G,IAAI,EAAU,KAARN,EAAIK,GAAUA,EAAQ,GAANL,EAAIK,GAC5DG,EAAkBpM,EAAQqM,SAASF,EAAeJ,EAAGI,EAAeJ,EAAGI,EAAeF,GACtFK,EAAkBtM,EAAQqM,SAASP,EAAgBC,EAAED,EAAgBE,EAAEF,EAAgBG,EAE3FpL,IACE0L,WAAYhB,EACZiB,OAAOJ,EACPK,WACEF,WAAWD,EACXE,OAAOJ,GAETM,OACEH,WAAWD,EACXE,OAAOJ,QAKXvL,IACE0L,WAAWhB,EACXiB,OAAOjB,EACPkB,WACEF,WAAWhB,EACXiB,OAAOjB,GAETmB,OACEH,WAAWhB,EACXiB,OAAOjB,QAMb1K,MACAA,EAAE0L,WAAahB,EAAMgB,YAAc,QACnC1L,EAAE2L,OAASjB,EAAMiB,QAAU3L,EAAE0L,WAEzBvM,EAAQuE,SAASgH,EAAMkB,WACzB5L,EAAE4L,WACAD,OAAQjB,EAAMkB,UACdF,WAAYhB,EAAMkB,YAIpB5L,EAAE4L,aACF5L,EAAE4L,UAAUF,WAAahB,EAAMkB,WAAalB,EAAMkB,UAAUF,YAAc1L,EAAE0L,WAC5E1L,EAAE4L,UAAUD,OAASjB,EAAMkB,WAAalB,EAAMkB,UAAUD,QAAU3L,EAAE2L,QAGlExM,EAAQuE,SAASgH,EAAMmB,OACzB7L,EAAE6L,OACAF,OAAQjB,EAAMmB,MACdH,WAAYhB,EAAMmB,QAIpB7L,EAAE6L,SACF7L,EAAE6L,MAAMH,WAAahB,EAAMmB,OAASnB,EAAMmB,MAAMH,YAAc1L,EAAE0L,WAChE1L,EAAE6L,MAAMF,OAASjB,EAAMmB,OAASnB,EAAMmB,MAAMF,QAAU3L,EAAE2L,OAI5D,OAAO3L,IAYTb,EAAQ2M,SAAW,SAASzB,EAAIC,EAAMC,GACpCF,GAAQ,IAAKC,GAAY,IAAKC,GAAU,GACxC,IAAIwB,GAAStH,KAAK4G,IAAIhB,EAAI5F,KAAK4G,IAAIf,EAAMC,IACrCyB,EAASvH,KAAKwH,IAAI5B,EAAI5F,KAAKwH,IAAI3B,EAAMC,GAGzC,IAAIwB,GAAUC,EACZ,OAAQd,EAAE,EAAEC,EAAE,EAAEC,EAAEW,EAIpB,IAAIG,GAAK7B,GAAK0B,EAAUzB,EAAMC,EAASA,GAAMwB,EAAU1B,EAAIC,EAAQC,EAAKF,EACpEa,EAAKb,GAAK0B,EAAU,EAAMxB,GAAMwB,EAAU,EAAI,EAC9CI,EAAM,IAAIjB,EAAIgB,GAAGF,EAASD,IAAS,IACnCK,GAAcJ,EAASD,GAAQC,EAC/BpF,EAAQoF,CACZ,QAAQd,EAAEiB,EAAIhB,EAAEiB,EAAWhB,EAAExE,GAG/B,IAAIyF,IAEF/E,MAAO,SAAUgF,GACf,GAAIC,KAWJ,OATAD,GAAQhF,MAAM,KAAKM,QAAQ,SAAU4E,GACnC,GAAoB,IAAhBA,EAAMC,OAAc,CACtB,GAAIC,GAAQF,EAAMlF,MAAM,KACpBW,EAAMyE,EAAM,GAAGD,OACf7F,EAAQ8F,EAAM,GAAGD,MACrBF,GAAOtE,GAAOrB,KAIX2F,GAIT/E,KAAM,SAAU+E,GACd,MAAOzG,QAAO6G,KAAKJ,GACdK,IAAI,SAAU3E,GACb,MAAOA,GAAM,KAAOsE,EAAOtE,KAE5BT,KAAK,OASdrI,GAAQ0N,WAAa,SAAU1E,EAASmE,GACtC,GAAIQ,GAAgBT,EAAQ/E,MAAMa,EAAQqE,MAAMF,SAC5CS,EAAYV,EAAQ/E,MAAMgF,GAC1BC,EAASpN,EAAQ0F,OAAOiI,EAAeC,EAE3C5E,GAAQqE,MAAMF,QAAUD,EAAQ7E,KAAK+E,IAQvCpN,EAAQ6N,cAAgB,SAAU7E,EAASmE,GACzC,GAAIC,GAASF,EAAQ/E,MAAMa,EAAQqE,MAAMF,SACrCW,EAAeZ,EAAQ/E,MAAMgF,EAEjC,KAAK,GAAIrE,KAAOgF,GACVA,EAAa5H,eAAe4C,UACvBsE,GAAOtE,EAIlBE,GAAQqE,MAAMF,QAAUD,EAAQ7E,KAAK+E,IAWvCpN,EAAQ+N,SAAW,SAAShC,EAAGC,EAAGC,GAChC,GAAIpB,GAAGC,EAAGtE,EAENZ,EAAIN,KAAKC,MAAU,EAAJwG,GACfiC,EAAQ,EAAJjC,EAAQnG,EACZ9E,EAAImL,GAAK,EAAID,GACbiC,EAAIhC,GAAK,EAAI+B,EAAIhC,GACjBkC,EAAIjC,GAAK,GAAK,EAAI+B,GAAKhC,EAE3B,QAAQpG,EAAI,GACV,IAAK,GAAGiF,EAAIoB,EAAGnB,EAAIoD,EAAG1H,EAAI1F,CAAG,MAC7B,KAAK,GAAG+J,EAAIoD,EAAGnD,EAAImB,EAAGzF,EAAI1F,CAAG,MAC7B,KAAK,GAAG+J,EAAI/J,EAAGgK,EAAImB,EAAGzF,EAAI0H,CAAG,MAC7B,KAAK,GAAGrD,EAAI/J,EAAGgK,EAAImD,EAAGzH,EAAIyF,CAAG,MAC7B,KAAK,GAAGpB,EAAIqD,EAAGpD,EAAIhK,EAAG0F,EAAIyF,CAAG,MAC7B,KAAK,GAAGpB,EAAIoB,EAAGnB,EAAIhK,EAAG0F,EAAIyH,EAG5B,OAAQpD,EAAEvF,KAAKC,MAAU,IAAJsF,GAAUC,EAAExF,KAAKC,MAAU,IAAJuF,GAAUtE,EAAElB,KAAKC,MAAU,IAAJiB,KAGrExG,EAAQqM,SAAW,SAASN,EAAGC,EAAGC,GAChC,GAAIR,GAAMzL,EAAQ+N,SAAShC,EAAGC,EAAGC,EACjC,OAAOjM,GAAQiL,SAASQ,EAAIZ,EAAGY,EAAIX,EAAGW,EAAIjF,IAG5CxG,EAAQ6L,SAAW,SAASnB,GAC1B,GAAIe,GAAMzL,EAAQyK,SAASC,EAC3B,OAAO1K,GAAQ2M,SAASlB,EAAIZ,EAAGY,EAAIX,EAAGW,EAAIjF,IAG5CxG,EAAQ2L,WAAa,SAASjB,GAC5B,GAAIyD,GAAO,qCAAqCC,KAAK1D,EACrD,OAAOyD,IAGTnO,EAAQwL,WAAa,SAASC,GAC5BA,EAAMA,EAAIb,QAAQ,IAAI,GACtB,IAAIuD,GAAO,wCAAwCC,KAAK3C,EACxD,OAAO0C,IAUTnO,EAAQqO,sBAAwB,SAASC,EAAQC,GAC/C,GAA8B,gBAAnBA,GAA6B,CAEtC,IAAK,GADDC,GAAW7H,OAAO8H,OAAOF,GACpB3I,EAAI,EAAGA,EAAI0I,EAAOvI,OAAQH,IAC7B2I,EAAgBrI,eAAeoI,EAAO1I,KACC,gBAA9B2I,GAAgBD,EAAO1I,MAChC4I,EAASF,EAAO1I,IAAM5F,EAAQ0O,aAAaH,EAAgBD,EAAO1I,KAIxE,OAAO4I,GAGP,MAAO,OAWXxO,EAAQ0O,aAAe,SAASH,GAC9B,GAA8B,gBAAnBA,GAA6B,CACtC,GAAIC,GAAW7H,OAAO8H,OAAOF,EAC7B,KAAK,GAAI3I,KAAK2I,GACRA,EAAgBrI,eAAeN,IACA,gBAAtB2I,GAAgB3I,KACzB4I,EAAS5I,GAAK5F,EAAQ0O,aAAaH,EAAgB3I,IAIzD,OAAO4I,GAGP,MAAO,OAcXxO,EAAQ2O,aAAe,SAAUC,EAAaC,EAAS3E,GACrD,GAAwBtD,SAApBiI,EAAQ3E,GACV,GAA8B,iBAAnB2E,GAAQ3E,GACjB0E,EAAY1E,GAAQ4E,QAAUD,EAAQ3E,OAEnC,CACH0E,EAAY1E,GAAQ4E,SAAU,CAC9B,KAAK,GAAI7I,KAAQ4I,GAAQ3E,GACnB2E,EAAQ3E,GAAQhE,eAAeD,KACjC2I,EAAY1E,GAAQjE,GAAQ4I,EAAQ3E,GAAQjE,MAmBtDjG,EAAQ+O,mBAAqB,SAASC,EAAcC,EAAgBC,EAAOC,GAMzE,IALA,GAAIC,GAAgB,IAChBC,EAAY,EACZC,EAAM,EACNC,EAAOP,EAAajJ,OAAS,EAEnBwJ,GAAPD,GAA2BF,EAAZC,GAA2B,CAC/C,GAAIG,GAASlK,KAAKC,OAAO+J,EAAMC,GAAQ,GAEnCE,EAAOT,EAAaQ,GACpB/H,EAAoBb,SAAXuI,EAAwBM,EAAKP,GAASO,EAAKP,GAAOC,GAE3DO,EAAeT,EAAexH,EAClC,IAAoB,GAAhBiI,EACF,MAAOF,EAEgB,KAAhBE,EACPJ,EAAME,EAAS,EAGfD,EAAOC,EAAS,EAGlBH,IAGF,MAAO,IAeTrP,EAAQ2P,kBAAoB,SAASX,EAAclF,EAAQoF,EAAOU,GAOhE,IANA,GAIIC,GAAWpI,EAAOqI,EAAWN,EAJ7BJ,EAAgB,IAChBC,EAAY,EACZC,EAAM,EACNC,EAAOP,EAAajJ,OAAS,EAGnBwJ,GAAPD,GAA2BF,EAAZC,GAA2B,CAO/C,GALAG,EAASlK,KAAKC,MAAM,IAAKgK,EAAKD,IAC9BO,EAAYb,EAAa1J,KAAKwH,IAAI,EAAE0C,EAAS,IAAIN,GACjDzH,EAAYuH,EAAaQ,GAAQN,GACjCY,EAAYd,EAAa1J,KAAK4G,IAAI8C,EAAajJ,OAAO,EAAEyJ,EAAS,IAAIN,GAEjEzH,GAASqC,EACX,MAAO0F,EAEJ,IAAgB1F,EAAZ+F,GAAsBpI,EAAQqC,EACrC,MAAyB,UAAlB8F,EAA6BtK,KAAKwH,IAAI,EAAE0C,EAAS,GAAKA,CAE1D,IAAY1F,EAARrC,GAAkBqI,EAAYhG,EACrC,MAAyB,UAAlB8F,EAA6BJ,EAASlK,KAAK4G,IAAI8C,EAAajJ,OAAO,EAAEyJ,EAAS,EAGzE1F,GAARrC,EACF6H,EAAME,EAAS,EAGfD,EAAOC,EAAS,EAGpBH,IAIF,MAAO,IAYTrP,EAAQ+P,cAAgB,SAAU7B,EAAG8B,EAAOC,EAAKC,GAC/C,GAAIC,GAASF,EAAMD,CAEnB,OADA9B,IAAKgC,EAAS,EACN,EAAJhC,EAAciC,EAAO,EAAEjC,EAAEA,EAAI8B,GACjC9B,KACQiC,EAAO,GAAKjC,GAAGA,EAAE,GAAK,GAAK8B,IAUrChQ,EAAQoQ,iBAENC,OAAQ,SAAUnC,GAChB,MAAOA,IAGToC,WAAY,SAAUpC,GACpB,MAAOA,GAAIA,GAGbqC,YAAa,SAAUrC,GACrB,MAAOA,IAAK,EAAIA,IAGlB6B,cAAe,SAAU7B,GACvB,MAAW,GAAJA,EAAS,EAAIA,EAAIA,EAAI,IAAM,EAAI,EAAIA,GAAKA,GAGjDsC,YAAa,SAAUtC,GACrB,MAAOA,GAAIA,EAAIA,GAGjBuC,aAAc,SAAUvC,GACtB,QAAUA,EAAKA,EAAIA,EAAI,GAGzBwC,eAAgB,SAAUxC,GACxB,MAAW,GAAJA,EAAS,EAAIA,EAAIA,EAAIA,GAAKA,EAAI,IAAM,EAAIA,EAAI,IAAM,EAAIA,EAAI,GAAK,GAGxEyC,YAAa,SAAUzC,GACrB,MAAOA,GAAIA,EAAIA,EAAIA,GAGrB0C,aAAc,SAAU1C,GACtB,MAAO,MAAOA,EAAKA,EAAIA,EAAIA,GAG7B2C,eAAgB,SAAU3C,GACxB,MAAW,GAAJA,EAAS,EAAIA,EAAIA,EAAIA,EAAIA,EAAI,EAAI,IAAOA,EAAKA,EAAIA,EAAIA,GAG9D4C,YAAa,SAAU5C,GACrB,MAAOA,GAAIA,EAAIA,EAAIA,EAAIA,GAGzB6C,aAAc,SAAU7C,GACtB,MAAO,KAAOA,EAAKA,EAAIA,EAAIA,EAAIA,GAGjC8C,eAAgB,SAAU9C,GACxB,MAAW,GAAJA,EAAS,GAAKA,EAAIA,EAAIA,EAAIA,EAAIA,EAAI,EAAI,KAAQA,EAAKA,EAAIA,EAAIA,EAAIA,KAMtE,SAASjO,EAAQD,GASrBA,EAAQiR,gBAAkB,SAASC,GAEjC,IAAK,GAAIC,KAAeD,GAClBA,EAAchL,eAAeiL,KAC/BD,EAAcC,GAAaC,UAAYF,EAAcC,GAAaE,KAClEH,EAAcC,GAAaE,UAYjCrR,EAAQsR,gBAAkB,SAASJ,GAEjC,IAAK,GAAIC,KAAeD,GACtB,GAAIA,EAAchL,eAAeiL,IAC3BD,EAAcC,GAAaC,UAAW,CACxC,IAAK,GAAIxL,GAAI,EAAGA,EAAIsL,EAAcC,GAAaC,UAAUrL,OAAQH,IAC/DsL,EAAcC,GAAaC,UAAUxL,GAAGqE,WAAWsH,YAAYL,EAAcC,GAAaC,UAAUxL,GAEtGsL,GAAcC,GAAaC,eAgBnCpR,EAAQwR,cAAgB,SAAUL,EAAaD,EAAeO,GAC5D,GAAIzI,EAqBJ,OAnBIkI,GAAchL,eAAeiL,GAE3BD,EAAcC,GAAaC,UAAUrL,OAAS,GAChDiD,EAAUkI,EAAcC,GAAaC,UAAU,GAC/CF,EAAcC,GAAaC,UAAUM,UAIrC1I,EAAU2I,SAASC,gBAAgB,6BAA8BT,GACjEM,EAAaI,YAAY7I,KAK3BA,EAAU2I,SAASC,gBAAgB,6BAA8BT,GACjED,EAAcC,IAAgBE,QAAUD,cACxCK,EAAaI,YAAY7I,IAE3BkI,EAAcC,GAAaE,KAAKjJ,KAAKY,GAC9BA,GAcThJ,EAAQ8R,cAAgB,SAAUX,EAAaD,EAAea,EAAcC,GAC1E,GAAIhJ,EA+BJ,OA7BIkI,GAAchL,eAAeiL,GAE3BD,EAAcC,GAAaC,UAAUrL,OAAS,GAChDiD,EAAUkI,EAAcC,GAAaC,UAAU,GAC/CF,EAAcC,GAAaC,UAAUM,UAIrC1I,EAAU2I,SAASM,cAAcd,GACZvK,SAAjBoL,EACFD,EAAaC,aAAahJ,EAASgJ,GAGnCD,EAAaF,YAAY7I,KAM7BA,EAAU2I,SAASM,cAAcd,GACjCD,EAAcC,IAAgBE,QAAUD,cACnBxK,SAAjBoL,EACFD,EAAaC,aAAahJ,EAASgJ,GAGnCD,EAAaF,YAAY7I,IAG7BkI,EAAcC,GAAaE,KAAKjJ,KAAKY,GAC9BA,GAkBThJ,EAAQkS,UAAY,SAASC,EAAGC,EAAGC,EAAOnB,EAAeO,GACvD,GAAIa,EAmBJ,OAlBsC,UAAlCD,EAAMxD,QAAQ0D,WAAWlF,OAC3BiF,EAAQtS,EAAQwR,cAAc,SAASN,EAAcO,GACrDa,EAAME,eAAe,KAAM,KAAML,GACjCG,EAAME,eAAe,KAAM,KAAMJ,GACjCE,EAAME,eAAe,KAAM,IAAK,GAAMH,EAAMxD,QAAQ0D,WAAWE,QAG/DH,EAAQtS,EAAQwR,cAAc,OAAON,EAAcO,GACnDa,EAAME,eAAe,KAAM,IAAKL,EAAI,GAAIE,EAAMxD,QAAQ0D,WAAWE,MACjEH,EAAME,eAAe,KAAM,IAAKJ,EAAI,GAAIC,EAAMxD,QAAQ0D,WAAWE,MACjEH,EAAME,eAAe,KAAM,QAASH,EAAMxD,QAAQ0D,WAAWE,MAC7DH,EAAME,eAAe,KAAM,SAAUH,EAAMxD,QAAQ0D,WAAWE,OAGzB7L,SAApCyL,EAAMxD,QAAQ0D,WAAWnF,QAC1BkF,EAAME,eAAe,KAAM,QAASH,EAAMA,MAAMxD,QAAQ0D,WAAWnF,QAErEkF,EAAME,eAAe,KAAM,QAASH,EAAMpK,UAAY,UAC/CqK,GAUTtS,EAAQ0S,QAAU,SAAUP,EAAGC,EAAGO,EAAOC,EAAQ3K,EAAWiJ,EAAeO,GACzE,GAAc,GAAVmB,EAAa,CACF,EAATA,IACFA,GAAU,GACVR,GAAKQ,EAEP,IAAIC,GAAO7S,EAAQwR,cAAc,OAAON,EAAeO,EACvDoB,GAAKL,eAAe,KAAM,IAAKL,EAAI,GAAMQ,GACzCE,EAAKL,eAAe,KAAM,IAAKJ,GAC/BS,EAAKL,eAAe,KAAM,QAASG,GACnCE,EAAKL,eAAe,KAAM,SAAUI,GACpCC,EAAKL,eAAe,KAAM,QAASvK,MAMnC,SAAShI,EAAQD,EAASM,GAgD9B,QAASW,GAAS6R,EAAMjE,GActB,IAZIiE,GAASzM,MAAMC,QAAQwM,IAAU/R,EAAKiE,YAAY8N,KACpDjE,EAAUiE,EACVA,EAAO,MAGT1S,KAAK2S,SAAWlE,MAChBzO,KAAK4S,SACL5S,KAAK6S,SAAW7S,KAAK2S,SAASG,SAAW,KACzC9S,KAAK+S,SAID/S,KAAK2S,SAAS7L,KAChB,IAAK,GAAIgI,KAAS9O,MAAK2S,SAAS7L,KAC9B,GAAI9G,KAAK2S,SAAS7L,KAAKhB,eAAegJ,GAAQ,CAC5C,GAAIzH,GAAQrH,KAAK2S,SAAS7L,KAAKgI,EAE7B9O,MAAK+S,MAAMjE,GADA,QAATzH,GAA4B,WAATA,GAA+B,WAATA,EACvB,OAGAA,EAO5B,GAAIrH,KAAK2S,SAAS9L,QAChB,KAAM,IAAIjD,OAAM,sDAGlB5D,MAAKgT,gBAGDN,GACF1S,KAAKiT,IAAIP,GAGX1S,KAAKkT,WAAWzE,GAtFlB,GAAI9N,GAAOT,EAAoB,GAC3Ba,EAAQb,EAAoB,EAiGhCW,GAAQsS,UAAUD,WAAa,SAASzE,GAClCA,GAA6BjI,SAAlBiI,EAAQ2E,QACjB3E,EAAQ2E,SAAU,EAEhBpT,KAAKqT,SACPrT,KAAKqT,OAAOC,gBACLtT,MAAKqT,SAKTrT,KAAKqT,SACRrT,KAAKqT,OAAStS,EAAMuE,OAAOtF,MACzBwK,SAAU,MAAO,SAAU,aAIF,gBAAlBiE,GAAQ2E,OACjBpT,KAAKqT,OAAOH,WAAWzE,EAAQ2E,UAevCvS,EAAQsS,UAAUI,GAAK,SAASjK,EAAOhB,GACrC,GAAIkL,GAAcxT,KAAKgT,aAAa1J,EAC/BkK,KACHA,KACAxT,KAAKgT,aAAa1J,GAASkK,GAG7BA,EAAYxL,MACVM,SAAUA,KAKdzH,EAAQsS,UAAUM,UAAY5S,EAAQsS,UAAUI,GAOhD1S,EAAQsS,UAAUO,IAAM,SAASpK,EAAOhB,GACtC,GAAIkL,GAAcxT,KAAKgT,aAAa1J,EAChCkK,KACFxT,KAAKgT,aAAa1J,GAASkK,EAAYG,OAAO,SAAU7K,GACtD,MAAQA,GAASR,UAAYA,MAMnCzH,EAAQsS,UAAUS,YAAc/S,EAAQsS,UAAUO,IASlD7S,EAAQsS,UAAUU,SAAW,SAAUvK,EAAOwK,EAAQC,GACpD,GAAa,KAATzK,EACF,KAAM,IAAI1F,OAAM,yBAGlB,IAAI4P,KACAlK,KAAStJ,MAAKgT,eAChBQ,EAAcA,EAAYQ,OAAOhU,KAAKgT,aAAa1J,KAEjD,KAAOtJ,MAAKgT,eACdQ,EAAcA,EAAYQ,OAAOhU,KAAKgT,aAAa,MAGrD,KAAK,GAAIxN,GAAI,EAAGA,EAAIgO,EAAY7N,OAAQH,IAAK,CAC3C,GAAIyO,GAAaT,EAAYhO,EACzByO,GAAW3L,UACb2L,EAAW3L,SAASgB,EAAOwK,EAAQC,GAAY,QAYrDlT,EAAQsS,UAAUF,IAAM,SAAUP,EAAMqB,GACtC,GACI1T,GADA6T,KAEAC,EAAKnU,IAET,IAAIiG,MAAMC,QAAQwM,GAEhB,IAAK,GAAIlN,GAAI,EAAGC,EAAMiN,EAAK/M,OAAYF,EAAJD,EAASA,IAC1CnF,EAAK8T,EAAGC,SAAS1B,EAAKlN,IACtB0O,EAASlM,KAAK3H,OAGb,IAAIM,EAAKiE,YAAY8N,GAGxB,IAAK,GADD2B,GAAUrU,KAAKsU,gBAAgB5B,GAC1B6B,EAAM,EAAGC,EAAO9B,EAAK+B,kBAAyBD,EAAND,EAAYA,IAAO,CAElE,IAAK,GADDlF,MACKqF,EAAM,EAAGC,EAAON,EAAQ1O,OAAcgP,EAAND,EAAYA,IAAO,CAC1D,GAAI5F,GAAQuF,EAAQK,EACpBrF,GAAKP,GAAS4D,EAAKkC,SAASL,EAAKG,GAGnCrU,EAAK8T,EAAGC,SAAS/E,GACjB6E,EAASlM,KAAK3H,OAGb,CAAA,KAAIqS,YAAgBnM,SAMvB,KAAM,IAAI3C,OAAM,mBAJhBvD,GAAK8T,EAAGC,SAAS1B,GACjBwB,EAASlM,KAAK3H,GAUhB,MAJI6T,GAASvO,QACX3F,KAAK6T,SAAS,OAAQ5R,MAAOiS,GAAWH,GAGnCG,GASTrT,EAAQsS,UAAU0B,OAAS,SAAUnC,EAAMqB,GACzC,GAAIG,MACAY,KACAC,KACAZ,EAAKnU,KACL8S,EAAUqB,EAAGtB,SAEbmC,EAAc,SAAU3F,GAC1B,GAAIhP,GAAKgP,EAAKyD,EACVqB,GAAGvB,MAAMvS,IAEXA,EAAK8T,EAAGc,YAAY5F,GACpByF,EAAW9M,KAAK3H,GAChB0U,EAAY/M,KAAKqH,KAIjBhP,EAAK8T,EAAGC,SAAS/E,GACjB6E,EAASlM,KAAK3H,IAIlB,IAAI4F,MAAMC,QAAQwM,GAEhB,IAAK,GAAIlN,GAAI,EAAGC,EAAMiN,EAAK/M,OAAYF,EAAJD,EAASA,IAC1CwP,EAAYtC,EAAKlN,QAGhB,IAAI7E,EAAKiE,YAAY8N,GAGxB,IAAK,GADD2B,GAAUrU,KAAKsU,gBAAgB5B,GAC1B6B,EAAM,EAAGC,EAAO9B,EAAK+B,kBAAyBD,EAAND,EAAYA,IAAO,CAElE,IAAK,GADDlF,MACKqF,EAAM,EAAGC,EAAON,EAAQ1O,OAAcgP,EAAND,EAAYA,IAAO,CAC1D,GAAI5F,GAAQuF,EAAQK,EACpBrF,GAAKP,GAAS4D,EAAKkC,SAASL,EAAKG,GAGnCM,EAAY3F,OAGX,CAAA,KAAIqD,YAAgBnM,SAKvB,KAAM,IAAI3C,OAAM,mBAHhBoR,GAAYtC,GAad,MAPIwB,GAASvO,QACX3F,KAAK6T,SAAS,OAAQ5R,MAAOiS,GAAWH,GAEtCe,EAAWnP,QACb3F,KAAK6T,SAAS,UAAW5R,MAAO6S,EAAYpC,KAAMqC,GAAchB,GAG3DG,EAASF,OAAOc,IAsCzBjU,EAAQsS,UAAU+B,IAAM,WACtB,GAGI7U,GAAI8U,EAAK1G,EAASiE,EAHlByB,EAAKnU,KAILoV,EAAYzU,EAAKwG,QAAQzB,UAAU,GACtB,WAAb0P,GAAsC,UAAbA,GAE3B/U,EAAKqF,UAAU,GACf+I,EAAU/I,UAAU,GACpBgN,EAAOhN,UAAU,IAEG,SAAb0P,GAEPD,EAAMzP,UAAU,GAChB+I,EAAU/I,UAAU,GACpBgN,EAAOhN,UAAU,KAIjB+I,EAAU/I,UAAU,GACpBgN,EAAOhN,UAAU,GAInB,IAAI2P,EACJ,IAAI5G,GAAWA,EAAQ4G,WAAY,CACjC,GAAIC,IAAiB,YAAa,QAAS,SAG3C,IAFAD,EAA0D,IAA7CC,EAAc3O,QAAQ8H,EAAQ4G,YAAoB,QAAU5G,EAAQ4G,WAE7E3C,GAAS2C,GAAc1U,EAAKwG,QAAQuL,GACtC,KAAM,IAAI9O,OAAM,6BAA+BjD,EAAKwG,QAAQuL,GAAQ,sDACVjE,EAAQ3H,KAAO,IAE3E,IAAkB,aAAduO,IAA8B1U,EAAKiE,YAAY8N,GACjD,KAAM,IAAI9O,OAAM,6EAKlByR,GADO3C,GAC6B,aAAtB/R,EAAKwG,QAAQuL,GAAwB,YAGtC,OAIf,IAEgBrD,GAAMkG,EAAQ/P,EAAGC,EAF7BqB,EAAO2H,GAAWA,EAAQ3H,MAAQ9G,KAAK2S,SAAS7L,KAChD6M,EAASlF,GAAWA,EAAQkF,OAC5B1R,IAGJ,IAAUuE,QAANnG,EAEFgP,EAAO8E,EAAGqB,SAASnV,EAAIyG,GACnB6M,IAAWA,EAAOtE,KACpBA,EAAO,UAGN,IAAW7I,QAAP2O,EAEP,IAAK3P,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IACrC6J,EAAO8E,EAAGqB,SAASL,EAAI3P,GAAIsB,KACtB6M,GAAUA,EAAOtE,KACpBpN,EAAM+F,KAAKqH,OAMf,KAAKkG,IAAUvV,MAAK4S,MACd5S,KAAK4S,MAAM9M,eAAeyP,KAC5BlG,EAAO8E,EAAGqB,SAASD,EAAQzO,KACtB6M,GAAUA,EAAOtE,KACpBpN,EAAM+F,KAAKqH,GAYnB,IALIZ,GAAWA,EAAQgH,OAAejP,QAANnG,GAC9BL,KAAK0V,MAAMzT,EAAOwM,EAAQgH,OAIxBhH,GAAWA,EAAQP,OAAQ,CAC7B,GAAIA,GAASO,EAAQP,MACrB,IAAU1H,QAANnG,EACFgP,EAAOrP,KAAK2V,cAActG,EAAMnB,OAGhC,KAAK1I,EAAI,EAAGC,EAAMxD,EAAM0D,OAAYF,EAAJD,EAASA,IACvCvD,EAAMuD,GAAKxF,KAAK2V,cAAc1T,EAAMuD,GAAI0I,GAM9C,GAAkB,aAAdmH,EAA2B,CAC7B,GAAIhB,GAAUrU,KAAKsU,gBAAgB5B,EACnC,IAAUlM,QAANnG,EAEF8T,EAAGyB,WAAWlD,EAAM2B,EAAShF,OAI7B,KAAK7J,EAAI,EAAGA,EAAIvD,EAAM0D,OAAQH,IAC5B2O,EAAGyB,WAAWlD,EAAM2B,EAASpS,EAAMuD,GAGvC,OAAOkN,GAEJ,GAAkB,UAAd2C,EAAwB,CAC/B,GAAI1K,KACJ,KAAKnF,EAAI,EAAGA,EAAIvD,EAAM0D,OAAQH,IAC5BmF,EAAO1I,EAAMuD,GAAGnF,IAAM4B,EAAMuD,EAE9B,OAAOmF,GAIP,GAAUnE,QAANnG,EAEF,MAAOgP,EAIP,IAAIqD,EAAM,CAER,IAAKlN,EAAI,EAAGC,EAAMxD,EAAM0D,OAAYF,EAAJD,EAASA,IACvCkN,EAAK1K,KAAK/F,EAAMuD,GAElB,OAAOkN,GAIP,MAAOzQ,IAcfpB,EAAQsS,UAAU0C,OAAS,SAAUpH,GACnC,GAIIjJ,GACAC,EACApF,EACAgP,EACApN,EARAyQ,EAAO1S,KAAK4S,MACZe,EAASlF,GAAWA,EAAQkF,OAC5B8B,EAAQhH,GAAWA,EAAQgH,MAC3B3O,EAAO2H,GAAWA,EAAQ3H,MAAQ9G,KAAK2S,SAAS7L,KAMhDqO,IAEJ,IAAIxB,EAEF,GAAI8B,EAAO,CAETxT,IACA,KAAK5B,IAAMqS,GACLA,EAAK5M,eAAezF,KACtBgP,EAAOrP,KAAKwV,SAASnV,EAAIyG,GACrB6M,EAAOtE,IACTpN,EAAM+F,KAAKqH,GAOjB,KAFArP,KAAK0V,MAAMzT,EAAOwT,GAEbjQ,EAAI,EAAGC,EAAMxD,EAAM0D,OAAYF,EAAJD,EAASA,IACvC2P,EAAI3P,GAAKvD,EAAMuD,GAAGxF,KAAK6S,cAKzB,KAAKxS,IAAMqS,GACLA,EAAK5M,eAAezF,KACtBgP,EAAOrP,KAAKwV,SAASnV,EAAIyG,GACrB6M,EAAOtE,IACT8F,EAAInN,KAAKqH,EAAKrP,KAAK6S,gBAQ3B,IAAI4C,EAAO,CAETxT,IACA,KAAK5B,IAAMqS,GACLA,EAAK5M,eAAezF,IACtB4B,EAAM+F,KAAK0K,EAAKrS,GAMpB,KAFAL,KAAK0V,MAAMzT,EAAOwT,GAEbjQ,EAAI,EAAGC,EAAMxD,EAAM0D,OAAYF,EAAJD,EAASA,IACvC2P,EAAI3P,GAAKvD,EAAMuD,GAAGxF,KAAK6S,cAKzB,KAAKxS,IAAMqS,GACLA,EAAK5M,eAAezF,KACtBgP,EAAOqD,EAAKrS,GACZ8U,EAAInN,KAAKqH,EAAKrP,KAAK6S,WAM3B,OAAOsC,IAOTtU,EAAQsS,UAAU2C,WAAa,WAC7B,MAAO9V,OAaTa,EAAQsS,UAAU9K,QAAU,SAAUC,EAAUmG,GAC9C,GAGIY,GACAhP,EAJAsT,EAASlF,GAAWA,EAAQkF,OAC5B7M,EAAO2H,GAAWA,EAAQ3H,MAAQ9G,KAAK2S,SAAS7L,KAChD4L,EAAO1S,KAAK4S,KAIhB,IAAInE,GAAWA,EAAQgH,MAIrB,IAAK,GAFDxT,GAAQjC,KAAKkV,IAAIzG,GAEZjJ,EAAI,EAAGC,EAAMxD,EAAM0D,OAAYF,EAAJD,EAASA,IAC3C6J,EAAOpN,EAAMuD,GACbnF,EAAKgP,EAAKrP,KAAK6S,UACfvK,EAAS+G,EAAMhP,OAKjB,KAAKA,IAAMqS,GACLA,EAAK5M,eAAezF,KACtBgP,EAAOrP,KAAKwV,SAASnV,EAAIyG,KACpB6M,GAAUA,EAAOtE,KACpB/G,EAAS+G,EAAMhP,KAkBzBQ,EAAQsS,UAAU9F,IAAM,SAAU/E,EAAUmG,GAC1C,GAIIY,GAJAsE,EAASlF,GAAWA,EAAQkF,OAC5B7M,EAAO2H,GAAWA,EAAQ3H,MAAQ9G,KAAK2S,SAAS7L,KAChDiP,KACArD,EAAO1S,KAAK4S,KAIhB,KAAK,GAAIvS,KAAMqS,GACTA,EAAK5M,eAAezF,KACtBgP,EAAOrP,KAAKwV,SAASnV,EAAIyG,KACpB6M,GAAUA,EAAOtE,KACpB0G,EAAY/N,KAAKM,EAAS+G,EAAMhP,IAUtC,OAJIoO,IAAWA,EAAQgH,OACrBzV,KAAK0V,MAAMK,EAAatH,EAAQgH,OAG3BM,GAUTlV,EAAQsS,UAAUwC,cAAgB,SAAUtG,EAAMnB,GAChD,GAAI8H,KAEJ,KAAK,GAAIlH,KAASO,GACZA,EAAKvJ,eAAegJ,IAAoC,IAAzBZ,EAAOvH,QAAQmI,KAChDkH,EAAalH,GAASO,EAAKP,GAI/B,OAAOkH,IASTnV,EAAQsS,UAAUuC,MAAQ,SAAUzT,EAAOwT,GACzC,GAAI9U,EAAKwD,SAASsR,GAAQ,CAExB,GAAIQ,GAAOR,CACXxT,GAAMiU,KAAK,SAAU3Q,EAAGa,GACtB,GAAI+P,GAAK5Q,EAAE0Q,GACPG,EAAKhQ,EAAE6P,EACX,OAAQE,GAAKC,EAAM,EAAWA,EAALD,EAAW,GAAK,QAGxC,CAAA,GAAqB,kBAAVV,GAOd,KAAM,IAAIpP,WAAU,uCALpBpE,GAAMiU,KAAKT,KAgBf5U,EAAQsS,UAAUkD,OAAS,SAAUhW,EAAI0T,GACvC,GACIvO,GAAGC,EAAK6Q,EADRC,IAGJ,IAAItQ,MAAMC,QAAQ7F,GAChB,IAAKmF,EAAI,EAAGC,EAAMpF,EAAGsF,OAAYF,EAAJD,EAASA,IACpC8Q,EAAYtW,KAAKwW,QAAQnW,EAAGmF,IACX,MAAb8Q,GACFC,EAAWvO,KAAKsO,OAKpBA,GAAYtW,KAAKwW,QAAQnW,GACR,MAAbiW,GACFC,EAAWvO,KAAKsO,EAQpB,OAJIC,GAAW5Q,QACb3F,KAAK6T,SAAS,UAAW5R,MAAOsU,GAAaxC,GAGxCwC,GAST1V,EAAQsS,UAAUqD,QAAU,SAAUnW,GACpC,GAAIM,EAAKqD,SAAS3D,IAAOM,EAAKwD,SAAS9D,IACrC,GAAIL,KAAK4S,MAAMvS,GAEb,aADOL,MAAK4S,MAAMvS,GACXA,MAGN,IAAIA,YAAckG,QAAQ,CAC7B,GAAIgP,GAASlV,EAAGL,KAAK6S,SACrB,IAAI0C,GAAUvV,KAAK4S,MAAM2C,GAEvB,aADOvV,MAAK4S,MAAM2C,GACXA,EAGX,MAAO,OAQT1U,EAAQsS,UAAUsD,MAAQ,SAAU1C,GAClC,GAAIoB,GAAM5O,OAAO6G,KAAKpN,KAAK4S,MAM3B,OAJA5S,MAAK4S,SAEL5S,KAAK6T,SAAS,UAAW5R,MAAOkT,GAAMpB,GAE/BoB,GAQTtU,EAAQsS,UAAUzG,IAAM,SAAUoC,GAChC,GAAI4D,GAAO1S,KAAK4S,MACZlG,EAAM,KACNgK,EAAW,IAEf,KAAK,GAAIrW,KAAMqS,GACb,GAAIA,EAAK5M,eAAezF,GAAK,CAC3B,GAAIgP,GAAOqD,EAAKrS,GACZsW,EAAYtH,EAAKP,EACJ,OAAb6H,KAAuBjK,GAAOiK,EAAYD,KAC5ChK,EAAM2C,EACNqH,EAAWC,GAKjB,MAAOjK,IAQT7L,EAAQsS,UAAUrH,IAAM,SAAUgD,GAChC,GAAI4D,GAAO1S,KAAK4S,MACZ9G,EAAM,KACN8K,EAAW,IAEf,KAAK,GAAIvW,KAAMqS,GACb,GAAIA,EAAK5M,eAAezF,GAAK,CAC3B,GAAIgP,GAAOqD,EAAKrS,GACZsW,EAAYtH,EAAKP,EACJ,OAAb6H,KAAuB7K,GAAmB8K,EAAZD,KAChC7K,EAAMuD,EACNuH,EAAWD,GAKjB,MAAO7K,IAUTjL,EAAQsS,UAAU0D,SAAW,SAAU/H,GACrC,GAIItJ,GAJAkN,EAAO1S,KAAK4S,MACZkE,KACAC,EAAY/W,KAAK2S,SAAS7L,MAAQ9G,KAAK2S,SAAS7L,KAAKgI,IAAU,KAC/DkI,EAAQ,CAGZ,KAAK,GAAInR,KAAQ6M,GACf,GAAIA,EAAK5M,eAAeD,GAAO,CAC7B,GAAIwJ,GAAOqD,EAAK7M,GACZwB,EAAQgI,EAAKP,GACbmI,GAAS,CACb,KAAKzR,EAAI,EAAOwR,EAAJxR,EAAWA,IACrB,GAAIsR,EAAOtR,IAAM6B,EAAO,CACtB4P,GAAS,CACT,OAGCA,GAAqBzQ,SAAVa,IACdyP,EAAOE,GAAS3P,EAChB2P,KAKN,GAAID,EACF,IAAKvR,EAAI,EAAGA,EAAIsR,EAAOnR,OAAQH,IAC7BsR,EAAOtR,GAAK7E,EAAKkG,QAAQiQ,EAAOtR,GAAIuR,EAIxC,OAAOD,IASTjW,EAAQsS,UAAUiB,SAAW,SAAU/E,GACrC,GAAIhP,GAAKgP,EAAKrP,KAAK6S,SAEnB,IAAUrM,QAANnG,GAEF,GAAIL,KAAK4S,MAAMvS,GAEb,KAAM,IAAIuD,OAAM,iCAAmCvD,EAAK,uBAK1DA,GAAKM,EAAKqE,aACVqK,EAAKrP,KAAK6S,UAAYxS,CAGxB,IAAIsM,KACJ,KAAK,GAAImC,KAASO,GAChB,GAAIA,EAAKvJ,eAAegJ,GAAQ,CAC9B,GAAIiI,GAAY/W,KAAK+S,MAAMjE,EAC3BnC,GAAEmC,GAASnO,EAAKkG,QAAQwI,EAAKP,GAAQiI,GAKzC,MAFA/W,MAAK4S,MAAMvS,GAAMsM,EAEVtM,GAUTQ,EAAQsS,UAAUqC,SAAW,SAAUnV,EAAI6W,GACzC,GAAIpI,GAAOzH,EAGP8P,EAAMnX,KAAK4S,MAAMvS,EACrB,KAAK8W,EACH,MAAO,KAIT,IAAIC,KACJ,IAAIF,EACF,IAAKpI,IAASqI,GACRA,EAAIrR,eAAegJ,KACrBzH,EAAQ8P,EAAIrI,GACZsI,EAAUtI,GAASnO,EAAKkG,QAAQQ,EAAO6P,EAAMpI,SAMjD,KAAKA,IAASqI,GACRA,EAAIrR,eAAegJ,KACrBzH,EAAQ8P,EAAIrI,GACZsI,EAAUtI,GAASzH,EAIzB,OAAO+P,IAWTvW,EAAQsS,UAAU8B,YAAc,SAAU5F,GACxC,GAAIhP,GAAKgP,EAAKrP,KAAK6S,SACnB,IAAUrM,QAANnG,EACF,KAAM,IAAIuD,OAAM,6CAA+CyT,KAAKC,UAAUjI,GAAQ,IAExF,IAAI1C,GAAI3M,KAAK4S,MAAMvS,EACnB,KAAKsM,EAEH,KAAM,IAAI/I,OAAM,uCAAyCvD,EAAK,SAIhE,KAAK,GAAIyO,KAASO,GAChB,GAAIA,EAAKvJ,eAAegJ,GAAQ,CAC9B,GAAIiI,GAAY/W,KAAK+S,MAAMjE,EAC3BnC,GAAEmC,GAASnO,EAAKkG,QAAQwI,EAAKP,GAAQiI,GAIzC,MAAO1W,IASTQ,EAAQsS,UAAUmB,gBAAkB,SAAUiD,GAE5C,IAAK,GADDlD,MACKK,EAAM,EAAGC,EAAO4C,EAAUC,qBAA4B7C,EAAND,EAAYA,IACnEL,EAAQK,GAAO6C,EAAUE,YAAY/C,IAAQ6C,EAAUG,eAAehD,EAExE,OAAOL,IAUTxT,EAAQsS,UAAUyC,WAAa,SAAU2B,EAAWlD,EAAShF,GAG3D,IAAK,GAFDkF,GAAMgD,EAAUI,SAEXjD,EAAM,EAAGC,EAAON,EAAQ1O,OAAcgP,EAAND,EAAYA,IAAO,CAC1D,GAAI5F,GAAQuF,EAAQK,EACpB6C,GAAUK,SAASrD,EAAKG,EAAKrF,EAAKP,MAItCjP,EAAOD,QAAUiB,GAKb,SAAShB,EAAQD,EAASM,GAe9B,QAASY,GAAU4R,EAAMjE,GACvBzO,KAAK4S,MAAQ,KACb5S,KAAK6X,QACL7X,KAAK2S,SAAWlE,MAChBzO,KAAK6S,SAAW,KAChB7S,KAAKgT,eAEL,IAAImB,GAAKnU,IACTA,MAAK8I,SAAW,WACdqL,EAAG2D,SAASC,MAAM5D,EAAIzO,YAGxB1F,KAAKgY,QAAQtF,GAzBf,GAAI/R,GAAOT,EAAoB,GAC3BW,EAAUX,EAAoB,EAkClCY,GAASqS,UAAU6E,QAAU,SAAUtF,GACrC,GAAIyC,GAAK3P,EAAGC,CAEZ,IAAIzF,KAAK4S,MAAO,CAEV5S,KAAK4S,MAAMgB,aACb5T,KAAK4S,MAAMgB,YAAY,IAAK5T,KAAK8I,UAInCqM,IACA,KAAK,GAAI9U,KAAML,MAAK6X,KACd7X,KAAK6X,KAAK/R,eAAezF,IAC3B8U,EAAInN,KAAK3H,EAGbL,MAAK6X,QACL7X,KAAK6T,SAAS,UAAW5R,MAAOkT,IAKlC,GAFAnV,KAAK4S,MAAQF,EAET1S,KAAK4S,MAAO,CAQd,IANA5S,KAAK6S,SAAW7S,KAAK2S,SAASG,SACzB9S,KAAK4S,OAAS5S,KAAK4S,MAAMnE,SAAWzO,KAAK4S,MAAMnE,QAAQqE,SACxD,KAGJqC,EAAMnV,KAAK4S,MAAMiD,QAAQlC,OAAQ3T,KAAK2S,UAAY3S,KAAK2S,SAASgB,SAC3DnO,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IACrCnF,EAAK8U,EAAI3P,GACTxF,KAAK6X,KAAKxX,IAAM,CAElBL,MAAK6T,SAAS,OAAQ5R,MAAOkT,IAGzBnV,KAAK4S,MAAMW,IACbvT,KAAK4S,MAAMW,GAAG,IAAKvT,KAAK8I,YAuC9BhI,EAASqS,UAAU+B,IAAM,WACvB,GAGIC,GAAK1G,EAASiE,EAHdyB,EAAKnU,KAILoV,EAAYzU,EAAKwG,QAAQzB,UAAU,GACtB,WAAb0P,GAAsC,UAAbA,GAAsC,SAAbA,GAEpDD,EAAMzP,UAAU,GAChB+I,EAAU/I,UAAU,GACpBgN,EAAOhN,UAAU,KAIjB+I,EAAU/I,UAAU,GACpBgN,EAAOhN,UAAU,GAInB,IAAIuS,GAActX,EAAK2E,UAAWtF,KAAK2S,SAAUlE,EAG7CzO,MAAK2S,SAASgB,QAAUlF,GAAWA,EAAQkF,SAC7CsE,EAAYtE,OAAS,SAAUtE,GAC7B,MAAO8E,GAAGxB,SAASgB,OAAOtE,IAASZ,EAAQkF,OAAOtE,IAKtD,IAAI6I,KAOJ,OANW1R,SAAP2O,GACF+C,EAAalQ,KAAKmN,GAEpB+C,EAAalQ,KAAKiQ,GAClBC,EAAalQ,KAAK0K,GAEX1S,KAAK4S,OAAS5S,KAAK4S,MAAMsC,IAAI6C,MAAM/X,KAAK4S,MAAOsF,IAWxDpX,EAASqS,UAAU0C,OAAS,SAAUpH,GACpC,GAAI0G,EAEJ,IAAInV,KAAK4S,MAAO,CACd,GACIe,GADAwE,EAAgBnY,KAAK2S,SAASgB,MAK9BA,GAFAlF,GAAWA,EAAQkF,OACjBwE,EACO,SAAU9I,GACjB,MAAO8I,GAAc9I,IAASZ,EAAQkF,OAAOtE,IAItCZ,EAAQkF,OAIVwE,EAGXhD,EAAMnV,KAAK4S,MAAMiD,QACflC,OAAQA,EACR8B,MAAOhH,GAAWA,EAAQgH,YAI5BN,KAGF,OAAOA,IAQTrU,EAASqS,UAAU2C,WAAa,WAE9B,IADA,GAAIsC,GAAUpY,KACPoY,YAAmBtX,IACxBsX,EAAUA,EAAQxF,KAEpB,OAAOwF,IAAW,MAYpBtX,EAASqS,UAAU2E,SAAW,SAAUxO,EAAOwK,EAAQC,GACrD,GAAIvO,GAAGC,EAAKpF,EAAIgP,EACZ8F,EAAMrB,GAAUA,EAAO7R,MACvByQ,EAAO1S,KAAK4S,MACZyF,KACAC,KACAC,IAEJ,IAAIpD,GAAOzC,EAAM,CACf,OAAQpJ,GACN,IAAK,MAEH,IAAK9D,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IACrCnF,EAAK8U,EAAI3P,GACT6J,EAAOrP,KAAKkV,IAAI7U,GACZgP,IACFrP,KAAK6X,KAAKxX,IAAM,EAChBgY,EAAMrQ,KAAK3H,GAIf,MAEF,KAAK,SAGH,IAAKmF,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IACrCnF,EAAK8U,EAAI3P,GACT6J,EAAOrP,KAAKkV,IAAI7U,GAEZgP,EACErP,KAAK6X,KAAKxX,GACZiY,EAAQtQ,KAAK3H,IAGbL,KAAK6X,KAAKxX,IAAM,EAChBgY,EAAMrQ,KAAK3H,IAITL,KAAK6X,KAAKxX,WACLL,MAAK6X,KAAKxX,GACjBkY,EAAQvQ,KAAK3H,GAQnB,MAEF,KAAK,SAEH,IAAKmF,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IACrCnF,EAAK8U,EAAI3P,GACLxF,KAAK6X,KAAKxX,WACLL,MAAK6X,KAAKxX,GACjBkY,EAAQvQ,KAAK3H,IAOjBgY,EAAM1S,QACR3F,KAAK6T,SAAS,OAAQ5R,MAAOoW,GAAQtE,GAEnCuE,EAAQ3S,QACV3F,KAAK6T,SAAS,UAAW5R,MAAOqW,GAAUvE,GAExCwE,EAAQ5S,QACV3F,KAAK6T,SAAS,UAAW5R,MAAOsW,GAAUxE,KAMhDjT,EAASqS,UAAUI,GAAK1S,EAAQsS,UAAUI,GAC1CzS,EAASqS,UAAUO,IAAM7S,EAAQsS,UAAUO,IAC3C5S,EAASqS,UAAUU,SAAWhT,EAAQsS,UAAUU,SAGhD/S,EAASqS,UAAUM,UAAY3S,EAASqS,UAAUI,GAClDzS,EAASqS,UAAUS,YAAc9S,EAASqS,UAAUO,IAEpD7T,EAAOD,QAAUkB,GAIb,SAASjB,GAeb,QAASkB,GAAM0N,GAEbzO,KAAKwY,MAAQ,KACbxY,KAAK0M,IAAM+L,IAGXzY,KAAKqT,UACLrT,KAAK0Y,SAAW,KAChB1Y,KAAK2Y,UAAY,KAEjB3Y,KAAKkT,WAAWzE,GAgBlB1N,EAAMoS,UAAUD,WAAa,SAAUzE,GACjCA,GAAoC,mBAAlBA,GAAQ+J,QAC5BxY,KAAKwY,MAAQ/J,EAAQ+J,OAEnB/J,GAAkC,mBAAhBA,GAAQ/B,MAC5B1M,KAAK0M,IAAM+B,EAAQ/B,KAGrB1M,KAAK4Y,kBAsBP7X,EAAMuE,OAAS,SAAUrB,EAAQwK,GAC/B,GAAI2E,GAAQ,GAAIrS,GAAM0N,EAEtB,IAAqBjI,SAAjBvC,EAAO4U,MACT,KAAM,IAAIjV,OAAM,6CAElBK,GAAO4U,MAAQ,WACbzF,EAAMyF,QAGR,IAAIC,KACF7C,KAAM,QACN8C,SAAUvS,QAGZ,IAAIiI,GAAWA,EAAQjE,QACrB,IAAK,GAAIhF,GAAI,EAAGA,EAAIiJ,EAAQjE,QAAQ7E,OAAQH,IAAK,CAC/C,GAAIyQ,GAAOxH,EAAQjE,QAAQhF,EAC3BsT,GAAQ9Q,MACNiO,KAAMA,EACN8C,SAAU9U,EAAOgS,KAEnB7C,EAAM5I,QAAQvG,EAAQgS,GAS1B,MALA7C,GAAMuF,WACJ1U,OAAQA,EACR6U,QAASA,GAGJ1F,GAOTrS,EAAMoS,UAAUG,QAAU,WAGxB,GAFAtT,KAAK6Y,QAED7Y,KAAK2Y,UAAW,CAGlB,IAAK,GAFD1U,GAASjE,KAAK2Y,UAAU1U,OACxB6U,EAAU9Y,KAAK2Y,UAAUG,QACpBtT,EAAI,EAAGA,EAAIsT,EAAQnT,OAAQH,IAAK,CACvC,GAAIwT,GAASF,EAAQtT,EACjBwT,GAAOD,SACT9U,EAAO+U,EAAO/C,MAAQ+C,EAAOD,eAGtB9U,GAAO+U,EAAO/C,MAGzBjW,KAAK2Y,UAAY,OASrB5X,EAAMoS,UAAU3I,QAAU,SAASvG,EAAQ+U,GACzC,GAAI7E,GAAKnU,KACL+Y,EAAW9U,EAAO+U,EACtB,KAAKD,EACH,KAAM,IAAInV,OAAM,UAAYoV,EAAS,aAGvC/U,GAAO+U,GAAU,WAGf,IAAK,GADDC,MACKzT,EAAI,EAAGA,EAAIE,UAAUC,OAAQH,IACpCyT,EAAKzT,GAAKE,UAAUF,EAItB2O,GAAGf,OACD6F,KAAMA,EACNC,GAAIH,EACJI,QAASnZ,SASfe,EAAMoS,UAAUC,MAAQ,SAASgG,GAE7BpZ,KAAKqT,OAAOrL,KADO,kBAAVoR,IACSF,GAAIE,GAGLA,GAGnBpZ,KAAK4Y,kBAOP7X,EAAMoS,UAAUyF,eAAiB,WAQ/B,GANI5Y,KAAKqT,OAAO1N,OAAS3F,KAAK0M,KAC5B1M,KAAK6Y,QAIPQ,aAAarZ,KAAK0Y,UACd1Y,KAAKoT,MAAMzN,OAAS,GAA2B,gBAAf3F,MAAKwY,MAAoB,CAC3D,GAAIrE,GAAKnU,IACTA,MAAK0Y,SAAWY,WAAW,WACzBnF,EAAG0E,SACF7Y,KAAKwY,SAOZzX,EAAMoS,UAAU0F,MAAQ,WACtB,KAAO7Y,KAAKqT,OAAO1N,OAAS,GAAG,CAC7B,GAAIyT,GAAQpZ,KAAKqT,OAAO/B,OACxB8H,GAAMF,GAAGnB,MAAMqB,EAAMD,SAAWC,EAAMF,GAAIE,EAAMH,YAIpDpZ,EAAOD,QAAUmB,GAKb,SAASlB,EAAQD,EAASM,GAwB9B,QAASc,GAAQuY,EAAW7G,EAAMjE,GAChC,KAAMzO,eAAgBgB,IACpB,KAAM,IAAIwY,aAAY,mDAIxBxZ,MAAKyZ,iBAAmBF,EACxBvZ,KAAKuS,MAAQ,QACbvS,KAAKwS,OAAS,QACdxS,KAAK0Z,OAAS,GACd1Z,KAAK2Z,eAAiB,MACtB3Z,KAAK4Z,eAAiB,MAEtB5Z,KAAK6Z,OAAS,IACd7Z,KAAK8Z,OAAS,IACd9Z,KAAK+Z,OAAS,GAEd,IAAIC,GAAc,SAASnO,GAAK,MAAOA,GACvC7L,MAAKia,YAAcD,EACnBha,KAAKka,YAAcF,EACnBha,KAAKma,YAAcH,EAEnBha,KAAKoa,YAAc,OACnBpa,KAAKqa,YAAc,QAEnBra,KAAKiN,MAAQjM,EAAQsZ,MAAMC,IAC3Bva,KAAKwa,iBAAkB,EACvBxa,KAAKya,UAAW,EAChBza,KAAK0a,iBAAkB,EACvB1a,KAAK2a,YAAa,EAClB3a,KAAK4a,gBAAiB,EACtB5a,KAAK6a,aAAc,EACnB7a,KAAK8a,cAAgB,GAErB9a,KAAK+a,kBAAoB,IACzB/a,KAAKgb,kBAAmB,EAExBhb,KAAKib,OAAS,GAAI/Z,GAClBlB,KAAKkb,IAAM,GAAI7Z,GAAQ,EAAG,EAAG,IAE7BrB,KAAKuX,UAAY,KACjBvX,KAAKmb,WAAa,KAGlBnb,KAAKob,KAAO5U,OACZxG,KAAKqb,KAAO7U,OACZxG,KAAKsb,KAAO9U,OACZxG,KAAKub,SAAW/U,OAChBxG,KAAKwb,UAAYhV,OAEjBxG,KAAKyb,KAAO,EACZzb,KAAK0b,MAAQlV,OACbxG,KAAK2b,KAAO,EACZ3b,KAAK4b,KAAO,EACZ5b,KAAK6b,MAAQrV,OACbxG,KAAK8b,KAAO,EACZ9b,KAAK+b,KAAO,EACZ/b,KAAKgc,MAAQxV,OACbxG,KAAKic,KAAO,EACZjc,KAAKkc,SAAW,EAChBlc,KAAKmc,SAAW,EAChBnc,KAAKoc,UAAY,EACjBpc,KAAKqc,UAAY,EAIjBrc,KAAKsc,UAAY,UACjBtc,KAAKuc,UAAY,UACjBvc,KAAKwc,SAAW,UAChBxc,KAAKyc,eAAiB,UAGtBzc,KAAKqO,SAGLrO,KAAKkT,WAAWzE,GAGZiE,GACF1S,KAAKgY,QAAQtF,GAknEjB,QAASgK,GAAWpT,GAClB,MAAI,WAAaA,GAAcA,EAAMqT,QAC9BrT,EAAMsT,cAAc,IAAMtT,EAAMsT,cAAc,GAAGD,SAAW,EAQrE,QAASE,GAAWvT,GAClB,MAAI,WAAaA,GAAcA,EAAMwT,QAC9BxT,EAAMsT,cAAc,IAAMtT,EAAMsT,cAAc,GAAGE,SAAW,EAnuErE,GAAIC,GAAU7c,EAAoB,IAC9BW,EAAUX,EAAoB,GAC9BY,EAAWZ,EAAoB,GAC/BS,EAAOT,EAAoB,GAC3BmB,EAAUnB,EAAoB,IAC9BkB,EAAUlB,EAAoB,GAC9BgB,EAAShB,EAAoB,GAC7BiB,EAASjB,EAAoB,GAC7BoB,EAASpB,EAAoB,IAC7BqB,EAAarB,EAAoB,GAiGrC6c,GAAQ/b,EAAQmS,WAKhBnS,EAAQmS,UAAU6J,UAAY,WAC5Bhd,KAAKid,MAAQ,GAAI5b,GAAQ,GAAKrB,KAAK2b,KAAO3b,KAAKyb,MAC7C,GAAKzb,KAAK8b,KAAO9b,KAAK4b,MACtB,GAAK5b,KAAKic,KAAOjc,KAAK+b,OAGpB/b,KAAK0a,kBACH1a,KAAKid,MAAMlL,EAAI/R,KAAKid,MAAMjL,EAE5BhS,KAAKid,MAAMjL,EAAIhS,KAAKid,MAAMlL,EAI1B/R,KAAKid,MAAMlL,EAAI/R,KAAKid,MAAMjL,GAK9BhS,KAAKid,MAAMC,GAAKld,KAAK8a,cAIrB9a,KAAKid,MAAM5V,MAAQ,GAAKrH,KAAKmc,SAAWnc,KAAKkc,SAG7C,IAAIiB,IAAWnd,KAAK2b,KAAO3b,KAAKyb,MAAQ,EAAIzb,KAAKid,MAAMlL,EACnDqL,GAAWpd,KAAK8b,KAAO9b,KAAK4b,MAAQ,EAAI5b,KAAKid,MAAMjL,EACnDqL,GAAWrd,KAAKic,KAAOjc,KAAK+b,MAAQ,EAAI/b,KAAKid,MAAMC,CACvDld,MAAKib,OAAOqC,eAAeH,EAASC,EAASC,IAU/Crc,EAAQmS,UAAUoK,eAAiB,SAASC,GAC1C,GAAIC,GAAczd,KAAK0d,2BAA2BF,EAClD,OAAOxd,MAAK2d,4BAA4BF,IAW1Czc,EAAQmS,UAAUuK,2BAA6B,SAASF,GACtD,GAAII,GAAKJ,EAAQzL,EAAI/R,KAAKid,MAAMlL,EAC9B8L,EAAKL,EAAQxL,EAAIhS,KAAKid,MAAMjL,EAC5B8L,EAAKN,EAAQN,EAAIld,KAAKid,MAAMC,EAE5Ba,EAAK/d,KAAKib,OAAO+C,oBAAoBjM,EACrCkM,EAAKje,KAAKib,OAAO+C,oBAAoBhM,EACrCkM,EAAKle,KAAKib,OAAO+C,oBAAoBd,EAGrCiB,EAAQjZ,KAAKkZ,IAAIpe,KAAKib,OAAOoD,oBAAoBtM,GACjDuM,EAAQpZ,KAAKqZ,IAAIve,KAAKib,OAAOoD,oBAAoBtM,GACjDyM,EAAQtZ,KAAKkZ,IAAIpe,KAAKib,OAAOoD,oBAAoBrM,GACjDyM,EAAQvZ,KAAKqZ,IAAIve,KAAKib,OAAOoD,oBAAoBrM,GACjD0M,EAAQxZ,KAAKkZ,IAAIpe,KAAKib,OAAOoD,oBAAoBnB,GACjDyB,EAAQzZ,KAAKqZ,IAAIve,KAAKib,OAAOoD,oBAAoBnB,GAGjD0B,EAAKH,GAASC,GAASb,EAAKI,GAAMU,GAASf,EAAKG,IAAOS,GAASV,EAAKI,GACrEW,EAAKV,GAASM,GAASX,EAAKI,GAAMM,GAASE,GAASb,EAAKI,GAAMU,GAASf,EAAKG,KAAQO,GAASK,GAASd,EAAKI,GAAMS,GAASd,EAAGG,IAC9He,EAAKR,GAASG,GAASX,EAAKI,GAAMM,GAASE,GAASb,EAAKI,GAAMU,GAASf,EAAKG,KAAQI,GAASQ,GAASd,EAAKI,GAAMS,GAASd,EAAGG,GAEhI,OAAO,IAAI1c,GAAQud,EAAIC,EAAIC,IAU7B9d,EAAQmS,UAAUwK,4BAA8B,SAASF,GACvD,GAQIsB,GACAC,EATAC,EAAKjf,KAAKkb,IAAInJ,EAChBmN,EAAKlf,KAAKkb,IAAIlJ,EACdmN,EAAKnf,KAAKkb,IAAIgC,EACd0B,EAAKnB,EAAY1L,EACjB8M,EAAKpB,EAAYzL,EACjB8M,EAAKrB,EAAYP,CAgBnB,OAXIld,MAAKwa,iBACPuE,GAAMH,EAAKK,IAAOE,EAAKL,GACvBE,GAAMH,EAAKK,IAAOC,EAAKL,KAGvBC,EAAKH,IAAOO,EAAKnf,KAAKib,OAAOmE,gBAC7BJ,EAAKH,IAAOM,EAAKnf,KAAKib,OAAOmE,iBAKxB,GAAIhe,GACTpB,KAAKqf,QAAUN,EAAK/e,KAAKsf,MAAMC,OAAOC,YACtCxf,KAAKyf,QAAUT,EAAKhf,KAAKsf,MAAMC,OAAOC,cAO1Cxe,EAAQmS,UAAUuM,oBAAsB,SAASC,GAC/C,GAAIC,GAAO,QACPC,EAAS,OACTC,EAAc,CAElB,IAAgC,gBAAtB,GACRF,EAAOD,EACPE,EAAS,OACTC,EAAc,MAEX,IAAgC,gBAAtB,GACgBtZ,SAAzBmZ,EAAgBC,OAAuBA,EAAOD,EAAgBC,MACnCpZ,SAA3BmZ,EAAgBE,SAAyBA,EAASF,EAAgBE,QAClCrZ,SAAhCmZ,EAAgBG,cAA2BA,EAAcH,EAAgBG,iBAE1E,IAAyBtZ,SAApBmZ,EAIR,KAAM,qCAGR3f,MAAKsf,MAAMrS,MAAM0S,gBAAkBC,EACnC5f,KAAKsf,MAAMrS,MAAM8S,YAAcF,EAC/B7f,KAAKsf,MAAMrS,MAAM+S,YAAcF,EAAc,KAC7C9f,KAAKsf,MAAMrS,MAAMgT,YAAc,SAKjCjf,EAAQsZ,OACN4F,IAAK,EACLC,SAAU,EACVC,QAAS,EACT7F,IAAM,EACN8F,QAAU,EACVC,SAAU,EACVC,QAAS,EACTC,KAAO,EACPC,KAAM,EACNC,QAAU,GASZ1f,EAAQmS,UAAUwN,gBAAkB,SAASC,GAC3C,OAAQA,GACN,IAAK,MAAW,MAAO5f,GAAQsZ,MAAMC,GACrC,KAAK,WAAa,MAAOvZ,GAAQsZ,MAAM+F,OACvC,KAAK,YAAe,MAAOrf,GAAQsZ,MAAMgG,QACzC,KAAK,WAAa,MAAOtf,GAAQsZ,MAAMiG,OACvC,KAAK,OAAW,MAAOvf,GAAQsZ,MAAMmG,IACrC,KAAK,OAAW,MAAOzf,GAAQsZ,MAAMkG,IACrC,KAAK,UAAa,MAAOxf,GAAQsZ,MAAMoG,OACvC,KAAK,MAAW,MAAO1f,GAAQsZ,MAAM4F,GACrC,KAAK,YAAe,MAAOlf,GAAQsZ,MAAM6F,QACzC,KAAK,WAAa,MAAOnf,GAAQsZ,MAAM8F,QAGzC,MAAO,IAQTpf,EAAQmS,UAAU0N,wBAA0B,SAASnO,GACnD,GAAI1S,KAAKiN,QAAUjM,EAAQsZ,MAAMC,KAC/Bva,KAAKiN,QAAUjM,EAAQsZ,MAAM+F,SAC7BrgB,KAAKiN,QAAUjM,EAAQsZ,MAAMmG,MAC7BzgB,KAAKiN,QAAUjM,EAAQsZ,MAAMkG,MAC7BxgB,KAAKiN,QAAUjM,EAAQsZ,MAAMoG,SAC7B1gB,KAAKiN,QAAUjM,EAAQsZ,MAAM4F,IAE7BlgB,KAAKob,KAAO,EACZpb,KAAKqb,KAAO,EACZrb,KAAKsb,KAAO,EACZtb,KAAKub,SAAW/U,OAEZkM,EAAK8E,qBAAuB,IAC9BxX,KAAKwb,UAAY,OAGhB,CAAA,GAAIxb,KAAKiN,QAAUjM,EAAQsZ,MAAMgG,UACpCtgB,KAAKiN,QAAUjM,EAAQsZ,MAAMiG,SAC7BvgB,KAAKiN,QAAUjM,EAAQsZ,MAAM6F,UAC7BngB,KAAKiN,QAAUjM,EAAQsZ,MAAM8F,QAY7B,KAAM,kBAAoBpgB,KAAKiN,MAAQ,GAVvCjN,MAAKob,KAAO,EACZpb,KAAKqb,KAAO,EACZrb,KAAKsb,KAAO,EACZtb,KAAKub,SAAW,EAEZ7I,EAAK8E,qBAAuB,IAC9BxX,KAAKwb,UAAY,KAQvBxa,EAAQmS,UAAUsB,gBAAkB,SAAS/B,GAC3C,MAAOA,GAAK/M,QAId3E,EAAQmS,UAAUqE,mBAAqB,SAAS9E,GAC9C,GAAIoO,GAAU,CACd,KAAK,GAAIC,KAAUrO,GAAK,GAClBA,EAAK,GAAG5M,eAAeib,IACzBD,GAGJ,OAAOA,IAIT9f,EAAQmS,UAAU6N,kBAAoB,SAAStO,EAAMqO,GAEnD,IAAK,GADDE,MACKzb,EAAI,EAAGA,EAAIkN,EAAK/M,OAAQH,IACgB,IAA3Cyb,EAAeta,QAAQ+L,EAAKlN,GAAGub,KACjCE,EAAejZ,KAAK0K,EAAKlN,GAAGub,GAGhC,OAAOE,IAITjgB,EAAQmS,UAAU+N,eAAiB,SAASxO,EAAKqO,GAE/C,IAAK,GADDI,IAAUrV,IAAI4G,EAAK,GAAGqO,GAAQrU,IAAIgG,EAAK,GAAGqO,IACrCvb,EAAI,EAAGA,EAAIkN,EAAK/M,OAAQH,IAC3B2b,EAAOrV,IAAM4G,EAAKlN,GAAGub,KAAWI,EAAOrV,IAAM4G,EAAKlN,GAAGub,IACrDI,EAAOzU,IAAMgG,EAAKlN,GAAGub,KAAWI,EAAOzU,IAAMgG,EAAKlN,GAAGub,GAE3D,OAAOI,IASTngB,EAAQmS,UAAUiO,gBAAkB,SAAUC,GAC5C,GAAIlN,GAAKnU,IAOT,IAJIA,KAAKoY,SACPpY,KAAKoY,QAAQ1E,IAAI,IAAK1T,KAAKshB,WAGb9a,SAAZ6a,EAAJ,CAGIpb,MAAMC,QAAQmb,KAChBA,EAAU,GAAIxgB,GAAQwgB,GAGxB,IAAI3O,EACJ,MAAI2O,YAAmBxgB,IAAWwgB,YAAmBvgB,IAInD,KAAM,IAAI8C,OAAM,uCAGlB,IANE8O,EAAO2O,EAAQnM,MAME,GAAfxC,EAAK/M,OAAT,CAGA3F,KAAKoY,QAAUiJ,EACfrhB,KAAKuX,UAAY7E,EAGjB1S,KAAKshB,UAAY,WACfnN,EAAG6D,QAAQ7D,EAAGiE,UAEhBpY,KAAKoY,QAAQ7E,GAAG,IAAKvT,KAAKshB,WAS1BthB,KAAKob,KAAO,IACZpb,KAAKqb,KAAO,IACZrb,KAAKsb,KAAO,IACZtb,KAAKub,SAAW,QAChBvb,KAAKwb,UAAY,SAKb9I,EAAK,GAAG5M,eAAe,WACDU,SAApBxG,KAAKuhB,aACPvhB,KAAKuhB,WAAa,GAAIpgB,GAAOkgB,EAASrhB,KAAKwb,UAAWxb,MACtDA,KAAKuhB,WAAWC,kBAAkB,WAAYrN,EAAGsN,WAKrD,IAAIC,GAAW1hB,KAAKiN,OAASjM,EAAQsZ,MAAM4F,KACzClgB,KAAKiN,OAASjM,EAAQsZ,MAAM6F,UAC5BngB,KAAKiN,OAASjM,EAAQsZ,MAAM8F,OAG9B,IAAIsB,EAAU,CACZ,GAA8Blb,SAA1BxG,KAAK2hB,iBACP3hB,KAAKoc,UAAYpc,KAAK2hB,qBAEnB,CACH,GAAIC,GAAQ5hB,KAAKghB,kBAAkBtO,EAAK1S,KAAKob,KAC7Cpb,MAAKoc,UAAawF,EAAM,GAAKA,EAAM,IAAO,EAG5C,GAA8Bpb,SAA1BxG,KAAK6hB,iBACP7hB,KAAKqc,UAAYrc,KAAK6hB,qBAEnB,CACH,GAAIC,GAAQ9hB,KAAKghB,kBAAkBtO,EAAK1S,KAAKqb,KAC7Crb,MAAKqc,UAAayF,EAAM,GAAKA,EAAM,IAAO,GAK9C,GAAIC,GAAS/hB,KAAKkhB,eAAexO,EAAK1S,KAAKob,KACvCsG,KACFK,EAAOjW,KAAO9L,KAAKoc,UAAY,EAC/B2F,EAAOrV,KAAO1M,KAAKoc,UAAY,GAEjCpc,KAAKyb,KAA6BjV,SAArBxG,KAAKgiB,YAA6BhiB,KAAKgiB,YAAcD,EAAOjW,IACzE9L,KAAK2b,KAA6BnV,SAArBxG,KAAKiiB,YAA6BjiB,KAAKiiB,YAAcF,EAAOrV,IACrE1M,KAAK2b,MAAQ3b,KAAKyb,OAAMzb,KAAK2b,KAAO3b,KAAKyb,KAAO,GACpDzb,KAAK0b,MAA+BlV,SAAtBxG,KAAKkiB,aAA8BliB,KAAKkiB,cAAgBliB,KAAK2b,KAAK3b,KAAKyb,MAAM,CAE3F,IAAI0G,GAASniB,KAAKkhB,eAAexO,EAAK1S,KAAKqb,KACvCqG,KACFS,EAAOrW,KAAO9L,KAAKqc,UAAY,EAC/B8F,EAAOzV,KAAO1M,KAAKqc,UAAY,GAEjCrc,KAAK4b,KAA6BpV,SAArBxG,KAAKoiB,YAA6BpiB,KAAKoiB,YAAcD,EAAOrW,IACzE9L,KAAK8b,KAA6BtV,SAArBxG,KAAKqiB,YAA6BriB,KAAKqiB,YAAcF,EAAOzV,IACrE1M,KAAK8b,MAAQ9b,KAAK4b,OAAM5b,KAAK8b,KAAO9b,KAAK4b,KAAO,GACpD5b,KAAK6b,MAA+BrV,SAAtBxG,KAAKsiB,aAA8BtiB,KAAKsiB,cAAgBtiB,KAAK8b,KAAK9b,KAAK4b,MAAM,CAE3F;GAAI2G,GAASviB,KAAKkhB,eAAexO,EAAK1S,KAAKsb,KAM3C,IALAtb,KAAK+b,KAA6BvV,SAArBxG,KAAKwiB,YAA6BxiB,KAAKwiB,YAAcD,EAAOzW,IACzE9L,KAAKic,KAA6BzV,SAArBxG,KAAKyiB,YAA6BziB,KAAKyiB,YAAcF,EAAO7V,IACrE1M,KAAKic,MAAQjc,KAAK+b,OAAM/b,KAAKic,KAAOjc,KAAK+b,KAAO,GACpD/b,KAAKgc,MAA+BxV,SAAtBxG,KAAK0iB,aAA8B1iB,KAAK0iB,cAAgB1iB,KAAKic,KAAKjc,KAAK+b,MAAM,EAErEvV,SAAlBxG,KAAKub,SAAwB,CAC/B,GAAIoH,GAAa3iB,KAAKkhB,eAAexO,EAAK1S,KAAKub,SAC/Cvb,MAAKkc,SAAqC1V,SAAzBxG,KAAK4iB,gBAAiC5iB,KAAK4iB,gBAAkBD,EAAW7W,IACzF9L,KAAKmc,SAAqC3V,SAAzBxG,KAAK6iB,gBAAiC7iB,KAAK6iB,gBAAkBF,EAAWjW,IACrF1M,KAAKmc,UAAYnc,KAAKkc,WAAUlc,KAAKmc,SAAWnc,KAAKkc,SAAW,GAItElc,KAAKgd,eAUPhc,EAAQmS,UAAU2P,eAAiB,SAAUpQ,GAE3C,GAAIX,GAAGC,EAAGxM,EAAG0X,EAAG6F,EAAK7Q,EAEjBiJ,IAEJ,IAAInb,KAAKiN,QAAUjM,EAAQsZ,MAAMkG,MAC/BxgB,KAAKiN,QAAUjM,EAAQsZ,MAAMoG,QAAS,CAKtC,GAAIkB,MACAE,IACJ,KAAKtc,EAAI,EAAGA,EAAIxF,KAAKyU,gBAAgB/B,GAAOlN,IAC1CuM,EAAIW,EAAKlN,GAAGxF,KAAKob,OAAS,EAC1BpJ,EAAIU,EAAKlN,GAAGxF,KAAKqb,OAAS,EAED,KAArBuG,EAAMjb,QAAQoL,IAChB6P,EAAM5Z,KAAK+J,GAEY,KAArB+P,EAAMnb,QAAQqL,IAChB8P,EAAM9Z,KAAKgK,EAIf,IAAIgR,GAAa,SAAUzd,EAAGa,GAC5B,MAAOb,GAAIa,EAEbwb,GAAM1L,KAAK8M,GACXlB,EAAM5L,KAAK8M,EAGX,IAAIC,KACJ,KAAKzd,EAAI,EAAGA,EAAIkN,EAAK/M,OAAQH,IAAK,CAChCuM,EAAIW,EAAKlN,GAAGxF,KAAKob,OAAS,EAC1BpJ,EAAIU,EAAKlN,GAAGxF,KAAKqb,OAAS,EAC1B6B,EAAIxK,EAAKlN,GAAGxF,KAAKsb,OAAS,CAE1B,IAAI4H,GAAStB,EAAMjb,QAAQoL,GACvBoR,EAASrB,EAAMnb,QAAQqL,EAEAxL,UAAvByc,EAAWC,KACbD,EAAWC,MAGb,IAAI1F,GAAU,GAAInc,EAClBmc,GAAQzL,EAAIA,EACZyL,EAAQxL,EAAIA,EACZwL,EAAQN,EAAIA,EAEZ6F,KACAA,EAAI7Q,MAAQsL,EACZuF,EAAIK,MAAQ5c,OACZuc,EAAIM,OAAS7c,OACbuc,EAAIO,OAAS,GAAIjiB,GAAQ0Q,EAAGC,EAAGhS,KAAK+b,MAEpCkH,EAAWC,GAAQC,GAAUJ,EAE7B5H,EAAWnT,KAAK+a,GAIlB,IAAKhR,EAAI,EAAGA,EAAIkR,EAAWtd,OAAQoM,IACjC,IAAKC,EAAI,EAAGA,EAAIiR,EAAWlR,GAAGpM,OAAQqM,IAChCiR,EAAWlR,GAAGC,KAChBiR,EAAWlR,GAAGC,GAAGuR,WAAcxR,EAAIkR,EAAWtd,OAAO,EAAKsd,EAAWlR,EAAE,GAAGC,GAAKxL,OAC/Eyc,EAAWlR,GAAGC,GAAGwR,SAAcxR,EAAIiR,EAAWlR,GAAGpM,OAAO,EAAKsd,EAAWlR,GAAGC,EAAE,GAAKxL,OAClFyc,EAAWlR,GAAGC,GAAGyR,WACd1R,EAAIkR,EAAWtd,OAAO,GAAKqM,EAAIiR,EAAWlR,GAAGpM,OAAO,EACnDsd,EAAWlR,EAAE,GAAGC,EAAE,GAClBxL,YAOV,KAAKhB,EAAI,EAAGA,EAAIkN,EAAK/M,OAAQH,IAC3B0M,EAAQ,GAAI7Q,GACZ6Q,EAAMH,EAAIW,EAAKlN,GAAGxF,KAAKob,OAAS,EAChClJ,EAAMF,EAAIU,EAAKlN,GAAGxF,KAAKqb,OAAS,EAChCnJ,EAAMgL,EAAIxK,EAAKlN,GAAGxF,KAAKsb,OAAS,EAEV9U,SAAlBxG,KAAKub,WACPrJ,EAAM7K,MAAQqL,EAAKlN,GAAGxF,KAAKub,WAAa,GAG1CwH,KACAA,EAAI7Q,MAAQA,EACZ6Q,EAAIO,OAAS,GAAIjiB,GAAQ6Q,EAAMH,EAAGG,EAAMF,EAAGhS,KAAK+b,MAChDgH,EAAIK,MAAQ5c,OACZuc,EAAIM,OAAS7c,OAEb2U,EAAWnT,KAAK+a,EAIpB,OAAO5H,IASTna,EAAQmS,UAAU9E,OAAS,WAEzB,KAAOrO,KAAKyZ,iBAAiBiK,iBAC3B1jB,KAAKyZ,iBAAiBtI,YAAYnR,KAAKyZ,iBAAiBkK,WAG1D3jB,MAAKsf,MAAQ/N,SAASM,cAAc,OACpC7R,KAAKsf,MAAMrS,MAAM2W,SAAW,WAC5B5jB,KAAKsf,MAAMrS,MAAM4W,SAAW,SAG5B7jB,KAAKsf,MAAMC,OAAShO,SAASM,cAAe,UAC5C7R,KAAKsf,MAAMC,OAAOtS,MAAM2W,SAAW,WACnC5jB,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAMC,OAGhC,IAAIuE,GAAWvS,SAASM,cAAe,MACvCiS,GAAS7W,MAAM9B,MAAQ,MACvB2Y,EAAS7W,MAAM8W,WAAc,OAC7BD,EAAS7W,MAAM+W,QAAW,OAC1BF,EAASG,UAAa,mDACtBjkB,KAAKsf,MAAMC,OAAO9N,YAAYqS,GAGhC9jB,KAAKsf,MAAM3L,OAASpC,SAASM,cAAe,OAC5C7R,KAAKsf,MAAM3L,OAAO1G,MAAM2W,SAAW,WACnC5jB,KAAKsf,MAAM3L,OAAO1G,MAAMqW,OAAS,MACjCtjB,KAAKsf,MAAM3L,OAAO1G,MAAMxF,KAAO,MAC/BzH,KAAKsf,MAAM3L,OAAO1G,MAAMsF,MAAQ,OAChCvS,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAM3L,OAGlC,IAAIQ,GAAKnU,KACLkkB,EAAc,SAAU5a,GAAQ6K,EAAGgQ,aAAa7a,IAChD8a,EAAe,SAAU9a,GAAQ6K,EAAGkQ,cAAc/a,IAClDgb,EAAe,SAAUhb,GAAQ6K,EAAGoQ,SAASjb,IAC7Ckb,EAAY,SAAUlb,GAAQ6K,EAAGsQ,WAAWnb,GAGhD3I,GAAKgI,iBAAiB3I,KAAKsf,MAAMC,OAAQ,UAAWmF,WACpD/jB,EAAKgI,iBAAiB3I,KAAKsf,MAAMC,OAAQ,YAAa2E,GACtDvjB,EAAKgI,iBAAiB3I,KAAKsf,MAAMC,OAAQ,aAAc6E,GACvDzjB,EAAKgI,iBAAiB3I,KAAKsf,MAAMC,OAAQ,aAAc+E,GACvD3jB,EAAKgI,iBAAiB3I,KAAKsf,MAAMC,OAAQ,YAAaiF,GAGtDxkB,KAAKyZ,iBAAiBhI,YAAYzR,KAAKsf,QAWzCte,EAAQmS,UAAUwR,QAAU,SAASpS,EAAOC,GAC1CxS,KAAKsf,MAAMrS,MAAMsF,MAAQA,EACzBvS,KAAKsf,MAAMrS,MAAMuF,OAASA,EAE1BxS,KAAK4kB,iBAMP5jB,EAAQmS,UAAUyR,cAAgB,WAChC5kB,KAAKsf,MAAMC,OAAOtS,MAAMsF,MAAQ,OAChCvS,KAAKsf,MAAMC,OAAOtS,MAAMuF,OAAS,OAEjCxS,KAAKsf,MAAMC,OAAOhN,MAAQvS,KAAKsf,MAAMC,OAAOC,YAC5Cxf,KAAKsf,MAAMC,OAAO/M,OAASxS,KAAKsf,MAAMC,OAAOsF,aAG7C7kB,KAAKsf,MAAM3L,OAAO1G,MAAMsF,MAASvS,KAAKsf,MAAMC,OAAOC,YAAc,GAAU,MAM7Exe,EAAQmS,UAAU2R,eAAiB,WACjC,IAAK9kB,KAAKsf,MAAM3L,SAAW3T,KAAKsf,MAAM3L,OAAOoR,OAC3C,KAAM,wBAER/kB,MAAKsf,MAAM3L,OAAOoR,OAAOC,QAO3BhkB,EAAQmS,UAAU8R,cAAgB,WAC3BjlB,KAAKsf,MAAM3L,QAAW3T,KAAKsf,MAAM3L,OAAOoR,QAE7C/kB,KAAKsf,MAAM3L,OAAOoR,OAAOG,QAU3BlkB,EAAQmS,UAAUgS,cAAgB,WAG9BnlB,KAAKqf,QAD0D,MAA7Drf,KAAK2Z,eAAeyL,OAAOplB,KAAK2Z,eAAehU,OAAO,GAEtD0f,WAAWrlB,KAAK2Z,gBAAkB,IAChC3Z,KAAKsf,MAAMC,OAAOC,YAGP6F,WAAWrlB,KAAK2Z,gBAK/B3Z,KAAKyf,QAD0D,MAA7Dzf,KAAK4Z,eAAewL,OAAOplB,KAAK4Z,eAAejU,OAAO,GAEtD0f,WAAWrlB,KAAK4Z,gBAAkB,KAC/B5Z,KAAKsf,MAAMC,OAAOsF,aAAe7kB,KAAKsf,MAAM3L,OAAOkR,cAGzCQ,WAAWrlB,KAAK4Z,iBAoBnC5Y,EAAQmS,UAAUmS,kBAAoB,SAASC,GACjC/e,SAAR+e,IAImB/e,SAAnB+e,EAAIC,YAA6Chf,SAAjB+e,EAAIE,UACtCzlB,KAAKib,OAAOyK,eAAeH,EAAIC,WAAYD,EAAIE,UAG5Bjf,SAAjB+e,EAAII,UACN3lB,KAAKib,OAAO2K,aAAaL,EAAII,UAG/B3lB,KAAKyhB,WASPzgB,EAAQmS,UAAU0S,kBAAoB,WACpC,GAAIN,GAAMvlB,KAAKib,OAAO6K,gBAEtB,OADAP,GAAII,SAAW3lB,KAAKib,OAAOmE,eACpBmG,GAMTvkB,EAAQmS,UAAU4S,UAAY,SAASrT,GAErC1S,KAAKohB,gBAAgB1O,EAAM1S,KAAKiN,OAK9BjN,KAAKmb,WAFHnb,KAAKuhB,WAEWvhB,KAAKuhB,WAAWuB,iBAIhB9iB,KAAK8iB,eAAe9iB,KAAKuX,WAI7CvX,KAAKgmB,iBAOPhlB,EAAQmS,UAAU6E,QAAU,SAAUtF,GACpC1S,KAAK+lB,UAAUrT,GACf1S,KAAKyhB,SAGDzhB,KAAKimB,oBAAsBjmB,KAAKuhB,YAClCvhB,KAAK8kB,kBAQT9jB,EAAQmS,UAAUD,WAAa,SAAUzE,GACvC,GAAIyX,GAAiB1f,MAIrB,IAFAxG,KAAKilB,gBAEWze,SAAZiI,EAAuB,CAkBzB,GAhBsBjI,SAAlBiI,EAAQ8D,QAA2BvS,KAAKuS,MAAQ9D,EAAQ8D,OACrC/L,SAAnBiI,EAAQ+D,SAA2BxS,KAAKwS,OAAS/D,EAAQ+D,QAErChM,SAApBiI,EAAQ0O,UAA2Bnd,KAAK2Z,eAAiBlL,EAAQ0O,SAC7C3W,SAApBiI,EAAQ2O,UAA2Bpd,KAAK4Z,eAAiBnL,EAAQ2O,SAEzC5W,SAAxBiI,EAAQ2L,cAA+Bpa,KAAKoa,YAAc3L,EAAQ2L,aAC1C5T,SAAxBiI,EAAQ4L,cAA+Bra,KAAKqa,YAAc5L,EAAQ4L,aAC/C7T,SAAnBiI,EAAQoL,SAA0B7Z,KAAK6Z,OAASpL,EAAQoL,QACrCrT,SAAnBiI,EAAQqL,SAA0B9Z,KAAK8Z,OAASrL,EAAQqL,QACrCtT,SAAnBiI,EAAQsL,SAA0B/Z,KAAK+Z,OAAStL,EAAQsL,QAEhCvT,SAAxBiI,EAAQwL,cAA+Bja,KAAKia,YAAcxL,EAAQwL,aAC1CzT,SAAxBiI,EAAQyL,cAA+Bla,KAAKka,YAAczL,EAAQyL,aAC1C1T,SAAxBiI,EAAQ0L,cAA+Bna,KAAKma,YAAc1L,EAAQ0L,aAEhD3T,SAAlBiI,EAAQxB,MAAqB,CAC/B,GAAIkZ,GAAcnmB,KAAK2gB,gBAAgBlS,EAAQxB,MAC3B,MAAhBkZ,IACFnmB,KAAKiN,MAAQkZ,GAGQ3f,SAArBiI,EAAQgM,WAA6Bza,KAAKya,SAAWhM,EAAQgM,UACjCjU,SAA5BiI,EAAQ+L,kBAAiCxa,KAAKwa,gBAAkB/L,EAAQ+L,iBACjDhU,SAAvBiI,EAAQkM,aAA6B3a,KAAK2a,WAAalM,EAAQkM,YAC3CnU,SAApBiI,EAAQ2X,UAA6BpmB,KAAK6a,YAAcpM,EAAQ2X,SAC9B5f,SAAlCiI,EAAQ4X,wBAAqCrmB,KAAKqmB,sBAAwB5X,EAAQ4X,uBACtD7f,SAA5BiI,EAAQiM,kBAAiC1a,KAAK0a,gBAAkBjM,EAAQiM,iBAC9ClU,SAA1BiI,EAAQqM,gBAA+B9a,KAAK8a,cAAgBrM,EAAQqM,eAEtCtU,SAA9BiI,EAAQsM,oBAAiC/a,KAAK+a,kBAAoBtM,EAAQsM,mBAC7CvU,SAA7BiI,EAAQuM,mBAAiChb,KAAKgb,iBAAmBvM,EAAQuM,kBAC1CxU,SAA/BiI,EAAQwX,qBAAiCjmB,KAAKimB,mBAAqBxX,EAAQwX,oBAErDzf,SAAtBiI,EAAQ2N,YAAyBpc,KAAK2hB,iBAAmBlT,EAAQ2N,WAC3C5V,SAAtBiI,EAAQ4N,YAAyBrc,KAAK6hB,iBAAmBpT,EAAQ4N,WAEhD7V,SAAjBiI,EAAQgN,OAAoBzb,KAAKgiB,YAAcvT,EAAQgN,MACrCjV,SAAlBiI,EAAQiN,QAAqB1b,KAAKkiB,aAAezT,EAAQiN,OACxClV,SAAjBiI,EAAQkN,OAAoB3b,KAAKiiB,YAAcxT,EAAQkN,MACtCnV,SAAjBiI,EAAQmN,OAAoB5b,KAAKoiB,YAAc3T,EAAQmN,MACrCpV,SAAlBiI,EAAQoN,QAAqB7b,KAAKsiB,aAAe7T,EAAQoN,OACxCrV,SAAjBiI,EAAQqN,OAAoB9b,KAAKqiB,YAAc5T,EAAQqN,MACtCtV,SAAjBiI,EAAQsN,OAAoB/b,KAAKwiB,YAAc/T,EAAQsN,MACrCvV,SAAlBiI,EAAQuN,QAAqBhc,KAAK0iB,aAAejU,EAAQuN,OACxCxV,SAAjBiI,EAAQwN,OAAoBjc,KAAKyiB,YAAchU,EAAQwN,MAClCzV,SAArBiI,EAAQyN,WAAwBlc,KAAK4iB,gBAAkBnU,EAAQyN,UAC1C1V,SAArBiI,EAAQ0N,WAAwBnc,KAAK6iB,gBAAkBpU,EAAQ0N,UAEpC3V,SAA3BiI,EAAQyX,iBAA8BA,EAAiBzX,EAAQyX,gBAE5C1f,SAAnB0f,GACFlmB,KAAKib,OAAOyK,eAAeQ,EAAeV,WAAYU,EAAeT,UACrEzlB,KAAKib,OAAO2K,aAAaM,EAAeP,YAGxC3lB,KAAKib,OAAOyK,eAAe,EAAK,IAChC1lB,KAAKib,OAAO2K,aAAa,MAI7B5lB,KAAK0f,oBAAoBjR,GAAWA,EAAQkR,iBAE5C3f,KAAK2kB,QAAQ3kB,KAAKuS,MAAOvS,KAAKwS,QAG1BxS,KAAKuX,WACPvX,KAAKgY,QAAQhY,KAAKuX,WAIhBvX,KAAKimB,oBAAsBjmB,KAAKuhB,YAClCvhB,KAAK8kB,kBAOT9jB,EAAQmS,UAAUsO,OAAS,WACzB,GAAwBjb,SAApBxG,KAAKmb,WACP,KAAM,mCAGRnb,MAAK4kB,gBACL5kB,KAAKmlB,gBACLnlB,KAAKsmB,gBACLtmB,KAAKumB,eACLvmB,KAAKwmB,cAEDxmB,KAAKiN,QAAUjM,EAAQsZ,MAAMkG,MAC/BxgB,KAAKiN,QAAUjM,EAAQsZ,MAAMoG,QAC7B1gB,KAAKymB,kBAEEzmB,KAAKiN,QAAUjM,EAAQsZ,MAAMmG,KACpCzgB,KAAK0mB,kBAEE1mB,KAAKiN,QAAUjM,EAAQsZ,MAAM4F,KACpClgB,KAAKiN,QAAUjM,EAAQsZ,MAAM6F,UAC7BngB,KAAKiN,QAAUjM,EAAQsZ,MAAM8F,QAC7BpgB,KAAK2mB,iBAIL3mB,KAAK4mB,iBAGP5mB,KAAK6mB,cACL7mB,KAAK8mB,iBAMP9lB,EAAQmS,UAAUoT,aAAe,WAC/B,GAAIhH,GAASvf,KAAKsf,MAAMC,OACpBwH,EAAMxH,EAAOyH,WAAW,KAE5BD,GAAIE,UAAU,EAAG,EAAG1H,EAAOhN,MAAOgN,EAAO/M,SAO3CxR,EAAQmS,UAAU2T,cAAgB,WAChC,GAAI9U,EAEJ,IAAIhS,KAAKiN,QAAUjM,EAAQsZ,MAAMgG,UAC/BtgB,KAAKiN,QAAUjM,EAAQsZ,MAAMiG,QAAS,CAEtC,GAEI2G,GAAUC,EAFVC,EAAmC,IAAzBpnB,KAAKsf,MAAME,WAGrBxf,MAAKiN,QAAUjM,EAAQsZ,MAAMiG,SAC/B2G,EAAWE,EAAU,EACrBD,EAAWC,EAAU,EAAc,EAAVA,IAGzBF,EAAW,GACXC,EAAW,GAGb,IAAI3U,GAAStN,KAAKwH,IAA8B,IAA1B1M,KAAKsf,MAAMuF,aAAqB,KAClDld,EAAM3H,KAAK0Z,OACX2N,EAAQrnB,KAAKsf,MAAME,YAAcxf,KAAK0Z,OACtCjS,EAAO4f,EAAQF,EACf7D,EAAS3b,EAAM6K,EAGrB,GAAI+M,GAASvf,KAAKsf,MAAMC,OACpBwH,EAAMxH,EAAOyH,WAAW,KAI5B,IAHAD,EAAIO,UAAY,EAChBP,EAAIQ,KAAO,aAEPvnB,KAAKiN,QAAUjM,EAAQsZ,MAAMgG,SAAU,CAEzC,GAAIkH,GAAO,EACPC,EAAOjV,CACX,KAAKR,EAAIwV,EAAUC,EAAJzV,EAAUA,IAAK,CAC5B,GAAIpE,IAAKoE,EAAIwV,IAASC,EAAOD,GAGzB5a,EAAU,IAAJgB,EACNzC,EAAQnL,KAAK0nB,SAAS9a,EAAK,EAAG,EAElCma,GAAIY,YAAcxc,EAClB4b,EAAIa,YACJb,EAAIc,OAAOpgB,EAAME,EAAMqK,GACvB+U,EAAIe,OAAOT,EAAO1f,EAAMqK,GACxB+U,EAAIlH,SAGNkH,EAAIY,YAAe3nB,KAAKsc,UACxByK,EAAIgB,WAAWtgB,EAAME,EAAKwf,EAAU3U,GAiBtC,GAdIxS,KAAKiN,QAAUjM,EAAQsZ,MAAMiG,UAE/BwG,EAAIY,YAAe3nB,KAAKsc,UACxByK,EAAIiB,UAAahoB,KAAKwc,SACtBuK,EAAIa,YACJb,EAAIc,OAAOpgB,EAAME,GACjBof,EAAIe,OAAOT,EAAO1f,GAClBof,EAAIe,OAAOT,EAAQF,EAAWD,EAAU5D,GACxCyD,EAAIe,OAAOrgB,EAAM6b,GACjByD,EAAIkB,YACJlB,EAAInH,OACJmH,EAAIlH,UAGF7f,KAAKiN,QAAUjM,EAAQsZ,MAAMgG,UAC/BtgB,KAAKiN,QAAUjM,EAAQsZ,MAAMiG,QAAS,CAEtC,GAAI2H,GAAc,EACdC,EAAO,GAAI5mB,GAAWvB,KAAKkc,SAAUlc,KAAKmc,UAAWnc,KAAKmc,SAASnc,KAAKkc,UAAU,GAAG,EAKzF,KAJAiM,EAAKvY,QACDuY,EAAKC,aAAepoB,KAAKkc,UAC3BiM,EAAKE,QAECF,EAAKtY,OACXmC,EAAIsR,GAAU6E,EAAKC,aAAepoB,KAAKkc,WAAalc,KAAKmc,SAAWnc,KAAKkc,UAAY1J,EAErFuU,EAAIa,YACJb,EAAIc,OAAOpgB,EAAOygB,EAAalW,GAC/B+U,EAAIe,OAAOrgB,EAAMuK,GACjB+U,EAAIlH,SAEJkH,EAAIuB,UAAY,QAChBvB,EAAIwB,aAAe,SACnBxB,EAAIiB,UAAYhoB,KAAKsc,UACrByK,EAAIyB,SAASL,EAAKC,aAAc3gB,EAAO,EAAIygB,EAAalW,GAExDmW,EAAKE,MAGPtB,GAAIuB,UAAY,QAChBvB,EAAIwB,aAAe,KACnB,IAAIE,GAAQzoB,KAAKqa,WACjB0M,GAAIyB,SAASC,EAAOpB,EAAO/D,EAAStjB,KAAK0Z,UAO7C1Y,EAAQmS,UAAU6S,cAAgB,WAGhC,GAFAhmB,KAAKsf,MAAM3L,OAAOsQ,UAAY,GAE1BjkB,KAAKuhB,WAAY,CACnB,GAAI9S,IACFia,QAAW1oB,KAAKqmB,uBAEdtB,EAAS,GAAIzjB,GAAOtB,KAAKsf,MAAM3L,OAAQlF,EAC3CzO,MAAKsf,MAAM3L,OAAOoR,OAASA,EAG3B/kB,KAAKsf,MAAM3L,OAAO1G,MAAM+W,QAAU,OAGlCe,EAAO4D,UAAU3oB,KAAKuhB,WAAWzK,QACjCiO,EAAO6D,gBAAgB5oB,KAAK+a,kBAG5B,IAAI5G,GAAKnU,KACL6oB,EAAW,WACb,GAAI1gB,GAAQ4c,EAAO+D,UAEnB3U,GAAGoN,WAAWwH,YAAY5gB,GAC1BgM,EAAGgH,WAAahH,EAAGoN,WAAWuB,iBAE9B3O,EAAGsN,SAELsD,GAAOiE,oBAAoBH,OAG3B7oB,MAAKsf,MAAM3L,OAAOoR,OAASve,QAO/BxF,EAAQmS,UAAUmT,cAAgB,WACE9f,SAA7BxG,KAAKsf,MAAM3L,OAAOoR,QACrB/kB,KAAKsf,MAAM3L,OAAOoR,OAAOtD,UAQ7BzgB,EAAQmS,UAAU0T,YAAc,WAC9B,GAAI7mB,KAAKuhB,WAAY,CACnB,GAAIhC,GAASvf,KAAKsf,MAAMC,OACpBwH,EAAMxH,EAAOyH,WAAW,KAE5BD,GAAIQ,KAAO,aACXR,EAAIkC,UAAY,OAChBlC,EAAIiB,UAAY,OAChBjB,EAAIuB,UAAY,OAChBvB,EAAIwB,aAAe,KAEnB,IAAIxW,GAAI/R,KAAK0Z,OACT1H,EAAIhS,KAAK0Z,MACbqN,GAAIyB,SAASxoB,KAAKuhB,WAAW2H,WAAa,KAAOlpB,KAAKuhB,WAAW4H,mBAAoBpX,EAAGC,KAQ5FhR,EAAQmS,UAAUqT,YAAc,WAC9B,GAEE4C,GAAMC,EAAIlB,EAAMmB,EAChBC,EAAMC,EAAOC,EAAOC,EACpBC,EAAQC,EAASC,EACjBC,EAAQC,EALNxK,EAASvf,KAAKsf,MAAMC,OACtBwH,EAAMxH,EAAOyH,WAAW,KAQ1BD,GAAIQ,KAAO,GAAKvnB,KAAKib,OAAOmE,eAAiB,UAG7C,IAAI4K,GAAW,KAAQhqB,KAAKid,MAAMlL,EAC9BkY,EAAW,KAAQjqB,KAAKid,MAAMjL,EAC9BkY,EAAa,EAAIlqB,KAAKib,OAAOmE,eAC7B+K,EAAWnqB,KAAKib,OAAO6K,iBAAiBN,UAU5C,KAPAuB,EAAIO,UAAY,EAChBgC,EAAoC9iB,SAAtBxG,KAAKkiB,aACnBiG,EAAO,GAAI5mB,GAAWvB,KAAKyb,KAAMzb,KAAK2b,KAAM3b,KAAK0b,MAAO4N,GACxDnB,EAAKvY,QACDuY,EAAKC,aAAepoB,KAAKyb,MAC3B0M,EAAKE,QAECF,EAAKtY,OAAO,CAClB,GAAIkC,GAAIoW,EAAKC,YAETpoB,MAAKya,UACP2O,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQ0Q,EAAG/R,KAAK4b,KAAM5b,KAAK+b,OAC1DsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQ0Q,EAAG/R,KAAK8b,KAAM9b,KAAK+b,OACxDgL,EAAIY,YAAc3nB,KAAKuc,UACvBwK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,WAGJuJ,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQ0Q,EAAG/R,KAAK4b,KAAM5b,KAAK+b,OAC1DsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQ0Q,EAAG/R,KAAK4b,KAAKoO,EAAUhqB,KAAK+b,OACjEgL,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,SAEJuJ,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQ0Q,EAAG/R,KAAK8b,KAAM9b,KAAK+b,OAC1DsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQ0Q,EAAG/R,KAAK8b,KAAKkO,EAAUhqB,KAAK+b,OACjEgL,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,UAGN4J,EAASvkB,KAAKqZ,IAAI4L,GAAY,EAAKnqB,KAAK4b,KAAO5b,KAAK8b,KACpDyN,EAAOvpB,KAAKud,eAAe,GAAIlc,GAAQ0Q,EAAG0X,EAAOzpB,KAAK+b,OAClD7W,KAAKqZ,IAAe,EAAX4L,GAAgB,GAC3BpD,EAAIuB,UAAY,SAChBvB,EAAIwB,aAAe,MACnBgB,EAAKvX,GAAKkY,GAEHhlB,KAAKkZ,IAAe,EAAX+L,GAAgB,GAChCpD,EAAIuB,UAAY,QAChBvB,EAAIwB,aAAe,WAGnBxB,EAAIuB,UAAY,OAChBvB,EAAIwB,aAAe,UAErBxB,EAAIiB,UAAYhoB,KAAKsc,UACrByK,EAAIyB,SAAS,KAAOxoB,KAAKia,YAAYkO,EAAKC,cAAgB,KAAMmB,EAAKxX,EAAGwX,EAAKvX,GAE7EmW,EAAKE,OAWP,IAPAtB,EAAIO,UAAY,EAChBgC,EAAoC9iB,SAAtBxG,KAAKsiB,aACnB6F,EAAO,GAAI5mB,GAAWvB,KAAK4b,KAAM5b,KAAK8b,KAAM9b,KAAK6b,MAAOyN,GACxDnB,EAAKvY,QACDuY,EAAKC,aAAepoB,KAAK4b,MAC3BuM,EAAKE,QAECF,EAAKtY,OACP7P,KAAKya,UACP2O,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAKyb,KAAM0M,EAAKC,aAAcpoB,KAAK+b,OAC1EsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAK2b,KAAMwM,EAAKC,aAAcpoB,KAAK+b,OACxEgL,EAAIY,YAAc3nB,KAAKuc,UACvBwK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,WAGJuJ,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAKyb,KAAM0M,EAAKC,aAAcpoB,KAAK+b,OAC1EsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAKyb,KAAKwO,EAAU9B,EAAKC,aAAcpoB,KAAK+b,OACjFgL,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,SAEJuJ,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAK2b,KAAMwM,EAAKC,aAAcpoB,KAAK+b,OAC1EsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAK2b,KAAKsO,EAAU9B,EAAKC,aAAcpoB,KAAK+b,OACjFgL,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,UAGN2J,EAAStkB,KAAKkZ,IAAI+L,GAAa,EAAKnqB,KAAKyb,KAAOzb,KAAK2b,KACrD4N,EAAOvpB,KAAKud,eAAe,GAAIlc,GAAQmoB,EAAOrB,EAAKC,aAAcpoB,KAAK+b,OAClE7W,KAAKqZ,IAAe,EAAX4L,GAAgB,GAC3BpD,EAAIuB,UAAY,SAChBvB,EAAIwB,aAAe,MACnBgB,EAAKvX,GAAKkY,GAEHhlB,KAAKkZ,IAAe,EAAX+L,GAAgB,GAChCpD,EAAIuB,UAAY,QAChBvB,EAAIwB,aAAe,WAGnBxB,EAAIuB,UAAY,OAChBvB,EAAIwB,aAAe,UAErBxB,EAAIiB,UAAYhoB,KAAKsc,UACrByK,EAAIyB,SAAS,KAAOxoB,KAAKka,YAAYiO,EAAKC,cAAgB,KAAMmB,EAAKxX,EAAGwX,EAAKvX,GAE7EmW,EAAKE,MAaP,KATAtB,EAAIO,UAAY,EAChBgC,EAAoC9iB,SAAtBxG,KAAK0iB,aACnByF,EAAO,GAAI5mB,GAAWvB,KAAK+b,KAAM/b,KAAKic,KAAMjc,KAAKgc,MAAOsN,GACxDnB,EAAKvY,QACDuY,EAAKC,aAAepoB,KAAK+b,MAC3BoM,EAAKE,OAEPmB,EAAStkB,KAAKqZ,IAAI4L,GAAa,EAAKnqB,KAAKyb,KAAOzb,KAAK2b,KACrD8N,EAASvkB,KAAKkZ,IAAI+L,GAAa,EAAKnqB,KAAK4b,KAAO5b,KAAK8b,MAC7CqM,EAAKtY,OAEXuZ,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQmoB,EAAOC,EAAOtB,EAAKC,eAC1DrB,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOsB,EAAKrX,EAAImY,EAAYd,EAAKpX,GACrC+U,EAAIlH,SAEJkH,EAAIuB,UAAY,QAChBvB,EAAIwB,aAAe,SACnBxB,EAAIiB,UAAYhoB,KAAKsc,UACrByK,EAAIyB,SAASxoB,KAAKma,YAAYgO,EAAKC,cAAgB,IAAKgB,EAAKrX,EAAI,EAAGqX,EAAKpX,GAEzEmW,EAAKE,MAEPtB,GAAIO,UAAY,EAChB8B,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQmoB,EAAOC,EAAOzpB,KAAK+b,OAC1DsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQmoB,EAAOC,EAAOzpB,KAAKic,OACxD8K,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,SAGJkH,EAAIO,UAAY,EAEhBwC,EAAS9pB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAKyb,KAAMzb,KAAK4b,KAAM5b,KAAK+b,OACpEgO,EAAS/pB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAK2b,KAAM3b,KAAK4b,KAAM5b,KAAK+b,OACpEgL,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOiC,EAAO/X,EAAG+X,EAAO9X,GAC5B+U,EAAIe,OAAOiC,EAAOhY,EAAGgY,EAAO/X,GAC5B+U,EAAIlH,SAEJiK,EAAS9pB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAKyb,KAAMzb,KAAK8b,KAAM9b,KAAK+b,OACpEgO,EAAS/pB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAK2b,KAAM3b,KAAK8b,KAAM9b,KAAK+b,OACpEgL,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOiC,EAAO/X,EAAG+X,EAAO9X,GAC5B+U,EAAIe,OAAOiC,EAAOhY,EAAGgY,EAAO/X,GAC5B+U,EAAIlH,SAGJkH,EAAIO,UAAY,EAEhB8B,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAKyb,KAAMzb,KAAK4b,KAAM5b,KAAK+b,OAClEsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAKyb,KAAMzb,KAAK8b,KAAM9b,KAAK+b,OAChEgL,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,SAEJuJ,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAK2b,KAAM3b,KAAK4b,KAAM5b,KAAK+b,OAClEsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAK2b,KAAM3b,KAAK8b,KAAM9b,KAAK+b,OAChEgL,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,QAGJ,IAAIhG,GAAS7Z,KAAK6Z,MACdA,GAAOlU,OAAS,IAClBkkB,EAAU,GAAM7pB,KAAKid,MAAMjL,EAC3BwX,GAASxpB,KAAKyb,KAAOzb,KAAK2b,MAAQ,EAClC8N,EAASvkB,KAAKqZ,IAAI4L,GAAY,EAAKnqB,KAAK4b,KAAOiO,EAAS7pB,KAAK8b,KAAO+N,EACpEN,EAAOvpB,KAAKud,eAAe,GAAIlc,GAAQmoB,EAAOC,EAAOzpB,KAAK+b,OACtD7W,KAAKqZ,IAAe,EAAX4L,GAAgB,GAC3BpD,EAAIuB,UAAY,SAChBvB,EAAIwB,aAAe,OAEZrjB,KAAKkZ,IAAe,EAAX+L,GAAgB,GAChCpD,EAAIuB,UAAY,QAChBvB,EAAIwB,aAAe,WAGnBxB,EAAIuB,UAAY,OAChBvB,EAAIwB,aAAe,UAErBxB,EAAIiB,UAAYhoB,KAAKsc,UACrByK,EAAIyB,SAAS3O,EAAQ0P,EAAKxX,EAAGwX,EAAKvX,GAIpC,IAAI8H,GAAS9Z,KAAK8Z,MACdA,GAAOnU,OAAS,IAClBikB,EAAU,GAAM5pB,KAAKid,MAAMlL,EAC3ByX,EAAStkB,KAAKkZ,IAAI+L,GAAa,EAAKnqB,KAAKyb,KAAOmO,EAAU5pB,KAAK2b,KAAOiO,EACtEH,GAASzpB,KAAK4b,KAAO5b,KAAK8b,MAAQ,EAClCyN,EAAOvpB,KAAKud,eAAe,GAAIlc,GAAQmoB,EAAOC,EAAOzpB,KAAK+b,OACtD7W,KAAKqZ,IAAe,EAAX4L,GAAgB,GAC3BpD,EAAIuB,UAAY,SAChBvB,EAAIwB,aAAe,OAEZrjB,KAAKkZ,IAAe,EAAX+L,GAAgB,GAChCpD,EAAIuB,UAAY,QAChBvB,EAAIwB,aAAe,WAGnBxB,EAAIuB,UAAY,OAChBvB,EAAIwB,aAAe,UAErBxB,EAAIiB,UAAYhoB,KAAKsc,UACrByK,EAAIyB,SAAS1O,EAAQyP,EAAKxX,EAAGwX,EAAKvX,GAIpC,IAAI+H,GAAS/Z,KAAK+Z,MACdA,GAAOpU,OAAS,IAClBgkB,EAAS,GACTH,EAAStkB,KAAKqZ,IAAI4L,GAAa,EAAKnqB,KAAKyb,KAAOzb,KAAK2b,KACrD8N,EAASvkB,KAAKkZ,IAAI+L,GAAa,EAAKnqB,KAAK4b,KAAO5b,KAAK8b,KACrD4N,GAAS1pB,KAAK+b,KAAO/b,KAAKic,MAAQ,EAClCsN,EAAOvpB,KAAKud,eAAe,GAAIlc,GAAQmoB,EAAOC,EAAOC,IACrD3C,EAAIuB,UAAY,QAChBvB,EAAIwB,aAAe,SACnBxB,EAAIiB,UAAYhoB,KAAKsc,UACrByK,EAAIyB,SAASzO,EAAQwP,EAAKxX,EAAI4X,EAAQJ,EAAKvX,KAU/ChR,EAAQmS,UAAUuU,SAAW,SAAS0C,EAAGC,EAAGC,GAC1C,GAAIC,GAAGC,EAAGC,EAAGC,EAAGC,EAAIC,CAMpB,QAJAF,EAAIJ,EAAID,EACRM,EAAKzlB,KAAKC,MAAMilB,EAAE,IAClBQ,EAAIF,GAAK,EAAIxlB,KAAK2lB,IAAMT,EAAE,GAAM,EAAK,IAE7BO,GACN,IAAK,GAAGJ,EAAIG,EAAGF,EAAII,EAAGH,EAAI,CAAG,MAC7B,KAAK,GAAGF,EAAIK,EAAGJ,EAAIE,EAAGD,EAAI,CAAG,MAC7B,KAAK,GAAGF,EAAI,EAAGC,EAAIE,EAAGD,EAAIG,CAAG,MAC7B,KAAK,GAAGL,EAAI,EAAGC,EAAII,EAAGH,EAAIC,CAAG,MAC7B,KAAK,GAAGH,EAAIK,EAAGJ,EAAI,EAAGC,EAAIC,CAAG,MAC7B,KAAK,GAAGH,EAAIG,EAAGF,EAAI,EAAGC,EAAIG,CAAG,MAE7B,SAASL,EAAI,EAAGC,EAAI,EAAGC,EAAI,EAG7B,MAAO,OAAS7f,SAAW,IAAF2f,GAAS,IAAM3f,SAAW,IAAF4f,GAAS,IAAM5f,SAAW,IAAF6f,GAAS,KAQpFzpB,EAAQmS,UAAUsT,gBAAkB,WAClC,GAEEvU,GAAOmV,EAAO1f,EAAKmjB,EACnBtlB,EACAulB,EAAgB/C,EAAWL,EAAaL,EACxC3b,EAAGC,EAAGC,EAAGmf,EALPzL,EAASvf,KAAKsf,MAAMC,OACtBwH,EAAMxH,EAAOyH,WAAW,KAO1B,MAAwBxgB,SAApBxG,KAAKmb,YAA4Bnb,KAAKmb,WAAWxV,QAAU,GAA/D,CAIA,IAAKH,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IAAK,CAC3C,GAAI4d,GAAQpjB,KAAK0d,2BAA2B1d,KAAKmb,WAAW3V,GAAG0M,OAC3DmR,EAASrjB,KAAK2d,4BAA4ByF,EAE9CpjB,MAAKmb,WAAW3V,GAAG4d,MAAQA,EAC3BpjB,KAAKmb,WAAW3V,GAAG6d,OAASA,CAG5B,IAAI4H,GAAcjrB,KAAK0d,2BAA2B1d,KAAKmb,WAAW3V,GAAG8d,OACrEtjB,MAAKmb,WAAW3V,GAAG0lB,KAAOlrB,KAAKwa,gBAAkByQ,EAAYtlB,UAAYslB,EAAY/N,EAIvF,GAAIiO,GAAY,SAAU5lB,EAAGa,GAC3B,MAAOA,GAAE8kB,KAAO3lB,EAAE2lB,KAIpB,IAFAlrB,KAAKmb,WAAWjF,KAAKiV,GAEjBnrB,KAAKiN,QAAUjM,EAAQsZ,MAAMoG,SAC/B,IAAKlb,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IAMtC,GALA0M,EAAQlS,KAAKmb,WAAW3V,GACxB6hB,EAAQrnB,KAAKmb,WAAW3V,GAAG+d,WAC3B5b,EAAQ3H,KAAKmb,WAAW3V,GAAGge,SAC3BsH,EAAQ9qB,KAAKmb,WAAW3V,GAAGie,WAEbjd,SAAV0L,GAAiC1L,SAAV6gB,GAA+B7gB,SAARmB,GAA+BnB,SAAVskB,EAAqB,CAE1F,GAAI9qB,KAAK4a,gBAAkB5a,KAAK2a,WAAY,CAK1C,GAAIyQ,GAAQ/pB,EAAQgqB,SAASP,EAAM1H,MAAOlR,EAAMkR,OAC5CkI,EAAQjqB,EAAQgqB,SAAS1jB,EAAIyb,MAAOiE,EAAMjE,OAC1CmI,EAAelqB,EAAQmqB,aAAaJ,EAAOE,GAC3C7lB,EAAM8lB,EAAa5lB,QAGvBolB,GAAkBQ,EAAarO,EAAI,MAGnC6N,IAAiB,CAGfA,IAEFC,GAAQ9Y,EAAMA,MAAMgL,EAAImK,EAAMnV,MAAMgL,EAAIvV,EAAIuK,MAAMgL,EAAI4N,EAAM5Y,MAAMgL,GAAK,EACvEvR,EAAoE,KAA/D,GAAKqf,EAAOhrB,KAAK+b,MAAQ/b,KAAKid,MAAMC,EAAKld,KAAK8a,eACnDlP,EAAI,EAEA5L,KAAK2a,YACP9O,EAAI3G,KAAK4G,IAAI,EAAKyf,EAAaxZ,EAAItM,EAAO,EAAG,GAC7CuiB,EAAYhoB,KAAK0nB,SAAS/b,EAAGC,EAAGC,GAChC8b,EAAcK,IAGdnc,EAAI,EACJmc,EAAYhoB,KAAK0nB,SAAS/b,EAAGC,EAAGC,GAChC8b,EAAc3nB,KAAKsc,aAIrB0L,EAAY,OACZL,EAAc3nB,KAAKsc,WAErBgL,EAAY,GAEZP,EAAIO,UAAYA,EAChBP,EAAIiB,UAAYA,EAChBjB,EAAIY,YAAcA,EAClBZ,EAAIa,YACJb,EAAIc,OAAO3V,EAAMmR,OAAOtR,EAAGG,EAAMmR,OAAOrR,GACxC+U,EAAIe,OAAOT,EAAMhE,OAAOtR,EAAGsV,EAAMhE,OAAOrR,GACxC+U,EAAIe,OAAOgD,EAAMzH,OAAOtR,EAAG+Y,EAAMzH,OAAOrR,GACxC+U,EAAIe,OAAOngB,EAAI0b,OAAOtR,EAAGpK,EAAI0b,OAAOrR,GACpC+U,EAAIkB,YACJlB,EAAInH,OACJmH,EAAIlH,cAKR,KAAKra,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IACtC0M,EAAQlS,KAAKmb,WAAW3V,GACxB6hB,EAAQrnB,KAAKmb,WAAW3V,GAAG+d,WAC3B5b,EAAQ3H,KAAKmb,WAAW3V,GAAGge,SAEbhd,SAAV0L,IAEAoV,EADEtnB,KAAKwa,gBACK,GAAKtI,EAAMkR,MAAMlG,EAGjB,IAAMld,KAAKkb,IAAIgC,EAAIld,KAAKib,OAAOmE,iBAIjC5Y,SAAV0L,GAAiC1L,SAAV6gB,IAEzB2D,GAAQ9Y,EAAMA,MAAMgL,EAAImK,EAAMnV,MAAMgL,GAAK,EACzCvR,EAAoE,KAA/D,GAAKqf,EAAOhrB,KAAK+b,MAAQ/b,KAAKid,MAAMC,EAAKld,KAAK8a,eAEnDiM,EAAIO,UAAYA,EAChBP,EAAIY,YAAc3nB,KAAK0nB,SAAS/b,EAAG,EAAG,GACtCob,EAAIa,YACJb,EAAIc,OAAO3V,EAAMmR,OAAOtR,EAAGG,EAAMmR,OAAOrR,GACxC+U,EAAIe,OAAOT,EAAMhE,OAAOtR,EAAGsV,EAAMhE,OAAOrR,GACxC+U,EAAIlH,UAGQrZ,SAAV0L,GAA+B1L,SAARmB,IAEzBqjB,GAAQ9Y,EAAMA,MAAMgL,EAAIvV,EAAIuK,MAAMgL,GAAK,EACvCvR,EAAoE,KAA/D,GAAKqf,EAAOhrB,KAAK+b,MAAQ/b,KAAKid,MAAMC,EAAKld,KAAK8a,eAEnDiM,EAAIO,UAAYA,EAChBP,EAAIY,YAAc3nB,KAAK0nB,SAAS/b,EAAG,EAAG,GACtCob,EAAIa,YACJb,EAAIc,OAAO3V,EAAMmR,OAAOtR,EAAGG,EAAMmR,OAAOrR,GACxC+U,EAAIe,OAAOngB,EAAI0b,OAAOtR,EAAGpK,EAAI0b,OAAOrR,GACpC+U,EAAIlH,YAWZ7e,EAAQmS,UAAUyT,eAAiB,WACjC,GAEIphB,GAFA+Z,EAASvf,KAAKsf,MAAMC,OACpBwH,EAAMxH,EAAOyH,WAAW,KAG5B,MAAwBxgB,SAApBxG,KAAKmb,YAA4Bnb,KAAKmb,WAAWxV,QAAU,GAA/D,CAIA,IAAKH,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IAAK,CAC3C,GAAI4d,GAAQpjB,KAAK0d,2BAA2B1d,KAAKmb,WAAW3V,GAAG0M,OAC3DmR,EAASrjB,KAAK2d,4BAA4ByF,EAC9CpjB,MAAKmb,WAAW3V,GAAG4d,MAAQA,EAC3BpjB,KAAKmb,WAAW3V,GAAG6d,OAASA,CAG5B,IAAI4H,GAAcjrB,KAAK0d,2BAA2B1d,KAAKmb,WAAW3V,GAAG8d,OACrEtjB,MAAKmb,WAAW3V,GAAG0lB,KAAOlrB,KAAKwa,gBAAkByQ,EAAYtlB,UAAYslB,EAAY/N,EAIvF,GAAIiO,GAAY,SAAU5lB,EAAGa,GAC3B,MAAOA,GAAE8kB,KAAO3lB,EAAE2lB,KAEpBlrB,MAAKmb,WAAWjF,KAAKiV,EAGrB,IAAI/D,GAAmC,IAAzBpnB,KAAKsf,MAAME,WACzB,KAAKha,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IAAK,CAC3C,GAAI0M,GAAQlS,KAAKmb,WAAW3V,EAE5B,IAAIxF,KAAKiN,QAAUjM,EAAQsZ,MAAM+F,QAAS,CAGxC,GAAI+I,GAAOppB,KAAKud,eAAerL,EAAMoR,OACrCyD,GAAIO,UAAY,EAChBP,EAAIY,YAAc3nB,KAAKuc,UACvBwK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAO5V,EAAMmR,OAAOtR,EAAGG,EAAMmR,OAAOrR,GACxC+U,EAAIlH,SAIN,GAAIxN,EAEFA,GADErS,KAAKiN,QAAUjM,EAAQsZ,MAAMiG,QACxB6G,EAAQ,EAAI,EAAEA,GAAWlV,EAAMA,MAAM7K,MAAQrH,KAAKkc,WAAalc,KAAKmc,SAAWnc,KAAKkc,UAGpFkL,CAGT,IAAIqE,EAEFA,GADEzrB,KAAKwa,gBACEnI,GAAQH,EAAMkR,MAAMlG,EAGpB7K,IAASrS,KAAKkb,IAAIgC,EAAIld,KAAKib,OAAOmE,gBAEhC,EAATqM,IACFA,EAAS,EAGX,IAAI7e,GAAKzB,EAAO4U,CACZ/f,MAAKiN,QAAUjM,EAAQsZ,MAAMgG,UAE/B1T,EAAqE,KAA9D,GAAKsF,EAAMA,MAAM7K,MAAQrH,KAAKkc,UAAYlc,KAAKid,MAAM5V,OAC5D8D,EAAQnL,KAAK0nB,SAAS9a,EAAK,EAAG,GAC9BmT,EAAc/f,KAAK0nB,SAAS9a,EAAK,EAAG,KAE7B5M,KAAKiN,QAAUjM,EAAQsZ,MAAMiG,SACpCpV,EAAQnL,KAAKwc,SACbuD,EAAc/f,KAAKyc,iBAInB7P,EAA+E,KAAxE,GAAKsF,EAAMA,MAAMgL,EAAIld,KAAK+b,MAAQ/b,KAAKid,MAAMC,EAAKld,KAAK8a,eAC9D3P,EAAQnL,KAAK0nB,SAAS9a,EAAK,EAAG,GAC9BmT,EAAc/f,KAAK0nB,SAAS9a,EAAK,EAAG,KAItCma,EAAIO,UAAY,EAChBP,EAAIY,YAAc5H,EAClBgH,EAAIiB,UAAY7c,EAChB4b,EAAIa,YACJb,EAAI2E,IAAIxZ,EAAMmR,OAAOtR,EAAGG,EAAMmR,OAAOrR,EAAGyZ,EAAQ,EAAW,EAARvmB,KAAKymB,IAAM,GAC9D5E,EAAInH,OACJmH,EAAIlH,YAQR7e,EAAQmS,UAAUwT,eAAiB,WACjC,GAEInhB,GAAGomB,EAAGC,EAASC,EAFfvM,EAASvf,KAAKsf,MAAMC,OACpBwH,EAAMxH,EAAOyH,WAAW,KAG5B,MAAwBxgB,SAApBxG,KAAKmb,YAA4Bnb,KAAKmb,WAAWxV,QAAU,GAA/D,CAIA,IAAKH,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IAAK,CAC3C,GAAI4d,GAAQpjB,KAAK0d,2BAA2B1d,KAAKmb,WAAW3V,GAAG0M,OAC3DmR,EAASrjB,KAAK2d,4BAA4ByF,EAC9CpjB,MAAKmb,WAAW3V,GAAG4d,MAAQA,EAC3BpjB,KAAKmb,WAAW3V,GAAG6d,OAASA,CAG5B,IAAI4H,GAAcjrB,KAAK0d,2BAA2B1d,KAAKmb,WAAW3V,GAAG8d,OACrEtjB,MAAKmb,WAAW3V,GAAG0lB,KAAOlrB,KAAKwa,gBAAkByQ,EAAYtlB,UAAYslB,EAAY/N,EAIvF,GAAIiO,GAAY,SAAU5lB,EAAGa,GAC3B,MAAOA,GAAE8kB,KAAO3lB,EAAE2lB,KAEpBlrB,MAAKmb,WAAWjF,KAAKiV,EAGrB,IAAIY,GAAS/rB,KAAKoc,UAAY,EAC1B4P,EAAShsB,KAAKqc,UAAY,CAC9B,KAAK7W,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IAAK,CAC3C,GAGIoH,GAAKzB,EAAO4U,EAHZ7N,EAAQlS,KAAKmb,WAAW3V,EAIxBxF,MAAKiN,QAAUjM,EAAQsZ,MAAM6F,UAE/BvT,EAAqE,KAA9D,GAAKsF,EAAMA,MAAM7K,MAAQrH,KAAKkc,UAAYlc,KAAKid,MAAM5V,OAC5D8D,EAAQnL,KAAK0nB,SAAS9a,EAAK,EAAG,GAC9BmT,EAAc/f,KAAK0nB,SAAS9a,EAAK,EAAG,KAE7B5M,KAAKiN,QAAUjM,EAAQsZ,MAAM8F,SACpCjV,EAAQnL,KAAKwc,SACbuD,EAAc/f,KAAKyc,iBAInB7P,EAA+E,KAAxE,GAAKsF,EAAMA,MAAMgL,EAAIld,KAAK+b,MAAQ/b,KAAKid,MAAMC,EAAKld,KAAK8a,eAC9D3P,EAAQnL,KAAK0nB,SAAS9a,EAAK,EAAG,GAC9BmT,EAAc/f,KAAK0nB,SAAS9a,EAAK,EAAG,KAIlC5M,KAAKiN,QAAUjM,EAAQsZ,MAAM8F,UAC/B2L,EAAU/rB,KAAKoc,UAAY,IAAOlK,EAAMA,MAAM7K,MAAQrH,KAAKkc,WAAalc,KAAKmc,SAAWnc,KAAKkc,UAAY,GAAM,IAC/G8P,EAAUhsB,KAAKqc,UAAY,IAAOnK,EAAMA,MAAM7K,MAAQrH,KAAKkc,WAAalc,KAAKmc,SAAWnc,KAAKkc,UAAY,GAAM,IAIjH,IAAI/H,GAAKnU,KACLwd,EAAUtL,EAAMA,MAChBvK,IACDuK,MAAO,GAAI7Q,GAAQmc,EAAQzL,EAAIga,EAAQvO,EAAQxL,EAAIga,EAAQxO,EAAQN,KACnEhL,MAAO,GAAI7Q,GAAQmc,EAAQzL,EAAIga,EAAQvO,EAAQxL,EAAIga,EAAQxO,EAAQN,KACnEhL,MAAO,GAAI7Q,GAAQmc,EAAQzL,EAAIga,EAAQvO,EAAQxL,EAAIga,EAAQxO,EAAQN,KACnEhL,MAAO,GAAI7Q,GAAQmc,EAAQzL,EAAIga,EAAQvO,EAAQxL,EAAIga,EAAQxO,EAAQN,KAElEoG,IACDpR,MAAO,GAAI7Q,GAAQmc,EAAQzL,EAAIga,EAAQvO,EAAQxL,EAAIga,EAAQhsB,KAAK+b,QAChE7J,MAAO,GAAI7Q,GAAQmc,EAAQzL,EAAIga,EAAQvO,EAAQxL,EAAIga,EAAQhsB,KAAK+b,QAChE7J,MAAO,GAAI7Q,GAAQmc,EAAQzL,EAAIga,EAAQvO,EAAQxL,EAAIga,EAAQhsB,KAAK+b,QAChE7J,MAAO,GAAI7Q,GAAQmc,EAAQzL,EAAIga,EAAQvO,EAAQxL,EAAIga,EAAQhsB,KAAK+b,OAInEpU,GAAIU,QAAQ,SAAU0a,GACpBA,EAAIM,OAASlP,EAAGoJ,eAAewF,EAAI7Q,SAErCoR,EAAOjb,QAAQ,SAAU0a,GACvBA,EAAIM,OAASlP,EAAGoJ,eAAewF,EAAI7Q,QAIrC,IAAI+Z,KACDH,QAASnkB,EAAKukB,OAAQ7qB,EAAQ8qB,IAAI7I,EAAO,GAAGpR,MAAOoR,EAAO,GAAGpR,SAC7D4Z,SAAUnkB,EAAI,GAAIA,EAAI,GAAI2b,EAAO,GAAIA,EAAO,IAAK4I,OAAQ7qB,EAAQ8qB,IAAI7I,EAAO,GAAGpR,MAAOoR,EAAO,GAAGpR,SAChG4Z,SAAUnkB,EAAI,GAAIA,EAAI,GAAI2b,EAAO,GAAIA,EAAO,IAAK4I,OAAQ7qB,EAAQ8qB,IAAI7I,EAAO,GAAGpR,MAAOoR,EAAO,GAAGpR,SAChG4Z,SAAUnkB,EAAI,GAAIA,EAAI,GAAI2b,EAAO,GAAIA,EAAO,IAAK4I,OAAQ7qB,EAAQ8qB,IAAI7I,EAAO,GAAGpR,MAAOoR,EAAO,GAAGpR,SAChG4Z,SAAUnkB,EAAI,GAAIA,EAAI,GAAI2b,EAAO,GAAIA,EAAO,IAAK4I,OAAQ7qB,EAAQ8qB,IAAI7I,EAAO,GAAGpR,MAAOoR,EAAO,GAAGpR,QAKnG,KAHAA,EAAM+Z,SAAWA,EAGZL,EAAI,EAAGA,EAAIK,EAAStmB,OAAQimB,IAAK,CACpCC,EAAUI,EAASL,EACnB,IAAIQ,GAAcpsB,KAAK0d,2BAA2BmO,EAAQK,OAC1DL,GAAQX,KAAOlrB,KAAKwa,gBAAkB4R,EAAYzmB,UAAYymB,EAAYlP,EAwB5E,IAjBA+O,EAAS/V,KAAK,SAAU3Q,EAAGa,GACzB,GAAIimB,GAAOjmB,EAAE8kB,KAAO3lB,EAAE2lB,IACtB,OAAImB,GAAaA,EAGb9mB,EAAEumB,UAAYnkB,EAAY,EAC1BvB,EAAE0lB,UAAYnkB,EAAY,GAGvB,IAITof,EAAIO,UAAY,EAChBP,EAAIY,YAAc5H,EAClBgH,EAAIiB,UAAY7c,EAEXygB,EAAI,EAAGA,EAAIK,EAAStmB,OAAQimB,IAC/BC,EAAUI,EAASL,GACnBE,EAAUD,EAAQC,QAClB/E,EAAIa,YACJb,EAAIc,OAAOiE,EAAQ,GAAGzI,OAAOtR,EAAG+Z,EAAQ,GAAGzI,OAAOrR,GAClD+U,EAAIe,OAAOgE,EAAQ,GAAGzI,OAAOtR,EAAG+Z,EAAQ,GAAGzI,OAAOrR,GAClD+U,EAAIe,OAAOgE,EAAQ,GAAGzI,OAAOtR,EAAG+Z,EAAQ,GAAGzI,OAAOrR,GAClD+U,EAAIe,OAAOgE,EAAQ,GAAGzI,OAAOtR,EAAG+Z,EAAQ,GAAGzI,OAAOrR,GAClD+U,EAAIe,OAAOgE,EAAQ,GAAGzI,OAAOtR,EAAG+Z,EAAQ,GAAGzI,OAAOrR,GAClD+U,EAAInH,OACJmH,EAAIlH,YAUV7e,EAAQmS,UAAUuT,gBAAkB,WAClC,GAEExU,GAAO1M,EAFL+Z,EAASvf,KAAKsf,MAAMC,OACtBwH,EAAMxH,EAAOyH,WAAW,KAG1B,MAAwBxgB,SAApBxG,KAAKmb,YAA4Bnb,KAAKmb,WAAWxV,QAAU,GAA/D,CAIA,IAAKH,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IAAK,CAC3C,GAAI4d,GAAQpjB,KAAK0d,2BAA2B1d,KAAKmb,WAAW3V,GAAG0M,OAC3DmR,EAASrjB,KAAK2d,4BAA4ByF,EAE9CpjB,MAAKmb,WAAW3V,GAAG4d,MAAQA,EAC3BpjB,KAAKmb,WAAW3V,GAAG6d,OAASA,EAc9B,IAVIrjB,KAAKmb,WAAWxV,OAAS,IAC3BuM,EAAQlS,KAAKmb,WAAW,GAExB4L,EAAIO,UAAY,EAChBP,EAAIY,YAAc,OAClBZ,EAAIa,YACJb,EAAIc,OAAO3V,EAAMmR,OAAOtR,EAAGG,EAAMmR,OAAOrR,IAIrCxM,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IACtC0M,EAAQlS,KAAKmb,WAAW3V,GACxBuhB,EAAIe,OAAO5V,EAAMmR,OAAOtR,EAAGG,EAAMmR,OAAOrR,EAItChS,MAAKmb,WAAWxV,OAAS,GAC3BohB,EAAIlH,WASR7e,EAAQmS,UAAUgR,aAAe,SAAS7a,GAWxC,GAVAA,EAAQA,GAASC,OAAOD,MAIpBtJ,KAAKssB,gBACPtsB,KAAKusB,WAAWjjB,GAIlBtJ,KAAKssB,eAAiBhjB,EAAMkjB,MAAyB,IAAhBljB,EAAMkjB,MAAiC,IAAjBljB,EAAMmjB,OAC5DzsB,KAAKssB,gBAAmBtsB,KAAK0sB,UAAlC,CAGA1sB,KAAK2sB,YAAcjQ,EAAUpT,GAC7BtJ,KAAK4sB,YAAc/P,EAAUvT,GAE7BtJ,KAAK6sB,WAAa,GAAIvoB,MAAKtE,KAAK4P,OAChC5P,KAAK8sB,SAAW,GAAIxoB,MAAKtE,KAAK6P,KAC9B7P,KAAK+sB,iBAAmB/sB,KAAKib,OAAO6K,iBAEpC9lB,KAAKsf,MAAMrS,MAAM+f,OAAS,MAK1B,IAAI7Y,GAAKnU,IACTA,MAAKitB,YAAc,SAAU3jB,GAAQ6K,EAAG+Y,aAAa5jB,IACrDtJ,KAAKmtB,UAAc,SAAU7jB,GAAQ6K,EAAGoY,WAAWjjB,IACnD3I,EAAKgI,iBAAiB4I,SAAU,YAAa4C,EAAG8Y,aAChDtsB,EAAKgI,iBAAiB4I,SAAU,UAAW4C,EAAGgZ,WAC9CxsB,EAAK0I,eAAeC,KAStBtI,EAAQmS,UAAU+Z,aAAe,SAAU5jB,GACzCA,EAAQA,GAASC,OAAOD,KAGxB,IAAI8jB,GAAQ/H,WAAW3I,EAAUpT,IAAUtJ,KAAK2sB,YAC5CU,EAAQhI,WAAWxI,EAAUvT,IAAUtJ,KAAK4sB,YAE5CU,EAAgBttB,KAAK+sB,iBAAiBvH,WAAa4H,EAAQ,IAC3DG,EAAcvtB,KAAK+sB,iBAAiBtH,SAAW4H,EAAQ,IAEvDG,EAAY,EACZC,EAAYvoB,KAAKkZ,IAAIoP,EAAY,IAAM,EAAItoB,KAAKymB,GAIhDzmB,MAAK2lB,IAAI3lB,KAAKkZ,IAAIkP,IAAkBG,IACtCH,EAAgBpoB,KAAKwoB,MAAOJ,EAAgBpoB,KAAKymB,IAAOzmB,KAAKymB,GAAK,MAEhEzmB,KAAK2lB,IAAI3lB,KAAKqZ,IAAI+O,IAAkBG,IACtCH,GAAiBpoB,KAAKwoB,MAAOJ,EAAepoB,KAAKymB,GAAK,IAAQ,IAAOzmB,KAAKymB,GAAK,MAI7EzmB,KAAK2lB,IAAI3lB,KAAKkZ,IAAImP,IAAgBE,IACpCF,EAAcroB,KAAKwoB,MAAOH,EAAcroB,KAAKymB,IAAOzmB,KAAKymB,IAEvDzmB,KAAK2lB,IAAI3lB,KAAKqZ,IAAIgP,IAAgBE,IACpCF,GAAeroB,KAAKwoB,MAAOH,EAAaroB,KAAKymB,GAAK,IAAQ,IAAOzmB,KAAKymB,IAGxE3rB,KAAKib,OAAOyK,eAAe4H,EAAeC,GAC1CvtB,KAAKyhB,QAGL,IAAIkM,GAAa3tB,KAAK6lB,mBACtB7lB,MAAK4tB,KAAK,uBAAwBD,GAElChtB,EAAK0I,eAAeC,IAStBtI,EAAQmS,UAAUoZ,WAAa,SAAUjjB,GACvCtJ,KAAKsf,MAAMrS,MAAM+f,OAAS,OAC1BhtB,KAAKssB,gBAAiB,EAGtB3rB,EAAKwI,oBAAoBoI,SAAU,YAAavR,KAAKitB,aACrDtsB,EAAKwI,oBAAoBoI,SAAU,UAAavR,KAAKmtB,WACrDxsB,EAAK0I,eAAeC,IAOtBtI,EAAQmS,UAAUsR,WAAa,SAAUnb,GACvC,GAAIkP,GAAQ,IACRqV,EAAe7tB,KAAKsf,MAAM9X,wBAC1BsmB,EAASpR,EAAUpT,GAASukB,EAAapmB,KACzCsmB,EAASlR,EAAUvT,GAASukB,EAAalmB,GAE7C,IAAK3H,KAAK6a,YAAV,CASA,GALI7a,KAAKguB,gBACP3U,aAAarZ,KAAKguB,gBAIhBhuB,KAAKssB,eAEP,WADAtsB,MAAKiuB,cAIP,IAAIjuB,KAAKomB,SAAWpmB,KAAKomB,QAAQ8H,UAAW,CAE1C,GAAIA,GAAYluB,KAAKmuB,iBAAiBL,EAAQC,EAC1CG,KAAcluB,KAAKomB,QAAQ8H,YAEzBA,EACFluB,KAAKouB,aAAaF,GAGlBluB,KAAKiuB,oBAIN,CAEH,GAAI9Z,GAAKnU,IACTA,MAAKguB,eAAiB1U,WAAW,WAC/BnF,EAAG6Z,eAAiB,IAGpB,IAAIE,GAAY/Z,EAAGga,iBAAiBL,EAAQC,EACxCG,IACF/Z,EAAGia,aAAaF,IAEjB1V,MAOPxX,EAAQmS,UAAUkR,cAAgB,SAAS/a,GACzCtJ,KAAK0sB,WAAY,CAEjB,IAAIvY,GAAKnU,IACTA,MAAKquB,YAAc,SAAU/kB,GAAQ6K,EAAGma,aAAahlB,IACrDtJ,KAAKuuB,WAAc,SAAUjlB,GAAQ6K,EAAGqa,YAAYllB,IACpD3I,EAAKgI,iBAAiB4I,SAAU,YAAa4C,EAAGka,aAChD1tB,EAAKgI,iBAAiB4I,SAAU,WAAY4C,EAAGoa,YAE/CvuB,KAAKmkB,aAAa7a,IAMpBtI,EAAQmS,UAAUmb,aAAe,SAAShlB,GACxCtJ,KAAKktB,aAAa5jB,IAMpBtI,EAAQmS,UAAUqb,YAAc,SAASllB,GACvCtJ,KAAK0sB,WAAY,EAEjB/rB,EAAKwI,oBAAoBoI,SAAU,YAAavR,KAAKquB,aACrD1tB,EAAKwI,oBAAoBoI,SAAU,WAAcvR,KAAKuuB,YAEtDvuB,KAAKusB,WAAWjjB,IASlBtI,EAAQmS,UAAUoR,SAAW,SAASjb,GAC/BA,IACHA,EAAQC,OAAOD,MAGjB,IAAImlB,GAAQ,CAYZ,IAXInlB,EAAMolB,WACRD,EAAQnlB,EAAMolB,WAAW,IAChBplB,EAAMqlB,SAGfF,GAASnlB,EAAMqlB,OAAO,GAMpBF,EAAO,CACT,GAAIG,GAAY5uB,KAAKib,OAAOmE,eACxByP,EAAYD,GAAa,EAAIH,EAAQ,GAEzCzuB,MAAKib,OAAO2K,aAAaiJ,GACzB7uB,KAAKyhB,SAELzhB,KAAKiuB,eAIP,GAAIN,GAAa3tB,KAAK6lB,mBACtB7lB,MAAK4tB,KAAK,uBAAwBD,GAKlChtB,EAAK0I,eAAeC,IAUtBtI,EAAQmS,UAAU2b,gBAAkB,SAAU5c,EAAO6c,GAKnD,QAASC,GAAMjd,GACb,MAAOA,GAAI,EAAI,EAAQ,EAAJA,EAAQ,GAAK,EALlC,GAAIxM,GAAIwpB,EAAS,GACf3oB,EAAI2oB,EAAS,GACbtuB,EAAIsuB,EAAS,GAMXE,EAAKD,GAAM5oB,EAAE2L,EAAIxM,EAAEwM,IAAMG,EAAMF,EAAIzM,EAAEyM,IAAM5L,EAAE4L,EAAIzM,EAAEyM,IAAME,EAAMH,EAAIxM,EAAEwM,IACrEmd,EAAKF,GAAMvuB,EAAEsR,EAAI3L,EAAE2L,IAAMG,EAAMF,EAAI5L,EAAE4L,IAAMvR,EAAEuR,EAAI5L,EAAE4L,IAAME,EAAMH,EAAI3L,EAAE2L,IACrEod,EAAKH,GAAMzpB,EAAEwM,EAAItR,EAAEsR,IAAMG,EAAMF,EAAIvR,EAAEuR,IAAMzM,EAAEyM,EAAIvR,EAAEuR,IAAME,EAAMH,EAAItR,EAAEsR,GAGzE,SAAc,GAANkd,GAAiB,GAANC,GAAWD,GAAMC,GAC3B,GAANA,GAAiB,GAANC,GAAWD,GAAMC,GACtB,GAANF,GAAiB,GAANE,GAAWF,GAAME,IAUjCnuB,EAAQmS,UAAUgb,iBAAmB,SAAUpc,EAAGC,GAChD,GAAIxM,GACF4pB,EAAU,IACVlB,EAAY,KACZmB,EAAmB,KACnBC,EAAc,KACdpD,EAAS,GAAI9qB,GAAQ2Q,EAAGC,EAE1B,IAAIhS,KAAKiN,QAAUjM,EAAQsZ,MAAM4F,KAC/BlgB,KAAKiN,QAAUjM,EAAQsZ,MAAM6F,UAC7BngB,KAAKiN,QAAUjM,EAAQsZ,MAAM8F,QAE7B,IAAK5a,EAAIxF,KAAKmb,WAAWxV,OAAS,EAAGH,GAAK,EAAGA,IAAK,CAChD0oB,EAAYluB,KAAKmb,WAAW3V,EAC5B,IAAIymB,GAAYiC,EAAUjC,QAC1B,IAAIA,EACF,IAAK,GAAIrgB,GAAIqgB,EAAStmB,OAAS,EAAGiG,GAAK,EAAGA,IAAK,CAE7C,GAAIigB,GAAUI,EAASrgB,GACnBkgB,EAAUD,EAAQC,QAClByD,GAAazD,EAAQ,GAAGzI,OAAQyI,EAAQ,GAAGzI,OAAQyI,EAAQ,GAAGzI,QAC9DmM,GAAa1D,EAAQ,GAAGzI,OAAQyI,EAAQ,GAAGzI,OAAQyI,EAAQ,GAAGzI,OAClE,IAAIrjB,KAAK8uB,gBAAgB5C,EAAQqD,IAC/BvvB,KAAK8uB,gBAAgB5C,EAAQsD,GAE7B,MAAOtB,QAQf,KAAK1oB,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IAAK,CAC3C0oB,EAAYluB,KAAKmb,WAAW3V,EAC5B,IAAI0M,GAAQgc,EAAU7K,MACtB,IAAInR,EAAO,CACT,GAAIud,GAAQvqB,KAAK2lB,IAAI9Y,EAAIG,EAAMH,GAC3B2d,EAAQxqB,KAAK2lB,IAAI7Y,EAAIE,EAAMF,GAC3BkZ,EAAQhmB,KAAKyqB,KAAKF,EAAQA,EAAQC,EAAQA,IAEzB,OAAhBJ,GAA+BA,EAAPpE,IAA8BkE,EAAPlE,IAClDoE,EAAcpE,EACdmE,EAAmBnB,IAO3B,MAAOmB,IAQTruB,EAAQmS,UAAUib,aAAe,SAAUF,GACzC,GAAI0B,GAASC,EAAMC,CAEd9vB,MAAKomB,SAiCRwJ,EAAU5vB,KAAKomB,QAAQ2J,IAAIH,QAC3BC,EAAQ7vB,KAAKomB,QAAQ2J,IAAIF,KACzBC,EAAQ9vB,KAAKomB,QAAQ2J,IAAID,MAlCzBF,EAAUre,SAASM,cAAc,OACjC+d,EAAQ3iB,MAAM2W,SAAW,WACzBgM,EAAQ3iB,MAAM+W,QAAU,OACxB4L,EAAQ3iB,MAAMb,OAAS,oBACvBwjB,EAAQ3iB,MAAM9B,MAAQ,UACtBykB,EAAQ3iB,MAAMd,WAAa,wBAC3ByjB,EAAQ3iB,MAAM+iB,aAAe,MAC7BJ,EAAQ3iB,MAAMgjB,UAAY,qCAE1BJ,EAAOte,SAASM,cAAc,OAC9Bge,EAAK5iB,MAAM2W,SAAW,WACtBiM,EAAK5iB,MAAMuF,OAAS,OACpBqd,EAAK5iB,MAAMsF,MAAQ,IACnBsd,EAAK5iB,MAAMijB,WAAa,oBAExBJ,EAAMve,SAASM,cAAc,OAC7Bie,EAAI7iB,MAAM2W,SAAW,WACrBkM,EAAI7iB,MAAMuF,OAAS,IACnBsd,EAAI7iB,MAAMsF,MAAQ,IAClBud,EAAI7iB,MAAMb,OAAS,oBACnB0jB,EAAI7iB,MAAM+iB,aAAe,MAEzBhwB,KAAKomB,SACH8H,UAAW,KACX6B,KACEH,QAASA,EACTC,KAAMA,EACNC,IAAKA,KAUX9vB,KAAKiuB,eAELjuB,KAAKomB,QAAQ8H,UAAYA,EAEvB0B,EAAQ3L,UADsB,kBAArBjkB,MAAK6a,YACM7a,KAAK6a,YAAYqT,EAAUhc,OAG3B,6BACMgc,EAAUhc,MAAMH,EAAI,gCACpBmc,EAAUhc,MAAMF,EAAI,gCACpBkc,EAAUhc,MAAMgL,EAAI,qBAIhD0S,EAAQ3iB,MAAMxF,KAAQ,IACtBmoB,EAAQ3iB,MAAMtF,IAAQ,IACtB3H,KAAKsf,MAAM7N,YAAYme,GACvB5vB,KAAKsf,MAAM7N,YAAYoe,GACvB7vB,KAAKsf,MAAM7N,YAAYqe,EAGvB,IAAIK,GAAgBP,EAAQQ,YACxBC,EAAkBT,EAAQU,aAC1BC,EAAgBV,EAAKS,aACrBE,EAAcV,EAAIM,YAClBK,EAAgBX,EAAIQ,aAEpB7oB,EAAOymB,EAAU7K,OAAOtR,EAAIoe,EAAe,CAC/C1oB,GAAOvC,KAAK4G,IAAI5G,KAAKwH,IAAIjF,EAAM,IAAKzH,KAAKsf,MAAME,YAAc,GAAK2Q,GAElEN,EAAK5iB,MAAMxF,KAASymB,EAAU7K,OAAOtR,EAAI,KACzC8d,EAAK5iB,MAAMtF,IAAUumB,EAAU7K,OAAOrR,EAAIue,EAAc,KACxDX,EAAQ3iB,MAAMxF,KAAQA,EAAO,KAC7BmoB,EAAQ3iB,MAAMtF,IAASumB,EAAU7K,OAAOrR,EAAIue,EAAaF,EAAiB,KAC1EP,EAAI7iB,MAAMxF,KAAWymB,EAAU7K,OAAOtR,EAAIye,EAAW,EAAK,KAC1DV,EAAI7iB,MAAMtF,IAAWumB,EAAU7K,OAAOrR,EAAIye,EAAY,EAAK,MAO7DzvB,EAAQmS,UAAU8a,aAAe,WAC/B,GAAIjuB,KAAKomB,QAAS,CAChBpmB,KAAKomB,QAAQ8H,UAAY,IAEzB,KAAK,GAAIroB,KAAQ7F,MAAKomB,QAAQ2J,IAC5B,GAAI/vB,KAAKomB,QAAQ2J,IAAIjqB,eAAeD,GAAO,CACzC,GAAI0B,GAAOvH,KAAKomB,QAAQ2J,IAAIlqB,EACxB0B,IAAQA,EAAKsC,YACftC,EAAKsC,WAAWsH,YAAY5J,MA8BtC1H,EAAOD,QAAUoB,GAKb,SAASnB,EAAQD,EAASM,GAc9B,QAASgB,KACPlB,KAAK0wB,YAAc,GAAIrvB,GACvBrB,KAAK2wB,eACL3wB,KAAK2wB,YAAYnL,WAAa,EAC9BxlB,KAAK2wB,YAAYlL,SAAW,EAC5BzlB,KAAK4wB,UAAY,IAEjB5wB,KAAK6wB,eAAiB,GAAIxvB,GAC1BrB,KAAK8wB,eAAkB,GAAIzvB,GAAQ,GAAI6D,KAAKymB,GAAI,EAAG,GAEnD3rB,KAAK+wB,6BAtBP,GAAI1vB,GAAUnB,EAAoB,GA+BlCgB,GAAOiS,UAAUmK,eAAiB,SAASvL,EAAGC,EAAGkL,GAC/Cld,KAAK0wB,YAAY3e,EAAIA,EACrB/R,KAAK0wB,YAAY1e,EAAIA,EACrBhS,KAAK0wB,YAAYxT,EAAIA,EAErBld,KAAK+wB,8BAWP7vB,EAAOiS,UAAUuS,eAAiB,SAASF,EAAYC,GAClCjf,SAAfgf,IACFxlB,KAAK2wB,YAAYnL,WAAaA,GAGfhf,SAAbif,IACFzlB,KAAK2wB,YAAYlL,SAAWA,EACxBzlB,KAAK2wB,YAAYlL,SAAW,IAAGzlB,KAAK2wB,YAAYlL,SAAW,GAC3DzlB,KAAK2wB,YAAYlL,SAAW,GAAIvgB,KAAKymB,KAAI3rB,KAAK2wB,YAAYlL,SAAW,GAAIvgB,KAAKymB,MAGjEnlB,SAAfgf,GAAyChf,SAAbif,IAC9BzlB,KAAK+wB,8BAQT7vB,EAAOiS,UAAU2S,eAAiB,WAChC,GAAIkL,KAIJ,OAHAA,GAAIxL,WAAaxlB,KAAK2wB,YAAYnL,WAClCwL,EAAIvL,SAAWzlB,KAAK2wB,YAAYlL,SAEzBuL,GAOT9vB,EAAOiS,UAAUyS,aAAe,SAASjgB,GACxBa,SAAXb,IAGJ3F,KAAK4wB,UAAYjrB,EAKb3F,KAAK4wB,UAAY,MAAM5wB,KAAK4wB,UAAY,KACxC5wB,KAAK4wB,UAAY,IAAK5wB,KAAK4wB,UAAY,GAE3C5wB,KAAK+wB,+BAOP7vB,EAAOiS,UAAUiM,aAAe,WAC9B,MAAOpf,MAAK4wB,WAOd1vB,EAAOiS,UAAU6K,kBAAoB,WACnC,MAAOhe,MAAK6wB,gBAOd3vB,EAAOiS,UAAUkL,kBAAoB,WACnC,MAAOre,MAAK8wB,gBAOd5vB,EAAOiS,UAAU4d,2BAA6B,WAE5C/wB,KAAK6wB,eAAe9e,EAAI/R,KAAK0wB,YAAY3e,EAAI/R,KAAK4wB,UAAY1rB,KAAKkZ,IAAIpe,KAAK2wB,YAAYnL,YAActgB,KAAKqZ,IAAIve,KAAK2wB,YAAYlL,UAChIzlB,KAAK6wB,eAAe7e,EAAIhS,KAAK0wB,YAAY1e,EAAIhS,KAAK4wB,UAAY1rB,KAAKqZ,IAAIve,KAAK2wB,YAAYnL,YAActgB,KAAKqZ,IAAIve,KAAK2wB,YAAYlL,UAChIzlB,KAAK6wB,eAAe3T,EAAIld,KAAK0wB,YAAYxT,EAAIld,KAAK4wB,UAAY1rB,KAAKkZ,IAAIpe,KAAK2wB,YAAYlL,UAGxFzlB,KAAK8wB,eAAe/e,EAAI7M,KAAKymB,GAAG,EAAI3rB,KAAK2wB,YAAYlL,SACrDzlB,KAAK8wB,eAAe9e,EAAI,EACxBhS,KAAK8wB,eAAe5T,GAAKld,KAAK2wB,YAAYnL,YAG5C3lB,EAAOD,QAAUsB,GAIb,SAASrB,EAAQD,EAASM,GAW9B,QAASiB,GAAQuR,EAAMqO,EAAQkQ,GAC7BjxB,KAAK0S,KAAOA,EACZ1S,KAAK+gB,OAASA,EACd/gB,KAAKixB,MAAQA,EAEbjxB,KAAKmI,MAAQ3B,OACbxG,KAAKqH,MAAQb,OAGbxG,KAAK8W,OAASma,EAAMjQ,kBAAkBtO,EAAKwC,MAAOlV,KAAK+gB,QAGvD/gB,KAAK8W,OAAOZ,KAAK,SAAU3Q,EAAGa,GAC5B,MAAOb,GAAIa,EAAI,EAAQA,EAAJb,EAAQ,GAAK,IAG9BvF,KAAK8W,OAAOnR,OAAS,GACvB3F,KAAK+oB,YAAY,GAInB/oB,KAAKmb,cAELnb,KAAKM,QAAS,EACdN,KAAKkxB,eAAiB1qB,OAElByqB,EAAMjW,kBACRhb,KAAKM,QAAS,EACdN,KAAKmxB,oBAGLnxB,KAAKM,QAAS,EAxClB,GAAIQ,GAAWZ,EAAoB,EAiDnCiB,GAAOgS,UAAUie,SAAW,WAC1B,MAAOpxB,MAAKM,QAQda,EAAOgS,UAAUke,kBAAoB,WAInC,IAHA,GAAI5rB,GAAMzF,KAAK8W,OAAOnR,OAElBH,EAAI,EACDxF,KAAKmb,WAAW3V,IACrBA,GAGF,OAAON,MAAKwoB,MAAMloB,EAAIC,EAAM,MAQ9BtE,EAAOgS,UAAU+V,SAAW,WAC1B,MAAOlpB,MAAKixB,MAAM7W,aAQpBjZ,EAAOgS,UAAUme,UAAY,WAC3B,MAAOtxB,MAAK+gB,QAOd5f,EAAOgS,UAAUgW,iBAAmB,WAClC,MAAmB3iB,UAAfxG,KAAKmI,MACA3B,OAEFxG,KAAK8W,OAAO9W,KAAKmI,QAO1BhH,EAAOgS,UAAUoe,UAAY,WAC3B,MAAOvxB,MAAK8W,QAQd3V,EAAOgS,UAAUyB,SAAW,SAASzM,GACnC,GAAIA,GAASnI,KAAK8W,OAAOnR,OACvB,KAAM,2BAER,OAAO3F,MAAK8W,OAAO3O,IASrBhH,EAAOgS,UAAU2P,eAAiB,SAAS3a,GAIzC,GAHc3B,SAAV2B,IACFA,EAAQnI,KAAKmI,OAED3B,SAAV2B,EACF,QAEF,IAAIgT,EACJ,IAAInb,KAAKmb,WAAWhT,GAClBgT,EAAanb,KAAKmb,WAAWhT,OAE1B,CACH,GAAIyF,KACJA,GAAEmT,OAAS/gB,KAAK+gB,OAChBnT,EAAEvG,MAAQrH,KAAK8W,OAAO3O,EAEtB,IAAIqpB,GAAW,GAAI1wB,GAASd,KAAK0S,MAAMiB,OAAQ,SAAUtE,GAAO,MAAQA,GAAKzB,EAAEmT,SAAWnT,EAAEvG,SAAW6N,KACvGiG,GAAanb,KAAKixB,MAAMnO,eAAe0O,GAEvCxxB,KAAKmb,WAAWhT,GAASgT,EAG3B,MAAOA,IAQTha,EAAOgS,UAAUqO,kBAAoB,SAASlZ,GAC5CtI,KAAKkxB,eAAiB5oB,GASxBnH,EAAOgS,UAAU4V,YAAc,SAAS5gB,GACtC,GAAIA,GAASnI,KAAK8W,OAAOnR,OACvB,KAAM,2BAER3F,MAAKmI,MAAQA,EACbnI,KAAKqH,MAAQrH,KAAK8W,OAAO3O,IAO3BhH,EAAOgS,UAAUge,iBAAmB,SAAShpB,GAC7B3B,SAAV2B,IACFA,EAAQ,EAEV,IAAImX,GAAQtf,KAAKixB,MAAM3R,KAEvB,IAAInX,EAAQnI,KAAK8W,OAAOnR,OAAQ,CAC9B,CAAqB3F,KAAK8iB,eAAe3a,GAIlB3B,SAAnB8Y,EAAMmS,WACRnS,EAAMmS,SAAWlgB,SAASM,cAAc,OACxCyN,EAAMmS,SAASxkB,MAAM2W,SAAW,WAChCtE,EAAMmS,SAASxkB,MAAM9B,MAAQ,OAC7BmU,EAAM7N,YAAY6N,EAAMmS,UAE1B,IAAIA,GAAWzxB,KAAKqxB,mBACpB/R,GAAMmS,SAASxN,UAAY,wBAA0BwN,EAAW,IAEhEnS,EAAMmS,SAASxkB,MAAMqW,OAAS,OAC9BhE,EAAMmS,SAASxkB,MAAMxF,KAAO,MAE5B,IAAI0M,GAAKnU,IACTsZ,YAAW,WAAYnF,EAAGgd,iBAAiBhpB,EAAM,IAAM,IACvDnI,KAAKM,QAAS,MAGdN,MAAKM,QAAS,EAGSkG,SAAnB8Y,EAAMmS,WACRnS,EAAMnO,YAAYmO,EAAMmS,UACxBnS,EAAMmS,SAAWjrB,QAGfxG,KAAKkxB,gBACPlxB,KAAKkxB,kBAIXrxB,EAAOD,QAAUuB,GAKb,SAAStB,GAOb,QAASuB,GAAS2Q,EAAGC,GACnBhS,KAAK+R,EAAUvL,SAANuL,EAAkBA,EAAI,EAC/B/R,KAAKgS,EAAUxL,SAANwL,EAAkBA,EAAI,EAGjCnS,EAAOD,QAAUwB,GAKb,SAASvB,GAQb,QAASwB,GAAQ0Q,EAAGC,EAAGkL,GACrBld,KAAK+R,EAAUvL,SAANuL,EAAkBA,EAAI,EAC/B/R,KAAKgS,EAAUxL,SAANwL,EAAkBA,EAAI,EAC/BhS,KAAKkd,EAAU1W,SAAN0W,EAAkBA,EAAI,EASjC7b,EAAQgqB,SAAW,SAAS9lB,EAAGa,GAC7B,GAAIsrB,GAAM,GAAIrwB,EAId,OAHAqwB,GAAI3f,EAAIxM,EAAEwM,EAAI3L,EAAE2L,EAChB2f,EAAI1f,EAAIzM,EAAEyM,EAAI5L,EAAE4L,EAChB0f,EAAIxU,EAAI3X,EAAE2X,EAAI9W,EAAE8W,EACTwU,GASTrwB,EAAQ4R,IAAM,SAAS1N,EAAGa,GACxB,GAAIurB,GAAM,GAAItwB,EAId;MAHAswB,GAAI5f,EAAIxM,EAAEwM,EAAI3L,EAAE2L,EAChB4f,EAAI3f,EAAIzM,EAAEyM,EAAI5L,EAAE4L,EAChB2f,EAAIzU,EAAI3X,EAAE2X,EAAI9W,EAAE8W,EACTyU,GASTtwB,EAAQ8qB,IAAM,SAAS5mB,EAAGa,GACxB,MAAO,IAAI/E,IACFkE,EAAEwM,EAAI3L,EAAE2L,GAAK,GACbxM,EAAEyM,EAAI5L,EAAE4L,GAAK,GACbzM,EAAE2X,EAAI9W,EAAE8W,GAAK,IAWxB7b,EAAQmqB,aAAe,SAASjmB,EAAGa,GACjC,GAAImlB,GAAe,GAAIlqB,EAMvB,OAJAkqB,GAAaxZ,EAAIxM,EAAEyM,EAAI5L,EAAE8W,EAAI3X,EAAE2X,EAAI9W,EAAE4L,EACrCuZ,EAAavZ,EAAIzM,EAAE2X,EAAI9W,EAAE2L,EAAIxM,EAAEwM,EAAI3L,EAAE8W,EACrCqO,EAAarO,EAAI3X,EAAEwM,EAAI3L,EAAE4L,EAAIzM,EAAEyM,EAAI5L,EAAE2L,EAE9BwZ,GAQTlqB,EAAQ8R,UAAUxN,OAAS,WACzB,MAAOT,MAAKyqB,KACJ3vB,KAAK+R,EAAI/R,KAAK+R,EACd/R,KAAKgS,EAAIhS,KAAKgS,EACdhS,KAAKkd,EAAIld,KAAKkd,IAIxBrd,EAAOD,QAAUyB,GAKb,SAASxB,EAAQD,EAASM,GAa9B,QAASoB,GAAOiY,EAAW9K,GACzB,GAAkBjI,SAAd+S,EACF,KAAM,qCAKR,IAHAvZ,KAAKuZ,UAAYA,EACjBvZ,KAAK0oB,QAAWja,GAA8BjI,QAAnBiI,EAAQia,QAAwBja,EAAQia,SAAU,EAEzE1oB,KAAK0oB,QAAS,CAChB1oB,KAAKsf,MAAQ/N,SAASM,cAAc,OAEpC7R,KAAKsf,MAAMrS,MAAMsF,MAAQ,OACzBvS,KAAKsf,MAAMrS,MAAM2W,SAAW,WAC5B5jB,KAAKuZ,UAAU9H,YAAYzR,KAAKsf,OAEhCtf,KAAKsf,MAAMsS,KAAOrgB,SAASM,cAAc,SACzC7R,KAAKsf,MAAMsS,KAAK9qB,KAAO,SACvB9G,KAAKsf,MAAMsS,KAAKvqB,MAAQ,OACxBrH,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAMsS,MAElC5xB,KAAKsf,MAAM0F,KAAOzT,SAASM,cAAc,SACzC7R,KAAKsf,MAAM0F,KAAKle,KAAO,SACvB9G,KAAKsf,MAAM0F,KAAK3d,MAAQ,OACxBrH,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAM0F,MAElChlB,KAAKsf,MAAM+I,KAAO9W,SAASM,cAAc,SACzC7R,KAAKsf,MAAM+I,KAAKvhB,KAAO,SACvB9G,KAAKsf,MAAM+I,KAAKhhB,MAAQ,OACxBrH,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAM+I,MAElCroB,KAAKsf,MAAMuS,IAAMtgB,SAASM,cAAc,SACxC7R,KAAKsf,MAAMuS,IAAI/qB,KAAO,SACtB9G,KAAKsf,MAAMuS,IAAI5kB,MAAM2W,SAAW,WAChC5jB,KAAKsf,MAAMuS,IAAI5kB,MAAMb,OAAS,gBAC9BpM,KAAKsf,MAAMuS,IAAI5kB,MAAMsF,MAAQ,QAC7BvS,KAAKsf,MAAMuS,IAAI5kB,MAAMuF,OAAS,MAC9BxS,KAAKsf,MAAMuS,IAAI5kB,MAAM+iB,aAAe,MACpChwB,KAAKsf,MAAMuS,IAAI5kB,MAAM6kB,gBAAkB,MACvC9xB,KAAKsf,MAAMuS,IAAI5kB,MAAMb,OAAS,oBAC9BpM,KAAKsf,MAAMuS,IAAI5kB,MAAM0S,gBAAkB,UACvC3f,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAMuS,KAElC7xB,KAAKsf,MAAMyS,MAAQxgB,SAASM,cAAc,SAC1C7R,KAAKsf,MAAMyS,MAAMjrB,KAAO,SACxB9G,KAAKsf,MAAMyS,MAAM9kB,MAAMyM,OAAS,MAChC1Z,KAAKsf,MAAMyS,MAAM1qB,MAAQ,IACzBrH,KAAKsf,MAAMyS,MAAM9kB,MAAM2W,SAAW,WAClC5jB,KAAKsf,MAAMyS,MAAM9kB,MAAMxF,KAAO,SAC9BzH,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAMyS,MAGlC,IAAI5d,GAAKnU,IACTA,MAAKsf,MAAMyS,MAAM7N,YAAc,SAAU5a,GAAQ6K,EAAGgQ,aAAa7a,IACjEtJ,KAAKsf,MAAMsS,KAAKI,QAAU,SAAU1oB,GAAQ6K,EAAGyd,KAAKtoB,IACpDtJ,KAAKsf,MAAM0F,KAAKgN,QAAU,SAAU1oB,GAAQ6K,EAAG8d,WAAW3oB,IAC1DtJ,KAAKsf,MAAM+I,KAAK2J,QAAU,SAAU1oB,GAAQ6K,EAAGkU,KAAK/e,IAGtDtJ,KAAKkyB,iBAAmB1rB,OAExBxG,KAAK8W,UACL9W,KAAKmI,MAAQ3B,OAEbxG,KAAKmyB,YAAc3rB,OACnBxG,KAAKoyB,aAAe,IACpBpyB,KAAKqyB,UAAW,EA3ElB,GAAI1xB,GAAOT,EAAoB,EAiF/BoB,GAAO6R,UAAUye,KAAO,WACtB,GAAIzpB,GAAQnI,KAAK8oB,UACb3gB,GAAQ,IACVA,IACAnI,KAAKsyB,SAASnqB,KAOlB7G,EAAO6R,UAAUkV,KAAO,WACtB,GAAIlgB,GAAQnI,KAAK8oB,UACb3gB,GAAQnI,KAAK8W,OAAOnR,OAAS,IAC/BwC,IACAnI,KAAKsyB,SAASnqB,KAOlB7G,EAAO6R,UAAUof,SAAW,WAC1B,GAAI3iB,GAAQ,GAAItL,MAEZ6D,EAAQnI,KAAK8oB,UACb3gB,GAAQnI,KAAK8W,OAAOnR,OAAS,GAC/BwC,IACAnI,KAAKsyB,SAASnqB,IAEPnI,KAAKqyB,WAEZlqB,EAAQ,EACRnI,KAAKsyB,SAASnqB,GAGhB,IAAI0H,GAAM,GAAIvL,MACV+nB,EAAQxc,EAAMD,EAId4iB,EAAWttB,KAAKwH,IAAI1M,KAAKoyB,aAAe/F,EAAM,GAG9ClY,EAAKnU,IACTA,MAAKmyB,YAAc7Y,WAAW,WAAYnF,EAAGoe,YAAcC,IAM7DlxB,EAAO6R,UAAU8e,WAAa,WACHzrB,SAArBxG,KAAKmyB,YACPnyB,KAAKglB,OAELhlB,KAAKklB,QAOT5jB,EAAO6R,UAAU6R,KAAO,WAElBhlB,KAAKmyB,cAETnyB,KAAKuyB,WAEDvyB,KAAKsf,QACPtf,KAAKsf,MAAM0F,KAAK3d,MAAQ,UAO5B/F,EAAO6R,UAAU+R,KAAO,WACtBuN,cAAczyB,KAAKmyB,aACnBnyB,KAAKmyB,YAAc3rB,OAEfxG,KAAKsf,QACPtf,KAAKsf,MAAM0F,KAAK3d,MAAQ,SAQ5B/F,EAAO6R,UAAU6V,oBAAsB,SAAS1gB,GAC9CtI,KAAKkyB,iBAAmB5pB,GAO1BhH,EAAO6R,UAAUyV,gBAAkB,SAAS4J,GAC1CxyB,KAAKoyB,aAAeI,GAOtBlxB,EAAO6R,UAAUuf,gBAAkB,WACjC,MAAO1yB,MAAKoyB,cASd9wB,EAAO6R,UAAUwf,YAAc,SAASC,GACtC5yB,KAAKqyB,SAAWO,GAOlBtxB,EAAO6R,UAAU0f,SAAW,WACIrsB,SAA1BxG,KAAKkyB,kBACPlyB,KAAKkyB,oBAOT5wB,EAAO6R,UAAUsO,OAAS,WACxB,GAAIzhB,KAAKsf,MAAO,CAEdtf,KAAKsf,MAAMuS,IAAI5kB,MAAMtF,IAAO3H,KAAKsf,MAAMuF,aAAa,EAChD7kB,KAAKsf,MAAMuS,IAAIvB,aAAa,EAAK,KACrCtwB,KAAKsf,MAAMuS,IAAI5kB,MAAMsF,MAASvS,KAAKsf,MAAME,YACrCxf,KAAKsf,MAAMsS,KAAKpS,YAChBxf,KAAKsf,MAAM0F,KAAKxF,YAChBxf,KAAKsf,MAAM+I,KAAK7I,YAAc,GAAO,IAGzC,IAAI/X,GAAOzH,KAAK8yB,YAAY9yB,KAAKmI,MACjCnI,MAAKsf,MAAMyS,MAAM9kB,MAAMxF,KAAO,EAAS,OAS3CnG,EAAO6R,UAAUwV,UAAY,SAAS7R,GACpC9W,KAAK8W,OAASA,EAEV9W,KAAK8W,OAAOnR,OAAS,EACvB3F,KAAKsyB,SAAS,GAEdtyB,KAAKmI,MAAQ3B,QAOjBlF,EAAO6R,UAAUmf,SAAW,SAASnqB,GACnC,KAAIA,EAAQnI,KAAK8W,OAAOnR,QAOtB,KAAM,2BANN3F,MAAKmI,MAAQA,EAEbnI,KAAKyhB,SACLzhB,KAAK6yB,YAWTvxB,EAAO6R,UAAU2V,SAAW,WAC1B,MAAO9oB,MAAKmI,OAQd7G,EAAO6R,UAAU+B,IAAM,WACrB,MAAOlV,MAAK8W,OAAO9W,KAAKmI,QAI1B7G,EAAO6R,UAAUgR,aAAe,SAAS7a,GAEvC,GAAIgjB,GAAiBhjB,EAAMkjB,MAAyB,IAAhBljB,EAAMkjB,MAAiC,IAAjBljB,EAAMmjB,MAChE,IAAKH,EAAL,CAEAtsB,KAAK+yB,aAAezpB,EAAMqT,QAC1B3c,KAAKgzB,YAAc3N,WAAWrlB,KAAKsf,MAAMyS,MAAM9kB,MAAMxF,MAErDzH,KAAKsf,MAAMrS,MAAM+f,OAAS,MAK1B,IAAI7Y,GAAKnU,IACTA,MAAKitB,YAAc,SAAU3jB,GAAQ6K,EAAG+Y,aAAa5jB,IACrDtJ,KAAKmtB,UAAc,SAAU7jB,GAAQ6K,EAAGoY,WAAWjjB,IACnD3I,EAAKgI,iBAAiB4I,SAAU,YAAavR,KAAKitB,aAClDtsB,EAAKgI,iBAAiB4I,SAAU,UAAavR,KAAKmtB,WAClDxsB,EAAK0I,eAAeC,KAItBhI,EAAO6R,UAAU8f,YAAc,SAAUxrB,GACvC,GAAI8K,GAAQ8S,WAAWrlB,KAAKsf,MAAMuS,IAAI5kB,MAAMsF,OACxCvS,KAAKsf,MAAMyS,MAAMvS,YAAc,GAC/BzN,EAAItK,EAAO,EAEXU,EAAQjD,KAAKwoB,MAAM3b,EAAIQ,GAASvS,KAAK8W,OAAOnR,OAAO,GAIvD,OAHY,GAARwC,IAAWA,EAAQ,GACnBA,EAAQnI,KAAK8W,OAAOnR,OAAO,IAAGwC,EAAQnI,KAAK8W,OAAOnR,OAAO,GAEtDwC,GAGT7G,EAAO6R,UAAU2f,YAAc,SAAU3qB,GACvC,GAAIoK,GAAQ8S,WAAWrlB,KAAKsf,MAAMuS,IAAI5kB,MAAMsF,OACxCvS,KAAKsf,MAAMyS,MAAMvS,YAAc,GAE/BzN,EAAI5J,GAASnI,KAAK8W,OAAOnR,OAAO,GAAK4M,EACrC9K,EAAOsK,EAAI,CAEf,OAAOtK,IAKTnG,EAAO6R,UAAU+Z,aAAe,SAAU5jB,GACxC,GAAI+iB,GAAO/iB,EAAMqT,QAAU3c,KAAK+yB,aAC5BhhB,EAAI/R,KAAKgzB,YAAc3G,EAEvBlkB,EAAQnI,KAAKizB,YAAYlhB,EAE7B/R,MAAKsyB,SAASnqB,GAEdxH,EAAK0I,kBAIP/H,EAAO6R,UAAUoZ,WAAa,WAC5BvsB,KAAKsf,MAAMrS,MAAM+f,OAAS,OAG1BrsB,EAAKwI,oBAAoBoI,SAAU,YAAavR,KAAKitB,aACrDtsB,EAAKwI,oBAAoBoI,SAAU,UAAWvR,KAAKmtB,WAEnDxsB,EAAK0I,kBAGPxJ,EAAOD,QAAU0B,GAKb,SAASzB,GA2Bb,QAAS0B,GAAWqO,EAAOC,EAAKsY,EAAMmB,GAEpCtpB,KAAKkzB,OAAS,EACdlzB,KAAKmzB,KAAO,EACZnzB,KAAKozB,MAAQ,EACbpzB,KAAKspB,YAAa,EAClBtpB,KAAKqzB,UAAY,EAEjBrzB,KAAKszB,SAAW,EAChBtzB,KAAKuzB,SAAS3jB,EAAOC,EAAKsY,EAAMmB,GAYlC/nB,EAAW4R,UAAUogB,SAAW,SAAS3jB,EAAOC,EAAKsY,EAAMmB,GACzDtpB,KAAKkzB,OAAStjB,EAAQA,EAAQ,EAC9B5P,KAAKmzB,KAAOtjB,EAAMA,EAAM,EAExB7P,KAAKwzB,QAAQrL,EAAMmB,IASrB/nB,EAAW4R,UAAUqgB,QAAU,SAASrL,EAAMmB,GAC/B9iB,SAAT2hB,GAA8B,GAARA,IAGP3hB,SAAf8iB,IACFtpB,KAAKspB,WAAaA,GAGlBtpB,KAAKozB,MADHpzB,KAAKspB,cAAe,EACT/nB,EAAWkyB,oBAAoBtL,GAE/BA,IAUjB5mB,EAAWkyB,oBAAsB,SAAUtL,GACzC,GAAIuL,GAAQ,SAAU3hB,GAAI,MAAO7M,MAAKyuB,IAAI5hB,GAAK7M,KAAK0uB,MAGhDC,EAAQ3uB,KAAK4uB,IAAI,GAAI5uB,KAAKwoB,MAAMgG,EAAMvL,KACtC4L,EAAQ,EAAI7uB,KAAK4uB,IAAI,GAAI5uB,KAAKwoB,MAAMgG,EAAMvL,EAAO,KACjD6L,EAAQ,EAAI9uB,KAAK4uB,IAAI,GAAI5uB,KAAKwoB,MAAMgG,EAAMvL,EAAO,KAGjDmB,EAAauK,CASjB,OARI3uB,MAAK2lB,IAAIkJ,EAAQ5L,IAASjjB,KAAK2lB,IAAIvB,EAAanB,KAAOmB,EAAayK,GACpE7uB,KAAK2lB,IAAImJ,EAAQ7L,IAASjjB,KAAK2lB,IAAIvB,EAAanB,KAAOmB,EAAa0K,GAGtD,GAAd1K,IACFA,EAAa,GAGRA,GAOT/nB,EAAW4R,UAAUiV,WAAa,WAChC,MAAO/C,YAAWrlB,KAAKszB,SAASW,YAAYj0B,KAAKqzB,aAOnD9xB,EAAW4R,UAAU+gB,QAAU,WAC7B,MAAOl0B,MAAKozB,OAOd7xB,EAAW4R,UAAUvD,MAAQ,WAC3B5P,KAAKszB,SAAWtzB,KAAKkzB,OAASlzB,KAAKkzB,OAASlzB,KAAKozB,OAMnD7xB,EAAW4R,UAAUkV,KAAO,WAC1BroB,KAAKszB,UAAYtzB,KAAKozB,OAOxB7xB,EAAW4R,UAAUtD,IAAM,WACzB,MAAQ7P,MAAKszB,SAAWtzB,KAAKmzB,MAG/BtzB,EAAOD,QAAU2B,GAKb,SAAS1B,EAAQD,EAASM,GAuB9B,QAASsB,GAAU+X,EAAWtX,EAAOkyB,EAAQ1lB,GAC3C,KAAMzO,eAAgBwB,IACpB,KAAM,IAAIgY,aAAY,mDAIxB,MAAMvT,MAAMC,QAAQiuB,IAAWA,YAAkBtzB,KAAYszB,YAAkB5tB,QAAQ,CACrF,GAAI6tB,GAAgB3lB,CACpBA,GAAU0lB,EACVA,EAASC,EAGX,GAAIjgB,GAAKnU,IACTA,MAAKq0B,gBACHzkB,MAAO,KACPC,IAAO,KAEPykB,YAAY,EAEZC,YAAa,SACbhiB,MAAO,KACPC,OAAQ,KACRgiB,UAAW,KACXC,UAAW,MAEbz0B,KAAKyO,QAAU9N,EAAK8F,cAAezG,KAAKq0B,gBAGxCr0B,KAAK00B,QAAQnb,GAGbvZ,KAAKgC,cAELhC,KAAK20B,MACH5E,IAAK/vB,KAAK+vB,IACV6E,SAAU50B,KAAKgG,MACf6uB,SACEthB,GAAIvT,KAAKuT,GAAGuhB,KAAK90B,MACjB0T,IAAK1T,KAAK0T,IAAIohB,KAAK90B,MACnB4tB,KAAM5tB,KAAK4tB,KAAKkH,KAAK90B,OAEvB+0B,eACAp0B,MACEq0B,KAAM,KACNC,SAAU9gB,EAAG+gB,UAAUJ,KAAK3gB,GAC5BghB,eAAgBhhB,EAAGihB,gBAAgBN,KAAK3gB,GACxCkhB,OAAQlhB,EAAGmhB,QAAQR,KAAK3gB,GACxBohB,aAAephB,EAAGqhB,cAAcV,KAAK3gB,KAKzCnU,KAAKy1B,MAAQ,GAAI5zB,GAAM7B,KAAK20B,MAC5B30B,KAAKgC,WAAWgG,KAAKhI,KAAKy1B,OAC1Bz1B,KAAK20B,KAAKc,MAAQz1B,KAAKy1B,MAGvBz1B,KAAK01B,SAAW,GAAIzyB,GAASjD,KAAK20B,MAClC30B,KAAKgC,WAAWgG,KAAKhI,KAAK01B,UAC1B11B,KAAK20B,KAAKh0B,KAAKq0B,KAAOh1B,KAAK01B,SAASV,KAAKF,KAAK90B,KAAK01B,UAGnD11B,KAAK21B,YAAc,GAAInzB,GAAYxC,KAAK20B,MACxC30B,KAAKgC,WAAWgG,KAAKhI,KAAK21B,aAI1B31B,KAAK41B,WAAa,GAAInzB,GAAWzC,KAAK20B,MACtC30B,KAAKgC,WAAWgG,KAAKhI,KAAK41B,YAG1B51B,KAAK61B,QAAU,GAAI/yB,GAAQ9C,KAAK20B,MAChC30B,KAAKgC,WAAWgG,KAAKhI,KAAK61B,SAE1B71B,KAAK81B,UAAY,KACjB91B,KAAK+1B,WAAa,KAGdtnB,GACFzO,KAAKkT,WAAWzE,GAId0lB,GACFn0B,KAAKg2B,UAAU7B,GAIblyB,EACFjC,KAAKi2B,SAASh0B,GAGdjC,KAAKyhB,SAjHT,GAEI9gB,IAFUT,EAAoB,IACrBA,EAAoB,IACtBA,EAAoB,IAC3BW,EAAUX,EAAoB,GAC9BY,EAAWZ,EAAoB,GAC/B2B,EAAQ3B,EAAoB,IAC5Bg2B,EAAOh2B,EAAoB,IAC3B+C,EAAW/C,EAAoB,IAC/BsC,EAActC,EAAoB,IAClCuC,EAAavC,EAAoB,IACjC4C,EAAU5C,EAAoB,GA4GlCsB,GAAS2R,UAAY,GAAI+iB,GAMzB10B,EAAS2R,UAAU8iB,SAAW,SAASh0B,GACrC,GAGIk0B,GAHAC,EAAiC,MAAlBp2B,KAAK81B,SAwBxB,IAhBEK,EAJGl0B,EAGIA,YAAiBpB,IAAWoB,YAAiBnB,GACvCmB,EAIA,GAAIpB,GAAQoB,GACvB6E,MACE8I,MAAO,OACPC,IAAK,UAVI,KAgBf7P,KAAK81B,UAAYK,EACjBn2B,KAAK61B,SAAW71B,KAAK61B,QAAQI,SAASE,GAElCC,EACF,GAA0B5vB,QAAtBxG,KAAKyO,QAAQmB,OAA0CpJ,QAApBxG,KAAKyO,QAAQoB,IAAkB,CACpE,GAA0BrJ,QAAtBxG,KAAKyO,QAAQmB,OAA0CpJ,QAApBxG,KAAKyO,QAAQoB,IAClD,GAAIwmB,GAAYr2B,KAAKs2B,eAGvB,IAAI1mB,GAA8BpJ,QAAtBxG,KAAKyO,QAAQmB,MAAqB5P,KAAKyO,QAAQmB,MAAQymB,EAAUzmB,MACzEC,EAA4BrJ,QAApBxG,KAAKyO,QAAQoB,IAAqB7P,KAAKyO,QAAQoB,IAAQwmB,EAAUxmB,GAE7E7P,MAAKu2B,UAAU3mB,EAAOC,GAAM2mB,SAAS,QAGrCx2B,MAAKy2B,KAAKD,SAAS,KASzBh1B,EAAS2R,UAAU6iB,UAAY,SAAS7B,GAEtC,GAAIgC,EAKFA,GAJGhC,EAGIA,YAAkBtzB,IAAWszB,YAAkBrzB,GACzCqzB,EAIA,GAAItzB,GAAQszB,GAPZ,KAUfn0B,KAAK+1B,WAAaI,EAClBn2B,KAAK61B,QAAQG,UAAUG,IAmBzB30B,EAAS2R,UAAUujB,aAAe,SAASvhB,EAAK1G,GAC9CzO,KAAK61B,SAAW71B,KAAK61B,QAAQa,aAAavhB,GAEtC1G,GAAWA,EAAQkoB,OACrB32B,KAAK22B,MAAMxhB,EAAK1G,IAQpBjN,EAAS2R,UAAUyjB,aAAe,WAChC,MAAO52B,MAAK61B,SAAW71B,KAAK61B,QAAQe,oBAetCp1B,EAAS2R,UAAUwjB,MAAQ,SAASt2B,EAAIoO,GACtC,GAAKzO,KAAK81B,WAAmBtvB,QAANnG,EAAvB,CAEA,GAAI8U,GAAMlP,MAAMC,QAAQ7F,GAAMA,GAAMA,GAGhCy1B,EAAY91B,KAAK81B,UAAUhgB,aAAaZ,IAAIC,GAC9CrO,MACE8I,MAAO,OACPC,IAAK,UAKLD,EAAQ,KACRC,EAAM,IAcV,IAbAimB,EAAUztB,QAAQ,SAAUwuB,GAC1B,GAAIjrB,GAAIirB,EAASjnB,MAAM5I,UACnB8vB,EAAI,OAASD,GAAWA,EAAShnB,IAAI7I,UAAY6vB,EAASjnB,MAAM5I,WAEtD,OAAV4I,GAAsBA,EAAJhE,KACpBgE,EAAQhE,IAGE,OAARiE,GAAgBinB,EAAIjnB,KACtBA,EAAMinB,KAII,OAAVlnB,GAA0B,OAARC,EAAc,CAElC,GAAIT,IAAUQ,EAAQC,GAAO,EACzB2iB,EAAWttB,KAAKwH,IAAK1M,KAAKy1B,MAAM5lB,IAAM7P,KAAKy1B,MAAM7lB,MAAwB,KAAfC,EAAMD,IAEhE4mB,EAAW/nB,GAA+BjI,SAApBiI,EAAQ+nB,QAAyB/nB,EAAQ+nB,SAAU,CAC7Ex2B,MAAKy1B,MAAMlC,SAASnkB,EAASojB,EAAW,EAAGpjB,EAASojB,EAAW,EAAGgE,MAUtEh1B,EAAS2R,UAAU4jB,aAAe,WAEhC,GAAIC,GAAUh3B,KAAK81B,UAAUhgB,aAC3BhK,EAAM,KACNY,EAAM,IAER,IAAIsqB,EAAS,CAEX,GAAIC,GAAUD,EAAQlrB,IAAI,QAC1BA,GAAMmrB,EAAUt2B,EAAKkG,QAAQowB,EAAQrnB,MAAO,QAAQ5I,UAAY,IAKhE,IAAIkwB,GAAeF,EAAQtqB,IAAI,QAC3BwqB,KACFxqB,EAAM/L,EAAKkG,QAAQqwB,EAAatnB,MAAO,QAAQ5I,UAEjD,IAAImwB,GAAaH,EAAQtqB,IAAI,MACzByqB,KAEAzqB,EADS,MAAPA,EACI/L,EAAKkG,QAAQswB,EAAWtnB,IAAK,QAAQ7I,UAGrC9B,KAAKwH,IAAIA,EAAK/L,EAAKkG,QAAQswB,EAAWtnB,IAAK,QAAQ7I,YAK/D,OACE8E,IAAa,MAAPA,EAAe,GAAIxH,MAAKwH,GAAO,KACrCY,IAAa,MAAPA,EAAe,GAAIpI,MAAKoI,GAAO,OAKzC7M,EAAOD,QAAU4B,GAKb,SAAS3B,EAAQD,EAASM,GAsB9B,QAASuB,GAAS8X,EAAWtX,EAAOkyB,EAAQ1lB,GAE1C,KAAMxI,MAAMC,QAAQiuB,IAAWA,YAAkBtzB,KAAYszB,YAAkB5tB,QAAQ,CACrF,GAAI6tB,GAAgB3lB,CACpBA,GAAU0lB,EACVA,EAASC,EAGX,GAAIjgB,GAAKnU,IACTA,MAAKq0B,gBACHzkB,MAAO,KACPC,IAAO,KAEPykB,YAAY,EAEZC,YAAa,SACbhiB,MAAO,KACPC,OAAQ,KACRgiB,UAAW,KACXC,UAAW,MAEbz0B,KAAKyO,QAAU9N,EAAK8F,cAAezG,KAAKq0B,gBAGxCr0B,KAAK00B,QAAQnb,GAGbvZ,KAAKgC,cAELhC,KAAK20B,MACH5E,IAAK/vB,KAAK+vB,IACV6E,SAAU50B,KAAKgG,MACf6uB,SACEthB,GAAIvT,KAAKuT,GAAGuhB,KAAK90B,MACjB0T,IAAK1T,KAAK0T,IAAIohB,KAAK90B,MACnB4tB,KAAM5tB,KAAK4tB,KAAKkH,KAAK90B,OAEvB+0B,eACAp0B,MACEq0B,KAAM,KACNC,SAAU9gB,EAAG+gB,UAAUJ,KAAK3gB,GAC5BghB,eAAgBhhB,EAAGihB,gBAAgBN,KAAK3gB,GACxCkhB,OAAQlhB,EAAGmhB,QAAQR,KAAK3gB,GACxBohB,aAAephB,EAAGqhB,cAAcV,KAAK3gB,KAKzCnU,KAAKy1B,MAAQ,GAAI5zB,GAAM7B,KAAK20B,MAC5B30B,KAAKgC,WAAWgG,KAAKhI,KAAKy1B,OAC1Bz1B,KAAK20B,KAAKc,MAAQz1B,KAAKy1B,MAGvBz1B,KAAK01B,SAAW,GAAIzyB,GAASjD,KAAK20B,MAClC30B,KAAKgC,WAAWgG,KAAKhI,KAAK01B,UAC1B11B,KAAK20B,KAAKh0B,KAAKq0B,KAAOh1B,KAAK01B,SAASV,KAAKF,KAAK90B,KAAK01B,UAGnD11B,KAAK21B,YAAc,GAAInzB,GAAYxC,KAAK20B,MACxC30B,KAAKgC,WAAWgG,KAAKhI,KAAK21B,aAI1B31B,KAAK41B,WAAa,GAAInzB,GAAWzC,KAAK20B,MACtC30B,KAAKgC,WAAWgG,KAAKhI,KAAK41B,YAG1B51B,KAAKo3B,UAAY,GAAIp0B,GAAUhD,KAAK20B,MACpC30B,KAAKgC,WAAWgG,KAAKhI,KAAKo3B,WAE1Bp3B,KAAK81B,UAAY,KACjB91B,KAAK+1B,WAAa,KAGdtnB,GACFzO,KAAKkT,WAAWzE,GAId0lB,GACFn0B,KAAKg2B,UAAU7B,GAIblyB,EACFjC,KAAKi2B,SAASh0B,GAGdjC,KAAKyhB,SA5GT,GAEI9gB,IAFUT,EAAoB,IACrBA,EAAoB,IACtBA,EAAoB,IAC3BW,EAAUX,EAAoB,GAC9BY,EAAWZ,EAAoB,GAC/B2B,EAAQ3B,EAAoB,IAC5Bg2B,EAAOh2B,EAAoB,IAC3B+C,EAAW/C,EAAoB,IAC/BsC,EAActC,EAAoB,IAClCuC,EAAavC,EAAoB,IACjC8C,EAAY9C,EAAoB,GAuGpCuB,GAAQ0R,UAAY,GAAI+iB,GAMxBz0B,EAAQ0R,UAAU8iB,SAAW,SAASh0B,GACpC,GAGIk0B,GAHAC,EAAiC,MAAlBp2B,KAAK81B,SAwBxB,IAhBEK,EAJGl0B,EAGIA,YAAiBpB,IAAWoB,YAAiBnB,GACvCmB,EAIA,GAAIpB,GAAQoB,GACvB6E,MACE8I,MAAO,OACPC,IAAK,UAVI,KAgBf7P,KAAK81B,UAAYK,EACjBn2B,KAAKo3B,WAAap3B,KAAKo3B,UAAUnB,SAASE,GAEtCC,EACF,GAA0B5vB,QAAtBxG,KAAKyO,QAAQmB,OAA0CpJ,QAApBxG,KAAKyO,QAAQoB,IAAkB,CACpE,GAAID,GAA8BpJ,QAAtBxG,KAAKyO,QAAQmB,MAAqB5P,KAAKyO,QAAQmB,MAAQ,KAC/DC,EAA4BrJ,QAApBxG,KAAKyO,QAAQoB,IAAqB7P,KAAKyO,QAAQoB,IAAM,IAEjE7P,MAAKu2B,UAAU3mB,EAAOC,GAAM2mB,SAAS,QAGrCx2B,MAAKy2B,KAAKD,SAAS,KASzB/0B,EAAQ0R,UAAU6iB,UAAY,SAAS7B,GAErC,GAAIgC,EAKFA,GAJGhC,EAGIA,YAAkBtzB,IAAWszB,YAAkBrzB,GACzCqzB,EAIA,GAAItzB,GAAQszB,GAPZ,KAUfn0B,KAAK+1B,WAAaI,EAClBn2B,KAAKo3B,UAAUpB,UAAUG,IAS3B10B,EAAQ0R,UAAUkkB,UAAY,SAASC,EAAS/kB,EAAOC,GAGrD,MAFehM,UAAX+L,IAAuBA,EAAS,IACrB/L,SAAXgM,IAAuBA,EAAS,IACGhM,SAAnCxG,KAAKo3B,UAAUjD,OAAOmD,GACjBt3B,KAAKo3B,UAAUjD,OAAOmD,GAASD,UAAU9kB,EAAMC,GAG/C,qBAAwB8kB,GASnC71B,EAAQ0R,UAAUokB,eAAiB,SAASD,GAC1C,MAAuC9wB,UAAnCxG,KAAKo3B,UAAUjD,OAAOmD,GAChBt3B,KAAKo3B,UAAUjD,OAAOmD,GAAS5O,UAAkEliB,SAAtDxG,KAAKo3B,UAAU3oB,QAAQ0lB,OAAOqD,WAAWF,IAA+E,GAArDt3B,KAAKo3B,UAAU3oB,QAAQ0lB,OAAOqD,WAAWF,KAGxJ,GAWX71B,EAAQ0R,UAAU4jB,aAAe,WAC/B,GAAIjrB,GAAM,KACNY,EAAM,IAGV,KAAK,GAAI4qB,KAAWt3B,MAAKo3B,UAAUjD,OACjC,GAAIn0B,KAAKo3B,UAAUjD,OAAOruB,eAAewxB,IACO,GAA1Ct3B,KAAKo3B,UAAUjD,OAAOmD,GAAS5O,QACjC,IAAK,GAAIljB,GAAI,EAAGA,EAAIxF,KAAKo3B,UAAUjD,OAAOmD,GAASxB,UAAUnwB,OAAQH,IAAK,CACxE,GAAI6J,GAAOrP,KAAKo3B,UAAUjD,OAAOmD,GAASxB,UAAUtwB,GAChD6B,EAAQ1G,EAAKkG,QAAQwI,EAAK0C,EAAG,QAAQ/K,SACzC8E,GAAa,MAAPA,EAAczE,EAAQyE,EAAMzE,EAAQA,EAAQyE,EAClDY,EAAa,MAAPA,EAAcrF,EAAcA,EAANqF,EAAcrF,EAAQqF,EAM1D,OACEZ,IAAa,MAAPA,EAAe,GAAIxH,MAAKwH,GAAO,KACrCY,IAAa,MAAPA,EAAe,GAAIpI,MAAKoI,GAAO,OAMzC7M,EAAOD,QAAU6B,GAKb,SAAS5B,EAAQD,EAASM,GAK9B,GAAI2D,GAAS3D,EAAoB,GAQjCN,GAAQ63B,qBAAuB,SAAS9C,EAAMI,GAE5C,GADAJ,EAAKI,eACDA,GACgC,GAA9B9uB,MAAMC,QAAQ6uB,GAAsB,CACtC,IAAK,GAAIvvB,GAAI,EAAGA,EAAIuvB,EAAYpvB,OAAQH,IACtC,GAA8BgB,SAA1BuuB,EAAYvvB,GAAGkyB,OAAsB,CACvC,GAAIC,KACJA,GAAS/nB,MAAQ/L,EAAOkxB,EAAYvvB,GAAGoK,OAAO1I,SAASF,UACvD2wB,EAAS9nB,IAAMhM,EAAOkxB,EAAYvvB,GAAGqK,KAAK3I,SAASF,UACnD2tB,EAAKI,YAAY/sB,KAAK2vB,GAG1BhD,EAAKI,YAAY7e,KAAK,SAAU3Q,EAAGa,GACjC,MAAOb,GAAEqK,MAAQxJ,EAAEwJ,UAY3BhQ,EAAQg4B,kBAAoB,SAAUjD,EAAMI,GAC1C,GAAIA,GAAuDvuB,SAAxCmuB,EAAKC,SAASiD,gBAAgBtlB,MAAqB,CACpE3S,EAAQ63B,qBAAqB9C,EAAMI,EAQnC,KAAK,GANDnlB,GAAQ/L,EAAO8wB,EAAKc,MAAM7lB,OAC1BC,EAAMhM,EAAO8wB,EAAKc,MAAM5lB,KAExBioB,EAAcnD,EAAKc,MAAM5lB,IAAM8kB,EAAKc,MAAM7lB,MAC1CmoB,EAAYD,EAAanD,EAAKC,SAASiD,gBAAgBtlB,MAElD/M,EAAI,EAAGA,EAAIuvB,EAAYpvB,OAAQH,IACtC,GAA8BgB,SAA1BuuB,EAAYvvB,GAAGkyB,OAAsB,CACvC,GAAIM,GAAYn0B,EAAOkxB,EAAYvvB,GAAGoK,OAClCqoB,EAAUp0B,EAAOkxB,EAAYvvB,GAAGqK,IAEpC,IAAoB,gBAAhBmoB,EAAUE,GACZ,KAAM,IAAIt0B,OAAM,qCAAuCmxB,EAAYvvB,GAAGoK,MAExE,IAAkB,gBAAdqoB,EAAQC,GACV,KAAM,IAAIt0B,OAAM,mCAAqCmxB,EAAYvvB,GAAGqK,IAGtE,IAAIC,GAAWmoB,EAAUD,CACzB,IAAIloB,GAAY,EAAIioB,EAAW,CAE7B,GAAIpO,GAAS,EACTwO,EAAWtoB,EAAIuoB,OACnB,QAAQrD,EAAYvvB,GAAGkyB,QACrB,IAAK,QACCM,EAAUK,OAASJ,EAAQI,QAC7B1O,EAAS,GAEXqO,EAAUM,UAAU1oB,EAAM0oB,aAC1BN,EAAUO,KAAK3oB,EAAM2oB,QACrBP,EAAU3M,SAAS,EAAE,QAErB4M,EAAQK,UAAU1oB,EAAM0oB,aACxBL,EAAQM,KAAK3oB,EAAM2oB,QACnBN,EAAQ5M,SAAS,EAAI1B,EAAO,QAE5BwO,EAASllB,IAAI,EAAG,QAChB,MACF,KAAK,SACH,GAAIulB,GAAYP,EAAQ5L,KAAK2L,EAAU,QACnCK,EAAML,EAAUK,KAGpBL,GAAUS,KAAK7oB,EAAM6oB,QACrBT,EAAUU,MAAM9oB,EAAM8oB,SACtBV,EAAUO,KAAK3oB,EAAM2oB,QACrBN,EAAUD,EAAUI,QAGpBJ,EAAUK,IAAIA,GACdJ,EAAQI,IAAIA,GACZJ,EAAQhlB,IAAIulB,EAAU,QAEtBR,EAAU3M,SAAS,EAAE,SACrB4M,EAAQ5M,SAAS,EAAE,SAEnB8M,EAASllB,IAAI,EAAG,QAChB,MACF,KAAK,UACC+kB,EAAUU,SAAWT,EAAQS,UAC/B/O,EAAS,GAEXqO,EAAUU,MAAM9oB,EAAM8oB,SACtBV,EAAUO,KAAK3oB,EAAM2oB,QACrBP,EAAU3M,SAAS,EAAE,UAErB4M,EAAQS,MAAM9oB,EAAM8oB,SACpBT,EAAQM,KAAK3oB,EAAM2oB,QACnBN,EAAQ5M,SAAS,EAAE,UACnB4M,EAAQhlB,IAAI0W,EAAO,UAEnBwO,EAASllB,IAAI,EAAG,SAChB,MACF,KAAK,SACC+kB,EAAUO,QAAUN,EAAQM,SAC9B5O,EAAS,GAEXqO,EAAUO,KAAK3oB,EAAM2oB,QACrBP,EAAU3M,SAAS,EAAE,SACrB4M,EAAQM,KAAK3oB,EAAM2oB,QACnBN,EAAQ5M,SAAS,EAAE,SACnB4M,EAAQhlB,IAAI0W,EAAO,SAEnBwO,EAASllB,IAAI,EAAG,QAChB,MACF,SAEE,WADA0lB,SAAQhF,IAAI,2EAA4EoB,EAAYvvB,GAAGkyB,QAG3G,KAAmBS,EAAZH,GAEL,OADArD,EAAKI,YAAY/sB,MAAM4H,MAAOooB,EAAUhxB,UAAW6I,IAAKooB,EAAQjxB,YACxD+tB,EAAYvvB,GAAGkyB,QACrB,IAAK,QACHM,EAAU/kB,IAAI,EAAG,QACjBglB,EAAQhlB,IAAI,EAAG,OACf,MACF,KAAK,SACH+kB,EAAU/kB,IAAI,EAAG,SACjBglB,EAAQhlB,IAAI,EAAG,QACf,MACF,KAAK,UACH+kB,EAAU/kB,IAAI,EAAG,UACjBglB,EAAQhlB,IAAI,EAAG,SACf,MACF,KAAK,SACH+kB,EAAU/kB,IAAI,EAAG,KACjBglB,EAAQhlB,IAAI,EAAG,IACf,MACF,SAEE,WADA0lB,SAAQhF,IAAI,2EAA4EoB,EAAYvvB,GAAGkyB,QAI7G/C,EAAKI,YAAY/sB,MAAM4H,MAAOooB,EAAUhxB,UAAW6I,IAAKooB,EAAQjxB,aAKtEpH,EAAQg5B,iBAAiBjE,EAEzB,IAAIkE,GAAcj5B,EAAQk5B,SAASnE,EAAKc,MAAM7lB,MAAO+kB,EAAKI,aACtDgE,EAAYn5B,EAAQk5B,SAASnE,EAAKc,MAAM5lB,IAAI8kB,EAAKI,aACjDiE,EAAarE,EAAKc,MAAM7lB,MACxBqpB,EAAWtE,EAAKc,MAAM5lB,GACA,IAAtBgpB,EAAYK,SAAiBF,EAAwC,GAA3BrE,EAAKc,MAAM0D,aAAuBN,EAAYb,UAAY,EAAIa,EAAYZ,QAAU,GAC1G,GAApBc,EAAUG,SAAmBD,EAAsC,GAAzBtE,EAAKc,MAAM2D,WAAuBL,EAAUf,UAAY,EAAMe,EAAUd,QAAU,IACtG,GAAtBY,EAAYK,QAAsC,GAApBH,EAAUG,SAC1CvE,EAAKc,MAAM4D,YAAYL,EAAYC,KAYzCr5B,EAAQg5B,iBAAmB,SAASjE,GAGlC,IAAK,GAFDI,GAAcJ,EAAKI,YACnBuE,KACK9zB,EAAI,EAAGA,EAAIuvB,EAAYpvB,OAAQH,IACtC,IAAK,GAAIomB,GAAI,EAAGA,EAAImJ,EAAYpvB,OAAQimB,IAClCpmB,GAAKomB,GAA8B,GAAzBmJ,EAAYnJ,GAAGvV,QAA2C,GAAzB0e,EAAYvvB,GAAG6Q,SAExD0e,EAAYnJ,GAAGhc,OAASmlB,EAAYvvB,GAAGoK,OAASmlB,EAAYnJ,GAAG/b,KAAOklB,EAAYvvB,GAAGqK,IACvFklB,EAAYnJ,GAAGvV,QAAS,EAGjB0e,EAAYnJ,GAAGhc,OAASmlB,EAAYvvB,GAAGoK,OAASmlB,EAAYnJ,GAAGhc,OAASmlB,EAAYvvB,GAAGqK,KAC9FklB,EAAYvvB,GAAGqK,IAAMklB,EAAYnJ,GAAG/b,IACpCklB,EAAYnJ,GAAGvV,QAAS,GAGjB0e,EAAYnJ,GAAG/b,KAAOklB,EAAYvvB,GAAGoK,OAASmlB,EAAYnJ,GAAG/b,KAAOklB,EAAYvvB,GAAGqK,MAC1FklB,EAAYvvB,GAAGoK,MAAQmlB,EAAYnJ,GAAGhc,MACtCmlB,EAAYnJ,GAAGvV,QAAS,GAMhC,KAAK,GAAI7Q,GAAI,EAAGA,EAAIuvB,EAAYpvB,OAAQH,IAClCuvB,EAAYvvB,GAAG6Q,UAAW,GAC5BijB,EAAUtxB,KAAK+sB,EAAYvvB,GAI/BmvB,GAAKI,YAAcuE,EACnB3E,EAAKI,YAAY7e,KAAK,SAAU3Q,EAAGa,GACjC,MAAOb,GAAEqK,MAAQxJ,EAAEwJ,SAIvBhQ,EAAQ25B,WAAa,SAASC,GAC5B,IAAK,GAAIh0B,GAAG,EAAGA,EAAIg0B,EAAM7zB,OAAQH,IAC/BmzB,QAAQhF,IAAInuB,EAAG,GAAIlB,MAAKk1B,EAAMh0B,GAAGoK,OAAO,GAAItL,MAAKk1B,EAAMh0B,GAAGqK,KAAM2pB,EAAMh0B,GAAGoK,MAAO4pB,EAAMh0B,GAAGqK,IAAK2pB,EAAMh0B,GAAG6Q,SAS3GzW,EAAQ65B,oBAAsB,SAASC,EAAUC,GAG/C,IAAK,GAFDC,IAAe,EACfC,EAAeH,EAASI,QAAQ9yB,UAC3BxB,EAAI,EAAGA,EAAIk0B,EAAS3E,YAAYpvB,OAAQH,IAAK,CACpD,GAAIwyB,GAAY0B,EAAS3E,YAAYvvB,GAAGoK,MACpCqoB,EAAUyB,EAAS3E,YAAYvvB,GAAGqK,GACtC,IAAIgqB,GAAgB7B,GAA4BC,EAAf4B,EAAwB,CACvDD,GAAe,CACf,QAIJ,GAAoB,GAAhBA,GAAwBC,EAAeH,EAASvG,KAAKnsB,WAAa6yB,GAAgBF,EAAc,CAClG,GAAIlqB,GAAY5L,EAAO81B,GACnBI,EAAWl2B,EAAOo0B,EAElBxoB,GAAU8oB,QAAUwB,EAASxB,OAASmB,EAASM,cAAe,EACzDvqB,EAAUipB,SAAWqB,EAASrB,QAAUgB,EAASO,eAAgB,EACjExqB,EAAU6oB,aAAeyB,EAASzB,cAAcoB,EAASQ,aAAc,GAEhFR,EAASI,QAAUC,EAAS7yB,WAmChCtH,EAAQq1B,SAAW,SAASiB,EAAMiE,EAAM5nB,GACtC,GAAoC,GAAhC2jB,EAAKvB,KAAKI,YAAYpvB,OAAa,CACrC,GAAIy0B,GAAalE,EAAKT,MAAM2E,WAAW7nB,EACvC,QAAQ4nB,EAAKnzB,UAAYozB,EAAWzQ,QAAUyQ,EAAWnd,MAGzD,GAAIic,GAASt5B,EAAQk5B,SAASqB,EAAMjE,EAAKvB,KAAKI,YACzB,IAAjBmE,EAAOA,SACTiB,EAAOjB,EAAOlB,UAGhB,IAAIloB,GAAWlQ,EAAQy6B,yBAAyBnE,EAAKvB,KAAKI,YAAamB,EAAKT,MAAM7lB,MAAOsmB,EAAKT,MAAM5lB,IACpGsqB,GAAOv6B,EAAQ06B,qBAAqBpE,EAAKvB,KAAKI,YAAamB,EAAKT,MAAO0E,EAEvE,IAAIC,GAAalE,EAAKT,MAAM2E,WAAW7nB,EAAOzC,EAC9C,QAAQqqB,EAAKnzB,UAAYozB,EAAWzQ,QAAUyQ,EAAWnd,OAa7Drd,EAAQy1B,OAAS,SAASa,EAAMnkB,EAAGQ,GACjC,GAAoC,GAAhC2jB,EAAKvB,KAAKI,YAAYpvB,OAAa,CACrC,GAAIy0B,GAAalE,EAAKT,MAAM2E,WAAW7nB,EACvC,OAAO,IAAIjO,MAAKyN,EAAIqoB,EAAWnd,MAAQmd,EAAWzQ,QAGlD,GAAI4Q,GAAiB36B,EAAQy6B,yBAAyBnE,EAAKvB,KAAKI,YAAamB,EAAKT,MAAM7lB,MAAOsmB,EAAKT,MAAM5lB,KACtG2qB,EAAgBtE,EAAKT,MAAM5lB,IAAMqmB,EAAKT,MAAM7lB,MAAQ2qB,EACpDE,EAAkBD,EAAgBzoB,EAAIQ,EACtCmoB,EAA4B96B,EAAQ+6B,6BAA6BzE,EAAKvB,KAAKI,YAAamB,EAAKT,MAAOgF,GAEpGG,EAAU,GAAIt2B,MAAKo2B,EAA4BD,EAAkBvE,EAAKT,MAAM7lB,MAChF,OAAOgrB,IAYXh7B,EAAQy6B,yBAA2B,SAAStF,EAAanlB,EAAOC,GAE9D,IAAK,GADDC,GAAW,EACNtK,EAAI,EAAGA,EAAIuvB,EAAYpvB,OAAQH,IAAK,CAC3C,GAAIwyB,GAAYjD,EAAYvvB,GAAGoK,MAC3BqoB,EAAUlD,EAAYvvB,GAAGqK,GAEzBmoB,IAAapoB,GAAmBC,EAAVooB,IACxBnoB,GAAYmoB,EAAUD,GAG1B,MAAOloB,IAWTlQ,EAAQ06B,qBAAuB,SAASvF,EAAaU,EAAO0E,GAG1D,MAFAA,GAAOt2B,EAAOs2B,GAAMjzB,SAASF,UAC7BmzB,GAAQv6B,EAAQi7B,wBAAwB9F,EAAYU,EAAM0E,IAI5Dv6B,EAAQi7B,wBAA0B,SAAS9F,EAAaU,EAAO0E,GAC7D,GAAIW,GAAa,CACjBX,GAAOt2B,EAAOs2B,GAAMjzB,SAASF,SAE7B,KAAK,GAAIxB,GAAI,EAAGA,EAAIuvB,EAAYpvB,OAAQH,IAAK,CAC3C,GAAIwyB,GAAYjD,EAAYvvB,GAAGoK,MAC3BqoB,EAAUlD,EAAYvvB,GAAGqK,GAEzBmoB,IAAavC,EAAM7lB,OAASqoB,EAAUxC,EAAM5lB,KAC1CsqB,GAAQlC,IACV6C,GAAe7C,EAAUD,GAI/B,MAAO8C,IAWTl7B,EAAQ+6B,6BAA+B,SAAS5F,EAAaU,EAAOsF,GAKlE,IAAK,GAJDR,GAAiB,EACjBzqB,EAAW,EACXkrB,EAAgBvF,EAAM7lB,MAEjBpK,EAAI,EAAGA,EAAIuvB,EAAYpvB,OAAQH,IAAK,CAC3C,GAAIwyB,GAAYjD,EAAYvvB,GAAGoK,MAC3BqoB,EAAUlD,EAAYvvB,GAAGqK,GAE7B,IAAImoB,GAAavC,EAAM7lB,OAASqoB,EAAUxC,EAAM5lB,IAAK,CAGnD,GAFAC,GAAYkoB,EAAYgD,EACxBA,EAAgB/C,EACZnoB,GAAYirB,EACd,KAGAR,IAAkBtC,EAAUD,GAKlC,MAAOuC,IAaT36B,EAAQq7B,mBAAqB,SAASlG,EAAaoF,EAAMe,EAAWC,GAClE,GAAIrC,GAAWl5B,EAAQk5B,SAASqB,EAAMpF,EACtC,OAAuB,IAAnB+D,EAASI,OACK,EAAZgC,EACuB,GAArBC,EACKrC,EAASd,WAAac,EAASb,QAAUkC,GAAQ,EAGjDrB,EAASd,UAAY,EAIL,GAArBmD,EACKrC,EAASb,SAAWkC,EAAOrB,EAASd,WAAa,EAGjDc,EAASb,QAAU,EAKvBkC,GAaXv6B,EAAQk5B,SAAW,SAASqB,EAAMpF,GAChC,IAAK,GAAIvvB,GAAI,EAAGA,EAAIuvB,EAAYpvB,OAAQH,IAAK,CAC3C,GAAIwyB,GAAYjD,EAAYvvB,GAAGoK,MAC3BqoB,EAAUlD,EAAYvvB,GAAGqK,GAE7B,IAAIsqB,GAAQnC,GAAoBC,EAAPkC,EACvB,OAAQjB,QAAQ,EAAMlB,UAAWA,EAAWC,QAASA,GAIzD,OAAQiB,QAAQ,EAAOlB,UAAWA,EAAWC,QAASA,KAKpD,SAASp4B,GA4Bb,QAAS+B,GAASgO,EAAOC,EAAKurB,EAAaC,EAAiBC,EAAaC,GAEvEv7B,KAAK85B,QAAU,EAEf95B,KAAKw7B,WAAY,EACjBx7B,KAAKy7B,UAAY,EACjBz7B,KAAKmoB,KAAO,EACZnoB,KAAKid,MAAQ,EAEbjd,KAAK07B,YACL17B,KAAK27B,UACL37B,KAAK47B,UAAY,EAEjB57B,KAAK67B,YAAc,EAAO,EAAM,EAAI,IACpC77B,KAAK87B,YAAc,IAAO,GAAM,EAAI,GAEpC97B,KAAKu7B,WAAaA,EAElBv7B,KAAKuzB,SAAS3jB,EAAOC,EAAKurB,EAAaC,EAAiBC,GAe1D15B,EAASuR,UAAUogB,SAAW,SAAS3jB,EAAOC,EAAKurB,EAAaC,EAAiBC,GAC/Et7B,KAAKkzB,OAA6B1sB,SAApB80B,EAAYxvB,IAAoB8D,EAAQ0rB,EAAYxvB,IAClE9L,KAAKmzB,KAA2B3sB,SAApB80B,EAAY5uB,IAAoBmD,EAAMyrB,EAAY5uB,IAE1D1M,KAAKkzB,QAAUlzB,KAAKmzB,OACtBnzB,KAAKkzB,QAAU,IACflzB,KAAKmzB,MAAQ,GAGO,GAAlBnzB,KAAKw7B,WACPx7B,KAAK+7B,eAAeX,EAAaC,GAGnCr7B,KAAKg8B,SAASV,IAOhB15B,EAASuR,UAAU4oB,eAAiB,SAASX,EAAaC,GAExD,GAAIhpB,GAAOrS,KAAKmzB,KAAOnzB,KAAKkzB,OACxB+I,EAAkB,IAAP5pB,EACX6pB,EAAmBd,GAAea,EAAWZ,GAC7Cc,EAAmBj3B,KAAKwoB,MAAMxoB,KAAKyuB,IAAIsI,GAAU/2B,KAAK0uB,MAEtDwI,EAAe,GACfC,EAAkBn3B,KAAK4uB,IAAI,GAAGqI,GAE9BvsB,EAAQ,CACW,GAAnBusB,IACFvsB,EAAQusB,EAIV,KAAK,GADDG,IAAgB,EACX92B,EAAIoK,EAAO1K,KAAK2lB,IAAIrlB,IAAMN,KAAK2lB,IAAIsR,GAAmB32B,IAAK,CAClE62B,EAAkBn3B,KAAK4uB,IAAI,GAAGtuB,EAC9B,KAAK,GAAIomB,GAAI,EAAGA,EAAI5rB,KAAK87B,WAAWn2B,OAAQimB,IAAK,CAC/C,GAAI2Q,GAAWF,EAAkBr8B,KAAK87B,WAAWlQ,EACjD,IAAI2Q,GAAYL,EAAkB,CAChCI,GAAgB,EAChBF,EAAexQ,CACf,QAGJ,GAAqB,GAAjB0Q,EACF,MAGJt8B,KAAKy7B,UAAYW,EACjBp8B,KAAKid,MAAQof,EACbr8B,KAAKmoB,KAAOkU,EAAkBr8B,KAAK87B,WAAWM,IAShDx6B,EAASuR,UAAU6oB,SAAW,SAASV,GACjB90B,SAAhB80B,IACFA,KAGF,IAAIkB,GAAgCh2B,SAApB80B,EAAYxvB,IAAoB9L,KAAKkzB,OAAuB,EAAblzB,KAAKid,MAAYjd,KAAK87B,WAAW97B,KAAKy7B,WAAcH,EAAYxvB,IAC3H2wB,EAA8Bj2B,SAApB80B,EAAY5uB,IAAoB1M,KAAKmzB,KAAQnzB,KAAKid,MAAQjd,KAAK87B,WAAW97B,KAAKy7B,WAAcH,EAAY5uB,GAEvH1M,MAAK27B,UAAgCn1B,SAApB80B,EAAY5uB,IAAoB1M,KAAK08B,aAAaD,GAAWnB,EAAY5uB,IAC1F1M,KAAK07B,YAAkCl1B,SAApB80B,EAAYxvB,IAAoB9L,KAAK08B,aAAaF,GAAalB,EAAYxvB,IAGvE,GAAnB9L,KAAKu7B,aAAuBv7B,KAAK27B,UAAY37B,KAAK07B,aAAe17B,KAAKmoB,MAAQ,IAChFnoB,KAAK27B,WAAa37B,KAAK27B,UAAY37B,KAAKmoB,MAG1CnoB,KAAK47B,UAAY57B,KAAK08B,aAAaD,GAAWA,EAAUz8B,KAAK08B,aAAaF,GAAaA,EACvFx8B,KAAK28B,YAAc38B,KAAK27B,UAAY37B,KAAK07B,YAGzC17B,KAAK85B,QAAU95B,KAAK27B,WAGtB/5B,EAASuR,UAAUupB,aAAe,SAASr1B,GACzC,GAAIu1B,GAAUv1B,EAASA,GAASrH,KAAKid,MAAQjd,KAAK87B,WAAW97B,KAAKy7B,WAClE,OAAIp0B,IAASrH,KAAKid,MAAQjd,KAAK87B,WAAW97B,KAAKy7B,YAAc,GAAOz7B,KAAKid,MAAQjd,KAAK87B,WAAW97B,KAAKy7B,WAC7FmB,EAAW58B,KAAKid,MAAQjd,KAAK87B,WAAW97B,KAAKy7B,WAG7CmB,GASXh7B,EAASuR,UAAU0pB,QAAU,WAC3B,MAAQ78B,MAAK85B,SAAW95B,KAAK07B,aAM/B95B,EAASuR,UAAUkV,KAAO,WACxB,GAAIuJ,GAAO5xB,KAAK85B,OAChB95B,MAAK85B,SAAW95B,KAAKmoB,KAGjBnoB,KAAK85B,SAAWlI,IAClB5xB,KAAK85B,QAAU95B,KAAKmzB,OAOxBvxB,EAASuR,UAAU2pB,SAAW,WAC5B98B,KAAK85B,SAAW95B,KAAKmoB,KACrBnoB,KAAK27B,WAAa37B,KAAKmoB,KACvBnoB,KAAK28B,YAAc38B,KAAK27B,UAAY37B,KAAK07B,aAS3C95B,EAASuR,UAAUiV,WAAa,SAAS2U,GAEvC,GAAIjD,GAAW50B,KAAK2lB,IAAI7qB,KAAK85B,SAAW95B,KAAKmoB,KAAO,EAAK,EAAInoB,KAAK85B,QAC9D7F,EAAc,GAAK/vB,OAAO41B,GAAS7F,YAAY,EAGnD,IAAgBztB,SAAbu2B,GAA2Br4B,MAAMR,OAAO64B,KAqCzC,GAAgC,IAA5B9I,EAAYttB,QAAQ,MAA0C,IAA5BstB,EAAYttB,QAAQ,KAExD,IAAK,GAAInB,GAAIyuB,EAAYtuB,OAAS,EAAGH,EAAI,EAAGA,IAAK,CAC/C,GAAsB,KAAlByuB,EAAYzuB,GAGX,CAAA,GAAsB,KAAlByuB,EAAYzuB,IAA+B,KAAlByuB,EAAYzuB,GAAW,CACvDyuB,EAAcA,EAAYhpB,MAAM,EAAGzF,EACnC,OAGA,MAPAyuB,EAAcA,EAAYhpB,MAAM,EAAGzF,QAzCY,CAErD,GAAIw3B,GAAM,GACN70B,EAAQ8rB,EAAYttB,QAAQ,IAoBhC,IAnBY,IAATwB,IAED60B,EAAM/I,EAAYhpB,MAAM9C,GAExB8rB,EAAcA,EAAYhpB,MAAM,EAAG9C,IAErCA,EAAQjD,KAAKwH,IAAIunB,EAAYttB,QAAQ,KAAMstB,EAAYttB,QAAQ,MAClD,KAAVwB,GAEe,IAAb40B,IACD9I,GAAe,KAGjB9rB,EAAQ8rB,EAAYtuB,OAASo3B,GAEV,IAAbA,IAEN50B,GAAS40B,EAAW,GAEnB50B,EAAQ8rB,EAAYtuB,OAErB,IAAI,GAAIs3B,GAAM90B,EAAQ8rB,EAAYtuB,OAAQs3B,EAAM,EAAGA,IACjDhJ,GAAe,QAKjBA,GAAcA,EAAYhpB,MAAM,EAAG9C,EAGrC8rB,IAAe+I,EAoBjB,MAAO/I,IAWTryB,EAASuR,UAAU6hB,KAAO,aAS1BpzB,EAASuR,UAAU+pB,QAAU,WAC3B,MAAQl9B,MAAK85B,SAAW95B,KAAKid,MAAQjd,KAAK67B,WAAW77B,KAAKy7B,aAAe,GAG3E57B,EAAOD,QAAUgC,GAKb,SAAS/B,EAAQD,EAASM,GAgB9B,QAAS2B,GAAM8yB,EAAMlmB,GACnB,GAAI0uB,GAAMt5B,IAASu5B,MAAM,GAAGC,QAAQ,GAAGC,QAAQ,GAAGC,aAAa,EAC/Dv9B,MAAK4P,MAAQutB,EAAI/E,QAAQnlB,IAAI,GAAI,QAAQjM,UACzChH,KAAK6P,IAAMstB,EAAI/E,QAAQnlB,IAAI,EAAG,QAAQjM,UAEtChH,KAAK20B,KAAOA,EACZ30B,KAAKw9B,gBAAkB,EACvBx9B,KAAKy9B,YAAc,EACnBz9B,KAAKm5B,cAAe,EACpBn5B,KAAKo5B,YAAa,EAGlBp5B,KAAKq0B,gBACHzkB,MAAO,KACPC,IAAK,KACLqrB,UAAW,aACXwC,UAAU,EACVC,UAAU,EACV7xB,IAAK,KACLY,IAAK,KACLkxB,QAAS,GACTC,QAAS,UAEX79B,KAAKyO,QAAU9N,EAAK2E,UAAWtF,KAAKq0B,gBAEpCr0B,KAAKgG,OACH83B,UAEF99B,KAAK+9B,aAAe,KAGpB/9B,KAAK20B,KAAKE,QAAQthB,GAAG,WAAYvT,KAAKg+B,aAAalJ,KAAK90B,OACxDA,KAAK20B,KAAKE,QAAQthB,GAAG,UAAYvT,KAAKi+B,QAAQnJ,KAAK90B,OACnDA,KAAK20B,KAAKE,QAAQthB,GAAG,SAAYvT,KAAKk+B,WAAWpJ,KAAK90B,OAGtDA,KAAK20B,KAAKE,QAAQthB,GAAG,QAASvT,KAAKm+B,QAAQrJ,KAAK90B,OAGhDA,KAAK20B,KAAKE,QAAQthB,GAAG,aAAcvT,KAAKo+B,cAActJ,KAAK90B,OAG3DA,KAAK20B,KAAKE,QAAQthB,GAAG,QAASvT,KAAKq+B,SAASvJ,KAAK90B,OACjDA,KAAK20B,KAAKE,QAAQthB,GAAG,QAASvT,KAAKs+B,SAASxJ,KAAK90B,OAEjDA,KAAKkT,WAAWzE,GAsClB,QAAS8vB,GAAmBrD,GAC1B,GAAiB,cAAbA,GAA0C,YAAbA,EAC/B,KAAM,IAAI70B,WAAU,sBAAwB60B,EAAY,yCAof5D,QAASsD,GAAYV,EAAOl1B,GAC1B,OACEmJ,EAAG+rB,EAAM/rB,EAAIpR,EAAK2G,gBAAgBsB,GAClCoJ,EAAG8rB,EAAM9rB,EAAIrR,EAAK+G,eAAekB,IA1lBrC,GAAIjI,GAAOT,EAAoB,GAE3B2D,GADa3D,EAAoB,IACxBA,EAAoB,KAC7BqC,EAAYrC,EAAoB,IAChCyB,EAAWzB,EAAoB,GA0DnC2B,GAAMsR,UAAY,GAAI5Q,GAkBtBV,EAAMsR,UAAUD,WAAa,SAAUzE,GACrC,GAAIA,EAAS,CAEX,GAAIP,IAAU,YAAa,MAAO,MAAO,UAAW,UAAW,WAAY,WAAY,WAAY,cACnGvN,GAAKoF,gBAAgBmI,EAAQlO,KAAKyO,QAASA,IAEvC,SAAWA,IAAW,OAASA,KAEjCzO,KAAKuzB,SAAS9kB,EAAQmB,MAAOnB,EAAQoB,OA4B3ChO,EAAMsR,UAAUogB,SAAW,SAAS3jB,EAAOC,EAAK2mB,EAASiI,GACnDA,KAAW,IACbA,GAAS,EAEX,IAAIvL,GAAkB1sB,QAAToJ,EAAqBjP,EAAKkG,QAAQ+I,EAAO,QAAQ5I,UAAY,KACtEmsB,EAAgB3sB,QAAPqJ,EAAqBlP,EAAKkG,QAAQgJ,EAAK,QAAQ7I,UAAc,IAG1E,IAFAhH,KAAK0+B,mBAEDlI,EAAS,CACX,GAAIriB,GAAKnU,KACL2+B,EAAY3+B,KAAK4P,MACjBgvB,EAAU5+B,KAAK6P,IACfC,EAA8B,gBAAZ0mB,GAAuBA,EAAU,IACnDqI,GAAW,GAAIv6B,OAAO0C,UACtB83B,GAAa,EAEbzW,EAAO,WACT,IAAKlU,EAAGnO,MAAM83B,MAAMiB,SAAU,CAC5B,GAAI5B,IAAM,GAAI74B,OAAO0C,UACjBmzB,EAAOgD,EAAM0B,EACbG,EAAO7E,EAAOrqB,EACdlE,EAAKozB,GAAmB,OAAX9L,EAAmBA,EAASvyB,EAAKgP,cAAcwqB,EAAMwE,EAAWzL,EAAQpjB,GACrFgnB,EAAKkI,GAAiB,OAAT7L,EAAmBA,EAASxyB,EAAKgP,cAAcwqB,EAAMyE,EAASzL,EAAMrjB,EAErFmvB,GAAU9qB,EAAGklB,YAAYztB,EAAGkrB,GAC5Bn1B,EAASi2B,kBAAkBzjB,EAAGwgB,KAAMxgB,EAAG1F,QAAQsmB,aAC/C+J,EAAaA,GAAcG,EACvBA,GACF9qB,EAAGwgB,KAAKE,QAAQjH,KAAK,eAAgBhe,MAAO,GAAItL,MAAK6P,EAAGvE,OAAQC,IAAK,GAAIvL,MAAK6P,EAAGtE,KAAM4uB,OAAOA,IAG5FO,EACEF,GACF3qB,EAAGwgB,KAAKE,QAAQjH,KAAK,gBAAiBhe,MAAO,GAAItL,MAAK6P,EAAGvE,OAAQC,IAAK,GAAIvL,MAAK6P,EAAGtE,KAAM4uB,OAAOA,IAMjGtqB,EAAG4pB,aAAezkB,WAAW+O,EAAM,KAKzC,OAAOA,KAGP,GAAI4W,GAAUj/B,KAAKq5B,YAAYnG,EAAQC,EAEvC,IADAxxB,EAASi2B,kBAAkB53B,KAAK20B,KAAM30B,KAAKyO,QAAQsmB,aAC/CkK,EAAS,CACX,GAAInrB,IAAUlE,MAAO,GAAItL,MAAKtE,KAAK4P,OAAQC,IAAK,GAAIvL,MAAKtE,KAAK6P,KAAM4uB,OAAOA,EAC3Ez+B,MAAK20B,KAAKE,QAAQjH,KAAK,cAAe9Z,GACtC9T,KAAK20B,KAAKE,QAAQjH,KAAK,eAAgB9Z,KAS7CjS,EAAMsR,UAAUurB,iBAAmB,WAC7B1+B,KAAK+9B,eACP1kB,aAAarZ,KAAK+9B,cAClB/9B,KAAK+9B,aAAe,OAaxBl8B,EAAMsR,UAAUkmB,YAAc,SAASzpB,EAAOC,GAC5C,GAIIwc,GAJA6S,EAAqB,MAATtvB,EAAiBjP,EAAKkG,QAAQ+I,EAAO,QAAQ5I,UAAYhH,KAAK4P,MAC1EuvB,EAAmB,MAAPtvB,EAAiBlP,EAAKkG,QAAQgJ,EAAK,QAAQ7I,UAAchH,KAAK6P,IAC1EnD,EAA2B,MAApB1M,KAAKyO,QAAQ/B,IAAe/L,EAAKkG,QAAQ7G,KAAKyO,QAAQ/B,IAAK,QAAQ1F,UAAY,KACtF8E,EAA2B,MAApB9L,KAAKyO,QAAQ3C,IAAenL,EAAKkG,QAAQ7G,KAAKyO,QAAQ3C,IAAK,QAAQ9E,UAAY,IAI1F,IAAItC,MAAMw6B,IAA0B,OAAbA,EACrB,KAAM,IAAIt7B,OAAM,kBAAoBgM,EAAQ,IAE9C,IAAIlL,MAAMy6B,IAAsB,OAAXA,EACnB,KAAM,IAAIv7B,OAAM,gBAAkBiM,EAAM,IAyC1C,IArCaqvB,EAATC,IACFA,EAASD,GAIC,OAARpzB,GACaA,EAAXozB,IACF7S,EAAQvgB,EAAMozB,EACdA,GAAY7S,EACZ8S,GAAU9S,EAGC,MAAP3f,GACEyyB,EAASzyB,IACXyyB,EAASzyB,IAOL,OAARA,GACEyyB,EAASzyB,IACX2f,EAAQ8S,EAASzyB,EACjBwyB,GAAY7S,EACZ8S,GAAU9S,EAGC,MAAPvgB,GACaA,EAAXozB,IACFA,EAAWpzB,IAOU,OAAzB9L,KAAKyO,QAAQmvB,QAAkB,CACjC,GAAIA,GAAUvY,WAAWrlB,KAAKyO,QAAQmvB,QACxB,GAAVA,IACFA,EAAU,GAEcA,EAArBuB,EAASD,IACPl/B,KAAK6P,IAAM7P,KAAK4P,QAAWguB,GAE9BsB,EAAWl/B,KAAK4P,MAChBuvB,EAASn/B,KAAK6P,MAIdwc,EAAQuR,GAAWuB,EAASD,GAC5BA,GAAY7S,EAAO,EACnB8S,GAAU9S,EAAO,IAMvB,GAA6B,OAAzBrsB,KAAKyO,QAAQovB,QAAkB,CACjC,GAAIA,GAAUxY,WAAWrlB,KAAKyO,QAAQovB,QACxB,GAAVA,IACFA,EAAU,GAEPsB,EAASD,EAAYrB,IACnB79B,KAAK6P,IAAM7P,KAAK4P,QAAWiuB,GAE9BqB,EAAWl/B,KAAK4P,MAChBuvB,EAASn/B,KAAK6P,MAIdwc,EAAS8S,EAASD,EAAYrB,EAC9BqB,GAAY7S,EAAO,EACnB8S,GAAU9S,EAAO,IAKvB,GAAI4S,GAAWj/B,KAAK4P,OAASsvB,GAAYl/B,KAAK6P,KAAOsvB,CAUrD,OAPOD,IAAYl/B,KAAK4P,OAASsvB,GAAcl/B,KAAK6P,KAASsvB,GAAYn/B,KAAK4P,OAASuvB,GAAYn/B,KAAK6P,KACjG7P,KAAK4P,OAASsvB,GAAYl/B,KAAK4P,OAASuvB,GAAcn/B,KAAK6P,KAAOqvB,GAAcl/B,KAAK6P,KAAOsvB,GACjGn/B,KAAK20B,KAAKE,QAAQjH,KAAK,oBAGzB5tB,KAAK4P,MAAQsvB,EACbl/B,KAAK6P,IAAMsvB,EACJF,GAOTp9B,EAAMsR,UAAUisB,SAAW,WACzB,OACExvB,MAAO5P,KAAK4P,MACZC,IAAK7P,KAAK6P,MAUdhO,EAAMsR,UAAUinB,WAAa,SAAU7nB,EAAO8sB,GAC5C,MAAOx9B,GAAMu4B,WAAWp6B,KAAK4P,MAAO5P,KAAK6P,IAAK0C,EAAO8sB,IAWvDx9B,EAAMu4B,WAAa,SAAUxqB,EAAOC,EAAK0C,EAAO8sB,GAI9C,MAHoB74B,UAAhB64B,IACFA,EAAc,GAEH,GAAT9sB,GAAe1C,EAAMD,GAAS,GAE9B+Z,OAAQ/Z,EACRqN,MAAO1K,GAAS1C,EAAMD,EAAQyvB,KAK9B1V,OAAQ,EACR1M,MAAO,IAUbpb,EAAMsR,UAAU6qB,aAAe,SAAS10B,GACtCtJ,KAAKw9B,gBAAkB,EACvBx9B,KAAKs/B,cAAgB,EAEhBt/B,KAAKyO,QAAQivB,UAIb19B,KAAKgG,MAAM83B,MAAMyB,gBAEtBv/B,KAAKgG,MAAM83B,MAAMluB,MAAQ5P,KAAK4P,MAC9B5P,KAAKgG,MAAM83B,MAAMjuB,IAAM7P,KAAK6P,IAC5B7P,KAAKgG,MAAM83B,MAAMiB,UAAW,EAExB/+B,KAAK20B,KAAK5E,IAAIrwB,OAChBM,KAAK20B,KAAK5E,IAAIrwB,KAAKuN,MAAM+f,OAAS,QAGpC1jB,EAAMD,mBAQRxH,EAAMsR,UAAU8qB,QAAU,SAAU30B,GAElC,GAAKtJ,KAAKyO,QAAQivB,UAKb19B,KAAKgG,MAAM83B,MAAMyB,cAAtB,CAEA,GAAIrE,GAAYl7B,KAAKyO,QAAQysB,SAC7BqD,GAAkBrD,EAClB,IAAIzM,GAAsB,cAAbyM,EAA6B5xB,EAAMk2B,OAASl2B,EAAMm2B,MAC/DhR,IAASzuB,KAAKw9B,eACd,IAAIhL,GAAYxyB,KAAKgG,MAAM83B,MAAMjuB,IAAM7P,KAAKgG,MAAM83B,MAAMluB,MAGpDE,EAAWnO,EAAS04B,yBAAyBr6B,KAAK20B,KAAKI,YAAa/0B,KAAK4P,MAAO5P,KAAK6P,IACzF2iB,IAAY1iB,CAEZ,IAAIyC,GAAsB,cAAb2oB,EAA6Bl7B,KAAK20B,KAAKC,SAAS1I,OAAO3Z,MAAQvS,KAAK20B,KAAKC,SAAS1I,OAAO1Z,OAClGktB,GAAajR,EAAQlc,EAAQigB,EAC7B0M,EAAWl/B,KAAKgG,MAAM83B,MAAMluB,MAAQ8vB,EACpCP,EAASn/B,KAAKgG,MAAM83B,MAAMjuB,IAAM6vB,EAGhCC,EAAYh+B,EAASs5B,mBAAmBj7B,KAAK20B,KAAKI,YAAamK,EAAUl/B,KAAKs/B,cAAc7Q,GAAO,GACnGmR,EAAUj+B,EAASs5B,mBAAmBj7B,KAAK20B,KAAKI,YAAaoK,EAAQn/B,KAAKs/B,cAAc7Q,GAAO,EACnG,IAAIkR,GAAaT,GAAYU,GAAWT,EAKtC,MAJAn/B,MAAKw9B,iBAAmB/O,EACxBzuB,KAAKgG,MAAM83B,MAAMluB,MAAQ+vB,EACzB3/B,KAAKgG,MAAM83B,MAAMjuB,IAAM+vB,MACvB5/B,MAAKi+B,QAAQ30B,EAIftJ,MAAKs/B,cAAgB7Q,EACrBzuB,KAAKq5B,YAAY6F,EAAUC,GAG3Bn/B,KAAK20B,KAAKE,QAAQjH,KAAK,eACrBhe,MAAO,GAAItL,MAAKtE,KAAK4P,OACrBC,IAAO,GAAIvL,MAAKtE,KAAK6P,KACrB4uB,QAAQ,IAGVn1B,EAAMD,mBAQRxH,EAAMsR,UAAU+qB,WAAa,WAEtBl+B,KAAKyO,QAAQivB,UAKb19B,KAAKgG,MAAM83B,MAAMyB,gBAEtBv/B,KAAKgG,MAAM83B,MAAMiB,UAAW,EACxB/+B,KAAK20B,KAAK5E,IAAIrwB,OAChBM,KAAK20B,KAAK5E,IAAIrwB,KAAKuN,MAAM+f,OAAS,QAIpChtB,KAAK20B,KAAKE,QAAQjH,KAAK,gBACrBhe,MAAO,GAAItL,MAAKtE,KAAK4P,OACrBC,IAAO,GAAIvL,MAAKtE,KAAK6P,KACrB4uB,QAAQ,MAUZ58B,EAAMsR,UAAUirB,cAAgB,SAAS90B,GAEvC,GAAMtJ,KAAKyO,QAAQkvB,UAAY39B,KAAKyO,QAAQivB,SAA5C,CAGA,GAAIjP,GAAQ,CAYZ,IAXInlB,EAAMolB,WACRD,EAAQnlB,EAAMolB,WAAa,IAClBplB,EAAMqlB,SAGfF,GAASnlB,EAAMqlB,OAAS,GAMtBF,EAAO,CAKT,GAAIxR,EAEFA,GADU,EAARwR,EACM,EAAKA,EAAQ,EAGb,GAAK,EAAKA,EAAQ,EAI5B,IAAIoR,GAAUrB,GAAYzsB,EAAGzI,EAAMw2B,MAAO9tB,EAAG1I,EAAMy2B,OAAQ//B,KAAK20B,KAAK5E,IAAI7D,QACrE8T,EAAchgC,KAAKigC,eAAeJ,EAEtC7/B,MAAKkgC,KAAKjjB,EAAO+iB,EAAavR,GAKhCnlB,EAAMD,mBAORxH,EAAMsR,UAAUkrB,SAAW,WACzBr+B,KAAKgG,MAAM83B,MAAMluB,MAAQ5P,KAAK4P,MAC9B5P,KAAKgG,MAAM83B,MAAMjuB,IAAM7P,KAAK6P,IAC5B7P,KAAKgG,MAAM83B,MAAMyB,eAAgB,EACjCv/B,KAAKgG,MAAM83B,MAAM5R,OAAS,KAC1BlsB,KAAKy9B,YAAc,EACnBz9B,KAAKw9B,gBAAkB,GAOzB37B,EAAMsR,UAAUgrB,QAAU,WACxBn+B,KAAKgG,MAAM83B,MAAMyB,eAAgB,GAQnC19B,EAAMsR,UAAUmrB,SAAW,SAAUh1B,GAEnC,GAAMtJ,KAAKyO,QAAQkvB,UAAY39B,KAAKyO,QAAQivB,SAA5C,CAEA19B,KAAKgG,MAAM83B,MAAMyB,eAAgB,EAE5Bv/B,KAAKgG,MAAM83B,MAAM5R,SACpBlsB,KAAKgG,MAAM83B,MAAM5R,OAASsS,EAAWl1B,EAAM4iB,OAAQlsB,KAAK20B,KAAK5E,IAAI7D,QAGnE,IAAIjP,GAAQ,GAAK3T,EAAM2T,MAAQjd,KAAKy9B,aAChC0C,EAAangC,KAAKigC,eAAejgC,KAAKgG,MAAM83B,MAAM5R,QAElDqO,EAAiB54B,EAAS04B,yBAAyBr6B,KAAK20B,KAAKI,YAAa/0B,KAAK4P,MAAO5P,KAAK6P,KAC3FuwB,EAAuBz+B,EAASk5B,wBAAwB76B,KAAK20B,KAAKI,YAAa/0B,KAAMmgC,GACrFE,EAAsB9F,EAAiB6F,EAGvClB,EAAYiB,EAAaC,GAAyBpgC,KAAKgG,MAAM83B,MAAMluB,OAASuwB,EAAaC,IAAyBnjB,EAClHkiB,EAAUgB,EAAaE,GAAwBrgC,KAAKgG,MAAM83B,MAAMjuB,KAAOswB,EAAaE,IAAwBpjB,CAGhHjd,MAAKm5B,aAA4B,GAAb,EAAIlc,EACxBjd,KAAKo5B,WAA0B,GAAbnc,EAAQ,CAE1B,IAAI0iB,GAAYh+B,EAASs5B,mBAAmBj7B,KAAK20B,KAAKI,YAAamK,EAAU,EAAIjiB,GAAO,GACpF2iB,EAAUj+B,EAASs5B,mBAAmBj7B,KAAK20B,KAAKI,YAAaoK,EAAQliB,EAAQ,GAAG,IAChF0iB,GAAaT,GAAYU,GAAWT,KACtCn/B,KAAKgG,MAAM83B,MAAMluB,MAAQ+vB,EACzB3/B,KAAKgG,MAAM83B,MAAMjuB,IAAM+vB,EACvB5/B,KAAKy9B,YAAc,EAAIn0B,EAAM2T,MAC7BiiB,EAAWS,EACXR,EAASS,GAGX5/B,KAAKuzB,SAAS2L,EAAUC,GAAQ,GAAO,GAEvCn/B,KAAKm5B,cAAe,EACpBn5B,KAAKo5B,YAAa,EAElB9vB,EAAMD,mBASRxH,EAAMsR,UAAU8sB,eAAiB,SAAUJ,GACzC,GAAIzF,GACAc,EAAYl7B,KAAKyO,QAAQysB,SAI7B,IAFAqD,EAAkBrD,GAED,cAAbA,EACF,MAAOl7B,MAAK20B,KAAKh0B,KAAK00B,OAAOwK,EAAQ9tB,GAAG/K,SAGxC,IAAIwL,GAASxS,KAAK20B,KAAKC,SAAS1I,OAAO1Z,MAEvC,OADA4nB,GAAap6B,KAAKo6B,WAAW5nB,GACtBqtB,EAAQ7tB,EAAIooB,EAAWnd,MAAQmd,EAAWzQ,QA4BrD9nB,EAAMsR,UAAU+sB,KAAO,SAASjjB,EAAOiP,EAAQuC,GAE/B,MAAVvC,IACFA,GAAUlsB,KAAK4P,MAAQ5P,KAAK6P,KAAO,EAGrC,IAAI0qB,GAAiB54B,EAAS04B,yBAAyBr6B,KAAK20B,KAAKI,YAAa/0B,KAAK4P,MAAO5P,KAAK6P,KAC3FuwB,EAAuBz+B,EAASk5B,wBAAwB76B,KAAK20B,KAAKI,YAAa/0B,KAAMksB,GACrFmU,EAAsB9F,EAAiB6F,EAGvClB,EAAYhT,EAAOkU,GAAyBpgC,KAAK4P,OAASsc,EAAOkU,IAAyBnjB,EAC1FkiB,EAAYjT,EAAOmU,GAAwBrgC,KAAK6P,KAAOqc,EAAOmU,IAAwBpjB,CAG1Fjd,MAAKm5B,aAAe1K,EAAQ,GAAI,GAAQ,EACxCzuB,KAAKo5B,YAAc3K,EAAS,GAAI,GAAQ,CACxC,IAAIkR,GAAYh+B,EAASs5B,mBAAmBj7B,KAAK20B,KAAKI,YAAamK,EAAUzQ,GAAO,GAChFmR,EAAUj+B,EAASs5B,mBAAmBj7B,KAAK20B,KAAKI,YAAaoK,GAAS1Q,GAAO,IAC7EkR,GAAaT,GAAYU,GAAWT,KACtCD,EAAWS,EACXR,EAASS,GAGX5/B,KAAKuzB,SAAS2L,EAAUC,GAAQ,GAAO,GAEvCn/B,KAAKm5B,cAAe,EACpBn5B,KAAKo5B,YAAa,GAWpBv3B,EAAMsR,UAAUmtB,KAAO,SAAS7R,GAE9B,GAAIpC,GAAQrsB,KAAK6P,IAAM7P,KAAK4P,MAGxBsvB,EAAWl/B,KAAK4P,MAAQyc,EAAOoC,EAC/B0Q,EAASn/B,KAAK6P,IAAMwc,EAAOoC,CAI/BzuB,MAAK4P,MAAQsvB,EACbl/B,KAAK6P,IAAMsvB,GAObt9B,EAAMsR,UAAU0U,OAAS,SAASA,GAChC,GAAIqE,IAAUlsB,KAAK4P,MAAQ5P,KAAK6P,KAAO,EAEnCwc,EAAOH,EAASrE,EAGhBqX,EAAWl/B,KAAK4P,MAAQyc,EACxB8S,EAASn/B,KAAK6P,IAAMwc,CAExBrsB,MAAKuzB,SAAS2L,EAAUC,IAG1Bt/B,EAAOD,QAAUiC,GAKb,SAAShC,EAAQD,GAGrB,GAAI2gC,GAAU,IAMd3gC,GAAQ4gC,aAAe,SAASv+B,GAC9BA,EAAMiU,KAAK,SAAU3Q,EAAGa,GACtB,MAAOb,GAAEmN,KAAK9C,MAAQxJ,EAAEsM,KAAK9C,SASjChQ,EAAQ6gC,WAAa,SAASx+B,GAC5BA,EAAMiU,KAAK,SAAU3Q,EAAGa,GACtB,GAAIs6B,GAAS,OAASn7B,GAAEmN,KAAQnN,EAAEmN,KAAK7C,IAAMtK,EAAEmN,KAAK9C,MAChD+wB,EAAS,OAASv6B,GAAEsM,KAAQtM,EAAEsM,KAAK7C,IAAMzJ,EAAEsM,KAAK9C,KAEpD,OAAO8wB,GAAQC,KAenB/gC,EAAQkC,MAAQ,SAASG,EAAOyX,EAAQknB,GACtC,GAAIp7B,GAAGq7B,CAEP,IAAID,EAEF,IAAKp7B,EAAI,EAAGq7B,EAAO5+B,EAAM0D,OAAYk7B,EAAJr7B,EAAUA,IACzCvD,EAAMuD,GAAGmC,IAAM,IAKnB,KAAKnC,EAAI,EAAGq7B,EAAO5+B,EAAM0D,OAAYk7B,EAAJr7B,EAAUA,IAAK,CAC9C,GAAI6J,GAAOpN,EAAMuD,EACjB,IAAI6J,EAAKvN,OAAsB,OAAbuN,EAAK1H,IAAc,CAEnC0H,EAAK1H,IAAM+R,EAAOonB,IAElB,GAAG,CAID,IAAK,GADDC,GAAgB,KACXnV,EAAI,EAAGoV,EAAK/+B,EAAM0D,OAAYq7B,EAAJpV,EAAQA,IAAK,CAC9C,GAAIhmB,GAAQ3D,EAAM2pB,EAClB,IAAkB,OAAdhmB,EAAM+B,KAAgB/B,IAAUyJ,GAAQzJ,EAAM9D,OAASlC,EAAQqhC,UAAU5xB,EAAMzJ,EAAO8T,EAAOrK,MAAO,CACtG0xB,EAAgBn7B,CAChB,QAIiB,MAAjBm7B,IAEF1xB,EAAK1H,IAAMo5B,EAAcp5B,IAAMo5B,EAAcvuB,OAASkH,EAAOrK,KAAKoW,gBAE7Dsb,MAafnhC,EAAQshC,QAAU,SAASj/B,EAAOyX,EAAQynB,GACxC,GAAI37B,GAAGq7B,EAAMO,CAGb,KAAK57B,EAAI,EAAGq7B,EAAO5+B,EAAM0D,OAAYk7B,EAAJr7B,EAAUA,IACzC,GAA+BgB,SAA3BvE,EAAMuD,GAAGkN,KAAK2uB,SAAwB,CACxCD,EAAS1nB,EAAOonB,IAChB,KAAK,GAAIO,KAAYF,GACfA,EAAUr7B,eAAeu7B,IACQ,GAA/BF,EAAUE,GAAU3Y,SAAmByY,EAAUE,GAAUl5B,MAAQg5B,EAAUl/B,EAAMuD,GAAGkN,KAAK2uB,UAAUl5B,QACvGi5B,GAAUD,EAAUE,GAAU7uB,OAASkH,EAAOrK,KAAKoW,SAIzDxjB,GAAMuD,GAAGmC,IAAMy5B,MAGfn/B,GAAMuD,GAAGmC,IAAM+R,EAAOonB,MAe5BlhC,EAAQqhC,UAAY,SAAS17B,EAAGa,EAAGsT,GACjC,MAASnU,GAAEkC,KAAOiS,EAAO8L,WAAa+a,EAAkBn6B,EAAEqB,KAAOrB,EAAEmM,OAC9DhN,EAAEkC,KAAOlC,EAAEgN,MAAQmH,EAAO8L,WAAa+a,EAAWn6B,EAAEqB,MACpDlC,EAAEoC,IAAM+R,EAAO+L,SAAW8a,EAAyBn6B,EAAEuB,IAAMvB,EAAEoM,QAC7DjN,EAAEoC,IAAMpC,EAAEiN,OAASkH,EAAO+L,SAAW8a,EAAan6B,EAAEuB,MAMvD,SAAS9H,EAAQD,EAASM,GAgC9B,QAAS6B,GAAS6N,EAAOC,EAAKurB,EAAarG,GAEzC/0B,KAAK85B,QAAU,GAAIx1B,MACnBtE,KAAKkzB,OAAS,GAAI5uB,MAClBtE,KAAKmzB,KAAO,GAAI7uB,MAEhBtE,KAAKw7B,WAAa,EAClBx7B,KAAKid,MAAQ,MACbjd,KAAKmoB,KAAO,EAGZnoB,KAAKuzB,SAAS3jB,EAAOC,EAAKurB,GAG1Bp7B,KAAKk6B,aAAc,EACnBl6B,KAAKi6B,eAAgB,EACrBj6B,KAAKg6B,cAAe,EACpBh6B,KAAK+0B,YAAcA,EACCvuB,SAAhBuuB,IACF/0B,KAAK+0B,gBAGP/0B,KAAKshC,OAASv/B,EAASw/B,OApDzB,GAAI19B,GAAS3D,EAAoB,IAC7ByB,EAAWzB,EAAoB,IAC/BS,EAAOT,EAAoB,EAsD/B6B,GAASw/B,QACPC,aACEC,YAAY,MACZC,OAAY,IACZC,OAAY,QACZC,KAAY,QACZC,QAAY,QACZxJ,IAAY,IACZK,MAAY,MACZH,KAAY,QAEduJ,aACEL,YAAY,WACZC,OAAY,eACZC,OAAY,aACZC,KAAY,aACZC,QAAY,YACZxJ,IAAY,YACZK,MAAY,OACZH,KAAY,KAUhBx2B,EAASoR,UAAU4uB,UAAY,SAAUT,GACvC,GAAIU,GAAgBrhC,EAAK8F,cAAe1E,EAASw/B,OACjDvhC,MAAKshC,OAAS3gC,EAAK8F,WAAWu7B,EAAeV,IAa/Cv/B,EAASoR,UAAUogB,SAAW,SAAS3jB,EAAOC,EAAKurB,GACjD,KAAMxrB,YAAiBtL,OAAWuL,YAAevL,OAC/C,KAAO,+CAGTtE,MAAKkzB,OAAmB1sB,QAAToJ,EAAsB,GAAItL,MAAKsL,EAAM5I,WAAa,GAAI1C,MACrEtE,KAAKmzB,KAAe3sB,QAAPqJ,EAAoB,GAAIvL,MAAKuL,EAAI7I,WAAa,GAAI1C,MAE3DtE,KAAKw7B,WACPx7B,KAAK+7B,eAAeX,IAOxBr5B,EAASoR,UAAU8uB,MAAQ,WACzBjiC,KAAK85B,QAAU,GAAIx1B,MAAKtE,KAAKkzB,OAAOlsB,WACpChH,KAAK08B,gBAOP36B,EAASoR,UAAUupB,aAAe,WAIhC,OAAQ18B,KAAKid,OACX,IAAK,OACHjd,KAAK85B,QAAQoI,YAAYliC,KAAKmoB,KAAOjjB,KAAKC,MAAMnF,KAAK85B,QAAQqI,cAAgBniC,KAAKmoB,OAClFnoB,KAAK85B,QAAQsI,SAAS,EACxB,KAAK,QAAgBpiC,KAAK85B,QAAQuI,QAAQ,EAC1C,KAAK,MACL,IAAK,UAAgBriC,KAAK85B,QAAQwI,SAAS,EAC3C,KAAK,OAAgBtiC,KAAK85B,QAAQyI,WAAW,EAC7C,KAAK,SAAgBviC,KAAK85B,QAAQ0I,WAAW,EAC7C,KAAK,SAAgBxiC,KAAK85B,QAAQ2I,gBAAgB,GAIpD,GAAiB,GAAbziC,KAAKmoB,KAEP,OAAQnoB,KAAKid,OACX,IAAK,cAAgBjd,KAAK85B,QAAQ2I,gBAAgBziC,KAAK85B,QAAQ4I,kBAAoB1iC,KAAK85B,QAAQ4I,kBAAoB1iC,KAAKmoB,KAAQ,MACjI,KAAK,SAAgBnoB,KAAK85B,QAAQ0I,WAAWxiC,KAAK85B,QAAQ6I,aAAe3iC,KAAK85B,QAAQ6I,aAAe3iC,KAAKmoB,KAAO,MACjH,KAAK,SAAgBnoB,KAAK85B,QAAQyI,WAAWviC,KAAK85B,QAAQ8I,aAAe5iC,KAAK85B,QAAQ8I,aAAe5iC,KAAKmoB,KAAO,MACjH,KAAK,OAAgBnoB,KAAK85B,QAAQwI,SAAStiC,KAAK85B,QAAQ+I,WAAa7iC,KAAK85B,QAAQ+I,WAAa7iC,KAAKmoB,KAAO,MAC3G,KAAK,UACL,IAAK,MAAgBnoB,KAAK85B,QAAQuI,QAASriC,KAAK85B,QAAQgJ,UAAU,GAAM9iC,KAAK85B,QAAQgJ,UAAU,GAAK9iC,KAAKmoB,KAAO,EAAI,MACpH,KAAK,QAAgBnoB,KAAK85B,QAAQsI,SAASpiC,KAAK85B,QAAQiJ,WAAa/iC,KAAK85B,QAAQiJ,WAAa/iC,KAAKmoB,KAAQ,MAC5G,KAAK,OAAgBnoB,KAAK85B,QAAQoI,YAAYliC,KAAK85B,QAAQqI,cAAgBniC,KAAK85B,QAAQqI,cAAgBniC,KAAKmoB,QAUnHpmB,EAASoR,UAAU0pB,QAAU,WAC3B,MAAQ78B,MAAK85B,QAAQ9yB,WAAahH,KAAKmzB,KAAKnsB,WAM9CjF,EAASoR,UAAUkV,KAAO,WACxB,GAAIuJ,GAAO5xB,KAAK85B,QAAQ9yB,SAIxB,IAAIhH,KAAK85B,QAAQiJ,WAAa,EAC5B,OAAQ/iC,KAAKid,OACX,IAAK,cAEHjd,KAAK85B,QAAU,GAAIx1B,MAAKtE,KAAK85B,QAAQ9yB,UAAYhH,KAAKmoB,KAAO,MAC/D,KAAK,SAAgBnoB,KAAK85B,QAAU,GAAIx1B,MAAKtE,KAAK85B,QAAQ9yB,UAAwB,IAAZhH,KAAKmoB,KAAc,MACzF,KAAK,SAAgBnoB,KAAK85B,QAAU,GAAIx1B,MAAKtE,KAAK85B,QAAQ9yB,UAAwB,IAAZhH,KAAKmoB,KAAc,GAAK,MAC9F,KAAK,OACHnoB,KAAK85B,QAAU,GAAIx1B,MAAKtE,KAAK85B,QAAQ9yB,UAAwB,IAAZhH,KAAKmoB,KAAc,GAAK,GAEzE,IAAIxc,GAAI3L,KAAK85B,QAAQ+I,UACrB7iC,MAAK85B,QAAQwI,SAAS32B,EAAKA,EAAI3L,KAAKmoB,KACpC,MACF,KAAK,UACL,IAAK,MAAgBnoB,KAAK85B,QAAQuI,QAAQriC,KAAK85B,QAAQgJ,UAAY9iC,KAAKmoB,KAAO,MAC/E,KAAK,QAAgBnoB,KAAK85B,QAAQsI,SAASpiC,KAAK85B,QAAQiJ,WAAa/iC,KAAKmoB,KAAO,MACjF,KAAK,OAAgBnoB,KAAK85B,QAAQoI,YAAYliC,KAAK85B,QAAQqI,cAAgBniC,KAAKmoB,UAKlF,QAAQnoB,KAAKid,OACX,IAAK,cAAgBjd,KAAK85B,QAAU,GAAIx1B,MAAKtE,KAAK85B,QAAQ9yB,UAAYhH,KAAKmoB,KAAO,MAClF,KAAK,SAAgBnoB,KAAK85B,QAAQ0I,WAAWxiC,KAAK85B,QAAQ6I,aAAe3iC,KAAKmoB,KAAO,MACrF,KAAK,SAAgBnoB,KAAK85B,QAAQyI,WAAWviC,KAAK85B,QAAQ8I,aAAe5iC,KAAKmoB,KAAO;KACrF,KAAK,OAAgBnoB,KAAK85B,QAAQwI,SAAStiC,KAAK85B,QAAQ+I,WAAa7iC,KAAKmoB,KAAO,MACjF,KAAK,UACL,IAAK,MAAgBnoB,KAAK85B,QAAQuI,QAAQriC,KAAK85B,QAAQgJ,UAAY9iC,KAAKmoB,KAAO,MAC/E,KAAK,QAAgBnoB,KAAK85B,QAAQsI,SAASpiC,KAAK85B,QAAQiJ,WAAa/iC,KAAKmoB,KAAO,MACjF,KAAK,OAAgBnoB,KAAK85B,QAAQoI,YAAYliC,KAAK85B,QAAQqI,cAAgBniC,KAAKmoB,MAKpF,GAAiB,GAAbnoB,KAAKmoB,KAEP,OAAQnoB,KAAKid,OACX,IAAK,cAAmBjd,KAAK85B,QAAQ4I,kBAAoB1iC,KAAKmoB,MAAMnoB,KAAK85B,QAAQ2I,gBAAgB,EAAK,MACtG,KAAK,SAAmBziC,KAAK85B,QAAQ6I,aAAe3iC,KAAKmoB,MAAMnoB,KAAK85B,QAAQ0I,WAAW,EAAK,MAC5F,KAAK,SAAmBxiC,KAAK85B,QAAQ8I,aAAe5iC,KAAKmoB,MAAMnoB,KAAK85B,QAAQyI,WAAW,EAAK,MAC5F,KAAK,OAAmBviC,KAAK85B,QAAQ+I,WAAa7iC,KAAKmoB,MAAMnoB,KAAK85B,QAAQwI,SAAS,EAAK,MACxF,KAAK,UACL,IAAK,MAAmBtiC,KAAK85B,QAAQgJ,UAAY9iC,KAAKmoB,KAAK,GAAGnoB,KAAK85B,QAAQuI,QAAQ,EAAI,MACvF,KAAK,QAAmBriC,KAAK85B,QAAQiJ,WAAa/iC,KAAKmoB,MAAMnoB,KAAK85B,QAAQsI,SAAS,EAAK,MACxF,KAAK,QAMLpiC,KAAK85B,QAAQ9yB,WAAa4qB,IAC5B5xB,KAAK85B,QAAU,GAAIx1B,MAAKtE,KAAKmzB,KAAKnsB,YAGpCrF,EAAS83B,oBAAoBz5B,KAAM4xB,IAQrC7vB,EAASoR,UAAUiV,WAAa,WAC9B,MAAOpoB,MAAK85B,SAcd/3B,EAASoR,UAAU6vB,SAAW,SAASC,EAAUC,GAC/CljC,KAAKid,MAAQgmB,EAETC,EAAU,IACZljC,KAAKmoB,KAAO+a,GAGdljC,KAAKw7B,WAAY,GAOnBz5B,EAASoR,UAAUgwB,aAAe,SAAUC,GAC1CpjC,KAAKw7B,UAAY4H,GAQnBrhC,EAASoR,UAAU4oB,eAAiB,SAASX,GAC3C,GAAmB50B,QAAf40B,EAAJ,CAMA,GAAIiI,GAAiB,QACjBC,EAAiB,OACjBC,EAAiB,MACjBC,EAAiB,KACjBC,EAAiB,IACjBC,EAAiB,IACjBC,EAAiB,CAGR,KAATN,EAAgBjI,IAAqBp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,KACpE,IAATkb,EAAejI,IAAsBp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,KACpE,IAATkb,EAAejI,IAAsBp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,KACpE,GAATkb,EAAcjI,IAAuBp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,IACpE,GAATkb,EAAcjI,IAAuBp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,IACpE,EAATkb,EAAajI,IAAwBp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,GAC7Ekb,EAAWjI,IAA0Bp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,GACnE,EAAVmb,EAAclI,IAAuBp7B,KAAKid,MAAQ,QAAejd,KAAKmoB,KAAO,GAC7Emb,EAAYlI,IAAyBp7B,KAAKid,MAAQ,QAAejd,KAAKmoB,KAAO,GACrE,EAARob,EAAYnI,IAAyBp7B,KAAKid,MAAQ,MAAejd,KAAKmoB,KAAO,GACrE,EAARob,EAAYnI,IAAyBp7B,KAAKid,MAAQ,MAAejd,KAAKmoB,KAAO,GAC7Eob,EAAUnI,IAA2Bp7B,KAAKid,MAAQ,MAAejd,KAAKmoB,KAAO,GAC7Eob,EAAQ,EAAInI,IAAyBp7B,KAAKid,MAAQ,UAAejd,KAAKmoB,KAAO,GACpE,EAATqb,EAAapI,IAAwBp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,GAC7Eqb,EAAWpI,IAA0Bp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,GAClE,GAAXsb,EAAgBrI,IAAqBp7B,KAAKid,MAAQ,SAAejd,KAAKmoB,KAAO,IAClE,GAAXsb,EAAgBrI,IAAqBp7B,KAAKid,MAAQ,SAAejd,KAAKmoB,KAAO,IAClE,EAAXsb,EAAerI,IAAsBp7B,KAAKid,MAAQ,SAAejd,KAAKmoB,KAAO,GAC7Esb,EAAarI,IAAwBp7B,KAAKid,MAAQ,SAAejd,KAAKmoB,KAAO,GAClE,GAAXub,EAAgBtI,IAAqBp7B,KAAKid,MAAQ,SAAejd,KAAKmoB,KAAO,IAClE,GAAXub,EAAgBtI,IAAqBp7B,KAAKid,MAAQ,SAAejd,KAAKmoB,KAAO,IAClE,EAAXub,EAAetI,IAAsBp7B,KAAKid,MAAQ,SAAejd,KAAKmoB,KAAO,GAC7Eub,EAAatI,IAAwBp7B,KAAKid,MAAQ,SAAejd,KAAKmoB,KAAO,GAC7D,IAAhBwb,EAAsBvI,IAAep7B,KAAKid,MAAQ,cAAejd,KAAKmoB,KAAO,KAC7D,IAAhBwb,EAAsBvI,IAAep7B,KAAKid,MAAQ,cAAejd,KAAKmoB,KAAO,KAC7D,GAAhBwb,EAAqBvI,IAAgBp7B,KAAKid,MAAQ,cAAejd,KAAKmoB,KAAO,IAC7D,GAAhBwb,EAAqBvI,IAAgBp7B,KAAKid,MAAQ,cAAejd,KAAKmoB,KAAO,IAC7D,EAAhBwb,EAAoBvI,IAAiBp7B,KAAKid,MAAQ,cAAejd,KAAKmoB,KAAO,GAC7Ewb,EAAkBvI,IAAmBp7B,KAAKid,MAAQ,cAAejd,KAAKmoB,KAAO,KASnFpmB,EAASoR,UAAU6hB,KAAO,SAASyD,GACjC,GAAIL,GAAQ,GAAI9zB,MAAKm0B,EAAKzxB,UAE1B,IAAkB,QAAdhH,KAAKid,MAAiB,CACxB,GAAIsb,GAAOH,EAAM+J,cAAgBj9B,KAAKwoB,MAAM0K,EAAM2K,WAAa,GAC/D3K,GAAM8J,YAAYh9B,KAAKwoB,MAAM6K,EAAOv4B,KAAKmoB,MAAQnoB,KAAKmoB,MACtDiQ,EAAMgK,SAAS,GACfhK,EAAMiK,QAAQ,GACdjK,EAAMkK,SAAS,GACflK,EAAMmK,WAAW,GACjBnK,EAAMoK,WAAW,GACjBpK,EAAMqK,gBAAgB,OAEnB,IAAkB,SAAdziC,KAAKid,MACRmb,EAAM0K,UAAY,IACpB1K,EAAMiK,QAAQ,GACdjK,EAAMgK,SAAShK,EAAM2K,WAAa,IAIlC3K,EAAMiK,QAAQ,GAGhBjK,EAAMkK,SAAS,GACflK,EAAMmK,WAAW,GACjBnK,EAAMoK,WAAW,GACjBpK,EAAMqK,gBAAgB,OAEnB,IAAkB,OAAdziC,KAAKid,MAAgB,CAE5B,OAAQjd,KAAKmoB,MACX,IAAK,GACL,IAAK,GACHiQ,EAAMkK,SAA6C,GAApCp9B,KAAKwoB,MAAM0K,EAAMyK,WAAa,IAAW,MAC1D,SACEzK,EAAMkK,SAA6C,GAApCp9B,KAAKwoB,MAAM0K,EAAMyK,WAAa,KAEjDzK,EAAMmK,WAAW,GACjBnK,EAAMoK,WAAW,GACjBpK,EAAMqK,gBAAgB,OAEnB,IAAkB,WAAdziC,KAAKid,MAAoB,CAEhC,OAAQjd,KAAKmoB,MACX,IAAK,GACL,IAAK,GACHiQ,EAAMkK,SAA6C,GAApCp9B,KAAKwoB,MAAM0K,EAAMyK,WAAa,IAAW,MAC1D,SACEzK,EAAMkK,SAA4C,EAAnCp9B,KAAKwoB,MAAM0K,EAAMyK,WAAa,IAEjDzK,EAAMmK,WAAW,GACjBnK,EAAMoK,WAAW,GACjBpK,EAAMqK,gBAAgB,OAEnB,IAAkB,QAAdziC,KAAKid,MAAiB,CAC7B,OAAQjd,KAAKmoB,MACX,IAAK,GACHiQ,EAAMmK,WAAiD,GAAtCr9B,KAAKwoB,MAAM0K,EAAMwK,aAAe,IAAW,MAC9D,SACExK,EAAMmK,WAAiD,GAAtCr9B,KAAKwoB,MAAM0K,EAAMwK,aAAe,KAErDxK,EAAMoK,WAAW,GACjBpK,EAAMqK,gBAAgB,OACjB,IAAkB,UAAdziC,KAAKid,MAAmB,CAEjC,OAAQjd,KAAKmoB,MACX,IAAK,IACL,IAAK,IACHiQ,EAAMmK,WAAgD,EAArCr9B,KAAKwoB,MAAM0K,EAAMwK,aAAe,IACjDxK,EAAMoK,WAAW,EACjB,MACF,KAAK,GACHpK,EAAMoK,WAAiD,GAAtCt9B,KAAKwoB,MAAM0K,EAAMuK,aAAe,IAAW,MAC9D,SACEvK,EAAMoK,WAAiD,GAAtCt9B,KAAKwoB,MAAM0K,EAAMuK,aAAe,KAErDvK,EAAMqK,gBAAgB,OAEnB,IAAkB,UAAdziC,KAAKid,MAEZ,OAAQjd,KAAKmoB,MACX,IAAK,IACL,IAAK,IACHiQ,EAAMoK,WAAgD,EAArCt9B,KAAKwoB,MAAM0K,EAAMuK,aAAe,IACjDvK,EAAMqK,gBAAgB,EACtB,MACF,KAAK,GACHrK,EAAMqK,gBAA6D,IAA7Cv9B,KAAKwoB,MAAM0K,EAAMsK,kBAAoB,KAAe,MAC5E,SACEtK,EAAMqK,gBAA4D,IAA5Cv9B,KAAKwoB,MAAM0K,EAAMsK,kBAAoB,UAG5D,IAAkB,eAAd1iC,KAAKid,MAAwB,CACpC,GAAIkL,GAAOnoB,KAAKmoB,KAAO,EAAInoB,KAAKmoB,KAAO,EAAI,CAC3CiQ,GAAMqK,gBAAgBv9B,KAAKwoB,MAAM0K,EAAMsK,kBAAoBva,GAAQA,GAGrE,MAAOiQ,IAQTr2B,EAASoR,UAAU+pB,QAAU,WAC3B,GAAyB,GAArBl9B,KAAKg6B,aAEP,OADAh6B,KAAKg6B,cAAe,EACZh6B,KAAKid,OACX,IAAK,OACL,IAAK,QACL,IAAK,UACL,IAAK,MACL,IAAK,OACL,IAAK,SACL,IAAK,SACL,IAAK,cACH,OAAO,CACT,SACE,OAAO,MAGR,IAA0B,GAAtBjd,KAAKi6B,cAEZ,OADAj6B,KAAKi6B,eAAgB,EACbj6B,KAAKid,OACX,IAAK,UACL,IAAK,MACL,IAAK,OACL,IAAK,SACL,IAAK,SACL,IAAK,cACH,OAAO,CACT,SACE,OAAO,MAGR,IAAwB,GAApBjd,KAAKk6B,YAEZ,OADAl6B,KAAKk6B,aAAc,EACXl6B,KAAKid,OACX,IAAK,cACL,IAAK,SACL,IAAK,SACL,IAAK,OACH,OAAO,CACT,SACE,OAAO,EAIb,OAAQjd,KAAKid,OACX,IAAK,cACH,MAA0C,IAAlCjd,KAAK85B,QAAQ4I,iBACvB,KAAK,SACH,MAAqC,IAA7B1iC,KAAK85B,QAAQ6I,YACvB,KAAK,SACH,MAAmC,IAA3B3iC,KAAK85B,QAAQ+I,YAAkD,GAA7B7iC,KAAK85B,QAAQ8I,YACzD,KAAK,OACH,MAAmC,IAA3B5iC,KAAK85B,QAAQ+I,UACvB,KAAK,UACL,IAAK,MACH,MAAkC,IAA1B7iC,KAAK85B,QAAQgJ,SACvB,KAAK,QACH,MAAmC,IAA3B9iC,KAAK85B,QAAQiJ,UACvB,KAAK,OACH,OAAO,CACT,SACE,OAAO,IAWbhhC,EAASoR,UAAUywB,cAAgB,SAASnL,GAC9BjyB,QAARiyB,IACFA,EAAOz4B,KAAK85B,QAGd,IAAIwH,GAASthC,KAAKshC,OAAOE,YAAYxhC,KAAKid,MAC1C,OAAQqkB,IAAUA,EAAO37B,OAAS,EAAK9B,EAAO40B,GAAM6I,OAAOA,GAAU,IASvEv/B,EAASoR,UAAU0wB,cAAgB,SAASpL,GAC9BjyB,QAARiyB,IACFA,EAAOz4B,KAAK85B,QAGd,IAAIwH,GAASthC,KAAKshC,OAAOQ,YAAY9hC,KAAKid,MAC1C,OAAQqkB,IAAUA,EAAO37B,OAAS,EAAK9B,EAAO40B,GAAM6I,OAAOA,GAAU,IAGvEv/B,EAASoR,UAAU2wB,aAAe,WAKhC,QAASC,GAAK18B,GACZ,MAAQA,GAAQ8gB,EAAO,GAAK,EAAK,QAAU,OAG7C,QAAS6b,GAAMvL,GACb,MAAIA,GAAKwL,OAAO,GAAI3/B,MAAQ,OACnB,SAELm0B,EAAKwL,OAAOpgC,IAASoP,IAAI,EAAG,OAAQ,OAC/B,YAELwlB,EAAKwL,OAAOpgC,IAASoP,IAAI,GAAI,OAAQ,OAChC,aAEF,GAGT,QAASixB,GAAYzL,GACnB,MAAOA,GAAKwL,OAAO,GAAI3/B,MAAQ,QAAU,gBAAkB,GAG7D,QAAS6/B,GAAa1L,GACpB,MAAOA,GAAKwL,OAAO,GAAI3/B,MAAQ,SAAW,iBAAmB,GAG/D,QAAS8/B,GAAY3L,GACnB,MAAOA,GAAKwL,OAAO,GAAI3/B,MAAQ,QAAU,gBAAkB,GA9B7D,GAAI9D,GAAIqD,EAAO7D,KAAK85B,SAChBrB,EAAOj4B,EAAE6jC,OAAS7jC,EAAE6jC,OAAO,MAAQ7jC,EAAE8jC,KAAK,MAC1Cnc,EAAOnoB,KAAKmoB,IA+BhB,QAAQnoB,KAAKid,OACX,IAAK,cACH,MAAO8mB,GAAKtL,EAAK8E,gBAAgBrwB,MAEnC,KAAK,SACH,MAAO62B,GAAKtL,EAAK6E,WAAWpwB,MAE9B,KAAK,SACH,MAAO62B,GAAKtL,EAAK4E,WAAWnwB,MAE9B,KAAK,OACH,GAAIkwB,GAAQ3E,EAAK2E,OAIjB,OAHiB,IAAbp9B,KAAKmoB,OACPiV,EAAQA,EAAQ,KAAOA,EAAQ,IAE1BA,EAAQ,IAAM4G,EAAMvL,GAAQsL,EAAKtL,EAAK2E,QAE/C,KAAK,UACH,MAAO3E,GAAK6I,OAAO,QAAQiD,cACvBP,EAAMvL,GAAQyL,EAAYzL,GAAQsL,EAAKtL,EAAKA,OAElD,KAAK,MACH,GAAIJ,GAAMI,EAAKA,OACXC,EAAQD,EAAK6I,OAAO,QAAQiD,aAChC,OAAO,MAAQlM,EAAM,IAAMK,EAAQyL,EAAa1L,GAAQsL,EAAK1L,EAAM,EAErE,KAAK,QACH,MAAOI,GAAK6I,OAAO,QAAQiD,cACvBJ,EAAa1L,GAAQsL,EAAKtL,EAAKC,QAErC,KAAK,OACH,GAAIH,GAAOE,EAAKF,MAChB,OAAO,OAASA,EAAO6L,EAAY3L,GAAOsL,EAAKxL,EAEjD,SACE,MAAO,KAIb14B,EAAOD,QAAUmC,GAKb,SAASlC,GAOb,QAAS0C,KACPvC,KAAKyO,QAAU,KACfzO,KAAKgG,MAAQ,KAQfzD,EAAU4Q,UAAUD,WAAa,SAASzE,GACpCA,GACF9N,KAAK2E,OAAOtF,KAAKyO,QAASA,IAQ9BlM,EAAU4Q,UAAUsO,OAAS,WAE3B,OAAO,GAMTlf,EAAU4Q,UAAUG,QAAU,aAU9B/Q,EAAU4Q,UAAUqxB,WAAa,WAC/B,GAAIC,GAAWzkC,KAAKgG,MAAM0+B,iBAAmB1kC,KAAKgG,MAAMuM,OACpDvS,KAAKgG,MAAM2+B,kBAAoB3kC,KAAKgG,MAAMwM,MAK9C,OAHAxS,MAAKgG,MAAM0+B,eAAiB1kC,KAAKgG,MAAMuM,MACvCvS,KAAKgG,MAAM2+B,gBAAkB3kC,KAAKgG,MAAMwM,OAEjCiyB,GAGT5kC,EAAOD,QAAU2C,GAKb,SAAS1C,EAAQD,EAASM,GAe9B,QAASsC,GAAamyB,EAAMlmB,GAC1BzO,KAAK20B,KAAOA,EAGZ30B,KAAKq0B,gBACHuQ,iBAAiB,EAEjBC,QAASA,EACTR,OAAQ,MAEVrkC,KAAKyO,QAAU9N,EAAK2E,UAAWtF,KAAKq0B,gBACpCr0B,KAAK2pB,OAAS,EAEd3pB,KAAK00B,UAEL10B,KAAKkT,WAAWzE,GA5BlB,GAAI9N,GAAOT,EAAoB,GAC3BqC,EAAYrC,EAAoB,IAChC2D,EAAS3D,EAAoB,IAC7B2kC,EAAU3kC,EAAoB,GA4BlCsC,GAAY2Q,UAAY,GAAI5Q,GAM5BC,EAAY2Q,UAAUuhB,QAAU,WAC9B,GAAI7C,GAAMtgB,SAASM,cAAc,MACjCggB,GAAIhqB,UAAY,cAChBgqB,EAAI5kB,MAAM2W,SAAW,WACrBiO,EAAI5kB,MAAMtF,IAAM,MAChBkqB,EAAI5kB,MAAMuF,OAAS,OAEnBxS,KAAK6xB,IAAMA,GAMbrvB,EAAY2Q,UAAUG,QAAU,WAC9BtT,KAAKyO,QAAQm2B,iBAAkB,EAC/B5kC,KAAKyhB,SAELzhB,KAAK20B,KAAO,MAQdnyB,EAAY2Q,UAAUD,WAAa,SAASzE,GACtCA,GAEF9N,EAAKoF,iBAAiB,kBAAmB,SAAU,WAAY/F,KAAKyO,QAASA,IAQjFjM,EAAY2Q,UAAUsO,OAAS,WAC7B,GAAIzhB,KAAKyO,QAAQm2B,gBAAiB,CAChC,GAAIE,GAAS9kC,KAAK20B,KAAK5E,IAAIgV,kBACvB/kC,MAAK6xB,IAAIhoB,YAAci7B,IAErB9kC,KAAK6xB,IAAIhoB,YACX7J,KAAK6xB,IAAIhoB,WAAWsH,YAAYnR,KAAK6xB,KAEvCiT,EAAOrzB,YAAYzR,KAAK6xB,KAExB7xB,KAAK4P,QAGP,IAAIutB,GAAM,GAAI74B,OAAK,GAAIA,OAAO0C,UAAYhH,KAAK2pB,QAC3C5X,EAAI/R,KAAK20B,KAAKh0B,KAAKs0B,SAASkI,GAE5BkH,EAASrkC,KAAKyO,QAAQo2B,QAAQ7kC,KAAKyO,QAAQ41B,QAC3CW,EAAQX,EAAOvK,QAAU,IAAMuK,EAAOlK,KAAO,KAAOt2B,EAAOs5B,GAAKmE,OAAO,8BAC3E0D,GAAQA,EAAM5f,OAAO,GAAG6f,cAAgBD,EAAME,UAAU,GAExDllC,KAAK6xB,IAAI5kB,MAAMxF,KAAOsK,EAAI,KAC1B/R,KAAK6xB,IAAImT,MAAQA,MAIbhlC,MAAK6xB,IAAIhoB,YACX7J,KAAK6xB,IAAIhoB,WAAWsH,YAAYnR,KAAK6xB,KAEvC7xB,KAAKklB,MAGP,QAAO,GAMT1iB,EAAY2Q,UAAUvD,MAAQ,WAG5B,QAASiF,KACPV,EAAG+Q,MAGH,IAAIjI,GAAQ9I,EAAGwgB,KAAKc,MAAM2E,WAAWjmB,EAAGwgB,KAAKC,SAAS1I,OAAO3Z,OAAO0K,MAChEuV,EAAW,EAAIvV,EAAQ,EACZ,IAAXuV,IAAiBA,EAAW,IAC5BA,EAAW,MAAMA,EAAW,KAEhCre,EAAGsN,SAGHtN,EAAGgxB,iBAAmB7rB,WAAWzE,EAAQ2d,GAd3C,GAAIre,GAAKnU,IAiBT6U,MAMFrS,EAAY2Q,UAAU+R,KAAO,WACG1e,SAA1BxG,KAAKmlC,mBACP9rB,aAAarZ,KAAKmlC,wBACXnlC,MAAKmlC,mBAUhB3iC,EAAY2Q,UAAUiyB,eAAiB,SAASjL,GAC9C,GAAIrsB,GAAInN,EAAKkG,QAAQszB,EAAM,QAAQnzB,UAC/Bm2B,GAAM,GAAI74B,OAAO0C,SACrBhH,MAAK2pB,OAAS7b,EAAIqvB,EAClBn9B,KAAKyhB,UAOPjf,EAAY2Q,UAAUkyB,eAAiB,WACrC,MAAO,IAAI/gC,OAAK,GAAIA,OAAO0C,UAAYhH,KAAK2pB,SAG9C9pB,EAAOD,QAAU4C,GAKb,SAAS3C,EAAQD,EAASM,GAiB9B,QAASuC,GAAYkyB,EAAMlmB,GACzBzO,KAAK20B,KAAOA,EAGZ30B,KAAKq0B,gBACHiR,gBAAgB,EAChBT,QAASA,EACTR,OAAQ,MAEVrkC,KAAKyO,QAAU9N,EAAK2E,UAAWtF,KAAKq0B,gBAEpCr0B,KAAK41B,WAAa,GAAItxB,MACtBtE,KAAKulC,eAGLvlC,KAAK00B,UAEL10B,KAAKkT,WAAWzE,GAhClB,GAAI1K,GAAS7D,EAAoB,IAC7BS,EAAOT,EAAoB,GAC3BqC,EAAYrC,EAAoB,IAChC2D,EAAS3D,EAAoB,IAC7B2kC,EAAU3kC,EAAoB,GA+BlCuC,GAAW0Q,UAAY,GAAI5Q,GAO3BE,EAAW0Q,UAAUD,WAAa,SAASzE,GACrCA,GAEF9N,EAAKoF,iBAAiB,iBAAkB,SAAU,WAAY/F,KAAKyO,QAASA,IAQhFhM,EAAW0Q,UAAUuhB,QAAU,WAC7B,GAAI7C,GAAMtgB,SAASM,cAAc,MACjCggB,GAAIhqB,UAAY,aAChBgqB,EAAI5kB,MAAM2W,SAAW,WACrBiO,EAAI5kB,MAAMtF,IAAM,MAChBkqB,EAAI5kB,MAAMuF,OAAS,OACnBxS,KAAK6xB,IAAMA,CAEX,IAAI2T,GAAOj0B,SAASM,cAAc,MAClC2zB,GAAKv4B,MAAM2W,SAAW,WACtB4hB,EAAKv4B,MAAMtF,IAAM,MACjB69B,EAAKv4B,MAAMxF,KAAO,QAClB+9B,EAAKv4B,MAAMuF,OAAS,OACpBgzB,EAAKv4B,MAAMsF,MAAQ,OACnBsf,EAAIpgB,YAAY+zB,GAGhBxlC,KAAK8D,OAAS,GAAIC,GAAOyhC,GACzBxlC,KAAK8D,OAAOyP,GAAG,WAAYvT,KAAKg+B,aAAalJ,KAAK90B,OAClDA,KAAK8D,OAAOyP,GAAG,UAAYvT,KAAKi+B,QAAQnJ,KAAK90B,OAC7CA,KAAK8D,OAAOyP,GAAG,SAAYvT,KAAKk+B,WAAWpJ,KAAK90B,OAChDA,KAAK8D,OAAOyP,GAAG,MAAS,SAAUjK,GAChCA,EAAMD,oBAOV5G,EAAW0Q,UAAUG,QAAU,WAC7BtT,KAAKyO,QAAQ62B,gBAAiB,EAC9BtlC,KAAKyhB,SAELzhB,KAAK8D,OAAOs/B,QAAO,GACnBpjC,KAAK8D,OAAS,KAEd9D,KAAK20B,KAAO,MAOdlyB,EAAW0Q,UAAUsO,OAAS,WAC5B,GAAIzhB,KAAKyO,QAAQ62B,eAAgB,CAC/B,GAAIR,GAAS9kC,KAAK20B,KAAK5E,IAAIgV,kBACvB/kC,MAAK6xB,IAAIhoB,YAAci7B,IAErB9kC,KAAK6xB,IAAIhoB,YACX7J,KAAK6xB,IAAIhoB,WAAWsH,YAAYnR,KAAK6xB,KAEvCiT,EAAOrzB,YAAYzR,KAAK6xB,KAG1B,IAAI9f,GAAI/R,KAAK20B,KAAKh0B,KAAKs0B,SAASj1B,KAAK41B,YAEjCyO,EAASrkC,KAAKyO,QAAQo2B,QAAQ7kC,KAAKyO,QAAQ41B,QAC3CW,EAAQX,EAAOlK,KAAO,KAAOt2B,EAAO7D,KAAK41B,YAAY0L,OAAO,8BAChE0D,GAAQA,EAAM5f,OAAO,GAAG6f,cAAgBD,EAAME,UAAU,GAExDllC,KAAK6xB,IAAI5kB,MAAMxF,KAAOsK,EAAI,KAC1B/R,KAAK6xB,IAAImT,MAAQA,MAIbhlC,MAAK6xB,IAAIhoB,YACX7J,KAAK6xB,IAAIhoB,WAAWsH,YAAYnR,KAAK6xB,IAIzC,QAAO,GAOTpvB,EAAW0Q,UAAUsyB,cAAgB,SAAStL,GAC5Cn6B,KAAK41B,WAAaj1B,EAAKkG,QAAQszB,EAAM,QACrCn6B,KAAKyhB,UAOPhf,EAAW0Q,UAAUuyB,cAAgB,WACnC,MAAO,IAAIphC,MAAKtE,KAAK41B,WAAW5uB,YAQlCvE,EAAW0Q,UAAU6qB,aAAe,SAAS10B,GAC3CtJ,KAAKulC,YAAYxG,UAAW,EAC5B/+B,KAAKulC,YAAY3P,WAAa51B,KAAK41B,WAEnCtsB,EAAMq8B,kBACNr8B,EAAMD,kBAQR5G,EAAW0Q,UAAU8qB,QAAU,SAAU30B,GACvC,GAAKtJ,KAAKulC,YAAYxG,SAAtB,CAEA,GAAIhtB,GAAI/R,KAAK20B,KAAKh0B,KAAKs0B,SAASj1B,KAAKulC,YAAY3P,YAActsB,EAAMk2B,OACjErF,EAAOn6B,KAAK20B,KAAKh0B,KAAK00B,OAAOtjB,EAEjC/R,MAAKylC,cAActL,GAGnBn6B,KAAK20B,KAAKE,QAAQjH,KAAK,cACrBuM,KAAM,GAAI71B,MAAKtE,KAAK41B,WAAW5uB,aAGjCsC,EAAMq8B,kBACNr8B,EAAMD,mBAQR5G,EAAW0Q,UAAU+qB,WAAa,SAAU50B,GACrCtJ,KAAKulC,YAAYxG,WAGtB/+B,KAAK20B,KAAKE,QAAQjH,KAAK,eACrBuM,KAAM,GAAI71B,MAAKtE,KAAK41B,WAAW5uB,aAGjCsC,EAAMq8B,kBACNr8B,EAAMD,mBAGRxJ,EAAOD,QAAU6C,GAKb,SAAS5C,EAAQD,EAASM,GAe9B,QAASwC,GAAUiyB,EAAMlmB,EAASm3B,EAAKC,GACrC7lC,KAAKK,GAAKM,EAAKqE,aACfhF,KAAK20B,KAAOA,EAEZ30B,KAAKq0B,gBACHE,YAAa,OACbuR,iBAAiB,EACjBC,iBAAiB,EACjBC,OAAO,EACPC,iBAAkB,EAClBC,iBAAkB,EAClBC,aAAc,GACdC,aAAc,EACdC,UAAW,GACX9zB,MAAO,OACPmW,SAAS,EACT6S,YAAY,EACZD,aACE7zB,MAAOqE,IAAItF,OAAWkG,IAAIlG,QAC1B6gB,OAAQvb,IAAItF,OAAWkG,IAAIlG,SAE7Bw+B,OACEv9B,MAAO8hB,KAAK/iB,QACZ6gB,OAAQkC,KAAK/iB,SAEf86B,QACE75B,MAAOs1B,SAAUv2B,QACjB6gB,OAAQ0V,SAAUv2B,UAItBxG,KAAK6lC,iBAAmBA,EACxB7lC,KAAKsmC,aAAeV,EACpB5lC,KAAKgG,SACLhG,KAAKumC,aACHC,SACAC,UACAzB,UAGFhlC,KAAK+vB,OAEL/vB,KAAKy1B,OAAS7lB,MAAM,EAAGC,IAAI,GAE3B7P,KAAKyO,QAAU9N,EAAK2E,UAAWtF,KAAKq0B,gBACpCr0B,KAAK0mC,iBAAmB,EAExB1mC,KAAKkT,WAAWzE,GAChBzO,KAAKuS,MAAQrO,QAAQ,GAAKlE,KAAKyO,QAAQ8D,OAAO/H,QAAQ,KAAK,KAC3DxK,KAAK2mC,SAAW3mC,KAAKuS,MACrBvS,KAAKwS,OAASxS,KAAKsmC,aAAahW,aAChCtwB,KAAKk5B,QAAS,EAEdl5B,KAAK4mC,WAAa,GAClB5mC,KAAK6mC,iBAAmB,GACxB7mC,KAAK8mC,aAAe,GAEpB9mC,KAAK+mC,WAAa,EAClB/mC,KAAKgnC,QAAS,EACdhnC,KAAKinC,eACLjnC,KAAKknC,cAAe,EAGpBlnC,KAAKm0B,UACLn0B,KAAKmnC,eAAiB,EAGtBnnC,KAAK00B,SAEL,IAAIvgB,GAAKnU,IACTA,MAAK20B,KAAKE,QAAQthB,GAAG,eAAgB,WACnCY,EAAG4b,IAAIqX,cAAcn6B,MAAMtF,IAAMwM,EAAGwgB,KAAKC,SAASyS,UAAY,OApFlE,GAAI1mC,GAAOT,EAAoB,GAC3BU,EAAUV,EAAoB,GAC9BqC,EAAYrC,EAAoB,IAChC0B,EAAW1B,EAAoB,GAqFnCwC,GAASyQ,UAAY,GAAI5Q,GAGzBG,EAASyQ,UAAUm0B,SAAW,SAAS7e,EAAO8e,GACvCvnC,KAAKm0B,OAAOruB,eAAe2iB,KAC9BzoB,KAAKm0B,OAAO1L,GAAS8e,GAEvBvnC,KAAKmnC,gBAAkB,GAGzBzkC,EAASyQ,UAAUq0B,YAAc,SAAS/e,EAAO8e,GAC/CvnC,KAAKm0B,OAAO1L,GAAS8e,GAGvB7kC,EAASyQ,UAAUs0B,YAAc,SAAShf,GACpCzoB,KAAKm0B,OAAOruB,eAAe2iB,WACtBzoB,MAAKm0B,OAAO1L,GACnBzoB,KAAKmnC,gBAAkB,IAK3BzkC,EAASyQ,UAAUD,WAAa,SAAUzE,GACxC,GAAIA,EAAS,CACX,GAAIgT,IAAS,CACTzhB,MAAKyO,QAAQ8lB,aAAe9lB,EAAQ8lB,aAAuC/tB,SAAxBiI,EAAQ8lB,cAC7D9S,GAAS,EAEX,IAAIvT,IACF,cACA,kBACA,kBACA,QACA,mBACA,mBACA,eACA,eACA,YACA,QACA,UACA,cACA,QACA,SACA,aAEFvN,GAAKoF,gBAAgBmI,EAAQlO,KAAKyO,QAASA,GAE3CzO,KAAK2mC,SAAWziC,QAAQ,GAAKlE,KAAKyO,QAAQ8D,OAAO/H,QAAQ,KAAK,KAEhD,GAAViX,GAAkBzhB,KAAK+vB,IAAIzQ,QAC7Btf,KAAK0nC,OACL1nC,KAAK2nC,UASXjlC,EAASyQ,UAAUuhB,QAAU,WAC3B10B,KAAK+vB,IAAIzQ,MAAQ/N,SAASM,cAAc,OACxC7R,KAAK+vB,IAAIzQ,MAAMrS,MAAMsF,MAAQvS,KAAKyO,QAAQ8D,MAC1CvS,KAAK+vB,IAAIzQ,MAAMrS,MAAMuF,OAASxS,KAAKwS,OAEnCxS,KAAK+vB,IAAIqX,cAAgB71B,SAASM,cAAc,OAChD7R,KAAK+vB,IAAIqX,cAAcn6B,MAAMsF,MAAQ,OACrCvS,KAAK+vB,IAAIqX,cAAcn6B,MAAMuF,OAASxS,KAAKwS,OAC3CxS,KAAK+vB,IAAIqX,cAAcn6B,MAAM2W,SAAW,WAGxC5jB,KAAK4lC,IAAMr0B,SAASC,gBAAgB,6BAA6B,OACjExR,KAAK4lC,IAAI34B,MAAM2W,SAAW,WAC1B5jB,KAAK4lC,IAAI34B,MAAMtF,IAAM,MACrB3H,KAAK4lC,IAAI34B,MAAMuF,OAAS,OACxBxS,KAAK4lC,IAAI34B,MAAMsF,MAAQ,OACvBvS,KAAK4lC,IAAI34B,MAAM26B,QAAU,QACzB5nC,KAAK+vB,IAAIzQ,MAAM7N,YAAYzR,KAAK4lC,MAGlCljC,EAASyQ,UAAU00B,kBAAoB,WACrCjnC,EAAQiQ,gBAAgB7Q,KAAKinC,YAE7B,IAAIl1B,GACAs0B,EAAYrmC,KAAKyO,QAAQ43B,UACzByB,EAAa,GACbC,EAAa,EACb/1B,EAAI+1B,EAAa,GAAMD,CAGzB/1B,GAD8B,QAA5B/R,KAAKyO,QAAQ8lB,YACXwT,EAGA/nC,KAAKuS,MAAQ8zB,EAAY0B,CAG/B,KAAK,GAAIzQ,KAAWt3B,MAAKm0B,OACnBn0B,KAAKm0B,OAAOruB,eAAewxB,KACO,GAAhCt3B,KAAKm0B,OAAOmD,GAAS5O,SAAkEliB,SAA9CxG,KAAK6lC,iBAAiBrO,WAAWF,IAAuE,GAA7Ct3B,KAAK6lC,iBAAiBrO,WAAWF,KACvIt3B,KAAKm0B,OAAOmD,GAAS0Q,SAASj2B,EAAGC,EAAGhS,KAAKinC,YAAajnC,KAAK4lC,IAAKS,EAAWyB,GAC3E91B,GAAK81B,EAAaC,GAKxBnnC,GAAQsQ,gBAAgBlR,KAAKinC,aAC7BjnC,KAAKknC,cAAe,GAGtBxkC,EAASyQ,UAAU80B,cAAgB,WACR,GAArBjoC,KAAKknC,eACPtmC,EAAQiQ,gBAAgB7Q,KAAKinC,aAC7BrmC,EAAQsQ,gBAAgBlR,KAAKinC,aAC7BjnC,KAAKknC,cAAe,IAOxBxkC,EAASyQ,UAAUw0B,KAAO,WACxB3nC,KAAKk5B,QAAS,EACTl5B,KAAK+vB,IAAIzQ,MAAMzV,aACc,QAA5B7J,KAAKyO,QAAQ8lB,YACfv0B,KAAK20B,KAAK5E,IAAItoB,KAAKgK,YAAYzR,KAAK+vB,IAAIzQ,OAGxCtf,KAAK20B,KAAK5E,IAAI1I,MAAM5V,YAAYzR,KAAK+vB,IAAIzQ,QAIxCtf,KAAK+vB,IAAIqX,cAAcv9B,YAC1B7J,KAAK20B,KAAK5E,IAAImY,qBAAqBz2B,YAAYzR,KAAK+vB,IAAIqX,gBAO5D1kC,EAASyQ,UAAUu0B,KAAO,WACxB1nC,KAAKk5B,QAAS,EACVl5B,KAAK+vB,IAAIzQ,MAAMzV,YACjB7J,KAAK+vB,IAAIzQ,MAAMzV,WAAWsH,YAAYnR,KAAK+vB,IAAIzQ,OAG7Ctf,KAAK+vB,IAAIqX,cAAcv9B,YACzB7J,KAAK+vB,IAAIqX,cAAcv9B,WAAWsH,YAAYnR,KAAK+vB,IAAIqX,gBAU3D1kC,EAASyQ,UAAUogB,SAAW,SAAU3jB,EAAOC,GAC1B,GAAf7P,KAAKgnC,QAA8C,GAA3BhnC,KAAKyO,QAAQ8sB,YAA2C,IAArBv7B,KAAK8mC,cAC9Dl3B,EAAQ,IACVA,EAAQ,GAGZ5P,KAAKy1B,MAAM7lB,MAAQA,EACnB5P,KAAKy1B,MAAM5lB,IAAMA,GAOnBnN,EAASyQ,UAAUsO,OAAS,WAC1B,GAAIgjB,IAAU,EACV0D,EAAe,CAGnBnoC,MAAK+vB,IAAIqX,cAAcn6B,MAAMtF,IAAM3H,KAAK20B,KAAKC,SAASyS,UAAY,IAElE,KAAK,GAAI/P,KAAWt3B,MAAKm0B,OACnBn0B,KAAKm0B,OAAOruB,eAAewxB,KACO,GAAhCt3B,KAAKm0B,OAAOmD,GAAS5O,SAAkEliB,SAA9CxG,KAAK6lC,iBAAiBrO,WAAWF,IAAuE,GAA7Ct3B,KAAK6lC,iBAAiBrO,WAAWF,IACvI6Q,IAIN,IAA2B,GAAvBnoC,KAAKmnC,gBAAuC,GAAhBgB,EAC9BnoC,KAAK0nC,WAEF,CACH1nC,KAAK2nC,OACL3nC,KAAKwS,OAAStO,OAAOlE,KAAKsmC,aAAar5B,MAAMuF,OAAOhI,QAAQ,KAAK,KAGjExK,KAAK+vB,IAAIqX,cAAcn6B,MAAMuF,OAASxS,KAAKwS,OAAS,KACpDxS,KAAKuS,MAAgC,GAAxBvS,KAAKyO,QAAQia,QAAkBxkB,QAAQ,GAAKlE,KAAKyO,QAAQ8D,OAAO/H,QAAQ,KAAK,KAAO,CAEjG,IAAIxE,GAAQhG,KAAKgG,MACbsZ,EAAQtf,KAAK+vB,IAAIzQ,KAGrBA,GAAMzX,UAAY,WAGlB7H,KAAKooC,oBAEL,IAAI7T,GAAcv0B,KAAKyO,QAAQ8lB,YAC3BuR,EAAkB9lC,KAAKyO,QAAQq3B,gBAC/BC,EAAkB/lC,KAAKyO,QAAQs3B,eAGnC//B,GAAMqiC,iBAAmBvC,EAAkB9/B,EAAMsiC,gBAAkB,EACnEtiC,EAAMuiC,iBAAmBxC,EAAkB//B,EAAMwiC,gBAAkB,EAEnExiC,EAAMyiC,eAAiBzoC,KAAK20B,KAAK5E,IAAImY,qBAAqB9X,YAAcpwB,KAAK+mC,WAAa/mC,KAAKuS,MAAQ,EAAIvS,KAAKyO,QAAQy3B,iBACxHlgC,EAAM0iC,gBAAkB,EACxB1iC,EAAM2iC,eAAiB3oC,KAAK20B,KAAK5E,IAAImY,qBAAqB9X,YAAcpwB,KAAK+mC,WAAa/mC,KAAKuS,MAAQ,EAAIvS,KAAKyO,QAAQw3B,iBACxHjgC,EAAM4iC,gBAAkB,EAGL,QAAfrU,GACFjV,EAAMrS,MAAMtF,IAAM,IAClB2X,EAAMrS,MAAMxF,KAAO,IACnB6X,EAAMrS,MAAMqW,OAAS,GACrBhE,EAAMrS,MAAMsF,MAAQvS,KAAKuS,MAAQ,KACjC+M,EAAMrS,MAAMuF,OAASxS,KAAKwS,OAAS,KACnCxS,KAAKgG,MAAMuM,MAAQvS,KAAK20B,KAAKC,SAASntB,KAAK8K,MAC3CvS,KAAKgG,MAAMwM,OAASxS,KAAK20B,KAAKC,SAASntB,KAAK+K,SAG5C8M,EAAMrS,MAAMtF,IAAM,GAClB2X,EAAMrS,MAAMqW,OAAS,IACrBhE,EAAMrS,MAAMxF,KAAO,IACnB6X,EAAMrS,MAAMsF,MAAQvS,KAAKuS,MAAQ,KACjC+M,EAAMrS,MAAMuF,OAASxS,KAAKwS,OAAS,KACnCxS,KAAKgG,MAAMuM,MAAQvS,KAAK20B,KAAKC,SAASvN,MAAM9U,MAC5CvS,KAAKgG,MAAMwM,OAASxS,KAAK20B,KAAKC,SAASvN,MAAM7U,QAG/CiyB,EAAUzkC,KAAK6oC,gBACfpE,EAAUzkC,KAAKwkC,cAAgBC,EAEL,GAAtBzkC,KAAKyO,QAAQu3B,MACfhmC,KAAK6nC,oBAGL7nC,KAAKioC,gBAGPjoC,KAAK8oC,aAAavU,GAEpB,MAAOkQ,IAOT/hC,EAASyQ,UAAU01B,cAAgB,WACjC,GAAIpE,IAAU,CACd7jC,GAAQiQ,gBAAgB7Q,KAAKumC,YAAYC,OACzC5lC,EAAQiQ,gBAAgB7Q,KAAKumC,YAAYE,OAEzC,IAAIlS,GAAcv0B,KAAKyO,QAAqB,YAGxC2sB,EAAcp7B,KAAKgnC,OAAShnC,KAAKgG,MAAMwiC,iBAAmB,GAAKxoC,KAAK6mC,iBAEpE1e,EAAO,GAAIvmB,GACb5B,KAAKy1B,MAAM7lB,MACX5P,KAAKy1B,MAAM5lB,IACXurB,EACAp7B,KAAK+vB,IAAIzQ,MAAMgR,aACftwB,KAAKyO,QAAQ6sB,YAAYt7B,KAAKyO,QAAQ8lB,aACvB,GAAfv0B,KAAKgnC,QAAmBhnC,KAAKyO,QAAQ8sB,WAGvCv7B,MAAKmoB,KAAOA,CAGZ,IAAIye,IAAc5mC,KAAK+vB,IAAIzQ,MAAMgR,aAAgBnI,EAAKyT,WAAa57B,KAAK+vB,IAAIzQ,MAAMgR,aAAenI,EAAKwU,gBAAoBxU,EAAKwU,YAAcxU,EAAKyT,WAAazT,EAAKA,KAEpKnoB,MAAK4mC,WAAaA,CAElB,IAAImC,GAAgB/oC,KAAKwS,OAASo0B,EAC9BoC,EAAiB,CAGrB,IAAmB,GAAfhpC,KAAKgnC,OAAiB,CACxBJ,EAAa5mC,KAAK6mC,iBAClBmC,EAAiB9jC,KAAKwoB,MAAO1tB,KAAK+vB,IAAIzQ,MAAMgR,aAAesW,EAAcmC,EACzE,KAAK,GAAIvjC,GAAI,EAAO,GAAMwjC,EAAVxjC,EAA0BA,IACxC2iB,EAAK2U,UAIP,IAFAiM,EAAgB/oC,KAAKwS,OAASo0B,EAEL,IAArB5mC,KAAK8mC,cAAiD,GAA3B9mC,KAAKyO,QAAQ8sB,WAAoB,CAC9D,GAAI0N,GAAsB9gB,EAAKwT,UAAYxT,EAAKA,KAAQnoB,KAAK8mC,YAC7D,IAAImC,EAAqB,EACvB,IAAK,GAAIzjC,GAAI,EAAOyjC,EAAJzjC,EAAwBA,IAAM2iB,EAAKE,WAEhD,IAAyB,EAArB4gB,EACP,IAAK,GAAIzjC,GAAI,GAAQyjC,EAALzjC,EAAyBA,IAAM2iB,EAAK2U,gBAKxDiM,IAAiB,GAInB/oC,MAAKkpC,YAAc/gB,EAAKwT,SACxB,IAMIoB,GANAoM,EAAiB,EAGjBz8B,EAAM,CAI8BlG,UAArCxG,KAAKyO,QAAQ6yB,OAAO/M,KACrBwI,EAAW/8B,KAAKyO,QAAQ6yB,OAAO/M,GAAawI,UAG9C/8B,KAAKopC,aAAe,CAEpB,KADA,GAAIp3B,GAAI,EACDtF,EAAMxH,KAAKwoB,MAAMqb,IAAgB,CACtC5gB,EAAKE,OACLrW,EAAI9M,KAAKwoB,MAAMhhB,EAAMk6B,GACrBuC,EAAiBz8B,EAAMk6B,CACvB,IAAI1J,GAAU/U,EAAK+U,WAEfl9B,KAAKyO,QAAyB,iBAAgB,GAAXyuB,GAAmC,GAAfl9B,KAAKgnC,QAAsD,GAAnChnC,KAAKyO,QAAyB,kBAC/GzO,KAAKqpC,aAAar3B,EAAI,EAAGmW,EAAKC,WAAW2U,GAAWxI,EAAa,cAAev0B,KAAKgG,MAAMsiC,iBAGzFpL,GAAWl9B,KAAKyO,QAAyB,iBAAoB,GAAfzO,KAAKgnC,QAChB,GAAnChnC,KAAKyO,QAAyB,iBAA6B,GAAfzO,KAAKgnC,QAA8B,GAAX9J,GAClElrB,GAAK,GACPhS,KAAKqpC,aAAar3B,EAAI,EAAGmW,EAAKC,WAAW2U,GAAWxI,EAAa,cAAev0B,KAAKgG,MAAMwiC,iBAE7FxoC,KAAKspC,YAAYt3B,EAAGuiB,EAAa,wBAAyBv0B,KAAKyO,QAAQw3B,iBAAkBjmC,KAAKgG,MAAM2iC,iBAGpG3oC,KAAKspC,YAAYt3B,EAAGuiB,EAAa,wBAAyBv0B,KAAKyO,QAAQy3B,iBAAkBlmC,KAAKgG,MAAMyiC,gBAGnF,GAAfzoC,KAAKgnC,QAAkC,GAAhB7e,EAAK2R,UAC9B95B,KAAK8mC,aAAep6B,GAGtBA,IAIA1M,KAAK0mC,iBADY,GAAf1mC,KAAKgnC,OACiBh1B,GAAKhS,KAAKkpC,YAAc/gB,EAAK2R,SAG7B95B,KAAK+vB,IAAIzQ,MAAMgR,aAAenI,EAAKwU,WAI7D,IAAI4M,GAAa,CACuB/iC,UAApCxG,KAAKyO,QAAQu2B,MAAMzQ,IAAuE/tB,SAAzCxG,KAAKyO,QAAQu2B,MAAMzQ,GAAahL,OACnFggB,EAAavpC,KAAKgG,MAAMwjC,gBAE1B,IAAI7f,GAA+B,GAAtB3pB,KAAKyO,QAAQu3B,MAAgB9gC,KAAKwH,IAAI1M,KAAKyO,QAAQ43B,UAAWkD,GAAcvpC,KAAKyO,QAAQ03B,aAAe,GAAKoD,EAAavpC,KAAKyO,QAAQ03B,aAAe,EA0BnK,OAvBInmC,MAAKopC,aAAgBppC,KAAKuS,MAAQoX,GAAmC,GAAxB3pB,KAAKyO,QAAQia,SAC5D1oB,KAAKuS,MAAQvS,KAAKopC,aAAezf,EACjC3pB,KAAKyO,QAAQ8D,MAAQvS,KAAKuS,MAAQ,KAClC3R,EAAQsQ,gBAAgBlR,KAAKumC,YAAYC,OACzC5lC,EAAQsQ,gBAAgBlR,KAAKumC,YAAYE,QACzCzmC,KAAKyhB,SACLgjB,GAAU,GAGHzkC,KAAKopC,aAAgBppC,KAAKuS,MAAQoX,GAAmC,GAAxB3pB,KAAKyO,QAAQia,SAAmB1oB,KAAKuS,MAAQvS,KAAK2mC,UACtG3mC,KAAKuS,MAAQrN,KAAKwH,IAAI1M,KAAK2mC,SAAS3mC,KAAKopC,aAAezf,GACxD3pB,KAAKyO,QAAQ8D,MAAQvS,KAAKuS,MAAQ,KAClC3R,EAAQsQ,gBAAgBlR,KAAKumC,YAAYC,OACzC5lC,EAAQsQ,gBAAgBlR,KAAKumC,YAAYE,QACzCzmC,KAAKyhB,SACLgjB,GAAU,IAGV7jC,EAAQsQ,gBAAgBlR,KAAKumC,YAAYC,OACzC5lC,EAAQsQ,gBAAgBlR,KAAKumC,YAAYE,QACzChC,GAAU,GAGLA,GAGT/hC,EAASyQ,UAAUs2B,aAAe,SAAUpiC,GAC1C,GAAIqiC,GAAgB1pC,KAAKkpC,YAAc7hC,EACnCsiC,EAAiBD,EAAgB1pC,KAAK0mC,gBAC1C,OAAOiD,IAYTjnC,EAASyQ,UAAUk2B,aAAe,SAAUr3B,EAAGuX,EAAMgL,EAAa1sB,EAAW+hC,GAE3E,GAAInhB,GAAQ7nB,EAAQ8Q,cAAc,MAAM1R,KAAKumC,YAAYE,OAAQzmC,KAAK+vB,IAAIzQ,MAC1EmJ,GAAM5gB,UAAYA,EAClB4gB,EAAMxE,UAAYsF,EACC,QAAfgL,GACF9L,EAAMxb,MAAMxF,KAAO,IAAMzH,KAAKyO,QAAQ03B,aAAe,KACrD1d,EAAMxb,MAAMqb,UAAY,UAGxBG,EAAMxb,MAAMoa,MAAQ,IAAMrnB,KAAKyO,QAAQ03B,aAAe,KACtD1d,EAAMxb,MAAMqb,UAAY,QAG1BG,EAAMxb,MAAMtF,IAAMqK,EAAI,GAAM43B,EAAkB5pC,KAAKyO,QAAQ23B,aAAe,KAE1E7c,GAAQ,EAER,IAAIsgB,GAAe3kC,KAAKwH,IAAI1M,KAAKgG,MAAM8jC,eAAe9pC,KAAKgG,MAAM+jC,eAC7D/pC,MAAKopC,aAAe7f,EAAK5jB,OAASkkC,IACpC7pC,KAAKopC,aAAe7f,EAAK5jB,OAASkkC,IAYtCnnC,EAASyQ,UAAUm2B,YAAc,SAAUt3B,EAAGuiB,EAAa1sB,EAAW8hB,EAAQpX,GAC5E,GAAmB,GAAfvS,KAAKgnC,OAAgB,CACvB,GAAInX,GAAOjvB,EAAQ8Q,cAAc,MAAM1R,KAAKumC,YAAYC,MAAOxmC,KAAK+vB,IAAIqX,cACxEvX,GAAKhoB,UAAYA,EACjBgoB,EAAK5L,UAAY,GAEE,QAAfsQ,EACF1E,EAAK5iB,MAAMxF,KAAQzH,KAAKuS,MAAQoX,EAAU,KAG1CkG,EAAK5iB,MAAMoa,MAASrnB,KAAKuS,MAAQoX,EAAU,KAG7CkG,EAAK5iB,MAAMsF,MAAQA,EAAQ,KAC3Bsd,EAAK5iB,MAAMtF,IAAMqK,EAAI,OASzBtP,EAASyQ,UAAU21B,aAAe,SAAUvU,GAI1C,GAHA3zB,EAAQiQ,gBAAgB7Q,KAAKumC,YAAYvB,OAGDx+B,SAApCxG,KAAKyO,QAAQu2B,MAAMzQ,IAAuE/tB,SAAzCxG,KAAKyO,QAAQu2B,MAAMzQ,GAAahL,KAAoB,CACvG,GAAIyb,GAAQpkC,EAAQ8Q,cAAc,MAAO1R,KAAKumC,YAAYvB,MAAOhlC,KAAK+vB,IAAIzQ,MAC1E0lB,GAAMn9B,UAAY,eAAiB0sB,EACnCyQ,EAAM/gB,UAAYjkB,KAAKyO,QAAQu2B,MAAMzQ,GAAahL,KAGJ/iB,SAA1CxG,KAAKyO,QAAQu2B,MAAMzQ,GAAatnB,OAClCtM,EAAK2M,WAAW03B,EAAOhlC,KAAKyO,QAAQu2B,MAAMzQ,GAAatnB,OAGtC,QAAfsnB,EACFyQ,EAAM/3B,MAAMxF,KAAOzH,KAAKgG,MAAMwjC,gBAAkB,KAGhDxE,EAAM/3B,MAAMoa,MAAQrnB,KAAKgG,MAAMwjC,gBAAkB,KAGnDxE,EAAM/3B,MAAMsF,MAAQvS,KAAKwS,OAAS,KAIpC5R,EAAQsQ,gBAAgBlR,KAAKumC,YAAYvB,QAW3CtiC,EAASyQ,UAAUi1B,mBAAqB,WAEtC,KAAM,mBAAqBpoC,MAAKgG,OAAQ,CACtC,GAAIgkC,GAAYz4B,SAAS04B,eAAe,KACpCC,EAAmB34B,SAASM,cAAc,MAC9Cq4B,GAAiBriC,UAAY,sBAC7BqiC,EAAiBz4B,YAAYu4B,GAC7BhqC,KAAK+vB,IAAIzQ,MAAM7N,YAAYy4B,GAE3BlqC,KAAKgG,MAAMsiC,gBAAkB4B,EAAiBrlB,aAC9C7kB,KAAKgG,MAAM+jC,eAAiBG,EAAiB1qB,YAE7Cxf,KAAK+vB,IAAIzQ,MAAMnO,YAAY+4B,GAG7B,KAAM,mBAAqBlqC,MAAKgG,OAAQ,CACtC,GAAImkC,GAAY54B,SAAS04B,eAAe,KACpCG,EAAmB74B,SAASM,cAAc,MAC9Cu4B,GAAiBviC,UAAY,sBAC7BuiC,EAAiB34B,YAAY04B,GAC7BnqC,KAAK+vB,IAAIzQ,MAAM7N,YAAY24B,GAE3BpqC,KAAKgG,MAAMwiC,gBAAkB4B,EAAiBvlB,aAC9C7kB,KAAKgG,MAAM8jC,eAAiBM,EAAiB5qB,YAE7Cxf,KAAK+vB,IAAIzQ,MAAMnO,YAAYi5B,GAG7B,KAAM,mBAAqBpqC,MAAKgG,OAAQ,CACtC,GAAIqkC,GAAY94B,SAAS04B,eAAe,KACpCK,EAAmB/4B,SAASM,cAAc,MAC9Cy4B,GAAiBziC,UAAY,sBAC7ByiC,EAAiB74B,YAAY44B,GAC7BrqC,KAAK+vB,IAAIzQ,MAAM7N,YAAY64B,GAE3BtqC,KAAKgG,MAAMwjC,gBAAkBc,EAAiBzlB,aAC9C7kB,KAAKgG,MAAMukC,eAAiBD,EAAiB9qB,YAE7Cxf,KAAK+vB,IAAIzQ,MAAMnO,YAAYm5B,KAU/B5nC,EAASyQ,UAAU6hB,KAAO,SAASyD,GACjC,MAAOz4B,MAAKmoB,KAAK6M,KAAKyD,IAGxB54B,EAAOD,QAAU8C,GAKb,SAAS7C,EAAQD,EAASM,GAkB9B,QAASyC,GAAYsP,EAAOqlB,EAAS7oB,EAAS+7B,GAC5CxqC,KAAKK,GAAKi3B,CACV,IAAIppB,IAAU,WAAW,QAAQ,OAAO,mBAAmB,WAAW,aAAa,SAAS,aAC5FlO,MAAKyO,QAAU9N,EAAKsN,sBAAsBC,EAAOO,GACjDzO,KAAKyqC,kBAAwCjkC,SAApByL,EAAMpK,UAC/B7H,KAAKwqC,yBAA2BA,EAChCxqC,KAAK0qC,aAAe,EACpB1qC,KAAK6U,OAAO5C,GACkB,GAA1BjS,KAAKyqC,oBACPzqC,KAAKwqC,yBAAyB,IAAM,GAEtCxqC,KAAK81B,aACL91B,KAAK0oB,QAA4BliB,SAAlByL,EAAMyW,SAAwB,EAAOzW,EAAMyW,QA5B5D,GAAI/nB,GAAOT,EAAoB,GAC3BU,EAAUV,EAAoB,GAC9ByqC,EAAOzqC,EAAoB,IAC3B0qC,EAAM1qC,EAAoB,IAC1B2qC,EAAS3qC,EAAoB,GAgCjCyC,GAAWwQ,UAAU8iB,SAAW,SAASh0B,GAC1B,MAATA,GACFjC,KAAK81B,UAAY7zB,EACQ,GAArBjC,KAAKyO,QAAQyH,MACflW,KAAK81B,UAAU5f,KAAK,SAAU3Q,EAAEa,GAAI,MAAOb,GAAEwM,EAAI3L,EAAE2L,KAIrD/R,KAAK81B,cASTnzB,EAAWwQ,UAAU23B,gBAAkB,SAASvlB,GAC9CvlB,KAAK0qC,aAAenlB,GAQtB5iB,EAAWwQ,UAAUD,WAAa,SAASzE,GACzC,GAAgBjI,SAAZiI,EAAuB,CACzB,GAAIP,IAAU,WAAW,QAAQ,OAAO,mBAAmB,WAC3DvN,GAAKwF,oBAAoB+H,EAAQlO,KAAKyO,QAASA,GAE/C9N,EAAK4N,aAAavO,KAAKyO,QAASA,EAAQ,cACxC9N,EAAK4N,aAAavO,KAAKyO,QAASA,EAAQ,cACxC9N,EAAK4N,aAAavO,KAAKyO,QAASA,EAAQ,UAEpCA,EAAQs8B,YACuB,gBAAtBt8B,GAAQs8B,YACbt8B,EAAQs8B,WAAWC,kBACqB,WAAtCv8B,EAAQs8B,WAAWC,gBACrBhrC,KAAKyO,QAAQs8B,WAAWE,MAAQ,EAEa,WAAtCx8B,EAAQs8B,WAAWC,gBAC1BhrC,KAAKyO,QAAQs8B,WAAWE,MAAQ,GAGhCjrC,KAAKyO,QAAQs8B,WAAWC,gBAAkB,cAC1ChrC,KAAKyO,QAAQs8B,WAAWE,MAAQ,KAOhB,QAAtBjrC,KAAKyO,QAAQxB,MACfjN,KAAK8G,KAAO,GAAI6jC,GAAK3qC,KAAKK,GAAIL,KAAKyO,SAEN,OAAtBzO,KAAKyO,QAAQxB,MACpBjN,KAAK8G,KAAO,GAAI8jC,GAAI5qC,KAAKK,GAAIL,KAAKyO,SAEL,UAAtBzO,KAAKyO,QAAQxB,QACpBjN,KAAK8G,KAAO,GAAI+jC,GAAO7qC,KAAKK,GAAIL,KAAKyO,WASzC9L,EAAWwQ,UAAU0B,OAAS,SAAS5C,GACrCjS,KAAKiS,MAAQA,EACbjS,KAAK4vB,QAAU3d,EAAM2d,SAAW,QAChC5vB,KAAK6H,UAAYoK,EAAMpK,WAAa7H,KAAK6H,WAAa,aAAe7H,KAAKwqC,yBAAyB,GAAK,GACxGxqC,KAAK0oB,QAA4BliB,SAAlByL,EAAMyW,SAAwB,EAAOzW,EAAMyW,QAC1D1oB,KAAKiN,MAAQgF,EAAMhF,MACnBjN,KAAKkT,WAAWjB,EAAMxD,UAcxB9L,EAAWwQ,UAAU60B,SAAW,SAASj2B,EAAGC,EAAGlB,EAAeo6B,EAAc7E,EAAWyB,GACrF,GACIqD,GAAMC,EADNC,EAA0B,GAAbvD,EAGbwD,EAAU1qC,EAAQwQ,cAAc,OAAQN,EAAeo6B,EAO3D,IANAI,EAAQl5B,eAAe,KAAM,IAAKL,GAClCu5B,EAAQl5B,eAAe,KAAM,IAAKJ,EAAIq5B,GACtCC,EAAQl5B,eAAe,KAAM,QAASi0B,GACtCiF,EAAQl5B,eAAe,KAAM,SAAU,EAAEi5B,GACzCC,EAAQl5B,eAAe,KAAM,QAAS,WAEZ,QAAtBpS,KAAKyO,QAAQxB,MACfk+B,EAAOvqC,EAAQwQ,cAAc,OAAQN,EAAeo6B,GACpDC,EAAK/4B,eAAe,KAAM,QAASpS,KAAK6H,WACtBrB,SAAfxG,KAAKiN,OACNk+B,EAAK/4B,eAAe,KAAM,QAASpS,KAAKiN,OAG1Ck+B,EAAK/4B,eAAe,KAAM,IAAK,IAAML,EAAI,IAAIC,EAAE,MAAQD,EAAIs0B,GAAa,IAAIr0B,GACzC,GAA/BhS,KAAKyO,QAAQ88B,OAAO78B,UACtB08B,EAAWxqC,EAAQwQ,cAAc,OAAQN,EAAeo6B,GACjB,OAAnClrC,KAAKyO,QAAQ88B,OAAOhX,YACtB6W,EAASh5B,eAAe,KAAM,IAAK,IAAIL,EAAE,MAAQC,EAAIq5B,GACnD,IAAIt5B,EAAE,IAAIC,EAAE,MAAOD,EAAIs0B,GAAa,IAAIr0B,EAAE,MAAOD,EAAIs0B,GAAa,KAAOr0B,EAAIq5B,IAG/ED,EAASh5B,eAAe,KAAM,IAAK,IAAIL,EAAE,IAAIC,EAAE,KACzCD,EAAE,KAAOC,EAAIq5B,GAAc,MACzBt5B,EAAIs0B,GAAa,KAAOr0B,EAAIq5B,GAClC,KAAMt5B,EAAIs0B,GAAa,IAAIr0B,GAE/Bo5B,EAASh5B,eAAe,KAAM,QAASpS,KAAK6H,UAAY,cAGnB,GAAnC7H,KAAKyO,QAAQ0D,WAAWzD,SAC1B9N,EAAQkR,UAAUC,EAAI,GAAMs0B,EAAUr0B,EAAGhS,KAAM8Q,EAAeo6B,OAG7D,CACH,GAAIM,GAAWtmC,KAAKwoB,MAAM,GAAM2Y,GAC5BoF,EAAavmC,KAAKwoB,MAAM,GAAMoa,GAC9B4D,EAAaxmC,KAAKwoB,MAAM,IAAOoa,GAE/Bne,EAASzkB,KAAKwoB,OAAO2Y,EAAa,EAAImF,GAAW,EAErD5qC,GAAQ0R,QAAQP,EAAI,GAAIy5B,EAAW7hB,EAAY3X,EAAIq5B,EAAaI,EAAa,EAAGD,EAAUC,EAAYzrC,KAAK6H,UAAY,OAAQiJ,EAAeo6B,GAC9ItqC,EAAQ0R,QAAQP,EAAI,IAAIy5B,EAAW7hB,EAAS,EAAG3X,EAAIq5B,EAAaK,EAAa,EAAGF,EAAUE,EAAY1rC,KAAK6H,UAAY,OAAQiJ,EAAeo6B,KAYlJvoC,EAAWwQ,UAAUkkB,UAAY,SAASgP,EAAWyB,GACnD,GAAIlC,GAAMr0B,SAASC,gBAAgB,6BAA6B,MAEhE,OADAxR,MAAKgoC,SAAS,EAAE,GAAIF,KAAclC,EAAIS,EAAUyB,IACxC6D,KAAM/F,EAAKnd,MAAOzoB,KAAK4vB,QAAS2E,YAAYv0B,KAAKyO,QAAQm9B,mBAGnEjpC,EAAWwQ,UAAU04B,UAAY,SAASC,GACxC,MAAO9rC,MAAK8G,KAAK+kC,UAAUC,IAG7BnpC,EAAWwQ,UAAU44B,KAAO,SAAS/U,EAAS/kB,EAAO+5B,GACnDhsC,KAAK8G,KAAKilC,KAAK/U,EAAS/kB,EAAO+5B,IAIjCnsC,EAAOD,QAAU+C,GAKb,SAAS9C,EAAQD,EAASM,GAY9B,QAAS0C,GAAO00B,EAAS5kB,EAAMmjB,GAC7B71B,KAAKs3B,QAAUA,EACft3B,KAAKmhC,aACLnhC,KAAKisC,cAAgB,EACrBjsC,KAAKksC,gBAAkBx5B,GAAQA,EAAKy5B,cACpCnsC,KAAK61B,QAAUA,EAEf71B,KAAK+vB,OACL/vB,KAAKgG,OACHyiB,OACElW,MAAO,EACPC,OAAQ,IAGZxS,KAAK6H,UAAY,KAEjB7H,KAAKiC,SACLjC,KAAKosC,gBACLpsC,KAAK4O,cACHy9B,WACAC,UAEFtsC,KAAKusC,kBAAmB,CACxB,IAAIp4B,GAAKnU,IACTA,MAAK61B,QAAQlB,KAAKE,QAAQthB,GAAG,mBAAoB,WAC/CY,EAAGo4B,kBAAmB,IAGxBvsC,KAAK00B,UAEL10B,KAAKgY,QAAQtF,GAxCf,CAAA,GAAI/R,GAAOT,EAAoB,GAC3B4B,EAAQ5B,EAAoB,GAChBA,GAAoB,IA6CpC0C,EAAMuQ,UAAUuhB,QAAU,WACxB,GAAIjM,GAAQlX,SAASM,cAAc,MACnC4W,GAAM5gB,UAAY,SAClB7H,KAAK+vB,IAAItH,MAAQA,CAEjB,IAAI+jB,GAAQj7B,SAASM,cAAc,MACnC26B,GAAM3kC,UAAY,QAClB4gB,EAAMhX,YAAY+6B,GAClBxsC,KAAK+vB,IAAIyc,MAAQA,CAEjB,IAAIC,GAAal7B,SAASM,cAAc,MACxC46B,GAAW5kC,UAAY,QACvB4kC,EAAW,kBAAoBzsC,KAC/BA,KAAK+vB,IAAI0c,WAAaA,EAEtBzsC,KAAK+vB,IAAI5jB,WAAaoF,SAASM,cAAc,OAC7C7R,KAAK+vB,IAAI5jB,WAAWtE,UAAY,QAEhC7H,KAAK+vB,IAAI+Q,KAAOvvB,SAASM,cAAc,OACvC7R,KAAK+vB,IAAI+Q,KAAKj5B,UAAY,QAK1B7H,KAAK+vB,IAAI2c,OAASn7B,SAASM,cAAc,OACzC7R,KAAK+vB,IAAI2c,OAAOz/B,MAAMuqB,WAAa,SACnCx3B,KAAK+vB,IAAI2c,OAAOzoB,UAAY,IAC5BjkB,KAAK+vB,IAAI5jB,WAAWsF,YAAYzR,KAAK+vB,IAAI2c,SAO3C9pC,EAAMuQ,UAAU6E,QAAU,SAAStF,GAEjC,GAAIkd,GAAUld,GAAQA,EAAKkd,OACvBA,aAAmB+c,SACrB3sC,KAAK+vB,IAAIyc,MAAM/6B,YAAYme,GAG3B5vB,KAAK+vB,IAAIyc,MAAMvoB,UADIzd,SAAZopB,GAAqC,OAAZA,EACLA,EAGA5vB,KAAKs3B,SAAW,GAI7Ct3B,KAAK+vB,IAAItH,MAAMuc,MAAQtyB,GAAQA,EAAKsyB,OAAS,GAExChlC,KAAK+vB,IAAIyc,MAAM7oB,WAIlBhjB,EAAKuH,gBAAgBlI,KAAK+vB,IAAIyc,MAAO,UAHrC7rC,EAAKiH,aAAa5H,KAAK+vB,IAAIyc,MAAO,SAOpC,IAAI3kC,GAAY6K,GAAQA,EAAK7K,WAAa,IACtCA,IAAa7H,KAAK6H,YAChB7H,KAAK6H,YACPlH,EAAKuH,gBAAgBlI,KAAK+vB,IAAItH,MAAOzoB,KAAK6H,WAC1ClH,EAAKuH,gBAAgBlI,KAAK+vB,IAAI0c,WAAYzsC,KAAK6H,WAC/ClH,EAAKuH,gBAAgBlI,KAAK+vB,IAAI5jB,WAAYnM,KAAK6H,WAC/ClH,EAAKuH,gBAAgBlI,KAAK+vB,IAAI+Q,KAAM9gC,KAAK6H,YAE3ClH,EAAKiH,aAAa5H,KAAK+vB,IAAItH,MAAO5gB,GAClClH,EAAKiH,aAAa5H,KAAK+vB,IAAI0c,WAAY5kC,GACvClH,EAAKiH,aAAa5H,KAAK+vB,IAAI5jB,WAAYtE,GACvClH,EAAKiH,aAAa5H,KAAK+vB,IAAI+Q,KAAMj5B,GACjC7H,KAAK6H,UAAYA,GAIf7H,KAAKiN,QACPtM,EAAK8M,cAAczN,KAAK+vB,IAAItH,MAAOzoB,KAAKiN,OACxCjN,KAAKiN,MAAQ,MAEXyF,GAAQA,EAAKzF,QACftM,EAAK2M,WAAWtN,KAAK+vB,IAAItH,MAAO/V,EAAKzF,OACrCjN,KAAKiN,MAAQyF,EAAKzF,QAQtBrK,EAAMuQ,UAAUy5B,cAAgB,WAC9B,MAAO5sC,MAAKgG,MAAMyiB,MAAMlW,OAW1B3P,EAAMuQ,UAAUsO,OAAS,SAASgU,EAAO/b,EAAQmzB,GAC/C,GAAIpI,IAAU,CAEdzkC,MAAKosC,aAAepsC,KAAK8sC,oBAAoB9sC,KAAK4O,aAAc5O,KAAKosC,aAAc3W,EAInF,IAAIsX,GAAe/sC,KAAK+vB,IAAI2c,OAAO7nB,YAC/BkoB,IAAgB/sC,KAAKgtC,mBACvBhtC,KAAKgtC,iBAAmBD,EAExBpsC,EAAK0H,QAAQrI,KAAKiC,MAAO,SAAUoN,GACjCA,EAAK49B,OAAQ,EACT59B,EAAK69B,WAAW79B,EAAKoS,WAG3BorB,GAAU,GAIR7sC,KAAK61B,QAAQpnB,QAAQ3M,MACvBA,EAAMA,MAAM9B,KAAKosC,aAAc1yB,EAAQmzB,GAGvC/qC,EAAMo/B,QAAQlhC,KAAKosC,aAAc1yB,EAAQ1Z,KAAKmhC,UAIhD,IAAI3uB,GAASxS,KAAKmtC,iBAAiBzzB,GAG/B+yB,EAAazsC,KAAK+vB,IAAI0c,UAC1BzsC,MAAK2H,IAAM8kC,EAAWW,UACtBptC,KAAKyH,KAAOglC,EAAWY,WACvBrtC,KAAKuS,MAAQk6B,EAAWrc,YACxBqU,EAAU9jC,EAAK8H,eAAezI,KAAM,SAAUwS,IAAWiyB,EAGzDA,EAAU9jC,EAAK8H,eAAezI,KAAKgG,MAAMyiB,MAAO,QAASzoB,KAAK+vB,IAAIyc,MAAMhtB,cAAgBilB,EACxFA,EAAU9jC,EAAK8H,eAAezI,KAAKgG,MAAMyiB,MAAO,SAAUzoB,KAAK+vB,IAAIyc,MAAM3nB,eAAiB4f,EAG1FzkC,KAAK+vB,IAAI5jB,WAAWc,MAAMuF,OAAUA,EAAS,KAC7CxS,KAAK+vB,IAAI0c,WAAWx/B,MAAMuF,OAAUA,EAAS,KAC7CxS,KAAK+vB,IAAItH,MAAMxb,MAAMuF,OAASA,EAAS,IAGvC,KAAK,GAAIhN,GAAI,EAAG8nC,EAAKttC,KAAKosC,aAAazmC,OAAY2nC,EAAJ9nC,EAAQA,IAAK,CAC1D,GAAI6J,GAAOrP,KAAKosC,aAAa5mC,EAC7B6J,GAAKk+B,YAAY7zB,GAGnB,MAAO+qB,IAST7hC,EAAMuQ,UAAUg6B,iBAAmB,SAAUzzB,GAE3C,GAAIlH,GACA45B,EAAepsC,KAAKosC,YAGxBpsC,MAAKwtC,gBACL,IAAIr5B,GAAKnU,IACT,IAAIosC,EAAazmC,OAAQ,CACvB,GAAImG,GAAMsgC,EAAa,GAAGzkC,IACtB+E,EAAM0/B,EAAa,GAAGzkC,IAAMykC,EAAa,GAAG55B,MAahD,IAZA7R,EAAK0H,QAAQ+jC,EAAc,SAAU/8B,GACnCvD,EAAM5G,KAAK4G,IAAIA,EAAKuD,EAAK1H,KACzB+E,EAAMxH,KAAKwH,IAAIA,EAAM2C,EAAK1H,IAAM0H,EAAKmD,QACVhM,SAAvB6I,EAAKqD,KAAK2uB,WACZltB,EAAGgtB,UAAU9xB,EAAKqD,KAAK2uB,UAAU7uB,OAAStN,KAAKwH,IAAIyH,EAAGgtB,UAAU9xB,EAAKqD,KAAK2uB,UAAU7uB,OAAOnD,EAAKmD,QAChG2B,EAAGgtB,UAAU9xB,EAAKqD,KAAK2uB,UAAU3Y,SAAU,KAO3C5c,EAAM4N,EAAOonB,KAAM,CAErB,GAAInX,GAAS7d,EAAM4N,EAAOonB,IAC1Bp0B,IAAOid,EACPhpB,EAAK0H,QAAQ+jC,EAAc,SAAU/8B,GACnCA,EAAK1H,KAAOgiB,IAGhBnX,EAAS9F,EAAMgN,EAAOrK,KAAKoW,SAAW,MAGtCjT,GAASkH,EAAOonB,KAAOpnB,EAAOrK,KAAKoW,QAIrC,OAFAjT,GAAStN,KAAKwH,IAAI8F,EAAQxS,KAAKgG,MAAMyiB,MAAMjW,SAQ7C5P,EAAMuQ,UAAUw0B,KAAO,WAChB3nC,KAAK+vB,IAAItH,MAAM5e,YAClB7J,KAAK61B,QAAQ9F,IAAI0d,SAASh8B,YAAYzR,KAAK+vB,IAAItH,OAG5CzoB,KAAK+vB,IAAI0c,WAAW5iC,YACvB7J,KAAK61B,QAAQ9F,IAAI0c,WAAWh7B,YAAYzR,KAAK+vB,IAAI0c,YAG9CzsC,KAAK+vB,IAAI5jB,WAAWtC,YACvB7J,KAAK61B,QAAQ9F,IAAI5jB,WAAWsF,YAAYzR,KAAK+vB,IAAI5jB,YAG9CnM,KAAK+vB,IAAI+Q,KAAKj3B,YACjB7J,KAAK61B,QAAQ9F,IAAI+Q,KAAKrvB,YAAYzR,KAAK+vB,IAAI+Q,OAO/Cl+B,EAAMuQ,UAAUu0B,KAAO,WACrB,GAAIjf,GAAQzoB,KAAK+vB,IAAItH,KACjBA,GAAM5e,YACR4e,EAAM5e,WAAWsH,YAAYsX,EAG/B,IAAIgkB,GAAazsC,KAAK+vB,IAAI0c,UACtBA,GAAW5iC,YACb4iC,EAAW5iC,WAAWsH,YAAYs7B,EAGpC,IAAItgC,GAAanM,KAAK+vB,IAAI5jB,UACtBA,GAAWtC,YACbsC,EAAWtC,WAAWsH,YAAYhF,EAGpC,IAAI20B,GAAO9gC,KAAK+vB,IAAI+Q,IAChBA,GAAKj3B,YACPi3B,EAAKj3B,WAAWsH,YAAY2vB,IAQhCl+B,EAAMuQ,UAAUF,IAAM,SAAS5D,GAc7B,GAbArP,KAAKiC,MAAMoN,EAAKhP,IAAMgP,EACtBA,EAAKq+B,UAAU1tC,MAGYwG,SAAvB6I,EAAKqD,KAAK2uB,WAC+B76B,SAAvCxG,KAAKmhC,UAAU9xB,EAAKqD,KAAK2uB,YAC3BrhC,KAAKmhC,UAAU9xB,EAAKqD,KAAK2uB,WAAa7uB,OAAO,EAAGkW,SAAS,EAAOvgB,MAAMnI,KAAKisC,cAAehqC,UAC1FjC,KAAKisC,iBAEPjsC,KAAKmhC,UAAU9xB,EAAKqD,KAAK2uB,UAAUp/B,MAAM+F,KAAKqH,IAEhDrP,KAAK2tC,iBAEkC,IAAnC3tC,KAAKosC,aAAazlC,QAAQ0I,GAAa,CACzC,GAAIomB,GAAQz1B,KAAK61B,QAAQlB,KAAKc,KAC9Bz1B,MAAK4tC,gBAAgBv+B,EAAMrP,KAAKosC,aAAc3W,KAIlD7yB,EAAMuQ,UAAUw6B,eAAiB,WAC/B,GAA6BnnC,SAAzBxG,KAAKksC,gBAA+B,CACtC,GAAI2B,KACJ,IAAmC,gBAAxB7tC,MAAKksC,gBAA6B,CAC3C,IAAK,GAAI7K,KAAYrhC,MAAKmhC,UACxB0M,EAAU7lC,MAAMq5B,SAAUA,EAAUyM,UAAW9tC,KAAKmhC,UAAUE,GAAUp/B,MAAM,GAAGyQ,KAAK1S,KAAKksC,kBAE7F2B,GAAU33B,KAAK,SAAU3Q,EAAGa,GAC1B,MAAOb,GAAEuoC,UAAY1nC,EAAE0nC,gBAGtB,IAAmC,kBAAxB9tC,MAAKksC,gBAA+B,CAClD,IAAK,GAAI7K,KAAYrhC,MAAKmhC,UACxB0M,EAAU7lC,KAAKhI,KAAKmhC,UAAUE,GAAUp/B,MAAM,GAAGyQ,KAEnDm7B,GAAU33B,KAAKlW,KAAKksC,iBAGtB,GAAI2B,EAAUloC,OAAS,EACrB,IAAK,GAAIH,GAAI,EAAGA,EAAIqoC,EAAUloC,OAAQH,IACpCxF,KAAKmhC,UAAU0M,EAAUroC,GAAG67B,UAAUl5B,MAAQ3C,IAMtD5C,EAAMuQ,UAAUq6B,eAAiB,WAC/B,IAAK,GAAInM,KAAYrhC,MAAKmhC,UACpBnhC,KAAKmhC,UAAUr7B,eAAeu7B,KAChCrhC,KAAKmhC,UAAUE,GAAU3Y,SAAU,IASzC9lB,EAAMuQ,UAAUkD,OAAS,SAAShH,SACzBrP,MAAKiC,MAAMoN,EAAKhP,IACvBgP,EAAKq+B,UAAU,KAGf,IAAIvlC,GAAQnI,KAAKosC,aAAazlC,QAAQ0I,EACzB,KAATlH,GAAanI,KAAKosC,aAAahkC,OAAOD,EAAO,IAUnDvF,EAAMuQ,UAAU46B,kBAAoB,SAAS1+B,GAC3CrP,KAAK61B,QAAQmY,WAAW3+B,EAAKhP,KAO/BuC,EAAMuQ,UAAUsC,MAAQ,WAKtB,IAAK,GAJDjN,GAAQ7H,EAAK4H,QAAQvI,KAAKiC,OAC1BgsC,KACAC,KAEK1oC,EAAI,EAAGA,EAAIgD,EAAM7C,OAAQH,IACNgB,SAAtBgC,EAAMhD,GAAGkN,KAAK7C,KAChBq+B,EAASlmC,KAAKQ,EAAMhD,IAEtByoC,EAAWjmC,KAAKQ,EAAMhD,GAExBxF,MAAK4O,cACHy9B,QAAS4B,EACT3B,MAAO4B,GAGTpsC,EAAM0+B,aAAaxgC,KAAK4O,aAAay9B,SACrCvqC,EAAM2+B,WAAWzgC,KAAK4O,aAAa09B,QAYrC1pC,EAAMuQ,UAAU25B,oBAAsB,SAASl+B,EAAcu/B,EAAiB1Y,GAC5E,GAKIpmB,GAAM7J,EALN4mC,KACAgC,KACA5b,GAAYiD,EAAM5lB,IAAM4lB,EAAM7lB,OAAS,EACvCy+B,EAAa5Y,EAAM7lB,MAAQ4iB,EAC3B8b,EAAa7Y,EAAM5lB,IAAM2iB,EAIzB3jB,EAAiB,SAAUxH,GAC7B,MAAiBgnC,GAARhnC,EAA6B,GACpBinC,GAATjnC,EAA8B,EACA,EAMzC,IAAI8mC,EAAgBxoC,OAAS,EAC3B,IAAKH,EAAI,EAAGA,EAAI2oC,EAAgBxoC,OAAQH,IACtCxF,KAAKuuC,6BAA6BJ,EAAgB3oC,GAAI4mC,EAAcgC,EAAoB3Y,EAK5F,IAAI+Y,GAAoB7tC,EAAKgO,mBAAmBC,EAAay9B,QAASx9B,EAAgB,OAAO,QAS7F,IANA7O,KAAKyuC,cAAcD,EAAmB5/B,EAAay9B,QAASD,EAAcgC,EAAoB,SAAU/+B,GACtG,MAAQA,GAAKqD,KAAK9C,MAAQy+B,GAAch/B,EAAKqD,KAAK9C,MAAQ0+B,IAK/B,GAAzBtuC,KAAKusC,iBAEP,IADAvsC,KAAKusC,kBAAmB,EACnB/mC,EAAI,EAAGA,EAAIoJ,EAAa09B,MAAM3mC,OAAQH,IACzCxF,KAAKuuC,6BAA6B3/B,EAAa09B,MAAM9mC,GAAI4mC,EAAcgC,EAAoB3Y,OAG1F,CAEH,GAAIiZ,GAAkB/tC,EAAKgO,mBAAmBC,EAAa09B,MAAOz9B,EAAgB,OAAO,MAGzF7O,MAAKyuC,cAAcC,EAAiB9/B,EAAa09B,MAAOF,EAAcgC,EAAoB,SAAU/+B,GAClG,MAAQA,GAAKqD,KAAK7C,IAAMw+B,GAAch/B,EAAKqD,KAAK7C,IAAMy+B,IAM1D,IAAK9oC,EAAI,EAAGA,EAAI4mC,EAAazmC,OAAQH,IACnC6J,EAAO+8B,EAAa5mC,GACf6J,EAAK69B,WAAW79B,EAAKs4B,OAE1Bt4B,EAAKs/B,aAgBP,OAAOvC,IAGTxpC,EAAMuQ,UAAUs7B,cAAgB,SAAUG,EAAY3sC,EAAOmqC,EAAcgC,EAAoBS,GAC7F,GAAIx/B,GACA7J,CAEJ,IAAkB,IAAdopC,EAAkB,CACpB,IAAKppC,EAAIopC,EAAYppC,GAAK,IACxB6J,EAAOpN,EAAMuD,IACTqpC,EAAex/B,IAFQ7J,IAMWgB,SAAhC4nC,EAAmB/+B,EAAKhP,MAC1B+tC,EAAmB/+B,EAAKhP,KAAM,EAC9B+rC,EAAapkC,KAAKqH,GAKxB,KAAK7J,EAAIopC,EAAa,EAAGppC,EAAIvD,EAAM0D,SACjC0J,EAAOpN,EAAMuD,IACTqpC,EAAex/B,IAFsB7J,IAMHgB,SAAhC4nC,EAAmB/+B,EAAKhP,MAC1B+tC,EAAmB/+B,EAAKhP,KAAM,EAC9B+rC,EAAapkC,KAAKqH,MAmB5BzM,EAAMuQ,UAAUy6B,gBAAkB,SAASv+B,EAAM+8B,EAAc3W,GACvDpmB,EAAKy/B,UAAUrZ,IACZpmB,EAAK69B,WAAW79B,EAAKs4B,OAE1Bt4B,EAAKs/B,cACLvC,EAAapkC,KAAKqH,IAGdA,EAAK69B,WAAW79B,EAAKq4B,QAgB/B9kC,EAAMuQ,UAAUo7B,6BAA+B,SAASl/B,EAAM+8B,EAAcgC,EAAoB3Y,GAC1FpmB,EAAKy/B,UAAUrZ,GACmBjvB,SAAhC4nC,EAAmB/+B,EAAKhP,MAC1B+tC,EAAmB/+B,EAAKhP,KAAM,EAC9B+rC,EAAapkC,KAAKqH,IAIhBA,EAAK69B,WAAW79B,EAAKq4B;EAM7B7nC,EAAOD,QAAUgD,GAKb,SAAS/C,EAAQD,EAASM,GAW9B,QAAS2C,GAAiBy0B,EAAS5kB,EAAMmjB,GACvCjzB,EAAMrC,KAAKP,KAAMs3B,EAAS5kB,EAAMmjB,GAEhC71B,KAAKuS,MAAQ,EACbvS,KAAKwS,OAAS,EACdxS,KAAK2H,IAAM,EACX3H,KAAKyH,KAAO,EAfd,GACI7E,IADO1C,EAAoB,GACnBA,EAAoB,IAiBhC2C,GAAgBsQ,UAAY5M,OAAO8H,OAAOzL,EAAMuQ,WAShDtQ,EAAgBsQ,UAAUsO,OAAS,SAASgU,EAAO/b,GACjD,GAAI+qB,IAAU,CAEdzkC,MAAKosC,aAAepsC,KAAK8sC,oBAAoB9sC,KAAK4O,aAAc5O,KAAKosC,aAAc3W,GAGnFz1B,KAAKuS,MAAQvS,KAAK+vB,IAAI5jB,WAAWikB,YAGjCpwB,KAAK+vB,IAAI5jB,WAAWc,MAAMuF,OAAU,GAGpC,KAAK,GAAIhN,GAAI,EAAG8nC,EAAKttC,KAAKosC,aAAazmC,OAAY2nC,EAAJ9nC,EAAQA,IAAK,CAC1D,GAAI6J,GAAOrP,KAAKosC,aAAa5mC,EAC7B6J,GAAKk+B,YAAY7zB,GAGnB,MAAO+qB,IAMT5hC,EAAgBsQ,UAAUw0B,KAAO,WAC1B3nC,KAAK+vB,IAAI5jB,WAAWtC,YACvB7J,KAAK61B,QAAQ9F,IAAI5jB,WAAWsF,YAAYzR,KAAK+vB,IAAI5jB,aAIrDtM,EAAOD,QAAUiD,GAKb,SAAShD,EAAQD,EAASM,GA2B9B,QAAS4C,GAAQ6xB,EAAMlmB,GACrBzO,KAAK20B,KAAOA,EAEZ30B,KAAKq0B,gBACHvtB,KAAM,KACNytB,YAAa,SACbwa,MAAO,OACPjtC,OAAO,EACPktC,WAAY,KAEZC,YAAY,EACZC,UACEC,YAAY,EACZ3H,aAAa,EACbv0B,KAAK,EACLoD,QAAQ,GAGV+4B,MAAO,SAAU//B,EAAM/G,GACrBA,EAAS+G,IAEXggC,SAAU,SAAUhgC,EAAM/G,GACxBA,EAAS+G,IAEXigC,OAAQ,SAAUjgC,EAAM/G,GACtBA,EAAS+G,IAEXkgC,SAAU,SAAUlgC,EAAM/G,GACxBA,EAAS+G,IAEXmgC,SAAU,SAAUngC,EAAM/G,GACxBA,EAAS+G,IAGXqK,QACErK,MACEmW,WAAY,GACZC,SAAU,IAEZqb,KAAM,IAER9c,QAAS,GAIXhkB,KAAKyO,QAAU9N,EAAK2E,UAAWtF,KAAKq0B,gBAGpCr0B,KAAKyvC,aACH3oC,MAAO8I,MAAO,OAAQC,IAAK,SAG7B7P,KAAKo6B,YACHnF,SAAUN,EAAKh0B,KAAKs0B,SACpBI,OAAQV,EAAKh0B,KAAK00B,QAEpBr1B,KAAK+vB,OACL/vB,KAAKgG,SACLhG,KAAK8D,OAAS,IAEd,IAAIqQ,GAAKnU,IACTA,MAAK81B,UAAY,KACjB91B,KAAK+1B,WAAa,KAGlB/1B,KAAK0vC,eACHz8B,IAAO,SAAU3J,EAAOwK,GACtBK,EAAGw7B,OAAO77B,EAAO7R,QAEnB4S,OAAU,SAAUvL,EAAOwK,GACzBK,EAAGy7B,UAAU97B,EAAO7R,QAEtBoU,OAAU,SAAU/M,EAAOwK,GACzBK,EAAG07B,UAAU/7B,EAAO7R,SAKxBjC,KAAK8vC,gBACH78B,IAAO,SAAU3J,EAAOwK,GACtBK,EAAG47B,aAAaj8B,EAAO7R,QAEzB4S,OAAU,SAAUvL,EAAOwK,GACzBK,EAAG67B,gBAAgBl8B,EAAO7R,QAE5BoU,OAAU,SAAU/M,EAAOwK,GACzBK,EAAG87B,gBAAgBn8B,EAAO7R,SAI9BjC,KAAKiC,SACLjC,KAAKm0B,UACLn0B,KAAKkwC,YAELlwC,KAAKmwC,aACLnwC,KAAKowC,YAAa,EAElBpwC,KAAKqwC,eAGLrwC,KAAK00B,UAEL10B,KAAKkT,WAAWzE,GA/HlB,GAAI1K,GAAS7D,EAAoB,IAC7BS,EAAOT,EAAoB,GAC3BW,EAAUX,EAAoB,GAC9BY,EAAWZ,EAAoB,GAC/BqC,EAAYrC,EAAoB,IAChC0C,EAAQ1C,EAAoB,IAC5B2C,EAAkB3C,EAAoB,IACtCkC,EAAUlC,EAAoB,IAC9BmC,EAAYnC,EAAoB,IAChCoC,EAAYpC,EAAoB,IAChCiC,EAAiBjC,EAAoB,IAGrCowC,EAAY,gBACZC,EAAa,gBAoHjBztC,GAAQqQ,UAAY,GAAI5Q,GAGxBO,EAAQoU,OACN/K,WAAYhK,EACZquC,IAAKpuC,EACLqzB,MAAOnzB,EACP4P,MAAO7P,GAMTS,EAAQqQ,UAAUuhB,QAAU,WAC1B,GAAIpV,GAAQ/N,SAASM,cAAc,MACnCyN,GAAMzX,UAAY,UAClByX,EAAM,oBAAsBtf,KAC5BA,KAAK+vB,IAAIzQ,MAAQA,CAGjB,IAAInT,GAAaoF,SAASM,cAAc,MACxC1F,GAAWtE,UAAY,aACvByX,EAAM7N,YAAYtF,GAClBnM,KAAK+vB,IAAI5jB,WAAaA,CAGtB,IAAIsgC,GAAal7B,SAASM,cAAc,MACxC46B,GAAW5kC,UAAY,aACvByX,EAAM7N,YAAYg7B,GAClBzsC,KAAK+vB,IAAI0c,WAAaA,CAGtB,IAAI3L,GAAOvvB,SAASM,cAAc,MAClCivB,GAAKj5B,UAAY,OACjB7H,KAAK+vB,IAAI+Q,KAAOA,CAGhB,IAAI2M,GAAWl8B,SAASM,cAAc,MACtC47B,GAAS5lC,UAAY,WACrB7H,KAAK+vB,IAAI0d,SAAWA,EAGpBztC,KAAKywC,kBAGL,IAAIC,GAAkB,GAAI7tC,GAAgB0tC,EAAY,KAAMvwC,KAC5D0wC,GAAgB/I,OAChB3nC,KAAKm0B,OAAOoc,GAAcG,EAM1B1wC,KAAK8D,OAAS,GAAIC,GAAO/D,KAAK20B,KAAK5E,IAAI8H,iBAGvC73B,KAAK8D,OAAOyP,GAAG,eAAgB,SAAUjK,GACnCA,EAAMqnC,SACR3wC,KAAKq+B,SAAS/0B,IAEhBwrB,KAAK90B,OACPA,KAAK8D,OAAOyP,GAAG,WAAYvT,KAAKg+B,aAAalJ,KAAK90B,OAClDA,KAAK8D,OAAOyP,GAAG,UAAYvT,KAAKi+B,QAAQnJ,KAAK90B,OAC7CA,KAAK8D,OAAOyP,GAAG,SAAYvT,KAAKk+B,WAAWpJ,KAAK90B,OAGhDA,KAAK8D,OAAOyP,GAAG,MAAQvT,KAAK4wC,cAAc9b,KAAK90B,OAG/CA,KAAK8D,OAAOyP,GAAG,QAASvT,KAAK6wC,mBAAmB/b,KAAK90B,OAGrDA,KAAK8D,OAAOyP,GAAG,YAAavT,KAAK8wC,WAAWhc,KAAK90B,OAGjDA,KAAK2nC,QAmEP7kC,EAAQqQ,UAAUD,WAAa,SAASzE,GACtC,GAAIA,EAAS,CAEX,GAAIP,IAAU,OAAQ,QAAS,cAAe,UAAW,QAAS,aAAc,aAAc,iBAAkB,WAAW,OAC3HvN,GAAKoF,gBAAgBmI,EAAQlO,KAAKyO,QAASA,GAEvC,UAAYA,KACgB,gBAAnBA,GAAQiL,QACjB1Z,KAAKyO,QAAQiL,OAAOonB,KAAOryB,EAAQiL,OACnC1Z,KAAKyO,QAAQiL,OAAOrK,KAAKmW,WAAa/W,EAAQiL,OAC9C1Z,KAAKyO,QAAQiL,OAAOrK,KAAKoW,SAAWhX,EAAQiL,QAEX,gBAAnBjL,GAAQiL,SACtB/Y,EAAKoF,iBAAiB,QAAS/F,KAAKyO,QAAQiL,OAAQjL,EAAQiL,QACxD,QAAUjL,GAAQiL,SACe,gBAAxBjL,GAAQiL,OAAOrK,MACxBrP,KAAKyO,QAAQiL,OAAOrK,KAAKmW,WAAa/W,EAAQiL,OAAOrK,KACrDrP,KAAKyO,QAAQiL,OAAOrK,KAAKoW,SAAWhX,EAAQiL,OAAOrK,MAEb,gBAAxBZ,GAAQiL,OAAOrK,MAC7B1O,EAAKoF,iBAAiB,aAAc,YAAa/F,KAAKyO,QAAQiL,OAAOrK,KAAMZ,EAAQiL,OAAOrK,SAM9F,YAAcZ,KACgB,iBAArBA,GAAQygC,UACjBlvC,KAAKyO,QAAQygC,SAASC,WAAc1gC,EAAQygC,SAC5ClvC,KAAKyO,QAAQygC,SAAS1H,YAAc/4B,EAAQygC,SAC5ClvC,KAAKyO,QAAQygC,SAASj8B,IAAcxE,EAAQygC,SAC5ClvC,KAAKyO,QAAQygC,SAAS74B,OAAc5H,EAAQygC,UAET,gBAArBzgC,GAAQygC,UACtBvuC,EAAKoF,iBAAiB,aAAc,cAAe,MAAO,UAAW/F,KAAKyO,QAAQygC,SAAUzgC,EAAQygC,UAKxG,IAAI6B,GAAc,SAAW96B,GAC3B,GAAIiD,GAAKzK,EAAQwH,EACjB,IAAIiD,EAAI,CACN,KAAMA,YAAc83B,WAClB,KAAM,IAAIptC,OAAM,UAAYqS,EAAO,uBAAyBA,EAAO,mBAErEjW,MAAKyO,QAAQwH,GAAQiD,IAEtB4b,KAAK90B,OACP,QAAS,WAAY,WAAY,SAAU,YAAYqI,QAAQ0oC,GAGhE/wC,KAAKixC,cAOTnuC,EAAQqQ,UAAU89B,UAAY,WAC5BjxC,KAAKkwC,YACLlwC,KAAKowC,YAAa,GAMpBttC,EAAQqQ,UAAUG,QAAU,WAC1BtT,KAAK0nC,OACL1nC,KAAKi2B,SAAS,MACdj2B,KAAKg2B,UAAU,MAEfh2B,KAAK8D,OAAS,KAEd9D,KAAK20B,KAAO,KACZ30B,KAAKo6B,WAAa,MAMpBt3B,EAAQqQ,UAAUu0B,KAAO,WAEnB1nC,KAAK+vB,IAAIzQ,MAAMzV,YACjB7J,KAAK+vB,IAAIzQ,MAAMzV,WAAWsH,YAAYnR,KAAK+vB,IAAIzQ,OAI7Ctf,KAAK+vB,IAAI+Q,KAAKj3B,YAChB7J,KAAK+vB,IAAI+Q,KAAKj3B,WAAWsH,YAAYnR,KAAK+vB,IAAI+Q,MAI5C9gC,KAAK+vB,IAAI0d,SAAS5jC,YACpB7J,KAAK+vB,IAAI0d,SAAS5jC,WAAWsH,YAAYnR,KAAK+vB,IAAI0d,WAQtD3qC,EAAQqQ,UAAUw0B,KAAO,WAElB3nC,KAAK+vB,IAAIzQ,MAAMzV,YAClB7J,KAAK20B,KAAK5E,IAAI7D,OAAOza,YAAYzR,KAAK+vB,IAAIzQ,OAIvCtf,KAAK+vB,IAAI+Q,KAAKj3B,YACjB7J,KAAK20B,KAAK5E,IAAIgV,mBAAmBtzB,YAAYzR,KAAK+vB,IAAI+Q,MAInD9gC,KAAK+vB,IAAI0d,SAAS5jC,YACrB7J,KAAK20B,KAAK5E,IAAItoB,KAAKgK,YAAYzR,KAAK+vB,IAAI0d,WAW5C3qC,EAAQqQ,UAAUujB,aAAe,SAASvhB,GACxC,GAAI3P,GAAG8nC,EAAIjtC,EAAIgP,CAMf,KAJW7I,QAAP2O,IAAkBA,MACjBlP,MAAMC,QAAQiP,KAAMA,GAAOA,IAG3B3P,EAAI,EAAG8nC,EAAKttC,KAAKmwC,UAAUxqC,OAAY2nC,EAAJ9nC,EAAQA,IAC9CnF,EAAKL,KAAKmwC,UAAU3qC,GACpB6J,EAAOrP,KAAKiC,MAAM5B,GACdgP,GAAMA,EAAK6hC,UAKjB,KADAlxC,KAAKmwC,aACA3qC,EAAI,EAAG8nC,EAAKn4B,EAAIxP,OAAY2nC,EAAJ9nC,EAAQA,IACnCnF,EAAK8U,EAAI3P,GACT6J,EAAOrP,KAAKiC,MAAM5B,GACdgP,IACFrP,KAAKmwC,UAAUnoC,KAAK3H,GACpBgP,EAAK8hC,WASXruC,EAAQqQ,UAAUyjB,aAAe,WAC/B,MAAO52B,MAAKmwC,UAAUn8B,YAOxBlR,EAAQqQ,UAAUi+B,gBAAkB,WAClC,GAAI3b,GAAQz1B,KAAK20B,KAAKc,MAAM2J,WACxB33B,EAAQzH,KAAK20B,KAAKh0B,KAAKs0B,SAASQ,EAAM7lB,OACtCyX,EAAQrnB,KAAK20B,KAAKh0B,KAAKs0B,SAASQ,EAAM5lB,KAEtCsF,IACJ,KAAK,GAAImiB,KAAWt3B,MAAKm0B,OACvB,GAAIn0B,KAAKm0B,OAAOruB,eAAewxB,GAM7B,IAAK,GALDrlB,GAAQjS,KAAKm0B,OAAOmD,GACpB+Z,EAAkBp/B,EAAMm6B,aAInB5mC,EAAI,EAAGA,EAAI6rC,EAAgB1rC,OAAQH,IAAK,CAC/C,GAAI6J,GAAOgiC,EAAgB7rC,EAEtB6J,GAAK5H,KAAO4f,GAAWhY,EAAK5H,KAAO4H,EAAKkD,MAAQ9K,GACnD0N,EAAInN,KAAKqH,EAAKhP,IAMtB,MAAO8U,IAQTrS,EAAQqQ,UAAUm+B,UAAY,SAASjxC,GAErC,IAAK,GADD8vC,GAAYnwC,KAAKmwC,UACZ3qC,EAAI,EAAG8nC,EAAK6C,EAAUxqC,OAAY2nC,EAAJ9nC,EAAQA,IAC7C,GAAI2qC,EAAU3qC,IAAMnF,EAAI,CACtB8vC,EAAU/nC,OAAO5C,EAAG,EACpB,SASN1C,EAAQqQ,UAAUsO,OAAS,WACzB,GAAI/H,GAAS1Z,KAAKyO,QAAQiL,OACtB+b,EAAQz1B,KAAK20B,KAAKc,MAClBtrB,EAASxJ,EAAKmJ,OAAOK,OACrBsE,EAAUzO,KAAKyO,QACf8lB,EAAc9lB,EAAQ8lB,YACtBkQ,GAAU,EACVnlB,EAAQtf,KAAK+vB,IAAIzQ,MACjB4vB,EAAWzgC,EAAQygC,SAASC,YAAc1gC,EAAQygC,SAAS1H,WAG/DxnC,MAAKgG,MAAM2B,IAAM3H,KAAK20B,KAAKC,SAASjtB,IAAI6K,OAASxS,KAAK20B,KAAKC,SAASxoB,OAAOzE,IAC3E3H,KAAKgG,MAAMyB,KAAOzH,KAAK20B,KAAKC,SAASntB,KAAK8K,MAAQvS,KAAK20B,KAAKC,SAASxoB,OAAO3E,KAG5E6X,EAAMzX,UAAY,WAAaqnC,EAAW,YAAc,IAGxDzK,EAAUzkC,KAAKuxC,gBAAkB9M,CAIjC,IAAI+M,GAAkB/b,EAAM5lB,IAAM4lB,EAAM7lB,MACpC6hC,EAAUD,GAAmBxxC,KAAK0xC,qBAAyB1xC,KAAKgG,MAAMuM,OAASvS,KAAKgG,MAAM2rC,SAC1FF,KAAQzxC,KAAKowC,YAAa,GAC9BpwC,KAAK0xC,oBAAsBF,EAC3BxxC,KAAKgG,MAAM2rC,UAAY3xC,KAAKgG,MAAMuM,KAElC,IAAIs6B,GAAU7sC,KAAKowC,WACfwB,EAAa5xC,KAAK6xC,cAClBC,GACFziC,KAAMqK,EAAOrK,KACbyxB,KAAMpnB,EAAOonB,MAEXiR,GACF1iC,KAAMqK,EAAOrK,KACbyxB,KAAMpnB,EAAOrK,KAAKoW,SAAW,GAE3BjT,EAAS,EACTiiB,EAAY/a,EAAOonB,KAAOpnB,EAAOrK,KAAKoW,QA+B1C,OA5BAzlB,MAAKm0B,OAAOoc,GAAY9uB,OAAOgU,EAAOsc,EAAgBlF,GAGtDlsC,EAAK0H,QAAQrI,KAAKm0B,OAAQ,SAAUliB,GAClC,GAAI+/B,GAAe//B,GAAS2/B,EAAcE,EAAcC,EACpDE,EAAehgC,EAAMwP,OAAOgU,EAAOuc,EAAanF,EACpDpI,GAAUwN,GAAgBxN,EAC1BjyB,GAAUP,EAAMO,SAElBA,EAAStN,KAAKwH,IAAI8F,EAAQiiB,GAC1Bz0B,KAAKowC,YAAa,EAGlB9wB,EAAMrS,MAAMuF,OAAUrI,EAAOqI,GAG7BxS,KAAKgG,MAAMuM,MAAQ+M,EAAM8Q,YACzBpwB,KAAKgG,MAAMwM,OAASA,EAGpBxS,KAAK+vB,IAAI+Q,KAAK7zB,MAAMtF,IAAMwC,EAAuB,OAAfoqB,EAC7Bv0B,KAAK20B,KAAKC,SAASjtB,IAAI6K,OAASxS,KAAK20B,KAAKC,SAASxoB,OAAOzE,IAC1D3H,KAAK20B,KAAKC,SAASjtB,IAAI6K,OAASxS,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,QACxExS,KAAK+vB,IAAI+Q,KAAK7zB,MAAMxF,KAAO,IAG3Bg9B,EAAUzkC,KAAKwkC,cAAgBC,GAUjC3hC,EAAQqQ,UAAU0+B,YAAc,WAC9B,GAAIK,GAA+C,OAA5BlyC,KAAKyO,QAAQ8lB,YAAwB,EAAKv0B,KAAKkwC,SAASvqC,OAAS,EACpFwsC,EAAenyC,KAAKkwC,SAASgC,GAC7BN,EAAa5xC,KAAKm0B,OAAOge,IAAiBnyC,KAAKm0B,OAAOmc,EAE1D,OAAOsB,IAAc,MAQvB9uC,EAAQqQ,UAAUs9B,iBAAmB,WACnC,CAAA,GAEIphC,GAAMkG,EAFN68B,EAAYpyC,KAAKm0B,OAAOmc,EACXtwC,MAAKm0B,OAAOoc,GAG7B,GAAIvwC,KAAK+1B,YAEP,GAAIqc,EAAW,CACbA,EAAU1K,aACH1nC,MAAKm0B,OAAOmc,EAEnB,KAAK/6B,IAAUvV,MAAKiC,MAClB,GAAIjC,KAAKiC,MAAM6D,eAAeyP,GAAS,CACrClG,EAAOrP,KAAKiC,MAAMsT,GAClBlG,EAAKy1B,QAAUz1B,EAAKy1B,OAAOzuB,OAAOhH,EAClC,IAAIioB,GAAUt3B,KAAKqyC,YAAYhjC,EAAKqD,MAChCT,EAAQjS,KAAKm0B,OAAOmD,EACxBrlB,IAASA,EAAMgB,IAAI5D,IAASA,EAAKq4B,aAOvC,KAAK0K,EAAW,CACd,GAAI/xC,GAAK,KACLqS,EAAO,IACX0/B,GAAY,GAAIxvC,GAAMvC,EAAIqS,EAAM1S,MAChCA,KAAKm0B,OAAOmc,GAAa8B,CAEzB,KAAK78B,IAAUvV,MAAKiC,MACdjC,KAAKiC,MAAM6D,eAAeyP,KAC5BlG,EAAOrP,KAAKiC,MAAMsT,GAClB68B,EAAUn/B,IAAI5D,GAIlB+iC,GAAUzK,SAShB7kC,EAAQqQ,UAAUm/B,YAAc,WAC9B,MAAOtyC,MAAK+vB,IAAI0d,UAOlB3qC,EAAQqQ,UAAU8iB,SAAW,SAASh0B,GACpC,GACIkT,GADAhB,EAAKnU,KAELuyC,EAAevyC,KAAK81B,SAGxB,IAAK7zB,EAGA,CAAA,KAAIA,YAAiBpB,IAAWoB,YAAiBnB,IAIpD,KAAM,IAAIuF,WAAU,kDAHpBrG,MAAK81B,UAAY7zB,MAHjBjC,MAAK81B,UAAY,IAoBnB,IAXIyc,IAEF5xC,EAAK0H,QAAQrI,KAAK0vC,cAAe,SAAUpnC,EAAUgB,GACnDipC,EAAa7+B,IAAIpK,EAAOhB,KAI1B6M,EAAMo9B,EAAa18B,SACnB7V,KAAK6vC,UAAU16B,IAGbnV,KAAK81B,UAAW,CAElB,GAAIz1B,GAAKL,KAAKK,EACdM,GAAK0H,QAAQrI,KAAK0vC,cAAe,SAAUpnC,EAAUgB,GACnD6K,EAAG2hB,UAAUviB,GAAGjK,EAAOhB,EAAUjI,KAInC8U,EAAMnV,KAAK81B,UAAUjgB,SACrB7V,KAAK2vC,OAAOx6B,GAGZnV,KAAKywC,qBAQT3tC,EAAQqQ,UAAUq/B,SAAW,WAC3B,MAAOxyC,MAAK81B,WAOdhzB,EAAQqQ,UAAU6iB,UAAY,SAAS7B,GACrC,GACIhf,GADAhB,EAAKnU,IAgBT,IAZIA,KAAK+1B,aACPp1B,EAAK0H,QAAQrI,KAAK8vC,eAAgB,SAAUxnC,EAAUgB,GACpD6K,EAAG4hB,WAAWniB,YAAYtK,EAAOhB,KAInC6M,EAAMnV,KAAK+1B,WAAWlgB,SACtB7V,KAAK+1B,WAAa,KAClB/1B,KAAKiwC,gBAAgB96B,IAIlBgf,EAGA,CAAA,KAAIA,YAAkBtzB,IAAWszB,YAAkBrzB,IAItD,KAAM,IAAIuF,WAAU,kDAHpBrG,MAAK+1B,WAAa5B,MAHlBn0B,MAAK+1B,WAAa,IASpB,IAAI/1B,KAAK+1B,WAAY,CAEnB,GAAI11B,GAAKL,KAAKK,EACdM,GAAK0H,QAAQrI,KAAK8vC,eAAgB,SAAUxnC,EAAUgB,GACpD6K,EAAG4hB,WAAWxiB,GAAGjK,EAAOhB,EAAUjI,KAIpC8U,EAAMnV,KAAK+1B,WAAWlgB,SACtB7V,KAAK+vC,aAAa56B,GAIpBnV,KAAKywC,mBAGLzwC,KAAKyyC,SAELzyC,KAAK20B,KAAKE,QAAQjH,KAAK,UAAWxa,OAAO,KAO3CtQ,EAAQqQ,UAAUu/B,UAAY,WAC5B,MAAO1yC,MAAK+1B,YAOdjzB,EAAQqQ,UAAU66B,WAAa,SAAS3tC,GACtC,GAAIgP,GAAOrP,KAAK81B,UAAU5gB,IAAI7U,GAC1B22B,EAAUh3B,KAAK81B,UAAUhgB,YAEzBzG,IAEFrP,KAAKyO,QAAQ8gC,SAASlgC,EAAM,SAAUA,GAChCA,GAGF2nB,EAAQ3gB,OAAOhW,MAYvByC,EAAQqQ,UAAUw/B,SAAW,SAAU9b,GACrC,MAAOA,GAAS/vB,MAAQ9G,KAAKyO,QAAQ3H,OAAS+vB,EAAShnB,IAAM,QAAU,QAUzE/M,EAAQqQ,UAAUk/B,YAAc,SAAUxb,GACxC,GAAI/vB,GAAO9G,KAAK2yC,SAAS9b,EACzB,OAAY,cAAR/vB,GAA0CN,QAAlBqwB,EAAS5kB,MAC7Bs+B,EAGCvwC,KAAK+1B,WAAac,EAAS5kB,MAAQq+B,GAS9CxtC,EAAQqQ,UAAUy8B,UAAY,SAASz6B,GACrC,GAAIhB,GAAKnU,IAETmV,GAAI9M,QAAQ,SAAUhI,GACpB,GAAIw2B,GAAW1iB,EAAG2hB,UAAU5gB,IAAI7U,EAAI8T,EAAGs7B,aACnCpgC,EAAO8E,EAAGlS,MAAM5B,GAChByG,EAAOqN,EAAGw+B,SAAS9b,GAEnBvwB,EAAcxD,EAAQoU,MAAMpQ,EAchC,IAZIuI,IAEG/I,GAAiB+I,YAAgB/I,GAMpC6N,EAAGc,YAAY5F,EAAMwnB,IAJrB1iB,EAAGy+B,YAAYvjC,GACfA,EAAO,QAONA,EAAM,CAET,IAAI/I,EAKC,KAEG,IAAID,WAFK,iBAARS,EAEa,4HAIA,sBAAwBA,EAAO,IAVnDuI,GAAO,GAAI/I,GAAYuwB,EAAU1iB,EAAGimB,WAAYjmB,EAAG1F,SACnDY,EAAKhP,GAAKA,EACV8T,EAAGC,SAAS/E,MAalBrP,KAAKyyC,SACLzyC,KAAKowC,YAAa,EAClBpwC,KAAK20B,KAAKE,QAAQjH,KAAK,UAAWxa,OAAO,KAQ3CtQ,EAAQqQ,UAAUw8B,OAAS7sC,EAAQqQ,UAAUy8B,UAO7C9sC,EAAQqQ,UAAU08B,UAAY,SAAS16B,GACrC,GAAI6B,GAAQ,EACR7C,EAAKnU,IACTmV,GAAI9M,QAAQ,SAAUhI,GACpB,GAAIgP,GAAO8E,EAAGlS,MAAM5B,EAChBgP,KACF2H,IACA7C,EAAGy+B,YAAYvjC,MAIf2H,IAEFhX,KAAKyyC,SACLzyC,KAAKowC,YAAa,EAClBpwC,KAAK20B,KAAKE,QAAQjH,KAAK,UAAWxa,OAAO,MAQ7CtQ,EAAQqQ,UAAUs/B,OAAS,WAGzB9xC,EAAK0H,QAAQrI,KAAKm0B,OAAQ,SAAUliB,GAClCA,EAAMwD,WASV3S,EAAQqQ,UAAU68B,gBAAkB,SAAS76B,GAC3CnV,KAAK+vC,aAAa56B,IAQpBrS,EAAQqQ,UAAU48B,aAAe,SAAS56B,GACxC,GAAIhB,GAAKnU,IAETmV,GAAI9M,QAAQ,SAAUhI,GACpB,GAAIyrC,GAAY33B,EAAG4hB,WAAW7gB,IAAI7U,GAC9B4R,EAAQkC,EAAGggB,OAAO9zB,EAEtB,IAAK4R,EA6BHA,EAAM+F,QAAQ8zB,OA7BJ,CAEV,GAAIzrC,GAAMiwC,GAAajwC,GAAMkwC,EAC3B,KAAM,IAAI3sC,OAAM,qBAAuBvD,EAAK,qBAG9C,IAAIwyC,GAAetsC,OAAO8H,OAAO8F,EAAG1F,QACpC9N,GAAK2E,OAAOutC,GACVrgC,OAAQ,OAGVP,EAAQ,GAAIrP,GAAMvC,EAAIyrC,EAAW33B,GACjCA,EAAGggB,OAAO9zB,GAAM4R,CAGhB,KAAK,GAAIsD,KAAUpB,GAAGlS,MACpB,GAAIkS,EAAGlS,MAAM6D,eAAeyP,GAAS,CACnC,GAAIlG,GAAO8E,EAAGlS,MAAMsT,EAChBlG,GAAKqD,KAAKT,OAAS5R,GACrB4R,EAAMgB,IAAI5D,GAKhB4C,EAAMwD,QACNxD,EAAM01B,UAQV3nC,KAAK20B,KAAKE,QAAQjH,KAAK,UAAWxa,OAAO,KAQ3CtQ,EAAQqQ,UAAU88B,gBAAkB,SAAS96B,GAC3C,GAAIgf,GAASn0B,KAAKm0B,MAClBhf,GAAI9M,QAAQ,SAAUhI,GACpB,GAAI4R,GAAQkiB,EAAO9zB,EAEf4R,KACFA,EAAMy1B,aACCvT,GAAO9zB,MAIlBL,KAAKixC,YAELjxC,KAAK20B,KAAKE,QAAQjH,KAAK,UAAWxa,OAAO,KAQ3CtQ,EAAQqQ,UAAUo+B,aAAe,WAC/B,GAAIvxC,KAAK+1B,WAAY,CAEnB,GAAIma,GAAWlwC,KAAK+1B,WAAWlgB,QAC7BJ,MAAOzV,KAAKyO,QAAQugC,aAGlB/P,GAAWt+B,EAAKiG,WAAWspC,EAAUlwC,KAAKkwC,SAC9C,IAAIjR,EAAS,CAEX,GAAI9K,GAASn0B,KAAKm0B,MAClB+b,GAAS7nC,QAAQ,SAAUivB,GACzBnD,EAAOmD,GAASoQ,SAIlBwI,EAAS7nC,QAAQ,SAAUivB,GACzBnD,EAAOmD,GAASqQ,SAGlB3nC,KAAKkwC,SAAWA,EAGlB,MAAOjR,GAGP,OAAO,GASXn8B,EAAQqQ,UAAUiB,SAAW,SAAS/E,GACpCrP,KAAKiC,MAAMoN,EAAKhP,IAAMgP,CAGtB,IAAIioB,GAAUt3B,KAAKqyC,YAAYhjC,EAAKqD,MAChCT,EAAQjS,KAAKm0B,OAAOmD,EACpBrlB,IAAOA,EAAMgB,IAAI5D,IASvBvM,EAAQqQ,UAAU8B,YAAc,SAAS5F,EAAMwnB,GAC7C,GAAIic,GAAazjC,EAAKqD,KAAKT,KAM3B,IAHA5C,EAAK2I,QAAQ6e,GAGTic,GAAczjC,EAAKqD,KAAKT,MAAO,CACjC,GAAI8gC,GAAW/yC,KAAKm0B,OAAO2e,EACvBC,IAAUA,EAAS18B,OAAOhH,EAE9B,IAAIioB,GAAUt3B,KAAKqyC,YAAYhjC,EAAKqD,MAChCT,EAAQjS,KAAKm0B,OAAOmD,EACpBrlB,IAAOA,EAAMgB,IAAI5D,KAUzBvM,EAAQqQ,UAAUy/B,YAAc,SAASvjC,GAEvCA,EAAKq4B,aAGE1nC,MAAKiC,MAAMoN,EAAKhP,GAGvB,IAAI8H,GAAQnI,KAAKmwC,UAAUxpC,QAAQ0I,EAAKhP,GAC3B,KAAT8H,GAAanI,KAAKmwC,UAAU/nC,OAAOD,EAAO,GAG9CkH,EAAKy1B,QAAUz1B,EAAKy1B,OAAOzuB,OAAOhH,IASpCvM,EAAQqQ,UAAU6/B,qBAAuB,SAASxqC,GAGhD,IAAK,GAFD0lC,MAEK1oC,EAAI,EAAGA,EAAIgD,EAAM7C,OAAQH,IAC5BgD,EAAMhD,YAAclD,IACtB4rC,EAASlmC,KAAKQ,EAAMhD,GAGxB,OAAO0oC,IAaTprC,EAAQqQ,UAAUkrB,SAAW,SAAU/0B,GAErCtJ,KAAKqwC,YAAYhhC,KAAOvM,EAAQmwC,eAAe3pC,GAC/CtJ,KAAKqwC,YAAY6C,aAAe5pC,EAAMI,OAAOwpC,eAAgB,EAC7DlzC,KAAKqwC,YAAY8C,cAAgB7pC,EAAMI,OAAOypC,gBAAiB,EAC/DnzC,KAAKqwC,YAAY+C,UAAY,MAQ/BtwC,EAAQqQ,UAAU6qB,aAAe,SAAU10B,GACzC,GAAKtJ,KAAKyO,QAAQygC,SAASC,YAAenvC,KAAKyO,QAAQygC,SAAS1H,YAAhE,CAIA,GAEIxhC,GAFAqJ,EAAOrP,KAAKqwC,YAAYhhC,MAAQ,KAChC8E,EAAKnU,IAGT,IAAIqP,GAAQA,EAAKgkC,SAAU,CACzB,GAAIH,GAAelzC,KAAKqwC,YAAY6C,aAChCC,EAAgBnzC,KAAKqwC,YAAY8C,aAEjCD,IACFltC,GACEqJ,KAAM6jC,EACNI,SAAUhqC,EAAM4iB,OAAOna,GAGrBoC,EAAG1F,QAAQygC,SAASC,aACtBnpC,EAAM4J,MAAQP,EAAKqD,KAAK9C,MAAM5I,WAE5BmN,EAAG1F,QAAQygC,SAAS1H,aAClB,SAAWn4B,GAAKqD,OAAM1M,EAAMiM,MAAQ5C,EAAKqD,KAAKT,OAGpDjS,KAAKqwC,YAAY+C,WAAaptC,IAEvBmtC,GACPntC,GACEqJ,KAAM8jC,EACNG,SAAUhqC,EAAM4iB,OAAOna,GAGrBoC,EAAG1F,QAAQygC,SAASC,aACtBnpC,EAAM6J,IAAMR,EAAKqD,KAAK7C,IAAI7I,WAExBmN,EAAG1F,QAAQygC,SAAS1H,aAClB,SAAWn4B,GAAKqD,OAAM1M,EAAMiM,MAAQ5C,EAAKqD,KAAKT,OAGpDjS,KAAKqwC,YAAY+C,WAAaptC,IAG9BhG,KAAKqwC,YAAY+C,UAAYpzC,KAAK42B,eAAevpB,IAAI,SAAUhN,GAC7D,GAAIgP,GAAO8E,EAAGlS,MAAM5B,GAChB2F,GACFqJ,KAAMA,EACNikC,SAAUhqC,EAAM4iB,OAAOna,EAWzB,OARIoC,GAAG1F,QAAQygC,SAASC,aAClB,SAAW9/B,GAAKqD,OAAM1M,EAAM4J,MAAQP,EAAKqD,KAAK9C,MAAM5I,WACpD,OAASqI,GAAKqD,OAAQ1M,EAAM6J,IAAMR,EAAKqD,KAAK7C,IAAI7I,YAElDmN,EAAG1F,QAAQygC,SAAS1H,aAClB,SAAWn4B,GAAKqD,OAAM1M,EAAMiM,MAAQ5C,EAAKqD,KAAKT,OAG7CjM,IAIXsD,EAAMq8B,kBACNr8B,EAAMD,oBASVvG,EAAQqQ,UAAU8qB,QAAU,SAAU30B,GAGpC,GAFAA,EAAMD,iBAEFrJ,KAAKqwC,YAAY+C,UAAW,CAC9B,GAAIj/B,GAAKnU,KACLg1B,EAAOh1B,KAAK20B,KAAKh0B,KAAKq0B,MAAQ,KAC9BpL,EAAU5pB,KAAK20B,KAAK5E,IAAIrwB,KAAK2tC,WAAartC,KAAK20B,KAAKC,SAASntB,KAAK8K,KAGtEvS,MAAKqwC,YAAY+C,UAAU/qC,QAAQ,SAAUrC,GAC3C,GAAIutC,MACAzZ,EAAU3lB,EAAGwgB,KAAKh0B,KAAK00B,OAAO/rB,EAAM4iB,OAAOna,EAAI6X,GAC/C4pB,EAAUr/B,EAAGwgB,KAAKh0B,KAAK00B,OAAOrvB,EAAMstC,SAAW1pB,GAC/CD,EAASmQ,EAAU0Z,CAEvB,IAAI,SAAWxtC,GAAO,CACpB,GAAI4J,GAAQ,GAAItL,MAAK0B,EAAM4J,MAAQ+Z,EACnC4pB,GAAS3jC,MAAQolB,EAAOA,EAAKplB,GAASA,EAGxC,GAAI,OAAS5J,GAAO,CAClB,GAAI6J,GAAM,GAAIvL,MAAK0B,EAAM6J,IAAM8Z,EAC/B4pB,GAAS1jC,IAAMmlB,EAAOA,EAAKnlB,GAAOA,EAGpC,GAAI,SAAW7J,GAAO,CAEpB,GAAIiM,GAAQnP,EAAQ2wC,gBAAgBnqC,EACpCiqC,GAASthC,MAAQA,GAASA,EAAMqlB,QAIlC,GAAIT,GAAWl2B,EAAK2E,UAAWU,EAAMqJ,KAAKqD,KAAM6gC,EAChDp/B,GAAG1F,QAAQ+gC,SAAS3Y,EAAU,SAAUA,GAClCA,GACF1iB,EAAGu/B,iBAAiB1tC,EAAMqJ,KAAMwnB,OAKtC72B,KAAKowC,YAAa,EAClBpwC,KAAK20B,KAAKE,QAAQjH,KAAK,UAEvBtkB,EAAMq8B,oBAUV7iC,EAAQqQ,UAAUugC,iBAAmB,SAASrkC,EAAMrJ,GAE9C,SAAWA,KAAOqJ,EAAKqD,KAAK9C,MAAQ5J,EAAM4J,OAC1C,OAAS5J,KAASqJ,EAAKqD,KAAK7C,IAAQ7J,EAAM6J,KAC1C,SAAW7J,IAASqJ,EAAKqD,KAAKT,OAASjM,EAAMiM,OAC/CjS,KAAK2zC,aAAatkC,EAAMrJ,EAAMiM,QAUlCnP,EAAQqQ,UAAUwgC,aAAe,SAAStkC,EAAMioB,GAC9C,GAAIrlB,GAAQjS,KAAKm0B,OAAOmD,EACxB,IAAIrlB,GAASA,EAAMqlB,SAAWjoB,EAAKqD,KAAKT,MAAO,CAC7C,GAAI8gC,GAAW1jC,EAAKy1B,MACpBiO,GAAS18B,OAAOhH,GAChB0jC,EAASt9B,QACTxD,EAAMgB,IAAI5D,GACV4C,EAAMwD,QAENpG,EAAKqD,KAAKT,MAAQA,EAAMqlB,UAS5Bx0B,EAAQqQ,UAAU+qB,WAAa,SAAU50B,GACvC,GAAItJ,KAAKqwC,YAAY+C,UAAW,CAE9B,GAAIQ,MACAz/B,EAAKnU,KACLg3B,EAAUh3B,KAAK81B,UAAUhgB,aAEzBs9B,EAAYpzC,KAAKqwC,YAAY+C,SACjCpzC,MAAKqwC,YAAY+C,UAAY,KAC7BA,EAAU/qC,QAAQ,SAAUrC,GAC1B,GAAI3F,GAAK2F,EAAMqJ,KAAKhP,GAChBw2B,EAAW1iB,EAAG2hB,UAAU5gB,IAAI7U,EAAI8T,EAAGs7B,aAEnCxQ,GAAU,CACV,UAAWj5B,GAAMqJ,KAAKqD,OACxBusB,EAAWj5B,EAAM4J,OAAS5J,EAAMqJ,KAAKqD,KAAK9C,MAAM5I,UAChD6vB,EAASjnB,MAAQjP,EAAKkG,QAAQb,EAAMqJ,KAAKqD,KAAK9C,MACtConB,EAAQrkB,SAAS7L,MAAQkwB,EAAQrkB,SAAS7L,KAAK8I,OAAS,SAE9D,OAAS5J,GAAMqJ,KAAKqD,OACtBusB,EAAUA,GAAaj5B,EAAM6J,KAAO7J,EAAMqJ,KAAKqD,KAAK7C,IAAI7I,UACxD6vB,EAAShnB,IAAMlP,EAAKkG,QAAQb,EAAMqJ,KAAKqD,KAAK7C,IACpCmnB,EAAQrkB,SAAS7L,MAAQkwB,EAAQrkB,SAAS7L,KAAK+I,KAAO,SAE5D,SAAW7J,GAAMqJ,KAAKqD,OACxBusB,EAAUA,GAAaj5B,EAAMiM,OAASjM,EAAMqJ,KAAKqD,KAAKT,MACtD4kB,EAAS5kB,MAAQjM,EAAMqJ,KAAKqD,KAAKT,OAI/BgtB,GACF9qB,EAAG1F,QAAQ6gC,OAAOzY,EAAU,SAAUA,GAChCA,GAEFA,EAASG,EAAQnkB,UAAYxS,EAC7BuzC,EAAQ5rC,KAAK6uB,KAIb1iB,EAAGu/B,iBAAiB1tC,EAAMqJ,KAAMrJ,GAEhCmO,EAAGi8B,YAAa,EAChBj8B,EAAGwgB,KAAKE,QAAQjH,KAAK,eAOzBgmB,EAAQjuC,QACVqxB,EAAQniB,OAAO++B,GAGjBtqC,EAAMq8B,oBASV7iC,EAAQqQ,UAAUy9B,cAAgB,SAAUtnC,GAC1C,GAAKtJ,KAAKyO,QAAQwgC,WAAlB,CAEA,GAAI4E,GAAWvqC,EAAMwqC,UAAYxqC,EAAMwqC,SAASD,QAC5CE,EAAWzqC,EAAMwqC,UAAYxqC,EAAMwqC,SAASC,QAChD,IAAIF,GAAWE,EAEb,WADA/zC,MAAK6wC,mBAAmBvnC,EAI1B,IAAI0qC,GAAeh0C,KAAK42B,eAEpBvnB,EAAOvM,EAAQmwC,eAAe3pC,GAC9B6mC,EAAY9gC,GAAQA,EAAKhP,MAC7BL,MAAK02B,aAAayZ,EAElB,IAAI8D,GAAej0C,KAAK42B,gBAIpBqd,EAAatuC,OAAS,GAAKquC,EAAaruC,OAAS,IACnD3F,KAAK20B,KAAKE,QAAQjH,KAAK,UACrB3rB,MAAOgyC,MAUbnxC,EAAQqQ,UAAU29B,WAAa,SAAUxnC,GACvC,GAAKtJ,KAAKyO,QAAQwgC,YACbjvC,KAAKyO,QAAQygC,SAASj8B,IAA3B,CAEA,GAAIkB,GAAKnU,KACLg1B,EAAOh1B,KAAK20B,KAAKh0B,KAAKq0B,MAAQ,KAC9B3lB,EAAOvM,EAAQmwC,eAAe3pC,EAElC,IAAI+F,EAAM,CAIR,GAAIwnB,GAAW1iB,EAAG2hB,UAAU5gB,IAAI7F,EAAKhP,GACrCL,MAAKyO,QAAQ4gC,SAASxY,EAAU,SAAUA,GACpCA,GACF1iB,EAAG2hB,UAAUhgB,aAAajB,OAAOgiB,SAIlC,CAEH,GAAIqd,GAAOvzC,EAAK2G,gBAAgBtH,KAAK+vB,IAAIzQ,OACrCvN,EAAIzI,EAAM4iB,OAAOna,EAAImiC,EACrBtkC,EAAQ5P,KAAK20B,KAAKh0B,KAAK00B,OAAOtjB,GAC9BoiC,GACFvkC,MAAOolB,EAAOA,EAAKplB,GAASA,EAC5BggB,QAAS,WAIX,IAA0B,UAAtB5vB,KAAKyO,QAAQ3H,KAAkB,CACjC,GAAI+I,GAAM7P,KAAK20B,KAAKh0B,KAAK00B,OAAOtjB,EAAI/R,KAAKgG,MAAMuM,MAAQ,EACvD4hC,GAAQtkC,IAAMmlB,EAAOA,EAAKnlB,GAAOA,EAGnCskC,EAAQn0C,KAAK81B,UAAUjjB,UAAYlS,EAAKqE,YAExC,IAAIiN,GAAQnP,EAAQ2wC,gBAAgBnqC,EAChC2I,KACFkiC,EAAQliC,MAAQA,EAAMqlB,SAIxBt3B,KAAKyO,QAAQ2gC,MAAM+E,EAAS,SAAU9kC,GAChCA,GACF8E,EAAG2hB,UAAUhgB,aAAa7C,IAAI5D,QAYtCvM,EAAQqQ,UAAU09B,mBAAqB,SAAUvnC,GAC/C,GAAKtJ,KAAKyO,QAAQwgC,WAAlB,CAEA,GAAIkB,GACA9gC,EAAOvM,EAAQmwC,eAAe3pC,EAElC,IAAI+F,EAAM,CAER8gC,EAAYnwC,KAAK42B,cAEjB,IAAImd,GAAWzqC,EAAMwqC,UAAYxqC,EAAMwqC,SAASC,WAAY,CAC5D,IAAIA,EAAU,CAIZ5D,EAAUnoC,KAAKqH,EAAKhP,GACpB,IAAIo1B,GAAQ3yB,EAAQsxC,cAAcp0C,KAAK81B,UAAU5gB,IAAIi7B,EAAWnwC,KAAKyvC,aAGrEU,KACA,KAAK,GAAI9vC,KAAML,MAAKiC,MAClB,GAAIjC,KAAKiC,MAAM6D,eAAezF,GAAK,CACjC,GAAIg0C,GAAQr0C,KAAKiC,MAAM5B,GACnBuP,EAAQykC,EAAM3hC,KAAK9C,MACnBC,EAA0BrJ,SAAnB6tC,EAAM3hC,KAAK7C,IAAqBwkC,EAAM3hC,KAAK7C,IAAMD,CAExDA,IAAS6lB,EAAM3pB,KAAO+D,GAAO4lB,EAAM/oB,KACrCyjC,EAAUnoC,KAAKqsC,EAAMh0C,SAKxB,CAEH,GAAI8H,GAAQgoC,EAAUxpC,QAAQ0I,EAAKhP,GACtB,KAAT8H,EAEFgoC,EAAUnoC,KAAKqH,EAAKhP,IAIpB8vC,EAAU/nC,OAAOD,EAAO,GAI5BnI,KAAK02B,aAAayZ,GAElBnwC,KAAK20B,KAAKE,QAAQjH,KAAK,UACrB3rB,MAAOjC,KAAK42B,oBAWlB9zB,EAAQsxC,cAAgB,SAASte,GAC/B,GAAIppB,GAAM,KACNZ,EAAM,IAmBV,OAjBAgqB,GAAUztB,QAAQ,SAAUqK,IACf,MAAP5G,GAAe4G,EAAK9C,MAAQ9D,KAC9BA,EAAM4G,EAAK9C,OAGGpJ,QAAZkM,EAAK7C,KACI,MAAPnD,GAAegG,EAAK7C,IAAMnD,KAC5BA,EAAMgG,EAAK7C,MAIF,MAAPnD,GAAegG,EAAK9C,MAAQlD,KAC9BA,EAAMgG,EAAK9C,UAMf9D,IAAKA,EACLY,IAAKA,IAUT5J,EAAQmwC,eAAiB,SAAS3pC,GAEhC,IADA,GAAII,GAASJ,EAAMI,OACZA,GAAQ,CACb,GAAIA,EAAO5D,eAAe,iBACxB,MAAO4D,GAAO,gBAEhBA,GAASA,EAAOG,WAGlB,MAAO,OAST/G,EAAQ2wC,gBAAkB,SAASnqC,GAEjC,IADA,GAAII,GAASJ,EAAMI,OACZA,GAAQ,CACb,GAAIA,EAAO5D,eAAe,kBACxB,MAAO4D,GAAO,iBAEhBA,GAASA,EAAOG,WAGlB,MAAO,OAST/G,EAAQwxC,kBAAoB,SAAShrC,GAEnC,IADA,GAAII,GAASJ,EAAMI,OACZA,GAAQ,CACb,GAAIA,EAAO5D,eAAe,oBACxB,MAAO4D,GAAO,mBAEhBA,GAASA,EAAOG,WAGlB,MAAO,OAGThK,EAAOD,QAAUkD,GAKb,SAASjD,EAAQD,EAASM,GAS9B,QAAS6C,GAAO4xB,EAAMlmB,EAAS8lC,EAAM1O,GACnC7lC,KAAK20B,KAAOA,EACZ30B,KAAKq0B,gBACH3lB,SAAS,EACTs3B,OAAO,EACPwO,SAAU,GACVC,YAAa,EACbhtC,MACEihB,SAAS,EACT9E,SAAU,YAEZyD,OACEqB,SAAS,EACT9E,SAAU,aAGd5jB,KAAKu0C,KAAOA,EACZv0C,KAAKyO,QAAU9N,EAAK2E,UAAUtF,KAAKq0B,gBACnCr0B,KAAK6lC,iBAAmBA,EAExB7lC,KAAKinC,eACLjnC,KAAK+vB,OACL/vB,KAAKm0B,UACLn0B,KAAKmnC,eAAiB,EACtBnnC,KAAK00B,UAEL10B,KAAKkT,WAAWzE,GAjClB,GAAI9N,GAAOT,EAAoB,GAC3BU,EAAUV,EAAoB,GAC9BqC,EAAYrC,EAAoB,GAkCpC6C,GAAOoQ,UAAY,GAAI5Q,GAEvBQ,EAAOoQ,UAAUsD,MAAQ,WACvBzW,KAAKm0B,UACLn0B,KAAKmnC,eAAiB,GAGxBpkC,EAAOoQ,UAAUm0B,SAAW,SAAS7e,EAAO8e,GAErCvnC,KAAKm0B,OAAOruB,eAAe2iB,KAC9BzoB,KAAKm0B,OAAO1L,GAAS8e,GAEvBvnC,KAAKmnC,gBAAkB,GAGzBpkC,EAAOoQ,UAAUq0B,YAAc,SAAS/e,EAAO8e,GAC7CvnC,KAAKm0B,OAAO1L,GAAS8e,GAGvBxkC,EAAOoQ,UAAUs0B,YAAc,SAAShf,GAClCzoB,KAAKm0B,OAAOruB,eAAe2iB,WACtBzoB,MAAKm0B,OAAO1L,GACnBzoB,KAAKmnC,gBAAkB,IAI3BpkC,EAAOoQ,UAAUuhB,QAAU,WACzB10B,KAAK+vB,IAAIzQ,MAAQ/N,SAASM,cAAc,OACxC7R,KAAK+vB,IAAIzQ,MAAMzX,UAAY,SAC3B7H,KAAK+vB,IAAIzQ,MAAMrS,MAAM2W,SAAW,WAChC5jB,KAAK+vB,IAAIzQ,MAAMrS,MAAMtF,IAAM,OAC3B3H,KAAK+vB,IAAIzQ,MAAMrS,MAAM26B,QAAU,QAE/B5nC,KAAK+vB,IAAI2kB,SAAWnjC,SAASM,cAAc,OAC3C7R,KAAK+vB,IAAI2kB,SAAS7sC,UAAY,aAC9B7H,KAAK+vB,IAAI2kB,SAASznC,MAAM2W,SAAW,WACnC5jB,KAAK+vB,IAAI2kB,SAASznC,MAAMtF,IAAM,MAE9B3H,KAAK4lC,IAAMr0B,SAASC,gBAAgB,6BAA6B,OACjExR,KAAK4lC,IAAI34B,MAAM2W,SAAW,WAC1B5jB,KAAK4lC,IAAI34B,MAAMtF,IAAM,MACrB3H,KAAK4lC,IAAI34B,MAAMsF,MAAQvS,KAAKyO,QAAQ+lC,SAAW,EAAI,KACnDx0C,KAAK4lC,IAAI34B,MAAMuF,OAAS,OAExBxS,KAAK+vB,IAAIzQ,MAAM7N,YAAYzR,KAAK4lC,KAChC5lC,KAAK+vB,IAAIzQ,MAAM7N,YAAYzR,KAAK+vB,IAAI2kB,WAMtC3xC,EAAOoQ,UAAUu0B,KAAO,WAElB1nC,KAAK+vB,IAAIzQ,MAAMzV,YACjB7J,KAAK+vB,IAAIzQ,MAAMzV,WAAWsH,YAAYnR,KAAK+vB,IAAIzQ,QAQnDvc,EAAOoQ,UAAUw0B,KAAO,WAEjB3nC,KAAK+vB,IAAIzQ,MAAMzV,YAClB7J,KAAK20B,KAAK5E,IAAI7D,OAAOza,YAAYzR,KAAK+vB,IAAIzQ,QAI9Cvc,EAAOoQ,UAAUD,WAAa,SAASzE,GACrC,GAAIP,IAAU,UAAU,cAAc,QAAQ,OAAO,QACrDvN,GAAKwF,oBAAoB+H,EAAQlO,KAAKyO,QAASA,IAGjD1L,EAAOoQ,UAAUsO,OAAS,WACxB,GAAI0mB,GAAe,CACnB,KAAK,GAAI7Q,KAAWt3B,MAAKm0B,OACnBn0B,KAAKm0B,OAAOruB,eAAewxB,KACO,GAAhCt3B,KAAKm0B,OAAOmD,GAAS5O,SAAkEliB,SAA9CxG,KAAK6lC,iBAAiBrO,WAAWF,IAAuE,GAA7Ct3B,KAAK6lC,iBAAiBrO,WAAWF,IACvI6Q,IAKN,IAAuC,GAAnCnoC,KAAKyO,QAAQzO,KAAKu0C,MAAM7rB,SAA2C,GAAvB1oB,KAAKmnC,gBAA+C,GAAxBnnC,KAAKyO,QAAQC,SAAoC,GAAhBy5B,EAC3GnoC,KAAK0nC,WAEF,CAqBH,GApBA1nC,KAAK2nC,OACmC,YAApC3nC,KAAKyO,QAAQzO,KAAKu0C,MAAM3wB,UAA8D,eAApC5jB,KAAKyO,QAAQzO,KAAKu0C,MAAM3wB,UAC5E5jB,KAAK+vB,IAAIzQ,MAAMrS,MAAMxF,KAAO,MAC5BzH,KAAK+vB,IAAIzQ,MAAMrS,MAAMqb,UAAY,OACjCtoB,KAAK+vB,IAAI2kB,SAASznC,MAAMqb,UAAY,OACpCtoB,KAAK+vB,IAAI2kB,SAASznC,MAAMxF,KAAQzH,KAAKyO,QAAQ+lC,SAAW,GAAM,KAC9Dx0C,KAAK+vB,IAAI2kB,SAASznC,MAAMoa,MAAQ,GAChCrnB,KAAK4lC,IAAI34B,MAAMxF,KAAO,MACtBzH,KAAK4lC,IAAI34B,MAAMoa,MAAQ,KAGvBrnB,KAAK+vB,IAAIzQ,MAAMrS,MAAMoa,MAAQ,MAC7BrnB,KAAK+vB,IAAIzQ,MAAMrS,MAAMqb,UAAY,QACjCtoB,KAAK+vB,IAAI2kB,SAASznC,MAAMqb,UAAY,QACpCtoB,KAAK+vB,IAAI2kB,SAASznC,MAAMoa,MAASrnB,KAAKyO,QAAQ+lC,SAAW,GAAM,KAC/Dx0C,KAAK+vB,IAAI2kB,SAASznC,MAAMxF,KAAO,GAC/BzH,KAAK4lC,IAAI34B,MAAMoa,MAAQ,MACvBrnB,KAAK4lC,IAAI34B,MAAMxF,KAAO,IAGgB,YAApCzH,KAAKyO,QAAQzO,KAAKu0C,MAAM3wB,UAA8D,aAApC5jB,KAAKyO,QAAQzO,KAAKu0C,MAAM3wB,SAC5E5jB,KAAK+vB,IAAIzQ,MAAMrS,MAAMtF,IAAM,EAAIzD,OAAOlE,KAAK20B,KAAK5E,IAAI7D,OAAOjf,MAAMtF,IAAI6C,QAAQ,KAAK,KAAO,KACzFxK,KAAK+vB,IAAIzQ,MAAMrS,MAAMqW,OAAS,OAE3B,CACH,GAAIqxB,GAAmB30C,KAAK20B,KAAKC,SAAS1I,OAAO1Z,OAASxS,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,MAC7FxS,MAAK+vB,IAAIzQ,MAAMrS,MAAMqW,OAAS,EAAIqxB,EAAmBzwC,OAAOlE,KAAK20B,KAAK5E,IAAI7D,OAAOjf,MAAMtF,IAAI6C,QAAQ,KAAK,KAAO,KAC/GxK,KAAK+vB,IAAIzQ,MAAMrS,MAAMtF,IAAM,GAGH,GAAtB3H,KAAKyO,QAAQu3B,OACfhmC,KAAK+vB,IAAIzQ,MAAMrS,MAAMsF,MAAQvS,KAAK+vB,IAAI2kB,SAAStkB,YAAc,GAAK,KAClEpwB,KAAK+vB,IAAI2kB,SAASznC,MAAMoa,MAAQ,GAChCrnB,KAAK+vB,IAAI2kB,SAASznC,MAAMxF,KAAO,GAC/BzH,KAAK4lC,IAAI34B,MAAMsF,MAAQ,QAGvBvS,KAAK+vB,IAAIzQ,MAAMrS,MAAMsF,MAAQvS,KAAKyO,QAAQ+lC,SAAW,GAAKx0C,KAAK+vB,IAAI2kB,SAAStkB,YAAc,GAAK,KAC/FpwB,KAAK40C,kBAGP,IAAIhlB,GAAU,EACd,KAAK,GAAI0H,KAAWt3B,MAAKm0B,OACnBn0B,KAAKm0B,OAAOruB,eAAewxB,KACO,GAAhCt3B,KAAKm0B,OAAOmD,GAAS5O,SAAkEliB,SAA9CxG,KAAK6lC,iBAAiBrO,WAAWF,IAAuE,GAA7Ct3B,KAAK6lC,iBAAiBrO,WAAWF,KACvI1H,GAAW5vB,KAAKm0B,OAAOmD,GAAS1H,QAAU,UAIhD5vB,MAAK+vB,IAAI2kB,SAASzwB,UAAY2L,EAC9B5vB,KAAK+vB,IAAI2kB,SAASznC,MAAMsjB,WAAe,IAAOvwB,KAAKyO,QAAQ+lC,SAAYx0C,KAAKyO,QAAQgmC,YAAe,OAIvG1xC,EAAOoQ,UAAUyhC,gBAAkB,WACjC,GAAI50C,KAAK+vB,IAAIzQ,MAAMzV,WAAY,CAC7BjJ,EAAQiQ,gBAAgB7Q,KAAKinC,YAC7B,IAAIjjB,GAAUza,OAAOsrC,iBAAiB70C,KAAK+vB,IAAIzQ,OAAOw1B,WAClD/M,EAAa7jC,OAAO8f,EAAQxZ,QAAQ,KAAK,KACzCuH,EAAIg2B,EACJ1B,EAAYrmC,KAAKyO,QAAQ+lC,SACzB1M,EAAa,IAAO9nC,KAAKyO,QAAQ+lC,SACjCxiC,EAAI+1B,EAAa,GAAMD,EAAa,CAExC9nC,MAAK4lC,IAAI34B,MAAMsF,MAAQ8zB,EAAY,EAAI0B,EAAa,IAEpD,KAAK,GAAIzQ,KAAWt3B,MAAKm0B,OACnBn0B,KAAKm0B,OAAOruB,eAAewxB,KACO,GAAhCt3B,KAAKm0B,OAAOmD,GAAS5O,SAAkEliB,SAA9CxG,KAAK6lC,iBAAiBrO,WAAWF,IAAuE,GAA7Ct3B,KAAK6lC,iBAAiBrO,WAAWF,KACvIt3B,KAAKm0B,OAAOmD,GAAS0Q,SAASj2B,EAAGC,EAAGhS,KAAKinC,YAAajnC,KAAK4lC,IAAKS,EAAWyB,GAC3E91B,GAAK81B,EAAa9nC,KAAKyO,QAAQgmC,aAKrC7zC,GAAQsQ,gBAAgBlR,KAAKinC,eAIjCpnC,EAAOD,QAAUmD,GAKb,SAASlD,EAAQD,EAASM,GAqB9B,QAAS8C,GAAU2xB,EAAMlmB,GACvBzO,KAAKK,GAAKM,EAAKqE,aACfhF,KAAK20B,KAAOA,EAEZ30B,KAAKq0B,gBACHuX,iBAAkB,OAClBmJ,aAAc,UACd7+B,MAAM,EACN8+B,UAAU,EACVC,YAAa,QACb1J,QACE78B,SAAS,EACT6lB,YAAa,UAEftnB,MAAO,OACPioC,UACE3iC,MAAO,GACP4iC,cAAe,UACfpG,MAAO,UAEThE,YACEr8B,SAAS,EACTs8B,gBAAiB,cACjBC,MAAO,IAET94B,YACEzD,SAAS,EACT2D,KAAM,EACNpF,MAAO,UAETmoC,UACEtP,iBAAiB,EACjBC,iBAAiB,EACjBC,OAAO,EACPzzB,MAAO,OACPmW,SAAS,EACT6S,YAAY,EACZD,aACE7zB,MAAOqE,IAAItF,OAAWkG,IAAIlG,QAC1B6gB,OAAQvb,IAAItF,OAAWkG,IAAIlG,UAkB/B6uC,QACE3mC,SAAS,EACTs3B,OAAO,EACPv+B,MACEihB,SAAS,EACT9E,SAAU,YAEZyD,OACEqB,SAAS,EACT9E,SAAU,cAGduQ,QACEqD,gBAKJx3B,KAAKyO,QAAU9N,EAAK2E,UAAWtF,KAAKq0B,gBACpCr0B,KAAK+vB,OACL/vB,KAAKgG,SACLhG,KAAK8D,OAAS,KACd9D,KAAKm0B,UACLn0B,KAAKs1C,oBAAqB,EAC1Bt1C,KAAKu1C,iBAAkB,EACvBv1C,KAAKw1C,yBAA0B,CAE/B,IAAIrhC,GAAKnU,IACTA,MAAK81B,UAAY,KACjB91B,KAAK+1B,WAAa,KAGlB/1B,KAAK0vC,eACHz8B,IAAO,SAAU3J,EAAOwK,GACtBK,EAAGw7B,OAAO77B,EAAO7R,QAEnB4S,OAAU,SAAUvL,EAAOwK,GACzBK,EAAGy7B,UAAU97B,EAAO7R,QAEtBoU,OAAU,SAAU/M,EAAOwK,GACzBK,EAAG07B,UAAU/7B,EAAO7R,SAKxBjC,KAAK8vC,gBACH78B,IAAO,SAAU3J,EAAOwK,GACtBK,EAAG47B,aAAaj8B,EAAO7R,QAEzB4S,OAAU,SAAUvL,EAAOwK,GACzBK,EAAG67B,gBAAgBl8B,EAAO7R,QAE5BoU,OAAU,SAAU/M,EAAOwK,GACzBK,EAAG87B,gBAAgBn8B,EAAO7R,SAI9BjC,KAAKiC,SACLjC,KAAKmwC,aACLnwC,KAAKy1C,UAAYz1C,KAAK20B,KAAKc,MAAM7lB,MACjC5P,KAAKqwC,eAELrwC,KAAKinC,eACLjnC,KAAKkT,WAAWzE,GAChBzO,KAAKwqC,0BAA4B,GACjCxqC,KAAK01C,QAAU,EACf11C,KAAK20B,KAAKE,QAAQthB,GAAG,eAAgB,WACnCY,EAAGshC,UAAYthC,EAAGwgB,KAAKc,MAAM7lB,MAC7BuE,EAAGyxB,IAAI34B,MAAMxF,KAAO9G,EAAKmJ,OAAOK,QAAQgK,EAAGnO,MAAMuM,OACjD4B,EAAGsN,OAAOlhB,KAAK4T,GAAG,KAIpBnU,KAAK00B,UACL10B,KAAKgsC,WAAapG,IAAK5lC,KAAK4lC,IAAKqB,YAAajnC,KAAKinC,YAAax4B,QAASzO,KAAKyO,QAAS0lB,OAAQn0B,KAAKm0B,QACpGn0B,KAAK20B,KAAKE,QAAQjH,KAAK,UAvJzB,GAAIjtB,GAAOT,EAAoB,GAC3BU,EAAUV,EAAoB,GAC9BW,EAAUX,EAAoB,GAC9BY,EAAWZ,EAAoB,GAC/BqC,EAAYrC,EAAoB,IAChCwC,EAAWxC,EAAoB,IAC/ByC,EAAazC,EAAoB,IACjC6C,EAAS7C,EAAoB,IAC7By1C,EAAoBz1C,EAAoB,IAExCowC,EAAY,eAiJhBttC,GAAUmQ,UAAY,GAAI5Q,GAK1BS,EAAUmQ,UAAUuhB,QAAU,WAC5B,GAAIpV,GAAQ/N,SAASM,cAAc,MACnCyN,GAAMzX,UAAY,YAClB7H,KAAK+vB,IAAIzQ,MAAQA,EAGjBtf,KAAK4lC,IAAMr0B,SAASC,gBAAgB,6BAA6B,OACjExR,KAAK4lC,IAAI34B,MAAM2W,SAAW,WAC1B5jB,KAAK4lC,IAAI34B,MAAMuF,QAAU,GAAKxS,KAAKyO,QAAQwmC,aAAazqC,QAAQ,KAAK,IAAM,KAC3ExK,KAAK4lC,IAAI34B,MAAM26B,QAAU,QACzBtoB,EAAM7N,YAAYzR,KAAK4lC,KAGvB5lC,KAAKyO,QAAQ2mC,SAAS7gB,YAAc,OACpCv0B,KAAK41C,UAAY,GAAIlzC,GAAS1C,KAAK20B,KAAM30B,KAAKyO,QAAQ2mC,SAAUp1C,KAAK4lC,IAAK5lC,KAAKyO,QAAQ0lB,QAEvFn0B,KAAKyO,QAAQ2mC,SAAS7gB,YAAc,QACpCv0B,KAAK61C,WAAa,GAAInzC,GAAS1C,KAAK20B,KAAM30B,KAAKyO,QAAQ2mC,SAAUp1C,KAAK4lC,IAAK5lC,KAAKyO,QAAQ0lB,cACjFn0B,MAAKyO,QAAQ2mC,SAAS7gB,YAG7Bv0B,KAAK81C,WAAa,GAAI/yC,GAAO/C,KAAK20B,KAAM30B,KAAKyO,QAAQ4mC,OAAQ,OAAQr1C,KAAKyO,QAAQ0lB,QAClFn0B,KAAK+1C,YAAc,GAAIhzC,GAAO/C,KAAK20B,KAAM30B,KAAKyO,QAAQ4mC,OAAQ,QAASr1C,KAAKyO,QAAQ0lB,QAEpFn0B,KAAK2nC,QAOP3kC,EAAUmQ,UAAUD,WAAa,SAASzE,GACxC,GAAIA,EAAS,CACX,GAAIP,IAAU,WAAW,eAAe,SAAS,cAAc,mBAAmB,QAAQ,WAAW,WAAW,OAAO,SAC3F1H,UAAxBiI,EAAQwmC,aAAgDzuC,SAAnBiI,EAAQ+D,QAAsEhM,SAA9CxG,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,QAC1GxS,KAAKu1C,iBAAkB,EACvBv1C,KAAKw1C,yBAA0B,GAEsBhvC,SAA9CxG,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,QAAgDhM,SAAxBiI,EAAQwmC,aACtErqC,UAAU6D,EAAQwmC,YAAc,IAAIzqC,QAAQ,KAAK,KAAOxK,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,SAC7FxS,KAAKu1C,iBAAkB,GAG3B50C,EAAKwF,oBAAoB+H,EAAQlO,KAAKyO,QAASA,GAC/C9N,EAAK4N,aAAavO,KAAKyO,QAASA,EAAQ,cACxC9N,EAAK4N,aAAavO,KAAKyO,QAASA,EAAQ,cACxC9N,EAAK4N,aAAavO,KAAKyO,QAASA,EAAQ,UACxC9N,EAAK4N,aAAavO,KAAKyO,QAASA,EAAQ,UAEpCA,EAAQs8B,YACuB,gBAAtBt8B,GAAQs8B,YACbt8B,EAAQs8B,WAAWC,kBACqB,WAAtCv8B,EAAQs8B,WAAWC,gBACrBhrC,KAAKyO,QAAQs8B,WAAWE,MAAQ,EAEa,WAAtCx8B,EAAQs8B,WAAWC,gBAC1BhrC,KAAKyO,QAAQs8B,WAAWE,MAAQ,GAGhCjrC,KAAKyO,QAAQs8B,WAAWC,gBAAkB,cAC1ChrC,KAAKyO,QAAQs8B,WAAWE,MAAQ,KAMpCjrC,KAAK41C,WACkBpvC,SAArBiI,EAAQ2mC,WACVp1C,KAAK41C,UAAU1iC,WAAWlT,KAAKyO,QAAQ2mC,UACvCp1C,KAAK61C,WAAW3iC,WAAWlT,KAAKyO,QAAQ2mC,WAIxCp1C,KAAK81C,YACgBtvC,SAAnBiI,EAAQ4mC,SACVr1C,KAAK81C,WAAW5iC,WAAWlT,KAAKyO,QAAQ4mC,QACxCr1C,KAAK+1C,YAAY7iC,WAAWlT,KAAKyO,QAAQ4mC,SAIzCr1C,KAAKm0B,OAAOruB,eAAewqC,IAC7BtwC,KAAKm0B,OAAOmc,GAAWp9B,WAAWzE,GAKlCzO,KAAK+vB,IAAIzQ,OACXtf,KAAKyhB,QAAO,IAOhBze,EAAUmQ,UAAUu0B,KAAO,WAErB1nC,KAAK+vB,IAAIzQ,MAAMzV,YACjB7J,KAAK+vB,IAAIzQ,MAAMzV,WAAWsH,YAAYnR,KAAK+vB,IAAIzQ,QASnDtc,EAAUmQ,UAAUw0B,KAAO,WAEpB3nC,KAAK+vB,IAAIzQ,MAAMzV,YAClB7J,KAAK20B,KAAK5E,IAAI7D,OAAOza,YAAYzR,KAAK+vB,IAAIzQ,QAS9Ctc,EAAUmQ,UAAU8iB,SAAW,SAASh0B,GACtC,GACEkT,GADEhB,EAAKnU,KAEPuyC,EAAevyC,KAAK81B,SAGtB,IAAK7zB,EAGA,CAAA,KAAIA,YAAiBpB,IAAWoB,YAAiBnB,IAIpD,KAAM,IAAIuF,WAAU,kDAHpBrG,MAAK81B,UAAY7zB,MAHjBjC,MAAK81B,UAAY,IAoBnB,IAXIyc,IAEF5xC,EAAK0H,QAAQrI,KAAK0vC,cAAe,SAAUpnC,EAAUgB,GACnDipC,EAAa7+B,IAAIpK,EAAOhB,KAI1B6M,EAAMo9B,EAAa18B,SACnB7V,KAAK6vC,UAAU16B,IAGbnV,KAAK81B,UAAW,CAElB,GAAIz1B,GAAKL,KAAKK,EACdM,GAAK0H,QAAQrI,KAAK0vC,cAAe,SAAUpnC,EAAUgB,GACnD6K,EAAG2hB,UAAUviB,GAAGjK,EAAOhB,EAAUjI,KAInC8U,EAAMnV,KAAK81B,UAAUjgB,SACrB7V,KAAK2vC,OAAOx6B,GAEdnV,KAAKywC,mBAELzwC,KAAKyhB,QAAO,IAQdze,EAAUmQ,UAAU6iB,UAAY,SAAS7B,GACvC,GACIhf,GADAhB,EAAKnU,IAgBT,IAZIA,KAAK+1B,aACPp1B,EAAK0H,QAAQrI,KAAK8vC,eAAgB,SAAUxnC,EAAUgB,GACpD6K,EAAG4hB,WAAWniB,YAAYtK,EAAOhB,KAInC6M,EAAMnV,KAAK+1B,WAAWlgB,SACtB7V,KAAK+1B,WAAa,KAClB/1B,KAAKiwC,gBAAgB96B,IAIlBgf,EAGA,CAAA,KAAIA,YAAkBtzB,IAAWszB,YAAkBrzB,IAItD,KAAM,IAAIuF,WAAU,kDAHpBrG,MAAK+1B,WAAa5B,MAHlBn0B,MAAK+1B,WAAa,IASpB,IAAI/1B,KAAK+1B,WAAY,CAEnB,GAAI11B,GAAKL,KAAKK,EACdM,GAAK0H,QAAQrI,KAAK8vC,eAAgB,SAAUxnC,EAAUgB,GACpD6K,EAAG4hB,WAAWxiB,GAAGjK,EAAOhB,EAAUjI,KAIpC8U,EAAMnV,KAAK+1B,WAAWlgB,SACtB7V,KAAK+vC,aAAa56B,GAEpBnV,KAAK4vC,aASP5sC,EAAUmQ,UAAUy8B,UAAY,WAC9B5vC,KAAKywC,mBACLzwC,KAAKg2C,sBAELh2C,KAAKyhB,QAAO,IAEdze,EAAUmQ,UAAUw8B,OAAkB,SAAUx6B,GAAMnV,KAAK4vC,UAAUz6B,IACrEnS,EAAUmQ,UAAU08B,UAAkB,SAAU16B,GAAMnV,KAAK4vC,UAAUz6B,IACrEnS,EAAUmQ,UAAU68B,gBAAmB,SAAUE,GAC/C,IAAK,GAAI1qC,GAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAAK,CACxC,GAAIyM,GAAQjS,KAAK+1B,WAAW7gB,IAAIg7B,EAAS1qC,GACzCxF,MAAKi2C,aAAahkC,EAAOi+B,EAAS1qC,IAIpCxF,KAAKyhB,QAAO,IAEdze,EAAUmQ,UAAU48B,aAAe,SAAUG,GAAWlwC,KAAKgwC,gBAAgBE,IAQ7EltC,EAAUmQ,UAAU88B,gBAAkB,SAAUC,GAC9C,IAAK,GAAI1qC,GAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAC/BxF,KAAKm0B,OAAOruB,eAAeoqC,EAAS1qC,MACmB,SAArDxF,KAAKm0B,OAAO+b,EAAS1qC,IAAIiJ,QAAQm9B,kBACnC5rC,KAAK61C,WAAWpO,YAAYyI,EAAS1qC,IACrCxF,KAAK+1C,YAAYtO,YAAYyI,EAAS1qC,IACtCxF,KAAK+1C,YAAYt0B,WAGjBzhB,KAAK41C,UAAUnO,YAAYyI,EAAS1qC,IACpCxF,KAAK81C,WAAWrO,YAAYyI,EAAS1qC,IACrCxF,KAAK81C,WAAWr0B,gBAEXzhB,MAAKm0B,OAAO+b,EAAS1qC,IAGhCxF,MAAKywC,mBAELzwC,KAAKyhB,QAAO,IAWdze,EAAUmQ,UAAU8iC,aAAe,SAAUhkC,EAAOqlB,GAC7Ct3B,KAAKm0B,OAAOruB,eAAewxB,IAY9Bt3B,KAAKm0B,OAAOmD,GAASziB,OAAO5C,GACyB,SAAjDjS,KAAKm0B,OAAOmD,GAAS7oB,QAAQm9B,kBAC/B5rC,KAAK61C,WAAWrO,YAAYlQ,EAASt3B,KAAKm0B,OAAOmD,IACjDt3B,KAAK+1C,YAAYvO,YAAYlQ,EAASt3B,KAAKm0B,OAAOmD,MAGlDt3B,KAAK41C,UAAUpO,YAAYlQ,EAASt3B,KAAKm0B,OAAOmD,IAChDt3B,KAAK81C,WAAWtO,YAAYlQ,EAASt3B,KAAKm0B,OAAOmD,OAlBnDt3B,KAAKm0B,OAAOmD,GAAW,GAAI30B,GAAWsP,EAAOqlB,EAASt3B,KAAKyO,QAASzO,KAAKwqC,0BACpB,SAAjDxqC,KAAKm0B,OAAOmD,GAAS7oB,QAAQm9B,kBAC/B5rC,KAAK61C,WAAWvO,SAAShQ,EAASt3B,KAAKm0B,OAAOmD,IAC9Ct3B,KAAK+1C,YAAYzO,SAAShQ,EAASt3B,KAAKm0B,OAAOmD,MAG/Ct3B,KAAK41C,UAAUtO,SAAShQ,EAASt3B,KAAKm0B,OAAOmD,IAC7Ct3B,KAAK81C,WAAWxO,SAAShQ,EAASt3B,KAAKm0B,OAAOmD,MAclDt3B,KAAK81C,WAAWr0B,SAChBzhB,KAAK+1C,YAAYt0B,UASnBze,EAAUmQ,UAAU6iC,oBAAsB,WACxC,GAAsB,MAAlBh2C,KAAK81B,UAAmB,CAC1B,GACIwB,GADA4e,IAEJ,KAAK5e,IAAWt3B,MAAKm0B,OACfn0B,KAAKm0B,OAAOruB,eAAewxB,KAC7B4e,EAAc5e,MAGlB,KAAK,GAAI/hB,KAAUvV,MAAK81B,UAAUljB,MAChC,GAAI5S,KAAK81B,UAAUljB,MAAM9M,eAAeyP,GAAS,CAC/C,GAAIlG,GAAOrP,KAAK81B,UAAUljB,MAAM2C,EAChC,IAAkC/O,SAA9B0vC,EAAc7mC,EAAK4C,OACrB,KAAM,IAAIrO,OAAM,4IAElByL,GAAK0C,EAAIpR,EAAKkG,QAAQwI,EAAK0C,EAAE,QAC7BmkC,EAAc7mC,EAAK4C,OAAOjK,KAAKqH,GAGnC,IAAKioB,IAAWt3B,MAAKm0B,OACfn0B,KAAKm0B,OAAOruB,eAAewxB,IAC7Bt3B,KAAKm0B,OAAOmD,GAASrB,SAASigB,EAAc5e,MAYpDt0B,EAAUmQ,UAAUs9B,iBAAmB,WACrC,GAAIzwC,KAAK81B,WAA+B,MAAlB91B,KAAK81B,UAAmB,CAC5C,GAAIqgB,GAAmB,CACvB,KAAK,GAAI5gC,KAAUvV,MAAK81B,UAAUljB,MAChC,GAAI5S,KAAK81B,UAAUljB,MAAM9M,eAAeyP,GAAS,CAC/C,GAAIlG,GAAOrP,KAAK81B,UAAUljB,MAAM2C,EACpB/O,SAAR6I,IACEA,EAAKvJ,eAAe,SACHU,SAAf6I,EAAK4C,QACP5C,EAAK4C,MAAQq+B,GAIfjhC,EAAK4C,MAAQq+B,EAEf6F,EAAmB9mC,EAAK4C,OAASq+B,EAAY6F,EAAmB,EAAIA,GAK1E,GAAwB,GAApBA,QACKn2C,MAAKm0B,OAAOmc,GACnBtwC,KAAK81C,WAAWrO,YAAY6I,GAC5BtwC,KAAK+1C,YAAYtO,YAAY6I,GAC7BtwC,KAAK41C,UAAUnO,YAAY6I,GAC3BtwC,KAAK61C,WAAWpO,YAAY6I,OAEzB,CACH,GAAIr+B,IAAS5R,GAAIiwC,EAAW1gB,QAAS5vB,KAAKyO,QAAQsmC,aAClD/0C,MAAKi2C,aAAahkC,EAAOq+B,eAIpBtwC,MAAKm0B,OAAOmc,GACnBtwC,KAAK81C,WAAWrO,YAAY6I,GAC5BtwC,KAAK+1C,YAAYtO,YAAY6I,GAC7BtwC,KAAK41C,UAAUnO,YAAY6I,GAC3BtwC,KAAK61C,WAAWpO,YAAY6I,EAG9BtwC,MAAK81C,WAAWr0B,SAChBzhB,KAAK+1C,YAAYt0B,UAQnBze,EAAUmQ,UAAUsO,OAAS,SAAS20B,GACpC,GAAI3R,IAAU,CAGdzkC,MAAKgG,MAAMuM,MAAQvS,KAAK+vB,IAAIzQ,MAAM8Q,YAClCpwB,KAAKgG,MAAMwM,OAASxS,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,OAGhChM,SAAnBxG,KAAK2xC,WAA2B3xC,KAAKgG,MAAMuM,QAC7C6jC,GAAmB,GAIrB3R,EAAUzkC,KAAKwkC,cAAgBC,CAG/B,IAAI+M,GAAkBxxC,KAAK20B,KAAKc,MAAM5lB,IAAM7P,KAAK20B,KAAKc,MAAM7lB,MACxD6hC,EAAUD,GAAmBxxC,KAAK0xC,mBA6BtC,IA5BA1xC,KAAK0xC,oBAAsBF,EAKZ,GAAX/M,IACFzkC,KAAK4lC,IAAI34B,MAAMsF,MAAQ5R,EAAKmJ,OAAOK,OAAO,EAAEnK,KAAKgG,MAAMuM,OACvDvS,KAAK4lC,IAAI34B,MAAMxF,KAAO9G,EAAKmJ,OAAOK,QAAQnK,KAAKgG,MAAMuM,QAGN,KAA1CvS,KAAKyO,QAAQ+D,OAAS,IAAI7L,QAAQ,MAA8C,GAAhC3G,KAAKw1C,2BACxDx1C,KAAKu1C,iBAAkB,IAKC,GAAxBv1C,KAAKu1C,iBACHv1C,KAAKyO,QAAQwmC,aAAej1C,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,OAAS,OAC1ExS,KAAKyO,QAAQwmC,YAAcj1C,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,OAAS,KACvExS,KAAK4lC,IAAI34B,MAAMuF,OAASxS,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,OAAS,MAEtExS,KAAKu1C,iBAAkB,GAGvBv1C,KAAK4lC,IAAI34B,MAAMuF,QAAU,GAAKxS,KAAKyO,QAAQwmC,aAAazqC,QAAQ,KAAK,IAAM,KAI9D,GAAXi6B,GAA6B,GAAVgN,GAA6C,GAA3BzxC,KAAKs1C,oBAAkD,GAApBc,EAC1E3R,EAAUzkC,KAAKq2C,gBAAkB5R,MAIjC,IAAsB,GAAlBzkC,KAAKy1C,UAAgB,CACvB,GAAI9rB,GAAS3pB,KAAK20B,KAAKc,MAAM7lB,MAAQ5P,KAAKy1C,UACtChgB,EAAQz1B,KAAK20B,KAAKc,MAAM5lB,IAAM7P,KAAK20B,KAAKc,MAAM7lB,KAClD,IAAwB,GAApB5P,KAAKgG,MAAMuM,MAAY,CACzB,GAAI+jC,GAAmBt2C,KAAKgG,MAAMuM,MAAMkjB,EACpC7L,EAAUD,EAAS2sB,CACvBt2C,MAAK4lC,IAAI34B,MAAMxF,MAASzH,KAAKgG,MAAMuM,MAAQqX,EAAW,MAO5D,MAFA5pB,MAAK81C,WAAWr0B,SAChBzhB,KAAK+1C,YAAYt0B,SACVgjB,GAQTzhC,EAAUmQ,UAAUkjC,aAAe,WAGjC,GADAz1C,EAAQiQ,gBAAgB7Q,KAAKinC,aACL,GAApBjnC,KAAKgG,MAAMuM,OAAgC,MAAlBvS,KAAK81B,UAAmB,CACnD,GAAI7jB,GAAOzM,EACP+wC,KACAC,KACAC,KACAC,GAAe,EAGfxG,IACJ,KAAK,GAAI5Y,KAAWt3B,MAAKm0B,OACnBn0B,KAAKm0B,OAAOruB,eAAewxB,KAC7BrlB,EAAQjS,KAAKm0B,OAAOmD,GACC,GAAjBrlB,EAAMyW,SAAgEliB,SAA5CxG,KAAKyO,QAAQ0lB,OAAOqD,WAAWF,IAAqE,GAA3Ct3B,KAAKyO,QAAQ0lB,OAAOqD,WAAWF,IACpH4Y,EAASloC,KAAKsvB,GAIpB,IAAI4Y,EAASvqC,OAAS,EAAG,CAEvB,GAAIgxC,GAAU32C,KAAK20B,KAAKh0B,KAAK40B,cAAcv1B,KAAK20B,KAAKC,SAASl1B,KAAK6S,OAC/DqkC,EAAU52C,KAAK20B,KAAKh0B,KAAK40B,aAAa,EAAIv1B,KAAK20B,KAAKC,SAASl1B,KAAK6S,OAClEwjB,IAQJ,KANA/1B,KAAK62C,iBAAiB3G,EAAUna,EAAY4gB,EAASC,GAGrD52C,KAAK82C,eAAe5G,EAAUna,GAGzBvwB,EAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAC/B+wC,EAAsBrG,EAAS1qC,IAAMxF,KAAK+2C,qBAAqBhhB,EAAWma,EAAS1qC,IAIrFxF,MAAKg3C,YAAY9G,EAAUqG,EAAuBE,GAIlDC,EAAe12C,KAAKi3C,aAAa/G,EAAUuG,EAC3C,IAAIS,GAAa,CACjB,IAAoB,GAAhBR,GAAwB12C,KAAK01C,QAAUwB,EAKzC,MAJAt2C,GAAQsQ,gBAAgBlR,KAAKinC,aAC7BjnC,KAAKs1C,oBAAqB,EAC1Bt1C,KAAK01C,UACL11C,KAAK20B,KAAKE,QAAQjH,KAAK,WAChB,CAUP,KAPI5tB,KAAK01C,QAAUwB,GACjBve,QAAQhF,IAAI,6EAEd3zB,KAAK01C,QAAU,EACf11C,KAAKs1C,oBAAqB,EAGrB9vC,EAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAC/ByM,EAAQjS,KAAKm0B,OAAO+b,EAAS1qC,IAC7BgxC,EAAmBtG,EAAS1qC,IAAMxF,KAAKm3C,qBAAqBphB,EAAWma,EAAS1qC,IAAKyM,EAIvF,KAAKzM,EAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAC/ByM,EAAQjS,KAAKm0B,OAAO+b,EAAS1qC,IACF,OAAvByM,EAAMxD,QAAQxB,OAChBgF,EAAM85B,KAAKyK,EAAmBtG,EAAS1qC,IAAKyM,EAAOjS,KAAKgsC,UAG5D2J,GAAkB5J,KAAKmE,EAAUsG,EAAoBx2C,KAAKgsC,YAOhE,MADAprC,GAAQsQ,gBAAgBlR,KAAKinC,cACtB,GAiBTjkC,EAAUmQ,UAAU0jC,iBAAmB,SAAU3G,EAAUna,EAAY4gB,EAASC,GAC9E,GAAI3kC,GAAOzM,EAAGomB,EAAGvc,CACjB,IAAI6gC,EAASvqC,OAAS,EACpB,IAAKH,EAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAAK,CACpCyM,EAAQjS,KAAKm0B,OAAO+b,EAAS1qC,IAC7BuwB,EAAWma,EAAS1qC,MACpB,IAAI4xC,GAAgBrhB,EAAWma,EAAS1qC,GAExC;GAA0B,GAAtByM,EAAMxD,QAAQyH,KAAc,CAC9B,GAAImhC,GAAQnyC,KAAKwH,IAAI,EAAG/L,EAAK4O,kBAAkB0C,EAAM6jB,UAAW6gB,EAAS,IAAK,UAC9E,KAAK/qB,EAAIyrB,EAAOzrB,EAAI3Z,EAAM6jB,UAAUnwB,OAAQimB,IAE1C,GADAvc,EAAO4C,EAAM6jB,UAAUlK,GACVplB,SAAT6I,EAAoB,CACtB,GAAIA,EAAK0C,EAAI6kC,EAAS,CACpBQ,EAAcpvC,KAAKqH,EACnB,OAGA+nC,EAAcpvC,KAAKqH,QAMzB,KAAKuc,EAAI,EAAGA,EAAI3Z,EAAM6jB,UAAUnwB,OAAQimB,IACtCvc,EAAO4C,EAAM6jB,UAAUlK,GACVplB,SAAT6I,GACEA,EAAK0C,EAAI4kC,GAAWtnC,EAAK0C,EAAI6kC,GAC/BQ,EAAcpvC,KAAKqH,KAgBjCrM,EAAUmQ,UAAU2jC,eAAiB,SAAU5G,EAAUna,GACvD,GAAI9jB,EACJ,IAAIi+B,EAASvqC,OAAS,EACpB,IAAK,GAAIH,GAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAEnC,GADAyM,EAAQjS,KAAKm0B,OAAO+b,EAAS1qC,IACC,GAA1ByM,EAAMxD,QAAQumC,SAAkB,CAClC,GAAIoC,GAAgBrhB,EAAWma,EAAS1qC,GACxC,IAAI4xC,EAAczxC,OAAS,EAAG,CAC5B,GAAI2xC,GAAY,EACZC,EAAiBH,EAAczxC,OAI/B6xC,EAAYx3C,KAAK20B,KAAKh0B,KAAKw0B,eAAeiiB,EAAcA,EAAczxC,OAAS,GAAGoM,GAAK/R,KAAK20B,KAAKh0B,KAAKw0B,eAAeiiB,EAAc,GAAGrlC,GACtI0lC,EAAiBF,EAAiBC,CACtCF,GAAYpyC,KAAK4G,IAAI5G,KAAKwyC,KAAK,GAAMH,GAAiBryC,KAAKwH,IAAI,EAAGxH,KAAKwoB,MAAM+pB,IAG7E,KAAK,GADDE,MACK/rB,EAAI,EAAO2rB,EAAJ3rB,EAAoBA,GAAK0rB,EACvCK,EAAY3vC,KAAKovC,EAAcxrB,GAGjCmK,GAAWma,EAAS1qC,IAAMmyC,KAgBpC30C,EAAUmQ,UAAU6jC,YAAc,SAAU9G,EAAUna,EAAY0gB,GAChE,GAAI3K,GAAW75B,EAAOzM,EAGlBiJ,EAFAmpC,KACAC,IAEJ,IAAI3H,EAASvqC,OAAS,EAAG,CACvB,IAAKH,EAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAC/BsmC,EAAY/V,EAAWma,EAAS1qC,IAChCiJ,EAAUzO,KAAKm0B,OAAO+b,EAAS1qC,IAAIiJ,QAC/Bq9B,EAAUnmC,OAAS,IACrBsM,EAAQjS,KAAKm0B,OAAO+b,EAAS1qC,IAES,SAAlCiJ,EAAQymC,SAASC,eAA6C,OAAjB1mC,EAAQxB,MACvB,QAA5BwB,EAAQm9B,iBAA6BgM,EAAuBA,EAAoB5jC,OAAO/B,EAAM45B,UAAUC,IAClE+L,EAAuBA,EAAqB7jC,OAAO/B,EAAM45B,UAAUC,IAG5G2K,EAAYvG,EAAS1qC,IAAMyM,EAAM45B,UAAUC,EAAUoE,EAAS1qC,IAMpEmwC,GAAkBmC,oBAAoBF,EAAsBnB,EAAavG,EAAU,iBAAmB,QACtGyF,EAAkBmC,oBAAoBD,EAAsBpB,EAAavG,EAAU,kBAAmB,WAW1GltC,EAAUmQ,UAAU8jC,aAAe,SAAU/G,EAAUuG,GACrD,GAGoEsB,GAAQC,EAHxEvT,GAAU,EACVwT,GAAgB,EAChBC,GAAiB,EACjBC,EAAU,IAAKC,EAAW,IAAKC,EAAU,KAAMC,EAAW,IAE9D,IAAIpI,EAASvqC,OAAS,EAAG,CAEvB,IAAK,GAAIH,GAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAAK,CACxC,GAAIyM,GAAQjS,KAAKm0B,OAAO+b,EAAS1qC,GAC7ByM,IAA2C,SAAlCA,EAAMxD,QAAQm9B,kBACzBqM,GAAgB,EAChBE,EAAU,EACVE,EAAU,GAEHpmC,GAASA,EAAMxD,QAAQm9B,mBAC9BsM,GAAiB,EACjBE,EAAW,EACXE,EAAW,GAKf,IAAK,GAAI9yC,GAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAC/BixC,EAAY3wC,eAAeoqC,EAAS1qC,KAClCixC,EAAYvG,EAAS1qC,IAAI+yC,UAAW,IACtCR,EAAStB,EAAYvG,EAAS1qC,IAAIsG,IAClCksC,EAASvB,EAAYvG,EAAS1qC,IAAIkH,IAEe,SAA7C+pC,EAAYvG,EAAS1qC,IAAIomC,kBAC3BqM,GAAgB,EAChBE,EAAUA,EAAUJ,EAASA,EAASI,EACtCE,EAAoBL,EAAVK,EAAmBL,EAASK,IAGtCH,GAAiB,EACjBE,EAAWA,EAAWL,EAASA,EAASK,EACxCE,EAAsBN,EAAXM,EAAoBN,EAASM,GAM3B,IAAjBL,GACFj4C,KAAK41C,UAAUriB,SAAS4kB,EAASE,GAEb,GAAlBH,GACFl4C,KAAK61C,WAAWtiB,SAAS6kB,EAAUE,GAoCvC,MAjCA7T,GAAUzkC,KAAKw4C,qBAAqBP,EAAgBj4C,KAAK41C,YAAenR,EACxEA,EAAUzkC,KAAKw4C,qBAAqBN,EAAgBl4C,KAAK61C,aAAepR,EAElD,GAAlByT,GAA2C,GAAjBD,GAC5Bj4C,KAAK41C,UAAU6C,WAAY,EAC3Bz4C,KAAK61C,WAAW4C,WAAY,IAG5Bz4C,KAAK41C,UAAU6C,WAAY,EAC3Bz4C,KAAK61C,WAAW4C,WAAY,GAE9Bz4C,KAAK61C,WAAW7O,QAAUiR,EACI,GAA1Bj4C,KAAK61C,WAAW7O,QACWhnC,KAAK41C,UAAU7O,WAAtB,GAAlBmR,EAAqDl4C,KAAK61C,WAAWtjC,MAChB,EAEzDkyB,EAAUzkC,KAAK41C,UAAUn0B,UAAYgjB,EACrCzkC,KAAK61C,WAAWhP,iBAAmB7mC,KAAK41C,UAAUhP,WAClD5mC,KAAK61C,WAAW/O,aAAe9mC,KAAK41C,UAAU9O,aAC9CrC,EAAUzkC,KAAK61C,WAAWp0B,UAAYgjB,GAGtCA,EAAUzkC,KAAK61C,WAAWp0B,UAAYgjB,EAIE,IAAtCyL,EAASvpC,QAAQ,mBACnBupC,EAAS9nC,OAAO8nC,EAASvpC,QAAQ,kBAAkB,GAEV,IAAvCupC,EAASvpC,QAAQ,oBACnBupC,EAAS9nC,OAAO8nC,EAASvpC,QAAQ,mBAAmB,GAG/C89B,GAYTzhC,EAAUmQ,UAAUqlC,qBAAuB,SAAUE,EAAU5X,GAC7D,GAAI7B,IAAU,CAad,OAZgB,IAAZyZ,EACE5X,EAAK/Q,IAAIzQ,MAAMzV,YAA6B,GAAfi3B,EAAK5H,SACpC4H,EAAK4G,OACLzI,GAAU,GAIP6B,EAAK/Q,IAAIzQ,MAAMzV,YAA6B,GAAfi3B,EAAK5H,SACrC4H,EAAK6G,OACL1I,GAAU,GAGPA,GAaTj8B,EAAUmQ,UAAU4jC,qBAAuB,SAAU4B,GAKnD,IAAK,GAHDC,GAAQC,EADRC,KAEA7jB,EAAWj1B,KAAK20B,KAAKh0B,KAAKs0B,SAErBzvB,EAAI,EAAGA,EAAImzC,EAAWhzC,OAAQH,IACrCozC,EAAS3jB,EAAS0jB,EAAWnzC,GAAGuM,GAAK/R,KAAKgG,MAAMuM,MAChDsmC,EAASF,EAAWnzC,GAAGwM,EACvB8mC,EAAc9wC,MAAM+J,EAAG6mC,EAAQ5mC,EAAG6mC,GAGpC,OAAOC,IAcT91C,EAAUmQ,UAAUgkC,qBAAuB,SAAUwB,EAAY1mC,GAC/D,GACI2mC,GAAQC,EADRC,KAEA7jB,EAAWj1B,KAAK20B,KAAKh0B,KAAKs0B,SAC1B6L,EAAO9gC,KAAK41C,UACZmD,EAAY70C,OAAOlE,KAAK4lC,IAAI34B,MAAMuF,OAAOhI,QAAQ,KAAK,IACpB,UAAlCyH,EAAMxD,QAAQm9B,mBAChB9K,EAAO9gC,KAAK61C,WAGd,KAAK,GAAIrwC,GAAI,EAAGA,EAAImzC,EAAWhzC,OAAQH,IACrCozC,EAAS3jB,EAAS0jB,EAAWnzC,GAAGuM,GAAK/R,KAAKgG,MAAMuM,MAChDsmC,EAAS3zC,KAAKwoB,MAAMoT,EAAK2I,aAAakP,EAAWnzC,GAAGwM,IACpD8mC,EAAc9wC,MAAM+J,EAAG6mC,EAAQ5mC,EAAG6mC,GAKpC,OAFA5mC,GAAM64B,gBAAgB5lC,KAAK4G,IAAIitC,EAAWjY,EAAK2I,aAAa,KAErDqP,GAITj5C,EAAOD,QAAUoD,GAKb,SAASnD,EAAQD,EAASM,GAgB9B,QAAS+C,GAAU0xB,EAAMlmB,GACvBzO,KAAK+vB,KACH0c,WAAY,KACZjG,SACAwS,cACAC,cACAjoC,WACEw1B,SACAwS,cACAC,gBAGJj5C,KAAKgG,OACHyvB,OACE7lB,MAAO,EACPC,IAAK,EACLurB,YAAa,GAEf8d,QAAS,GAGXl5C,KAAKq0B,gBACHE,YAAa,SAEbuR,iBAAiB,EACjBC,iBAAiB,EACjBzE,OAAQ,MAEVthC,KAAKyO,QAAU9N,EAAK2E,UAAWtF,KAAKq0B,gBAEpCr0B,KAAK20B,KAAOA,EAGZ30B,KAAK00B,UAEL10B,KAAKkT,WAAWzE,GAjDlB,GAAI9N,GAAOT,EAAoB,GAC3BqC,EAAYrC,EAAoB,IAChC6B,EAAW7B,EAAoB,IAC/ByB,EAAWzB,EAAoB,IAC/B2D,EAAS3D,EAAoB,GAgDjC+C,GAASkQ,UAAY,GAAI5Q,GAUzBU,EAASkQ,UAAUD,WAAa,SAASzE,GACnCA,IAEF9N,EAAKoF,iBACH,cACA,kBACA,kBACA,cACA,UACC/F,KAAKyO,QAASA,GAIb,UAAYA,KACe,kBAAlB5K,GAAOwgC,OAEhBxgC,EAAOwgC,OAAO51B,EAAQ41B,QAGtBxgC,EAAOygC,KAAK71B,EAAQ41B,WAS5BphC,EAASkQ,UAAUuhB,QAAU,WAC3B10B,KAAK+vB,IAAI0c,WAAal7B,SAASM,cAAc,OAC7C7R,KAAK+vB,IAAI5jB,WAAaoF,SAASM,cAAc,OAE7C7R,KAAK+vB,IAAI0c,WAAW5kC,UAAY,sBAChC7H,KAAK+vB,IAAI5jB,WAAWtE,UAAY,uBAMlC5E,EAASkQ,UAAUG,QAAU,WAEvBtT,KAAK+vB,IAAI0c,WAAW5iC,YACtB7J,KAAK+vB,IAAI0c,WAAW5iC,WAAWsH,YAAYnR,KAAK+vB,IAAI0c,YAElDzsC,KAAK+vB,IAAI5jB,WAAWtC,YACtB7J,KAAK+vB,IAAI5jB,WAAWtC,WAAWsH,YAAYnR,KAAK+vB,IAAI5jB,YAGtDnM,KAAK20B,KAAO,MAOd1xB,EAASkQ,UAAUsO,OAAS,WAC1B,GAAIhT,GAAUzO,KAAKyO,QACfzI,EAAQhG,KAAKgG,MACbymC,EAAazsC,KAAK+vB,IAAI0c,WACtBtgC,EAAanM,KAAK+vB,IAAI5jB,WAGtB24B,EAAiC,OAAvBr2B,EAAQ8lB,YAAwBv0B,KAAK20B,KAAK5E,IAAIpoB,IAAM3H,KAAK20B,KAAK5E,IAAIzM,OAC5E61B,EAAiB1M,EAAW5iC,aAAei7B,CAG/C9kC,MAAKooC,oBAGL,IACItC,IADc9lC,KAAKyO,QAAQ8lB,YACTv0B,KAAKyO,QAAQq3B,iBAC/BC,EAAkB/lC,KAAKyO,QAAQs3B,eAGnC//B,GAAMqiC,iBAAmBvC,EAAkB9/B,EAAMsiC,gBAAkB,EACnEtiC,EAAMuiC,iBAAmBxC,EAAkB//B,EAAMwiC,gBAAkB,EACnExiC,EAAMwM,OAASxM,EAAMqiC,iBAAmBriC,EAAMuiC,iBAC9CviC,EAAMuM,MAAQk6B,EAAWrc,YAEzBpqB,EAAM0iC,gBAAkB1oC,KAAK20B,KAAKC,SAASl1B,KAAK8S,OAASxM,EAAMuiC,kBACnC,OAAvB95B,EAAQ8lB,YAAuBv0B,KAAK20B,KAAKC,SAAStR,OAAO9Q,OAASxS,KAAK20B,KAAKC,SAASjtB,IAAI6K,QAC9FxM,EAAMyiC,eAAiB,EACvBziC,EAAM4iC,gBAAkB5iC,EAAM0iC,gBAAkB1iC,EAAMuiC,iBACtDviC,EAAM2iC,eAAiB,CAGvB,IAAIyQ,GAAwB3M,EAAW4M,YACnCC,EAAwBntC,EAAWktC,WAsBvC,OArBA5M,GAAW5iC,YAAc4iC,EAAW5iC,WAAWsH,YAAYs7B,GAC3DtgC,EAAWtC,YAAcsC,EAAWtC,WAAWsH,YAAYhF,GAE3DsgC,EAAWx/B,MAAMuF,OAASxS,KAAKgG,MAAMwM,OAAS,KAE9CxS,KAAKu5C,iBAGDH,EACFtU,EAAOlzB,aAAa66B,EAAY2M,GAGhCtU,EAAOrzB,YAAYg7B,GAEjB6M,EACFt5C,KAAK20B,KAAK5E,IAAIgV,mBAAmBnzB,aAAazF,EAAYmtC,GAG1Dt5C,KAAK20B,KAAK5E,IAAIgV,mBAAmBtzB,YAAYtF,GAGxCnM,KAAKwkC,cAAgB2U,GAO9Bl2C,EAASkQ,UAAUomC,eAAiB,WAClC,GAAIhlB,GAAcv0B,KAAKyO,QAAQ8lB,YAG3B3kB,EAAQjP,EAAKkG,QAAQ7G,KAAK20B,KAAKc,MAAM7lB,MAAO,UAC5CC,EAAMlP,EAAKkG,QAAQ7G,KAAK20B,KAAKc,MAAM5lB,IAAK,UACxC2pC,EAAgBx5C,KAAK20B,KAAKh0B,KAAK00B,OAA2C,GAAnCr1B,KAAKgG,MAAM+jC,gBAAkB,KAAS/iC,UAC7Eo0B,EAAcoe,EAAgB73C,EAASk5B,wBAAwB76B,KAAK20B,KAAKI,YAAa/0B,KAAK20B,KAAKc,MAAO+jB,EAC3Gpe,IAAep7B,KAAK20B,KAAKh0B,KAAK00B,OAAO,GAAGruB,SAExC,IAAImhB,GAAO,GAAIpmB,GAAS,GAAIuC,MAAKsL,GAAQ,GAAItL,MAAKuL,GAAMurB,EAAap7B,KAAK20B,KAAKI,YAC3E/0B,MAAKyO,QAAQ6yB,QACfnZ,EAAK4Z,UAAU/hC,KAAKyO,QAAQ6yB,QAE9BthC,KAAKmoB,KAAOA,CAKZ,IAAI4H,GAAM/vB,KAAK+vB,GACfA,GAAI/e,UAAUw1B,MAAQzW,EAAIyW,MAC1BzW,EAAI/e,UAAUgoC,WAAajpB,EAAIipB,WAC/BjpB,EAAI/e,UAAUioC,WAAalpB,EAAIkpB,WAC/BlpB,EAAIyW,SACJzW,EAAIipB,cACJjpB,EAAIkpB,aAEJ,IAAIQ,GAEAvc,EAGAwc,EAGA7xC,EAPAkK,EAAI,EAEJ4nC,EAAQ,EACRpnC,EAAQ,EAERqnC,EAAmBpzC,OACnBkG,EAAM,CAIV,KADAyb,EAAK8Z,QACE9Z,EAAK0U,WAAmB,IAANnwB,GACvBA,IAEA+sC,EAAMtxB,EAAKC,aACX8U,EAAU/U,EAAK+U,UACfr1B,EAAYsgB,EAAK2b,eAEjB6V,EAAQ5nC,EACRA,EAAI/R,KAAK20B,KAAKh0B,KAAKs0B,SAASwkB,GAC5BlnC,EAAQR,EAAI4nC,EACRD,IACFA,EAASzsC,MAAMsF,MAAQA,EAAQ,MAG7BvS,KAAKyO,QAAQq3B,iBACf9lC,KAAK65C,kBAAkB9nC,EAAGoW,EAAKyb,gBAAiBrP,EAAa1sB,GAG3Dq1B,GAAWl9B,KAAKyO,QAAQs3B,iBACtBh0B,EAAI,IACkBvL,QAApBozC,IACFA,EAAmB7nC,GAErB/R,KAAK85C,kBAAkB/nC,EAAGoW,EAAK0b,gBAAiBtP,EAAa1sB,IAE/D6xC,EAAW15C,KAAK+5C,kBAAkBhoC,EAAGwiB,EAAa1sB,IAGlD6xC,EAAW15C,KAAKg6C,kBAAkBjoC,EAAGwiB,EAAa1sB,GAGpDsgB,EAAKE,MAIP,IAAIroB,KAAKyO,QAAQs3B,gBAAiB,CAChC,GAAIkU,GAAWj6C,KAAK20B,KAAKh0B,KAAK00B,OAAO,GACjC6kB,EAAW/xB,EAAK0b,cAAcoW,GAC9BE,EAAYD,EAASv0C,QAAU3F,KAAKgG,MAAM8jC,gBAAkB,IAAM,IAE9CtjC,QAApBozC,GAA6CA,EAAZO,IACnCn6C,KAAK85C,kBAAkB,EAAGI,EAAU3lB,EAAa1sB,GAKrDlH,EAAK0H,QAAQrI,KAAK+vB,IAAI/e,UAAW,SAAUopC,GACzC,KAAOA,EAAIz0C,QAAQ,CACjB,GAAI4B,GAAO6yC,EAAIC,KACX9yC,IAAQA,EAAKsC,YACftC,EAAKsC,WAAWsH,YAAY5J,OAcpCtE,EAASkQ,UAAU0mC,kBAAoB,SAAU9nC,EAAGwX,EAAMgL,EAAa1sB,GAErE,GAAI4gB,GAAQzoB,KAAK+vB,IAAI/e,UAAUioC,WAAW3nC,OAE1C,KAAKmX,EAAO,CAEV,GAAImH,GAAUre,SAAS04B,eAAe,GACtCxhB,GAAQlX,SAASM,cAAc,OAC/B4W,EAAMhX,YAAYme,GAClB5vB,KAAK+vB,IAAI0c,WAAWh7B,YAAYgX,GAElCzoB,KAAK+vB,IAAIkpB,WAAWjxC,KAAKygB,GAEzBA,EAAM6xB,WAAW,GAAGC,UAAYhxB,EAEhCd,EAAMxb,MAAMtF,IAAsB,OAAf4sB,EAAyBv0B,KAAKgG,MAAMuiC,iBAAmB,KAAQ,IAClF9f,EAAMxb,MAAMxF,KAAOsK,EAAI,KACvB0W,EAAM5gB,UAAY,cAAgBA,GAYpC5E,EAASkQ,UAAU2mC,kBAAoB,SAAU/nC,EAAGwX,EAAMgL,EAAa1sB,GAErE,GAAI4gB,GAAQzoB,KAAK+vB,IAAI/e,UAAUgoC,WAAW1nC,OAE1C,KAAKmX,EAAO,CAEV,GAAImH,GAAUre,SAAS04B,eAAe1gB,EACtCd,GAAQlX,SAASM,cAAc,OAC/B4W,EAAMhX,YAAYme,GAClB5vB,KAAK+vB,IAAI0c,WAAWh7B,YAAYgX,GAElCzoB,KAAK+vB,IAAIipB,WAAWhxC,KAAKygB,GAEzBA,EAAM6xB,WAAW,GAAGC,UAAYhxB,EAChCd,EAAM5gB,UAAY,cAAgBA,EAGlC4gB,EAAMxb,MAAMtF,IAAsB,OAAf4sB,EAAwB,IAAOv0B,KAAKgG,MAAMqiC,iBAAoB,KACjF5f,EAAMxb,MAAMxF,KAAOsK,EAAI,MAWzB9O,EAASkQ,UAAU6mC,kBAAoB,SAAUjoC,EAAGwiB,EAAa1sB,GAE/D,GAAIgoB,GAAO7vB,KAAK+vB,IAAI/e,UAAUw1B,MAAMl1B,OAC/Bue,KAEHA,EAAOte,SAASM,cAAc,OAC9B7R,KAAK+vB,IAAI5jB,WAAWsF,YAAYoe,IAElC7vB,KAAK+vB,IAAIyW,MAAMx+B,KAAK6nB,EAEpB,IAAI7pB,GAAQhG,KAAKgG,KAYjB,OAVE6pB,GAAK5iB,MAAMtF,IADM,OAAf4sB,EACevuB,EAAMuiC,iBAAmB,KAGzBvoC,KAAK20B,KAAKC,SAASjtB,IAAI6K,OAAS,KAEnDqd,EAAK5iB,MAAMuF,OAASxM,EAAM0iC,gBAAkB,KAC5C7Y,EAAK5iB,MAAMxF,KAAQsK,EAAI/L,EAAMyiC,eAAiB,EAAK,KAEnD5Y,EAAKhoB,UAAY,uBAAyBA,EAEnCgoB,GAWT5sB,EAASkQ,UAAU4mC,kBAAoB,SAAUhoC,EAAGwiB,EAAa1sB,GAE/D,GAAIgoB,GAAO7vB,KAAK+vB,IAAI/e,UAAUw1B,MAAMl1B,OAC/Bue,KAEHA,EAAOte,SAASM,cAAc,OAC9B7R,KAAK+vB,IAAI5jB,WAAWsF,YAAYoe,IAElC7vB,KAAK+vB,IAAIyW,MAAMx+B,KAAK6nB,EAEpB,IAAI7pB,GAAQhG,KAAKgG,KAYjB,OAVE6pB,GAAK5iB,MAAMtF,IADM,OAAf4sB,EACe,IAGAv0B,KAAK20B,KAAKC,SAASjtB,IAAI6K,OAAS,KAEnDqd,EAAK5iB,MAAMxF,KAAQsK,EAAI/L,EAAM2iC,eAAiB,EAAK,KACnD9Y,EAAK5iB,MAAMuF,OAASxM,EAAM4iC,gBAAkB,KAE5C/Y,EAAKhoB,UAAY,uBAAyBA,EAEnCgoB,GAQT5sB,EAASkQ,UAAUi1B,mBAAqB,WAKjCpoC,KAAK+vB,IAAIma,mBACZlqC,KAAK+vB,IAAIma,iBAAmB34B,SAASM,cAAc,OACnD7R,KAAK+vB,IAAIma,iBAAiBriC,UAAY,qBACtC7H,KAAK+vB,IAAIma,iBAAiBj9B,MAAM2W,SAAW,WAE3C5jB,KAAK+vB,IAAIma,iBAAiBz4B,YAAYF,SAAS04B,eAAe,MAC9DjqC,KAAK+vB,IAAI0c,WAAWh7B,YAAYzR,KAAK+vB,IAAIma,mBAE3ClqC,KAAKgG,MAAMsiC,gBAAkBtoC,KAAK+vB,IAAIma,iBAAiBrlB,aACvD7kB,KAAKgG,MAAM+jC,eAAiB/pC,KAAK+vB,IAAIma,iBAAiB1qB,YAGjDxf,KAAK+vB,IAAIqa,mBACZpqC,KAAK+vB,IAAIqa,iBAAmB74B,SAASM,cAAc,OACnD7R,KAAK+vB,IAAIqa,iBAAiBviC,UAAY,qBACtC7H,KAAK+vB,IAAIqa,iBAAiBn9B,MAAM2W,SAAW,WAE3C5jB,KAAK+vB,IAAIqa,iBAAiB34B,YAAYF,SAAS04B,eAAe,MAC9DjqC,KAAK+vB,IAAI0c,WAAWh7B,YAAYzR,KAAK+vB,IAAIqa,mBAE3CpqC,KAAKgG,MAAMwiC,gBAAkBxoC,KAAK+vB,IAAIqa,iBAAiBvlB,aACvD7kB,KAAKgG,MAAM8jC,eAAiB9pC,KAAK+vB,IAAIqa,iBAAiB5qB,aASxDvc,EAASkQ,UAAU6hB,KAAO,SAASyD,GACjC,MAAOz4B,MAAKmoB,KAAK6M,KAAKyD,IAGxB54B,EAAOD,QAAUqD,GAKb,SAASpD,EAAQD,EAASM,GAc9B,QAASgC,GAAMwQ,EAAM0nB,EAAY3rB,GAC/BzO,KAAKK,GAAK,KACVL,KAAK8kC,OAAS,KACd9kC,KAAK0S,KAAOA,EACZ1S,KAAK+vB,IAAM,KACX/vB,KAAKo6B,WAAaA,MAClBp6B,KAAKyO,QAAUA,MAEfzO,KAAKqzC,UAAW,EAChBrzC,KAAKktC,WAAY,EACjBltC,KAAKitC,OAAQ,EAEbjtC,KAAK2H,IAAM,KACX3H,KAAKyH,KAAO,KACZzH,KAAKuS,MAAQ,KACbvS,KAAKwS,OAAS,KA3BhB,GAAIzO,GAAS7D,EAAoB,IAC7BS,EAAOT,EAAoB,EA6B/BgC,GAAKiR,UAAUrR,OAAQ,EAKvBI,EAAKiR,UAAUg+B,OAAS,WACtBnxC,KAAKqzC,UAAW,EAChBrzC,KAAKitC,OAAQ,EACTjtC,KAAKktC,WAAWltC,KAAKyhB,UAM3Bvf,EAAKiR,UAAU+9B,SAAW,WACxBlxC,KAAKqzC,UAAW,EAChBrzC,KAAKitC,OAAQ,EACTjtC,KAAKktC,WAAWltC,KAAKyhB,UAQ3Bvf,EAAKiR,UAAU6E,QAAU,SAAStF,GAChC1S,KAAK0S,KAAOA,EACZ1S,KAAKitC,OAAQ,EACTjtC,KAAKktC,WAAWltC,KAAKyhB,UAO3Bvf,EAAKiR,UAAUu6B,UAAY,SAAS5I,GAC9B9kC,KAAKktC,WACPltC,KAAK0nC,OACL1nC,KAAK8kC,OAASA,EACV9kC,KAAK8kC,QACP9kC,KAAK2nC,QAIP3nC,KAAK8kC,OAASA,GASlB5iC,EAAKiR,UAAU27B,UAAY,WAEzB,OAAO,GAOT5sC,EAAKiR,UAAUw0B,KAAO,WACpB,OAAO,GAOTzlC,EAAKiR,UAAUu0B,KAAO,WACpB,OAAO,GAMTxlC,EAAKiR,UAAUsO,OAAS,aAOxBvf,EAAKiR,UAAUw7B,YAAc,aAO7BzsC,EAAKiR,UAAUo6B,YAAc,aAS7BrrC,EAAKiR,UAAUqnC,qBAAuB,SAAUC,GAC9C,GAAIz6C,KAAKqzC,UAAYrzC,KAAKyO,QAAQygC,SAAS74B,SAAWrW,KAAK+vB,IAAI2qB,aAAc,CAE3E,GAAIvmC,GAAKnU,KAEL06C,EAAenpC,SAASM,cAAc,MAC1C6oC,GAAa7yC,UAAY,SACzB6yC,EAAa1V,MAAQ,mBAGrB,GAAIjhC,GAAO22C,GAAcnnC,GAAG,MAAO,SAAUjK,GAC3C6K,EAAG2wB,OAAOiJ,kBAAkB55B,GAC5B7K,EAAMq8B,kBACNr8B,EAAMD,mBAGRoxC,EAAOhpC,YAAYipC,GACnB16C,KAAK+vB,IAAI2qB,aAAeA,OAEhB16C,KAAKqzC,UAAYrzC,KAAK+vB,IAAI2qB,eAE9B16C,KAAK+vB,IAAI2qB,aAAa7wC,YACxB7J,KAAK+vB,IAAI2qB,aAAa7wC,WAAWsH,YAAYnR,KAAK+vB,IAAI2qB,cAExD16C,KAAK+vB,IAAI2qB,aAAe,OAS5Bx4C,EAAKiR,UAAUwnC,gBAAkB,SAAU/xC,GACzC,GAAIgnB,EACJ,IAAI5vB,KAAKyO,QAAQmsC,SAAU,CACzB,GAAI/jB,GAAW72B,KAAK8kC,OAAOjP,QAAQC,UAAU5gB,IAAIlV,KAAKK,GACtDuvB,GAAU5vB,KAAKyO,QAAQmsC,SAAS/jB,OAGhCjH,GAAU5vB,KAAK0S,KAAKkd,OAGtB,IAAGA,IAAY5vB,KAAK4vB,QAAS,CAE3B,GAAIA,YAAmB+c,SACrB/jC,EAAQqb,UAAY,GACpBrb,EAAQ6I,YAAYme,OAEjB,IAAeppB,QAAXopB,EACPhnB,EAAQqb,UAAY2L,MAGpB,IAAwB,cAAlB5vB,KAAK0S,KAAK5L,MAA8CN,SAAtBxG,KAAK0S,KAAKkd,QAChD,KAAM,IAAIhsB,OAAM,sCAAwC5D,KAAKK,GAIjEL,MAAK4vB,QAAUA,IASnB1tB,EAAKiR,UAAU0nC,aAAe,SAAUjyC,GACf,MAAnB5I,KAAK0S,KAAKsyB,MACZp8B,EAAQo8B,MAAQhlC,KAAK0S,KAAKsyB,OAAS,GAGnCp8B,EAAQkyC,gBAAgB,UAS3B54C,EAAKiR,UAAU4nC,sBAAwB,SAASnyC,GAC/C,GAAI5I,KAAKyO,QAAQusC,gBAAkBh7C,KAAKyO,QAAQusC,eAAer1C,OAAS,EAAG,CACzE,GAAIs1C,KAEJ,IAAIh1C,MAAMC,QAAQlG,KAAKyO,QAAQusC,gBAC7BC,EAAaj7C,KAAKyO,QAAQusC,mBAEvB,CAAA,GAAmC,OAA/Bh7C,KAAKyO,QAAQusC,eAIpB,MAHAC,GAAa10C,OAAO6G,KAAKpN,KAAK0S,MAMhC,IAAK,GAAIlN,GAAI,EAAGA,EAAIy1C,EAAWt1C,OAAQH,IAAK,CAC1C,GAAIyQ,GAAOglC,EAAWz1C,GAClB6B,EAAQrH,KAAK0S,KAAKuD,EAET,OAAT5O,EACFuB,EAAQsyC,aAAa,QAAUjlC,EAAM5O,GAGrCuB,EAAQkyC,gBAAgB,QAAU7kC,MAW1C/T,EAAKiR,UAAUgoC,aAAe,SAASvyC,GAEjC5I,KAAKiN,QACPtM,EAAK8M,cAAc7E,EAAS5I,KAAKiN,OACjCjN,KAAKiN,MAAQ,MAIXjN,KAAK0S,KAAKzF,QACZtM,EAAK2M,WAAW1E,EAAS5I,KAAK0S,KAAKzF,OACnCjN,KAAKiN,MAAQjN,KAAK0S,KAAKzF,QAI3BpN,EAAOD,QAAUsC,GAKb,SAASrC,EAAQD,EAASM,GAkB9B,QAASiC,GAAgBuQ,EAAM0nB,EAAY3rB,GASzC,GARAzO,KAAKgG,OACH4pB,SACErd,MAAO,IAGXvS,KAAK6jB,UAAW,EAGZnR,EAAM,CACR,GAAkBlM,QAAdkM,EAAK9C,MACP,KAAM,IAAIhM,OAAM,oCAAsC8O,EAAKrS,GAE7D,IAAgBmG,QAAZkM,EAAK7C,IACP,KAAM,IAAIjM,OAAM,kCAAoC8O,EAAKrS,IAI7D6B,EAAK3B,KAAKP,KAAM0S,EAAM0nB,EAAY3rB,GAElCzO,KAAKo7C,cAAe,EApCtB,GACIl5C,IADShC,EAAoB,IACtBA,EAAoB,KAC3B2C,EAAkB3C,EAAoB,IACtCoC,EAAYpC,EAAoB,GAoCpCiC,GAAegR,UAAY,GAAIjR,GAAM,KAAM,KAAM,MAEjDC,EAAegR,UAAUkoC,cAAgB,kBACzCl5C,EAAegR,UAAUrR,OAAQ,EAOjCK,EAAegR,UAAU27B,UAAY,SAASrZ,GAE5C,MAAQz1B,MAAK0S,KAAK9C,MAAQ6lB,EAAM5lB,KAAS7P,KAAK0S,KAAK7C,IAAM4lB,EAAM7lB,OAMjEzN,EAAegR,UAAUsO,OAAS,WAChC,GAAIsO,GAAM/vB,KAAK+vB,GAuBf,IAtBKA,IAEH/vB,KAAK+vB,OACLA,EAAM/vB,KAAK+vB,IAGXA,EAAIygB,IAAMj/B,SAASM,cAAc,OAIjCke,EAAIH,QAAUre,SAASM,cAAc,OACrCke,EAAIH,QAAQ/nB,UAAY,UACxBkoB,EAAIygB,IAAI/+B,YAAYse,EAAIH,SAMxB5vB,KAAKitC,OAAQ,IAIVjtC,KAAK8kC,OACR,KAAM,IAAIlhC,OAAM,yCAElB,KAAKmsB,EAAIygB,IAAI3mC,WAAY,CACvB,GAAIsC,GAAanM,KAAK8kC,OAAO/U,IAAI5jB,UACjC,KAAKA,EACH,KAAM,IAAIvI,OAAM,iEAElBuI,GAAWsF,YAAYse,EAAIygB,KAQ7B,GANAxwC,KAAKktC,WAAY,EAMbltC,KAAKitC,MAAO,CACdjtC,KAAK26C,gBAAgB36C,KAAK+vB,IAAIH,SAC9B5vB,KAAK66C,aAAa76C,KAAK+vB,IAAIH,SAC3B5vB,KAAK+6C,sBAAsB/6C,KAAK+vB,IAAIH,SACpC5vB,KAAKm7C,aAAan7C,KAAK+vB,IAAIygB,IAG3B,IAAI3oC,IAAa7H,KAAK0S,KAAK7K,UAAa,IAAM7H,KAAK0S,KAAK7K,UAAa,KAChE7H,KAAKqzC,SAAW,YAAc,GACnCtjB,GAAIygB,IAAI3oC,UAAY7H,KAAKq7C,cAAgBxzC,EAGzC7H,KAAK6jB,SAA6D,WAAlDta,OAAOsrC,iBAAiB9kB,EAAIH,SAAS/L,SAGrD7jB,KAAKgG,MAAM4pB,QAAQrd,MAAQvS,KAAK+vB,IAAIH,QAAQQ,YAC5CpwB,KAAKwS,OAAS,EAEdxS,KAAKitC,OAAQ,IAQjB9qC,EAAegR,UAAUw0B,KAAOrlC,EAAU6Q,UAAUw0B,KAMpDxlC,EAAegR,UAAUu0B,KAAOplC,EAAU6Q,UAAUu0B,KAMpDvlC,EAAegR,UAAUw7B,YAAcrsC,EAAU6Q,UAAUw7B,YAM3DxsC,EAAegR,UAAUo6B,YAAc,SAAS7zB,GAC9C,GAAI4hC,GAAqC,QAA7Bt7C,KAAKyO,QAAQ8lB,WACzBv0B,MAAK+vB,IAAIH,QAAQ3iB,MAAMtF,IAAM2zC,EAAQ,GAAK,IAC1Ct7C,KAAK+vB,IAAIH,QAAQ3iB,MAAMqW,OAASg4B,EAAQ,IAAM,EAC9C,IAAI9oC,EAGJ,IAA2BhM,SAAvBxG,KAAK0S,KAAK2uB,SAAwB,CACpC,GAAIka,GAAev7C,KAAK0S,KAAK2uB,SACzBF,EAAYnhC,KAAK8kC,OAAO3D,UACxB8K,EAAgB9K,EAAUoa,GAAcpzC,KAE5C,IAAa,GAATmzC,EAAe,CAEjB9oC,EAASxS,KAAK8kC,OAAO3D,UAAUoa,GAAc/oC,OAASkH,EAAOrK,KAAKoW,SAClEjT,GAA2B,GAAjBy5B,EAAqBvyB,EAAOonB,KAAO,GAAIpnB,EAAOrK,KAAKoW,SAAW,CACxE,IAAI2b,GAASphC,KAAK8kC,OAAOn9B,GACzB,KAAK,GAAI05B,KAAYF,GACfA,EAAUr7B,eAAeu7B,IACQ,GAA/BF,EAAUE,GAAU3Y,SAAmByY,EAAUE,GAAUl5B,MAAQ8jC,IACrE7K,GAAUD,EAAUE,GAAU7uB,OAASkH,EAAOrK,KAAKoW,SAMzD2b,IAA2B,GAAjB6K,EAAqBvyB,EAAOonB,KAAO,GAAMpnB,EAAOrK,KAAKoW,SAAW,EAC1EzlB,KAAK+vB,IAAIygB,IAAIvjC,MAAMtF,IAAMy5B,EAAS,KAClCphC,KAAK+vB,IAAIygB,IAAIvjC,MAAMqW,OAAS,OAGzB,CACH,GAAI8d,GAASphC,KAAK8kC,OAAOn9B,GACzB,KAAK,GAAI05B,KAAYF,GACfA,EAAUr7B,eAAeu7B,IACQ,GAA/BF,EAAUE,GAAU3Y,SAAmByY,EAAUE,GAAUl5B,MAAQ8jC,IACrE7K,GAAUD,EAAUE,GAAU7uB,OAASkH,EAAOrK,KAAKoW,SAIzDjT,GAASxS,KAAK8kC,OAAO3D,UAAUoa,GAAc/oC,OAASkH,EAAOrK,KAAKoW,SAClEzlB,KAAK+vB,IAAIygB,IAAIvjC,MAAMtF,IAAMy5B,EAAS,KAClCphC,KAAK+vB,IAAIygB,IAAIvjC,MAAMqW,OAAS,QAM1BtjB,MAAK8kC,iBAAkBjiC,IAEzB2P,EAAStN,KAAKwH,IAAI1M,KAAK8kC,OAAOtyB,OAC1BxS,KAAK8kC,OAAOjP,QAAQlB,KAAKC,SAAS1I,OAAO1Z,OACzCxS,KAAK8kC,OAAOjP,QAAQlB,KAAKC,SAASiD,gBAAgBrlB,QACtDxS,KAAK+vB,IAAIygB,IAAIvjC,MAAMtF,IAAM2zC,EAAQ,IAAM,GACvCt7C,KAAK+vB,IAAIygB,IAAIvjC,MAAMqW,OAASg4B,EAAQ,GAAK,MAGzC9oC,EAASxS,KAAK8kC,OAAOtyB,OAErBxS,KAAK+vB,IAAIygB,IAAIvjC,MAAMtF,IAAM3H,KAAK8kC,OAAOn9B,IAAM,KAC3C3H,KAAK+vB,IAAIygB,IAAIvjC,MAAMqW,OAAS,GAGhCtjB,MAAK+vB,IAAIygB,IAAIvjC,MAAMuF,OAASA,EAAS,MAGvC3S,EAAOD,QAAUuC,GAKb,SAAStC,EAAQD,EAASM,GAe9B,QAASkC,GAASsQ,EAAM0nB,EAAY3rB,GAalC,GAZAzO,KAAKgG,OACH8pB,KACEvd,MAAO,EACPC,OAAQ,GAEVqd,MACEtd,MAAO,EACPC,OAAQ,IAKRE,GACgBlM,QAAdkM,EAAK9C,MACP,KAAM,IAAIhM,OAAM,oCAAsC8O,EAI1DxQ,GAAK3B,KAAKP,KAAM0S,EAAM0nB,EAAY3rB,GAhCpC,CAAA,GAAIvM,GAAOhC,EAAoB,GACpBA,GAAoB,GAkC/BkC,EAAQ+Q,UAAY,GAAIjR,GAAM,KAAM,KAAM,MAO1CE,EAAQ+Q,UAAU27B,UAAY,SAASrZ,GAGrC,GAAIjD,IAAYiD,EAAM5lB,IAAM4lB,EAAM7lB,OAAS,CAC3C,OAAQ5P,MAAK0S,KAAK9C,MAAQ6lB,EAAM7lB,MAAQ4iB,GAAcxyB,KAAK0S,KAAK9C,MAAQ6lB,EAAM5lB,IAAM2iB,GAMtFpwB,EAAQ+Q,UAAUsO,OAAS,WACzB,GAAIsO,GAAM/vB,KAAK+vB,GA6Bf,IA5BKA,IAEH/vB,KAAK+vB,OACLA,EAAM/vB,KAAK+vB,IAGXA,EAAIygB,IAAMj/B,SAASM,cAAc,OAGjCke,EAAIH,QAAUre,SAASM,cAAc,OACrCke,EAAIH,QAAQ/nB,UAAY,UACxBkoB,EAAIygB,IAAI/+B,YAAYse,EAAIH,SAGxBG,EAAIF,KAAOte,SAASM,cAAc,OAClCke,EAAIF,KAAKhoB,UAAY,OAGrBkoB,EAAID,IAAMve,SAASM,cAAc,OACjCke,EAAID,IAAIjoB,UAAY,MAGpBkoB,EAAIygB,IAAI,iBAAmBxwC,KAE3BA,KAAKitC,OAAQ,IAIVjtC,KAAK8kC,OACR,KAAM,IAAIlhC,OAAM,yCAElB,KAAKmsB,EAAIygB,IAAI3mC,WAAY,CACvB,GAAI4iC,GAAazsC,KAAK8kC,OAAO/U,IAAI0c,UACjC,KAAKA,EAAY,KAAM,IAAI7oC,OAAM,iEACjC6oC,GAAWh7B,YAAYse,EAAIygB,KAE7B,IAAKzgB,EAAIF,KAAKhmB,WAAY,CACxB,GAAIsC,GAAanM,KAAK8kC,OAAO/U,IAAI5jB,UACjC,KAAKA,EAAY,KAAM,IAAIvI,OAAM,iEACjCuI,GAAWsF,YAAYse,EAAIF,MAE7B,IAAKE,EAAID,IAAIjmB,WAAY,CACvB,GAAIi3B,GAAO9gC,KAAK8kC,OAAO/U,IAAI+Q,IAC3B,KAAK30B,EAAY,KAAM,IAAIvI,OAAM,2DACjCk9B,GAAKrvB,YAAYse,EAAID,KAQvB,GANA9vB,KAAKktC,WAAY,EAMbltC,KAAKitC,MAAO,CACdjtC,KAAK26C,gBAAgB36C,KAAK+vB,IAAIH,SAC9B5vB,KAAK66C,aAAa76C,KAAK+vB,IAAIygB,KAC3BxwC,KAAK+6C,sBAAsB/6C,KAAK+vB,IAAIygB,KACpCxwC,KAAKm7C,aAAan7C,KAAK+vB,IAAIygB,IAG3B,IAAI3oC,IAAa7H,KAAK0S,KAAK7K,UAAW,IAAM7H,KAAK0S,KAAK7K,UAAY,KAC7D7H,KAAKqzC,SAAW,YAAc,GACnCtjB,GAAIygB,IAAI3oC,UAAY,WAAaA,EACjCkoB,EAAIF,KAAKhoB,UAAY,YAAcA,EACnCkoB,EAAID,IAAIjoB,UAAa,WAAaA,EAGlC7H,KAAKgG,MAAM8pB,IAAItd,OAASud,EAAID,IAAIQ,aAChCtwB,KAAKgG,MAAM8pB,IAAIvd,MAAQwd,EAAID,IAAIM,YAC/BpwB,KAAKgG,MAAM6pB,KAAKtd,MAAQwd,EAAIF,KAAKO,YACjCpwB,KAAKuS,MAAQwd,EAAIygB,IAAIpgB,YACrBpwB,KAAKwS,OAASud,EAAIygB,IAAIlgB,aAEtBtwB,KAAKitC,OAAQ,EAGfjtC,KAAKw6C,qBAAqBzqB,EAAIygB,MAOhCpuC,EAAQ+Q,UAAUw0B,KAAO,WAClB3nC,KAAKktC,WACRltC,KAAKyhB,UAOTrf,EAAQ+Q,UAAUu0B,KAAO,WACvB,GAAI1nC,KAAKktC,UAAW,CAClB,GAAInd,GAAM/vB,KAAK+vB,GAEXA,GAAIygB,IAAI3mC,YAAckmB,EAAIygB,IAAI3mC,WAAWsH,YAAY4e,EAAIygB,KACzDzgB,EAAIF,KAAKhmB,YAAakmB,EAAIF,KAAKhmB,WAAWsH,YAAY4e,EAAIF,MAC1DE,EAAID,IAAIjmB,YAAckmB,EAAID,IAAIjmB,WAAWsH,YAAY4e,EAAID,KAE7D9vB,KAAK2H,IAAM,KACX3H,KAAKyH,KAAO,KAEZzH,KAAKktC,WAAY,IAQrB9qC,EAAQ+Q,UAAUw7B,YAAc,WAC9B,GAAI/+B,GAAQ5P,KAAKo6B,WAAWnF,SAASj1B,KAAK0S,KAAK9C,OAC3Cm/B,EAAQ/uC,KAAKyO,QAAQsgC,MAErByB,EAAMxwC,KAAK+vB,IAAIygB,IACf3gB,EAAO7vB,KAAK+vB,IAAIF,KAChBC,EAAM9vB,KAAK+vB,IAAID,GAIjB9vB,MAAKyH,KADM,SAATsnC,EACUn/B,EAAQ5P,KAAKuS,MAET,QAATw8B,EACKn/B,EAIAA,EAAQ5P,KAAKuS,MAAQ,EAInCi+B,EAAIvjC,MAAMxF,KAAOzH,KAAKyH,KAAO,KAG7BooB,EAAK5iB,MAAMxF,KAAQmI,EAAQ5P,KAAKgG,MAAM6pB,KAAKtd,MAAQ,EAAK,KAGxDud,EAAI7iB,MAAMxF,KAAQmI,EAAQ5P,KAAKgG,MAAM8pB,IAAIvd,MAAQ,EAAK,MAOxDnQ,EAAQ+Q,UAAUo6B,YAAc,WAC9B,GAAIhZ,GAAcv0B,KAAKyO,QAAQ8lB,YAC3Bic,EAAMxwC,KAAK+vB,IAAIygB,IACf3gB,EAAO7vB,KAAK+vB,IAAIF,KAChBC,EAAM9vB,KAAK+vB,IAAID,GAEnB,IAAmB,OAAfyE,EACFic,EAAIvjC,MAAMtF,KAAW3H,KAAK2H,KAAO,GAAK,KAEtCkoB,EAAK5iB,MAAMtF,IAAS,IACpBkoB,EAAK5iB,MAAMuF,OAAUxS,KAAK8kC,OAAOn9B,IAAM3H,KAAK2H,IAAM,EAAK,KACvDkoB,EAAK5iB,MAAMqW,OAAS,OAEjB,CACH,GAAIk4B,GAAgBx7C,KAAK8kC,OAAOjP,QAAQ7vB,MAAMwM,OAC1C+d,EAAairB,EAAgBx7C,KAAK8kC,OAAOn9B,IAAM3H,KAAK8kC,OAAOtyB,OAASxS,KAAK2H,GAE7E6oC,GAAIvjC,MAAMtF,KAAW3H,KAAK8kC,OAAOtyB,OAASxS,KAAK2H,IAAM3H,KAAKwS,QAAU,GAAK,KACzEqd,EAAK5iB,MAAMtF,IAAU6zC,EAAgBjrB,EAAc,KACnDV,EAAK5iB,MAAMqW,OAAS,IAGtBwM,EAAI7iB,MAAMtF,KAAQ3H,KAAKgG,MAAM8pB,IAAItd,OAAS,EAAK,MAGjD3S,EAAOD,QAAUwC,GAKb,SAASvC,EAAQD,EAASM,GAc9B,QAASmC,GAAWqQ,EAAM0nB,EAAY3rB,GAcpC,GAbAzO,KAAKgG,OACH8pB,KACEnoB,IAAK,EACL4K,MAAO,EACPC,OAAQ,GAEVod,SACEpd,OAAQ,EACRipC,WAAY,IAKZ/oC,GACgBlM,QAAdkM,EAAK9C,MACP,KAAM,IAAIhM,OAAM,oCAAsC8O,EAI1DxQ,GAAK3B,KAAKP,KAAM0S,EAAM0nB,EAAY3rB,GAhCpC,GAAIvM,GAAOhC,EAAoB,GAmC/BmC,GAAU8Q,UAAY,GAAIjR,GAAM,KAAM,KAAM,MAO5CG,EAAU8Q,UAAU27B,UAAY,SAASrZ,GAGvC,GAAIjD,IAAYiD,EAAM5lB,IAAM4lB,EAAM7lB,OAAS,CAC3C,OAAQ5P,MAAK0S,KAAK9C,MAAQ6lB,EAAM7lB,MAAQ4iB,GAAcxyB,KAAK0S,KAAK9C,MAAQ6lB,EAAM5lB,IAAM2iB,GAMtFnwB,EAAU8Q,UAAUsO,OAAS,WAC3B,GAAIsO,GAAM/vB,KAAK+vB,GA0Bf,IAzBKA,IAEH/vB,KAAK+vB,OACLA,EAAM/vB,KAAK+vB,IAGXA,EAAI7d,MAAQX,SAASM,cAAc,OAInCke,EAAIH,QAAUre,SAASM,cAAc,OACrCke,EAAIH,QAAQ/nB,UAAY,UACxBkoB,EAAI7d,MAAMT,YAAYse,EAAIH,SAG1BG,EAAID,IAAMve,SAASM,cAAc,OACjCke,EAAI7d,MAAMT,YAAYse,EAAID,KAG1BC,EAAI7d,MAAM,iBAAmBlS,KAE7BA,KAAKitC,OAAQ,IAIVjtC,KAAK8kC,OACR,KAAM,IAAIlhC,OAAM,yCAElB,KAAKmsB,EAAI7d,MAAMrI,WAAY,CACzB,GAAI4iC,GAAazsC,KAAK8kC,OAAO/U,IAAI0c,UACjC,KAAKA,EACH,KAAM,IAAI7oC,OAAM,iEAElB6oC,GAAWh7B,YAAYse,EAAI7d,OAQ7B,GANAlS,KAAKktC,WAAY,EAMbltC,KAAKitC,MAAO,CACdjtC,KAAK26C,gBAAgB36C,KAAK+vB,IAAIH,SAC9B5vB,KAAK66C,aAAa76C,KAAK+vB,IAAI7d,OAC3BlS,KAAK+6C,sBAAsB/6C,KAAK+vB,IAAI7d,OACpClS,KAAKm7C,aAAan7C,KAAK+vB,IAAI7d,MAG3B,IAAIrK,IAAa7H,KAAK0S,KAAK7K,UAAW,IAAM7H,KAAK0S,KAAK7K,UAAY,KAC7D7H,KAAKqzC,SAAW,YAAc,GACnCtjB,GAAI7d,MAAMrK,UAAa,aAAeA,EACtCkoB,EAAID,IAAIjoB,UAAa,WAAaA,EAGlC7H,KAAKuS,MAAQwd,EAAI7d,MAAMke,YACvBpwB,KAAKwS,OAASud,EAAI7d,MAAMoe,aACxBtwB,KAAKgG,MAAM8pB,IAAIvd,MAAQwd,EAAID,IAAIM,YAC/BpwB,KAAKgG,MAAM8pB,IAAItd,OAASud,EAAID,IAAIQ,aAChCtwB,KAAKgG,MAAM4pB,QAAQpd,OAASud,EAAIH,QAAQU,aAGxCP,EAAIH,QAAQ3iB,MAAMwuC,WAAa,EAAIz7C,KAAKgG,MAAM8pB,IAAIvd,MAAQ,KAG1Dwd,EAAID,IAAI7iB,MAAMtF,KAAQ3H,KAAKwS,OAASxS,KAAKgG,MAAM8pB,IAAItd,QAAU,EAAK,KAClEud,EAAID,IAAI7iB,MAAMxF,KAAQzH,KAAKgG,MAAM8pB,IAAIvd,MAAQ,EAAK,KAElDvS,KAAKitC,OAAQ,EAGfjtC,KAAKw6C,qBAAqBzqB,EAAI7d,QAOhC7P,EAAU8Q,UAAUw0B,KAAO,WACpB3nC,KAAKktC,WACRltC,KAAKyhB,UAOTpf,EAAU8Q,UAAUu0B,KAAO,WACrB1nC,KAAKktC,YACHltC,KAAK+vB,IAAI7d,MAAMrI,YACjB7J,KAAK+vB,IAAI7d,MAAMrI,WAAWsH,YAAYnR,KAAK+vB,IAAI7d,OAGjDlS,KAAK2H,IAAM,KACX3H,KAAKyH,KAAO,KAEZzH,KAAKktC,WAAY,IAQrB7qC,EAAU8Q,UAAUw7B,YAAc,WAChC,GAAI/+B,GAAQ5P,KAAKo6B,WAAWnF,SAASj1B,KAAK0S,KAAK9C,MAE/C5P,MAAKyH,KAAOmI,EAAQ5P,KAAKgG,MAAM8pB,IAAIvd,MAGnCvS,KAAK+vB,IAAI7d,MAAMjF,MAAMxF,KAAOzH,KAAKyH,KAAO,MAO1CpF,EAAU8Q,UAAUo6B,YAAc,WAChC,GAAIhZ,GAAcv0B,KAAKyO,QAAQ8lB,YAC3BriB,EAAQlS,KAAK+vB,IAAI7d,KAGnBA,GAAMjF,MAAMtF,IADK,OAAf4sB,EACgBv0B,KAAK2H,IAAM,KAGV3H,KAAK8kC,OAAOtyB,OAASxS,KAAK2H,IAAM3H,KAAKwS,OAAU,MAItE3S,EAAOD,QAAUyC,GAKb,SAASxC,EAAQD,EAASM,GAe9B,QAASoC,GAAWoQ,EAAM0nB,EAAY3rB,GASpC,GARAzO,KAAKgG,OACH4pB,SACErd,MAAO,IAGXvS,KAAK6jB,UAAW,EAGZnR,EAAM,CACR,GAAkBlM,QAAdkM,EAAK9C,MACP,KAAM,IAAIhM,OAAM,oCAAsC8O,EAAKrS,GAE7D,IAAgBmG,QAAZkM,EAAK7C,IACP,KAAM,IAAIjM,OAAM,kCAAoC8O,EAAKrS,IAI7D6B,EAAK3B,KAAKP,KAAM0S,EAAM0nB,EAAY3rB,GA/BpC,GACIvM,IADShC,EAAoB,IACtBA,EAAoB,IAiC/BoC,GAAU6Q,UAAY,GAAIjR,GAAM,KAAM,KAAM,MAE5CI,EAAU6Q,UAAUkoC,cAAgB,aAOpC/4C,EAAU6Q,UAAU27B,UAAY,SAASrZ,GAEvC,MAAQz1B,MAAK0S,KAAK9C,MAAQ6lB,EAAM5lB,KAAS7P,KAAK0S,KAAK7C,IAAM4lB,EAAM7lB,OAMjEtN,EAAU6Q,UAAUsO,OAAS,WAC3B,GAAIsO,GAAM/vB,KAAK+vB,GAsBf,IArBKA,IAEH/vB,KAAK+vB,OACLA,EAAM/vB,KAAK+vB,IAGXA,EAAIygB,IAAMj/B,SAASM,cAAc,OAIjCke,EAAIH,QAAUre,SAASM,cAAc,OACrCke,EAAIH,QAAQ/nB,UAAY,UACxBkoB,EAAIygB,IAAI/+B,YAAYse,EAAIH,SAGxBG,EAAIygB,IAAI,iBAAmBxwC,KAE3BA,KAAKitC,OAAQ,IAIVjtC,KAAK8kC,OACR,KAAM,IAAIlhC,OAAM,yCAElB,KAAKmsB,EAAIygB,IAAI3mC,WAAY,CACvB,GAAI4iC,GAAazsC,KAAK8kC,OAAO/U,IAAI0c,UACjC,KAAKA,EACH,KAAM,IAAI7oC,OAAM,iEAElB6oC,GAAWh7B,YAAYse,EAAIygB,KAQ7B,GANAxwC,KAAKktC,WAAY,EAMbltC,KAAKitC,MAAO,CACdjtC,KAAK26C,gBAAgB36C,KAAK+vB,IAAIH,SAC9B5vB,KAAK66C,aAAa76C,KAAK+vB,IAAIygB,KAC3BxwC,KAAK+6C,sBAAsB/6C,KAAK+vB,IAAIygB,KACpCxwC,KAAKm7C,aAAan7C,KAAK+vB,IAAIygB,IAG3B,IAAI3oC,IAAa7H,KAAK0S,KAAK7K,UAAa,IAAM7H,KAAK0S,KAAK7K,UAAa,KAChE7H,KAAKqzC,SAAW,YAAc,GACnCtjB,GAAIygB,IAAI3oC,UAAY7H,KAAKq7C,cAAgBxzC,EAGzC7H,KAAK6jB,SAA6D,WAAlDta,OAAOsrC,iBAAiB9kB,EAAIH,SAAS/L,SAKrD7jB,KAAK+vB,IAAIH,QAAQ3iB,MAAMyuC,SAAW,OAClC17C,KAAKgG,MAAM4pB,QAAQrd,MAAQvS,KAAK+vB,IAAIH,QAAQQ,YAC5CpwB,KAAKwS,OAASxS,KAAK+vB,IAAIygB,IAAIlgB,aAC3BtwB,KAAK+vB,IAAIH,QAAQ3iB,MAAMyuC,SAAW,GAElC17C,KAAKitC,OAAQ,EAGfjtC,KAAKw6C,qBAAqBzqB,EAAIygB,KAC9BxwC,KAAK27C,mBACL37C,KAAK47C,qBAOPt5C,EAAU6Q,UAAUw0B,KAAO,WACpB3nC,KAAKktC,WACRltC,KAAKyhB,UAQTnf,EAAU6Q,UAAUu0B,KAAO,WACzB,GAAI1nC,KAAKktC,UAAW,CAClB,GAAIsD,GAAMxwC,KAAK+vB,IAAIygB,GAEfA,GAAI3mC,YACN2mC,EAAI3mC,WAAWsH,YAAYq/B,GAG7BxwC,KAAK2H,IAAM,KACX3H,KAAKyH,KAAO,KAEZzH,KAAKktC,WAAY,IAQrB5qC,EAAU6Q,UAAUw7B,YAAc,WAChC,GAGIkN,GACA1rB,EAJA2rB,EAAc97C,KAAK8kC,OAAOvyB,MAC1B3C,EAAQ5P,KAAKo6B,WAAWnF,SAASj1B,KAAK0S,KAAK9C,OAC3CC,EAAM7P,KAAKo6B,WAAWnF,SAASj1B,KAAK0S,KAAK7C,MAKhCisC,EAATlsC,IACFA,GAASksC,GAEPjsC,EAAM,EAAIisC,IACZjsC,EAAM,EAAIisC,EAEZ,IAAIC,GAAW72C,KAAKwH,IAAImD,EAAMD,EAAO,EAoBrC,QAlBI5P,KAAK6jB,UACP7jB,KAAKyH,KAAOmI,EACZ5P,KAAKuS,MAAQwpC,EAAW/7C,KAAKgG,MAAM4pB,QAAQrd,MAC3C4d,EAAenwB,KAAKgG,MAAM4pB,QAAQrd,QAOlCvS,KAAKyH,KAAOmI,EACZ5P,KAAKuS,MAAQwpC,EACb5rB,EAAejrB,KAAK4G,IAAI+D,EAAMD,EAAQ,EAAI5P,KAAKyO,QAAQuV,QAAShkB,KAAKgG,MAAM4pB,QAAQrd,QAGrFvS,KAAK+vB,IAAIygB,IAAIvjC,MAAMxF,KAAOzH,KAAKyH,KAAO,KACtCzH,KAAK+vB,IAAIygB,IAAIvjC,MAAMsF,MAAQwpC,EAAW,KAE9B/7C,KAAKyO,QAAQsgC,OACnB,IAAK,OACH/uC,KAAK+vB,IAAIH,QAAQ3iB,MAAMxF,KAAO,GAC9B,MAEF,KAAK,QACHzH,KAAK+vB,IAAIH,QAAQ3iB,MAAMxF,KAAOvC,KAAKwH,IAAKqvC,EAAW5rB,EAAe,EAAInwB,KAAKyO,QAAQuV,QAAU,GAAK,IAClG,MAEF,KAAK,SACHhkB,KAAK+vB,IAAIH,QAAQ3iB,MAAMxF,KAAOvC,KAAKwH,KAAKqvC,EAAW5rB,EAAe,EAAInwB,KAAKyO,QAAQuV,SAAW,EAAG,GAAK,IACtG,MAEF,SAIM63B,EAFA77C,KAAK6jB,SACHhU,EAAM,EACM3K,KAAKwH,KAAKkD,EAAO,IAGhBugB,EAIL,EAARvgB,EACY1K,KAAK4G,KAAK8D,EACnBC,EAAMD,EAAQugB,EAAe,EAAInwB,KAAKyO,QAAQuV,SAIrC,EAGlBhkB,KAAK+vB,IAAIH,QAAQ3iB,MAAMxF,KAAOo0C,EAAc,OAQlDv5C,EAAU6Q,UAAUo6B,YAAc,WAChC,GAAIhZ,GAAcv0B,KAAKyO,QAAQ8lB,YAC3Bic,EAAMxwC,KAAK+vB,IAAIygB,GAGjBA,GAAIvjC,MAAMtF,IADO,OAAf4sB,EACcv0B,KAAK2H,IAAM,KAGV3H,KAAK8kC,OAAOtyB,OAASxS,KAAK2H,IAAM3H,KAAKwS,OAAU,MAQpElQ,EAAU6Q,UAAUwoC,iBAAmB,WACrC,GAAI37C,KAAKqzC,UAAYrzC,KAAKyO,QAAQygC,SAASC,aAAenvC,KAAK+vB,IAAIisB,SAAU,CAE3E,GAAIA,GAAWzqC,SAASM,cAAc,MACtCmqC,GAASn0C,UAAY,YACrBm0C,EAAS9I,aAAelzC,KASxBA,KAAK+vB,IAAIygB,IAAI/+B,YAAYuqC,GACzBh8C,KAAK+vB,IAAIisB,SAAWA,OAEZh8C,KAAKqzC,UAAYrzC,KAAK+vB,IAAIisB,WAE9Bh8C,KAAK+vB,IAAIisB,SAASnyC,YACpB7J,KAAK+vB,IAAIisB,SAASnyC,WAAWsH,YAAYnR,KAAK+vB,IAAIisB,UAEpDh8C,KAAK+vB,IAAIisB,SAAW,OAQxB15C,EAAU6Q,UAAUyoC,kBAAoB,WACtC,GAAI57C,KAAKqzC,UAAYrzC,KAAKyO,QAAQygC,SAASC,aAAenvC,KAAK+vB,IAAIksB,UAAW,CAE5E,GAAIA,GAAY1qC,SAASM,cAAc,MACvCoqC,GAAUp0C,UAAY,aACtBo0C,EAAU9I,cAAgBnzC,KAS1BA,KAAK+vB,IAAIygB,IAAI/+B,YAAYwqC,GACzBj8C,KAAK+vB,IAAIksB,UAAYA,OAEbj8C,KAAKqzC,UAAYrzC,KAAK+vB,IAAIksB,YAE9Bj8C,KAAK+vB,IAAIksB,UAAUpyC,YACrB7J,KAAK+vB,IAAIksB,UAAUpyC,WAAWsH,YAAYnR,KAAK+vB,IAAIksB,WAErDj8C,KAAK+vB,IAAIksB,UAAY,OAIzBp8C,EAAOD,QAAU0C,GAKb,SAASzC,EAAQD,EAASM,GAkC9B,QAASgD,GAASqW,EAAW7G,EAAMjE,GACjC,KAAMzO,eAAgBkD,IACpB,KAAM,IAAIsW,aAAY,mDAGxBxZ,MAAKk8C,0BACLl8C,KAAKm8C,0BAGLn8C,KAAKyZ,iBAAmBF,EAGxBvZ,KAAKo8C,kBAAoB,GACzBp8C,KAAKq8C,eAAiB,IAAOr8C,KAAKo8C,kBAClCp8C,KAAKs8C,WAAa,EAClBt8C,KAAKu8C,YAAc,EACnBv8C,KAAKw8C,gBAAiB,EACtBx8C,KAAKy8C,wBAA0B,GAE/Bz8C,KAAK08C,cAAe,EAEpB18C,KAAK28C,kBAAoB1pC,IAAI,KAAK2pC,KAAK,KAAKC,SAAS,KAAKC,QAAQ,KAAKC,IAAI,MAG3E/8C,KAAKq0B,gBACH2oB,OACEC,KAAM,EACNC,UAAW,GACXC,UAAW,GACX1xB,OAAQ,GACR2xB,MAAO,UACPC,MAAO72C,OACP0gB,SAAU,GACVC,SAAU,GACVm2B,UAAW,QACXC,SAAU,GACVC,SAAU,UACVC,SAAUj3C,OACVk3C,gBAAiB,EACjBC,gBAAiB,QACjBC,MAAO,GACPzyC,OACIiB,OAAQ,UACRD,WAAY,UACdE,WACED,OAAQ,UACRD,WAAY,WAEdG,OACEF,OAAQ,UACRD,WAAY,YAGhB8F,MAAOzL,OACPwZ,YAAa,EACb69B,oBAAqBr3C,QAEvBs3C,OACE52B,SAAU,EACVC,SAAU,GACV5U,MAAO,EACPwrC,yBAA0B,EAC1BC,WAAY,IACZ/wC,MAAO,OACP9B,OACEA,MAAM,UACNkB,UAAU,UACVC,MAAO,WAETgxC,UAAW,UACXC,SAAU,GACVC,SAAU,QACVC,SAAU,QACVC,gBAAiB,EACjBC,gBAAiB,QACjBM,eAAe,aACfC,iBAAkB,EAClBC,MACEx4C,OAAQ,GACRy4C,IAAK,EACLC,UAAW73C,QAEb83C,aAAc,QAEhBC,kBAAiB,EACjBC,SACEC,WACE/vC,SAAS,EACTgwC,cAAe,EACfC,sBAAuB,KACvBC,eAAgB,GAChBC,aAAc,GACdC,eAAgB,IAChBC,QAAS,KAEXC,WACEJ,eAAgB,EAChBC,aAAc,IACdC,eAAgB,IAChBG,aAAc,IACdF,QAAS,KAEXG,uBACExwC,SAAS,EACTkwC,eAAgB,EAChBC,aAAc,IACdC,eAAgB,IAChBG,aAAc,IACdF,QAAS,KAEXA,QAAS,KACTH,eAAgB,KAChBC,aAAc,KACdC,eAAgB,MAElBK,YACEzwC,SAAS,EACT0wC,gBAAiB,IACjBC,iBAAiB,IACjBC,cAAc,IACdC,eAAgB,GAChBC,qBAAsB,GACtBC,gBAAiB,IACjBC,oBAAqB,GACrBC,mBAAoB,EACpBC,YAAa,IACbC,mBAAoB,GACpBC,sBAAuB,GACvBC,WAAY,GACZC,aAAcztC,MAAQ,EACRC,OAAQ,EACRiZ,OAAQ,GACtBw0B,sBAAuB,IACvBC,kBAAmB,GACnBC,uBAAwB,GAE1BC,YACE1xC,SAAS,GAEX2xC,UACE3xC,SAAS,EACT4xC,OAAQvuC,EAAG,GAAIC,EAAG,GAAIkuB,KAAM,MAE9BqgB,kBACE7xC,SAAS,EACT8xC,kBAAkB,GAEpBC,oBACE/xC,SAAQ,EACRgyC,gBAAiB,IACjBC,YAAa,IACbzlB,UAAW,KACX0lB,OAAQ,WAEVC,wBAAwB,EACxBC,cACEpyC,SAAS,EACTqyC,SAAS,EACTj6C,KAAM,aACNk6C,UAAW,IAEbC,YAAc,GACdC,YAAc,GACdC,WAAW,EACXC,wBAAyB,IACzBC,uBAAuB,EACvBhd,OAAQ,KACRQ,QAASA,EACTze,SACE5N,MAAO,IACP8kC,UAAW,QACXC,SAAU,GACVC,SAAU,UACVryC,OACEiB,OAAQ,OACRD,WAAY,YAGhBm1C,aAAa,EACbC,WAAW,EACX5jB,UAAU,EACVrxB,OAAO,EACPk1C,iBAAiB,EACjBC,iBAAiB,EACjBlvC,MAAQ,OACRC,OAAS,OACTy8B,YAAY,GAEdjvC,KAAK0hD,UAAY/gD,EAAK2E,UAAWtF,KAAKq0B,gBACtCr0B,KAAK2hD,WAAa,EAGlB3hD,KAAK4hD,UAAY5E,SAASc,UAC1B99C,KAAK6hD,oBAAqB,EAC1B7hD,KAAK8hD,mBAAqBC,YAAaC,SAGvChiD,KAAKiiD,eAAiB,EAAEjiD,KAAKo8C,kBAC7Bp8C,KAAKkiD,wBAA0B,iBAC/BliD,KAAKmiD,WAAa,EAClBniD,KAAKoiD,YAAc,EACnBpiD,KAAKqiD,YAAc,EACnBriD,KAAKsiD,kBAAoB,EACzBtiD,KAAKuiD,kBAAoB,EACzBviD,KAAKwiD,eAAiB,KACtBxiD,KAAKyiD,mBAAqB,KAC1BziD,KAAK0iD,UAAY,CAGjB,IAAIv/C,GAAUnD,IACdA,MAAKm0B,OAAS,GAAI9wB,GAClBrD,KAAK2iD,OAAS,GAAIr/C,GAClBtD,KAAK2iD,OAAOC,kBAAkB,WAC5Bz/C,EAAQ0/C,YAIV7iD,KAAK8iD,WAAa,EAClB9iD,KAAK+iD,WAAa,EAClB/iD,KAAKgjD,cAAgB,EAIrBhjD,KAAKijD,qBAELjjD,KAAK00B,UAEL10B,KAAKkjD,oBAELljD,KAAKmjD,qBAELnjD,KAAKojD,uBAELpjD,KAAKqjD,uBAILrjD,KAAKsjD,gBAAgBtjD,KAAKsf,MAAME,YAAc,EAAGxf,KAAKsf,MAAMuF,aAAe,GAC3E7kB,KAAKgd,UAAU,GACfhd,KAAKkT,WAAWzE,GAGhBzO,KAAKujD,kBAAmB,EACxBvjD,KAAKwjD,mBACLxjD,KAAKyjD,sBAAuB,EAC5BzjD,KAAK0jD,YAAa,EAClB1jD,KAAKohD,wBAA0B,KAC/BphD,KAAK2jD,eAAgB,EAGrB3jD,KAAK4jD,oBACL5jD,KAAK6jD,0BACL7jD,KAAK8jD,eACL9jD,KAAKg9C,SACLh9C,KAAK89C,SAGL99C,KAAK+jD,eAAqBhyC,EAAK,EAAEC,EAAK,GACtChS,KAAKgkD,mBAAqBjyC,EAAK,EAAEC,EAAK,GACtChS,KAAKikD,iBAAmBlyC,EAAK,EAAEC,EAAK,GACpChS,KAAKkkD,cACLlkD,KAAKid,MAAQ,EACbjd,KAAKmkD,cAAgBnkD,KAAKid,MAG1Bjd,KAAKokD,UAAY,KACjBpkD,KAAKqkD,UAAY,KAGjBrkD,KAAKskD,gBACHrxC,IAAO,SAAU3J,EAAOwK,GACtB3Q,EAAQohD,UAAUzwC,EAAO7R,OACzBkB,EAAQyM,SAEViF,OAAU,SAAUvL,EAAOwK,GACzB3Q,EAAQqhD,aAAa1wC,EAAO7R,MAAO6R,EAAOpB,MAC1CvP,EAAQyM,SAEVyG,OAAU,SAAU/M,EAAOwK,GACzB3Q,EAAQshD,aAAa3wC,EAAO7R,OAC5BkB,EAAQyM,UAGZ5P,KAAK0kD,gBACHzxC,IAAO,SAAU3J,EAAOwK,GACtB3Q,EAAQwhD,UAAU7wC,EAAO7R,OACzBkB,EAAQyM,SAEViF,OAAU,SAAUvL,EAAOwK,GACzB3Q,EAAQyhD,aAAa9wC,EAAO7R,OAC5BkB,EAAQyM,SAEVyG,OAAU,SAAU/M,EAAOwK,GACzB3Q,EAAQ0hD,aAAa/wC,EAAO7R,OAC5BkB,EAAQyM,UAKZ5P,KAAK8kD,QAAS,EACd9kD,KAAK+kD,MAAQv+C,OAGbxG,KAAKgY,QAAQtF,EAAK1S,KAAK0hD,UAAUvC,WAAWzwC,SAAW1O,KAAK0hD,UAAUjB,mBAAmB/xC,SAGzF1O,KAAK08C,cAAe,EAC6B,GAA7C18C,KAAK0hD,UAAUjB,mBAAmB/xC,QACpC1O,KAAKglD,2BAI2B,GAA5BhlD,KAAK0hD,UAAUP,WACjBnhD,KAAKilD,WAAWz+C,QAAW,EAAKxG,KAAK0hD,UAAUvC,WAAWzwC,SAK1D1O,KAAK0hD,UAAUvC,WAAWzwC,SAC5B1O,KAAKklD,sBA/VT,GAAInoC,GAAU7c,EAAoB,IAC9B6D,EAAS7D,EAAoB,IAC7BilD,EAAWjlD,EAAoB,IAC/BS,EAAOT,EAAoB,GAC3BklD,EAAallD,EAAoB,IACjCW,EAAUX,EAAoB,GAC9BY,EAAWZ,EAAoB,GAC/BuD,EAAYvD,EAAoB,IAChCwD,EAAcxD,EAAoB,IAClCmD,EAASnD,EAAoB,IAC7BoD,EAASpD,EAAoB,IAC7BqD,EAAOrD,EAAoB,IAC3BkD,EAAOlD,EAAoB,IAC3BsD,EAAQtD,EAAoB,IAC5BmlD,EAAcnlD,EAAoB,IAClColD,EAAYplD,EAAoB,IAChC2kC,EAAU3kC,EAAoB,GAGlCA,GAAoB,IAiVpB6c,EAAQ7Z,EAAQiQ,WAOhBjQ,EAAQiQ,UAAU+oC,wBAA0B,WAC1C,GAAIqJ,GAAcv8C,UAAUC,UAAUs7B,aACtCvkC,MAAKwlD,iBAAkB,EACgB,IAAnCD,EAAY5+C,QAAQ,YACtB3G,KAAKwlD,iBAAkB,EAEiB,IAAjCD,EAAY5+C,QAAQ,WACvB4+C,EAAY5+C,QAAQ,WAAa,KACnC3G,KAAKwlD,iBAAkB,IAa7BtiD,EAAQiQ,UAAUsyC,eAAiB,WAIjC,IAAK,GAHDC,GAAUn0C,SAASo0C,qBAAsB,UAGpCngD,EAAI,EAAGA,EAAIkgD,EAAQ//C,OAAQH,IAAK,CACvC,GAAIogD,GAAMF,EAAQlgD,GAAGogD,IACjBrhD,EAAQqhD,GAAO,qBAAqBnhD,KAAKmhD,EAC7C,IAAIrhD,EAEF,MAAOqhD,GAAI1gB,UAAU,EAAG0gB,EAAIjgD,OAASpB,EAAM,GAAGoB,QAIlD,MAAO,OAQTzC,EAAQiQ,UAAU0yC,UAAY,WAC5B,GAAsDC,GAAlDC,EAAO,IAAKC,EAAO,KAAMC,EAAO,IAAKC,EAAO,IAChD,KAAK,GAAIC,KAAUnmD,MAAKg9C,MAClBh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GACdF,EAAQH,EAAKM,YAAgB,OAAIH,EAAOH,EAAKM,YAAY3+C,MACzDy+C,EAAQJ,EAAKM,YAAiB,QAAIF,EAAOJ,EAAKM,YAAY/+B,OAC1D0+B,EAAQD,EAAKM,YAAkB,SAAIL,EAAOD,EAAKM,YAAY9iC,QAC3D0iC,EAAQF,EAAKM,YAAe,MAAIJ,EAAOF,EAAKM,YAAYz+C,KAMhE,OAHY,MAARs+C,GAAuB,MAARC,GAAwB,KAARH,GAAuB,MAARC,IAChDD,EAAO,EAAGC,EAAO,EAAGC,EAAO,EAAGC,EAAO,IAE/BD,KAAMA,EAAMC,KAAMA,EAAMH,KAAMA,EAAMC,KAAMA,IASpD9iD,EAAQiQ,UAAUkzC,YAAc,SAAS5wB,GACvC,OAAQ1jB,EAAI,IAAO0jB,EAAMywB,KAAOzwB,EAAMwwB,MAC9Bj0C,EAAI,IAAOyjB,EAAMuwB,KAAOvwB,EAAMswB,QAUxC7iD,EAAQiQ,UAAU8xC,WAAa,SAASqB,EAAkBC,EAAaC,GACrExmD,KAAK6iD,SAAQ,GAEOr8C,SAAhB+/C,IACFA,GAAc,GAEK//C,SAAjBggD,IACFA,GAAe,GAEQhgD,SAArB8/C,IACFA,GAAmB,EAGrB,IACIG,GADAhxB,EAAQz1B,KAAK6lD,WAGjB,IAAmB,GAAfU,EAAqB,CACvB,GAAIG,GAAgB1mD,KAAK8jD,YAAYn+C,MAIjC8gD,GAH+B,GAA/BzmD,KAAK0hD,UAAUZ,aACwB,GAArC9gD,KAAK0hD,UAAUvC,WAAWzwC,SAC5Bg4C,GAAiB1mD,KAAK0hD,UAAUvC,WAAWC,gBAC/B,UAAYsH,EAAgB,WAAa,SAGzC,QAAUA,EAAgB,QAAU,SAIT,GAArC1mD,KAAK0hD,UAAUvC,WAAWzwC,SAC1Bg4C,GAAiB1mD,KAAK0hD,UAAUvC,WAAWC,gBACjC,YAAcsH,EAAgB,YAAc,cAG5C,YAAcA,EAAgB,aAAe,SAK7D,IAAIC,GAASzhD,KAAK4G,IAAI9L,KAAKsf,MAAMC,OAAOC,YAAc,IAAKxf,KAAKsf,MAAMC,OAAOsF,aAAe,IAC5F4hC,IAAaE,MAEV,CACH,GAAInP,GAAgD,IAApCtyC,KAAK2lB,IAAI4K,EAAMywB,KAAOzwB,EAAMwwB,MACxCW,EAAgD,IAApC1hD,KAAK2lB,IAAI4K,EAAMuwB,KAAOvwB,EAAMswB,MAExCc,EAAa7mD,KAAKsf,MAAMC,OAAOC,YAAeg4B,EAC9CsP,EAAa9mD,KAAKsf,MAAMC,OAAOsF,aAAe+hC,CAElDH,GAA2BK,GAAdD,EAA4BA,EAAaC,EAGpDL,EAAY,IACdA,EAAY,EAId,IAAIv6B,GAASlsB,KAAKqmD,YAAY5wB,EAC9B,IAAoB,GAAhB+wB,EAAuB,CACzB,GAAI/3C,IAAWmV,SAAUsI,EAAQjP,MAAOwpC,EAAWM,UAAWT,EAC9DtmD,MAAK6nB,OAAOpZ,GACZzO,KAAK8kD,QAAS,EACd9kD,KAAK4P,YAGLsc,GAAOna,GAAK00C,EACZv6B,EAAOla,GAAKy0C,EACZv6B,EAAOna,GAAK,GAAM/R,KAAKsf,MAAMC,OAAOC,YACpC0M,EAAOla,GAAK,GAAMhS,KAAKsf,MAAMC,OAAOsF,aACpC7kB,KAAKgd,UAAUypC,GACfzmD,KAAKsjD,iBAAiBp3B,EAAOna,GAAGma,EAAOla,IAS3C9O,EAAQiQ,UAAU6zC,qBAAuB,WACvChnD,KAAKinD,qBACL,KAAK,GAAIC,KAAOlnD,MAAKg9C,MACfh9C,KAAKg9C,MAAMl3C,eAAeohD,IAC5BlnD,KAAK8jD,YAAY97C,KAAKk/C,IAiB5BhkD,EAAQiQ,UAAU6E,QAAU,SAAStF,EAAM8zC,GAOzC,GANqBhgD,SAAjBggD,IACFA,GAAe,GAGjBxmD,KAAK08C,cAAe,EAEhBhqC,GAAQA,EAAKod,MAAQpd,EAAKsqC,OAAStqC,EAAKorC,OAC1C,KAAM,IAAItkC,aAAY,iGAYxB,IAP+C,GAA3CxZ,KAAK0hD,UAAUnB,iBAAiB7xC,SAClC1O,KAAKmnD,wBAIPnnD,KAAKkT,WAAWR,GAAQA,EAAKjE,SAEzBiE,GAAQA,EAAKod,KAEf,GAAGpd,GAAQA,EAAKod,IAAK,CACnB,GAAIs3B,GAAU3jD,EAAU4jD,WAAW30C,EAAKod,IAExC,YADA9vB,MAAKgY,QAAQovC,QAIZ,IAAI10C,GAAQA,EAAK40C,OAEpB,GAAG50C,GAAQA,EAAK40C,MAAO,CACrB,GAAIC,GAAY7jD,EAAY8jD,WAAW90C,EAAK40C,MAE5C,YADAtnD,MAAKgY,QAAQuvC,QAKfvnD,MAAKynD,UAAU/0C,GAAQA,EAAKsqC,OAC5Bh9C,KAAK0nD,UAAUh1C,GAAQA,EAAKorC,MAE9B99C,MAAK2nD,mBACe,GAAhBnB,IAC+C,GAA7CxmD,KAAK0hD,UAAUjB,mBAAmB/xC,SACpC1O,KAAK4nD,eACL5nD,KAAKglD,4BAIDhlD,KAAK0hD,UAAUP,WACjBnhD,KAAK6nD,aAGT7nD,KAAK4P,SAEP5P,KAAK08C,cAAe,GAOtBx5C,EAAQiQ,UAAUD,WAAa,SAAUzE,GACvC,GAAIA,EAAS,CACX,GAAI5I,GACAqI,GAAU,QAAQ,QAAQ,eAAe,qBAAqB,aAAa,aAC7E,WAAW,mBAAmB,QAAQ,SAAS,aAAa,YAAY,WAAW,aAOrF,IAJAvN,EAAK+F,uBAAuBwH,EAAOlO,KAAK0hD,UAAWjzC,GACnD9N,EAAK+F,wBAAwB,SAAS1G,KAAK0hD,UAAU1E,MAAOvuC,EAAQuuC,OACpEr8C,EAAK+F,wBAAwB,QAAQ,UAAU1G,KAAK0hD,UAAU5D,MAAOrvC,EAAQqvC,OAEzErvC,EAAQ+vC,UACV79C,EAAK4N,aAAavO,KAAK0hD,UAAUlD,QAAS/vC,EAAQ+vC,QAAQ,aAC1D79C,EAAK4N,aAAavO,KAAK0hD,UAAUlD,QAAS/vC,EAAQ+vC,QAAQ,aAEtD/vC,EAAQ+vC,QAAQU,uBAAuB,CACzCl/C,KAAK0hD,UAAUjB,mBAAmB/xC,SAAU,EAC5C1O,KAAK0hD,UAAUlD,QAAQU,sBAAsBxwC,SAAU,EACvD1O,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,SAAU,CAC3C,KAAK7I,IAAQ4I,GAAQ+vC,QAAQU,sBACvBzwC,EAAQ+vC,QAAQU,sBAAsBp5C,eAAeD,KACvD7F,KAAK0hD,UAAUlD,QAAQU,sBAAsBr5C,GAAQ4I,EAAQ+vC,QAAQU,sBAAsBr5C;CAkDnG,GA5CI4I,EAAQ2gC,QAAQpvC,KAAK28C,iBAAiB1pC,IAAMxE,EAAQ2gC,OACpD3gC,EAAQq5C,SAAS9nD,KAAK28C,iBAAiBC,KAAOnuC,EAAQq5C,QACtDr5C,EAAQs5C,aAAa/nD,KAAK28C,iBAAiBE,SAAWpuC,EAAQs5C,YAC9Dt5C,EAAQu5C,YAAYhoD,KAAK28C,iBAAiBG,QAAUruC,EAAQu5C,WAC5Dv5C,EAAQw5C,WAAWjoD,KAAK28C,iBAAiBI,IAAMtuC,EAAQw5C,UAE3DtnD,EAAK4N,aAAavO,KAAK0hD,UAAWjzC,EAAQ,gBAC1C9N,EAAK4N,aAAavO,KAAK0hD,UAAWjzC,EAAQ,sBAC1C9N,EAAK4N,aAAavO,KAAK0hD,UAAWjzC,EAAQ,cAC1C9N,EAAK4N,aAAavO,KAAK0hD,UAAWjzC,EAAQ,cAC1C9N,EAAK4N,aAAavO,KAAK0hD,UAAWjzC,EAAQ,YAC1C9N,EAAK4N,aAAavO,KAAK0hD,UAAWjzC,EAAQ,oBAGtCA,EAAQ8xC,mBACVvgD,KAAKkoD,SAAWloD,KAAK0hD,UAAUnB,iBAAiBC,kBAK9C/xC,EAAQqvC,QACkBt3C,SAAxBiI,EAAQqvC,MAAM3yC,QACZxK,EAAKwD,SAASsK,EAAQqvC,MAAM3yC,QAC9BnL,KAAK0hD,UAAU5D,MAAM3yC,SACrBnL,KAAK0hD,UAAU5D,MAAM3yC,MAAMA,MAAQsD,EAAQqvC,MAAM3yC,MACjDnL,KAAK0hD,UAAU5D,MAAM3yC,MAAMkB,UAAYoC,EAAQqvC,MAAM3yC,MACrDnL,KAAK0hD,UAAU5D,MAAM3yC,MAAMmB,MAAQmC,EAAQqvC,MAAM3yC,QAGf3E,SAA9BiI,EAAQqvC,MAAM3yC,MAAMA,QAA0BnL,KAAK0hD,UAAU5D,MAAM3yC,MAAMA,MAAQsD,EAAQqvC,MAAM3yC,MAAMA,OACnE3E,SAAlCiI,EAAQqvC,MAAM3yC,MAAMkB,YAA0BrM,KAAK0hD,UAAU5D,MAAM3yC,MAAMkB,UAAYoC,EAAQqvC,MAAM3yC,MAAMkB,WAC3E7F,SAA9BiI,EAAQqvC,MAAM3yC,MAAMmB,QAA0BtM,KAAK0hD,UAAU5D,MAAM3yC,MAAMmB,MAAQmC,EAAQqvC,MAAM3yC,MAAMmB,QAE3GtM,KAAK0hD,UAAU5D,MAAMQ,cAAe,GAGjC7vC,EAAQqvC,MAAMR,WACW92C,SAAxBiI,EAAQqvC,MAAM3yC,QACZxK,EAAKwD,SAASsK,EAAQqvC,MAAM3yC,OAAmBnL,KAAK0hD,UAAU5D,MAAMR,UAAY7uC,EAAQqvC,MAAM3yC,MAC3D3E,SAA9BiI,EAAQqvC,MAAM3yC,MAAMA,QAAsBnL,KAAK0hD,UAAU5D,MAAMR,UAAY7uC,EAAQqvC,MAAM3yC,MAAMA,SAK1GsD,EAAQuuC,OACNvuC,EAAQuuC,MAAM7xC,MAAO,CACvB,GAAIg9C,GAAcxnD,EAAKuK,WAAWuD,EAAQuuC,MAAM7xC,MAChDnL,MAAK0hD,UAAU1E,MAAM7xC,MAAMgB,WAAag8C,EAAYh8C,WACpDnM,KAAK0hD,UAAU1E,MAAM7xC,MAAMiB,OAAS+7C,EAAY/7C,OAChDpM,KAAK0hD,UAAU1E,MAAM7xC,MAAMkB,UAAUF,WAAag8C,EAAY97C,UAAUF,WACxEnM,KAAK0hD,UAAU1E,MAAM7xC,MAAMkB,UAAUD,OAAS+7C,EAAY97C,UAAUD,OACpEpM,KAAK0hD,UAAU1E,MAAM7xC,MAAMmB,MAAMH,WAAag8C,EAAY77C,MAAMH,WAChEnM,KAAK0hD,UAAU1E,MAAM7xC,MAAMmB,MAAMF,OAAS+7C,EAAY77C,MAAMF,OAGhE,GAAIqC,EAAQ0lB,OACV,IAAK,GAAIi0B,KAAa35C,GAAQ0lB,OAC5B,GAAI1lB,EAAQ0lB,OAAOruB,eAAesiD,GAAY,CAC5C,GAAIn2C,GAAQxD,EAAQ0lB,OAAOi0B,EAC3BpoD,MAAKm0B,OAAOlhB,IAAIm1C,EAAWn2C,GAKjC,GAAIxD,EAAQ2X,QAAS,CACnB,IAAKvgB,IAAQ4I,GAAQ2X,QACf3X,EAAQ2X,QAAQtgB,eAAeD,KACjC7F,KAAK0hD,UAAUt7B,QAAQvgB,GAAQ4I,EAAQ2X,QAAQvgB,GAG/C4I,GAAQ2X,QAAQjb,QAClBnL,KAAK0hD,UAAUt7B,QAAQjb,MAAQxK,EAAKuK,WAAWuD,EAAQ2X,QAAQjb,QAmBnE,GAfI,cAAgBsD,KACdA,EAAQ45C,WACLroD,KAAKsoD,YACRtoD,KAAKsoD,UAAY,GAAIhD,GAAUtlD,KAAKsf,OACpCtf,KAAKsoD,UAAU/0C,GAAG,SAAUvT,KAAKuoD,gBAAgBzzB,KAAK90B,QAIpDA,KAAKsoD,YACPtoD,KAAKsoD,UAAUh1C,gBACRtT,MAAKsoD,YAKd75C,EAAQg4B,OACV,KAAM,IAAI7iC,OAAM,6EAMlB5D,MAAKijD,qBAELjjD,KAAKwoD,0BAELxoD,KAAKyoD,0BAELzoD,KAAK0oD,yBAIL1oD,KAAKuoD,kBAELvoD,KAAK2kB,QAAQ3kB,KAAK0hD,UAAUnvC,MAAOvS,KAAK0hD,UAAUlvC,QAClDxS,KAAK8kD,QAAS,EACd9kD,KAAK4P,UAaT1M,EAAQiQ,UAAUuhB,QAAU,WAE1B,KAAO10B,KAAKyZ,iBAAiBiK,iBAC3B1jB,KAAKyZ,iBAAiBtI,YAAYnR,KAAKyZ,iBAAiBkK,WAe1D,IAZA3jB,KAAKsf,MAAQ/N,SAASM,cAAc,OACpC7R,KAAKsf,MAAMzX,UAAY,oBACvB7H,KAAKsf,MAAMrS,MAAM2W,SAAW,WAC5B5jB,KAAKsf,MAAMrS,MAAM4W,SAAW,SAK5B7jB,KAAKsf,MAAMC,OAAShO,SAASM,cAAc,UAC3C7R,KAAKsf,MAAMC,OAAOtS,MAAM2W,SAAW,WACnC5jB,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAMC,QAE7Bvf,KAAKsf,MAAMC,OAAOyH,WAQlB,CACH,GAAID,GAAM/mB,KAAKsf,MAAMC,OAAOyH,WAAW,KACvChnB,MAAK2hD,YAAcp4C,OAAOo/C,kBAAoB,IAAM5hC,EAAI6hC,8BAC9C7hC,EAAI8hC,2BACJ9hC,EAAI+hC,0BACJ/hC,EAAIgiC,yBACJhiC,EAAIiiC,wBAA0B,GAExChpD,KAAKsf,MAAMC,OAAOyH,WAAW,MAAMiiC,aAAajpD,KAAK2hD,WAAY,EAAG,EAAG3hD,KAAK2hD,WAAY,EAAG,OAhB1D,CACjC,GAAI79B,GAAWvS,SAASM,cAAe,MACvCiS,GAAS7W,MAAM9B,MAAQ,MACvB2Y,EAAS7W,MAAM8W,WAAc,OAC7BD,EAAS7W,MAAM+W,QAAW,OAC1BF,EAASG,UAAa,mDACtBjkB,KAAKsf,MAAMC,OAAO9N,YAAYqS,GAgBhC,GAAI3P,GAAKnU,IACTA,MAAKwlC,QACLxlC,KAAKkpD,SACLlpD,KAAK8D,OAAS,GAAIC,GAAO/D,KAAKsf,MAAMC,QACpCvf,KAAK8D,OAAOoR,IAAI,SAASi0C,KAAK/lB,QAAQ,IAEtCpjC,KAAK8D,OAAOyP,GAAG,MAAaY,EAAGi1C,OAAOt0B,KAAK3gB,IAC3CnU,KAAK8D,OAAOyP,GAAG,YAAaY,EAAGk1C,aAAav0B,KAAK3gB,IACjDnU,KAAK8D,OAAOyP,GAAG,QAAaY,EAAGgqB,QAAQrJ,KAAK3gB,IAC5CnU,KAAK8D,OAAOyP,GAAG,QAAaY,EAAGmqB,SAASxJ,KAAK3gB,IAC7CixC,EAAWkE,QAAQtpD,KAAK8D,OAAQqQ,EAAGkqB,SAASvJ,KAAK3gB,IACjDnU,KAAK8D,OAAOyP,GAAG,WAAaY,EAAG6pB,aAAalJ,KAAK3gB,IACjDnU,KAAK8D,OAAOyP,GAAG,UAAaY,EAAG8pB,QAAQnJ,KAAK3gB,IAC5CnU,KAAK8D,OAAOyP,GAAG,SAAaY,EAAG+pB,WAAWpJ,KAAK3gB,IAE/CnU,KAAKsf,MAAMC,OAAO5W,iBAAiB,YAAawL,EAAGo1C,kBAAkBz0B,KAAK3gB,IAC1EnU,KAAKsf,MAAMC,OAAO5W,iBAAiB,aAAcwL,EAAGiqB,cAActJ,KAAK3gB,IACvEnU,KAAKsf,MAAMC,OAAO5W,iBAAiB,iBAAkBwL,EAAGiqB,cAActJ,KAAK3gB,IAG3EnU,KAAKyZ,iBAAiBhI,YAAYzR,KAAKsf,QAQzCpc,EAAQiQ,UAAUo1C,gBAAkB,WAClC,GAAIp0C,GAAKnU,IACawG,UAAlBxG,KAAKmlD,UACPnlD,KAAKmlD,SAAS7xC,UAEhBtT,KAAKmlD,SAAWA,IAEhBnlD,KAAKmlD,SAASqE,QAEVxpD,KAAK0hD,UAAUrB,SAAS3xC,SAAW1O,KAAKypD,aAC1CzpD,KAAKmlD,SAASrwB,KAAK,KAAQ90B,KAAK0pD,QAAQ50B,KAAK3gB,GAAQ,WACrDnU,KAAKmlD,SAASrwB,KAAK,KAAQ90B,KAAK2pD,aAAa70B,KAAK3gB,GAAK,SACvDnU,KAAKmlD,SAASrwB,KAAK,OAAQ90B,KAAK4pD,UAAU90B,KAAK3gB,GAAM,WACrDnU,KAAKmlD,SAASrwB,KAAK,OAAQ90B,KAAK2pD,aAAa70B,KAAK3gB,GAAK,SACvDnU,KAAKmlD,SAASrwB,KAAK,OAAQ90B,KAAK6pD,UAAU/0B,KAAK3gB,GAAM,WACrDnU,KAAKmlD,SAASrwB,KAAK,OAAQ90B,KAAK8pD,aAAah1B,KAAK3gB,GAAK,SACvDnU,KAAKmlD,SAASrwB,KAAK,QAAQ90B,KAAK+pD,WAAWj1B,KAAK3gB,GAAK,WACrDnU,KAAKmlD,SAASrwB,KAAK,QAAQ90B,KAAK8pD,aAAah1B,KAAK3gB,GAAK,SACvDnU,KAAKmlD,SAASrwB,KAAK,IAAQ90B,KAAKgqD,QAAQl1B,KAAK3gB,GAAQ,WACrDnU,KAAKmlD,SAASrwB,KAAK,IAAQ90B,KAAKiqD,UAAUn1B,KAAK3gB,GAAQ,SACvDnU,KAAKmlD,SAASrwB,KAAK,OAAQ90B,KAAKgqD,QAAQl1B,KAAK3gB,GAAQ,WACrDnU,KAAKmlD,SAASrwB,KAAK,OAAQ90B,KAAKiqD,UAAUn1B,KAAK3gB,GAAQ,SACvDnU,KAAKmlD,SAASrwB,KAAK,OAAQ90B,KAAKkqD,SAASp1B,KAAK3gB,GAAO,WACrDnU,KAAKmlD,SAASrwB,KAAK,OAAQ90B,KAAKiqD,UAAUn1B,KAAK3gB,GAAQ,SACvDnU,KAAKmlD,SAASrwB,KAAK,IAAQ90B,KAAKkqD,SAASp1B,KAAK3gB,GAAO,WACrDnU,KAAKmlD,SAASrwB,KAAK,IAAQ90B,KAAKiqD,UAAUn1B,KAAK3gB,GAAQ,SACvDnU,KAAKmlD,SAASrwB,KAAK,IAAQ90B,KAAKgqD,QAAQl1B,KAAK3gB,GAAQ,WACrDnU,KAAKmlD,SAASrwB,KAAK,IAAQ90B,KAAKiqD,UAAUn1B,KAAK3gB,GAAQ,SACvDnU,KAAKmlD,SAASrwB,KAAK,IAAQ90B,KAAKkqD,SAASp1B,KAAK3gB,GAAO,WACrDnU,KAAKmlD,SAASrwB,KAAK,IAAQ90B,KAAKiqD,UAAUn1B,KAAK3gB,GAAQ,SACvDnU,KAAKmlD,SAASrwB,KAAK,SAAS90B,KAAKgqD,QAAQl1B,KAAK3gB,GAAO,WACrDnU,KAAKmlD,SAASrwB,KAAK,SAAS90B,KAAKiqD,UAAUn1B,KAAK3gB,GAAO,SACvDnU,KAAKmlD,SAASrwB,KAAK,WAAW90B,KAAKkqD,SAASp1B,KAAK3gB,GAAI,WACrDnU,KAAKmlD,SAASrwB,KAAK,WAAW90B,KAAKiqD,UAAUn1B,KAAK3gB,GAAK,UAGV,GAA3CnU,KAAK0hD,UAAUnB,iBAAiB7xC,UAClC1O,KAAKmlD,SAASrwB,KAAK,MAAM90B,KAAKmnD,sBAAsBryB,KAAK3gB,IACzDnU,KAAKmlD,SAASrwB,KAAK,SAAS90B,KAAKmqD,gBAAgBr1B,KAAK3gB,MAU1DjR,EAAQiQ,UAAUG,QAAU,WAC1BtT,KAAK4P,MAAQ,aACb5P,KAAKyhB,OAAS,aACdzhB,KAAK+kD,OAAQ,EAGb/kD,KAAKoqD,+BAGLpqD,KAAKmlD,SAASqE,QAGdxpD,KAAK8D,OAAOwP,UAGZtT,KAAK0T,MAEL1T,KAAKqqD,oBAAoBrqD,KAAKyZ,mBAGhCvW,EAAQiQ,UAAUk3C,oBAAsB,SAASC,GAC/C,KAAoC,GAA7BA,EAAU5mC,iBACf1jB,KAAKqqD,oBAAoBC,EAAU3mC,YACnC2mC,EAAUn5C,YAAYm5C,EAAU3mC,aAUpCzgB,EAAQiQ,UAAUo3C,YAAc,SAAUr+B,GACxC,OACEna,EAAGma,EAAOna,EAAIpR,EAAK2G,gBAAgBtH,KAAKsf,MAAMC,QAC9CvN,EAAGka,EAAOla,EAAIrR,EAAK+G,eAAe1H,KAAKsf,MAAMC,UASjDrc,EAAQiQ,UAAUkrB,SAAW,SAAU/0B,IACjC,GAAIhF,OAAO0C,UAAYhH,KAAK0iD,UAAY,MAC1C1iD,KAAKwlC,KAAK3F,QAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,QAC3ClsB,KAAKwlC,KAAKglB,SAAU,EACpBxqD,KAAKkpD,MAAMjsC,MAAQjd,KAAKyqD,YAGxBzqD,KAAK0iD,WAAY,GAAIp+C,OAAO0C,UAE5BhH,KAAK0qD,aAAa1qD,KAAKwlC,KAAK3F,WAQhC38B,EAAQiQ,UAAU6qB,aAAe,SAAU10B,GACzCtJ,KAAK2qD,iBAAiBrhD,IAUxBpG,EAAQiQ,UAAUw3C,iBAAmB,SAASrhD,GAElB9C,SAAtBxG,KAAKwlC,KAAK3F,SACZ7/B,KAAKq+B,SAAS/0B,EAGhB,IAAIw8C,GAAO9lD,KAAK4qD,WAAW5qD,KAAKwlC,KAAK3F,QASrC,IANA7/B,KAAKwlC,KAAKzG,UAAW,EACrB/+B,KAAKwlC,KAAK2K,aACVnwC,KAAKwlC,KAAK/nB,YAAczd,KAAK6qD,kBAC7B7qD,KAAKwlC,KAAK2gB,OAAS,KACnBnmD,KAAK2jD,eAAgB,EAET,MAARmC,GAA4C,GAA5B9lD,KAAK0hD,UAAUH,UAAmB,CACpDvhD,KAAK2jD,eAAgB,EACrB3jD,KAAKwlC,KAAK2gB,OAASL,EAAKzlD,GAEnBylD,EAAKgF,cACR9qD,KAAK+qD,cAAcjF,GAAK,GAG1B9lD,KAAK4tB,KAAK,aAAao9B,QAAQhrD,KAAK42B,eAAeomB,OAGnD,KAAK,GAAIiO,KAAYjrD,MAAKkrD,aAAalO,MACrC,GAAIh9C,KAAKkrD,aAAalO,MAAMl3C,eAAemlD,GAAW,CACpD,GAAIhnD,GAASjE,KAAKkrD,aAAalO,MAAMiO,GACjCr/C,GACFvL,GAAI4D,EAAO5D,GACXylD,KAAM7hD,EAGN8N,EAAG9N,EAAO8N,EACVC,EAAG/N,EAAO+N,EACVm5C,OAAQlnD,EAAOknD,OACfC,OAAQnnD,EAAOmnD,OAGjBnnD,GAAOknD,QAAS,EAChBlnD,EAAOmnD,QAAS,EAEhBprD,KAAKwlC,KAAK2K,UAAUnoC,KAAK4D,IAK/BtC,EAAMD,kBAQRnG,EAAQiQ,UAAU8qB,QAAU,SAAU30B,GACpCtJ,KAAKqrD,cAAc/hD,IAUrBpG,EAAQiQ,UAAUk4C,cAAgB,SAAS/hD,GACzC,IAAItJ,KAAKwlC,KAAKglB,QAAd,CAKAxqD,KAAKsrD,aAEL,IAAIzrB,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,QACjC/X,EAAKnU,KACLwlC,EAAOxlC,KAAKwlC,KACZ2K,EAAY3K,EAAK2K,SACrB,IAAIA,GAAaA,EAAUxqC,QAAsC,GAA5B3F,KAAK0hD,UAAUH,UAAmB,CAErE,GAAI/hB,GAASK,EAAQ9tB,EAAIyzB,EAAK3F,QAAQ9tB,EAClC0tB,EAASI,EAAQ7tB,EAAIwzB,EAAK3F,QAAQ7tB,CAGtCm+B,GAAU9nC,QAAQ,SAAUuD,GAC1B,GAAIk6C,GAAOl6C,EAAEk6C,IAERl6C,GAAEu/C,SACLrF,EAAK/zC,EAAIoC,EAAGo3C,qBAAqBp3C,EAAGq3C,qBAAqB5/C,EAAEmG,GAAKytB,IAG7D5zB,EAAEw/C,SACLtF,EAAK9zC,EAAImC,EAAGs3C,qBAAqBt3C,EAAGu3C,qBAAqB9/C,EAAEoG,GAAKytB,MAM/Dz/B,KAAK8kD,SACR9kD,KAAK8kD,QAAS,EACd9kD,KAAK4P,aAKP,IAAkC,GAA9B5P,KAAK0hD,UAAUJ,YAAqB,CAEtC,GAA0B96C,SAAtBxG,KAAKwlC,KAAK3F,QAEZ,WADA7/B,MAAK2qD,iBAAiBrhD,EAGxB,IAAI8jB,GAAQyS,EAAQ9tB,EAAI/R,KAAKwlC,KAAK3F,QAAQ9tB,EACtCsb,EAAQwS,EAAQ7tB,EAAIhS,KAAKwlC,KAAK3F,QAAQ7tB,CAE1ChS,MAAKsjD,gBACHtjD,KAAKwlC,KAAK/nB,YAAY1L,EAAIqb,EAC1BptB,KAAKwlC,KAAK/nB,YAAYzL,EAAIqb,GAE5BrtB,KAAK6iD,UAITv5C,EAAMD,mBAORnG,EAAQiQ,UAAU+qB,WAAa,SAAU50B,GACvCtJ,KAAK2rD,eAAeriD,IAItBpG,EAAQiQ,UAAUw4C,eAAiB,SAASriD,GAC1CtJ,KAAKwlC,KAAKzG,UAAW,CACrB,IAAIoR,GAAYnwC,KAAKwlC,KAAK2K,SACtBA,IAAaA,EAAUxqC,QACzBwqC,EAAU9nC,QAAQ,SAAUuD,GAE1BA,EAAEk6C,KAAKqF,OAASv/C,EAAEu/C,OAClBv/C,EAAEk6C,KAAKsF,OAASx/C,EAAEw/C,SAEpBprD,KAAK8kD,QAAS,EACd9kD,KAAK4P,SAGL5P,KAAK6iD,UAEmB,GAAtB7iD,KAAK2jD,cACP3jD,KAAK4tB,KAAK,WAAWo9B,aAGrBhrD,KAAK4tB,KAAK,WAAWo9B,QAAQhrD,KAAK42B,eAAeomB,QAGnD1zC,EAAMD,kBAORnG,EAAQiQ,UAAUi2C,OAAS,SAAU9/C,GACnC,GAAIu2B,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,OACrClsB,MAAKikD,gBAAkBpkB,EACvB7/B,KAAK4rD,WAAW/rB,IASlB38B,EAAQiQ,UAAUk2C,aAAe,SAAU//C,GACzC,GAAIu2B,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,OACrClsB,MAAK6rD,iBAAiBhsB,IAQxB38B,EAAQiQ,UAAUgrB,QAAU,SAAU70B,GACpC,GAAIu2B,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,OACrClsB,MAAKikD,gBAAkBpkB,EACvB7/B,KAAK8rD,cAAcjsB,IAQrB38B,EAAQiQ,UAAU44C,WAAa,SAAUziD,GACvC,GAAIu2B,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,OACrClsB,MAAKgsD,iBAAiBnsB,IAQxB38B,EAAQiQ,UAAUmrB,SAAW,SAAUh1B,GACrC,GAAIu2B,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,OAErClsB,MAAKwlC,KAAKglB,SAAU,EACd,SAAWxqD,MAAKkpD,QACpBlpD,KAAKkpD,MAAMjsC,MAAQ,EAIrB,IAAIA,GAAQjd,KAAKkpD,MAAMjsC,MAAQ3T,EAAM2T,KACrCjd,MAAKisD,MAAMhvC,EAAO4iB,IAUpB38B,EAAQiQ,UAAU84C,MAAQ,SAAShvC,EAAO4iB,GACxC,GAA+B,GAA3B7/B,KAAK0hD,UAAU/jB,SAAkB,CACnC,GAAIuuB,GAAWlsD,KAAKyqD,WACR,MAARxtC,IACFA,EAAQ,MAENA,EAAQ,KACVA,EAAQ,GAGV,IAAIkvC,GAAsB,IACR3lD,UAAdxG,KAAKwlC,MACmB,GAAtBxlC,KAAKwlC,KAAKzG,WACZotB,EAAsBnsD,KAAKosD,YAAYpsD,KAAKwlC,KAAK3F,SAIrD,IAAIpiB,GAAczd,KAAK6qD,kBAEnBwB,EAAYpvC,EAAQivC,EACpBI,GAAM,EAAID,GAAaxsB,EAAQ9tB,EAAI0L,EAAY1L,EAAIs6C,EACnDE,GAAM,EAAIF,GAAaxsB,EAAQ7tB,EAAIyL,EAAYzL,EAAIq6C,CASvD,IAPArsD,KAAKkkD,YAAcnyC,EAAM/R,KAAKurD,qBAAqB1rB,EAAQ9tB,GACxCC,EAAMhS,KAAKyrD,qBAAqB5rB,EAAQ7tB,IAE3DhS,KAAKgd,UAAUC,GACfjd,KAAKsjD,gBAAgBgJ,EAAIC,GACzBvsD,KAAKwsD,wBAEsB,MAAvBL,EAA6B,CAC/B,GAAIM,GAAuBzsD,KAAK0sD,YAAYP,EAC5CnsD,MAAKwlC,KAAK3F,QAAQ9tB,EAAI06C,EAAqB16C,EAC3C/R,KAAKwlC,KAAK3F,QAAQ7tB,EAAIy6C,EAAqBz6C,EAY7C,MATAhS,MAAK6iD,UAEU5lC,EAAXivC,EACFlsD,KAAK4tB,KAAK,QAASsN,UAAU,MAG7Bl7B,KAAK4tB,KAAK,QAASsN,UAAU,MAGxBje,IAYX/Z,EAAQiQ,UAAUirB,cAAgB,SAAS90B,GAEzC,GAAImlB,GAAQ,CAYZ,IAXInlB,EAAMolB,WACRD,EAAQnlB,EAAMolB,WAAW,IAChBplB,EAAMqlB,SAGfF,GAASnlB,EAAMqlB,OAAO,GAMpBF,EAAO,CAET,GAAIxR,GAAQjd,KAAKyqD,YACbvqB,EAAOzR,EAAQ,EACP,GAARA,IACFyR,GAAe,EAAIA,GAErBjjB,GAAU,EAAIijB,CAGd,IAAIL,GAAU7/B,KAAKuqD,aAAax4C,EAAGzI,EAAMw2B,MAAO9tB,EAAG1I,EAAMy2B,OAGzD//B,MAAKisD,MAAMhvC,EAAO4iB,GAIpBv2B,EAAMD,kBASRnG,EAAQiQ,UAAUo2C,kBAAoB,SAAUjgD,GAC9C,GAAIu2B,GAAU7/B,KAAKuqD,aAAax4C,EAAGzI,EAAMw2B,MAAO9tB,EAAG1I,EAAMy2B,OAGrD//B,MAAK2sD,UACP3sD,KAAK4sD,gBAAgB/sB,EAKvB,IAAI1rB,GAAKnU,KACL6sD,EAAY,WACd14C,EAAG24C,gBAAgBjtB,GAarB,IAXI7/B,KAAK+sD,YACPt6B,cAAczyB,KAAK+sD,YAEhB/sD,KAAKwlC,KAAKzG,WACb/+B,KAAK+sD,WAAazzC,WAAWuzC,EAAW7sD,KAAK0hD,UAAUt7B,QAAQ5N,QAOrC,GAAxBxY,KAAK0hD,UAAUp1C,MAAe,CAEhC,IAAK,GAAI0gD,KAAUhtD,MAAK4hD,SAAS9D,MAC3B99C,KAAK4hD,SAAS9D,MAAMh4C,eAAeknD,KACrChtD,KAAK4hD,SAAS9D,MAAMkP,GAAQ1gD,OAAQ,QAC7BtM,MAAK4hD,SAAS9D,MAAMkP,GAK/B,IAAIjqC,GAAM/iB,KAAK4qD,WAAW/qB,EACf,OAAP9c,IACFA,EAAM/iB,KAAKitD,WAAWptB,IAEb,MAAP9c,GACF/iB,KAAKktD,aAAanqC,EAIpB,KAAK,GAAIojC,KAAUnmD,MAAK4hD,SAAS5E,MAC3Bh9C,KAAK4hD,SAAS5E,MAAMl3C,eAAeqgD,KACjCpjC,YAAexf,IAAQwf,EAAI1iB,IAAM8lD,GAAUpjC,YAAe3f,IAAe,MAAP2f,KACpE/iB,KAAKmtD,YAAYntD,KAAK4hD,SAAS5E,MAAMmJ,UAC9BnmD,MAAK4hD,SAAS5E,MAAMmJ,GAIjCnmD,MAAKyhB,WAYTve,EAAQiQ,UAAU25C,gBAAkB,SAAUjtB,GAC5C,GAOIx/B,GAPA0iB,GACFtb,KAAQzH,KAAKurD,qBAAqB1rB,EAAQ9tB,GAC1CpK,IAAQ3H,KAAKyrD,qBAAqB5rB,EAAQ7tB,GAC1CqV,MAAQrnB,KAAKurD,qBAAqB1rB,EAAQ9tB,GAC1CuR,OAAQtjB,KAAKyrD,qBAAqB5rB,EAAQ7tB,IAIxCo7C,EAAgBptD,KAAK2sD,SACrBU,GAAkB,CAEtB,IAAqB7mD,QAAjBxG,KAAK2sD,SAAuB,CAE9B,GAAI3P,GAAQh9C,KAAKg9C,MACbsQ,IACJ,KAAKjtD,IAAM28C,GACT,GAAIA,EAAMl3C,eAAezF,GAAK,CAC5B,GAAIylD,GAAO9I,EAAM38C,EACbylD,GAAKyH,kBAAkBxqC,IACDvc,SAApBs/C,EAAK0H,YACPF,EAAiBtlD,KAAK3H,GAM1BitD,EAAiB3nD,OAAS,IAG5B3F,KAAK2sD,SAAW3sD,KAAKg9C,MAAMsQ,EAAiBA,EAAiB3nD,OAAS,IAEtE0nD,GAAkB,GAItB,GAAsB7mD,SAAlBxG,KAAK2sD,UAA6C,GAAnBU,EAA0B,CAE3D,GAAIvP,GAAQ99C,KAAK89C,MACb2P,IACJ,KAAKptD,IAAMy9C,GACT,GAAIA,EAAMh4C,eAAezF,GAAK,CAC5B,GAAIqtD,GAAO5P,EAAMz9C,EACbqtD,GAAKC,WAAkCnnD,SAApBknD,EAAKF,YACxBE,EAAKH,kBAAkBxqC,IACzB0qC,EAAiBzlD,KAAK3H,GAKxBotD,EAAiB9nD,OAAS,IAC5B3F,KAAK2sD,SAAW3sD,KAAK89C,MAAM2P,EAAiBA,EAAiB9nD,OAAS,KAI1E,GAAI3F,KAAK2sD,UAEP,GAAI3sD,KAAK2sD,UAAYS,EAAe,CAClC,GAAIj5C,GAAKnU,IACJmU,GAAGy5C,QACNz5C,EAAGy5C,MAAQ,GAAIpqD,GAAM2Q,EAAGmL,MAAOnL,EAAGutC,UAAUt7B,UAM9CjS,EAAGy5C,MAAMC,YAAYhuB,EAAQ9tB,EAAI,EAAG8tB,EAAQ7tB,EAAI,GAChDmC,EAAGy5C,MAAME,QAAQ35C,EAAGw4C,SAASa,YAC7Br5C,EAAGy5C,MAAMjmB,YAIP3nC,MAAK4tD,OACP5tD,KAAK4tD,MAAMlmB,QAYjBxkC,EAAQiQ,UAAUy5C,gBAAkB,SAAU/sB,GACvC7/B,KAAK2sD,UAAa3sD,KAAK4qD,WAAW/qB,KACrC7/B,KAAK2sD,SAAWnmD,OACZxG,KAAK4tD,OACP5tD,KAAK4tD,MAAMlmB,SAajBxkC,EAAQiQ,UAAUwR,QAAU,SAASpS,EAAOC,GAC1C,GAAIu7C,IAAY,EACZC,EAAWhuD,KAAKsf,MAAMC,OAAOhN,MAC7B07C,EAAYjuD,KAAKsf,MAAMC,OAAO/M,MAC9BD,IAASvS,KAAK0hD,UAAUnvC,OAASC,GAAUxS,KAAK0hD,UAAUlvC,QAAUxS,KAAKsf,MAAMrS,MAAMsF,OAASA,GAASvS,KAAKsf,MAAMrS,MAAMuF,QAAUA,GACpIxS,KAAKsf,MAAMrS,MAAMsF,MAAQA,EACzBvS,KAAKsf,MAAMrS,MAAMuF,OAASA,EAE1BxS,KAAKsf,MAAMC,OAAOtS,MAAMsF,MAAQ,OAChCvS,KAAKsf,MAAMC,OAAOtS,MAAMuF,OAAS,OAEjCxS,KAAKsf,MAAMC,OAAOhN,MAAQvS,KAAKsf,MAAMC,OAAOC,YAAcxf,KAAK2hD,WAC/D3hD,KAAKsf,MAAMC,OAAO/M,OAASxS,KAAKsf,MAAMC,OAAOsF,aAAe7kB,KAAK2hD,WAEjE3hD,KAAK0hD,UAAUnvC,MAAQA,EACvBvS,KAAK0hD,UAAUlvC,OAASA,EAExBu7C,GAAY,IAMR/tD,KAAKsf,MAAMC,OAAOhN,OAASvS,KAAKsf,MAAMC,OAAOC,YAAcxf,KAAK2hD,aAClE3hD,KAAKsf,MAAMC,OAAOhN,MAAQvS,KAAKsf,MAAMC,OAAOC,YAAcxf,KAAK2hD,WAC/DoM,GAAY,GAEV/tD,KAAKsf,MAAMC,OAAO/M,QAAUxS,KAAKsf,MAAMC,OAAOsF,aAAe7kB,KAAK2hD,aACpE3hD,KAAKsf,MAAMC,OAAO/M,OAASxS,KAAKsf,MAAMC,OAAOsF,aAAe7kB,KAAK2hD,WACjEoM,GAAY,IAIC,GAAbA,GACF/tD,KAAK4tB,KAAK,UAAWrb,MAAMvS,KAAKsf,MAAMC,OAAOhN,MAAQvS,KAAK2hD,WAAWnvC,OAAOxS,KAAKsf,MAAMC,OAAO/M,OAASxS,KAAK2hD,WAAYqM,SAAUA,EAAWhuD,KAAK2hD,WAAYsM,UAAWA,EAAYjuD,KAAK2hD,cAS9Lz+C,EAAQiQ,UAAUs0C,UAAY,SAASzK,GACrC,GAAIkR,GAAeluD,KAAKokD,SAExB,IAAIpH,YAAiBn8C,IAAWm8C,YAAiBl8C,GAC/Cd,KAAKokD,UAAYpH,MAEd,IAAI/2C,MAAMC,QAAQ82C,GACrBh9C,KAAKokD,UAAY,GAAIvjD,GACrBb,KAAKokD,UAAUnxC,IAAI+pC,OAEhB,CAAA,GAAKA,EAIR,KAAM,IAAI32C,WAAU,4BAHpBrG,MAAKokD,UAAY,GAAIvjD,GAgBvB,GAVIqtD,GAEFvtD,EAAK0H,QAAQrI,KAAKskD,eAAgB,SAAUh8C,EAAUgB,GACpD4kD,EAAax6C,IAAIpK,EAAOhB,KAK5BtI,KAAKg9C,SAEDh9C,KAAKokD,UAAW,CAElB,GAAIjwC,GAAKnU,IACTW,GAAK0H,QAAQrI,KAAKskD,eAAgB,SAAUh8C,EAAUgB,GACpD6K,EAAGiwC,UAAU7wC,GAAGjK,EAAOhB,IAIzB,IAAI6M,GAAMnV,KAAKokD,UAAUvuC,QACzB7V,MAAKukD,UAAUpvC,GAEjBnV,KAAKmuD,oBAQPjrD,EAAQiQ,UAAUoxC,UAAY,SAASpvC,GAErC,IAAK,GADD9U,GACKmF,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IAAK,CAC9CnF,EAAK8U,EAAI3P,EACT,IAAIkN,GAAO1S,KAAKokD,UAAUlvC,IAAI7U,GAC1BylD,EAAO,GAAIviD,GAAKmP,EAAM1S,KAAK2iD,OAAQ3iD,KAAKm0B,OAAQn0B,KAAK0hD,UAEzD,IADA1hD,KAAKg9C,MAAM38C,GAAMylD,IACG,GAAfA,EAAKqF,QAAkC,GAAfrF,EAAKsF,QAAgC,OAAXtF,EAAK/zC,GAAyB,OAAX+zC,EAAK9zC,GAAa,CAC1F,GAAIyZ,GAAS,EAAStW,EAAIxP,OAAS,GAC/ByoD,EAAQ,EAAIlpD,KAAKymB,GAAKzmB,KAAKE,QACZ,IAAf0gD,EAAKqF,SAAkBrF,EAAK/zC,EAAI0Z,EAASvmB,KAAKqZ,IAAI6vC,IACnC,GAAftI,EAAKsF,SAAkBtF,EAAK9zC,EAAIyZ,EAASvmB,KAAKkZ,IAAIgwC,IAExDpuD,KAAK8kD,QAAS,EAGhB9kD,KAAKgnD,uBAC4C,GAA7ChnD,KAAK0hD,UAAUjB,mBAAmB/xC,SAAwC,GAArB1O,KAAK08C,eAC5D18C,KAAK4nD,eACL5nD,KAAKglD,4BAEPhlD,KAAKquD,0BACLruD,KAAKsuD,kBACLtuD,KAAKuuD,kBAAkBvuD,KAAKg9C,OAC5Bh9C,KAAKwuD,gBAQPtrD,EAAQiQ,UAAUqxC,aAAe,SAASrvC,EAAIs5C,GAE5C,IAAK,GADDzR,GAAQh9C,KAAKg9C,MACRx3C,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IAAK,CAC9C,GAAInF,GAAK8U,EAAI3P,GACTsgD,EAAO9I,EAAM38C,GACbqS,EAAO+7C,EAAYjpD,EACnBsgD,GAEFA,EAAK4I,cAAch8C,EAAM1S,KAAK0hD,YAI9BoE,EAAO,GAAIviD,GAAKorD,WAAY3uD,KAAK2iD,OAAQ3iD,KAAKm0B,OAAQn0B,KAAK0hD,WAC3D1E,EAAM38C,GAAMylD,GAGhB9lD,KAAK8kD,QAAS,EACmC,GAA7C9kD,KAAK0hD,UAAUjB,mBAAmB/xC,SAAwC,GAArB1O,KAAK08C,eAC5D18C,KAAK4nD,eACL5nD,KAAKglD,4BAEPhlD,KAAKgnD,uBACLhnD,KAAKuuD,kBAAkBvR,IAQzB95C,EAAQiQ,UAAUsxC,aAAe,SAAStvC,GAExC,IAAK,GADD6nC,GAAQh9C,KAAKg9C,MACRx3C,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IAAK,CAC9C,GAAInF,GAAK8U,EAAI3P,SACNw3C,GAAM38C,GAEfL,KAAKgnD,uBAC4C,GAA7ChnD,KAAK0hD,UAAUjB,mBAAmB/xC,SAAwC,GAArB1O,KAAK08C,eAC5D18C,KAAK4nD,eACL5nD,KAAKglD,4BAEPhlD,KAAKquD,0BACLruD,KAAKsuD,kBACLtuD,KAAKmuD,mBACLnuD,KAAKuuD,kBAAkBvR,IASzB95C,EAAQiQ,UAAUu0C,UAAY,SAAS5J,GACrC,GAAI8Q,GAAe5uD,KAAKqkD,SAExB,IAAIvG,YAAiBj9C,IAAWi9C,YAAiBh9C,GAC/Cd,KAAKqkD,UAAYvG,MAEd,IAAI73C,MAAMC,QAAQ43C,GACrB99C,KAAKqkD,UAAY,GAAIxjD,GACrBb,KAAKqkD,UAAUpxC,IAAI6qC,OAEhB,CAAA,GAAKA,EAIR,KAAM,IAAIz3C,WAAU,4BAHpBrG,MAAKqkD,UAAY,GAAIxjD,GAgBvB,GAVI+tD,GAEFjuD,EAAK0H,QAAQrI,KAAK0kD,eAAgB,SAAUp8C,EAAUgB,GACpDslD,EAAal7C,IAAIpK,EAAOhB,KAK5BtI,KAAK89C,SAED99C,KAAKqkD,UAAW,CAElB,GAAIlwC,GAAKnU,IACTW,GAAK0H,QAAQrI,KAAK0kD,eAAgB,SAAUp8C,EAAUgB,GACpD6K,EAAGkwC,UAAU9wC,GAAGjK,EAAOhB,IAIzB,IAAI6M,GAAMnV,KAAKqkD,UAAUxuC,QACzB7V,MAAK2kD,UAAUxvC,GAGjBnV,KAAKsuD,mBAQPprD,EAAQiQ,UAAUwxC,UAAY,SAAUxvC,GAItC,IAAK,GAHD2oC,GAAQ99C,KAAK89C,MACbuG,EAAYrkD,KAAKqkD,UAEZ7+C,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IAAK,CAC9C,GAAInF,GAAK8U,EAAI3P,GAETqpD,EAAU/Q,EAAMz9C,EAChBwuD,IACFA,EAAQC,YAGV,IAAIp8C,GAAO2xC,EAAUnvC,IAAI7U,GAAK0uD,iBAAoB,GAClDjR,GAAMz9C,GAAM,GAAI+C,GAAKsP,EAAM1S,KAAMA,KAAK0hD,WAExC1hD,KAAK8kD,QAAS,EACd9kD,KAAKuuD,kBAAkBzQ,GACvB99C,KAAKgvD,qBACLhvD,KAAKquD,0BAC4C,GAA7CruD,KAAK0hD,UAAUjB,mBAAmB/xC,SAAwC,GAArB1O,KAAK08C,eAC5D18C,KAAK4nD,eACL5nD,KAAKglD,6BAST9hD,EAAQiQ,UAAUyxC,aAAe,SAAUzvC,GAGzC,IAAK,GAFD2oC,GAAQ99C,KAAK89C,MACbuG,EAAYrkD,KAAKqkD,UACZ7+C,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IAAK,CAC9C,GAAInF,GAAK8U,EAAI3P,GAETkN,EAAO2xC,EAAUnvC,IAAI7U,GACrBqtD,EAAO5P,EAAMz9C,EACbqtD,IAEFA,EAAKoB,aACLpB,EAAKgB,cAAch8C,EAAM1S,KAAK0hD,WAC9BgM,EAAK5Q,YAIL4Q,EAAO,GAAItqD,GAAKsP,EAAM1S,KAAMA,KAAK0hD,WACjC1hD,KAAK89C,MAAMz9C,GAAMqtD,GAIrB1tD,KAAKgvD,qBAC4C,GAA7ChvD,KAAK0hD,UAAUjB,mBAAmB/xC,SAAwC,GAArB1O,KAAK08C,eAC5D18C,KAAK4nD,eACL5nD,KAAKglD,4BAEPhlD,KAAK8kD,QAAS,EACd9kD,KAAKuuD,kBAAkBzQ,IAQzB56C,EAAQiQ,UAAU0xC,aAAe,SAAU1vC,GAEzC,IAAK,GADD2oC,GAAQ99C,KAAK89C,MACRt4C,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IAAK,CAC9C,GAAInF,GAAK8U,EAAI3P,GACTkoD,EAAO5P,EAAMz9C,EACbqtD,KACc,MAAZA,EAAKuB,WACAjvD,MAAKkvD,QAAiB,QAAS,MAAExB,EAAKuB,IAAI5uD,IAEnDqtD,EAAKoB,mBACEhR,GAAMz9C,IAIjBL,KAAK8kD,QAAS,EACd9kD,KAAKuuD,kBAAkBzQ,GAC0B,GAA7C99C,KAAK0hD,UAAUjB,mBAAmB/xC,SAAwC,GAArB1O,KAAK08C,eAC5D18C,KAAK4nD,eACL5nD,KAAKglD,4BAEPhlD,KAAKquD,2BAOPnrD,EAAQiQ,UAAUm7C,gBAAkB,WAClC,GAAIjuD,GACA28C,EAAQh9C,KAAKg9C,MACbc,EAAQ99C,KAAK89C,KACjB,KAAKz9C,IAAM28C,GACLA,EAAMl3C,eAAezF,KACvB28C,EAAM38C,GAAIy9C,SACVd,EAAM38C,GAAI8uD,gBAId,KAAK9uD,IAAMy9C,GACT,GAAIA,EAAMh4C,eAAezF,GAAK,CAC5B,GAAIqtD,GAAO5P,EAAMz9C,EACjBqtD,GAAKtkC,KAAO,KACZskC,EAAKrkC,GAAK,KACVqkC,EAAK5Q,YAaX55C,EAAQiQ,UAAUo7C,kBAAoB,SAASxrC,GAC7C,GAAI1iB,GAGA6b,EAAW1V,OACX2V,EAAW3V,MACf,KAAKnG,IAAM0iB,GACT,GAAIA,EAAIjd,eAAezF,GAAK,CAC1B,GAAIgH,GAAQ0b,EAAI1iB,GAAIuU,UACNpO,UAAVa,IACF6U,EAAyB1V,SAAb0V,EAA0B7U,EAAQnC,KAAK4G,IAAIzE,EAAO6U,GAC9DC,EAAyB3V,SAAb2V,EAA0B9U,EAAQnC,KAAKwH,IAAIrF,EAAO8U,IAMpE,GAAiB3V,SAAb0V,GAAuC1V,SAAb2V,EAC5B,IAAK9b,IAAM0iB,GACLA,EAAIjd,eAAezF,IACrB0iB,EAAI1iB,GAAI+uD,cAAclzC,EAAUC,IAUxCjZ,EAAQiQ,UAAUsO,OAAS,WACzBzhB,KAAK2kB,QAAQ3kB,KAAK0hD,UAAUnvC,MAAOvS,KAAK0hD,UAAUlvC,QAClDxS,KAAK6iD,WAQP3/C,EAAQiQ,UAAU0vC,QAAU,SAAS3pB,GACnC,GAAInS,GAAM/mB,KAAKsf,MAAMC,OAAOyH,WAAW,KAEvCD,GAAIkiC,aAAajpD,KAAK2hD,WAAY,EAAG,EAAG3hD,KAAK2hD,WAAY,EAAG,EAG5D,IAAI0N,GAAIrvD,KAAKsf,MAAMC,OAAOhN,MAASvS,KAAK2hD,WACpCh2C,EAAI3L,KAAKsf,MAAMC,OAAO/M,OAAUxS,KAAK2hD,UACzC56B,GAAIE,UAAU,EAAG,EAAGooC,EAAG1jD,GAGvBob,EAAIuoC,OACJvoC,EAAIwoC,UAAUvvD,KAAKyd,YAAY1L,EAAG/R,KAAKyd,YAAYzL,GACnD+U,EAAI9J,MAAMjd,KAAKid,MAAOjd,KAAKid,OAE3Bjd,KAAK+jD,eACHhyC,EAAK/R,KAAKurD,qBAAqB,GAC/Bv5C,EAAKhS,KAAKyrD,qBAAqB,IAEjCzrD,KAAKgkD,mBACHjyC,EAAK/R,KAAKurD,qBAAqBvrD,KAAKsf,MAAMC,OAAOC,YAAcxf,KAAK2hD,YACpE3vC,EAAKhS,KAAKyrD,qBAAqBzrD,KAAKsf,MAAMC,OAAOsF,aAAe7kB,KAAK2hD,aAGvD,GAAVzoB,IACJl5B,KAAKwvD,gBAAgB,sBAAuBzoC,IAClB,GAAtB/mB,KAAKwlC,KAAKzG,UAA4Cv4B,SAAvBxG,KAAKwlC,KAAKzG,UAA4D,GAAlC/+B,KAAK0hD,UAAUF,kBACpFxhD,KAAKwvD,gBAAgB,aAAczoC,KAIb,GAAtB/mB,KAAKwlC,KAAKzG,UAA4Cv4B,SAAvBxG,KAAKwlC,KAAKzG,UAA4D,GAAlC/+B,KAAK0hD,UAAUD,kBACpFzhD,KAAKwvD,gBAAgB,aAAazoC,GAAI,GAGxB,GAAVmS,GAC2B,GAA3Bl5B,KAAK6hD,oBACP7hD,KAAKwvD,gBAAgB,oBAAqBzoC,GAQ9CA,EAAI0oC,UAEU,GAAVv2B,GACFnS,EAAIE,UAAU,EAAG,EAAGooC,EAAG1jD,IAU3BzI,EAAQiQ,UAAUmwC,gBAAkB,SAASoM,EAASC,GAC3BnpD,SAArBxG,KAAKyd,cACPzd,KAAKyd,aACH1L,EAAG,EACHC,EAAG,IAISxL,SAAZkpD,IACF1vD,KAAKyd,YAAY1L,EAAI29C,GAEPlpD,SAAZmpD,IACF3vD,KAAKyd,YAAYzL,EAAI29C,GAGvB3vD,KAAK4tB,KAAK,gBAQZ1qB,EAAQiQ,UAAU03C,gBAAkB,WAClC,OACE94C,EAAG/R,KAAKyd,YAAY1L,EACpBC,EAAGhS,KAAKyd,YAAYzL,IASxB9O,EAAQiQ,UAAU6J,UAAY,SAASC,GACrCjd,KAAKid,MAAQA,GAQf/Z,EAAQiQ,UAAUs3C,UAAY,WAC5B,MAAOzqD,MAAKid,OAUd/Z,EAAQiQ,UAAUo4C,qBAAuB,SAASx5C,GAChD,OAAQA,EAAI/R,KAAKyd,YAAY1L,GAAK/R,KAAKid,OAUzC/Z,EAAQiQ,UAAUq4C,qBAAuB,SAASz5C,GAChD,MAAOA,GAAI/R,KAAKid,MAAQjd,KAAKyd,YAAY1L,GAU3C7O,EAAQiQ,UAAUs4C,qBAAuB,SAASz5C,GAChD,OAAQA,EAAIhS,KAAKyd,YAAYzL,GAAKhS,KAAKid,OAUzC/Z,EAAQiQ,UAAUu4C,qBAAuB,SAAS15C,GAChD,MAAOA,GAAIhS,KAAKid,MAAQjd,KAAKyd,YAAYzL,GAU3C9O,EAAQiQ,UAAUu5C,YAAc,SAAUnnC,GACxC,OAAQxT,EAAG/R,KAAKwrD,qBAAqBjmC,EAAIxT,GAAIC,EAAGhS,KAAK0rD,qBAAqBnmC,EAAIvT,KAShF9O,EAAQiQ,UAAUi5C,YAAc,SAAU7mC,GACxC,OAAQxT,EAAG/R,KAAKurD,qBAAqBhmC,EAAIxT,GAAIC,EAAGhS,KAAKyrD,qBAAqBlmC,EAAIvT,KAUhF9O,EAAQiQ,UAAUy8C,WAAa,SAAS7oC,EAAI8oC,GACvBrpD,SAAfqpD,IACFA,GAAa,EAIf,IAAI7S,GAAQh9C,KAAKg9C,MACb3J,IAEJ,KAAK,GAAIhzC,KAAM28C,GACTA,EAAMl3C,eAAezF,KACvB28C,EAAM38C,GAAIyvD,eAAe9vD,KAAKid,MAAMjd,KAAK+jD,cAAc/jD,KAAKgkD,mBACxDhH,EAAM38C,GAAIyqD,aACZzX,EAASrrC,KAAK3H,IAGV28C,EAAM38C,GAAI0vD,UAAYF,IACxB7S,EAAM38C,GAAI0rC,KAAKhlB,GAOvB,KAAK,GAAInb,GAAI,EAAGokD,EAAO3c,EAAS1tC,OAAYqqD,EAAJpkD,EAAUA,KAC5CoxC,EAAM3J,EAASznC,IAAImkD,UAAYF,IACjC7S,EAAM3J,EAASznC,IAAImgC,KAAKhlB,IAW9B7jB,EAAQiQ,UAAU88C,WAAa,SAASlpC,GACtC,GAAI+2B,GAAQ99C,KAAK89C,KACjB,KAAK,GAAIz9C,KAAMy9C,GACb,GAAIA,EAAMh4C,eAAezF,GAAK,CAC5B,GAAIqtD,GAAO5P,EAAMz9C,EACjBqtD,GAAK1qB,SAAShjC,KAAKid,OACfywC,EAAKC,WACP7P,EAAMz9C,GAAI0rC,KAAKhlB,KAYvB7jB,EAAQiQ,UAAU+8C,kBAAoB,SAASnpC,GAC7C,GAAI+2B,GAAQ99C,KAAK89C,KACjB,KAAK,GAAIz9C,KAAMy9C,GACTA,EAAMh4C,eAAezF,IACvBy9C,EAAMz9C,GAAI6vD,kBAAkBnpC,IASlC7jB,EAAQiQ,UAAU00C,WAAa,WACgB,GAAzC7nD,KAAK0hD,UAAUb,wBACjB7gD,KAAKmwD,qBAKP,KADA,GAAIn5C,GAAQ,EACLhX,KAAK8kD,QAAU9tC,EAAQhX,KAAK0hD,UAAUN,yBAC3CphD,KAAKowD,eACLp5C,GAG0C,IAAxChX,KAAK0hD,UAAUL,uBACjBrhD,KAAKilD,WAAWz+C,QAAW,GAAO,GAGS,GAAzCxG,KAAK0hD,UAAUb,wBACjB7gD,KAAKqwD,uBAUTntD,EAAQiQ,UAAUg9C,oBAAsB,WACtC,GAAInT,GAAQh9C,KAAKg9C,KACjB,KAAK,GAAI38C,KAAM28C,GACTA,EAAMl3C,eAAezF,IACJ,MAAf28C,EAAM38C,GAAI0R,GAA4B,MAAfirC,EAAM38C,GAAI2R,IACnCgrC,EAAM38C,GAAIiwD,UAAUv+C,EAAIirC,EAAM38C,GAAI8qD,OAClCnO,EAAM38C,GAAIiwD,UAAUt+C,EAAIgrC,EAAM38C,GAAI+qD,OAClCpO,EAAM38C,GAAI8qD,QAAS,EACnBnO,EAAM38C,GAAI+qD,QAAS,IAW3BloD,EAAQiQ,UAAUk9C,oBAAsB,WACtC,GAAIrT,GAAQh9C,KAAKg9C,KACjB,KAAK,GAAI38C,KAAM28C,GACTA,EAAMl3C,eAAezF,IACM,MAAzB28C,EAAM38C,GAAIiwD,UAAUv+C,IACtBirC,EAAM38C,GAAI8qD,OAASnO,EAAM38C,GAAIiwD,UAAUv+C,EACvCirC,EAAM38C,GAAI+qD,OAASpO,EAAM38C,GAAIiwD,UAAUt+C,IAa/C9O,EAAQiQ,UAAUo9C,UAAY,SAASC,GACrC,GAAIxT,GAAQh9C,KAAKg9C,KACjB,KAAK,GAAI38C,KAAM28C,GACb,GAAIA,EAAMl3C,eAAezF,IAAO28C,EAAM38C,GAAIowD,SAASD,GACjD,OAAO,CAGX,QAAO,GAUTttD,EAAQiQ,UAAUu9C,mBAAqB,WACrC,GAEIvK,GAFA3zB,EAAWxyB,KAAKy8C,wBAChBO,EAAQh9C,KAAKg9C,MAEb2T,GAAe,CAEnB,IAAI3wD,KAAK0hD,UAAUT,YAAc,EAC/B,IAAKkF,IAAUnJ,GACTA,EAAMl3C,eAAeqgD,KACvBnJ,EAAMmJ,GAAQyK,oBAAoBp+B,EAAUxyB,KAAK0hD,UAAUT,aAC3D0P,GAAe,OAKnB,KAAKxK,IAAUnJ,GACTA,EAAMl3C,eAAeqgD,KACvBnJ,EAAMmJ,GAAQ0K,aAAar+B,GAC3Bm+B,GAAe,EAKrB,IAAoB,GAAhBA,EAAsB,CACxB,GAAIG,GAAgB9wD,KAAK0hD,UAAUR,YAAch8C,KAAKwH,IAAI1M,KAAKid,MAAM,IACrE,OAAI6zC,GAAgB,GAAI9wD,KAAK0hD,UAAUT,aAC9B,EAGAjhD,KAAKuwD,UAAUO,GAG1B,OAAO,GAIT5tD,EAAQiQ,UAAU49C,oBAAsB,WACtC,GAAI/T,GAAQh9C,KAAKg9C,KACjB,KAAK,GAAImJ,KAAUnJ,GACbA,EAAMl3C,eAAeqgD,IACvBnJ,EAAMmJ,GAAQ6K,kBAKpB9tD,EAAQiQ,UAAU89C,mBAAqB,WACrCjxD,KAAKkxD,sBAAsB,uBACgB,GAAvClxD,KAAK0hD,UAAUZ,aAAapyC,SAA0D,GAAvC1O,KAAK0hD,UAAUZ,aAAaC,SAC7E/gD,KAAKmxD,mBAAmB,wBAS5BjuD,EAAQiQ,UAAUi9C,aAAe,WAC/B,IAAKpwD,KAAKujD,kBACW,GAAfvjD,KAAK8kD,OAAgB,CACvB,GAAIsM,IAAmB,EACnBC,GAAsB,CAE1BrxD,MAAKkxD,sBAAsB,8BAC3B,IAAII,GAAatxD,KAAKkxD,sBAAsB,qBACD,IAAvClxD,KAAK0hD,UAAUZ,aAAapyC,SAA0D,GAAvC1O,KAAK0hD,UAAUZ,aAAaC,UAC7EsQ,EAAsBrxD,KAAKmxD,mBAAmB,sBAIhD,KAAK,GAAI3rD,GAAI,EAAGA,EAAI8rD,EAAW3rD,OAAQH,IAAM4rD,EAAmBE,EAAW,IAAMF,CAGjFpxD,MAAK8kD,OAASsM,GAAoBC,EAEf,GAAfrxD,KAAK8kD,OACP9kD,KAAKixD,qBAI4B,GAA7BjxD,KAAKyjD,uBACPzjD,KAAK4tB,KAAK,sBACV5tB,KAAKyjD,sBAAuB,GAIhCzjD,KAAKohD,4BAYXl+C,EAAQiQ,UAAUo+C,eAAiB,WAEjCvxD,KAAK+kD,MAAQv+C,OAGbxG,KAAKwxD,mBAEL,IAAIC,GAAYntD,KAAK64B,KACrBn9B,MAAKowD,cACL,IAAI7T,GAAcj4C,KAAK64B,MAAQs0B,GAG1BzxD,KAAKq8C,eAAiBr8C,KAAKs8C,WAAa,EAAIC,GAAsC,GAAvBv8C,KAAKw8C,iBAA2C,GAAfx8C,KAAK8kD,SACpG9kD,KAAKowD,eAGkB,GAAnBpwD,KAAKs8C,aACPt8C,KAAKw8C,gBAAiB,GAI1B,IAAIkV,GAAkBptD,KAAK64B,KAC3Bn9B,MAAK6iD,UACL7iD,KAAKs8C,WAAah4C,KAAK64B,MAAQu0B,EAG/B1xD,KAAK4P,SAGe,mBAAXrG,UACTA,OAAOooD,sBAAwBpoD,OAAOooD,uBAAyBpoD,OAAOqoD,0BACvCroD,OAAOsoD,6BAA+BtoD,OAAOuoD,yBAM9E5uD,EAAQiQ,UAAUvD,MAAQ,WACxB,GAAmB,GAAf5P,KAAK8kD,QAAqC,GAAnB9kD,KAAK8iD,YAAsC,GAAnB9iD,KAAK+iD,YAAyC,GAAtB/iD,KAAKgjD,cACzEhjD,KAAK+kD,QAEN/kD,KAAK+kD,MADqB,GAAxB/kD,KAAKwlD,gBACMj8C,OAAO+P,WAAWtZ,KAAKuxD,eAAez8B,KAAK90B,MAAOA,KAAKq8C,gBAGvD9yC,OAAOooD,sBAAsB3xD,KAAKuxD,eAAez8B,KAAK90B,YAOvE,IAFAA,KAAK6iD,UAED7iD,KAAKohD,wBAA0B,EAAG,CAKpC,GAAIjtC,GAAKnU,KACL8T,GACFi+C,WAAY59C,EAAGitC,wBAEjBphD,MAAKohD,wBAA0B,EAC/BphD,KAAKyjD,sBAAuB,EAC5BnqC,WAAW,WACTnF,EAAGyZ,KAAK,aAAc9Z,IACrB,OAGH9T,MAAKohD,wBAA0B,GAWrCl+C,EAAQiQ,UAAUq+C,kBAAoB,WACpC,GAAuB,GAAnBxxD,KAAK8iD,YAAsC,GAAnB9iD,KAAK+iD,WAAiB,CAChD,GAAItlC,GAAczd,KAAK6qD,iBACvB7qD,MAAKsjD,gBAAgB7lC,EAAY1L,EAAE/R,KAAK8iD,WAAYrlC,EAAYzL,EAAEhS,KAAK+iD,YAEzE,GAA0B,GAAtB/iD,KAAKgjD,cAAoB,CAC3B,GAAI92B,IACFna,EAAG/R,KAAKsf,MAAMC,OAAOC,YAAc,EACnCxN,EAAGhS,KAAKsf,MAAMC,OAAOsF,aAAe,EAEtC7kB,MAAKisD,MAAMjsD,KAAKid,OAAO,EAAIjd,KAAKgjD,eAAgB92B,KAQpDhpB,EAAQiQ,UAAU6+C,aAAe,WACF,GAAzBhyD,KAAKujD,iBACPvjD,KAAKujD,kBAAmB,GAGxBvjD,KAAKujD,kBAAmB,EACxBvjD,KAAK4P,UAWT1M,EAAQiQ,UAAUu1C,uBAAyB,SAASlC,GAIlD,GAHqBhgD,SAAjBggD,IACFA,GAAe,GAE0B,GAAvCxmD,KAAK0hD,UAAUZ,aAAapyC,SAA0D,GAAvC1O,KAAK0hD,UAAUZ,aAAaC,QAAiB,CAC9F/gD,KAAKgvD,oBAEL,KAAK,GAAI7I,KAAUnmD,MAAKkvD,QAAiB,QAAS,MAC5ClvD,KAAKkvD,QAAiB,QAAS,MAAEppD,eAAeqgD,IACwB3/C,SAAtExG,KAAK89C,MAAM99C,KAAKkvD,QAAiB,QAAS,MAAE/I,GAAQ8L,qBAC/CjyD,MAAKkvD,QAAiB,QAAS,MAAE/I,OAK3C,CAEHnmD,KAAKkvD,QAAiB,QAAS,QAC/B,KAAK,GAAIlC,KAAUhtD,MAAK89C,MAClB99C,KAAK89C,MAAMh4C,eAAeknD,KAC5BhtD,KAAK89C,MAAMkP,GAAQiC,IAAM,MAM/BjvD,KAAKquD,0BACA7H,IACHxmD,KAAK8kD,QAAS,EACd9kD,KAAK4P,UAWT1M,EAAQiQ,UAAU67C,mBAAqB,WACrC,GAA2C,GAAvChvD,KAAK0hD,UAAUZ,aAAapyC,SAA0D,GAAvC1O,KAAK0hD,UAAUZ,aAAaC,QAC7E,IAAK,GAAIiM,KAAUhtD,MAAK89C,MACtB,GAAI99C,KAAK89C,MAAMh4C,eAAeknD,GAAS,CACrC,GAAIU,GAAO1tD,KAAK89C,MAAMkP,EACtB,IAAgB,MAAZU,EAAKuB,IAAa,CACpB,GAAI9I,GAAS,UAAUnyC,OAAO05C,EAAKrtD,GACnCL,MAAKkvD,QAAiB,QAAS,MAAE/I,GAAU,GAAI5iD,IACtClD,GAAG8lD,EACFlJ,KAAK,EACLG,MAAM,SACNC,MAAM,GACN6U,mBAAmB,SACblyD,KAAK0hD,WACrBgM,EAAKuB,IAAMjvD,KAAKkvD,QAAiB,QAAS,MAAE/I,GAC5CuH,EAAKuB,IAAIgD,aAAevE,EAAKrtD,GAC7BqtD,EAAKyE,wBAYfjvD,EAAQiQ,UAAUgpC,wBAA0B,WAC1C,IAAK,GAAIiW,KAAS/M,GACZA,EAAYv/C,eAAessD,KAC7BlvD,EAAQiQ,UAAUi/C,GAAS/M,EAAY+M,KAQ7ClvD,EAAQiQ,UAAUk/C,cAAgB,WAChC15B,QAAQhF,IAAI,mEACZ3zB,KAAKsyD,kBAMPpvD,EAAQiQ,UAAUm/C,eAAiB,WACjC,GAAIC,KACJ,KAAK,GAAIpM,KAAUnmD,MAAKg9C,MACtB,GAAIh9C,KAAKg9C,MAAMl3C,eAAeqgD,GAAS,CACrC,GAAIL,GAAO9lD,KAAKg9C,MAAMmJ,GAClBqM,GAAkBxyD,KAAKg9C,MAAMmO,OAC7BsH,GAAkBzyD,KAAKg9C,MAAMoO,QAC7BprD,KAAKokD,UAAUxxC,MAAMuzC,GAAQp0C,GAAK7M,KAAKwoB,MAAMo4B,EAAK/zC,IAAM/R,KAAKokD,UAAUxxC,MAAMuzC,GAAQn0C,GAAK9M,KAAKwoB,MAAMo4B,EAAK9zC,KAC5GugD,EAAUvqD,MAAM3H,GAAG8lD,EAAOp0C,EAAE7M,KAAKwoB,MAAMo4B,EAAK/zC,GAAGC,EAAE9M,KAAKwoB,MAAMo4B,EAAK9zC,GAAGwgD,eAAeA,EAAeC,eAAeA,IAIvHzyD,KAAKokD,UAAUvvC,OAAO09C,IAMxBrvD,EAAQiQ,UAAUu/C,aAAe,SAASv9C,GACxC,GAAIo9C,KACJ,IAAY/rD,SAAR2O,GACF,GAA0B,GAAtBlP,MAAMC,QAAQiP,IAChB,IAAK,GAAI3P,GAAI,EAAGA,EAAI2P,EAAIxP,OAAQH,IAC9B,GAA2BgB,SAAvBxG,KAAKg9C,MAAM7nC,EAAI3P,IAAmB,CACpC,GAAIsgD,GAAO9lD,KAAKg9C,MAAM7nC,EAAI3P,GAC1B+sD,GAAUp9C,EAAI3P,KAAOuM,EAAG7M,KAAKwoB,MAAMo4B,EAAK/zC,GAAIC,EAAG9M,KAAKwoB,MAAMo4B,EAAK9zC,SAKnE,IAAwBxL,SAApBxG,KAAKg9C,MAAM7nC,GAAoB,CACjC,GAAI2wC,GAAO9lD,KAAKg9C,MAAM7nC,EACtBo9C,GAAUp9C,IAAQpD,EAAG7M,KAAKwoB,MAAMo4B,EAAK/zC,GAAIC,EAAG9M,KAAKwoB,MAAMo4B,EAAK9zC,SAKhE,KAAK,GAAIm0C,KAAUnmD,MAAKg9C,MACtB,GAAIh9C,KAAKg9C,MAAMl3C,eAAeqgD,GAAS,CACrC,GAAIL,GAAO9lD,KAAKg9C,MAAMmJ,EACtBoM,GAAUpM,IAAWp0C,EAAG7M,KAAKwoB,MAAMo4B,EAAK/zC,GAAIC,EAAG9M,KAAKwoB,MAAMo4B,EAAK9zC,IAIrE,MAAOugD,IAWTrvD,EAAQiQ,UAAUw/C,YAAc,SAAUxM,EAAQ13C,GAChD,GAAIzO,KAAKg9C,MAAMl3C,eAAeqgD,GAAS,CACrB3/C,SAAZiI,IACFA,KAEF,IAAImkD,IAAgB7gD,EAAG/R,KAAKg9C,MAAMmJ,GAAQp0C,EAAGC,EAAGhS,KAAKg9C,MAAMmJ,GAAQn0C,EACnEvD,GAAQmV,SAAWgvC,EACnBnkD,EAAQokD,aAAe1M,EAEvBnmD,KAAK6nB,OAAOpZ,OAGZkqB,SAAQhF,IAAI,iCAWhBzwB,EAAQiQ,UAAU0U,OAAS,SAAUpZ,GACnC,MAAgBjI,UAAZiI,OACFA,OAGwBjI,SAAtBiI,EAAQkb,SAAoClb,EAAQkb,QAAa5X,EAAG,EAAGC,EAAG,IACpDxL,SAAtBiI,EAAQkb,OAAO5X,IAA6BtD,EAAQkb,OAAO5X,EAAK,GAC1CvL,SAAtBiI,EAAQkb,OAAO3X,IAA6BvD,EAAQkb,OAAO3X,EAAK,GAC1CxL,SAAtBiI,EAAQwO,QAAoCxO,EAAQwO,MAAYjd,KAAKyqD,aAC/CjkD,SAAtBiI,EAAQmV,WAAoCnV,EAAQmV,SAAY5jB,KAAK6qD,mBAC/CrkD,SAAtBiI,EAAQs4C,YAAoCt4C,EAAQs4C,WAAaj3C,SAAS,IAC1ErB,EAAQs4C,aAAc,IAAsBt4C,EAAQs4C,WAAaj3C,SAAS,IAC1ErB,EAAQs4C,aAAc,IAAsBt4C,EAAQs4C,cACrBvgD,SAA/BiI,EAAQs4C,UAAUj3C,WAA0BrB,EAAQs4C,UAAUj3C,SAAW,KACpCtJ,SAArCiI,EAAQs4C,UAAU+L,iBAAgCrkD,EAAQs4C,UAAU+L,eAAiB,qBAEzF9yD,MAAK+yD,YAAYtkD,KAcnBvL,EAAQiQ,UAAU4/C,YAAc,SAAUtkD,GACxC,GAAgBjI,SAAZiI,EAEF,YADAA,KAKFzO,MAAKsrD,cACiB,GAAlB78C,EAAQukD,SACVhzD,KAAKwiD,eAAiB/zC,EAAQokD,aAC9B7yD,KAAKyiD,mBAAqBh0C,EAAQkb,QAIb,GAAnB3pB,KAAKmiD,YACPniD,KAAKizD,kBAAkB,GAGzBjzD,KAAKoiD,YAAcpiD,KAAKyqD,YACxBzqD,KAAKsiD,kBAAoBtiD,KAAK6qD,kBAC9B7qD,KAAKqiD,YAAc5zC,EAAQwO,MAI3Bjd,KAAKgd,UAAUhd,KAAKqiD,YACpB,IAAI6Q,GAAalzD,KAAKosD,aAAar6C,EAAG,GAAM/R,KAAKsf,MAAMC,OAAOC,YAAaxN,EAAG,GAAMhS,KAAKsf,MAAMC,OAAOsF,eAClGsuC,GACFphD,EAAGmhD,EAAWnhD,EAAItD,EAAQmV,SAAS7R,EACnCC,EAAGkhD,EAAWlhD,EAAIvD,EAAQmV,SAAS5R,EAErChS,MAAKuiD,mBACHxwC,EAAG/R,KAAKsiD,kBAAkBvwC,EAAIohD,EAAmBphD,EAAI/R,KAAKqiD,YAAc5zC,EAAQkb,OAAO5X,EACvFC,EAAGhS,KAAKsiD,kBAAkBtwC,EAAImhD,EAAmBnhD,EAAIhS,KAAKqiD,YAAc5zC,EAAQkb,OAAO3X,GAIvD,GAA9BvD,EAAQs4C,UAAUj3C,SACO,MAAvB9P,KAAKwiD,gBACPxiD,KAAKozD,eAAiBpzD,KAAK6iD,QAC3B7iD,KAAK6iD,QAAU7iD,KAAKqzD,gBAGpBrzD,KAAKgd,UAAUhd,KAAKqiD,aACpBriD,KAAKsjD,gBAAgBtjD,KAAKuiD,kBAAkBxwC,EAAG/R,KAAKuiD,kBAAkBvwC,GACtEhS,KAAK6iD,YAIP7iD,KAAKiiD,eAAiB,GAAKjiD,KAAKo8C,kBAAoB3tC,EAAQs4C,UAAUj3C,SAAW,OAAU,EAAI9P,KAAKo8C,kBACpGp8C,KAAKkiD,wBAA0BzzC,EAAQs4C,UAAU+L,eACjD9yD,KAAKozD,eAAiBpzD,KAAK6iD,QAC3B7iD,KAAK6iD,QAAU7iD,KAAKizD,kBACpBjzD,KAAK6iD,UACL7iD,KAAK8kD,QAAS,EACd9kD,KAAK4P,UAQT1M,EAAQiQ,UAAUkgD,cAAgB,WAChC,GAAIT,IAAgB7gD,EAAG/R,KAAKg9C,MAAMh9C,KAAKwiD,gBAAgBzwC,EAAGC,EAAGhS,KAAKg9C,MAAMh9C,KAAKwiD,gBAAgBxwC,GACzFkhD,EAAalzD,KAAKosD,aAAar6C,EAAG,GAAM/R,KAAKsf,MAAMC,OAAOC,YAAaxN,EAAG,GAAMhS,KAAKsf,MAAMC,OAAOsF,eAClGsuC,GACFphD,EAAGmhD,EAAWnhD,EAAI6gD,EAAa7gD,EAC/BC,EAAGkhD,EAAWlhD,EAAI4gD,EAAa5gD,GAE7BswC,EAAoBtiD,KAAK6qD,kBACzBtI,GACFxwC,EAAGuwC,EAAkBvwC,EAAIohD,EAAmBphD,EAAI/R,KAAKid,MAAQjd,KAAKyiD,mBAAmB1wC,EACrFC,EAAGswC,EAAkBtwC,EAAImhD,EAAmBnhD,EAAIhS,KAAKid,MAAQjd,KAAKyiD,mBAAmBzwC,EAGvFhS,MAAKsjD,gBAAgBf,EAAkBxwC,EAAEwwC,EAAkBvwC,GAC3DhS,KAAKozD,kBAGPlwD,EAAQiQ,UAAUm4C,YAAc,WACH,MAAvBtrD,KAAKwiD,iBACPxiD,KAAK6iD,QAAU7iD,KAAKozD,eACpBpzD,KAAKwiD,eAAiB,KACtBxiD,KAAKyiD,mBAAqB,OAS9Bv/C,EAAQiQ,UAAU8/C,kBAAoB,SAAU9Q,GAC9CniD,KAAKmiD,WAAaA,GAAcniD,KAAKmiD,WAAaniD,KAAKiiD,eACvDjiD,KAAKmiD,YAAcniD,KAAKiiD,cAExB,IAAIxwB,GAAW9wB,EAAKqP,gBAAgBhQ,KAAKkiD,yBAAyBliD,KAAKmiD,WAEvEniD,MAAKgd,UAAUhd,KAAKoiD,aAAepiD,KAAKqiD,YAAcriD,KAAKoiD,aAAe3wB,GAC1EzxB,KAAKsjD,gBACHtjD,KAAKsiD,kBAAkBvwC,GAAK/R,KAAKuiD,kBAAkBxwC,EAAI/R,KAAKsiD,kBAAkBvwC,GAAK0f,EACnFzxB,KAAKsiD,kBAAkBtwC,GAAKhS,KAAKuiD,kBAAkBvwC,EAAIhS,KAAKsiD,kBAAkBtwC,GAAKyf,GAGrFzxB,KAAKozD,iBACLpzD,KAAK8kD,QAAS,EAGV9kD,KAAKmiD,YAAc,IACrBniD,KAAKmiD,WAAa,EAEhBniD,KAAK6iD,QADoB,MAAvB7iD,KAAKwiD,eACQxiD,KAAKqzD,cAGLrzD,KAAKozD,eAEtBpzD,KAAK4tB,KAAK,uBAId1qB,EAAQiQ,UAAUigD,eAAiB,aAQnClwD,EAAQiQ,UAAUs2C,SAAW,WAC3B,OAAQzpD,KAAKsoD,WAAatoD,KAAKsoD,UAAUgL,QAQ3CpwD,EAAQiQ,UAAU6vB,SAAW,WAC3B,MAAOhjC,MAAKgd,aAQd9Z,EAAQiQ,UAAUogD,SAAW,WAC3B,MAAOvzD,MAAKyqD,aAQdvnD,EAAQiQ,UAAUqgD,qBAAuB,WACvC,MAAOxzD,MAAKosD,aAAar6C,EAAG,GAAM/R,KAAKsf,MAAMC,OAAOC,YAAaxN,EAAG,GAAMhS,KAAKsf,MAAMC,OAAOsF,gBAI9F3hB,EAAQiQ,UAAUsgD,eAAiB,SAAStN,GAC1C,MAA2B3/C,UAAvBxG,KAAKg9C,MAAMmJ,GACNnmD,KAAKg9C,MAAMmJ,GAAQC,YAD5B,QAKFvmD,EAAOD,QAAUsD,GAKb,SAASrD,EAAQD,EAASM,GAoB9B,QAASkD,GAAMurD,EAAYxrD,EAASuwD,GAClC,IAAKvwD,EACH,KAAM,qBAER,IAAI+K,IAAU,QAAQ,WAClBwzC,EAAY/gD,EAAKsN,sBAAsBC,EAAOwlD,EAClD1zD,MAAKyO,QAAUizC,EAAU5D,MACzB99C,KAAKw+C,QAAUkD,EAAUlD,QACzBx+C,KAAKyO,QAAsB,aAAIilD,EAA+B,aAG9D1zD,KAAKmD,QAAUA,EAGfnD,KAAKK,GAASmG,OACdxG,KAAK2zD,OAASntD,OACdxG,KAAK4zD,KAASptD,OACdxG,KAAKglC,MAASx+B,OACdxG,KAAK6zD,cAAgB7zD,KAAKyO,QAAQ8D,MAAQvS,KAAKyO,QAAQsvC,yBACvD/9C,KAAKqH,MAASb,OACdxG,KAAKqzC,UAAW,EAChBrzC,KAAKsM,OAAQ,EACbtM,KAAK8zD,iBAAmBnsD,IAAI,EAAEF,KAAK,EAAE8K,MAAM,EAAEC,OAAO,EAAEuhD,MAAM,GAC5D/zD,KAAKg0D,YAAa,EAElBh0D,KAAKopB,KAAO,KACZppB,KAAKqpB,GAAK,KACVrpB,KAAKivD,IAAM,KAEXjvD,KAAKi0D,WAAa,KAClBj0D,KAAKk0D,SAAW,KAIhBl0D,KAAKm0D,kBACLn0D,KAAKo0D,gBAELp0D,KAAK2tD,WAAY,EAEjB3tD,KAAKq0D,YAAc,EACnBr0D,KAAKs0D,aAAc,EAEnBt0D,KAAK0uD,cAAcC,GAEnB3uD,KAAKu0D,qBAAsB,EAC3Bv0D,KAAKw0D,cAAgBprC,KAAK,KAAMC,GAAG,KAAMorC,cACzCz0D,KAAK00D,cAAgB,KAhEvB,GAAI/zD,GAAOT,EAAoB,GAC3BqD,EAAOrD,EAAoB,GAuE/BkD,GAAK+P,UAAUu7C,cAAgB,SAASC,GACtC,GAAKA,EAAL,CAIA,GAAIzgD,IAAU,QAAQ,WAAW,WAAW,YAAY,WAAW,kBAAkB,kBAAkB,QACrG,2BAA2B,aAAa,mBAAmB,OAAO,eAAe,iBAoCnF,QAlCAvN,EAAKwF,oBAAoB+H,EAAQlO,KAAKyO,QAASkgD,GAEvBnoD,SAApBmoD,EAAWvlC,OAA+BppB,KAAK2zD,OAAShF,EAAWvlC,MACjD5iB,SAAlBmoD,EAAWtlC,KAA+BrpB,KAAK4zD,KAAOjF,EAAWtlC,IAE/C7iB,SAAlBmoD,EAAWtuD,KAA+BL,KAAKK,GAAKsuD,EAAWtuD,IAC1CmG,SAArBmoD,EAAWlmC,QAA+BzoB,KAAKyoB,MAAQkmC,EAAWlmC,MAAOzoB,KAAKg0D,YAAa,GAEtExtD,SAArBmoD,EAAW3pB,QAA6BhlC,KAAKglC,MAAQ2pB,EAAW3pB,OAC3Cx+B,SAArBmoD,EAAWtnD,QAA6BrH,KAAKqH,MAAQsnD,EAAWtnD,OAC1Cb,SAAtBmoD,EAAWhpD,SAA6B3F,KAAKw+C,QAAQK,aAAe8P,EAAWhpD,QAE1Da,SAArBmoD,EAAWxjD,QACbnL,KAAKyO,QAAQ6vC,cAAe,EACxB39C,EAAKwD,SAASwqD,EAAWxjD,QAC3BnL,KAAKyO,QAAQtD,MAAMA,MAAQwjD,EAAWxjD,MACtCnL,KAAKyO,QAAQtD,MAAMkB,UAAYsiD,EAAWxjD,QAGX3E,SAA3BmoD,EAAWxjD,MAAMA,QAA0BnL,KAAKyO,QAAQtD,MAAMA,MAAQwjD,EAAWxjD,MAAMA,OACxD3E,SAA/BmoD,EAAWxjD,MAAMkB,YAA0BrM,KAAKyO,QAAQtD,MAAMkB,UAAYsiD,EAAWxjD,MAAMkB,WAChE7F,SAA3BmoD,EAAWxjD,MAAMmB,QAA0BtM,KAAKyO,QAAQtD,MAAMmB,MAAQqiD,EAAWxjD,MAAMmB,SAK/FtM,KAAK88C,UAEL98C,KAAKq0D,WAAar0D,KAAKq0D,YAAoC7tD,SAArBmoD,EAAWp8C,MACjDvS,KAAKs0D,YAAct0D,KAAKs0D,aAAsC9tD,SAAtBmoD,EAAWhpD,OAEnD3F,KAAK6zD,cAAgB7zD,KAAKyO,QAAQ8D,MAAOvS,KAAKyO,QAAQsvC,yBAG9C/9C,KAAKyO,QAAQxB,OACnB,IAAK,OAAiBjN,KAAK+rC,KAAO/rC,KAAK20D,SAAW,MAClD,KAAK,QAAiB30D,KAAK+rC,KAAO/rC,KAAK40D,UAAY,MACnD,KAAK,eAAiB50D,KAAK+rC,KAAO/rC,KAAK60D,gBAAkB,MACzD,KAAK,YAAiB70D,KAAK+rC,KAAO/rC,KAAK80D,aAAe,MACtD,SAAsB90D,KAAK+rC,KAAO/rC,KAAK20D,aAQ3CvxD,EAAK+P,UAAU2pC,QAAU,WACvB98C,KAAK8uD,aAEL9uD,KAAKopB,KAAOppB,KAAKmD,QAAQ65C,MAAMh9C,KAAK2zD,SAAW,KAC/C3zD,KAAKqpB,GAAKrpB,KAAKmD,QAAQ65C,MAAMh9C,KAAK4zD,OAAS,KAC3C5zD,KAAK2tD,UAAa3tD,KAAKopB,MAAQppB,KAAKqpB,GAEhCrpB,KAAK2tD,WACP3tD,KAAKopB,KAAK2rC,WAAW/0D,MACrBA,KAAKqpB,GAAG0rC,WAAW/0D,QAGfA,KAAKopB,MACPppB,KAAKopB,KAAK4rC,WAAWh1D,MAEnBA,KAAKqpB,IACPrpB,KAAKqpB,GAAG2rC,WAAWh1D,QAQzBoD,EAAK+P,UAAU27C,WAAa,WACtB9uD,KAAKopB,OACPppB,KAAKopB,KAAK4rC,WAAWh1D,MACrBA,KAAKopB,KAAO,MAEVppB,KAAKqpB,KACPrpB,KAAKqpB,GAAG2rC,WAAWh1D,MACnBA,KAAKqpB,GAAK,MAGZrpB,KAAK2tD,WAAY,GAQnBvqD,EAAK+P,UAAUq6C,SAAW,WACxB,MAA6B,kBAAfxtD,MAAKglC,MAAuBhlC,KAAKglC,QAAUhlC,KAAKglC;EAQhE5hC,EAAK+P,UAAUyB,SAAW,WACxB,MAAO5U,MAAKqH,OASdjE,EAAK+P,UAAUi8C,cAAgB,SAAStjD,EAAKY,GAC3C,IAAK1M,KAAKq0D,YAA6B7tD,SAAfxG,KAAKqH,MAAqB,CAChD,GAAI4V,IAASjd,KAAKyO,QAAQ0Y,SAAWnnB,KAAKyO,QAAQyY,WAAaxa,EAAMZ,EACrE9L,MAAKyO,QAAQ8D,OAAQvS,KAAKqH,MAAQyE,GAAOmR,EAAQjd,KAAKyO,QAAQyY,SAC9DlnB,KAAK6zD,cAAgB7zD,KAAKyO,QAAQ8D,MAAOvS,KAAKyO,QAAQsvC,2BAU1D36C,EAAK+P,UAAU44B,KAAO,WACpB,KAAM,uCAQR3oC,EAAK+P,UAAUo6C,kBAAoB,SAASxqC,GAC1C,GAAI/iB,KAAK2tD,UAAW,CAClB,GAAIv+B,GAAU,GACV6lC,EAAQj1D,KAAKopB,KAAKrX,EAClBmjD,EAAQl1D,KAAKopB,KAAKpX,EAClBmjD,EAAMn1D,KAAKqpB,GAAGtX,EACdqjD,EAAMp1D,KAAKqpB,GAAGrX,EACdqjD,EAAOtyC,EAAItb,KACX6tD,EAAOvyC,EAAIpb,IAEXujB,EAAOlrB,KAAKu1D,mBAAmBN,EAAOC,EAAOC,EAAKC,EAAKC,EAAMC,EAEjE,OAAelmC,GAAPlE,EAGR,OAAO,GAIX9nB,EAAK+P,UAAUqiD,UAAY,WACzB,GAAIC,GAAWz1D,KAAKyO,QAAQtD,KAgB5B,OAfiC,MAA7BnL,KAAKyO,QAAQ6vC,aACfmX,GACEppD,UAAWrM,KAAKqpB,GAAG5a,QAAQtD,MAAMkB,UAAUD,OAC3CE,MAAOtM,KAAKqpB,GAAG5a,QAAQtD,MAAMmB,MAAMF,OACnCjB,MAAOnL,KAAKqpB,GAAG5a,QAAQtD,MAAMiB,SAGK,QAA7BpM,KAAKyO,QAAQ6vC,cAAuD,GAA7Bt+C,KAAKyO,QAAQ6vC,gBAC3DmX,GACEppD,UAAWrM,KAAKopB,KAAK3a,QAAQtD,MAAMkB,UAAUD,OAC7CE,MAAOtM,KAAKopB,KAAK3a,QAAQtD,MAAMmB,MAAMF,OACrCjB,MAAOnL,KAAKopB,KAAK3a,QAAQtD,MAAMiB,SAId,GAAjBpM,KAAKqzC,SAA4BoiB,EAASppD,UACvB,GAAdrM,KAAKsM,MAAuBmpD,EAASnpD,MACTmpD,EAAStqD,OAWhD/H,EAAK+P,UAAUwhD,UAAY,SAAS5tC,GAKlC,GAHAA,EAAIY,YAAc3nB,KAAKw1D,YACvBzuC,EAAIO,UAActnB,KAAK01D,gBAEnB11D,KAAKopB,MAAQppB,KAAKqpB,GAAI,CAExB,GAGInX,GAHA+8C,EAAMjvD,KAAK21D,MAAM5uC,EAIrB,IAAI/mB,KAAKyoB,MAAO,CACd,GAAyC,GAArCzoB,KAAKyO,QAAQqyC,aAAapyC,SAA0B,MAAPugD,EAAa,CAC5D,GAAI2G,GAAY,IAAK,IAAK51D,KAAKopB,KAAKrX,EAAIk9C,EAAIl9C,GAAK,IAAK/R,KAAKqpB,GAAGtX,EAAIk9C,EAAIl9C,IAClE8jD,EAAY,IAAK,IAAK71D,KAAKopB,KAAKpX,EAAIi9C,EAAIj9C,GAAK,IAAKhS,KAAKqpB,GAAGrX,EAAIi9C,EAAIj9C,GACtEE,IAASH,EAAE6jD,EAAW5jD,EAAE6jD,OAGxB3jD,GAAQlS,KAAK81D,aAAa,GAE5B91D,MAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOvW,EAAMH,EAAGG,EAAMF,QAG3C,CACH,GAAID,GAAGC,EACHyZ,EAASzrB,KAAKw+C,QAAQK,aAAe,EACrCiH,EAAO9lD,KAAKopB,IACX08B,GAAKvzC,OACRuzC,EAAKkQ,OAAOjvC,GAEV++B,EAAKvzC,MAAQuzC,EAAKtzC,QACpBT,EAAI+zC,EAAK/zC,EAAI+zC,EAAKvzC,MAAQ,EAC1BP,EAAI8zC,EAAK9zC,EAAIyZ,IAGb1Z,EAAI+zC,EAAK/zC,EAAI0Z,EACbzZ,EAAI8zC,EAAK9zC,EAAI8zC,EAAKtzC,OAAS,GAE7BxS,KAAKi2D,QAAQlvC,EAAKhV,EAAGC,EAAGyZ,GACxBvZ,EAAQlS,KAAKk2D,eAAenkD,EAAGC,EAAGyZ,EAAQ,IAC1CzrB,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOvW,EAAMH,EAAGG,EAAMF,KAUhD5O,EAAK+P,UAAUuiD,cAAgB,WAC7B,MAAqB,IAAjB11D,KAAKqzC,SACCnuC,KAAKwH,IAAIxH,KAAK4G,IAAI9L,KAAK6zD,cAAe7zD,KAAKyO,QAAQ0Y,UAAW,GAAInnB,KAAKm2D,iBAG7D,GAAdn2D,KAAKsM,MACApH,KAAKwH,IAAIxH,KAAK4G,IAAI9L,KAAKyO,QAAQuvC,WAAYh+C,KAAKyO,QAAQ0Y,UAAW,GAAInnB,KAAKm2D,iBAG5EjxD,KAAKwH,IAAI1M,KAAKyO,QAAQ8D,MAAO,GAAIvS,KAAKm2D,kBAKnD/yD,EAAK+P,UAAUijD,mBAAqB,WAClC,GAAyC,GAArCp2D,KAAKyO,QAAQqyC,aAAaC,SAAwD,GAArC/gD,KAAKyO,QAAQqyC,aAAapyC,QACzE,MAAO1O,MAAKivD,GAET,IAAyC,GAArCjvD,KAAKyO,QAAQqyC,aAAapyC,QACjC,OAAQqD,EAAE,EAAEC,EAAE,EAGd,IAAIqkD,GAAO,KACPC,EAAO,KACP3P,EAAS3mD,KAAKyO,QAAQqyC,aAAaE,UACnCl6C,EAAO9G,KAAKyO,QAAQqyC,aAAah6C,KAEjC8X,EAAK1Z,KAAK2lB,IAAI7qB,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GACpC8M,EAAK3Z,KAAK2lB,IAAI7qB,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,EA2JxC,OA1JY,YAARlL,GAA8B,iBAARA,EACpB5B,KAAK2lB,IAAI7qB,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAAK7M,KAAK2lB,IAAI7qB,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,IACjEhS,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,EACpBhS,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GACxBskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS9nC,EAC9By3C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS9nC,GAEvB7e,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,IAC7BskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS9nC,EAC9By3C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS9nC,GAGzB7e,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,IACzBhS,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GACxBskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS9nC,EAC9By3C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS9nC,GAEvB7e,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,IAC7BskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS9nC,EAC9By3C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS9nC,IAGtB,YAAR/X,IACFuvD,EAAY1P,EAAS9nC,EAAdD,EAAmB5e,KAAKopB,KAAKrX,EAAIskD,IAGnCnxD,KAAK2lB,IAAI7qB,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAAK7M,KAAK2lB,IAAI7qB,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,KACtEhS,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,EACpBhS,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GACxBskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS/nC,GAEvB5e,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,IAC7BskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS/nC,GAGzB5e,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,IACzBhS,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GACxBskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS/nC,GAEvB5e,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,IAC7BskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS/nC,IAGtB,YAAR9X,IACFwvD,EAAY3P,EAAS/nC,EAAdC,EAAmB7e,KAAKopB,KAAKpX,EAAIskD,IAI7B,iBAARxvD,EACH5B,KAAK2lB,IAAI7qB,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAAK7M,KAAK2lB,IAAI7qB,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,IACrEqkD,EAAOr2D,KAAKopB,KAAKrX,EAEfukD,EADEt2D,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,EACjBhS,KAAKqpB,GAAGrX,GAAK,EAAI20C,GAAU9nC,EAG3B7e,KAAKqpB,GAAGrX,GAAK,EAAI20C,GAAU9nC,GAG7B3Z,KAAK2lB,IAAI7qB,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAAK7M,KAAK2lB,IAAI7qB,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,KAExEqkD,EADEr2D,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,EACjB/R,KAAKqpB,GAAGtX,GAAK,EAAI40C,GAAU/nC,EAG3B5e,KAAKqpB,GAAGtX,GAAK,EAAI40C,GAAU/nC,EAEpC03C,EAAOt2D,KAAKopB,KAAKpX,GAGJ,cAARlL,GAELuvD,EADEr2D,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,EACjB/R,KAAKqpB,GAAGtX,GAAK,EAAI40C,GAAU/nC,EAG3B5e,KAAKqpB,GAAGtX,GAAK,EAAI40C,GAAU/nC,EAEpC03C,EAAOt2D,KAAKopB,KAAKpX,GAEF,YAARlL,GACPuvD,EAAOr2D,KAAKopB,KAAKrX,EAEfukD,EADEt2D,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,EACjBhS,KAAKqpB,GAAGrX,GAAK,EAAI20C,GAAU9nC,EAG3B7e,KAAKqpB,GAAGrX,GAAK,EAAI20C,GAAU9nC,GAIhC3Z,KAAK2lB,IAAI7qB,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAAK7M,KAAK2lB,IAAI7qB,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,GACjEhS,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,EACpBhS,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAExBskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS9nC,EAC9By3C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS9nC,EAC9Bw3C,EAAOr2D,KAAKqpB,GAAGtX,EAAIskD,EAAOr2D,KAAKqpB,GAAGtX,EAAIskD,GAE/Br2D,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,IAE7BskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS9nC,EAC9By3C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS9nC,EAC9Bw3C,EAAOr2D,KAAKqpB,GAAGtX,EAAIskD,EAAOr2D,KAAKqpB,GAAGtX,EAAIskD,GAGjCr2D,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,IACzBhS,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAExBskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS9nC,EAC9By3C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS9nC,EAC9Bw3C,EAAOr2D,KAAKqpB,GAAGtX,EAAIskD,EAAOr2D,KAAKqpB,GAAGtX,EAAIskD,GAE/Br2D,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,IAE7BskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS9nC,EAC9By3C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS9nC,EAC9Bw3C,EAAOr2D,KAAKqpB,GAAGtX,EAAIskD,EAAOr2D,KAAKqpB,GAAGtX,EAAIskD,IAInCnxD,KAAK2lB,IAAI7qB,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAAK7M,KAAK2lB,IAAI7qB,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,KACtEhS,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,EACpBhS,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAExBskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKqpB,GAAGrX,EAAIskD,EAAOt2D,KAAKqpB,GAAGrX,EAAIskD,GAE/Bt2D,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,IAE7BskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKqpB,GAAGrX,EAAIskD,EAAOt2D,KAAKqpB,GAAGrX,EAAIskD,GAGjCt2D,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,IACzBhS,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAExBskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKqpB,GAAGrX,EAAIskD,EAAOt2D,KAAKqpB,GAAGrX,EAAIskD,GAE/Bt2D,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,IAE7BskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKqpB,GAAGrX,EAAIskD,EAAOt2D,KAAKqpB,GAAGrX,EAAIskD,MAOtCvkD,EAAGskD,EAAMrkD,EAAGskD,IASxBlzD,EAAK+P,UAAUwiD,MAAQ,SAAU5uC,GAI/B,GAFAA,EAAIa,YACJb,EAAIc,OAAO7nB,KAAKopB,KAAKrX,EAAG/R,KAAKopB,KAAKpX,GACO,GAArChS,KAAKyO,QAAQqyC,aAAapyC,QAAiB,CAC7C,GAAyC,GAArC1O,KAAKyO,QAAQqyC,aAAaC,QAAkB,CAC9C,GAAIkO,GAAMjvD,KAAKo2D,oBACf,OAAa,OAATnH,EAAIl9C,GACNgV,EAAIe,OAAO9nB,KAAKqpB,GAAGtX,EAAG/R,KAAKqpB,GAAGrX,GAC9B+U,EAAIlH,SACG,OAKPkH,EAAIwvC,iBAAiBtH,EAAIl9C,EAAEk9C,EAAIj9C,EAAEhS,KAAKqpB,GAAGtX,EAAG/R,KAAKqpB,GAAGrX,GACpD+U,EAAIlH,SACGovC,GAMT,MAFAloC,GAAIwvC,iBAAiBv2D,KAAKivD,IAAIl9C,EAAE/R,KAAKivD,IAAIj9C,EAAEhS,KAAKqpB,GAAGtX,EAAG/R,KAAKqpB,GAAGrX,GAC9D+U,EAAIlH,SACG7f,KAAKivD,IAMd,MAFAloC,GAAIe,OAAO9nB,KAAKqpB,GAAGtX,EAAG/R,KAAKqpB,GAAGrX,GAC9B+U,EAAIlH,SACG,MAYXzc,EAAK+P,UAAU8iD,QAAU,SAAUlvC,EAAKhV,EAAGC,EAAGyZ,GAE5C1E,EAAIa,YACJb,EAAI2E,IAAI3Z,EAAGC,EAAGyZ,EAAQ,EAAG,EAAIvmB,KAAKymB,IAAI,GACtC5E,EAAIlH,UAWNzc,EAAK+P,UAAU4iD,OAAS,SAAUhvC,EAAKwC,EAAMxX,EAAGC,GAC9C,GAAIuX,EAAM,CACRxC,EAAIQ,MAASvnB,KAAKopB,KAAKiqB,UAAYrzC,KAAKqpB,GAAGgqB,SAAY,QAAU,IACjErzC,KAAKyO,QAAQ8uC,SAAW,MAAQv9C,KAAKyO,QAAQ+uC,QAC7C,IAAIuW,EAEJ,IAAuB,GAAnB/zD,KAAKg0D,WAAoB,CAC3B,GAAIxtB,GAAQpiC,OAAOmlB,GAAMxhB,MAAM,MAC3ByuD,EAAYhwB,EAAM7gC,OAClB43C,EAAWr5C,OAAOlE,KAAKyO,QAAQ8uC,SACnCwW,GAAQ/hD,GAAK,EAAIwkD,GAAa,EAAIjZ,CAGlC,KAAK,GADDhrC,GAAQwU,EAAI0vC,YAAYjwB,EAAM,IAAIj0B,MAC7B/M,EAAI,EAAOgxD,EAAJhxD,EAAeA,IAAK,CAClC,GAAI8hB,GAAYP,EAAI0vC,YAAYjwB,EAAMhhC,IAAI+M,KAC1CA,GAAQ+U,EAAY/U,EAAQ+U,EAAY/U,EAE1C,GAAIC,GAASxS,KAAKyO,QAAQ8uC,SAAWiZ,EACjC/uD,EAAOsK,EAAIQ,EAAQ,EACnB5K,EAAMqK,EAAIQ,EAAS,CAGvBxS,MAAK8zD,iBAAmBnsD,IAAIA,EAAIF,KAAKA,EAAK8K,MAAMA,EAAMC,OAAOA,EAAOuhD,MAAMA,GAG/E,GAAIA,GAAQ/zD,KAAK8zD,gBAAgBC,KAEjChtC,GAAIuoC,OAE+B,cAA/BtvD,KAAKyO,QAAQwvC,iBAChBl3B,EAAIwoC,UAAUx9C,EAAGgiD,GACjB/zD,KAAK02D,yBAAyB3vC,GAC9BhV,EAAI,EACJgiD,EAAQ,GAIT/zD,KAAK22D,eAAe5vC,GACpB/mB,KAAK42D,eAAe7vC,EAAIhV,EAAEgiD,EAAOvtB,EAAOgwB,EAAWjZ,GAEnDx2B,EAAI0oC,YASLrsD,EAAK+P,UAAUujD,yBAA2B,SAAS3vC,GAClD,GAAIlI,GAAK7e,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,EAC3B4M,EAAK5e,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,EAC3B8kD,EAAiB3xD,KAAK4xD,MAAMj4C,EAAID,IAGf,GAAjBi4C,GAA4B,EAALj4C,GAAYi4C,EAAiB,GAAU,EAALj4C,KAC5Di4C,GAAkC3xD,KAAKymB,IAGxC5E,EAAIgwC,OAAOF,IASZzzD,EAAK+P,UAAUwjD,eAAiB,SAAS5vC,GACxC,GAA8BvgB,SAA1BxG,KAAKyO,QAAQgvC,UAAoD,OAA1Bz9C,KAAKyO,QAAQgvC,UAA+C,SAA1Bz9C,KAAKyO,QAAQgvC,SAAqB,CAC9G12B,EAAIiB,UAAYhoB,KAAKyO,QAAQgvC,QAE7B,IAAIuZ,GAAa,CAEoB,gBAA/Bh3D,KAAKyO,QAAQwvC,eACfl3B,EAAIkwC,SAAuC,IAA7Bj3D,KAAK8zD,gBAAgBvhD,MAA4C,IAA9BvS,KAAK8zD,gBAAgBthD,OAAcxS,KAAK8zD,gBAAgBvhD,MAAOvS,KAAK8zD,gBAAgBthD,QAE/F,cAA/BxS,KAAKyO,QAAQwvC,eACpBl3B,EAAIkwC,SAAuC,IAA7Bj3D,KAAK8zD,gBAAgBvhD,QAAevS,KAAK8zD,gBAAgBthD,OAASwkD,GAAah3D,KAAK8zD,gBAAgBvhD,MAAOvS,KAAK8zD,gBAAgBthD,QAExG,cAA/BxS,KAAKyO,QAAQwvC,eACpBl3B,EAAIkwC,SAAuC,IAA7Bj3D,KAAK8zD,gBAAgBvhD,MAAaykD,EAAYh3D,KAAK8zD,gBAAgBvhD,MAAOvS,KAAK8zD,gBAAgBthD,QAG7GuU,EAAIkwC,SAASj3D,KAAK8zD,gBAAgBrsD,KAAMzH,KAAK8zD,gBAAgBnsD,IAAK3H,KAAK8zD,gBAAgBvhD,MAAOvS,KAAK8zD,gBAAgBthD,UAezHpP,EAAK+P,UAAUyjD,eAAiB,SAAS7vC,EAAKhV,EAAGgiD,EAAOvtB,EAAOgwB,EAAWjZ,GAMxE,GAJDx2B,EAAIiB,UAAYhoB,KAAKyO,QAAQ6uC,WAAa,QAC1Cv2B,EAAIuB,UAAY,SAGoB,cAA/BtoB,KAAKyO,QAAQwvC,eAAgC,CAC/C,GAAI+Y,GAAa,CACkB,eAA/Bh3D,KAAKyO,QAAQwvC,gBACfl3B,EAAIwB,aAAe,aACnBwrC,GAAS,EAAIiD,GAEyB,cAA/Bh3D,KAAKyO,QAAQwvC,gBACpBl3B,EAAIwB,aAAe,UACnBwrC,GAAS,EAAIiD,GAGbjwC,EAAIwB,aAAe,aAIrBxB,GAAIwB,aAAe,QAIjBvoB,MAAKyO,QAAQivC,gBAAkB,IACjC32B,EAAIO,UAActnB,KAAKyO,QAAQivC,gBAC/B32B,EAAIY,YAAc3nB,KAAKyO,QAAQkvC,gBAC/B52B,EAAImwC,SAAc,QAErB,KAAK,GAAI1xD,GAAI,EAAOgxD,EAAJhxD,EAAeA,IACzBxF,KAAKyO,QAAQivC,gBAAkB,GAChC32B,EAAIowC,WAAW3wB,EAAMhhC,GAAIuM,EAAGgiD,GAEhChtC,EAAIyB,SAASge,EAAMhhC,GAAIuM,EAAGgiD,GAC1BA,GAASxW,GAaXn6C,EAAK+P,UAAU2hD,cAAgB,SAAS/tC,GAEtCA,EAAIY,YAAc3nB,KAAKw1D,YACvBzuC,EAAIO,UAAYtnB,KAAK01D,eAErB,IAAIzG,GAAM,IAEV,IAAwBzoD,SAApBugB,EAAIqwC,YAA2B,CACjCrwC,EAAIuoC,MAEJ,IAAI+H,IAAW,EAEbA,GAD+B7wD,SAA7BxG,KAAKyO,QAAQ0vC,KAAKx4C,QAAkDa,SAA1BxG,KAAKyO,QAAQ0vC,KAAKC,KACnDp+C,KAAKyO,QAAQ0vC,KAAKx4C,OAAO3F,KAAKyO,QAAQ0vC,KAAKC,MAG3C,EAAE,GAIfr3B,EAAIqwC,YAAYC,GAChBtwC,EAAIuwC,eAAiB,EAGrBrI,EAAMjvD,KAAK21D,MAAM5uC,GAGjBA,EAAIqwC,aAAa,IACjBrwC,EAAIuwC,eAAiB,EACrBvwC,EAAI0oC,cAIJ1oC,GAAIa,YACJb,EAAIwwC,QAAU,QACsB/wD,SAAhCxG,KAAKyO,QAAQ0vC,KAAKE,UAEpBt3B,EAAIywC,WAAWx3D,KAAKopB,KAAKrX,EAAE/R,KAAKopB,KAAKpX,EAAEhS,KAAKqpB,GAAGtX,EAAE/R,KAAKqpB,GAAGrX,GACpDhS,KAAKyO,QAAQ0vC,KAAKx4C,OAAO3F,KAAKyO,QAAQ0vC,KAAKC,IAAIp+C,KAAKyO,QAAQ0vC,KAAKE,UAAUr+C,KAAKyO,QAAQ0vC,KAAKC,MAE9D53C,SAA7BxG,KAAKyO,QAAQ0vC,KAAKx4C,QAAkDa,SAA1BxG,KAAKyO,QAAQ0vC,KAAKC,IAEnEr3B,EAAIywC,WAAWx3D,KAAKopB,KAAKrX,EAAE/R,KAAKopB,KAAKpX,EAAEhS,KAAKqpB,GAAGtX,EAAE/R,KAAKqpB,GAAGrX,GACpDhS,KAAKyO,QAAQ0vC,KAAKx4C,OAAO3F,KAAKyO,QAAQ0vC,KAAKC,OAIhDr3B,EAAIc,OAAO7nB,KAAKopB,KAAKrX,EAAG/R,KAAKopB,KAAKpX,GAClC+U,EAAIe,OAAO9nB,KAAKqpB,GAAGtX,EAAG/R,KAAKqpB,GAAGrX,IAEhC+U,EAAIlH,QAIN,IAAI7f,KAAKyoB,MAAO,CACd,GAAIvW,EACJ,IAAyC,GAArClS,KAAKyO,QAAQqyC,aAAapyC,SAA0B,MAAPugD,EAAa,CAC5D,GAAI2G,GAAY,IAAK,IAAK51D,KAAKopB,KAAKrX,EAAIk9C,EAAIl9C,GAAK,IAAK/R,KAAKqpB,GAAGtX,EAAIk9C,EAAIl9C,IAClE8jD,EAAY,IAAK,IAAK71D,KAAKopB,KAAKpX,EAAIi9C,EAAIj9C,GAAK,IAAKhS,KAAKqpB,GAAGrX,EAAIi9C,EAAIj9C,GACtEE,IAASH,EAAE6jD,EAAW5jD,EAAE6jD,OAGxB3jD,GAAQlS,KAAK81D,aAAa,GAE5B91D,MAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOvW,EAAMH,EAAGG,EAAMF,KAUhD5O,EAAK+P,UAAU2iD,aAAe,SAAU2B,GACtC,OACE1lD,GAAI,EAAI0lD,GAAcz3D,KAAKopB,KAAKrX,EAAI0lD,EAAaz3D,KAAKqpB,GAAGtX,EACzDC,GAAI,EAAIylD,GAAcz3D,KAAKopB,KAAKpX,EAAIylD,EAAaz3D,KAAKqpB,GAAGrX,IAa7D5O,EAAK+P,UAAU+iD,eAAiB,SAAUnkD,EAAGC,EAAGyZ,EAAQgsC,GACtD,GAAIrJ,GAA6B,GAApBqJ,EAAa,EAAE,GAASvyD,KAAKymB,EAC1C,QACE5Z,EAAGA,EAAI0Z,EAASvmB,KAAKqZ,IAAI6vC,GACzBp8C,EAAGA,EAAIyZ,EAASvmB,KAAKkZ,IAAIgwC,KAW7BhrD,EAAK+P,UAAU0hD,iBAAmB,SAAS9tC,GACzC,GAAI7U,EAMJ,IAJA6U,EAAIY,YAAc3nB,KAAKw1D,YACvBzuC,EAAIiB,UAAYjB,EAAIY,YACpBZ,EAAIO,UAAYtnB,KAAK01D,gBAEjB11D,KAAKopB,MAAQppB,KAAKqpB,GAAI,CAExB,GAAI4lC,GAAMjvD,KAAK21D,MAAM5uC,GAEjBqnC,EAAQlpD,KAAK4xD,MAAO92D,KAAKqpB,GAAGrX,EAAIhS,KAAKopB,KAAKpX,EAAKhS,KAAKqpB,GAAGtX,EAAI/R,KAAKopB,KAAKrX,GACrEpM,GAAU,GAAK,EAAI3F,KAAKyO,QAAQ8D,OAASvS,KAAKyO,QAAQyvC,gBAE1D,IAAyC,GAArCl+C,KAAKyO,QAAQqyC,aAAapyC,SAA0B,MAAPugD,EAAa,CAC5D,GAAI2G,GAAY,IAAK,IAAK51D,KAAKopB,KAAKrX,EAAIk9C,EAAIl9C,GAAK,IAAK/R,KAAKqpB,GAAGtX,EAAIk9C,EAAIl9C,IAClE8jD,EAAY,IAAK,IAAK71D,KAAKopB,KAAKpX,EAAIi9C,EAAIj9C,GAAK,IAAKhS,KAAKqpB,GAAGrX,EAAIi9C,EAAIj9C,GACtEE,IAASH,EAAE6jD,EAAW5jD,EAAE6jD,OAGxB3jD,GAAQlS,KAAK81D,aAAa,GAG5B/uC,GAAI2wC,MAAMxlD,EAAMH,EAAGG,EAAMF,EAAGo8C,EAAOzoD,GACnCohB,EAAInH,OACJmH,EAAIlH,SAGA7f,KAAKyoB,OACPzoB,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOvW,EAAMH,EAAGG,EAAMF,OAG3C,CAEH,GAAID,GAAGC,EACHyZ,EAAS,IAAOvmB,KAAKwH,IAAI,IAAI1M,KAAKw+C,QAAQK,cAC1CiH,EAAO9lD,KAAKopB,IACX08B,GAAKvzC,OACRuzC,EAAKkQ,OAAOjvC,GAEV++B,EAAKvzC,MAAQuzC,EAAKtzC,QACpBT,EAAI+zC,EAAK/zC,EAAiB,GAAb+zC,EAAKvzC,MAClBP,EAAI8zC,EAAK9zC,EAAIyZ,IAGb1Z,EAAI+zC,EAAK/zC,EAAI0Z,EACbzZ,EAAI8zC,EAAK9zC,EAAkB,GAAd8zC,EAAKtzC,QAEpBxS,KAAKi2D,QAAQlvC,EAAKhV,EAAGC,EAAGyZ,EAGxB,IAAI2iC,GAAQ,GAAMlpD,KAAKymB,GACnBhmB,GAAU,GAAK,EAAI3F,KAAKyO,QAAQ8D,OAASvS,KAAKyO,QAAQyvC,gBAC1DhsC,GAAQlS,KAAKk2D,eAAenkD,EAAGC,EAAGyZ,EAAQ,IAC1C1E,EAAI2wC,MAAMxlD,EAAMH,EAAGG,EAAMF,EAAGo8C,EAAOzoD,GACnCohB,EAAInH,OACJmH,EAAIlH,SAGA7f,KAAKyoB,QACPvW,EAAQlS,KAAKk2D,eAAenkD,EAAGC,EAAGyZ,EAAQ,IAC1CzrB,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOvW,EAAMH,EAAGG,EAAMF,MAKlD5O,EAAK+P,UAAUwkD,eAAiB,SAAS7pD,GACvC,GAAImhD,GAAMjvD,KAAKo2D,qBAEXrkD,EAAI7M,KAAK4uB,IAAI,EAAEhmB,EAAE,GAAG9N,KAAKopB,KAAKrX,EAAK,EAAEjE,GAAG,EAAIA,GAAImhD,EAAIl9C,EAAI7M,KAAK4uB,IAAIhmB,EAAE,GAAG9N,KAAKqpB,GAAGtX,EAC9EC,EAAI9M,KAAK4uB,IAAI,EAAEhmB,EAAE,GAAG9N,KAAKopB,KAAKpX,EAAK,EAAElE,GAAG,EAAIA,GAAImhD,EAAIj9C,EAAI9M,KAAK4uB,IAAIhmB,EAAE,GAAG9N,KAAKqpB,GAAGrX,CAElF,QAAQD,EAAEA,EAAEC,EAAEA,IAWhB5O,EAAK+P,UAAUykD,oBAAsB,SAASxuC,EAAKrC,GACjD,GAIIxB,GAAI6oC,EAAMyJ,EAAkBC,EAAiBC,EAJ7C/oD,EAAgB,GAChBC,EAAY,EACZC,EAAM,EACNC,EAAO,EAEP6oD,EAAY,GACZlS,EAAO9lD,KAAKqpB,EAKhB,KAJY,GAARD,IACF08B,EAAO9lD,KAAKopB,MAGAja,GAAPD,GAA2BF,EAAZC,GAA2B,CAC/C,GAAIG,GAAwB,IAAdF,EAAMC,EAOpB,IALAoW,EAAMvlB,KAAK23D,eAAevoD,GAC1Bg/C,EAAQlpD,KAAK4xD,MAAOhR,EAAK9zC,EAAIuT,EAAIvT,EAAK8zC,EAAK/zC,EAAIwT,EAAIxT,GACnD8lD,EAAmB/R,EAAK+R,iBAAiB9wC,EAAIqnC,GAC7C0J,EAAkB5yD,KAAKyqB,KAAKzqB,KAAK4uB,IAAIvO,EAAIxT,EAAE+zC,EAAK/zC,EAAE,GAAK7M,KAAK4uB,IAAIvO,EAAIvT,EAAE8zC,EAAK9zC,EAAE,IAC7E+lD,EAAaF,EAAmBC,EAC5B5yD,KAAK2lB,IAAIktC,GAAcC,EACzB,KAEoB,GAAbD,EACK,GAAR3uC,EACFla,EAAME,EAGND,EAAOC,EAIG,GAARga,EACFja,EAAOC,EAGPF,EAAME,EAIVH,IAIF,MAFAsW,GAAIzX,EAAIsB,EAEDmW,GAUTniB,EAAK+P,UAAUyhD,WAAa,SAAS7tC,GAEnCA,EAAIY,YAAc3nB,KAAKw1D,YACvBzuC,EAAIiB,UAAYjB,EAAIY,YACpBZ,EAAIO,UAAYtnB,KAAK01D,eAGrB,IAAItH,GAAOzoD,EAAQsyD,CAGnB,IAAIj4D,KAAKopB,MAAQppB,KAAKqpB,GAAI,CAKxB,GAHArpB,KAAK21D,MAAM5uC,GAG8B,GAArC/mB,KAAKyO,QAAQqyC,aAAapyC,QAAiB,CAC7C,GAAIugD,GAAMjvD,KAAKo2D,oBACf6B,GAAWj4D,KAAK43D,qBAAoB,EAAO7wC,EAC3C,IAAImxC,GAAWl4D,KAAK23D,eAAezyD,KAAKwH,IAAI,EAAKurD,EAASnqD,EAAI,IAC9DsgD,GAAQlpD,KAAK4xD,MAAOmB,EAASjmD,EAAIkmD,EAASlmD,EAAKimD,EAASlmD,EAAImmD,EAASnmD,OAElE,CACHq8C,EAAQlpD,KAAK4xD,MAAO92D,KAAKqpB,GAAGrX,EAAIhS,KAAKopB,KAAKpX,EAAKhS,KAAKqpB,GAAGtX,EAAI/R,KAAKopB,KAAKrX,EACrE,IAAI6M,GAAM5e,KAAKqpB,GAAGtX,EAAI/R,KAAKopB,KAAKrX,EAC5B8M,EAAM7e,KAAKqpB,GAAGrX,EAAIhS,KAAKopB,KAAKpX,EAC5BmmD,EAAoBjzD,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAC7Cu5C,EAAep4D,KAAKqpB,GAAGwuC,iBAAiB9wC,EAAKqnC,GAC7CiK,GAAiBF,EAAoBC,GAAgBD,CAEzDF,MACAA,EAASlmD,GAAK,EAAIsmD,GAAiBr4D,KAAKopB,KAAKrX,EAAIsmD,EAAgBr4D,KAAKqpB,GAAGtX,EACzEkmD,EAASjmD,GAAK,EAAIqmD,GAAiBr4D,KAAKopB,KAAKpX,EAAIqmD,EAAgBr4D,KAAKqpB,GAAGrX,EAU3E,GANArM,GAAU,GAAK,EAAI3F,KAAKyO,QAAQ8D,OAASvS,KAAKyO,QAAQyvC,iBACtDn3B,EAAI2wC,MAAMO,EAASlmD,EAAEkmD,EAASjmD,EAAGo8C,EAAOzoD,GACxCohB,EAAInH,OACJmH,EAAIlH,SAGA7f,KAAKyoB,MAAO,CACd,GAAIvW,EAEFA,GADuC,GAArClS,KAAKyO,QAAQqyC,aAAapyC,SAA0B,MAAPugD,EACvCjvD,KAAK23D,eAAe,IAGpB33D,KAAK81D,aAAa,IAE5B91D,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOvW,EAAMH,EAAGG,EAAMF,QAG3C,CAEH,GACID,GAAGC,EAAG0lD,EADN5R,EAAO9lD,KAAKopB,KAEZqC,EAAS,IAAOvmB,KAAKwH,IAAI,IAAI1M,KAAKw+C,QAAQK,aACzCiH,GAAKvzC,OACRuzC,EAAKkQ,OAAOjvC,GAEV++B,EAAKvzC,MAAQuzC,EAAKtzC,QACpBT,EAAI+zC,EAAK/zC,EAAiB,GAAb+zC,EAAKvzC,MAClBP,EAAI8zC,EAAK9zC,EAAIyZ,EACbisC,GACE3lD,EAAGA,EACHC,EAAG8zC,EAAK9zC,EACRo8C,MAAO,GAAMlpD,KAAKymB,MAIpB5Z,EAAI+zC,EAAK/zC,EAAI0Z,EACbzZ,EAAI8zC,EAAK9zC,EAAkB,GAAd8zC,EAAKtzC,OAClBklD,GACE3lD,EAAG+zC,EAAK/zC,EACRC,EAAGA,EACHo8C,MAAO,GAAMlpD,KAAKymB,KAGtB5E,EAAIa,YAEJb,EAAI2E,IAAI3Z,EAAGC,EAAGyZ,EAAQ,EAAG,EAAIvmB,KAAKymB,IAAI,GACtC5E,EAAIlH,QAGJ,IAAIla,IAAU,GAAK,EAAI3F,KAAKyO,QAAQ8D,OAASvS,KAAKyO,QAAQyvC,gBAC1Dn3B,GAAI2wC,MAAMA,EAAM3lD,EAAG2lD,EAAM1lD,EAAG0lD,EAAMtJ,MAAOzoD,GACzCohB,EAAInH,OACJmH,EAAIlH,SAGA7f,KAAKyoB,QACPvW,EAAQlS,KAAKk2D,eAAenkD,EAAGC,EAAGyZ,EAAQ,IAC1CzrB,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOvW,EAAMH,EAAGG,EAAMF,MAiBlD5O,EAAK+P,UAAUoiD,mBAAqB,SAAU+C,EAAGC,EAAIC,EAAGC,EAAIC,EAAGC,GAC7D,GAAInvD,GAAc,CAClB,IAAIxJ,KAAKopB,MAAQppB,KAAKqpB,GACpB,GAAyC,GAArCrpB,KAAKyO,QAAQqyC,aAAapyC,QAAiB,CAC7C,GAAI2nD,GAAMC,CACV,IAAyC,GAArCt2D,KAAKyO,QAAQqyC,aAAapyC,SAAwD,GAArC1O,KAAKyO,QAAQqyC,aAAaC,QACzEsV,EAAOr2D,KAAKivD,IAAIl9C,EAChBukD,EAAOt2D,KAAKivD,IAAIj9C,MAEb,CACH,GAAIi9C,GAAMjvD,KAAKo2D,oBACfC,GAAOpH,EAAIl9C,EACXukD,EAAOrH,EAAIj9C,EAEb,GACI2T,GACAngB,EAAEsI,EAAEiE,EAAEC,EAAG4mD,EAAOC,EAFhBC,EAAc,GAGlB,KAAKtzD,EAAI,EAAO,GAAJA,EAAQA,IAClBsI,EAAI,GAAItI,EACRuM,EAAI7M,KAAK4uB,IAAI,EAAEhmB,EAAE,GAAGwqD,EAAM,EAAExqD,GAAG,EAAIA,GAAIuoD,EAAOnxD,KAAK4uB,IAAIhmB,EAAE,GAAG0qD,EAC5DxmD,EAAI9M,KAAK4uB,IAAI,EAAEhmB,EAAE,GAAGyqD,EAAM,EAAEzqD,GAAG,EAAIA,GAAIwoD,EAAOpxD,KAAK4uB,IAAIhmB,EAAE,GAAG2qD,EACxDjzD,EAAI,IACNmgB,EAAW3lB,KAAK+4D,mBAAmBH,EAAMC,EAAM9mD,EAAEC,EAAG0mD,EAAGC,GACvDG,EAAyBA,EAAXnzC,EAAyBA,EAAWmzC,GAEpDF,EAAQ7mD,EAAG8mD,EAAQ7mD,CAErBxI,GAAcsvD,MAGdtvD,GAAcxJ,KAAK+4D,mBAAmBT,EAAGC,EAAGC,EAAGC,EAAGC,EAAGC,OAGpD,CACH,GAAI5mD,GAAGC,EAAG4M,EAAIC,EACV4M,EAAS,IAAOzrB,KAAKw+C,QAAQK,aAC7BiH,EAAO9lD,KAAKopB,IACZ08B,GAAKvzC,MAAQuzC,EAAKtzC,QACpBT,EAAI+zC,EAAK/zC,EAAI,GAAM+zC,EAAKvzC,MACxBP,EAAI8zC,EAAK9zC,EAAIyZ,IAGb1Z,EAAI+zC,EAAK/zC,EAAI0Z,EACbzZ,EAAI8zC,EAAK9zC,EAAI,GAAM8zC,EAAKtzC,QAE1BoM,EAAK7M,EAAI2mD,EACT75C,EAAK7M,EAAI2mD,EACTnvD,EAActE,KAAK2lB,IAAI3lB,KAAKyqB,KAAK/Q,EAAGA,EAAKC,EAAGA,GAAM4M,GAGpD,MAAIzrB,MAAK8zD,gBAAgBrsD,KAAOixD,GAC9B14D,KAAK8zD,gBAAgBrsD,KAAOzH,KAAK8zD,gBAAgBvhD,MAAQmmD,GACzD14D,KAAK8zD,gBAAgBnsD,IAAMgxD,GAC3B34D,KAAK8zD,gBAAgBnsD,IAAM3H,KAAK8zD,gBAAgBthD,OAASmmD,EAClD,EAGAnvD,GAIXpG,EAAK+P,UAAU4lD,mBAAqB,SAAST,EAAGC,EAAGC,EAAGC,EAAGC,EAAGC,GAC1D,GAAIK,GAAKR,EAAGF,EACVW,EAAKR,EAAGF,EACRW,EAAYF,EAAGA,EAAKC,EAAGA,EACvBE,IAAOT,EAAKJ,GAAMU,GAAML,EAAKJ,GAAMU,GAAMC,CAEvCC,GAAI,EACNA,EAAI,EAEO,EAAJA,IACPA,EAAI,EAGN,IAAIpnD,GAAIumD,EAAKa,EAAIH,EACfhnD,EAAIumD,EAAKY,EAAIF,EACbr6C,EAAK7M,EAAI2mD,EACT75C,EAAK7M,EAAI2mD,CAQX,OAAOzzD,MAAKyqB,KAAK/Q,EAAGA,EAAKC,EAAGA,IAQ9Bzb,EAAK+P,UAAU6vB,SAAW,SAAS/lB,GACjCjd,KAAKm2D,gBAAkB,EAAIl5C,GAI7B7Z,EAAK+P,UAAUg+B,OAAS,WACtBnxC,KAAKqzC,UAAW,GAGlBjwC,EAAK+P,UAAU+9B,SAAW,WACxBlxC,KAAKqzC,UAAW,GAGlBjwC,EAAK+P,UAAUg/C,mBAAqB,WACjB,OAAbnyD,KAAKivD,KAA8B,OAAdjvD,KAAKopB,MAA6B,OAAZppB,KAAKqpB,IAClDrpB,KAAKivD,IAAIl9C,EAAI,IAAO/R,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAC1C/R,KAAKivD,IAAIj9C,EAAI,IAAOhS,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,KAG1ChS,KAAKivD,IAAIl9C,EAAI,EACb/R,KAAKivD,IAAIj9C,EAAI,IASjB5O,EAAK+P,UAAU+8C,kBAAoB,SAASnpC,GAC1C,GAAgC,GAA5B/mB,KAAKu0D,oBAA6B,CACpC,GAA+B,OAA3Bv0D,KAAKw0D,aAAaprC,MAA0C,OAAzBppB,KAAKw0D,aAAanrC,GAAa,CACpE,GAAI+vC,GAAa,cAAcplD,OAAOhU,KAAKK,IACvCg5D,EAAW,YAAYrlD,OAAOhU,KAAKK,IACnCqhD,GACY1E,OAAO/qC,MAAM,GAAIwZ,OAAO,EAAGzL,YAAY,EAAG69B,oBAAqB,GAC/DW,SAASO,QAAQ,GACjBI,YAAac,sBAAuB,EAAGD,aAAcztC,MAAM,EAAGC,OAAQ,EAAGiZ,OAAO,IAEhGzrB,MAAKw0D,aAAaprC,KAAO,GAAI7lB,IAC1BlD,GAAG+4D,EACFhc,MAAM,MACJjyC,OAAOgB,WAAW,UAAWC,OAAO,UAAWC,WAAYF,WAAW,mBAClEu1C,GACV1hD,KAAKw0D,aAAanrC,GAAK,GAAI9lB,IACxBlD,GAAGg5D,EACFjc,MAAM,MACNjyC,OAAOgB,WAAW,UAAWC,OAAO,UAAWC,WAAYF,WAAW,mBAChEu1C,GAGZ1hD,KAAKw0D,aAAaC,aACqB,GAAnCz0D,KAAKw0D,aAAaprC,KAAKiqB,WACzBrzC,KAAKw0D,aAAaC,UAAUrrC,KAAOppB,KAAKs5D,2BAA2BvyC,GACnE/mB,KAAKw0D,aAAaprC,KAAKrX,EAAI/R,KAAKw0D,aAAaC,UAAUrrC,KAAKrX,EAC5D/R,KAAKw0D,aAAaprC,KAAKpX,EAAIhS,KAAKw0D,aAAaC,UAAUrrC,KAAKpX,GAEzB,GAAjChS,KAAKw0D,aAAanrC,GAAGgqB,WACvBrzC,KAAKw0D,aAAaC,UAAUprC,GAAKrpB,KAAKu5D,yBAAyBxyC,GAC/D/mB,KAAKw0D,aAAanrC,GAAGtX,EAAI/R,KAAKw0D,aAAaC,UAAUprC,GAAGtX,EACxD/R,KAAKw0D,aAAanrC,GAAGrX,EAAIhS,KAAKw0D,aAAaC,UAAUprC,GAAGrX,GAG1DhS,KAAKw0D,aAAaprC,KAAK2iB,KAAKhlB,GAC5B/mB,KAAKw0D,aAAanrC,GAAG0iB,KAAKhlB,OAG1B/mB,MAAKw0D,cAAgBprC,KAAK,KAAMC,GAAG,KAAMorC,eAQ7CrxD,EAAK+P,UAAUqmD,oBAAsB,WACnCx5D,KAAKi0D,WAAaj0D,KAAKopB,KACvBppB,KAAKk0D,SAAWl0D,KAAKqpB,GACrBrpB,KAAKu0D,qBAAsB,GAO7BnxD,EAAK+P,UAAUsmD,qBAAuB,WACpCz5D,KAAK2zD,OAAS3zD,KAAKopB,KAAK/oB,GACxBL,KAAK4zD,KAAO5zD,KAAKqpB,GAAGhpB,GAChBL,KAAK2zD,QAAU3zD,KAAKi0D,WAAW5zD,GACjCL,KAAKi0D,WAAWe,WAAWh1D,MAEpBA,KAAK4zD,MAAQ5zD,KAAKk0D,SAAS7zD,IAClCL,KAAKk0D,SAASc,WAAWh1D,MAG3BA,KAAKi0D,WAAa,KAClBj0D,KAAKk0D,SAAW,KAChBl0D,KAAKu0D,qBAAsB,GAW7BnxD,EAAK+P,UAAUumD,wBAA0B,SAAS3nD,EAAEC,GAClD,GAAIyiD,GAAYz0D,KAAKw0D,aAAaC,UAC9BkF,EAAez0D,KAAKyqB,KAAKzqB,KAAK4uB,IAAI/hB,EAAI0iD,EAAUrrC,KAAKrX,EAAE,GAAK7M,KAAK4uB,IAAI9hB,EAAIyiD,EAAUrrC,KAAKpX,EAAE,IAC1F4nD,EAAe10D,KAAKyqB,KAAKzqB,KAAK4uB,IAAI/hB,EAAI0iD,EAAUprC,GAAGtX,EAAI,GAAK7M,KAAK4uB,IAAI9hB,EAAIyiD,EAAUprC,GAAGrX,EAAI,GAE9F,OAAmB,IAAf2nD,GACF35D,KAAK00D,cAAgB10D,KAAKopB,KAC1BppB,KAAKopB,KAAOppB,KAAKw0D,aAAaprC,KACvBppB,KAAKw0D,aAAaprC,MAEL,GAAbwwC,GACP55D,KAAK00D,cAAgB10D,KAAKqpB,GAC1BrpB,KAAKqpB,GAAKrpB,KAAKw0D,aAAanrC,GACrBrpB,KAAKw0D,aAAanrC,IAGlB,MASXjmB,EAAK+P,UAAU0mD,qBAAuB,WACG,GAAnC75D,KAAKw0D,aAAaprC,KAAKiqB,UACzBrzC,KAAKopB,KAAOppB,KAAK00D,cACjB10D,KAAK00D,cAAgB,KACrB10D,KAAKw0D,aAAaprC,KAAK8nB,YAEiB,GAAjClxC,KAAKw0D,aAAanrC,GAAGgqB,WAC5BrzC,KAAKqpB,GAAKrpB,KAAK00D,cACf10D,KAAK00D,cAAgB,KACrB10D,KAAKw0D,aAAanrC,GAAG6nB,aAUzB9tC,EAAK+P,UAAUmmD,2BAA6B,SAASvyC,GAEnD,GAAI+yC,EACJ,IAAyC,GAArC95D,KAAKyO,QAAQqyC,aAAapyC,QAC5BorD,EAAqB95D,KAAK43D,qBAAoB,EAAM7wC,OAEjD,CACH,GAAIqnC,GAAQlpD,KAAK4xD,MAAO92D,KAAKqpB,GAAGrX,EAAIhS,KAAKopB,KAAKpX,EAAKhS,KAAKqpB,GAAGtX,EAAI/R,KAAKopB,KAAKrX,GACrE6M,EAAM5e,KAAKqpB,GAAGtX,EAAI/R,KAAKopB,KAAKrX,EAC5B8M,EAAM7e,KAAKqpB,GAAGrX,EAAIhS,KAAKopB,KAAKpX,EAC5BmmD,EAAoBjzD,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAE7Ck7C,EAAiB/5D,KAAKopB,KAAKyuC,iBAAiB9wC,EAAKqnC,EAAQlpD,KAAKymB,IAC9DquC,GAAmB7B,EAAoB4B,GAAkB5B,CAC7D2B,MACAA,EAAmB/nD,EAAI,EAAoB/R,KAAKopB,KAAKrX,GAAK,EAAIioD,GAAmBh6D,KAAKqpB,GAAGtX,EACzF+nD,EAAmB9nD,EAAI,EAAoBhS,KAAKopB,KAAKpX,GAAK,EAAIgoD,GAAmBh6D,KAAKqpB,GAAGrX,EAG3F,MAAO8nD,IAST12D,EAAK+P,UAAUomD,yBAA2B,SAASxyC,GAEjD,GAAuBkzC,EACvB,IAAyC,GAArCj6D,KAAKyO,QAAQqyC,aAAapyC,QAC5BurD,EAAmBj6D,KAAK43D,qBAAoB,EAAO7wC,OAEhD,CACH,GAAIqnC,GAAQlpD,KAAK4xD,MAAO92D,KAAKqpB,GAAGrX,EAAIhS,KAAKopB,KAAKpX,EAAKhS,KAAKqpB,GAAGtX,EAAI/R,KAAKopB,KAAKrX,GACrE6M,EAAM5e,KAAKqpB,GAAGtX,EAAI/R,KAAKopB,KAAKrX,EAC5B8M,EAAM7e,KAAKqpB,GAAGrX,EAAIhS,KAAKopB,KAAKpX,EAC5BmmD,EAAoBjzD,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAC7Cu5C,EAAep4D,KAAKqpB,GAAGwuC,iBAAiB9wC,EAAKqnC,GAC7CiK,GAAiBF,EAAoBC,GAAgBD,CAEzD8B,MACAA,EAAiBloD,GAAK,EAAIsmD,GAAiBr4D,KAAKopB,KAAKrX,EAAIsmD,EAAgBr4D,KAAKqpB,GAAGtX,EACjFkoD,EAAiBjoD,GAAK,EAAIqmD,GAAiBr4D,KAAKopB,KAAKpX,EAAIqmD,EAAgBr4D,KAAKqpB,GAAGrX,EAGnF,MAAOioD,IAGTp6D,EAAOD,QAAUwD,GAIb,SAASvD,EAAQD,EAASM,GAQ9B,QAASmD,KACPrD,KAAKyW,QACLzW,KAAKk6D,aAAe,EARXh6D,EAAoB,EAe/BmD,GAAO82D,UACJ/tD,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aAO3I9I,EAAO8P,UAAUsD,MAAQ,WACvBzW,KAAKm0B,UACLn0B,KAAKm0B,OAAOxuB,OAAS,WAEnB,GAAIH,GAAI,CACR,KAAM,GAAI9E,KAAKV,MACTA,KAAK8F,eAAepF,IACtB8E,GAGJ,OAAOA,KAWXnC,EAAO8P,UAAU+B,IAAM,SAAUkzC,GAC/B,GAAIn2C,GAAQjS,KAAKm0B,OAAOi0B,EACxB,IAAa5hD,QAATyL,EAAoB,CAEtB,GAAI9J,GAAQnI,KAAKk6D,aAAe72D,EAAO82D,QAAQx0D,MAC/C3F,MAAKk6D,eACLjoD,KACAA,EAAM9G,MAAQ9H,EAAO82D,QAAQhyD,GAC7BnI,KAAKm0B,OAAOi0B,GAAan2C,EAG3B,MAAOA,IAUT5O,EAAO8P,UAAUF,IAAM,SAAUm1C,EAAWn7C,GAE1C,MADAjN,MAAKm0B,OAAOi0B,GAAan7C,EAClBA,GAGTpN,EAAOD,QAAUyD,GAKb,SAASxD,GAMb,QAASyD,KACPtD,KAAK2iD,UACL3iD,KAAKo6D,eACLp6D,KAAKsI,SAAW9B,OAQlBlD,EAAO6P,UAAUyvC,kBAAoB,SAASt6C,GAC5CtI,KAAKsI,SAAWA,GASlBhF,EAAO6P,UAAUknD,KAAO,SAASC,EAAKC,GACpC,GAAIC,GAAMx6D,KAAK2iD,OAAO2X,EACtB,IAAY9zD,SAARg0D,EAAmB,CAErB,GAAIrmD,GAAKnU,IACTw6D,GAAM,GAAIC,OACVD,EAAIE,OAAS,WAEO,GAAd16D,KAAKuS,QACPhB,SAASojB,KAAKljB,YAAYzR,MAC1BA,KAAKuS,MAAQvS,KAAKowB,YAClBpwB,KAAKwS,OAASxS,KAAKswB,aACnB/e,SAASojB,KAAKxjB,YAAYnR,OAGxBmU,EAAG7L,WACL6L,EAAGwuC,OAAO2X,GAAOE,EACjBrmD,EAAG7L,SAAStI,QAIhBw6D,EAAIG,QAAU,WACMn0D,SAAd+zD,GACF5hC,QAAQiiC,MAAM,wBAAyBN,SAChCt6D,MAAK4lD,IACRzxC,EAAG7L,UACL6L,EAAG7L,SAAStI,OAGPmU,EAAGimD,YAAYE,MAAS,GAC/B3hC,QAAQiiC,MAAM,8BAA+BL,SACtCv6D,MAAK4lD,IACRzxC,EAAG7L,UACL6L,EAAG7L,SAAStI,QAIdA,KAAK4lD,IAAM2U,EACXpmD,EAAGimD,YAAYE,IAAO,IAI1BE,EAAI5U,IAAM0U,EAGZ,MAAOE,IAGT36D,EAAOD,QAAU0D,GAKb,SAASzD,EAAQD,EAASM,GA6B9B,QAASqD,GAAKorD,EAAYkM,EAAWC,EAAWpH,GAC9C,GAAIhS,GAAY/gD,EAAKsN,uBAAuB,SAASylD,EACrD1zD,MAAKyO,QAAUizC,EAAU1E,MAEzBh9C,KAAKqzC,UAAW,EAChBrzC,KAAKsM,OAAQ,EAEbtM,KAAK89C,SACL99C,KAAKmvD,gBACLnvD,KAAK+6D,iBAEL/6D,KAAKg7D,kBAAoB,EAGzBh7D,KAAKK,GAAKmG,OACVxG,KAAKwyD,gBAAiB,EACtBxyD,KAAKyyD,gBAAiB,EACtBzyD,KAAKmrD,QAAS,EACdnrD,KAAKorD,QAAS,EACdprD,KAAKi7D,qBAAsB,EAC3Bj7D,KAAKk7D,kBAAsB,EAC3Bl7D,KAAKm7D,gBAAkBzH,EAAiB1W,MAAMvxB,OAC9CzrB,KAAKo7D,aAAc,EACnBp7D,KAAK49C,MAAQ,GACb59C,KAAKq7D,kBAAmB,EACxBr7D,KAAKs7D,qBAAsB,EAC3Bt7D,KAAK8zD,iBAAmBnsD,IAAI,EAAGF,KAAK,EAAG8K,MAAM,EAAGC,OAAO,EAAGuhD,MAAM,GAChE/zD,KAAKomD,aAAez+C,IAAI,EAAGF,KAAK,EAAG4f,MAAM,EAAG/D,OAAO,GAEnDtjB,KAAK66D,UAAYA,EACjB76D,KAAK86D,UAAYA,EAGjB96D,KAAKu7D,GAAK,EACVv7D,KAAKw7D,GAAK,EACVx7D,KAAKy7D,GAAK,EACVz7D,KAAK07D,GAAK,EACV17D,KAAK+R,EAAI,KACT/R,KAAKgS,EAAI,KAGThS,KAAK27D,eAAiBF,GAAG,EAAEC,GAAG,EAAE3pD,EAAE,EAAEC,EAAE,GAEtChS,KAAK++C,QAAU2U,EAAiBlV,QAAQO,QACxC/+C,KAAKswD,WAAav+C,EAAE,KAAKC,EAAE,MAE3BhS,KAAK0uD,cAAcC,EAAYjN,GAG/B1hD,KAAK47D,eACL57D,KAAK67D,mBAAqB,EAC1B77D,KAAK87D,eAAiB,EACtB97D,KAAK+7D,uBAA0BrI,EAAiBvU,WAAWa,YAAYztC,MACvEvS,KAAKg8D,wBAA0BtI,EAAiBvU,WAAWa,YAAYxtC,OACvExS,KAAKi8D,wBAA0BvI,EAAiBvU,WAAWa,YAAYv0B,OACvEzrB,KAAKigD,sBAAwByT,EAAiBvU,WAAWc,sBACzDjgD,KAAKk8D,gBAAkB,EAGvBl8D,KAAKm2D,gBAAkB,EACvBn2D,KAAKm8D,aAAe,EACpBn8D,KAAK+jD,eAAiBhyC,EAAK,KAAMC,EAAK,MACtChS,KAAKgkD,mBAAqBjyC,EAAM,IAAKC,EAAM,KAC3ChS,KAAKiyD,aAAe,KA1FtB,GAAItxD,GAAOT,EAAoB,EAiG/BqD,GAAK4P,UAAU69C,eAAiB,WAC9BhxD,KAAK+R,EAAI/R,KAAK27D,cAAc5pD,EAC5B/R,KAAKgS,EAAIhS,KAAK27D,cAAc3pD,EAC5BhS,KAAKy7D,GAAKz7D,KAAK27D,cAAcF,GAC7Bz7D,KAAK07D,GAAK17D,KAAK27D,cAAcD,IAO/Bn4D,EAAK4P,UAAUyoD,aAAe,WAE5B57D,KAAKo8D,eAAiB51D,OACtBxG,KAAKq8D,YAAc,EACnBr8D,KAAKs8D,kBACLt8D,KAAKu8D,kBACLv8D,KAAKw8D,oBAOPj5D,EAAK4P,UAAU4hD,WAAa,SAASrH,GACH,IAA5B1tD,KAAK89C,MAAMn3C,QAAQ+mD,IACrB1tD,KAAK89C,MAAM91C,KAAK0lD,GAEqB,IAAnC1tD,KAAKmvD,aAAaxoD,QAAQ+mD,IAC5B1tD,KAAKmvD,aAAannD,KAAK0lD,GAEzB1tD,KAAK67D,mBAAqB77D,KAAKmvD,aAAaxpD,QAO9CpC,EAAK4P,UAAU6hD,WAAa,SAAStH,GACnC,GAAIvlD,GAAQnI,KAAK89C,MAAMn3C,QAAQ+mD,EAClB,KAATvlD,GACFnI,KAAK89C,MAAM11C,OAAOD,EAAO,GAE3BA,EAAQnI,KAAKmvD,aAAaxoD,QAAQ+mD,GACrB,IAATvlD,GACFnI,KAAKmvD,aAAa/mD,OAAOD,EAAO,GAElCnI,KAAK67D,mBAAqB77D,KAAKmvD,aAAaxpD,QAS9CpC,EAAK4P,UAAUu7C,cAAgB,SAASC,EAAYjN,GAClD,GAAKiN,EAAL,CAIA,GAAIzgD,IAAU,cAAc,sBAAsB,QAAQ,QAAQ,cAAc,SAAS,YACvF,WAAW,WAAW,WAAW,kBAAkB,kBAAkB,QAAQ,OAkB/E,IAhBAvN,EAAKwF,oBAAoB+H,EAAQlO,KAAKyO,QAASkgD,GAGzBnoD,SAAlBmoD,EAAWtuD,KAA0BL,KAAKK,GAAKsuD,EAAWtuD,IACrCmG,SAArBmoD,EAAWlmC,QAA0BzoB,KAAKyoB,MAAQkmC,EAAWlmC,MAAOzoB,KAAKy8D,cAAgB9N,EAAWlmC,OAC/EjiB,SAArBmoD,EAAW3pB,QAA0BhlC,KAAKglC,MAAQ2pB,EAAW3pB,OAC5Cx+B,SAAjBmoD,EAAW58C,IAA0B/R,KAAK+R,EAAI48C,EAAW58C,GACxCvL,SAAjBmoD,EAAW38C,IAA0BhS,KAAKgS,EAAI28C,EAAW38C,GACpCxL,SAArBmoD,EAAWtnD,QAA0BrH,KAAKqH,MAAQsnD,EAAWtnD,OACxCb,SAArBmoD,EAAW/Q,QAA0B59C,KAAK49C,MAAQ+Q,EAAW/Q,MAAO59C,KAAKq7D,kBAAmB,GAGzD70D,SAAnCmoD,EAAWsM,sBAAoCj7D,KAAKi7D,oBAAsBtM,EAAWsM,qBAClDz0D,SAAnCmoD,EAAWuM,mBAAoCl7D,KAAKk7D,iBAAsBvM,EAAWuM,kBAClD10D,SAAnCmoD,EAAW+N,kBAAoC18D,KAAK08D,gBAAsB/N,EAAW+N,iBAEzEl2D,SAAZxG,KAAKK,GACP,KAAM,sBAIR,IAAkC,gBAAvBL,MAAKyO,QAAQwD,OAAqD,gBAAvBjS,MAAKyO,QAAQwD,OAA4C,IAAtBjS,KAAKyO,QAAQwD,MAAc,CAClH,GAAI0qD,GAAW38D,KAAK86D,UAAU5lD,IAAIlV,KAAKyO,QAAQwD,MAC/CtR,GAAK8F,WAAWzG,KAAKyO,QAASkuD,GAE9B38D,KAAKyO,QAAQtD,MAAQxK,EAAKuK,WAAWlL,KAAKyO,QAAQtD,OAMpD,GAF0B3E,SAAtBmoD,EAAWljC,SAA+BzrB,KAAKm7D,gBAAkBn7D,KAAKyO,QAAQgd,QACzDjlB,SAArBmoD,EAAWxjD,QAA+BnL,KAAKyO,QAAQtD,MAAQxK,EAAKuK,WAAWyjD,EAAWxjD,QACnE3E,SAAvBxG,KAAKyO,QAAQ4uC,OAA4C,IAArBr9C,KAAKyO,QAAQ4uC,MAAY,CAC/D,IAAIr9C,KAAK66D,UAIP,KAAM,uBAHN76D,MAAK48D,SAAW58D,KAAK66D,UAAUR,KAAKr6D,KAAKyO,QAAQ4uC,MAAOr9C,KAAKyO,QAAQouD,aAgCzE,OAzBkCr2D,SAA9BmoD,EAAW6D,gBACbxyD,KAAKmrD,QAAUwD,EAAW6D,eAC1BxyD,KAAKwyD,eAAiB7D,EAAW6D,gBAEThsD,SAAjBmoD,EAAW58C,GAA0C,GAAvB/R,KAAKwyD,iBAC1CxyD,KAAKmrD,QAAS,GAIkB3kD,SAA9BmoD,EAAW8D,gBACbzyD,KAAKorD,QAAUuD,EAAW8D,eAC1BzyD,KAAKyyD,eAAiB9D,EAAW8D,gBAETjsD,SAAjBmoD,EAAW38C,GAA0C,GAAvBhS,KAAKyyD,iBAC1CzyD,KAAKorD,QAAS,GAGhBprD,KAAKo7D,YAAcp7D,KAAKo7D,aAAsC50D,SAAtBmoD,EAAWljC,QAExB,UAAvBzrB,KAAKyO,QAAQ2uC,OAA4C,kBAAvBp9C,KAAKyO,QAAQ2uC,SACjDp9C,KAAKyO,QAAQyuC,UAAYwE,EAAU1E,MAAM91B,SACzClnB,KAAKyO,QAAQ0uC,UAAYuE,EAAU1E,MAAM71B,UAInCnnB,KAAKyO,QAAQ2uC,OACnB,IAAK,WAAiBp9C,KAAK+rC,KAAO/rC,KAAK88D,cAAe98D,KAAKg2D,OAASh2D,KAAK+8D,eAAiB,MAC1F,KAAK,MAAiB/8D,KAAK+rC,KAAO/rC,KAAKg9D,SAAUh9D,KAAKg2D,OAASh2D,KAAKi9D,UAAY,MAChF,KAAK,SAAiBj9D,KAAK+rC,KAAO/rC,KAAKk9D,YAAal9D,KAAKg2D,OAASh2D,KAAKm9D,aAAe,MACtF,KAAK,UAAiBn9D,KAAK+rC,KAAO/rC,KAAKo9D,aAAcp9D,KAAKg2D,OAASh2D,KAAKq9D,cAAgB,MAExF,KAAK,QAAiBr9D,KAAK+rC,KAAO/rC,KAAKs9D,WAAYt9D,KAAKg2D,OAASh2D,KAAKu9D,YAAc,MACpF,KAAK,gBAAiBv9D,KAAK+rC,KAAO/rC,KAAKw9D,mBAAoBx9D,KAAKg2D,OAASh2D,KAAKy9D,oBAAsB,MACpG,KAAK,OAAiBz9D,KAAK+rC,KAAO/rC,KAAK09D,UAAW19D,KAAKg2D,OAASh2D,KAAK29D,WAAa,MAClF,KAAK,MAAiB39D,KAAK+rC,KAAO/rC,KAAK49D,SAAU59D,KAAKg2D,OAASh2D,KAAK69D,YAAc,MAClF,KAAK,SAAiB79D,KAAK+rC,KAAO/rC,KAAK89D,YAAa99D,KAAKg2D,OAASh2D,KAAK69D,YAAc,MACrF,KAAK,WAAiB79D,KAAK+rC,KAAO/rC,KAAK+9D,cAAe/9D,KAAKg2D,OAASh2D,KAAK69D,YAAc,MACvF,KAAK,eAAiB79D,KAAK+rC,KAAO/rC,KAAKg+D,kBAAmBh+D,KAAKg2D,OAASh2D,KAAK69D,YAAc,MAC3F,KAAK,OAAiB79D,KAAK+rC,KAAO/rC,KAAKi+D,UAAWj+D,KAAKg2D,OAASh2D,KAAK69D,YAAc,MACnF,SAAsB79D,KAAK+rC,KAAO/rC,KAAKo9D,aAAcp9D,KAAKg2D,OAASh2D,KAAKq9D,eAG1Er9D,KAAKk+D,WAOP36D,EAAK4P,UAAUg+B,OAAS,WACtBnxC,KAAKqzC,UAAW,EAChBrzC,KAAKk+D,UAMP36D,EAAK4P,UAAU+9B,SAAW,WACxBlxC,KAAKqzC,UAAW,EAChBrzC,KAAKk+D,UAOP36D,EAAK4P,UAAUgrD,eAAiB,WAC9Bn+D,KAAKk+D,UAOP36D,EAAK4P,UAAU+qD,OAAS,WACtBl+D,KAAKuS,MAAQ/L,OACbxG,KAAKwS,OAAShM,QAQhBjD,EAAK4P,UAAUq6C,SAAW,WACxB,MAA6B,kBAAfxtD,MAAKglC,MAAuBhlC,KAAKglC,QAAUhlC,KAAKglC,OAShEzhC,EAAK4P,UAAU0kD,iBAAmB,SAAU9wC,EAAKqnC,GAC/C,GAAIpuC,GAAc,CAMlB,QAJKhgB,KAAKuS,OACRvS,KAAKg2D,OAAOjvC,GAGN/mB,KAAKyO,QAAQ2uC,OACnB,IAAK,SACL,IAAK,MACH,MAAOp9C,MAAKyO,QAAQgd,OAAQzL,CAE9B,KAAK,UACH,GAAIza,GAAIvF,KAAKuS,MAAQ,EACjBnM,EAAIpG,KAAKwS,OAAS,EAClB68C,EAAKnqD,KAAKkZ,IAAIgwC,GAAS7oD,EACvBoG,EAAKzG,KAAKqZ,IAAI6vC,GAAShoD,CAC3B,OAAOb,GAAIa,EAAIlB,KAAKyqB,KAAK0/B,EAAIA,EAAI1jD,EAAIA,EAMvC,KAAK,MACL,IAAK,QACL,IAAK,OACL,QACE,MAAI3L,MAAKuS,MACArN,KAAK4G,IACR5G,KAAK2lB,IAAI7qB,KAAKuS,MAAQ,EAAIrN,KAAKqZ,IAAI6vC,IACnClpD,KAAK2lB,IAAI7qB,KAAKwS,OAAS,EAAItN,KAAKkZ,IAAIgwC,KAAWpuC,EAI5C,IAYfzc,EAAK4P,UAAUirD,UAAY,SAAS7C,EAAIC,GACtCx7D,KAAKu7D,GAAKA,EACVv7D,KAAKw7D,GAAKA,GASZj4D,EAAK4P,UAAUkrD,UAAY,SAAS9C,EAAIC,GACtCx7D,KAAKu7D,IAAMA,EACXv7D,KAAKw7D,IAAMA,GAMbj4D,EAAK4P,UAAUmrD,WAAa,WAC1Bt+D,KAAK27D,cAAc5pD,EAAI/R,KAAK+R,EAC5B/R,KAAK27D,cAAc3pD,EAAIhS,KAAKgS,EAC5BhS,KAAK27D,cAAcF,GAAKz7D,KAAKy7D,GAC7Bz7D,KAAK27D,cAAcD,GAAK17D,KAAK07D,IAO/Bn4D,EAAK4P,UAAU09C,aAAe,SAASr+B,GAErC,GADAxyB,KAAKs+D,aACAt+D,KAAKmrD,OAORnrD,KAAKu7D,GAAK,EACVv7D,KAAKy7D,GAAK,MARM,CAChB,GAAI78C,GAAO5e,KAAK++C,QAAU/+C,KAAKy7D,GAC3B79C,GAAQ5d,KAAKu7D,GAAK38C,GAAM5e,KAAKyO,QAAQwuC,IACzCj9C,MAAKy7D,IAAM79C,EAAK4U,EAChBxyB,KAAK+R,GAAM/R,KAAKy7D,GAAKjpC,EAOvB,GAAKxyB,KAAKorD,OAORprD,KAAKw7D,GAAK,EACVx7D,KAAK07D,GAAK,MARM,CAChB,GAAI78C,GAAO7e,KAAK++C,QAAU/+C,KAAK07D,GAC3B79C,GAAQ7d,KAAKw7D,GAAK38C,GAAM7e,KAAKyO,QAAQwuC,IACzCj9C,MAAK07D,IAAM79C,EAAK2U,EAChBxyB,KAAKgS,GAAMhS,KAAK07D,GAAKlpC,IAezBjvB,EAAK4P,UAAUy9C,oBAAsB,SAASp+B,EAAUyuB,GAEtD,GADAjhD,KAAKs+D,aACAt+D,KAAKmrD,OAQRnrD,KAAKu7D,GAAK,EACVv7D,KAAKy7D,GAAK,MATM,CAChB,GAAI78C,GAAO5e,KAAK++C,QAAU/+C,KAAKy7D,GAC3B79C,GAAQ5d,KAAKu7D,GAAK38C,GAAM5e,KAAKyO,QAAQwuC,IACzCj9C,MAAKy7D,IAAM79C,EAAK4U,EAChBxyB,KAAKy7D,GAAMv2D,KAAK2lB,IAAI7qB,KAAKy7D,IAAMxa,EAAiBjhD,KAAKy7D,GAAK,EAAKxa,GAAeA,EAAejhD,KAAKy7D,GAClGz7D,KAAK+R,GAAM/R,KAAKy7D,GAAKjpC,EAOvB,GAAKxyB,KAAKorD,OAQRprD,KAAKw7D,GAAK,EACVx7D,KAAK07D,GAAK,MATM,CAChB,GAAI78C,GAAO7e,KAAK++C,QAAU/+C,KAAK07D,GAC3B79C,GAAQ7d,KAAKw7D,GAAK38C,GAAM7e,KAAKyO,QAAQwuC,IACzCj9C,MAAK07D,IAAM79C,EAAK2U,EAChBxyB,KAAK07D,GAAMx2D,KAAK2lB,IAAI7qB,KAAK07D,IAAMza,EAAiBjhD,KAAK07D,GAAK,EAAKza,GAAeA,EAAejhD,KAAK07D,GAClG17D,KAAKgS,GAAMhS,KAAK07D,GAAKlpC,IAYzBjvB,EAAK4P,UAAUorD,QAAU,WACvB,MAAQv+D,MAAKmrD,QAAUnrD,KAAKorD,QAQ9B7nD,EAAK4P,UAAUs9C,SAAW,SAASD,GACjC,GAAIgO,GAAWt5D,KAAKyqB,KAAKzqB,KAAK4uB,IAAI9zB,KAAKy7D,GAAG,GAAKv2D,KAAK4uB,IAAI9zB,KAAK07D,GAAG,GAEhE,OAAQ8C,GAAWhO,GAOrBjtD,EAAK4P,UAAU23C,WAAa,WAC1B,MAAO9qD,MAAKqzC,UAOd9vC,EAAK4P,UAAUyB,SAAW,WACxB,MAAO5U,MAAKqH,OASd9D,EAAK4P,UAAUsrD,YAAc,SAAS1sD,EAAGC,GACvC,GAAI4M,GAAK5e,KAAK+R,EAAIA,EACd8M,EAAK7e,KAAKgS,EAAIA,CAClB,OAAO9M,MAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,IAUlCtb,EAAK4P,UAAUi8C,cAAgB,SAAStjD,EAAKY,GAC3C,IAAK1M,KAAKo7D,aAA8B50D,SAAfxG,KAAKqH,MAC5B,GAAIqF,GAAOZ,EACT9L,KAAKyO,QAAQgd,QAASzrB,KAAKyO,QAAQyuC,UAAYl9C,KAAKyO,QAAQ0uC,WAAa,MAEtE,CACH,GAAIlgC,IAASjd,KAAKyO,QAAQ0uC,UAAYn9C,KAAKyO,QAAQyuC,YAAcxwC,EAAMZ,EACvE9L,MAAKyO,QAAQgd,QAASzrB,KAAKqH,MAAQyE,GAAOmR,EAAQjd,KAAKyO,QAAQyuC,UAGnEl9C,KAAKm7D,gBAAkBn7D,KAAKyO,QAAQgd,QAQtCloB,EAAK4P,UAAU44B,KAAO,WACpB,KAAM,wCAQRxoC,EAAK4P,UAAU6iD,OAAS,WACtB,KAAM,0CAQRzyD,EAAK4P,UAAUo6C,kBAAoB,SAASxqC,GAC1C,MAAQ/iB,MAAKyH,KAAoBsb,EAAIsE,OAC7BrnB,KAAKyH,KAAOzH,KAAKuS,MAAQwQ,EAAItb,MAC7BzH,KAAK2H,IAAoBob,EAAIO,QAC7BtjB,KAAK2H,IAAM3H,KAAKwS,OAASuQ,EAAIpb,KAGvCpE,EAAK4P,UAAUoqD,aAAe,WAG5B,IAAKv9D,KAAKuS,QAAUvS,KAAKwS,OAAQ,CAC/B,GAAID,GAAOC,CACX,IAAIxS,KAAKqH,MAAO,CACdrH,KAAKyO,QAAQgd,OAAQzrB,KAAKm7D,eAC1B,IAAIl+C,GAAQjd,KAAK48D,SAASpqD,OAASxS,KAAK48D,SAASrqD,KACnC/L,UAAVyW,GACF1K,EAAQvS,KAAKyO,QAAQgd,QAASzrB,KAAK48D,SAASrqD,MAC5CC,EAASxS,KAAKyO,QAAQgd,OAAQxO,GAASjd,KAAK48D,SAASpqD,SAGrDD,EAAQ,EACRC,EAAS,OAIXD,GAAQvS,KAAK48D,SAASrqD,MACtBC,EAASxS,KAAK48D,SAASpqD,MAEzBxS,MAAKuS,MAASA,EACdvS,KAAKwS,OAASA,EAEdxS,KAAKk8D,gBAAkB,EACnBl8D,KAAKuS,MAAQ,GAAKvS,KAAKwS,OAAS,IAClCxS,KAAKuS,OAAUrN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAA0BjgD,KAAK+7D,uBAClF/7D,KAAKwS,QAAUtN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKg8D,wBACjFh8D,KAAKyO,QAAQgd,QAASvmB,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKi8D,wBACxFj8D,KAAKk8D,gBAAkBl8D,KAAKuS,MAAQA,KAK1ChP,EAAK4P,UAAUurD,qBAAuB,SAAU33C,GAC9C,GAA2B,GAAvB/mB,KAAK48D,SAASrqD,MAAa,CAE7B,GAAIvS,KAAKq8D,YAAc,EAAG,CACxB,GAAI/0C,GAActnB,KAAKq8D,YAAc,EAAK,GAAK,CAC/C/0C,IAAatnB,KAAKm2D,gBAClB7uC,EAAYpiB,KAAK4G,IAAI,GAAM9L,KAAKuS,MAAM+U,GAEtCP,EAAI43C,YAAc,GAClB53C,EAAI63C,UAAU5+D,KAAK48D,SAAU58D,KAAKyH,KAAO6f,EAAWtnB,KAAK2H,IAAM2f,EAAWtnB,KAAKuS,MAAQ,EAAE+U,EAAWtnB,KAAKwS,OAAS,EAAE8U,GAItHP,EAAI43C,YAAc,EAClB53C,EAAI63C,UAAU5+D,KAAK48D,SAAU58D,KAAKyH,KAAMzH,KAAK2H,IAAK3H,KAAKuS,MAAOvS,KAAKwS,UAIvEjP,EAAK4P,UAAU0rD,gBAAkB,SAAU93C,GACzC,GAAIjN,GACA6P,EAAS,CAEb,IAAI3pB,KAAKwS,OAAO,CACdmX,EAAS3pB,KAAKwS,OAAS,CACvB,IAAIshD,GAAkB9zD,KAAK8+D,YAAY/3C,EAEnC+sC,GAAgB0C,WAAa,IAC/B7sC,GAAUmqC,EAAgBthD,OAAS,EACnCmX,GAAU,GAId7P,EAAS9Z,KAAKgS,EAAI2X,EAElB3pB,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOzoB,KAAK+R,EAAG+H,EAAQtT,SAG/CjD,EAAK4P,UAAUmqD,WAAa,SAAUv2C,GACpC/mB,KAAKu9D,aAAax2C,GAClB/mB,KAAKyH,KAASzH,KAAK+R,EAAI/R,KAAKuS,MAAQ,EACpCvS,KAAK2H,IAAS3H,KAAKgS,EAAIhS,KAAKwS,OAAS,EAErCxS,KAAK0+D,qBAAqB33C,GAE1B/mB,KAAKomD,YAAYz+C,IAAM3H,KAAK2H,IAC5B3H,KAAKomD,YAAY3+C,KAAOzH,KAAKyH,KAC7BzH,KAAKomD,YAAY/+B,MAAQrnB,KAAKyH,KAAOzH,KAAKuS,MAC1CvS,KAAKomD,YAAY9iC,OAAStjB,KAAK2H,IAAM3H,KAAKwS,OAE1CxS,KAAK6+D,gBAAgB93C,GACrB/mB,KAAKomD,YAAY3+C,KAAOvC,KAAK4G,IAAI9L,KAAKomD,YAAY3+C,KAAMzH,KAAK8zD,gBAAgBrsD,MAC7EzH,KAAKomD,YAAY/+B,MAAQniB,KAAKwH,IAAI1M,KAAKomD,YAAY/+B,MAAOrnB,KAAK8zD,gBAAgBrsD,KAAOzH,KAAK8zD,gBAAgBvhD,OAC3GvS,KAAKomD,YAAY9iC,OAASpe,KAAKwH,IAAI1M,KAAKomD,YAAY9iC,OAAQtjB,KAAKomD,YAAY9iC,OAAStjB,KAAK8zD,gBAAgBthD,SAG7GjP,EAAK4P,UAAUsqD,qBAAuB,SAAU12C,GAC9C,GAAI/mB,KAAK48D,SAAShX,KAAQ5lD,KAAK48D,SAASrqD,OAAUvS,KAAK48D,SAASpqD,OAe1DxS,KAAK++D,oCACP/+D,KAAKuS,MAAQ,EACbvS,KAAKwS,OAAS,QACPxS,MAAK++D,mCAEd/+D,KAAKu9D,aAAax2C,OAnBlB,KAAK/mB,KAAKuS,MAAO,CACf,GAAIysD,GAAiC,EAAtBh/D,KAAKyO,QAAQgd,MAC5BzrB,MAAKuS,MAAQysD,EACbh/D,KAAKwS,OAASwsD,EAKdh/D,KAAKyO,QAAQgd,QAAuE,GAA7DvmB,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAA+BjgD,KAAKi8D,wBAC/Fj8D,KAAKk8D,gBAAkBl8D,KAAKyO,QAAQgd,OAAQ,GAAIuzC,EAChDh/D,KAAK++D,mCAAoC,IAc/Cx7D,EAAK4P,UAAUqqD,mBAAqB,SAAUz2C,GAC5C/mB,KAAKy9D,qBAAqB12C,GAE1B/mB,KAAKyH,KAASzH,KAAK+R,EAAI/R,KAAKuS,MAAQ,EACpCvS,KAAK2H,IAAS3H,KAAKgS,EAAIhS,KAAKwS,OAAS,CAErC,IAAIysD,GAAUj/D,KAAKyH,KAAQzH,KAAKuS,MAAQ,EACpC2sD,EAAUl/D,KAAK2H,IAAO3H,KAAKwS,OAAS,EACpCiZ,EAASvmB,KAAK2lB,IAAI7qB,KAAKwS,OAAS,EAEpCxS,MAAKm/D,eAAep4C,EAAKk4C,EAASC,EAASzzC,GAE3C1E,EAAIuoC,OACJvoC,EAAIq4C,OAAOp/D,KAAK+R,EAAG/R,KAAKgS,EAAGyZ,GAC3B1E,EAAIlH,SACJkH,EAAIs4C,OAEJr/D,KAAK0+D,qBAAqB33C,GAE1BA,EAAI0oC,UAEJzvD,KAAKomD,YAAYz+C,IAAM3H,KAAKgS,EAAIhS,KAAKyO,QAAQgd,OAC7CzrB,KAAKomD,YAAY3+C,KAAOzH,KAAK+R,EAAI/R,KAAKyO,QAAQgd,OAC9CzrB,KAAKomD,YAAY/+B,MAAQrnB,KAAK+R,EAAI/R,KAAKyO,QAAQgd,OAC/CzrB,KAAKomD,YAAY9iC,OAAStjB,KAAKgS,EAAIhS,KAAKyO,QAAQgd,OAEhDzrB,KAAK6+D,gBAAgB93C,GAErB/mB,KAAKomD,YAAY3+C,KAAOvC,KAAK4G,IAAI9L,KAAKomD,YAAY3+C,KAAMzH,KAAK8zD,gBAAgBrsD,MAC7EzH,KAAKomD,YAAY/+B,MAAQniB,KAAKwH,IAAI1M,KAAKomD,YAAY/+B,MAAOrnB,KAAK8zD,gBAAgBrsD,KAAOzH,KAAK8zD,gBAAgBvhD,OAC3GvS,KAAKomD,YAAY9iC,OAASpe,KAAKwH,IAAI1M,KAAKomD,YAAY9iC,OAAQtjB,KAAKomD,YAAY9iC,OAAStjB,KAAK8zD,gBAAgBthD,SAG7GjP,EAAK4P,UAAU8pD,WAAa,SAAUl2C,GACpC,IAAK/mB,KAAKuS,MAAO,CACf,GAAImH,GAAS,EACT4lD,EAAWt/D,KAAK8+D,YAAY/3C,EAChC/mB,MAAKuS,MAAQ+sD,EAAS/sD,MAAQ,EAAImH,EAClC1Z,KAAKwS,OAAS8sD,EAAS9sD,OAAS,EAAIkH,EAEpC1Z,KAAKuS,OAAuE,GAA7DrN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAA+BjgD,KAAK+7D,uBACvF/7D,KAAKwS,QAAuE,GAA7DtN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAA+BjgD,KAAKg8D,wBACvFh8D,KAAKk8D,gBAAkBl8D,KAAKuS,OAAS+sD,EAAS/sD,MAAQ,EAAImH,KAM9DnW,EAAK4P,UAAU6pD,SAAW,SAAUj2C,GAClC/mB,KAAKi9D,WAAWl2C,GAEhB/mB,KAAKyH,KAAOzH,KAAK+R,EAAI/R,KAAKuS,MAAQ,EAClCvS,KAAK2H,IAAM3H,KAAKgS,EAAIhS,KAAKwS,OAAS,CAElC,IAAI+sD,GAAmB,IACnBv/C,EAAchgB,KAAKyO,QAAQuR,YAC3Bw/C,EAAqBx/D,KAAKyO,QAAQovC,qBAAuB,EAAI79C,KAAKyO,QAAQuR,WAE9E+G,GAAIY,YAAc3nB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUD,OAASpM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMF,OAASpM,KAAKyO,QAAQtD,MAAMiB,OAGtIpM,KAAKq8D,YAAc,IACrBt1C,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAI04C,UAAUz/D,KAAKyH,KAAK,EAAEsf,EAAIO,UAAWtnB,KAAK2H,IAAI,EAAEof,EAAIO,UAAWtnB,KAAKuS,MAAM,EAAEwU,EAAIO,UAAWtnB,KAAKwS,OAAO,EAAEuU,EAAIO,UAAWtnB,KAAKyO,QAAQgd,QACzI1E,EAAIlH,UAENkH,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAIiB,UAAYhoB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUF,WAAanM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMH,WAAanM,KAAKyO,QAAQtD,MAAMgB,WAEhJ4a,EAAI04C,UAAUz/D,KAAKyH,KAAMzH,KAAK2H,IAAK3H,KAAKuS,MAAOvS,KAAKwS,OAAQxS,KAAKyO,QAAQgd,QACzE1E,EAAInH,OACJmH,EAAIlH,SAEJ7f,KAAKomD,YAAYz+C,IAAM3H,KAAK2H,IAC5B3H,KAAKomD,YAAY3+C,KAAOzH,KAAKyH,KAC7BzH,KAAKomD,YAAY/+B,MAAQrnB,KAAKyH,KAAOzH,KAAKuS,MAC1CvS,KAAKomD,YAAY9iC,OAAStjB,KAAK2H,IAAM3H,KAAKwS,OAE1CxS,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOzoB,KAAK+R,EAAG/R,KAAKgS,IAI5CzO,EAAK4P,UAAU4pD,gBAAkB,SAAUh2C,GACzC,IAAK/mB,KAAKuS,MAAO,CACf,GAAImH,GAAS,EACT4lD,EAAWt/D,KAAK8+D,YAAY/3C,GAC5B1U,EAAOitD,EAAS/sD,MAAQ,EAAImH,CAChC1Z,MAAKuS,MAAQF,EACbrS,KAAKwS,OAASH,EAGdrS,KAAKuS,OAAUrN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAK+7D,uBACjF/7D,KAAKwS,QAAUtN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKg8D,wBACjFh8D,KAAKyO,QAAQgd,QAASvmB,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKi8D,wBACxFj8D,KAAKk8D,gBAAkBl8D,KAAKuS,MAAQF,IAIxC9O,EAAK4P,UAAU2pD,cAAgB,SAAU/1C,GACvC/mB,KAAK+8D,gBAAgBh2C,GACrB/mB,KAAKyH,KAAOzH,KAAK+R,EAAI/R,KAAKuS,MAAQ,EAClCvS,KAAK2H,IAAM3H,KAAKgS,EAAIhS,KAAKwS,OAAS,CAElC,IAAI+sD,GAAmB,IACnBv/C,EAAchgB,KAAKyO,QAAQuR,YAC3Bw/C,EAAqBx/D,KAAKyO,QAAQovC,qBAAuB,EAAI79C,KAAKyO,QAAQuR,WAE9E+G,GAAIY,YAAc3nB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUD,OAASpM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMF,OAASpM,KAAKyO,QAAQtD,MAAMiB,OAGtIpM,KAAKq8D,YAAc,IACrBt1C,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAI24C,SAAS1/D,KAAK+R,EAAI/R,KAAKuS,MAAM,EAAI,EAAEwU,EAAIO,UAAWtnB,KAAKgS,EAAgB,GAAZhS,KAAKwS,OAAa,EAAEuU,EAAIO,UAAWtnB,KAAKuS,MAAQ,EAAEwU,EAAIO,UAAWtnB,KAAKwS,OAAS,EAAEuU,EAAIO,WACpJP,EAAIlH,UAENkH,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAIiB,UAAYhoB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUF,WAAanM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMH,WAAanM,KAAKyO,QAAQtD,MAAMgB,WAChJ4a,EAAI24C,SAAS1/D,KAAK+R,EAAI/R,KAAKuS,MAAM,EAAGvS,KAAKgS,EAAgB,GAAZhS,KAAKwS,OAAYxS,KAAKuS,MAAOvS,KAAKwS,QAC/EuU,EAAInH,OACJmH,EAAIlH,SAEJ7f,KAAKomD,YAAYz+C,IAAM3H,KAAK2H,IAC5B3H,KAAKomD,YAAY3+C,KAAOzH,KAAKyH,KAC7BzH,KAAKomD,YAAY/+B,MAAQrnB,KAAKyH,KAAOzH,KAAKuS,MAC1CvS,KAAKomD,YAAY9iC,OAAStjB,KAAK2H,IAAM3H,KAAKwS,OAE1CxS,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOzoB,KAAK+R,EAAG/R,KAAKgS;EAI5CzO,EAAK4P,UAAUgqD,cAAgB,SAAUp2C,GACvC,IAAK/mB,KAAKuS,MAAO,CACf,GAAImH,GAAS,EACT4lD,EAAWt/D,KAAK8+D,YAAY/3C,GAC5Bi4C,EAAW95D,KAAKwH,IAAI4yD,EAAS/sD,MAAO+sD,EAAS9sD,QAAU,EAAIkH,CAC/D1Z,MAAKyO,QAAQgd,OAASuzC,EAAW,EAEjCh/D,KAAKuS,MAAQysD,EACbh/D,KAAKwS,OAASwsD,EAKdh/D,KAAKyO,QAAQgd,QAAuE,GAA7DvmB,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAA+BjgD,KAAKi8D,wBAC/Fj8D,KAAKk8D,gBAAkBl8D,KAAKyO,QAAQgd,OAAQ,GAAIuzC,IAIpDz7D,EAAK4P,UAAUgsD,eAAiB,SAAUp4C,EAAKhV,EAAGC,EAAGyZ,GACnD,GAAI8zC,GAAmB,IACnBv/C,EAAchgB,KAAKyO,QAAQuR,YAC3Bw/C,EAAqBx/D,KAAKyO,QAAQovC,qBAAuB,EAAI79C,KAAKyO,QAAQuR,WAE9E+G,GAAIY,YAAc3nB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUD,OAASpM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMF,OAASpM,KAAKyO,QAAQtD,MAAMiB,OAGtIpM,KAAKq8D,YAAc,IACrBt1C,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAIq4C,OAAOrtD,EAAGC,EAAGyZ,EAAO,EAAE1E,EAAIO,WAC9BP,EAAIlH,UAENkH,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAIiB,UAAYhoB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUF,WAAanM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMH,WAAanM,KAAKyO,QAAQtD,MAAMgB,WAChJ4a,EAAIq4C,OAAOp/D,KAAK+R,EAAG/R,KAAKgS,EAAGyZ,GAC3B1E,EAAInH,OACJmH,EAAIlH,UAGNtc,EAAK4P,UAAU+pD,YAAc,SAAUn2C,GACrC/mB,KAAKm9D,cAAcp2C,GACnB/mB,KAAKyH,KAAOzH,KAAK+R,EAAI/R,KAAKuS,MAAQ,EAClCvS,KAAK2H,IAAM3H,KAAKgS,EAAIhS,KAAKwS,OAAS,EAElCxS,KAAKm/D,eAAep4C,EAAK/mB,KAAK+R,EAAG/R,KAAKgS,EAAGhS,KAAKyO,QAAQgd,QAEtDzrB,KAAKomD,YAAYz+C,IAAM3H,KAAKgS,EAAIhS,KAAKyO,QAAQgd,OAC7CzrB,KAAKomD,YAAY3+C,KAAOzH,KAAK+R,EAAI/R,KAAKyO,QAAQgd,OAC9CzrB,KAAKomD,YAAY/+B,MAAQrnB,KAAK+R,EAAI/R,KAAKyO,QAAQgd,OAC/CzrB,KAAKomD,YAAY9iC,OAAStjB,KAAKgS,EAAIhS,KAAKyO,QAAQgd,OAEhDzrB,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOzoB,KAAK+R,EAAG/R,KAAKgS,IAG5CzO,EAAK4P,UAAUkqD,eAAiB,SAAUt2C,GACxC,IAAK/mB,KAAKuS,MAAO,CACf,GAAI+sD,GAAWt/D,KAAK8+D,YAAY/3C,EAEhC/mB,MAAKuS,MAAyB,IAAjB+sD,EAAS/sD,MACtBvS,KAAKwS,OAA2B,EAAlB8sD,EAAS9sD,OACnBxS,KAAKuS,MAAQvS,KAAKwS,SACpBxS,KAAKuS,MAAQvS,KAAKwS,OAEpB,IAAImtD,GAAc3/D,KAAKuS,KAGvBvS,MAAKuS,OAAUrN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAK+7D,uBACjF/7D,KAAKwS,QAAUtN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKg8D,wBACjFh8D,KAAKyO,QAAQgd,QAAUvmB,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKi8D,wBACzFj8D,KAAKk8D,gBAAkBl8D,KAAKuS,MAAQotD,IAIxCp8D,EAAK4P,UAAUiqD,aAAe,SAAUr2C,GACtC/mB,KAAKq9D,eAAet2C,GACpB/mB,KAAKyH,KAAOzH,KAAK+R,EAAI/R,KAAKuS,MAAQ,EAClCvS,KAAK2H,IAAM3H,KAAKgS,EAAIhS,KAAKwS,OAAS,CAElC,IAAI+sD,GAAmB,IACnBv/C,EAAchgB,KAAKyO,QAAQuR,YAC3Bw/C,EAAqBx/D,KAAKyO,QAAQovC,qBAAuB,EAAI79C,KAAKyO,QAAQuR,WAE9E+G,GAAIY,YAAc3nB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUD,OAASpM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMF,OAASpM,KAAKyO,QAAQtD,MAAMiB,OAGtIpM,KAAKq8D,YAAc,IACrBt1C,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAI64C,QAAQ5/D,KAAKyH,KAAK,EAAEsf,EAAIO,UAAWtnB,KAAK2H,IAAI,EAAEof,EAAIO,UAAWtnB,KAAKuS,MAAM,EAAEwU,EAAIO,UAAWtnB,KAAKwS,OAAO,EAAEuU,EAAIO,WAC/GP,EAAIlH,UAENkH,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAIiB,UAAYhoB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUF,WAAanM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMH,WAAanM,KAAKyO,QAAQtD,MAAMgB,WAEhJ4a,EAAI64C,QAAQ5/D,KAAKyH,KAAMzH,KAAK2H,IAAK3H,KAAKuS,MAAOvS,KAAKwS,QAClDuU,EAAInH,OACJmH,EAAIlH,SAEJ7f,KAAKomD,YAAYz+C,IAAM3H,KAAK2H,IAC5B3H,KAAKomD,YAAY3+C,KAAOzH,KAAKyH,KAC7BzH,KAAKomD,YAAY/+B,MAAQrnB,KAAKyH,KAAOzH,KAAKuS,MAC1CvS,KAAKomD,YAAY9iC,OAAStjB,KAAK2H,IAAM3H,KAAKwS,OAE1CxS,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOzoB,KAAK+R,EAAG/R,KAAKgS,IAG5CzO,EAAK4P,UAAUyqD,SAAW,SAAU72C,GAClC/mB,KAAK6/D,WAAW94C,EAAK,WAGvBxjB,EAAK4P,UAAU4qD,cAAgB,SAAUh3C,GACvC/mB,KAAK6/D,WAAW94C,EAAK,aAGvBxjB,EAAK4P,UAAU6qD,kBAAoB,SAAUj3C,GAC3C/mB,KAAK6/D,WAAW94C,EAAK,iBAGvBxjB,EAAK4P,UAAU2qD,YAAc,SAAU/2C,GACrC/mB,KAAK6/D,WAAW94C,EAAK,WAGvBxjB,EAAK4P,UAAU8qD,UAAY,SAAUl3C,GACnC/mB,KAAK6/D,WAAW94C,EAAK,SAGvBxjB,EAAK4P,UAAU0qD,aAAe,WAC5B,IAAK79D,KAAKuS,MAAO,CACfvS,KAAKyO,QAAQgd,OAAQzrB,KAAKm7D,eAC1B,IAAI9oD,GAAO,EAAIrS,KAAKyO,QAAQgd,MAC5BzrB,MAAKuS,MAAQF,EACbrS,KAAKwS,OAASH,EAGdrS,KAAKuS,OAAUrN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAK+7D,uBACjF/7D,KAAKwS,QAAUtN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKg8D,wBACjFh8D,KAAKyO,QAAQgd,QAAsE,GAA7DvmB,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAA+BjgD,KAAKi8D,wBAC9Fj8D,KAAKk8D,gBAAkBl8D,KAAKuS,MAAQF,IAIxC9O,EAAK4P,UAAU0sD,WAAa,SAAU94C,EAAKq2B,GACzCp9C,KAAK69D,aAAa92C,GAElB/mB,KAAKyH,KAAOzH,KAAK+R,EAAI/R,KAAKuS,MAAQ,EAClCvS,KAAK2H,IAAM3H,KAAKgS,EAAIhS,KAAKwS,OAAS,CAElC,IAAI+sD,GAAmB,IACnBv/C,EAAchgB,KAAKyO,QAAQuR,YAC3Bw/C,EAAqBx/D,KAAKyO,QAAQovC,qBAAuB,EAAI79C,KAAKyO,QAAQuR,YAC1E8/C,EAAmB,CAGvB,QAAQ1iB,GACN,IAAK,MAAiB0iB,EAAmB,CAAG,MAC5C,KAAK,SAAiBA,EAAmB,CAAG,MAC5C,KAAK,WAAiBA,EAAmB,CAAG,MAC5C,KAAK,eAAiBA,EAAmB,CAAG,MAC5C,KAAK,OAAiBA,EAAmB,EAG3C/4C,EAAIY,YAAc3nB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUD,OAASpM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMF,OAASpM,KAAKyO,QAAQtD,MAAMiB,OAEtIpM,KAAKq8D,YAAc,IACrBt1C,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAIq2B,GAAOp9C,KAAK+R,EAAG/R,KAAKgS,EAAGhS,KAAKyO,QAAQgd,OAAQq0C,EAAmB/4C,EAAIO,WACvEP,EAAIlH,UAENkH,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAIiB,UAAYhoB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUF,WAAanM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMH,WAAanM,KAAKyO,QAAQtD,MAAMgB,WAChJ4a,EAAIq2B,GAAOp9C,KAAK+R,EAAG/R,KAAKgS,EAAGhS,KAAKyO,QAAQgd,QACxC1E,EAAInH,OACJmH,EAAIlH,SAEJ7f,KAAKomD,YAAYz+C,IAAM3H,KAAKgS,EAAIhS,KAAKyO,QAAQgd,OAC7CzrB,KAAKomD,YAAY3+C,KAAOzH,KAAK+R,EAAI/R,KAAKyO,QAAQgd,OAC9CzrB,KAAKomD,YAAY/+B,MAAQrnB,KAAK+R,EAAI/R,KAAKyO,QAAQgd,OAC/CzrB,KAAKomD,YAAY9iC,OAAStjB,KAAKgS,EAAIhS,KAAKyO,QAAQgd,OAE5CzrB,KAAKyoB,QACPzoB,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOzoB,KAAK+R,EAAG/R,KAAKgS,EAAIhS,KAAKwS,OAAS,EAAGhM,OAAW,WAAU,GACpFxG,KAAKomD,YAAY3+C,KAAOvC,KAAK4G,IAAI9L,KAAKomD,YAAY3+C,KAAMzH,KAAK8zD,gBAAgBrsD,MAC7EzH,KAAKomD,YAAY/+B,MAAQniB,KAAKwH,IAAI1M,KAAKomD,YAAY/+B,MAAOrnB,KAAK8zD,gBAAgBrsD,KAAOzH,KAAK8zD,gBAAgBvhD,OAC3GvS,KAAKomD,YAAY9iC,OAASpe,KAAKwH,IAAI1M,KAAKomD,YAAY9iC,OAAQtjB,KAAKomD,YAAY9iC,OAAStjB,KAAK8zD,gBAAgBthD,UAI/GjP,EAAK4P,UAAUwqD,YAAc,SAAU52C,GACrC,IAAK/mB,KAAKuS,MAAO,CACf,GAAImH,GAAS,EACT4lD,EAAWt/D,KAAK8+D,YAAY/3C,EAChC/mB,MAAKuS,MAAQ+sD,EAAS/sD,MAAQ,EAAImH,EAClC1Z,KAAKwS,OAAS8sD,EAAS9sD,OAAS,EAAIkH,EAGpC1Z,KAAKuS,OAAUrN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAK+7D,uBACjF/7D,KAAKwS,QAAUtN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKg8D,wBACjFh8D,KAAKyO,QAAQgd,QAASvmB,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKi8D,wBACxFj8D,KAAKk8D,gBAAkBl8D,KAAKuS,OAAS+sD,EAAS/sD,MAAQ,EAAImH,KAI9DnW,EAAK4P,UAAUuqD,UAAY,SAAU32C,GACnC/mB,KAAK29D,YAAY52C,GACjB/mB,KAAKyH,KAAOzH,KAAK+R,EAAI/R,KAAKuS,MAAQ,EAClCvS,KAAK2H,IAAM3H,KAAKgS,EAAIhS,KAAKwS,OAAS,EAElCxS,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOzoB,KAAK+R,EAAG/R,KAAKgS,GAE1ChS,KAAKomD,YAAYz+C,IAAM3H,KAAK2H,IAC5B3H,KAAKomD,YAAY3+C,KAAOzH,KAAKyH,KAC7BzH,KAAKomD,YAAY/+B,MAAQrnB,KAAKyH,KAAOzH,KAAKuS,MAC1CvS,KAAKomD,YAAY9iC,OAAStjB,KAAK2H,IAAM3H,KAAKwS,QAI5CjP,EAAK4P,UAAU4iD,OAAS,SAAUhvC,EAAKwC,EAAMxX,EAAGC,EAAG+8B,EAAOgxB,EAAUC,GAClE,GAAIz2C,GAAQrlB,OAAOlE,KAAKyO,QAAQ8uC,UAAYv9C,KAAKm8D,aAAen8D,KAAKg7D,kBAAmB,CACtFj0C,EAAIQ,MAAQvnB,KAAKqzC,SAAW,QAAU,IAAMrzC,KAAKyO,QAAQ8uC,SAAW,MAAQv9C,KAAKyO,QAAQ+uC,QAEzF,IAAIhX,GAAQjd,EAAKxhB,MAAM,MACnByuD,EAAYhwB,EAAM7gC,OAClB43C,EAAWr5C,OAAOlE,KAAKyO,QAAQ8uC,UAC/BwW,EAAQ/hD,GAAK,EAAIwkD,GAAa,EAAIjZ,CAChB,IAAlByiB,IACFjM,EAAQ/hD,GAAK,EAAIwkD,IAAc,EAAIjZ,GAKrC,KAAK,GADDhrC,GAAQwU,EAAI0vC,YAAYjwB,EAAM,IAAIj0B,MAC7B/M,EAAI,EAAOgxD,EAAJhxD,EAAeA,IAAK,CAClC,GAAI8hB,GAAYP,EAAI0vC,YAAYjwB,EAAMhhC,IAAI+M,KAC1CA,GAAQ+U,EAAY/U,EAAQ+U,EAAY/U,EAE1C,GAAIC,GAASxS,KAAKyO,QAAQ8uC,SAAWiZ,EACjC/uD,EAAOsK,EAAIQ,EAAQ,EACnB5K,EAAMqK,EAAIQ,EAAS,CACP,YAAZutD,IACFp4D,GAAO,GAAM41C,EACb51C,GAAO,EACPosD,GAAS,GAEX/zD,KAAK8zD,iBAAmBnsD,IAAIA,EAAIF,KAAKA,EAAK8K,MAAMA,EAAMC,OAAOA,EAAOuhD,MAAMA,GAG5CvtD,SAA1BxG,KAAKyO,QAAQgvC,UAAoD,OAA1Bz9C,KAAKyO,QAAQgvC,UAA+C,SAA1Bz9C,KAAKyO,QAAQgvC,WACxF12B,EAAIiB,UAAYhoB,KAAKyO,QAAQgvC,SAC7B12B,EAAIkwC,SAASxvD,EAAME,EAAK4K,EAAOC,IAIjCuU,EAAIiB,UAAYhoB,KAAKyO,QAAQ6uC,WAAa,QAC1Cv2B,EAAIuB,UAAYymB,GAAS,SACzBhoB,EAAIwB,aAAew3C,GAAY,SAC3B//D,KAAKyO,QAAQivC,gBAAkB,IACjC32B,EAAIO,UAActnB,KAAKyO,QAAQivC,gBAC/B32B,EAAIY,YAAc3nB,KAAKyO,QAAQkvC,gBAC/B52B,EAAImwC,SAAc,QAEpB,KAAK,GAAI1xD,GAAI,EAAOgxD,EAAJhxD,EAAeA,IAC1BxF,KAAKyO,QAAQivC,iBACd32B,EAAIowC,WAAW3wB,EAAMhhC,GAAIuM,EAAGgiD,GAE9BhtC,EAAIyB,SAASge,EAAMhhC,GAAIuM,EAAGgiD,GAC1BA,GAASxW,IAMfh6C,EAAK4P,UAAU2rD,YAAc,SAAS/3C,GACpC,GAAmBvgB,SAAfxG,KAAKyoB,MAAqB,CAC5B1B,EAAIQ,MAAQvnB,KAAKqzC,SAAW,QAAU,IAAMrzC,KAAKyO,QAAQ8uC,SAAW,MAAQv9C,KAAKyO,QAAQ+uC,QAMzF,KAAK,GAJDhX,GAAQxmC,KAAKyoB,MAAM1gB,MAAM,MACzByK,GAAUtO,OAAOlE,KAAKyO,QAAQ8uC,UAAY,GAAK/W,EAAM7gC,OACrD4M,EAAQ,EAEH/M,EAAI,EAAGq7B,EAAO2F,EAAM7gC,OAAYk7B,EAAJr7B,EAAUA,IAC7C+M,EAAQrN,KAAKwH,IAAI6F,EAAOwU,EAAI0vC,YAAYjwB,EAAMhhC,IAAI+M,MAGpD,QAAQA,MAASA,EAAOC,OAAUA,EAAQgkD,UAAWhwB,EAAM7gC,QAG3D,OAAQ4M,MAAS,EAAGC,OAAU,EAAGgkD,UAAW,IAUhDjzD,EAAK4P,UAAU48C,OAAS,WACtB,MAAmBvpD,UAAfxG,KAAKuS,MACDvS,KAAK+R,EAAI/R,KAAKuS,MAAOvS,KAAKm2D,iBAAoBn2D,KAAK+jD,cAAchyC,GACjE/R,KAAK+R,EAAI/R,KAAKuS,MAAOvS,KAAKm2D,gBAAoBn2D,KAAKgkD,kBAAkBjyC,GACrE/R,KAAKgS,EAAIhS,KAAKwS,OAAOxS,KAAKm2D,iBAAoBn2D,KAAK+jD,cAAc/xC,GACjEhS,KAAKgS,EAAIhS,KAAKwS,OAAOxS,KAAKm2D,gBAAoBn2D,KAAKgkD,kBAAkBhyC,GAGpE,GAQXzO,EAAK4P,UAAU8sD,OAAS,WACtB,MAAQjgE,MAAK+R,GAAK/R,KAAK+jD,cAAchyC,GAC7B/R,KAAK+R,EAAI/R,KAAKgkD,kBAAkBjyC,GAChC/R,KAAKgS,GAAKhS,KAAK+jD,cAAc/xC,GAC7BhS,KAAKgS,EAAIhS,KAAKgkD,kBAAkBhyC,GAW1CzO,EAAK4P,UAAU28C,eAAiB,SAAS7yC,EAAM8mC,EAAcC,GAC3DhkD,KAAKm2D,gBAAkB,EAAIl5C,EAC3Bjd,KAAKm8D,aAAel/C,EACpBjd,KAAK+jD,cAAgBA,EACrB/jD,KAAKgkD,kBAAoBA,GAS3BzgD,EAAK4P,UAAU6vB,SAAW,SAAS/lB,GACjCjd,KAAKm2D,gBAAkB,EAAIl5C,EAC3Bjd,KAAKm8D,aAAel/C,GAQtB1Z,EAAK4P,UAAU+sD,cAAgB,WAC7BlgE,KAAKy7D,GAAK,EACVz7D,KAAK07D,GAAK,GASZn4D,EAAK4P,UAAUgtD,eAAiB,SAASC,GACvC,GAAIC,GAAergE,KAAKy7D,GAAKz7D,KAAKy7D,GAAK2E,CAEvCpgE,MAAKy7D,GAAKv2D,KAAKyqB,KAAK0wC,EAAargE,KAAKyO,QAAQwuC,MAC9CojB,EAAergE,KAAK07D,GAAK17D,KAAK07D,GAAK0E,EAEnCpgE,KAAK07D,GAAKx2D,KAAKyqB,KAAK0wC,EAAargE,KAAKyO,QAAQwuC,OAGhDp9C,EAAOD,QAAU2D,GAKb,SAAS1D,GAWb,QAAS2D,GAAM+V,EAAWxH,EAAGC,EAAGuX,EAAMtc,GAElCjN,KAAKuZ,UADHA,EACeA,EAGAhI,SAASojB,KAIdnuB,SAAVyG,IACe,gBAAN8E,IACT9E,EAAQ8E,EACRA,EAAIvL,QACqB,gBAAT+iB,IAChBtc,EAAQsc,EACRA,EAAO/iB,QAGPyG,GACEqwC,UAAW,QACXC,SAAU,GACVC,SAAU,UACVryC,OACEiB,OAAQ,OACRD,WAAY,aAMpBnM,KAAK+R,EAAI,EACT/R,KAAKgS,EAAI,EACThS,KAAKgkB,QAAU,EAELxd,SAANuL,GAAyBvL,SAANwL,GACrBhS,KAAK6tD,YAAY97C,EAAGC,GAETxL,SAAT+iB,GACFvpB,KAAK8tD,QAAQvkC,GAIfvpB,KAAKsf,MAAQ/N,SAASM,cAAc,MACpC,IAAIyuD,GAAYtgE,KAAKsf,MAAMrS,KAC3BqzD,GAAU18C,SAAW,WACrB08C,EAAU9oC,WAAa,SACvB8oC,EAAUl0D,OAAS,aAAea,EAAM9B,MAAMiB,OAC9Ck0D,EAAUn1D,MAAQ8B,EAAMqwC,UACxBgjB,EAAU/iB,SAAWtwC,EAAMswC,SAAW,KACtC+iB,EAAUC,WAAatzD,EAAMuwC,SAC7B8iB,EAAUt8C,QAAUhkB,KAAKgkB,QAAU,KACnCs8C,EAAU3gD,gBAAkB1S,EAAM9B,MAAMgB,WACxCm0D,EAAUtwC,aAAe,MACzBswC,EAAUxuC,gBAAkB,MAC5BwuC,EAAUE,mBAAqB,MAC/BF,EAAUrwC,UAAY,wCACtBqwC,EAAUG,WAAa,SACvBzgE,KAAKuZ,UAAU9H,YAAYzR,KAAKsf,OAOlC9b,EAAM2P,UAAU06C,YAAc,SAAS97C,EAAGC,GACxChS,KAAK+R,EAAInH,SAASmH,GAClB/R,KAAKgS,EAAIpH,SAASoH,IAOpBxO,EAAM2P,UAAU26C,QAAU,SAASl+B,GAC7BA,YAAmB+c,UACrB3sC,KAAKsf,MAAM2E,UAAY,GACvBjkB,KAAKsf,MAAM7N,YAAYme,IAGvB5vB,KAAKsf,MAAM2E,UAAY2L,GAQ3BpsB,EAAM2P,UAAUw0B,KAAO,SAAUA,GAK/B,GAJanhC,SAATmhC,IACFA,GAAO,GAGLA,EAAM,CACR,GAAIn1B,GAASxS,KAAKsf,MAAMuF,aACpBtS,EAASvS,KAAKsf,MAAME,YACpBgV,EAAYx0B,KAAKsf,MAAMzV,WAAWgb,aAClC62B,EAAW17C,KAAKsf,MAAMzV,WAAW2V,YAEjC7X,EAAO3H,KAAKgS,EAAIQ,CAChB7K,GAAM6K,EAASxS,KAAKgkB,QAAUwQ,IAChC7sB,EAAM6sB,EAAYhiB,EAASxS,KAAKgkB,SAE9Brc,EAAM3H,KAAKgkB,UACbrc,EAAM3H,KAAKgkB,QAGb,IAAIvc,GAAOzH,KAAK+R,CACZtK,GAAO8K,EAAQvS,KAAKgkB,QAAU03B,IAChCj0C,EAAOi0C,EAAWnpC,EAAQvS,KAAKgkB,SAE7Bvc,EAAOzH,KAAKgkB,UACdvc,EAAOzH,KAAKgkB,SAGdhkB,KAAKsf,MAAMrS,MAAMxF,KAAOA,EAAO,KAC/BzH,KAAKsf,MAAMrS,MAAMtF,IAAMA,EAAM,KAC7B3H,KAAKsf,MAAMrS,MAAMuqB,WAAa,cAG9Bx3B,MAAK0nC,QAOTlkC,EAAM2P,UAAUu0B,KAAO,WACrB1nC,KAAKsf,MAAMrS,MAAMuqB,WAAa,UAGhC33B,EAAOD,QAAU4D,GAKb,SAAS3D,EAAQD,GAarB,QAAS8gE,GAAUhuD,GAEjB,MADAod,GAAMpd,EACCiuD,IAoCT,QAAS1+B,KACP95B,EAAQ,EACR1H,EAAIqvB,EAAI1K,OAAO,GAQjB,QAASiD,KACPlgB,IACA1H,EAAIqvB,EAAI1K,OAAOjd,GAOjB,QAASy4D,KACP,MAAO9wC,GAAI1K,OAAOjd,EAAQ,GAS5B,QAAS04D,GAAepgE,GACtB,MAAOqgE,GAAkB9yD,KAAKvN,GAShC,QAASsgE,GAAOx7D,EAAGa,GAKjB,GAJKb,IACHA,MAGEa,EACF,IAAK,GAAI6P,KAAQ7P,GACXA,EAAEN,eAAemQ,KACnB1Q,EAAE0Q,GAAQ7P,EAAE6P,GAIlB,OAAO1Q,GAeT,QAASqS,GAASmL,EAAKooB,EAAM9jC,GAG3B,IAFA,GAAI+F,GAAO+9B,EAAKpjC,MAAM,KAClBi5D,EAAIj+C,EACD3V,EAAKzH,QAAQ,CAClB,GAAI+C,GAAM0E,EAAKkE,OACXlE,GAAKzH,QAEFq7D,EAAEt4D,KACLs4D,EAAEt4D,OAEJs4D,EAAIA,EAAEt4D,IAINs4D,EAAEt4D,GAAOrB,GAWf,QAAS45D,GAAQhwC,EAAO60B,GAOtB,IANA,GAAItgD,GAAGC,EACHq0B,EAAU,KAGVonC,GAAUjwC,GACVvxB,EAAOuxB,EACJvxB,EAAKolC,QACVo8B,EAAOl5D,KAAKtI,EAAKolC,QACjBplC,EAAOA,EAAKolC,MAId,IAAIplC,EAAKs9C,MACP,IAAKx3C,EAAI,EAAGC,EAAM/F,EAAKs9C,MAAMr3C,OAAYF,EAAJD,EAASA,IAC5C,GAAIsgD,EAAKzlD,KAAOX,EAAKs9C,MAAMx3C,GAAGnF,GAAI,CAChCy5B,EAAUp6B,EAAKs9C,MAAMx3C,EACrB,OAiBN,IAZKs0B,IAEHA,GACEz5B,GAAIylD,EAAKzlD,IAEP4wB,EAAM60B,OAERhsB,EAAQqnC,KAAOJ,EAAMjnC,EAAQqnC,KAAMlwC,EAAM60B,QAKxCtgD,EAAI07D,EAAOv7D,OAAS,EAAGH,GAAK,EAAGA,IAAK,CACvC,GAAIkF,GAAIw2D,EAAO17D,EAEVkF,GAAEsyC,QACLtyC,EAAEsyC,UAE4B,IAA5BtyC,EAAEsyC,MAAMr2C,QAAQmzB,IAClBpvB,EAAEsyC,MAAMh1C,KAAK8xB,GAKbgsB,EAAKqb,OACPrnC,EAAQqnC,KAAOJ,EAAMjnC,EAAQqnC,KAAMrb,EAAKqb,OAS5C,QAASC,GAAQnwC,EAAOy8B,GAKtB,GAJKz8B,EAAM6sB,QACT7sB,EAAM6sB,UAER7sB,EAAM6sB,MAAM91C,KAAK0lD,GACbz8B,EAAMy8B,KAAM,CACd,GAAIyT,GAAOJ,KAAU9vC,EAAMy8B,KAC3BA,GAAKyT,KAAOJ,EAAMI,EAAMzT,EAAKyT,OAajC,QAASE,GAAWpwC,EAAO7H,EAAMC,EAAIviB,EAAMq6D,GACzC,GAAIzT,IACFtkC,KAAMA,EACNC,GAAIA,EACJviB,KAAMA,EAQR,OALImqB,GAAMy8B,OACRA,EAAKyT,KAAOJ,KAAU9vC,EAAMy8B,OAE9BA,EAAKyT,KAAOJ,EAAMrT,EAAKyT,SAAYA,GAE5BzT,EAOT,QAAS4T,KAKP,IAJAC,EAAYC,EAAUC,KACtBC,EAAQ,GAGI,KAALjhE,GAAiB,KAALA,GAAkB,MAALA,GAAkB,MAALA,GAC3C4nB,GAGF,GAAG,CACD,GAAIs5C,IAAY,CAGhB,IAAS,KAALlhE,EAAU,CAGZ,IADA,GAAI+E,GAAI2C,EAAQ,EACQ,KAAjB2nB,EAAI1K,OAAO5f,IAA8B,KAAjBsqB,EAAI1K,OAAO5f,IACxCA,GAEF,IAAqB,MAAjBsqB,EAAI1K,OAAO5f,IAA+B,IAAjBsqB,EAAI1K,OAAO5f,GAAU,CAEhD,KAAY,IAAL/E,GAAgB,MAALA,GAChB4nB,GAEFs5C,IAAY,GAGhB,GAAS,KAALlhE,GAA6B,KAAjBmgE,IAAsB,CAEpC,KAAY,IAALngE,GAAgB,MAALA,GAChB4nB,GAEFs5C,IAAY,EAEd,GAAS,KAALlhE,GAA6B,KAAjBmgE,IAAsB,CAEpC,KAAY,IAALngE,GAAS,CACd,GAAS,KAALA,GAA6B,KAAjBmgE,IAAsB,CAEpCv4C,IACAA,GACA,OAGAA,IAGJs5C,GAAY,EAId,KAAY,KAALlhE,GAAiB,KAALA,GAAkB,MAALA,GAAkB,MAALA,GAC3C4nB,UAGGs5C,EAGP,IAAS,IAALlhE,EAGF,YADA8gE,EAAYC,EAAUI,UAKxB,IAAIC,GAAKphE,EAAImgE,GACb,IAAIkB,EAAWD,GAKb,MAJAN,GAAYC,EAAUI,UACtBF,EAAQG,EACRx5C,QACAA,IAKF,IAAIy5C,EAAWrhE,GAIb,MAHA8gE,GAAYC,EAAUI,UACtBF,EAAQjhE,MACR4nB,IAMF,IAAIw4C,EAAepgE,IAAW,KAALA,EAAU,CAIjC,IAHAihE,GAASjhE,EACT4nB,IAEOw4C,EAAepgE,IACpBihE,GAASjhE,EACT4nB,GAYF,OAVa,SAATq5C,EACFA,GAAQ,EAEQ,QAATA,EACPA,GAAQ,EAEAh9D,MAAMR,OAAOw9D,MACrBA,EAAQx9D,OAAOw9D,SAEjBH,EAAYC,EAAUO,YAKxB,GAAS,KAALthE,EAAU,CAEZ,IADA4nB,IACY,IAAL5nB,IAAiB,KAALA,GAAkB,KAALA,GAA6B,KAAjBmgE,MAC1Cc,GAASjhE,EACA,KAALA,GACF4nB,IAEFA,GAEF,IAAS,KAAL5nB,EACF,KAAMuhE,GAAe,2BAIvB,OAFA35C,UACAk5C,EAAYC,EAAUO,YAMxB,IADAR,EAAYC,EAAUS,QACV,IAALxhE,GACLihE,GAASjhE,EACT4nB,GAEF,MAAM,IAAI7O,aAAY,yBAA2B0oD,EAAKR,EAAO,IAAM,KAOrE,QAASf,KACP,GAAI1vC,KAwBJ,IAtBAgR,IACAq/B,IAGa,UAATI,IACFzwC,EAAMkxC,QAAS,EACfb,MAIW,SAATI,GAA6B,WAATA,KACtBzwC,EAAMnqB,KAAO46D,EACbJ,KAIEC,GAAaC,EAAUO,aACzB9wC,EAAM5wB,GAAKqhE,EACXJ,KAIW,KAATI,EACF,KAAMM,GAAe,2BAQvB,IANAV,IAGAc,EAAgBnxC,GAGH,KAATywC,EACF,KAAMM,GAAe,2BAKvB,IAHAV,IAGc,KAAVI,EACF,KAAMM,GAAe,uBASvB,OAPAV,WAGOrwC,GAAM60B,WACN70B,GAAMy8B,WACNz8B,GAAMA,MAENA,EAOT,QAASmxC,GAAiBnxC,GACxB,KAAiB,KAAVywC,GAAyB,KAATA,GACrBW,EAAepxC,GACF,KAATywC,GACFJ,IAWN,QAASe,GAAepxC,GAEtB,GAAIqxC,GAAWC,EAActxC,EAC7B,IAAIqxC,EAIF,WAFAE,GAAUvxC,EAAOqxC,EAMnB,IAAInB,GAAOsB,EAAwBxxC,EACnC,KAAIkwC,EAAJ,CAKA,GAAII,GAAaC,EAAUO,WACzB,KAAMC,GAAe,sBAEvB,IAAI3hE,GAAKqhE,CAGT,IAFAJ,IAEa,KAATI,EAAc,CAGhB,GADAJ,IACIC,GAAaC,EAAUO,WACzB,KAAMC,GAAe,sBAEvB/wC,GAAM5wB,GAAMqhE,EACZJ,QAIAoB,GAAmBzxC,EAAO5wB,IAS9B,QAASkiE,GAAetxC,GACtB,GAAIqxC,GAAW,IAgBf,IAba,YAATZ,IACFY,KACAA,EAASx7D,KAAO,WAChBw6D,IAGIC,GAAaC,EAAUO,aACzBO,EAASjiE,GAAKqhE,EACdJ,MAKS,KAATI,EAAc,CAehB,GAdAJ,IAEKgB,IACHA,MAEFA,EAASx9B,OAAS7T,EAClBqxC,EAASxc,KAAO70B,EAAM60B,KACtBwc,EAAS5U,KAAOz8B,EAAMy8B,KACtB4U,EAASrxC,MAAQA,EAAMA,MAGvBmxC,EAAgBE,GAGH,KAATZ,EACF,KAAMM,GAAe,2BAEvBV,WAGOgB,GAASxc,WACTwc,GAAS5U,WACT4U,GAASrxC,YACTqxC,GAASx9B,OAGX7T,EAAM0xC,YACT1xC,EAAM0xC,cAER1xC,EAAM0xC,UAAU36D,KAAKs6D,GAGvB,MAAOA,GAYT,QAASG,GAAyBxxC,GAEhC,MAAa,QAATywC,GACFJ,IAGArwC,EAAM60B,KAAO8c,IACN,QAES,QAATlB,GACPJ,IAGArwC,EAAMy8B,KAAOkV,IACN,QAES,SAATlB,GACPJ,IAGArwC,EAAMA,MAAQ2xC,IACP,SAGF,KAQT,QAASF,GAAmBzxC,EAAO5wB,GAEjC,GAAIylD,IACFzlD,GAAIA,GAEF8gE,EAAOyB,GACPzB,KACFrb,EAAKqb,KAAOA,GAEdF,EAAQhwC,EAAO60B,GAGf0c,EAAUvxC,EAAO5wB,GAQnB,QAASmiE,GAAUvxC,EAAO7H,GACxB,KAAgB,MAATs4C,GAA0B,MAATA,GAAe,CACrC,GAAIr4C,GACAviB,EAAO46D,CACXJ,IAEA,IAAIgB,GAAWC,EAActxC,EAC7B,IAAIqxC,EACFj5C,EAAKi5C,MAEF,CACH,GAAIf,GAAaC,EAAUO,WACzB,KAAMC,GAAe,kCAEvB34C,GAAKq4C,EACLT,EAAQhwC,GACN5wB,GAAIgpB,IAENi4C,IAIF,GAAIH,GAAOyB,IAGPlV,EAAO2T,EAAWpwC,EAAO7H,EAAMC,EAAIviB,EAAMq6D,EAC7CC,GAAQnwC,EAAOy8B,GAEftkC,EAAOC,GASX,QAASu5C,KAGP,IAFA,GAAIzB,GAAO,KAEK,KAATO,GAAc,CAGnB,IAFAJ,IACAH,KACiB,KAAVO,GAAyB,KAATA,GAAc,CACnC,GAAIH,GAAaC,EAAUO,WACzB,KAAMC,GAAe,0BAEvB,IAAI/rD,GAAOyrD,CAGX,IADAJ,IACa,KAATI,EACF,KAAMM,GAAe,wBAIvB,IAFAV,IAEIC,GAAaC,EAAUO,WACzB,KAAMC,GAAe,2BAEvB,IAAI36D,GAAQq6D,CACZ9pD,GAASupD,EAAMlrD,EAAM5O,GAErBi6D,IACY,KAARI,GACFJ,IAIJ,GAAa,KAATI,EACF,KAAMM,GAAe,qBAEvBV,KAGF,MAAOH,GAQT,QAASa,GAAea,GACtB,MAAO,IAAIrpD,aAAYqpD,EAAU,UAAYX,EAAKR,EAAO,IAAM,WAAav5D,EAAQ,KAStF,QAAS+5D,GAAM34C,EAAMu5C,GACnB,MAAQv5C,GAAK5jB,QAAUm9D,EAAav5C,EAAQA,EAAKje,OAAO,EAAG,IAAM,MASnE,QAASy3D,GAASC,EAAQC,EAAQ/pD,GAC5BjT,MAAMC,QAAQ88D,GAChBA,EAAO36D,QAAQ,SAAU66D,GACnBj9D,MAAMC,QAAQ+8D,GAChBA,EAAO56D,QAAQ,SAAU86D,GACvBjqD,EAAGgqD,EAAOC,KAIZjqD,EAAGgqD,EAAOD,KAKVh9D,MAAMC,QAAQ+8D,GAChBA,EAAO56D,QAAQ,SAAU86D,GACvBjqD,EAAG8pD,EAAQG,KAIbjqD,EAAG8pD,EAAQC,GAWjB,QAAS5b,GAAY30C,GAEnB,GAAI00C,GAAUsZ,EAAShuD,GACnB0wD,GACFpmB,SACAc,SACArvC,WAmBF,IAfI24C,EAAQpK,OACVoK,EAAQpK,MAAM30C,QAAQ,SAAUg7D,GAC9B,GAAIC,IACFjjE,GAAIgjE,EAAQhjE,GACZooB,MAAOrkB,OAAOi/D,EAAQ56C,OAAS46C,EAAQhjE,IAEzC0gE,GAAMuC,EAAWD,EAAQlC,MACrBmC,EAAUjmB,QACZimB,EAAUlmB,MAAQ,SAEpBgmB,EAAUpmB,MAAMh1C,KAAKs7D,KAKrBlc,EAAQtJ,MAAO,CAMjB,GAAIylB,GAAc,SAAUC,GAC1B,GAAIC,IACFr6C,KAAMo6C,EAAQp6C,KACdC,GAAIm6C,EAAQn6C,GAId,OAFA03C,GAAM0C,EAAWD,EAAQrC,MACzBsC,EAAUx2D,MAAyB,MAAhBu2D,EAAQ18D,KAAgB,QAAU,OAC9C28D,EAGTrc,GAAQtJ,MAAMz1C,QAAQ,SAAUm7D,GAC9B,GAAIp6C,GAAMC,CAERD,GADEo6C,EAAQp6C,eAAgB7iB,QACnBi9D,EAAQp6C,KAAK4zB,OAIlB38C,GAAImjE,EAAQp6C,MAKdC,EADEm6C,EAAQn6C,aAAc9iB,QACnBi9D,EAAQn6C,GAAG2zB,OAId38C,GAAImjE,EAAQn6C,IAIZm6C,EAAQp6C,eAAgB7iB,SAAUi9D,EAAQp6C,KAAK00B,OACjD0lB,EAAQp6C,KAAK00B,MAAMz1C,QAAQ,SAAUq7D,GACnC,GAAID,GAAYF,EAAYG,EAC5BN,GAAUtlB,MAAM91C,KAAKy7D,KAIzBV,EAAS35C,EAAMC,EAAI,SAAUD,EAAMC,GACjC,GAAIq6C,GAAUrC,EAAW+B,EAAWh6C,EAAK/oB,GAAIgpB,EAAGhpB,GAAImjE,EAAQ18D,KAAM08D,EAAQrC,MACtEsC,EAAYF,EAAYG,EAC5BN,GAAUtlB,MAAM91C,KAAKy7D,KAGnBD,EAAQn6C,aAAc9iB,SAAUi9D,EAAQn6C,GAAGy0B,OAC7C0lB,EAAQn6C,GAAGy0B,MAAMz1C,QAAQ,SAAUq7D,GACjC,GAAID,GAAYF,EAAYG,EAC5BN,GAAUtlB,MAAM91C,KAAKy7D,OAW7B,MAJIrc,GAAQ+Z,OACViC,EAAU30D,QAAU24C,EAAQ+Z,MAGvBiC,EAnyBT,GAAI5B,IACFC,KAAO,EACPG,UAAY,EACZG,WAAY,EACZE,QAAU,GAIRH,GACF6B,KAAK,EACLC,KAAK,EACLC,KAAK,EACLC,KAAK,EACLC,KAAK,EACLC,KAAK,EACLC,KAAK,EAELC,MAAM,EACNC,MAAM,GAGJr0C,EAAM,GACN3nB,EAAQ,EACR1H,EAAI,GACJihE,EAAQ,GACRH,EAAYC,EAAUC,KAmCtBX,EAAoB,iBA2uBxBlhE,GAAQ8gE,SAAWA,EACnB9gE,EAAQynD,WAAaA,GAKjB,SAASxnD,EAAQD,GAGrB,QAAS4nD,GAAW4c,EAAW31D,GAC7B,GAAIqvC,MACAd,IACJh9C,MAAKyO,SACHqvC,OACEQ,cAAc,GAEhBtB,OACEqnB,eAAe,EACfn5D,YAAY,IAIA1E,SAAZiI,IACFzO,KAAKyO,QAAQuuC,MAAqB,cAAIvuC,EAAQ41D,eAAgB,EAC9DrkE,KAAKyO,QAAQuuC,MAAkB,WAAOvuC,EAAQvD,YAAgB,EAC9DlL,KAAKyO,QAAQqvC,MAAoB,aAAKrvC,EAAQ6vC,cAAgB,EAKhE,KAAK,GAFDgmB,GAASF,EAAUtmB,MACnBymB,EAASH,EAAUpnB,MACdx3C,EAAI,EAAGA,EAAI8+D,EAAO3+D,OAAQH,IAAK,CACtC,GAAIkoD,MACA8W,EAAQF,EAAO9+D,EACnBkoD,GAAS,GAAI8W,EAAMnkE,GACnBqtD,EAAW,KAAI8W,EAAMC,OACrB/W,EAAS,GAAI8W,EAAM96D,OACnBgkD,EAAiB,WAAI8W,EAAMvpB,WAG3ByS,EAAY,MAAI8W,EAAMr5D,MACtBuiD,EAAmB,aAAsBlnD,SAAlBknD,EAAY,OAAkB,EAAQ1tD,KAAKyO,QAAQ6vC,aAC1ER,EAAM91C,KAAK0lD,GAGb,IAAK,GAAIloD,GAAI,EAAGA,EAAI++D,EAAO5+D,OAAQH,IAAK,CACtC,GAAIsgD,MACA4e,EAAQH,EAAO/+D,EACnBsgD,GAAS,GAAI4e,EAAMrkE,GACnBylD,EAAiB,WAAI4e,EAAMzpB,WAC3B6K,EAAQ,EAAI4e,EAAM3yD,EAClB+zC,EAAQ,EAAI4e,EAAM1yD,EAClB8zC,EAAY,MAAI4e,EAAMj8C,MAEpBq9B,EAAY,MADuB,GAAjC9lD,KAAKyO,QAAQuuC,MAAM9xC,WACLw5D,EAAMv5D,MAGU3E,SAAhBk+D,EAAMv5D,OAAuBgB,WAAWu4D,EAAMv5D,MAAOiB,OAAOs4D,EAAMv5D,OAAS3E,OAE7Fs/C,EAAa,OAAI4e,EAAMryD,KACvByzC,EAAqB,eAAI9lD,KAAKyO,QAAQuuC,MAAMqnB,cAC5Cve,EAAqB,eAAI9lD,KAAKyO,QAAQuuC,MAAMqnB,cAC5CrnB,EAAMh1C,KAAK89C,GAGb,OAAQ9I,MAAMA,EAAOc,MAAMA,GAG7Bl+C,EAAQ4nD,WAAaA,GAIjB,SAAS3nD,EAAQD,EAASM,GAI9BL,EAAOD,QAA6B,mBAAX2J,SAA2BA,OAAe,QAAKrJ,EAAoB,KAKxF,SAASL,EAAQD,EAASM,GAI9B,GAAsB,mBAAXqJ,QAAwB,CACjC,GAAIo7D,GAAczkE,EAAoB,IAClC6D,EAASwF,OAAe,QAAKrJ,EAAoB,GACrDL,GAAOD,QAAU+kE,EAAY5gE,OAG7BlE,GAAOD,QAAU,WACf,KAAMgE,OAAM,+DAOZ,SAAS/D,EAAQD,EAASM,GAmB9B,QAASg2B,MAjBT,GAAInZ,GAAU7c,EAAoB,IAC9B6D,EAAS7D,EAAoB,IAC7BS,EAAOT,EAAoB,GAK3BolD,GAJUplD,EAAoB,GACnBA,EAAoB,GACvBA,EAAoB,IAClBA,EAAoB,IAClBA,EAAoB,KAChCyB,EAAWzB,EAAoB,GAYnC6c,GAAQmZ,EAAK/iB,WASb+iB,EAAK/iB,UAAUuhB,QAAU,SAAUnb,GAmHjC,QAASqrD,GAAat7D,GAChB6K,EAAGs1C,YACLt1C,EAAGyZ,KAAK,aAActkB,GApH1BtJ,KAAK+vB,OAEL/vB,KAAK+vB,IAAIrwB,KAAuB6R,SAASM,cAAc,OACvD7R,KAAK+vB,IAAI5jB,WAAuBoF,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIgV,mBAAuBxzB,SAASM,cAAc,OACvD7R,KAAK+vB,IAAImY,qBAAuB32B,SAASM,cAAc,OACvD7R,KAAK+vB,IAAI8H,gBAAuBtmB,SAASM,cAAc,OACvD7R,KAAK+vB,IAAI80C,cAAuBtzD,SAASM,cAAc,OACvD7R,KAAK+vB,IAAI+0C,eAAuBvzD,SAASM,cAAc,OACvD7R,KAAK+vB,IAAI7D,OAAuB3a,SAASM,cAAc,OACvD7R,KAAK+vB,IAAItoB,KAAuB8J,SAASM,cAAc,OACvD7R,KAAK+vB,IAAI1I,MAAuB9V,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIpoB,IAAuB4J,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIzM,OAAuB/R,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIg1C,UAAuBxzD,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIi1C,aAAuBzzD,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIk1C,cAAuB1zD,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIm1C,iBAAuB3zD,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIo1C,eAAuB5zD,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIq1C,kBAAuB7zD,SAASM,cAAc,OAEvD7R,KAAK+vB,IAAIrwB,KAAKmI,UAA4B,oBAC1C7H,KAAK+vB,IAAI5jB,WAAWtE,UAAsB,sBAC1C7H,KAAK+vB,IAAIgV,mBAAmBl9B,UAAc,+BAC1C7H,KAAK+vB,IAAImY,qBAAqBrgC,UAAY,iCAC1C7H,KAAK+vB,IAAI8H,gBAAgBhwB,UAAiB,kBAC1C7H,KAAK+vB,IAAI80C,cAAch9D,UAAmB,gBAC1C7H,KAAK+vB,IAAI+0C,eAAej9D,UAAkB,iBAC1C7H,KAAK+vB,IAAIpoB,IAAIE,UAA6B,eAC1C7H,KAAK+vB,IAAIzM,OAAOzb,UAA0B,kBAC1C7H,KAAK+vB,IAAItoB,KAAKI,UAA4B,UAC1C7H,KAAK+vB,IAAI7D,OAAOrkB,UAA0B,UAC1C7H,KAAK+vB,IAAI1I,MAAMxf,UAA2B,UAC1C7H,KAAK+vB,IAAIg1C,UAAUl9D,UAAuB,aAC1C7H,KAAK+vB,IAAIi1C,aAAan9D,UAAoB,gBAC1C7H,KAAK+vB,IAAIk1C,cAAcp9D,UAAmB,aAC1C7H,KAAK+vB,IAAIm1C,iBAAiBr9D,UAAgB,gBAC1C7H,KAAK+vB,IAAIo1C,eAAet9D,UAAkB,aAC1C7H,KAAK+vB,IAAIq1C,kBAAkBv9D,UAAe,gBAE1C7H,KAAK+vB,IAAIrwB,KAAK+R,YAAYzR,KAAK+vB,IAAI5jB,YACnCnM,KAAK+vB,IAAIrwB,KAAK+R,YAAYzR,KAAK+vB,IAAIgV,oBACnC/kC,KAAK+vB,IAAIrwB,KAAK+R,YAAYzR,KAAK+vB,IAAImY,sBACnCloC,KAAK+vB,IAAIrwB,KAAK+R,YAAYzR,KAAK+vB,IAAI8H,iBACnC73B,KAAK+vB,IAAIrwB,KAAK+R,YAAYzR,KAAK+vB,IAAI80C,eACnC7kE,KAAK+vB,IAAIrwB,KAAK+R,YAAYzR,KAAK+vB,IAAI+0C,gBACnC9kE,KAAK+vB,IAAIrwB,KAAK+R,YAAYzR,KAAK+vB,IAAIpoB,KACnC3H,KAAK+vB,IAAIrwB,KAAK+R,YAAYzR,KAAK+vB,IAAIzM,QAEnCtjB,KAAK+vB,IAAI8H,gBAAgBpmB,YAAYzR,KAAK+vB,IAAI7D,QAC9ClsB,KAAK+vB,IAAI80C,cAAcpzD,YAAYzR,KAAK+vB,IAAItoB,MAC5CzH,KAAK+vB,IAAI+0C,eAAerzD,YAAYzR,KAAK+vB,IAAI1I,OAE7CrnB,KAAK+vB,IAAI8H,gBAAgBpmB,YAAYzR,KAAK+vB,IAAIg1C,WAC9C/kE,KAAK+vB,IAAI8H,gBAAgBpmB,YAAYzR,KAAK+vB,IAAIi1C,cAC9ChlE,KAAK+vB,IAAI80C,cAAcpzD,YAAYzR,KAAK+vB,IAAIk1C,eAC5CjlE,KAAK+vB,IAAI80C,cAAcpzD,YAAYzR,KAAK+vB,IAAIm1C,kBAC5CllE,KAAK+vB,IAAI+0C,eAAerzD,YAAYzR,KAAK+vB,IAAIo1C,gBAC7CnlE,KAAK+vB,IAAI+0C,eAAerzD,YAAYzR,KAAK+vB,IAAIq1C,mBAE7CplE,KAAKuT,GAAG,cAAevT,KAAKyhB,OAAOqT,KAAK90B,MAExC,IAAImU,GAAKnU,IACTA,MAAKuT,GAAG,SAAU,SAAUo7C,GACtBA,GAAkC,GAApBA,EAAWv7C,MAEtBe,EAAGkxD,eACNlxD,EAAGkxD,aAAe/rD,WAAW,WAC3BnF,EAAGkxD,aAAe,KAClBlxD,EAAGsN,UACF,IAKLtN,EAAGsN,WAMPzhB,KAAK8D,OAAS,GAAIC,GAAO/D,KAAK+vB,IAAIrwB,MAAO4lE,YAAa,UACtDtlE,KAAK8D,OAAOoR,IAAI,SAASi0C,KAAK/lB,QAAQ,IACtCpjC,KAAKulE,YAEL,IAAIC,IACF,MAAO,YAAa,QACpB,QACA,MAAO,WAAY,UAAW,SAsDhC,IA/CAA,EAAOn9D,QAAQ,SAAUvB,GACvB,GAAIgC,GAAW,SAAUQ,GACnB6K,EAAGs1C,YACLt1C,EAAGyZ,KAAK9mB,EAAMwC,GAGlB6K,GAAGrQ,OAAOyP,GAAGzM,EAAMgC,GACnBqL,EAAGoxD,UAAUz+D,GAAQgC,IAIvB9I,KAAK8D,OAAOyP,GAAG,eAAgB,SAAUjK,GACnCA,EAAMqnC,SACJx8B,EAAGs1C,YACLt1C,EAAGyZ,KAAK,QAAStkB,IAGrBwrB,KAAK90B,OAOPA,KAAK+vB,IAAIrwB,KAAKiJ,iBAAiB,aAAci8D,GAC7C5kE,KAAK+vB,IAAIrwB,KAAKiJ,iBAAiB,iBAAkBi8D,GAGjD5kE,KAAKgG,OACHtG,QACAyM,cACA0rB,mBACAgtC,iBACAC,kBACA54C,UACAzkB,QACA4f,SACA1f,OACA2b,UACAlX,UACAi7B,UAAW,EACXo+B,aAAc,GAGhBzlE,KAAK0lE,YAAc,GAGdnsD,EAAW,KAAM,IAAI3V,OAAM,wBAChC2V,GAAU9H,YAAYzR,KAAK+vB,IAAIrwB,OA4BjCw2B,EAAK/iB,UAAUD,WAAa,SAAUzE,GACpC,GAAIA,EAAS,CAEX,GAAIP,IAAU,QAAS,SAAU,YAAa,YAAa,aAAc,QAAS,MAAO,cAAe,aAAc,iBAAkB,cACxIvN,GAAKoF,gBAAgBmI,EAAQlO,KAAKyO,QAASA,GAEvC,eAAiBzO,MAAKyO,SACxB9M,EAAS81B,qBAAqBz3B,KAAK20B,KAAM30B,KAAKyO,QAAQsmB,aAGpD,cAAgBtmB,KACdA,EAAQ45C,WACLroD,KAAKsoD,YACRtoD,KAAKsoD,UAAY,GAAIhD,GAAUtlD,KAAK+vB,IAAIrwB,OAItCM,KAAKsoD,YACPtoD,KAAKsoD,UAAUh1C,gBACRtT,MAAKsoD,YAMlBtoD,KAAK2lE,kBASP,GALA3lE,KAAKgC,WAAWqG,QAAQ,SAAUu9D,GAChCA,EAAU1yD,WAAWzE,KAInBA,GAAWA,EAAQgH,MACrB,KAAM,IAAI7R,OAAM,wEAIlB5D,MAAKyhB,UAOPyU,EAAK/iB,UAAUs2C,SAAW,WACxB,OAAQzpD,KAAKsoD,WAAatoD,KAAKsoD,UAAUgL,QAM3Cp9B,EAAK/iB,UAAUG,QAAU,WAEvBtT,KAAKyW,QAGLzW,KAAK0T,MAGL1T,KAAK6lE,kBAGD7lE,KAAK+vB,IAAIrwB,KAAKmK,YAChB7J,KAAK+vB,IAAIrwB,KAAKmK,WAAWsH,YAAYnR,KAAK+vB,IAAIrwB,MAEhDM,KAAK+vB,IAAM,KAGP/vB,KAAKsoD,YACPtoD,KAAKsoD,UAAUh1C,gBACRtT,MAAKsoD,UAId,KAAK,GAAIh/C,KAAStJ,MAAKulE,UACjBvlE,KAAKulE,UAAUz/D,eAAewD,UACzBtJ,MAAKulE,UAAUj8D,EAG1BtJ,MAAKulE,UAAY,KACjBvlE,KAAK8D,OAAS,KAGd9D,KAAKgC,WAAWqG,QAAQ,SAAUu9D,GAChCA,EAAUtyD,YAGZtT,KAAK20B,KAAO,MAQduB,EAAK/iB,UAAUsyB,cAAgB,SAAUtL,GACvC,IAAKn6B,KAAK41B,WACR,KAAM,IAAIhyB,OAAM,yDAGlB5D,MAAK41B,WAAW6P,cAActL,IAOhCjE,EAAK/iB,UAAUuyB,cAAgB,WAC7B,IAAK1lC,KAAK41B,WACR,KAAM,IAAIhyB,OAAM,yDAGlB,OAAO5D,MAAK41B,WAAW8P,iBAQzBxP,EAAK/iB,UAAUi+B,gBAAkB,WAC/B,MAAOpxC,MAAK61B,SAAW71B,KAAK61B,QAAQub,uBAetClb,EAAK/iB,UAAUsD,MAAQ,SAASqvD,KAEzBA,GAAQA,EAAK7jE,QAChBjC,KAAKi2B,SAAS,QAIX6vC,GAAQA,EAAK3xC,SAChBn0B,KAAKg2B,UAAU,QAIZ8vC,GAAQA,EAAKr3D,WAChBzO,KAAKgC,WAAWqG,QAAQ,SAAUu9D,GAChCA,EAAU1yD,WAAW0yD,EAAUvxC,kBAGjCr0B,KAAKkT,WAAWlT,KAAKq0B,kBAazB6B,EAAK/iB,UAAUsjB,IAAM,SAAShoB,GAC5B,GAAIgnB,GAAQz1B,KAAKs2B,eAGjB,IAAoB,OAAhBb,EAAM7lB,OAAgC,OAAd6lB,EAAM5lB,IAAlC,CAIA,GAAI2mB,GAAW/nB,GAA+BjI,SAApBiI,EAAQ+nB,QAAyB/nB,EAAQ+nB,SAAU,CAC7Ex2B,MAAKy1B,MAAMlC,SAASkC,EAAM7lB,MAAO6lB,EAAM5lB,IAAK2mB,KAQ9CN,EAAK/iB,UAAUmjB,cAAgB,WAE7B,GAAID,GAAYr2B,KAAK+2B,eAGjBnnB,EAAQymB,EAAUvqB,IAClB+D,EAAMwmB,EAAU3pB,GACpB,IAAa,MAATkD,GAAwB,MAAPC,EAAa,CAChC,GAAI2iB,GAAY3iB,EAAI7I,UAAY4I,EAAM5I,SACtB,IAAZwrB,IAEFA,EAAW,OAEb5iB,EAAQ,GAAItL,MAAKsL,EAAM5I,UAAuB,IAAXwrB,GACnC3iB,EAAM,GAAIvL,MAAKuL,EAAI7I,UAAuB,IAAXwrB,GAGjC,OACE5iB,MAAOA,EACPC,IAAKA,IAuBTqmB,EAAK/iB,UAAUojB,UAAY,SAAS3mB,EAAOC,EAAKpB,GAC9C,GAAI+nB,GAAW/nB,GAA+BjI,SAApBiI,EAAQ+nB,QAAyB/nB,EAAQ+nB,SAAU,CAC7E,IAAwB,GAApB9wB,UAAUC,OAAa,CACzB,GAAI8vB,GAAQ/vB,UAAU,EACtB1F,MAAKy1B,MAAMlC,SAASkC,EAAM7lB,MAAO6lB,EAAM5lB,IAAK2mB,OAG5Cx2B,MAAKy1B,MAAMlC,SAAS3jB,EAAOC,EAAK2mB,IAcpCN,EAAK/iB,UAAU0U,OAAS,SAASsS,EAAM1rB,GACrC,GAAI+jB,GAAWxyB,KAAKy1B,MAAM5lB,IAAM7P,KAAKy1B,MAAM7lB,MACvC9B,EAAInN,EAAKkG,QAAQszB,EAAM,QAAQnzB,UAE/B4I,EAAQ9B,EAAI0kB,EAAW,EACvB3iB,EAAM/B,EAAI0kB,EAAW,EACrBgE,EAAW/nB,GAA+BjI,SAApBiI,EAAQ+nB,QAAyB/nB,EAAQ+nB,SAAU,CAE7Ex2B,MAAKy1B,MAAMlC,SAAS3jB,EAAOC,EAAK2mB,IAOlCN,EAAK/iB,UAAU4yD,UAAY,WACzB,GAAItwC,GAAQz1B,KAAKy1B,MAAM2J,UACvB,QACExvB,MAAO,GAAItL,MAAKmxB,EAAM7lB,OACtBC,IAAK,GAAIvL,MAAKmxB,EAAM5lB,OAQxBqmB,EAAK/iB,UAAUsO,OAAS,WACtB,GAAIgjB,IAAU,EACVh2B,EAAUzO,KAAKyO,QACfzI,EAAQhG,KAAKgG,MACb+pB,EAAM/vB,KAAK+vB,GAEf,IAAKA,EAAL,CAEApuB,EAASi2B,kBAAkB53B,KAAK20B,KAAM30B,KAAKyO,QAAQsmB,aAGxB,OAAvBtmB,EAAQ8lB,aACV5zB,EAAKiH,aAAamoB,EAAIrwB,KAAM,OAC5BiB,EAAKuH,gBAAgB6nB,EAAIrwB,KAAM,YAG/BiB,EAAKuH,gBAAgB6nB,EAAIrwB,KAAM,OAC/BiB,EAAKiH,aAAamoB,EAAIrwB,KAAM,WAI9BqwB,EAAIrwB,KAAKuN,MAAMunB,UAAY7zB,EAAKmJ,OAAOK,OAAOsE,EAAQ+lB,UAAW,IACjEzE,EAAIrwB,KAAKuN,MAAMwnB,UAAY9zB,EAAKmJ,OAAOK,OAAOsE,EAAQgmB,UAAW,IACjE1E,EAAIrwB,KAAKuN,MAAMsF,MAAQ5R,EAAKmJ,OAAOK,OAAOsE,EAAQ8D,MAAO,IAGzDvM,EAAMoG,OAAO3E,MAAUsoB,EAAI8H,gBAAgBzH,YAAcL,EAAI8H,gBAAgBrY,aAAe,EAC5FxZ,EAAMoG,OAAOib,MAASrhB,EAAMoG,OAAO3E,KACnCzB,EAAMoG,OAAOzE,KAAUooB,EAAI8H,gBAAgBvH,aAAeP,EAAI8H,gBAAgBhT,cAAgB,EAC9F7e,EAAMoG,OAAOkX,OAAStd,EAAMoG,OAAOzE,GACnC,IAAIq+D,GAAkBj2C,EAAIrwB,KAAK4wB,aAAeP,EAAIrwB,KAAKmlB,aACnDohD,EAAkBl2C,EAAIrwB,KAAK0wB,YAAcL,EAAIrwB,KAAK8f,WAIb,KAArCuQ,EAAI8H,gBAAgBhT,eACtB7e,EAAMoG,OAAO3E,KAAOzB,EAAMoG,OAAOzE,IACjC3B,EAAMoG,OAAOib,MAASrhB,EAAMoG,OAAO3E,MAEP,IAA1BsoB,EAAIrwB,KAAKmlB,eACXohD,EAAkBD,GAKpBhgE,EAAMkmB,OAAO1Z,OAASud,EAAI7D,OAAOoE,aACjCtqB,EAAMyB,KAAK+K,OAAWud,EAAItoB,KAAK6oB,aAC/BtqB,EAAMqhB,MAAM7U,OAAUud,EAAI1I,MAAMiJ,aAChCtqB,EAAM2B,IAAI6K,OAAYud,EAAIpoB,IAAIkd,eAAoB7e,EAAMoG,OAAOzE,IAC/D3B,EAAMsd,OAAO9Q,OAASud,EAAIzM,OAAOuB,eAAiB7e,EAAMoG,OAAOkX,MAM/D,IAAI+M,GAAgBnrB,KAAKwH,IAAI1G,EAAMyB,KAAK+K,OAAQxM,EAAMkmB,OAAO1Z,OAAQxM,EAAMqhB,MAAM7U,QAC7E0zD,EAAalgE,EAAM2B,IAAI6K,OAAS6d,EAAgBrqB,EAAMsd,OAAO9Q,OAC/DwzD,EAAmBhgE,EAAMoG,OAAOzE,IAAM3B,EAAMoG,OAAOkX,MACrDyM,GAAIrwB,KAAKuN,MAAMuF,OAAS7R,EAAKmJ,OAAOK,OAAOsE,EAAQ+D,OAAQ0zD,EAAa,MAGxElgE,EAAMtG,KAAK8S,OAASud,EAAIrwB,KAAK4wB,aAC7BtqB,EAAMmG,WAAWqG,OAASxM,EAAMtG,KAAK8S,OAASwzD,CAC9C,IAAI3qC,GAAkBr1B,EAAMtG,KAAK8S,OAASxM,EAAM2B,IAAI6K,OAASxM,EAAMsd,OAAO9Q,OACxEwzD,CACFhgE,GAAM6xB,gBAAgBrlB,OAAU6oB,EAChCr1B,EAAM6+D,cAAcryD,OAAY6oB,EAChCr1B,EAAM8+D,eAAetyD,OAAWxM,EAAM6+D,cAAcryD,OAGpDxM,EAAMtG,KAAK6S,MAAQwd,EAAIrwB,KAAK0wB,YAC5BpqB,EAAMmG,WAAWoG,MAAQvM,EAAMtG,KAAK6S,MAAQ0zD,EAC5CjgE,EAAMyB,KAAK8K,MAAQwd,EAAI80C,cAAcrlD,cAAkBxZ,EAAMoG,OAAO3E,KACpEzB,EAAM6+D,cAActyD,MAAQvM,EAAMyB,KAAK8K,MACvCvM,EAAMqhB,MAAM9U,MAAQwd,EAAI+0C,eAAetlD,cAAgBxZ,EAAMoG,OAAOib,MACpErhB,EAAM8+D,eAAevyD,MAAQvM,EAAMqhB,MAAM9U,KACzC,IAAI4zD,GAAcngE,EAAMtG,KAAK6S,MAAQvM,EAAMyB,KAAK8K,MAAQvM,EAAMqhB,MAAM9U,MAAQ0zD,CAC5EjgE,GAAMkmB,OAAO3Z,MAAiB4zD,EAC9BngE,EAAM6xB,gBAAgBtlB,MAAQ4zD,EAC9BngE,EAAM2B,IAAI4K,MAAoB4zD,EAC9BngE,EAAMsd,OAAO/Q,MAAiB4zD,EAG9Bp2C,EAAI5jB,WAAWc,MAAMuF,OAAmBxM,EAAMmG,WAAWqG,OAAS,KAClEud,EAAIgV,mBAAmB93B,MAAMuF,OAAWxM,EAAMmG,WAAWqG,OAAS,KAClEud,EAAImY,qBAAqBj7B,MAAMuF,OAASxM,EAAM6xB,gBAAgBrlB,OAAS,KACvEud,EAAI8H,gBAAgB5qB,MAAMuF,OAAcxM,EAAM6xB,gBAAgBrlB,OAAS,KACvEud,EAAI80C,cAAc53D,MAAMuF,OAAgBxM,EAAM6+D,cAAcryD,OAAS,KACrEud,EAAI+0C,eAAe73D,MAAMuF,OAAexM,EAAM8+D,eAAetyD,OAAS,KAEtEud,EAAI5jB,WAAWc,MAAMsF,MAAmBvM,EAAMmG,WAAWoG,MAAQ,KACjEwd,EAAIgV,mBAAmB93B,MAAMsF,MAAWvM,EAAM6xB,gBAAgBtlB,MAAQ,KACtEwd,EAAImY,qBAAqBj7B,MAAMsF,MAASvM,EAAMmG,WAAWoG,MAAQ,KACjEwd,EAAI8H,gBAAgB5qB,MAAMsF,MAAcvM,EAAMkmB,OAAO3Z,MAAQ,KAC7Dwd,EAAIpoB,IAAIsF,MAAMsF,MAA0BvM,EAAM2B,IAAI4K,MAAQ,KAC1Dwd,EAAIzM,OAAOrW,MAAMsF,MAAuBvM,EAAMsd,OAAO/Q,MAAQ,KAG7Dwd,EAAI5jB,WAAWc,MAAMxF,KAAiB,IACtCsoB,EAAI5jB,WAAWc,MAAMtF,IAAiB,IACtCooB,EAAIgV,mBAAmB93B,MAAMxF,KAAUzB,EAAMyB,KAAK8K,MAAQvM,EAAMoG,OAAO3E,KAAQ,KAC/EsoB,EAAIgV,mBAAmB93B,MAAMtF,IAAS,IACtCooB,EAAImY,qBAAqBj7B,MAAMxF,KAAO,IACtCsoB,EAAImY,qBAAqBj7B,MAAMtF,IAAO3B,EAAM2B,IAAI6K,OAAS,KACzDud,EAAI8H,gBAAgB5qB,MAAMxF,KAAYzB,EAAMyB,KAAK8K,MAAQ,KACzDwd,EAAI8H,gBAAgB5qB,MAAMtF,IAAY3B,EAAM2B,IAAI6K,OAAS,KACzDud,EAAI80C,cAAc53D,MAAMxF,KAAc,IACtCsoB,EAAI80C,cAAc53D,MAAMtF,IAAc3B,EAAM2B,IAAI6K,OAAS,KACzDud,EAAI+0C,eAAe73D,MAAMxF,KAAczB,EAAMyB,KAAK8K,MAAQvM,EAAMkmB,OAAO3Z,MAAS,KAChFwd,EAAI+0C,eAAe73D,MAAMtF,IAAa3B,EAAM2B,IAAI6K,OAAS,KACzDud,EAAIpoB,IAAIsF,MAAMxF,KAAwBzB,EAAMyB,KAAK8K,MAAQ,KACzDwd,EAAIpoB,IAAIsF,MAAMtF,IAAwB,IACtCooB,EAAIzM,OAAOrW,MAAMxF,KAAqBzB,EAAMyB,KAAK8K,MAAQ,KACzDwd,EAAIzM,OAAOrW,MAAMtF,IAAsB3B,EAAM2B,IAAI6K,OAASxM,EAAM6xB,gBAAgBrlB,OAAU,KAI1FxS,KAAKomE,kBAGL,IAAIz8C,GAAS3pB,KAAKgG,MAAMqhC,SACG,WAAvB54B,EAAQ8lB,cACV5K,GAAUzkB,KAAKwH,IAAI1M,KAAKgG,MAAM6xB,gBAAgBrlB,OAASxS,KAAKgG,MAAMkmB,OAAO1Z,OACvExS,KAAKgG,MAAMoG,OAAOzE,IAAM3H,KAAKgG,MAAMoG,OAAOkX,OAAQ,IAEtDyM,EAAI7D,OAAOjf,MAAMxF,KAAO,IACxBsoB,EAAI7D,OAAOjf,MAAMtF,IAAOgiB,EAAS,KACjCoG,EAAItoB,KAAKwF,MAAMxF,KAAS,IACxBsoB,EAAItoB,KAAKwF,MAAMtF,IAASgiB,EAAS,KACjCoG,EAAI1I,MAAMpa,MAAMxF,KAAQ,IACxBsoB,EAAI1I,MAAMpa,MAAMtF,IAAQgiB,EAAS,IAGjC,IAAI08C,GAAwC,GAAxBrmE,KAAKgG,MAAMqhC,UAAiB,SAAW,GACvDi/B,EAAmBtmE,KAAKgG,MAAMqhC,WAAarnC,KAAKgG,MAAMy/D,aAAe,SAAW,EAYpF,IAXA11C,EAAIg1C,UAAU93D,MAAMuqB,WAAsB6uC,EAC1Ct2C,EAAIi1C,aAAa/3D,MAAMuqB,WAAmB8uC,EAC1Cv2C,EAAIk1C,cAAch4D,MAAMuqB,WAAkB6uC,EAC1Ct2C,EAAIm1C,iBAAiBj4D,MAAMuqB,WAAe8uC,EAC1Cv2C,EAAIo1C,eAAel4D,MAAMuqB,WAAiB6uC,EAC1Ct2C,EAAIq1C,kBAAkBn4D,MAAMuqB,WAAc8uC,EAG1CtmE,KAAKgC,WAAWqG,QAAQ,SAAUu9D,GAChCnhC,EAAUmhC,EAAUnkD,UAAYgjB,IAE9BA,EAAS,CAEX,GAAI8hC,GAAc,CACdvmE,MAAK0lE,YAAca,GACrBvmE,KAAK0lE,cACL1lE,KAAKyhB,UAGLkX,QAAQhF,IAAI,qCAEd3zB,KAAK0lE,YAAc,EAGrB1lE,KAAK4tB,KAAK,oBAIZsI,EAAK/iB,UAAUqzD,QAAU,WACvB,KAAM,IAAI5iE,OAAM,wDAUlBsyB,EAAK/iB,UAAUiyB,eAAiB,SAASjL,GACvC,IAAKn6B,KAAK21B,YACR,KAAM,IAAI/xB,OAAM,sCAGlB5D,MAAK21B,YAAYyP,eAAejL,IAQlCjE,EAAK/iB,UAAUkyB,eAAiB,WAC9B,IAAKrlC,KAAK21B,YACR,KAAM,IAAI/xB,OAAM,sCAGlB,OAAO5D,MAAK21B,YAAY0P,kBAU1BnP,EAAK/iB,UAAUmiB,QAAU,SAASvjB,GAChC,MAAOpQ,GAAS0zB,OAAOr1B,KAAM+R,EAAG/R,KAAKgG,MAAMkmB,OAAO3Z,QAUpD2jB,EAAK/iB,UAAUqiB,cAAgB,SAASzjB,GACtC,MAAOpQ,GAAS0zB,OAAOr1B,KAAM+R,EAAG/R,KAAKgG,MAAMtG,KAAK6S,QAalD2jB,EAAK/iB,UAAU+hB,UAAY,SAASiF,GAClC,MAAOx4B,GAASszB,SAASj1B,KAAMm6B,EAAMn6B,KAAKgG,MAAMkmB,OAAO3Z,QAczD2jB,EAAK/iB,UAAUiiB,gBAAkB,SAAS+E,GACxC,MAAOx4B,GAASszB,SAASj1B,KAAMm6B,EAAMn6B,KAAKgG,MAAMtG,KAAK6S,QAUvD2jB,EAAK/iB,UAAUwyD,gBAAkB,WACA,GAA3B3lE,KAAKyO,QAAQ6lB,WACft0B,KAAKymE,mBAGLzmE,KAAK6lE,mBAST3vC,EAAK/iB,UAAUszD,iBAAmB,WAChC,GAAItyD,GAAKnU,IAETA,MAAK6lE,kBAEL7lE,KAAK0mE,UAAY,WACf,MAA6B,IAAzBvyD,EAAG1F,QAAQ6lB,eAEbngB,GAAG0xD,uBAID1xD,EAAG4b,IAAIrwB,OAKJyU,EAAG4b,IAAIrwB,KAAK0wB,aAAejc,EAAGnO,MAAM2rC,WACtCx9B,EAAG4b,IAAIrwB,KAAK4wB,cAAgBnc,EAAGnO,MAAM2gE,cACtCxyD,EAAGnO,MAAM2rC,UAAYx9B,EAAG4b,IAAIrwB,KAAK0wB,YACjCjc,EAAGnO,MAAM2gE,WAAaxyD,EAAG4b,IAAIrwB,KAAK4wB,aAElCnc,EAAGyZ,KAAK,aAMdjtB,EAAKgI,iBAAiBY,OAAQ,SAAUvJ,KAAK0mE,WAE7C1mE,KAAK4mE,WAAaC,YAAY7mE,KAAK0mE,UAAW,MAOhDxwC,EAAK/iB,UAAU0yD,gBAAkB,WAC3B7lE,KAAK4mE,aACPn0C,cAAczyB,KAAK4mE,YACnB5mE,KAAK4mE,WAAapgE,QAIpB7F,EAAKwI,oBAAoBI,OAAQ,SAAUvJ,KAAK0mE,WAChD1mE,KAAK0mE,UAAY,MASnBxwC,EAAK/iB,UAAU2zD,cAAgB,SAAUz/B,GAGvC,MAFArnC,MAAKgG,MAAMqhC,UAAYA,EACvBrnC,KAAKomE,mBACEpmE,KAAKgG,MAAMqhC,WAQpBnR,EAAK/iB,UAAUizD,iBAAmB,WAEhC,GAAIX,GAAevgE,KAAK4G,IAAI9L,KAAKgG,MAAM6xB,gBAAgBrlB,OAASxS,KAAKgG,MAAMkmB,OAAO1Z,OAAQ,EAc1F,OAbIizD,IAAgBzlE,KAAKgG,MAAMy/D,eAGG,UAA5BzlE,KAAKyO,QAAQ8lB,cACfv0B,KAAKgG,MAAMqhC,WAAco+B,EAAezlE,KAAKgG,MAAMy/D,cAErDzlE,KAAKgG,MAAMy/D,aAAeA,GAIxBzlE,KAAKgG,MAAMqhC,UAAY,IAAGrnC,KAAKgG,MAAMqhC,UAAY,GACjDrnC,KAAKgG,MAAMqhC,UAAYo+B,IAAczlE,KAAKgG,MAAMqhC,UAAYo+B,GAEzDzlE,KAAKgG,MAAMqhC,WAQpBnR,EAAK/iB,UAAU4zD,cAAgB,WAC7B,MAAO/mE,MAAKgG,MAAMqhC,WAGpBxnC,EAAOD,QAAUs2B,GAKb,SAASr2B,EAAQD,EAASM,GAEjBA,EAAoB,GAOjCN,GAAQ0pD,QAAU,SAAUxlD,EAAQwE,GAClCA,EAAS0+D,aAAe,SAAU19D,GAC5BA,EAAMqnC,SACRroC,EAASgB,IAIbxF,EAAOyP,GAAG,eAAgBjL,EAAS0+D,eAQrCpnE,EAAQqnE,UAAY,SAAUnjE,EAAQwE,GAOpC,MANAA,GAAS0+D,aAAe,SAAU19D,GAC5BA,EAAM49D,SACR5+D,EAASgB,IAINxF,EAAOyP,GAAG,eAAgBjL,EAAS0+D,eAQ5CpnE,EAAQunE,SAAW,SAAUrjE,EAAQwE,GACnCxE,EAAO4P,IAAI,eAAgBpL,EAAS0+D,eAQtCpnE,EAAQwnE,WAAaxnE,EAAQunE,UAKzB,SAAStnE,EAAQD,GAGrBA,EAAY,IACVk6B,QAAS,UACTK,KAAM,QAERv6B,EAAe,MAAIA,EAAY,GAC/BA,EAAe,MAAIA,EAAY,GAG/BA,EAAY,IACVynE,OAAQ,aACRltC,KAAM,QAERv6B,EAAe,MAAIA,EAAY,GAC/BA,EAAe,MAAIA,EAAY,IAK3B,SAASC,EAAQD,GAGrBA,EAAY,IACVg9C,KAAM,OACNG,IAAK,kBACLuqB,KAAM,OACNrG,QAAS,WACTG,QAAS,WACTmG,SAAU,YACV1qB,SAAU,YACV2qB,eAAgB,+CAChBC,gBAAiB,qEACjBC,oBAAqB,wEACrBC,gBAAiB,kCACjBC,mBAAoB,+BAEtBhoE,EAAe,MAAIA,EAAY,GAC/BA,EAAe,MAAIA,EAAY,GAG/BA,EAAY,IACVg9C,KAAM,WACNG,IAAK,uBACLuqB,KAAM,QACNrG,QAAS,iBACTG,QAAS,iBACTmG,SAAU,gBACV1qB,SAAU,gBACV2qB,eAAgB,uDAChBC,gBAAiB,6EACjBC,oBAAqB,kFACrBC,gBAAiB,wCACjBC,mBAAoB,2CAEtBhoE,EAAe,MAAIA,EAAY,GAC/BA,EAAe,MAAIA,EAAY,IAK3B,WAKoC,mBAA7BioE,4BAKTA,yBAAyB10D,UAAUisD,OAAS,SAASrtD,EAAGC,EAAGvH,GACzDzK,KAAK4nB,YACL5nB,KAAK0rB,IAAI3Z,EAAGC,EAAGvH,EAAG,EAAG,EAAEvF,KAAKymB,IAAI,IASlCk8C,yBAAyB10D,UAAU20D,OAAS,SAAS/1D,EAAGC,EAAGvH,GACzDzK,KAAK4nB,YACL5nB,KAAKyS,KAAKV,EAAItH,EAAGuH,EAAIvH,EAAO,EAAJA,EAAW,EAAJA,IASjCo9D,yBAAyB10D,UAAU4b,SAAW,SAAShd,EAAGC,EAAGvH,GAE3DzK,KAAK4nB,WAEL,IAAIhc,GAAQ,EAAJnB,EACJs9D,EAAKn8D,EAAI,EACTo8D,EAAK9iE,KAAKyqB,KAAK,GAAK,EAAI/jB,EACxBD,EAAIzG,KAAKyqB,KAAK/jB,EAAIA,EAAIm8D,EAAKA,EAE/B/nE,MAAK6nB,OAAO9V,EAAGC,GAAKrG,EAAIq8D,IACxBhoE,KAAK8nB,OAAO/V,EAAIg2D,EAAI/1D,EAAIg2D,GACxBhoE,KAAK8nB,OAAO/V,EAAIg2D,EAAI/1D,EAAIg2D,GACxBhoE,KAAK8nB,OAAO/V,EAAGC,GAAKrG,EAAIq8D,IACxBhoE,KAAKioB,aASP4/C,yBAAyB10D,UAAU80D,aAAe,SAASl2D,EAAGC,EAAGvH,GAE/DzK,KAAK4nB,WAEL,IAAIhc,GAAQ,EAAJnB,EACJs9D,EAAKn8D,EAAI,EACTo8D,EAAK9iE,KAAKyqB,KAAK,GAAK,EAAI/jB,EACxBD,EAAIzG,KAAKyqB,KAAK/jB,EAAIA,EAAIm8D,EAAKA,EAE/B/nE,MAAK6nB,OAAO9V,EAAGC,GAAKrG,EAAIq8D,IACxBhoE,KAAK8nB,OAAO/V,EAAIg2D,EAAI/1D,EAAIg2D,GACxBhoE,KAAK8nB,OAAO/V,EAAIg2D,EAAI/1D,EAAIg2D,GACxBhoE,KAAK8nB,OAAO/V,EAAGC,GAAKrG,EAAIq8D,IACxBhoE,KAAKioB,aASP4/C,yBAAyB10D,UAAU+0D,KAAO,SAASn2D,EAAGC,EAAGvH,GAEvDzK,KAAK4nB,WAEL,KAAK,GAAIugD,GAAI,EAAO,GAAJA,EAAQA,IAAK,CAC3B,GAAI18C,GAAU08C,EAAI,IAAM,EAAS,IAAJ19D,EAAc,GAAJA,CACvCzK,MAAK8nB,OACD/V,EAAI0Z,EAASvmB,KAAKkZ,IAAQ,EAAJ+pD,EAAQjjE,KAAKymB,GAAK,IACxC3Z,EAAIyZ,EAASvmB,KAAKqZ,IAAQ,EAAJ4pD,EAAQjjE,KAAKymB,GAAK,KAI9C3rB,KAAKioB,aAMP4/C,yBAAyB10D,UAAUssD,UAAY,SAAS1tD,EAAGC,EAAGq9C,EAAG1jD,EAAGlB,GAClE,GAAI29D,GAAMljE,KAAKymB,GAAG,GACE,GAAhB0jC,EAAM,EAAI5kD,IAAYA,EAAM4kD,EAAI,GAChB,EAAhB1jD,EAAM,EAAIlB,IAAYA,EAAMkB,EAAI,GACpC3L,KAAK4nB,YACL5nB,KAAK6nB,OAAO9V,EAAEtH,EAAEuH,GAChBhS,KAAK8nB,OAAO/V,EAAEs9C,EAAE5kD,EAAEuH,GAClBhS,KAAK0rB,IAAI3Z,EAAEs9C,EAAE5kD,EAAEuH,EAAEvH,EAAEA,EAAM,IAAJ29D,EAAY,IAAJA,GAAQ,GACrCpoE,KAAK8nB,OAAO/V,EAAEs9C,EAAEr9C,EAAErG,EAAElB,GACpBzK,KAAK0rB,IAAI3Z,EAAEs9C,EAAE5kD,EAAEuH,EAAErG,EAAElB,EAAEA,EAAE,EAAM,GAAJ29D,GAAO,GAChCpoE,KAAK8nB,OAAO/V,EAAEtH,EAAEuH,EAAErG,GAClB3L,KAAK0rB,IAAI3Z,EAAEtH,EAAEuH,EAAErG,EAAElB,EAAEA,EAAM,GAAJ29D,EAAW,IAAJA,GAAQ,GACpCpoE,KAAK8nB,OAAO/V,EAAEC,EAAEvH,GAChBzK,KAAK0rB,IAAI3Z,EAAEtH,EAAEuH,EAAEvH,EAAEA,EAAM,IAAJ29D,EAAY,IAAJA,GAAQ,IAMrCP,yBAAyB10D,UAAUysD,QAAU,SAAS7tD,EAAGC,EAAGq9C,EAAG1jD,GAC7D,GAAI08D,GAAQ,SACRC,EAAMjZ,EAAI,EAAKgZ,EACfE,EAAM58D,EAAI,EAAK08D,EACfG,EAAKz2D,EAAIs9C,EACToZ,EAAKz2D,EAAIrG,EACT+8D,EAAK32D,EAAIs9C,EAAI,EACbsZ,EAAK32D,EAAIrG,EAAI,CAEjB3L;KAAK4nB,YACL5nB,KAAK6nB,OAAO9V,EAAG42D,GACf3oE,KAAK4oE,cAAc72D,EAAG42D,EAAKJ,EAAIG,EAAKJ,EAAIt2D,EAAG02D,EAAI12D,GAC/ChS,KAAK4oE,cAAcF,EAAKJ,EAAIt2D,EAAGw2D,EAAIG,EAAKJ,EAAIC,EAAIG,GAChD3oE,KAAK4oE,cAAcJ,EAAIG,EAAKJ,EAAIG,EAAKJ,EAAIG,EAAIC,EAAID,GACjDzoE,KAAK4oE,cAAcF,EAAKJ,EAAIG,EAAI12D,EAAG42D,EAAKJ,EAAIx2D,EAAG42D,IAQjDd,yBAAyB10D,UAAUusD,SAAW,SAAS3tD,EAAGC,EAAGq9C,EAAG1jD,GAC9D,GAAIiC,GAAI,EAAE,EACNi7D,EAAWxZ,EACXyZ,EAAWn9D,EAAIiC,EAEfy6D,EAAQ,SACRC,EAAMO,EAAW,EAAKR,EACtBE,EAAMO,EAAW,EAAKT,EACtBG,EAAKz2D,EAAI82D,EACTJ,EAAKz2D,EAAI82D,EACTJ,EAAK32D,EAAI82D,EAAW,EACpBF,EAAK32D,EAAI82D,EAAW,EACpBC,EAAM/2D,GAAKrG,EAAIm9D,EAAS,GACxBE,EAAMh3D,EAAIrG,CAEd3L,MAAK4nB,YACL5nB,KAAK6nB,OAAO2gD,EAAIG,GAEhB3oE,KAAK4oE,cAAcJ,EAAIG,EAAKJ,EAAIG,EAAKJ,EAAIG,EAAIC,EAAID,GACjDzoE,KAAK4oE,cAAcF,EAAKJ,EAAIG,EAAI12D,EAAG42D,EAAKJ,EAAIx2D,EAAG42D,GAE/C3oE,KAAK4oE,cAAc72D,EAAG42D,EAAKJ,EAAIG,EAAKJ,EAAIt2D,EAAG02D,EAAI12D,GAC/ChS,KAAK4oE,cAAcF,EAAKJ,EAAIt2D,EAAGw2D,EAAIG,EAAKJ,EAAIC,EAAIG,GAEhD3oE,KAAK8nB,OAAO0gD,EAAIO,GAEhB/oE,KAAK4oE,cAAcJ,EAAIO,EAAMR,EAAIG,EAAKJ,EAAIU,EAAKN,EAAIM,GACnDhpE,KAAK4oE,cAAcF,EAAKJ,EAAIU,EAAKj3D,EAAGg3D,EAAMR,EAAIx2D,EAAGg3D,GAEjD/oE,KAAK8nB,OAAO/V,EAAG42D,IAOjBd,yBAAyB10D,UAAUukD,MAAQ,SAAS3lD,EAAGC,EAAGo8C,EAAOzoD,GAE/D,GAAIsjE,GAAKl3D,EAAIpM,EAAST,KAAKqZ,IAAI6vC,GAC3B8a,EAAKl3D,EAAIrM,EAAST,KAAKkZ,IAAIgwC,GAI3B+a,EAAKp3D,EAAa,GAATpM,EAAeT,KAAKqZ,IAAI6vC,GACjCgb,EAAKp3D,EAAa,GAATrM,EAAeT,KAAKkZ,IAAIgwC,GAGjCib,EAAKJ,EAAKtjE,EAAS,EAAIT,KAAKqZ,IAAI6vC,EAAQ,GAAMlpD,KAAKymB,IACnD29C,EAAKJ,EAAKvjE,EAAS,EAAIT,KAAKkZ,IAAIgwC,EAAQ,GAAMlpD,KAAKymB,IAGnD49C,EAAKN,EAAKtjE,EAAS,EAAIT,KAAKqZ,IAAI6vC,EAAQ,GAAMlpD,KAAKymB,IACnD69C,EAAKN,EAAKvjE,EAAS,EAAIT,KAAKkZ,IAAIgwC,EAAQ,GAAMlpD,KAAKymB,GAEvD3rB,MAAK4nB,YACL5nB,KAAK6nB,OAAO9V,EAAGC,GACfhS,KAAK8nB,OAAOuhD,EAAIC,GAChBtpE,KAAK8nB,OAAOqhD,EAAIC,GAChBppE,KAAK8nB,OAAOyhD,EAAIC,GAChBxpE,KAAKioB,aASP4/C,yBAAyB10D,UAAUqkD,WAAa,SAASzlD,EAAEC,EAAEwmD,EAAGC,EAAGgR,GAC5DA,IAAWA,GAAW,GAAG,IACd,GAAZC,IAAeA,EAAa,KAChC,IAAIC,GAAYF,EAAU9jE,MAC1B3F,MAAK6nB,OAAO9V,EAAGC,EAKf,KAJA,GAAI4M,GAAM45C,EAAGzmD,EAAI8M,EAAM45C,EAAGzmD,EACtB43D,EAAQ/qD,EAAGD,EACXirD,EAAgB3kE,KAAKyqB,KAAM/Q,EAAGA,EAAKC,EAAGA,GACtCirD,EAAU,EAAG/9B,GAAK,EACf89B,GAAe,IAAI,CACxB,GAAIH,GAAaD,EAAUK,IAAYH,EACnCD,GAAaG,IAAeH,EAAaG,EAC7C,IAAInuD,GAAQxW,KAAKyqB,KAAM+5C,EAAWA,GAAc,EAAIE,EAAMA,GACnD,GAAHhrD,IAAMlD,GAASA,GACnB3J,GAAK2J,EACL1J,GAAK43D,EAAMluD,EACX1b,KAAK+rC,EAAO,SAAW,UAAUh6B,EAAEC,GACnC63D,GAAiBH,EACjB39B,GAAQA,MAUV,SAASlsC,EAAQD,EAASM,GAQ9B,QAASyqC,GAAKrT,EAAS7oB,GACrBzO,KAAKs3B,QAAUA,EACft3B,KAAKyO,QAAUA,EALjB,GAAI7N,GAAUV,EAAoB,GAC9B2qC,EAAS3qC,EAAoB,GAOjCyqC,GAAKx3B,UAAU04B,UAAY,SAASC,GAGlC,IAAK,GAFDlwB,GAAOkwB,EAAU,GAAG95B,EACpB8J,EAAOgwB,EAAU,GAAG95B,EACf4Z,EAAI,EAAGA,EAAIkgB,EAAUnmC,OAAQimB,IACpChQ,EAAOA,EAAOkwB,EAAUlgB,GAAG5Z,EAAI85B,EAAUlgB,GAAG5Z,EAAI4J,EAChDE,EAAOA,EAAOgwB,EAAUlgB,GAAG5Z,EAAI85B,EAAUlgB,GAAG5Z,EAAI8J,CAElD,QAAQhQ,IAAK8P,EAAMlP,IAAKoP,EAAM8vB,iBAAkB5rC,KAAKyO,QAAQm9B,mBAU/DjB,EAAKx3B,UAAU44B,KAAO,SAAU/U,EAAS/kB,EAAO+5B,GAC9C,GAAe,MAAXhV,GACEA,EAAQrxB,OAAS,EAAG,CACtB,GAAIwlC,GAAMx+B,EACNosC,EAAY70C,OAAO8nC,EAAUpG,IAAI34B,MAAMuF,OAAOhI,QAAQ,KAAK,IAgB/D,IAfA2gC,EAAOvqC,EAAQwQ,cAAc,OAAQ46B,EAAU/E,YAAa+E,EAAUpG,KACtEuF,EAAK/4B,eAAe,KAAM,QAASH,EAAMpK,WACtBrB,SAAhByL,EAAMhF,OACPk+B,EAAK/4B,eAAe,KAAM,QAASH,EAAMhF,OAKzCN,EADsC,GAApCsF,EAAMxD,QAAQs8B,WAAWr8B,QACvBi8B,EAAKo/B,YAAY/yC,EAAS/kB,GAG1B04B,EAAKq/B,QAAQhzC,GAIiB,GAAhC/kB,EAAMxD,QAAQ88B,OAAO78B,QAAiB,CACxC,GACIu7D,GADA7+B,EAAWxqC,EAAQwQ,cAAc,OAAQ46B,EAAU/E,YAAa+E,EAAUpG,IAG5EqkC,GADsC,OAApCh4D,EAAMxD,QAAQ88B,OAAOhX,YACf,IAAMyC,EAAQ,GAAGjlB,EAAI,MAAgBpF,EAAI,IAAMqqB,EAAQA,EAAQrxB,OAAS,GAAGoM,EAAI,KAG/E,IAAMilB,EAAQ,GAAGjlB,EAAI,IAAMgnC,EAAY,IAAMpsC,EAAI,IAAMqqB,EAAQA,EAAQrxB,OAAS,GAAGoM,EAAI,IAAMgnC,EAEvG3N,EAASh5B,eAAe,KAAM,QAASH,EAAMpK,UAAY,SACvBrB,SAA/ByL,EAAMxD,QAAQ88B,OAAOt+B,OACtBm+B,EAASh5B,eAAe,KAAM,QAASH,EAAMxD,QAAQ88B,OAAOt+B,OAE9Dm+B,EAASh5B,eAAe,KAAM,IAAK63D,GAGrC9+B,EAAK/4B,eAAe,KAAM,IAAK,IAAMzF,GAGG,GAApCsF,EAAMxD,QAAQ0D,WAAWzD,SAC3Bm8B,EAAOkB,KAAK/U,EAAS/kB,EAAO+5B,KAepCrB,EAAKu/B,mBAAqB,SAASx3D,GAMjC,IAAK,GAJDy3D,GAAIC,EAAIC,EAAIC,EAAIC,EAAKC,EACrB79D,EAAIzH,KAAKwoB,MAAMhb,EAAK,GAAGX,GAAK,IAAM7M,KAAKwoB,MAAMhb,EAAK,GAAGV,GAAK,IAC1Dy4D,EAAgB,EAAE,EAClB9kE,EAAS+M,EAAK/M,OACTH,EAAI,EAAOG,EAAS,EAAbH,EAAgBA,IAE9B2kE,EAAW,GAAL3kE,EAAUkN,EAAK,GAAKA,EAAKlN,EAAE,GACjC4kE,EAAK13D,EAAKlN,GACV6kE,EAAK33D,EAAKlN,EAAE,GACZ8kE,EAAc3kE,EAARH,EAAI,EAAckN,EAAKlN,EAAE,GAAK6kE,EAUpCE,GAAQx4D,IAAMo4D,EAAGp4D,EAAI,EAAEq4D,EAAGr4D,EAAIs4D,EAAGt4D,GAAI04D,EAAgBz4D,IAAMm4D,EAAGn4D,EAAI,EAAEo4D,EAAGp4D,EAAIq4D,EAAGr4D,GAAIy4D,GAClFD,GAAQz4D,GAAMq4D,EAAGr4D,EAAI,EAAEs4D,EAAGt4D,EAAIu4D,EAAGv4D,GAAI04D,EAAgBz4D,GAAMo4D,EAAGp4D,EAAI,EAAEq4D,EAAGr4D,EAAIs4D,EAAGt4D,GAAIy4D,GAGlF99D,GAAK,IACL49D,EAAIx4D,EAAI,IACRw4D,EAAIv4D,EAAI,IACRw4D,EAAIz4D,EAAI,IACRy4D,EAAIx4D,EAAI,IACRq4D,EAAGt4D,EAAI,IACPs4D,EAAGr4D,EAAI,GAGT,OAAOrF,IAcTg+B,EAAKo/B,YAAc,SAASr3D,EAAMT,GAChC,GAAIg5B,GAAQh5B,EAAMxD,QAAQs8B,WAAWE,KACrC,IAAa,GAATA,GAAwBzkC,SAAVykC,EAChB,MAAOjrC,MAAKkqE,mBAAmBx3D,EAO/B,KAAK,GAJDy3D,GAAIC,EAAIC,EAAIC,EAAIC,EAAKC,EAAKE,EAAGC,EAAGC,EAAIC,EAAGpgD,EAAGqgD,EAAGC,EAC7CC,EAAQC,EAAQC,EAASC,EAASC,EAASC,EAC3C1+D,EAAIzH,KAAKwoB,MAAMhb,EAAK,GAAGX,GAAK,IAAM7M,KAAKwoB,MAAMhb,EAAK,GAAGV,GAAK,IAC1DrM,EAAS+M,EAAK/M,OACTH,EAAI,EAAOG,EAAS,EAAbH,EAAgBA,IAE9B2kE,EAAW,GAAL3kE,EAAUkN,EAAK,GAAKA,EAAKlN,EAAE,GACjC4kE,EAAK13D,EAAKlN,GACV6kE,EAAK33D,EAAKlN,EAAE,GACZ8kE,EAAc3kE,EAARH,EAAI,EAAckN,EAAKlN,EAAE,GAAK6kE,EAEpCK,EAAKxlE,KAAKyqB,KAAKzqB,KAAK4uB,IAAIq2C,EAAGp4D,EAAIq4D,EAAGr4D,EAAE,GAAK7M,KAAK4uB,IAAIq2C,EAAGn4D,EAAIo4D,EAAGp4D,EAAE,IAC9D24D,EAAKzlE,KAAKyqB,KAAKzqB,KAAK4uB,IAAIs2C,EAAGr4D,EAAIs4D,EAAGt4D,EAAE,GAAK7M,KAAK4uB,IAAIs2C,EAAGp4D,EAAIq4D,EAAGr4D,EAAE,IAC9D44D,EAAK1lE,KAAKyqB,KAAKzqB,KAAK4uB,IAAIu2C,EAAGt4D,EAAIu4D,EAAGv4D,EAAE,GAAK7M,KAAK4uB,IAAIu2C,EAAGr4D,EAAIs4D,EAAGt4D,EAAE,IAY9Dg5D,EAAU9lE,KAAK4uB,IAAI82C,EAAK3/B,GACxBigC,EAAUhmE,KAAK4uB,IAAI82C,EAAG,EAAE3/B,GACxBggC,EAAU/lE,KAAK4uB,IAAI62C,EAAK1/B,GACxBkgC,EAAUjmE,KAAK4uB,IAAI62C,EAAG,EAAE1/B,GACxBogC,EAAUnmE,KAAK4uB,IAAI42C,EAAKz/B,GACxBmgC,EAAUlmE,KAAK4uB,IAAI42C,EAAG,EAAEz/B,GAExB4/B,EAAI,EAAEO,EAAU,EAAEC,EAASJ,EAASE,EACpC1gD,EAAI,EAAEygD,EAAU,EAAEF,EAASC,EAASE,EACpCL,EAAI,EAAEO,GAAUA,EAASJ,GACrBH,EAAI,IAAIA,EAAI,EAAIA,GACpBC,EAAI,EAAEC,GAAUA,EAASC,GACrBF,EAAI,IAAIA,EAAI,EAAIA,GAEpBR,GAAQx4D,IAAMo5D,EAAUhB,EAAGp4D,EAAI84D,EAAET,EAAGr4D,EAAIq5D,EAAUf,EAAGt4D,GAAK+4D,EACxD94D,IAAMm5D,EAAUhB,EAAGn4D,EAAI64D,EAAET,EAAGp4D,EAAIo5D,EAAUf,EAAGr4D,GAAK84D,GAEpDN,GAAQz4D,GAAMm5D,EAAUd,EAAGr4D,EAAI0Y,EAAE4/C,EAAGt4D,EAAIo5D,EAAUb,EAAGv4D,GAAKg5D,EACxD/4D,GAAMk5D,EAAUd,EAAGp4D,EAAIyY,EAAE4/C,EAAGr4D,EAAIm5D,EAAUb,EAAGt4D,GAAK+4D,GAEvC,GAATR,EAAIx4D,GAAmB,GAATw4D,EAAIv4D,IAASu4D,EAAMH,GACxB,GAATI,EAAIz4D,GAAmB,GAATy4D,EAAIx4D,IAASw4D,EAAMH,GACrC19D,GAAK,IACL49D,EAAIx4D,EAAI,IACRw4D,EAAIv4D,EAAI,IACRw4D,EAAIz4D,EAAI,IACRy4D,EAAIx4D,EAAI,IACRq4D,EAAGt4D,EAAI,IACPs4D,EAAGr4D,EAAI,GAGT,OAAOrF,IAUXg+B,EAAKq/B,QAAU,SAASt3D,GAGtB,IAAK,GADD/F,GAAI,GACCnH,EAAI,EAAGA,EAAIkN,EAAK/M,OAAQH,IAE7BmH,GADO,GAALnH,EACGkN,EAAKlN,GAAGuM,EAAI,IAAMW,EAAKlN,GAAGwM,EAG1B,IAAMU,EAAKlN,GAAGuM,EAAI,IAAMW,EAAKlN,GAAGwM,CAGzC,OAAOrF,IAGT9M,EAAOD,QAAU+qC,GAKb,SAAS9qC,EAAQD,EAASM,GAQ9B,QAASorE,GAASh0C,EAAS7oB,GACzBzO,KAAKs3B,QAAUA,EACft3B,KAAKyO,QAAUA,EALjB,CAAA,GAAI7N,GAAUV,EAAoB,EACrBA,GAAoB,IAOjCorE,EAASn4D,UAAU04B,UAAY,SAASC,GACtC,GAA2C,SAAvC9rC,KAAKyO,QAAQymC,SAASC,cAA0B,CAGlD,IAAK,GAFDv5B,GAAOkwB,EAAU,GAAG95B,EACpB8J,EAAOgwB,EAAU,GAAG95B,EACf4Z,EAAI,EAAGA,EAAIkgB,EAAUnmC,OAAQimB,IACpChQ,EAAOA,EAAOkwB,EAAUlgB,GAAG5Z,EAAI85B,EAAUlgB,GAAG5Z,EAAI4J,EAChDE,EAAOA,EAAOgwB,EAAUlgB,GAAG5Z,EAAI85B,EAAUlgB,GAAG5Z,EAAI8J,CAElD,QAAQhQ,IAAK8P,EAAMlP,IAAKoP,EAAM8vB,iBAAkB5rC,KAAKyO,QAAQm9B,kBAI7D,IAAK,GADD2/B,MACK3/C,EAAI,EAAGA,EAAIkgB,EAAUnmC,OAAQimB,IACpC2/C,EAAgBvjE,MACd+J,EAAG+5B,EAAUlgB,GAAG7Z,EAChBC,EAAG85B,EAAUlgB,GAAG5Z,EAChBslB,QAASt3B,KAAKs3B,SAGlB,OAAOi0C,IAYXD,EAASv/B,KAAO,SAAUmE,EAAUsG,EAAoBxK,GACtD,GAEIw/B,GACA9iE,EAAK+iE,EACLx5D,EACAzM,EAAEomB,EALF8/C,KACAC,KAKAC,EAAY,CAGhB,KAAKpmE,EAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAE/B,GADAyM,EAAQ+5B,EAAU7X,OAAO+b,EAAS1qC,IACP,OAAvByM,EAAMxD,QAAQxB,OACK,GAAjBgF,EAAMyW,UAAyEliB,SAArDwlC,EAAUv9B,QAAQ0lB,OAAOqD,WAAW0Y,EAAS1qC,KAAyE,GAApDwmC,EAAUv9B,QAAQ0lB,OAAOqD,WAAW0Y,EAAS1qC,KAC3I,IAAKomB,EAAI,EAAGA,EAAI4qB,EAAmBtG,EAAS1qC,IAAIG,OAAQimB,IACtD8/C,EAAa1jE,MACX+J,EAAGykC,EAAmBtG,EAAS1qC,IAAIomB,GAAG7Z,EACtCC,EAAGwkC,EAAmBtG,EAAS1qC,IAAIomB,GAAG5Z,EACtCslB,QAAS4Y,EAAS1qC,KAEpBomE,GAAa,CAMrB,IAAiB,GAAbA,EAeJ,IAZAF,EAAax1D,KAAK,SAAU3Q,EAAGa,GAC7B,MAAIb,GAAEwM,GAAK3L,EAAE2L,EACJxM,EAAE+xB,QAAUlxB,EAAEkxB,QAEd/xB,EAAEwM,EAAI3L,EAAE2L,IAKnBu5D,EAASO,sBAAsBF,EAAeD,GAGzClmE,EAAI,EAAGA,EAAIkmE,EAAa/lE,OAAQH,IAAK,CACxCyM,EAAQ+5B,EAAU7X,OAAOu3C,EAAalmE,GAAG8xB,QACzC,IAAIqP,GAAW,GAAM10B,EAAMxD,QAAQymC,SAAS3iC,KAE5C7J,GAAMgjE,EAAalmE,GAAGuM,CACtB,IAAI+5D,GAAe,CACnB,IAA2BtlE,SAAvBmlE,EAAcjjE,GACZlD,EAAE,EAAIkmE,EAAa/lE,SAAS6lE,EAAetmE,KAAK2lB,IAAI6gD,EAAalmE,EAAE,GAAGuM,EAAIrJ,IAC1ElD,EAAI,IAAwBgmE,EAAetmE,KAAK4G,IAAI0/D,EAAatmE,KAAK2lB,IAAI6gD,EAAalmE,EAAE,GAAGuM,EAAIrJ,KACpG+iE,EAAWH,EAASS,iBAAiBP,EAAcv5D,EAAO00B,OAEvD,CACH,GAAIqlC,GAAUxmE,GAAKmmE,EAAcjjE,GAAKujE,OAASN,EAAcjjE,GAAKwjE,UAC9DC,EAAU3mE,GAAKmmE,EAAcjjE,GAAKwjE,SAAW,EAC7CF,GAAUN,EAAa/lE,SAAS6lE,EAAetmE,KAAK2lB,IAAI6gD,EAAaM,GAASj6D,EAAIrJ,IAClFyjE,EAAU,IAAsBX,EAAetmE,KAAK4G,IAAI0/D,EAAatmE,KAAK2lB,IAAI6gD,EAAaS,GAASp6D,EAAIrJ,KAC5G+iE,EAAWH,EAASS,iBAAiBP,EAAcv5D,EAAO00B,GAC1DglC,EAAcjjE,GAAKwjE,UAAY,EAEa,SAAxCj6D,EAAMxD,QAAQymC,SAASC,eACzB22B,EAAeH,EAAcjjE,GAAK0jE,YAClCT,EAAcjjE,GAAK0jE,aAAen6D,EAAMy4B,aAAeghC,EAAalmE,GAAGwM,GAExB,cAAxCC,EAAMxD,QAAQymC,SAASC,gBAC9Bs2B,EAASl5D,MAAQk5D,EAASl5D,MAAQo5D,EAAcjjE,GAAKujE,OACrDR,EAAS9hD,QAAWgiD,EAAcjjE,GAAa,SAAI+iE,EAASl5D,MAAS,GAAIk5D,EAASl5D,OAASo5D,EAAcjjE,GAAKujE,OAAO,GACjF,QAAhCh6D,EAAMxD,QAAQymC,SAASnG,MAAwB08B,EAAS9hD,QAAU,GAAI8hD,EAASl5D,MAC1C,SAAhCN,EAAMxD,QAAQymC,SAASnG,QAAmB08B,EAAS9hD,QAAU,GAAI8hD,EAASl5D,QAGvF3R,EAAQ0R,QAAQo5D,EAAalmE,GAAGuM,EAAI05D,EAAS9hD,OAAQ+hD,EAAalmE,GAAGwM,EAAI85D,EAAcL,EAASl5D,MAAON,EAAMy4B,aAAeghC,EAAalmE,GAAGwM,EAAGC,EAAMpK,UAAY,OAAQmkC,EAAU/E,YAAa+E,EAAUpG,KAElK,GAApC3zB,EAAMxD,QAAQ0D,WAAWzD,SAC3B9N,EAAQkR,UAAU45D,EAAalmE,GAAGuM,EAAI05D,EAAS9hD,OAAQ+hD,EAAalmE,GAAGwM,EAAGC,EAAO+5B,EAAU/E,YAAa+E,EAAUpG,OAYxH0lC,EAASO,sBAAwB,SAAUF,EAAeD,GAGxD,IAAK,GADDF,GACKhmE,EAAI,EAAGA,EAAIkmE,EAAa/lE,OAAQH,IACnCA,EAAI,EAAIkmE,EAAa/lE,SACvB6lE,EAAetmE,KAAK2lB,IAAI6gD,EAAalmE,EAAI,GAAGuM,EAAI25D,EAAalmE,GAAGuM,IAE9DvM,EAAI,IACNgmE,EAAetmE,KAAK4G,IAAI0/D,EAActmE,KAAK2lB,IAAI6gD,EAAalmE,EAAI,GAAGuM,EAAI25D,EAAalmE,GAAGuM,KAErE,GAAhBy5D,IACuChlE,SAArCmlE,EAAcD,EAAalmE,GAAGuM,KAChC45D,EAAcD,EAAalmE,GAAGuM,IAAMk6D,OAAQ,EAAGC,SAAU,EAAGE,YAAa,IAE3ET,EAAcD,EAAalmE,GAAGuM,GAAGk6D,QAAU,IAejDX,EAASS,iBAAmB,SAAUP,EAAcv5D,EAAO00B,GACzD,GAAIp0B,GAAOoX,CAwBX,OAvBI6hD,GAAev5D,EAAMxD,QAAQymC,SAAS3iC,OAASi5D,EAAe,GAChEj5D,EAAuBo0B,EAAf6kC,EAA0B7kC,EAAW6kC,EAE7C7hD,EAAS,EAC2B,QAAhC1X,EAAMxD,QAAQymC,SAASnG,MACzBplB,GAAU,GAAM6hD,EAEuB,SAAhCv5D,EAAMxD,QAAQymC,SAASnG,QAC9BplB,GAAU,GAAM6hD,KAKlBj5D,EAAQN,EAAMxD,QAAQymC,SAAS3iC,MAC/BoX,EAAS,EAC2B,QAAhC1X,EAAMxD,QAAQymC,SAASnG,MACzBplB,GAAU,GAAM1X,EAAMxD,QAAQymC,SAAS3iC,MAEA,SAAhCN,EAAMxD,QAAQymC,SAASnG,QAC9BplB,GAAU,GAAM1X,EAAMxD,QAAQymC,SAAS3iC,SAInCA,MAAOA,EAAOoX,OAAQA,IAGhC2hD,EAASxzB,oBAAsB,SAASyzB,EAAiB90B,EAAavG,EAAUm8B,EAAY93C,GAC1F,GAAIg3C,EAAgB5lE,OAAS,EAAG,CAE9B4lE,EAAgBr1D,KAAK,SAAU3Q,EAAGa,GAChC,MAAIb,GAAEwM,GAAK3L,EAAE2L,EACJxM,EAAE+xB,QAAUlxB,EAAEkxB,QAEd/xB,EAAEwM,EAAI3L,EAAE2L,GAGnB,IAAI45D,KAEJL,GAASO,sBAAsBF,EAAeJ,GAC9C90B,EAAY41B,GAAcf,EAASgB,qBAAqBX,EAAeJ,GACvE90B,EAAY41B,GAAYzgC,iBAAmBrX,EAC3C2b,EAASloC,KAAKqkE,KAIlBf,EAASgB,qBAAuB,SAAUX,EAAeD,GAIvD,IAAK,GAHDhjE,GACAkT,EAAO8vD,EAAa,GAAG15D,EACvB8J,EAAO4vD,EAAa,GAAG15D,EAClBxM,EAAI,EAAGA,EAAIkmE,EAAa/lE,OAAQH,IACvCkD,EAAMgjE,EAAalmE,GAAGuM,EACKvL,SAAvBmlE,EAAcjjE,IAChBkT,EAAOA,EAAO8vD,EAAalmE,GAAGwM,EAAI05D,EAAalmE,GAAGwM,EAAI4J,EACtDE,EAAOA,EAAO4vD,EAAalmE,GAAGwM,EAAI05D,EAAalmE,GAAGwM,EAAI8J,GAGtD6vD,EAAcjjE,GAAK0jE,aAAeV,EAAalmE,GAAGwM,CAGtD,KAAK,GAAIu6D,KAAQZ,GACXA,EAAc7lE,eAAeymE,KAC/B3wD,EAAOA,EAAO+vD,EAAcY,GAAMH,YAAcT,EAAcY,GAAMH,YAAcxwD,EAClFE,EAAOA,EAAO6vD,EAAcY,GAAMH,YAAcT,EAAcY,GAAMH,YAActwD,EAItF,QAAQhQ,IAAK8P,EAAMlP,IAAKoP,IAG1Bjc,EAAOD,QAAU0rE,GAIb,SAASzrE,EAAQD,EAASM,GAO9B,QAAS2qC,GAAOvT,EAAS7oB,GACvBzO,KAAKs3B,QAAUA,EACft3B,KAAKyO,QAAUA,EAJjB,GAAI7N,GAAUV,EAAoB,EAQlC2qC,GAAO13B,UAAU04B,UAAY,SAASC,GAGpC,IAAK,GAFDlwB,GAAOkwB,EAAU,GAAG95B,EACpB8J,EAAOgwB,EAAU,GAAG95B,EACf4Z,EAAI,EAAGA,EAAIkgB,EAAUnmC,OAAQimB,IACpChQ,EAAOA,EAAOkwB,EAAUlgB,GAAG5Z,EAAI85B,EAAUlgB,GAAG5Z,EAAI4J,EAChDE,EAAOA,EAAOgwB,EAAUlgB,GAAG5Z,EAAI85B,EAAUlgB,GAAG5Z,EAAI8J,CAElD,QAAQhQ,IAAK8P,EAAMlP,IAAKoP,EAAM8vB,iBAAkB5rC,KAAKyO,QAAQm9B,mBAG/Df,EAAO13B,UAAU44B,KAAO,SAAS/U,EAAS/kB,EAAO+5B,EAAWriB,GAC1DkhB,EAAOkB,KAAK/U,EAAS/kB,EAAO+5B,EAAWriB,IAYzCkhB,EAAOkB,KAAO,SAAU/U,EAAS/kB,EAAO+5B,EAAWriB,GAClCnjB,SAAXmjB,IAAuBA,EAAS,EACpC,KAAK,GAAInkB,GAAI,EAAGA,EAAIwxB,EAAQrxB,OAAQH,IAClC5E,EAAQkR,UAAUklB,EAAQxxB,GAAGuM,EAAI4X,EAAQqN,EAAQxxB,GAAGwM,EAAGC,EAAO+5B,EAAU/E,YAAa+E,EAAUpG,MAKnG/lC,EAAOD,QAAUirC,GAIb,SAAShrC,EAAQD,EAASM,GAE9B,GAAIssE,GAAetsE,EAAoB,IACnCusE,EAAevsE,EAAoB,IACnCwsE,EAAexsE,EAAoB,IACnCysE,EAAiBzsE,EAAoB,IACrC0sE,EAAoB1sE,EAAoB,IACxC2sE,EAAkB3sE,EAAoB,IACtC4sE,EAA0B5sE,EAAoB,GAQlDN,GAAQmtE,WAAa,SAAUC,GAC7B,IAAK,GAAIC,KAAiBD,GACpBA,EAAelnE,eAAemnE,KAChCjtE,KAAKitE,GAAiBD,EAAeC,KAY3CrtE,EAAQstE,YAAc,SAAUF,GAC9B,IAAK,GAAIC,KAAiBD,GACpBA,EAAelnE,eAAemnE,KAChCjtE,KAAKitE,GAAiBzmE,SAW5B5G,EAAQqjD,mBAAqB,WAC3BjjD,KAAK+sE,WAAWP,GAChBxsE,KAAKmtE,2BACkC,GAAnCntE,KAAK0hD,UAAUnD,iBACjBv+C,KAAKotE,4BAGLptE,KAAKoqD,gCAUTxqD,EAAQujD,mBAAqB,WAC3BnjD,KAAK87D,eAAiB,EACtB97D,KAAKqtE,aAAe,EACpBrtE,KAAK+sE,WAAWN,IASlB7sE,EAAQsjD,kBAAoB,WAC1BljD,KAAKkvD,WACLlvD,KAAKstE,cAAgB,WACrBttE,KAAKkvD,QAAgB,UACrBlvD,KAAKkvD,QAAgB,OAAE,YAAclS,SACnCc,SACAgG,eACAsY,eAAkB,EAClBmR,YAAe/mE,QACjBxG,KAAKkvD,QAAgB,UACrBlvD,KAAKkvD,QAAiB,SAAKlS,SACzBc,SACAgG,eACAsY,eAAkB,EAClBmR,YAAe/mE,QAEjBxG,KAAK8jD,YAAc9jD,KAAKkvD,QAAgB,OAAE,WAAwB,YAElElvD,KAAK+sE,WAAWL,IASlB9sE,EAAQwjD,qBAAuB,WAC7BpjD,KAAKkrD,cAAgBlO,SAAWc,UAEhC99C,KAAK+sE,WAAWJ,IASlB/sE,EAAQ6oD,wBAA0B,WAEhCzoD,KAAKwtE,8BAA+B,EACpCxtE,KAAKytE,sBAAuB,EAEmB,GAA3CztE,KAAK0hD,UAAUnB,iBAAiB7xC,SAELlI,SAAzBxG,KAAK0tE,kBACP1tE,KAAK0tE,gBAAkBn8D,SAASM,cAAc,OAC9C7R,KAAK0tE,gBAAgB7lE,UAAY,0BAE/B7H,KAAK0tE,gBAAgBzgE,MAAM26B,QADR,GAAjB5nC,KAAKkoD,SAC8B,QAGA,OAEvCloD,KAAKsf,MAAM7N,YAAYzR,KAAK0tE,kBAGLlnE,SAArBxG,KAAK2tE,cACP3tE,KAAK2tE,YAAcp8D,SAASM,cAAc,OAC1C7R,KAAK2tE,YAAY9lE,UAAY,gCAE3B7H,KAAK2tE,YAAY1gE,MAAM26B,QADJ,GAAjB5nC,KAAKkoD,SAC0B,OAGA,QAEnCloD,KAAKsf,MAAM7N,YAAYzR,KAAK2tE,cAGRnnE,SAAlBxG,KAAK4tE,WACP5tE,KAAK4tE,SAAWr8D,SAASM,cAAc,OACvC7R,KAAK4tE,SAAS/lE,UAAY,gCAC1B7H,KAAK4tE,SAAS3gE,MAAM26B,QAAU5nC,KAAK0tE,gBAAgBzgE,MAAM26B,QACzD5nC,KAAKsf,MAAM7N,YAAYzR,KAAK4tE,WAI9B5tE,KAAK+sE,WAAWH,GAGhB5sE,KAAKmnD,yBAGwB3gD,SAAzBxG,KAAK0tE,kBAEP1tE,KAAKmnD,wBAGLnnD,KAAKsf,MAAMnO,YAAYnR,KAAK0tE,iBAC5B1tE,KAAKsf,MAAMnO,YAAYnR,KAAK2tE,aAC5B3tE,KAAKsf,MAAMnO,YAAYnR,KAAK4tE,UAE5B5tE,KAAK0tE,gBAAkBlnE,OACvBxG,KAAK2tE,YAAcnnE,OACnBxG,KAAK4tE,SAAWpnE,OAEhBxG,KAAKktE,YAAYN,KAWvBhtE,EAAQ4oD,wBAA0B,WAChCxoD,KAAK+sE,WAAWF,GAEhB7sE,KAAK6tE,mBACoC,GAArC7tE,KAAK0hD,UAAUtB,WAAW1xC,SAC5B1O,KAAK8tE,2BAUTluE,EAAQyjD,qBAAuB,WAC7BrjD,KAAK+sE,WAAWD,KAMd,SAASjtE,EAAQD,EAASM,GAiB9B,QAASolD,GAAU/rC,GACjBvZ,KAAKszD,QAAS,EAEdtzD,KAAK+vB,KACHxW,UAAWA,GAGbvZ,KAAK+vB,IAAIg+C,QAAUx8D,SAASM,cAAc,OAC1C7R,KAAK+vB,IAAIg+C,QAAQlmE,UAAY,UAE7B7H,KAAK+vB,IAAIxW,UAAU9H,YAAYzR,KAAK+vB,IAAIg+C,SAExC/tE,KAAK8D,OAASC,EAAO/D,KAAK+vB,IAAIg+C,SAAUC,iBAAiB,IACzDhuE,KAAK8D,OAAOyP,GAAG,MAAOvT,KAAKiuE,cAAcn5C,KAAK90B,MAG9C,IAAImU,GAAKnU,KACLwlE,GACF,QAAS,QACT,YAAa,OACb,YAAa,OAAQ,UACrB,aAAc,iBAEhBA,GAAOn9D,QAAQ,SAAUiB,GACvB6K,EAAGrQ,OAAOyP,GAAGjK,EAAO,SAAUA,GAC5BA,EAAMq8B,sBAKV3lC,KAAKkuE,aAAenqE,EAAOwF,QAASykE,iBAAiB,IACrDhuE,KAAKkuE,aAAa36D,GAAG,MAAO,SAAUjK,GAE/B6kE,EAAW7kE,EAAMI,OAAQ6P,IAC5BpF,EAAGi6D,eAIe5nE,SAAlBxG,KAAKmlD,UACPnlD,KAAKmlD,SAAS7xC,UAEhBtT,KAAKmlD,SAAWA,IAGhBnlD,KAAKquE,YAAcruE,KAAKouE,WAAWt5C,KAAK90B,MAiF1C,QAASmuE,GAAWvlE,EAASk8B,GAC3B,KAAOl8B,GAAS,CACd,GAAIA,IAAYk8B,EACd,OAAO,CAETl8B,GAAUA,EAAQiB,WAEpB,OAAO,EAnJT,GAAIs7C,GAAWjlD,EAAoB,IAC/B6c,EAAU7c,EAAoB,IAC9B6D,EAAS7D,EAAoB,IAC7BS,EAAOT,EAAoB,EA4D/B6c,GAAQuoC,EAAUnyC,WAGlBmyC,EAAUxrB,QAAU,KAKpBwrB,EAAUnyC,UAAUG,QAAU,WAC5BtT,KAAKouE,aAGLpuE,KAAK+vB,IAAIg+C,QAAQlkE,WAAWsH,YAAYnR,KAAK+vB,IAAIg+C,SAGjD/tE,KAAK8D,OAAS,KACd9D,KAAKkuE,aAAe,MAQtB5oB,EAAUnyC,UAAUm7D,SAAW,WAEzBhpB,EAAUxrB,SACZwrB,EAAUxrB,QAAQs0C,aAEpB9oB,EAAUxrB,QAAU95B,KAEpBA,KAAKszD,QAAS,EACdtzD,KAAK+vB,IAAIg+C,QAAQ9gE,MAAM26B,QAAU,OACjCjnC,EAAKiH,aAAa5H,KAAK+vB,IAAIxW,UAAW,cAEtCvZ,KAAK4tB,KAAK,UACV5tB,KAAK4tB,KAAK,YAIV5tB,KAAKmlD,SAASrwB,KAAK,MAAO90B,KAAKquE,cAOjC/oB,EAAUnyC,UAAUi7D,WAAa,WAC/BpuE,KAAKszD,QAAS,EACdtzD,KAAK+vB,IAAIg+C,QAAQ9gE,MAAM26B,QAAU,GACjCjnC,EAAKuH,gBAAgBlI,KAAK+vB,IAAIxW,UAAW,cACzCvZ,KAAKmlD,SAASopB,OAAO,MAAOvuE,KAAKquE,aAEjCruE,KAAK4tB,KAAK,UACV5tB,KAAK4tB,KAAK,eAQZ03B,EAAUnyC,UAAU86D,cAAgB,SAAU3kE,GAE5CtJ,KAAKsuE,WACLhlE,EAAMq8B,mBAsBR9lC,EAAOD,QAAU0lD,GAKb,SAASzlD,GAeb,QAASkd,GAAQgG,GACf,MAAIA,GAAYqvC,EAAMrvC,GAAtB,OAWF,QAASqvC,GAAMrvC,GACb,IAAK,GAAIra,KAAOqU,GAAQ5J,UACtB4P,EAAIra,GAAOqU,EAAQ5J,UAAUzK,EAE/B,OAAOqa,GAxBTljB,EAAOD,QAAUmd,EAoCjBA,EAAQ5J,UAAUI,GAClBwJ,EAAQ5J,UAAUxK,iBAAmB,SAASW,EAAO4P,GAInD,MAHAlZ,MAAKwuE,WAAaxuE,KAAKwuE,gBACtBxuE,KAAKwuE,WAAWllE,GAAStJ,KAAKwuE,WAAWllE,QACvCtB,KAAKkR,GACDlZ,MAaT+c,EAAQ5J,UAAUs7D,KAAO,SAASnlE,EAAO4P,GAIvC,QAAS3F,KACPm7D,EAAKh7D,IAAIpK,EAAOiK,GAChB2F,EAAGnB,MAAM/X,KAAM0F,WALjB,GAAIgpE,GAAO1uE,IAUX,OATAA,MAAKwuE,WAAaxuE,KAAKwuE,eAOvBj7D,EAAG2F,GAAKA,EACRlZ,KAAKuT,GAAGjK,EAAOiK,GACRvT,MAaT+c,EAAQ5J,UAAUO,IAClBqJ,EAAQ5J,UAAUw7D,eAClB5xD,EAAQ5J,UAAUy7D,mBAClB7xD,EAAQ5J,UAAUhK,oBAAsB,SAASG,EAAO4P,GAItD,GAHAlZ,KAAKwuE,WAAaxuE,KAAKwuE,eAGnB,GAAK9oE,UAAUC,OAEjB,MADA3F,MAAKwuE,cACExuE,IAIT,IAAI6uE,GAAY7uE,KAAKwuE,WAAWllE,EAChC,KAAKulE,EAAW,MAAO7uE,KAGvB,IAAI,GAAK0F,UAAUC,OAEjB,aADO3F,MAAKwuE,WAAWllE,GAChBtJ,IAKT,KAAK,GADD8uE,GACKtpE,EAAI,EAAGA,EAAIqpE,EAAUlpE,OAAQH,IAEpC,GADAspE,EAAKD,EAAUrpE,GACXspE,IAAO51D,GAAM41D,EAAG51D,KAAOA,EAAI,CAC7B21D,EAAUzmE,OAAO5C,EAAG,EACpB,OAGJ,MAAOxF,OAWT+c,EAAQ5J,UAAUya,KAAO,SAAStkB,GAChCtJ,KAAKwuE,WAAaxuE,KAAKwuE,cACvB,IAAIv1D,MAAUhO,MAAM1K,KAAKmF,UAAW,GAChCmpE,EAAY7uE,KAAKwuE,WAAWllE,EAEhC,IAAIulE,EAAW,CACbA,EAAYA,EAAU5jE,MAAM,EAC5B,KAAK,GAAIzF,GAAI,EAAGC,EAAMopE,EAAUlpE,OAAYF,EAAJD,IAAWA,EACjDqpE,EAAUrpE,GAAGuS,MAAM/X,KAAMiZ,GAI7B,MAAOjZ,OAWT+c,EAAQ5J,UAAUoyD,UAAY,SAASj8D,GAErC,MADAtJ,MAAKwuE,WAAaxuE,KAAKwuE,eAChBxuE,KAAKwuE,WAAWllE,QAWzByT,EAAQ5J,UAAU47D,aAAe,SAASzlE,GACxC,QAAUtJ,KAAKulE,UAAUj8D,GAAO3D,SAM9B,SAAS9F,EAAQD,EAASM,GAE9B,GAAI8uE,IAA0D,SAASC,EAAQpvE,IAM/E,SAAW2G,GA+RP,QAAS0oE,GAAI3pE,EAAGa,EAAG3F,GACf,OAAQiF,UAAUC,QACd,IAAK,GAAG,MAAY,OAALJ,EAAYA,EAAIa,CAC/B,KAAK,GAAG,MAAY,OAALb,EAAYA,EAAS,MAALa,EAAYA,EAAI3F,CAC/C,SAAS,KAAM,IAAImD,OAAM,iBAIjC,QAASurE,GAAW5pE,EAAGa,GACnB,MAAON,IAAevF,KAAKgF,EAAGa,GAGlC,QAASgpE,KAGL,OACIC,OAAQ,EACRC,gBACAC,eACA1rD,SAAW,GACX2rD,cAAgB,EAChBC,WAAY,EACZC,aAAe,KACfC,eAAgB,EAChBC,iBAAkB,EAClBC,KAAK,GAIb,QAASC,GAASC,GACVlsE,GAAOmsE,+BAAgC,GAChB,mBAAZr3C,UAA2BA,QAAQs3C,MAC9Ct3C,QAAQs3C,KAAK,wBAA0BF,GAI/C,QAASG,GAAUH,EAAK72D,GACpB,GAAIi3D,IAAY,CAChB,OAAO7qE,GAAO,WAKV,MAJI6qE,KACAL,EAASC,GACTI,GAAY,GAETj3D,EAAGnB,MAAM/X,KAAM0F,YACvBwT,GAGP,QAASk3D,GAAgBn6D,EAAM85D,GACtBM,GAAap6D,KACd65D,EAASC,GACTM,GAAap6D,IAAQ,GAI7B,QAASq6D,GAASC,EAAMv5D,GACpB,MAAO,UAAUzR,GACb,MAAOirE,GAAaD,EAAKhwE,KAAKP,KAAMuF,GAAIyR,IAGhD,QAASy5D,GAAgBF,EAAMG,GAC3B,MAAO,UAAUnrE,GACb,MAAOvF,MAAK2wE,aAAaC,QAAQL,EAAKhwE,KAAKP,KAAMuF,GAAImrE,IAI7D,QAASG,GAAUtrE,EAAGa,GAElB,GAGI0qE,GAASC,EAHTC,EAA0C,IAAvB5qE,EAAEmyB,OAAShzB,EAAEgzB,SAAiBnyB,EAAEsyB,QAAUnzB,EAAEmzB,SAE/D+hB,EAASl1C,EAAE6yB,QAAQnlB,IAAI+9D,EAAgB,SAa3C,OAViB,GAAb5qE,EAAIq0C,GACJq2B,EAAUvrE,EAAE6yB,QAAQnlB,IAAI+9D,EAAiB,EAAG,UAE5CD,GAAU3qE,EAAIq0C,IAAWA,EAASq2B,KAElCA,EAAUvrE,EAAE6yB,QAAQnlB,IAAI+9D,EAAiB,EAAG,UAE5CD,GAAU3qE,EAAIq0C,IAAWq2B,EAAUr2B,MAG9Bu2B,EAAiBD,GAc9B,QAASE,GAAgB5sC,EAAQzC,EAAMsvC,GACnC,GAAIC,EAEJ,OAAgB,OAAZD,EAEOtvC,EAEgB,MAAvByC,EAAO+sC,aACA/sC,EAAO+sC,aAAaxvC,EAAMsvC,GACX,MAAf7sC,EAAOgtC,MAEdF,EAAO9sC,EAAOgtC,KAAKH,GACfC,GAAe,GAAPvvC,IACRA,GAAQ,IAEPuvC,GAAiB,KAATvvC,IACTA,EAAO,GAEJA,GAGAA,EAQf,QAAS0vC,MAIT,QAASC,GAAOC,EAAQC,GAChBA,KAAiB,GACjBC,EAAcF,GAElBG,EAAW3xE,KAAMwxE,GACjBxxE,KAAKk4B,GAAK,GAAI5zB,OAAMktE,EAAOt5C,IAGvB05C,MAAqB,IACrBA,IAAmB,EACnB/tE,GAAOguE,aAAa7xE,MACpB4xE,IAAmB,GAK3B,QAASE,GAAShiE,GACd,GAAIiiE,GAAkBC,EAAqBliE,GACvCmiE,EAAQF,EAAgBx5C,MAAQ,EAChC25C,EAAWH,EAAgBI,SAAW,EACtCC,EAASL,EAAgBr5C,OAAS,EAClC25C,EAAQN,EAAgBO,MAAQ,EAChCC,EAAOR,EAAgB15C,KAAO,EAC9B+E,EAAQ20C,EAAgBnwC,MAAQ,EAChCvE,EAAU00C,EAAgBpwC,QAAU,EACpCrE,EAAUy0C,EAAgBrwC,QAAU,EACpCnE,EAAew0C,EAAgBtwC,aAAe,CAGlDzhC,MAAKwyE,eAAiBj1C,EACR,IAAVD,EACU,IAAVD,EACQ,KAARD,EAGJp9B,KAAKyyE,OAASF,EACF,EAARF,EAIJryE,KAAK0yE,SAAWN,EACD,EAAXF,EACQ,GAARD,EAEJjyE,KAAK4S,SAEL5S,KAAK2yE,QAAU9uE,GAAO8sE,aAEtB3wE,KAAK4yE,UAQT,QAASttE,GAAOC,EAAGa,GACf,IAAK,GAAIZ,KAAKY,GACN+oE,EAAW/oE,EAAGZ,KACdD,EAAEC,GAAKY,EAAEZ,GAYjB,OARI2pE,GAAW/oE,EAAG,cACdb,EAAEF,SAAWe,EAAEf,UAGf8pE,EAAW/oE,EAAG,aACdb,EAAEyB,QAAUZ,EAAEY,SAGXzB,EAGX,QAASosE,GAAWtoD,EAAID,GACpB,GAAI5jB,GAAGK,EAAMgtE,CAiCb,IA/BqC,mBAA1BzpD,GAAK0pD,mBACZzpD,EAAGypD,iBAAmB1pD,EAAK0pD,kBAER,mBAAZ1pD,GAAK2pD,KACZ1pD,EAAG0pD,GAAK3pD,EAAK2pD,IAEM,mBAAZ3pD,GAAK4pD,KACZ3pD,EAAG2pD,GAAK5pD,EAAK4pD,IAEM,mBAAZ5pD,GAAK6pD,KACZ5pD,EAAG4pD,GAAK7pD,EAAK6pD,IAEW,mBAAjB7pD,GAAK8pD,UACZ7pD,EAAG6pD,QAAU9pD,EAAK8pD,SAEG,mBAAd9pD,GAAK+pD,OACZ9pD,EAAG8pD,KAAO/pD,EAAK+pD,MAEQ,mBAAhB/pD,GAAKgqD,SACZ/pD,EAAG+pD,OAAShqD,EAAKgqD,QAEO,mBAAjBhqD,GAAKiqD,UACZhqD,EAAGgqD,QAAUjqD,EAAKiqD,SAEE,mBAAbjqD,GAAKkqD,MACZjqD,EAAGiqD,IAAMlqD,EAAKkqD,KAEU,mBAAjBlqD,GAAKupD,UACZtpD,EAAGspD,QAAUvpD,EAAKupD,SAGlBY,GAAiB5tE,OAAS,EAC1B,IAAKH,IAAK+tE,IACN1tE,EAAO0tE,GAAiB/tE,GACxBqtE,EAAMzpD,EAAKvjB,GACQ,mBAARgtE,KACPxpD,EAAGxjB,GAAQgtE,EAKvB,OAAOxpD,GAGX,QAASmqD,GAASC,GACd,MAAa,GAATA,EACOvuE,KAAKwyC,KAAK+7B,GAEVvuE,KAAKC,MAAMsuE,GAM1B,QAASjD,GAAaiD,EAAQC,EAAcC,GAIxC,IAHA,GAAIC,GAAS,GAAK1uE,KAAK2lB,IAAI4oD,GACvBzkD,EAAOykD,GAAU,EAEdG,EAAOjuE,OAAS+tE,GACnBE,EAAS,IAAMA,CAEnB,QAAQ5kD,EAAQ2kD,EAAY,IAAM,GAAM,KAAOC,EAGnD,QAASC,GAA0BC,EAAMluE,GACrC,GAAImuE,IAAOx2C,aAAc,EAAG60C,OAAQ,EAUpC,OARA2B,GAAI3B,OAASxsE,EAAM8yB,QAAUo7C,EAAKp7C,QACC,IAA9B9yB,EAAM2yB,OAASu7C,EAAKv7C,QACrBu7C,EAAK17C,QAAQnlB,IAAI8gE,EAAI3B,OAAQ,KAAK4B,QAAQpuE,MACxCmuE,EAAI3B,OAGV2B,EAAIx2C,cAAgB33B,GAAUkuE,EAAK17C,QAAQnlB,IAAI8gE,EAAI3B,OAAQ,KAEpD2B,EAGX,QAASE,GAAkBH,EAAMluE,GAC7B,GAAImuE,EAUJ,OATAnuE,GAAQsuE,EAAOtuE,EAAOkuE,GAClBA,EAAKK,SAASvuE,GACdmuE,EAAMF,EAA0BC,EAAMluE,IAEtCmuE,EAAMF,EAA0BjuE,EAAOkuE,GACvCC,EAAIx2C,cAAgBw2C,EAAIx2C,aACxBw2C,EAAI3B,QAAU2B,EAAI3B,QAGf2B,EAIX,QAASK,GAAYl5C,EAAWjlB,GAC5B,MAAO,UAAU48D,EAAKnC,GAClB,GAAI2D,GAAKC,CAUT,OARe,QAAX5D,GAAoBhsE,OAAOgsE,KAC3BN,EAAgBn6D,EAAM,YAAcA,EAAQ,uDAAyDA,EAAO,qBAC5Gq+D,EAAMzB,EAAKA,EAAMnC,EAAQA,EAAS4D,GAGtCzB,EAAqB,gBAARA,IAAoBA,EAAMA,EACvCwB,EAAMxwE,GAAOiM,SAAS+iE,EAAKnC,GAC3B6D,EAAgCv0E,KAAMq0E,EAAKn5C,GACpCl7B,MAIf,QAASu0E,GAAgCC,EAAK1kE,EAAU2kE,EAAU5C,GAC9D,GAAIt0C,GAAeztB,EAAS0iE,cACxBD,EAAOziE,EAAS2iE,MAChBL,EAAStiE,EAAS4iE,OACtBb,GAA+B,MAAhBA,GAAuB,EAAOA,EAEzCt0C,GACAi3C,EAAIt8C,GAAGw8C,SAASF,EAAIt8C,GAAKqF,EAAek3C,GAExClC,GACAoC,GAAUH,EAAK,OAAQI,GAAUJ,EAAK,QAAUjC,EAAOkC,GAEvDrC,GACAyC,GAAeL,EAAKI,GAAUJ,EAAK,SAAWpC,EAASqC,GAEvD5C,GACAhuE,GAAOguE,aAAa2C,EAAKjC,GAAQH,GAKzC,QAASlsE,GAAQ4uE,GACb,MAAiD,mBAA1CvuE,OAAO4M,UAAU9N,SAAS9E,KAAKu0E,GAG1C,QAASzwE,GAAOywE,GACZ,MAAiD,kBAA1CvuE,OAAO4M,UAAU9N,SAAS9E,KAAKu0E,IAClCA,YAAiBxwE,MAIzB,QAASywE,GAAc/R,EAAQC,EAAQ+R,GACnC,GAGIxvE,GAHAC,EAAMP,KAAK4G,IAAIk3D,EAAOr9D,OAAQs9D,EAAOt9D,QACrCsvE,EAAa/vE,KAAK2lB,IAAIm4C,EAAOr9D,OAASs9D,EAAOt9D,QAC7CuvE,EAAQ,CAEZ,KAAK1vE,EAAI,EAAOC,EAAJD,EAASA,KACZwvE,GAAehS,EAAOx9D,KAAOy9D,EAAOz9D,KACnCwvE,GAAeG,EAAMnS,EAAOx9D,MAAQ2vE,EAAMlS,EAAOz9D,MACnD0vE,GAGR,OAAOA,GAAQD,EAGnB,QAASG,GAAeC,GACpB,GAAIA,EAAO,CACP,GAAIC,GAAUD,EAAM9wC,cAAc/5B,QAAQ,QAAS,KACnD6qE,GAAQE,GAAYF,IAAUG,GAAeF,IAAYA,EAE7D,MAAOD,GAGX,QAASrD,GAAqByD,GAC1B,GACIC,GACA7vE,EAFAksE,IAIJ,KAAKlsE,IAAQ4vE,GACLtG,EAAWsG,EAAa5vE,KACxB6vE,EAAiBN,EAAevvE,GAC5B6vE,IACA3D,EAAgB2D,GAAkBD,EAAY5vE,IAK1D,OAAOksE,GAGX,QAAS4D,GAAS7mE,GACd,GAAIkI,GAAO4+D,CAEX,IAA8B,IAA1B9mE,EAAMnI,QAAQ,QACdqQ,EAAQ,EACR4+D,EAAS,UAER,CAAA,GAA+B,IAA3B9mE,EAAMnI,QAAQ,SAKnB,MAJAqQ,GAAQ,GACR4+D,EAAS,QAMb/xE,GAAOiL,GAAS,SAAUwyB,EAAQn5B,GAC9B,GAAI3C,GAAGqwE,EACH78D,EAASnV,GAAO8uE,QAAQ7jE,GACxBgnE,IAYJ,IAVsB,gBAAXx0C,KACPn5B,EAAQm5B,EACRA,EAAS96B,GAGbqvE,EAAS,SAAUrwE,GACf,GAAIhF,GAAIqD,KAASkyE,MAAM5sB,IAAIysB,EAAQpwE,EACnC,OAAOwT,GAAOzY,KAAKsD,GAAO8uE,QAASnyE,EAAG8gC,GAAU,KAGvC,MAATn5B,EACA,MAAO0tE,GAAO1tE,EAGd,KAAK3C,EAAI,EAAOwR,EAAJxR,EAAWA,IACnBswE,EAAQ9tE,KAAK6tE,EAAOrwE,GAExB,OAAOswE,IAKnB,QAASX,GAAMa,GACX,GAAIC,IAAiBD,EACjB3uE,EAAQ,CAUZ,OARsB,KAAlB4uE,GAAuBC,SAASD,KAE5B5uE,EADA4uE,GAAiB,EACT/wE,KAAKC,MAAM8wE,GAEX/wE,KAAKwyC,KAAKu+B,IAInB5uE,EAGX,QAAS8uE,GAAY59C,EAAMG,GACvB,MAAO,IAAIp0B,MAAKA,KAAK8xE,IAAI79C,EAAMG,EAAQ,EAAG,IAAI29C,aAGlD,QAASC,GAAY/9C,EAAMg+C,EAAKC,GAC5B,MAAOC,IAAW5yE,IAAQ00B,EAAM,GAAI,GAAKg+C,EAAMC,IAAOD,EAAKC,GAAKlE,KAGpE,QAASoE,GAAWn+C,GAChB,MAAOo+C,GAAWp+C,GAAQ,IAAM,IAGpC,QAASo+C,GAAWp+C,GAChB,MAAQA,GAAO,IAAM,GAAKA,EAAO,MAAQ,GAAMA,EAAO,MAAQ,EAGlE,QAASm5C,GAAclxE,GACnB,GAAIqjB,EACArjB,GAAEo2E,IAAyB,KAAnBp2E,EAAE8yE,IAAIzvD,WACdA,EACIrjB,EAAEo2E,GAAGC,IAAS,GAAKr2E,EAAEo2E,GAAGC,IAAS,GAAKA,GACtCr2E,EAAEo2E,GAAGE,IAAQ,GAAKt2E,EAAEo2E,GAAGE,IAAQX,EAAY31E,EAAEo2E,GAAGG,IAAOv2E,EAAEo2E,GAAGC,KAAUC,GACtEt2E,EAAEo2E,GAAGI,IAAQ,GAAKx2E,EAAEo2E,GAAGI,IAAQ,IACX,KAAfx2E,EAAEo2E,GAAGI,MAAkC,IAAjBx2E,EAAEo2E,GAAGK,KACY,IAAjBz2E,EAAEo2E,GAAGM,KACiB,IAAtB12E,EAAEo2E,GAAGO,KAAuBH,GACvDx2E,EAAEo2E,GAAGK,IAAU,GAAKz2E,EAAEo2E,GAAGK,IAAU,GAAKA,GACxCz2E,EAAEo2E,GAAGM,IAAU,GAAK12E,EAAEo2E,GAAGM,IAAU,GAAKA,GACxC12E,EAAEo2E,GAAGO,IAAe,GAAK32E,EAAEo2E,GAAGO,IAAe,IAAMA,GACnD,GAEA32E,EAAE8yE,IAAI8D,qBAAkCL,GAAXlzD,GAAmBA,EAAWizD,MAC3DjzD,EAAWizD,IAGft2E,EAAE8yE,IAAIzvD,SAAWA,GAIzB,QAASwzD,GAAQ72E,GAiBb,MAhBkB,OAAdA,EAAE82E,WACF92E,EAAE82E,UAAY5yE,MAAMlE,EAAE03B,GAAGq/C,YACrB/2E,EAAE8yE,IAAIzvD,SAAW,IAChBrjB,EAAE8yE,IAAIjE,QACN7uE,EAAE8yE,IAAI5D,eACNlvE,EAAE8yE,IAAI7D,YACNjvE,EAAE8yE,IAAI3D,gBACNnvE,EAAE8yE,IAAI1D,gBAEPpvE,EAAE0yE,UACF1yE,EAAE82E,SAAW92E,EAAE82E,UACa,IAAxB92E,EAAE8yE,IAAI9D,eACwB,IAA9BhvE,EAAE8yE,IAAIhE,aAAa3pE,QACnBnF,EAAE8yE,IAAIkE,UAAYhxE,IAGvBhG,EAAE82E,SAGb,QAASG,GAAgB/uE,GACrB,MAAOA,GAAMA,EAAI67B,cAAc/5B,QAAQ,IAAK,KAAO9B,EAMvD,QAASgvE,GAAaC,GAGlB,IAFA,GAAW/rD,GAAGvD,EAAMgc,EAAQt8B,EAAxBvC,EAAI,EAEDA,EAAImyE,EAAMhyE,QAAQ,CAKrB,IAJAoC,EAAQ0vE,EAAgBE,EAAMnyE,IAAIuC,MAAM,KACxC6jB,EAAI7jB,EAAMpC,OACV0iB,EAAOovD,EAAgBE,EAAMnyE,EAAI,IACjC6iB,EAAOA,EAAOA,EAAKtgB,MAAM,KAAO,KACzB6jB,EAAI,GAAG,CAEV,GADAyY,EAASuzC,EAAW7vE,EAAMkD,MAAM,EAAG2gB,GAAG3jB,KAAK,MAEvC,MAAOo8B,EAEX,IAAIhc,GAAQA,EAAK1iB,QAAUimB,GAAKmpD,EAAchtE,EAAOsgB,GAAM,IAASuD,EAAI,EAEpE,KAEJA,KAEJpmB,IAEJ,MAAO,MAGX,QAASoyE,GAAW3hE,GAChB,GAAI4hE,GAAY,IAChB,KAAKhzC,GAAQ5uB,IAAS6hE,GAClB,IACID,EAAYh0E,GAAOwgC,UACjB,WAAkC,GAAIvN,GAAI,GAAIlzB,OAAM,gCAAiE,MAA7BkzB,GAAEihD,KAAO,mBAA0BjhD,KAE7HjzB,GAAOwgC,OAAOwzC,GAChB,MAAO/gD,IAEb,MAAO+N,IAAQ5uB,GAKnB,QAASi+D,GAAOY,EAAOkD,GACnB,GAAIjE,GAAK1nD,CACT,OAAI2rD,GAAM5E,QACNW,EAAMiE,EAAM5/C,QACZ/L,GAAQxoB,GAAOoD,SAAS6tE,IAAUzwE,EAAOywE,IAChCA,GAASjxE,GAAOixE,KAAYf,EAErCA,EAAI77C,GAAGw8C,SAASX,EAAI77C,GAAK7L,GACzBxoB,GAAOguE,aAAakC,GAAK,GAClBA,GAEAlwE,GAAOixE,GAAOmD,QA6N7B,QAASC,GAAuBpD,GAC5B,MAAIA,GAAMvwE,MAAM,YACLuwE,EAAMtqE,QAAQ,WAAY,IAE9BsqE,EAAMtqE,QAAQ,MAAO,IAGhC,QAAS2tE,GAAmB72C,GACxB,GAA4C97B,GAAGG,EAA3C6C,EAAQ84B,EAAO/8B,MAAM6zE,GAEzB,KAAK5yE,EAAI,EAAGG,EAAS6C,EAAM7C,OAAYA,EAAJH,EAAYA,IAEvCgD,EAAMhD,GADN6yE,GAAqB7vE,EAAMhD,IAChB6yE,GAAqB7vE,EAAMhD,IAE3B0yE,EAAuB1vE,EAAMhD,GAIhD,OAAO,UAAUgvE,GACb,GAAIZ,GAAS,EACb,KAAKpuE,EAAI,EAAOG,EAAJH,EAAYA,IACpBouE,GAAUprE,EAAMhD,YAAcwrC,UAAWxoC,EAAMhD,GAAGjF,KAAKi0E,EAAKlzC,GAAU94B,EAAMhD,EAEhF,OAAOouE,IAKf,QAAS0E,GAAa93E,EAAG8gC,GACrB,MAAK9gC,GAAE62E,WAIP/1C,EAASi3C,EAAaj3C,EAAQ9gC,EAAEmwE,cAE3B6H,GAAgBl3C,KACjBk3C,GAAgBl3C,GAAU62C,EAAmB72C,IAG1Ck3C,GAAgBl3C,GAAQ9gC,IATpBA,EAAEmwE,aAAa8H,cAY9B,QAASF,GAAaj3C,EAAQ+C,GAG1B,QAASq0C,GAA4B5D,GACjC,MAAOzwC,GAAOs0C,eAAe7D,IAAUA,EAH3C,GAAItvE,GAAI,CAOR,KADAozE,GAAsBC,UAAY,EAC3BrzE,GAAK,GAAKozE,GAAsB5qE,KAAKszB,IACxCA,EAASA,EAAO92B,QAAQouE,GAAuBF,GAC/CE,GAAsBC,UAAY,EAClCrzE,GAAK,CAGT,OAAO87B,GAUX,QAASw3C,GAAsBpX,EAAO8P,GAClC,GAAIjsE,GAAG48D,EAASqP,EAAO0B,OACvB,QAAQxR,GACR,IAAK,IACD,MAAOqX,GACX,KAAK,OACD,MAAOC,GACX,KAAK,OACL,IAAK,OACL,IAAK,OACD,MAAO7W,GAAS8W,GAAuBC,EAC3C,KAAK,IACL,IAAK,IACL,IAAK,IACD,MAAOC,GACX,KAAK,SACL,IAAK,QACL,IAAK,QACL,IAAK,QACD,MAAOhX,GAASiX,GAAsBC,EAC1C,KAAK,IACD,GAAIlX,EACA,MAAO4W,GAGf,KAAK,KACD,GAAI5W,EACA,MAAOmX,GAGf,KAAK,MACD,GAAInX,EACA,MAAO6W,GAGf,KAAK,MACD,MAAOO,GACX,KAAK,MACL,IAAK,OACL,IAAK,KACL,IAAK,MACL,IAAK,OACD,MAAOC,GACX,KAAK,IACL,IAAK,IACD,MAAOhI,GAAOmB,QAAQ8G,cAC1B,KAAK,IACD,MAAOC,GACX,KAAK,IACD,MAAOC,GACX,KAAK,IACL,IAAK,KACD,MAAOC,GACX,KAAK,IACD,MAAOC,GACX,KAAK,OACD,MAAOC,GACX,KAAK,KACL,IAAK,KACL,IAAK,KACL,IAAK,KACL,IAAK,KACL,IAAK,KACL,IAAK,KACL,IAAK,KACL,IAAK,KACL,IAAK,KACL,IAAK,KACD,MAAO3X,GAASmX,GAAsBS,EAC1C,KAAK,IACL,IAAK,IACL,IAAK,IACL,IAAK,IACL,IAAK,IACL,IAAK,IACL,IAAK,IACL,IAAK,IACL,IAAK,IACL,IAAK,IACL,IAAK,IACD,MAAOA,GACX,KAAK,KACD,MAAO5X,GAASqP,EAAOmB,QAAQqH,cAAgBxI,EAAOmB,QAAQsH,oBAClE,SAEI,MADA10E,GAAI,GAAI20E,QAAOC,GAAaC,GAAe1Y,EAAMl3D,QAAQ,KAAM,KAAM,OAK7E,QAAS6vE,GAAoBC,GACzBA,EAASA,GAAU,EACnB,IAAIC,GAAqBD,EAAO/1E,MAAMq1E,QAClCY,EAAUD,EAAkBA,EAAkB50E,OAAS,OACvDwH,GAASqtE,EAAU,IAAIj2E,MAAMk2E,MAA0B,IAAK,EAAG,GAC/Dp9C,IAAuB,GAAXlwB,EAAM,IAAWgoE,EAAMhoE,EAAM,GAE7C,OAAoB,MAAbA,EAAM,GAAakwB,GAAWA,EAIzC,QAASq9C,GAAwBhZ,EAAOoT,EAAOtD,GAC3C,GAAIjsE,GAAGo1E,EAAgBnJ,EAAOoF,EAE9B,QAAQlV,GAER,IAAK,IACY,MAAToT,IACA6F,EAAc9D,IAA8B,GAApB1B,EAAML,GAAS,GAE3C,MAEJ,KAAK,IACL,IAAK,KACY,MAATA,IACA6F,EAAc9D,IAAS1B,EAAML,GAAS,EAE1C,MACJ,KAAK,MACL,IAAK,OACDvvE,EAAIisE,EAAOmB,QAAQiI,YAAY9F,EAAOpT,EAAO8P,EAAO0B,SAE3C,MAAL3tE,EACAo1E,EAAc9D,IAAStxE,EAEvBisE,EAAO8B,IAAI5D,aAAeoF,CAE9B,MAEJ,KAAK,IACL,IAAK,KACY,MAATA,IACA6F,EAAc7D,IAAQ3B,EAAML,GAEhC,MACJ,KAAK,KACY,MAATA,IACA6F,EAAc7D,IAAQ3B,EAAMvqE,SAChBkqE,EAAMvwE,MAAM,WAAW,GAAI,KAE3C,MAEJ,KAAK,MACL,IAAK,OACY,MAATuwE,IACAtD,EAAOqJ,WAAa1F,EAAML,GAG9B,MAEJ,KAAK,KACD6F,EAAc5D,IAAQlzE,GAAOi3E,kBAAkBhG,EAC/C,MACJ,KAAK,OACL,IAAK,QACL,IAAK,SACD6F,EAAc5D,IAAQ5B,EAAML,EAC5B,MAEJ,KAAK,IACL,IAAK,IACDtD,EAAOuJ,UAAYjG,CAEnB,MAEJ,KAAK,IACL,IAAK,KACDtD,EAAO8B,IAAIkE,SAAU,CAEzB,KAAK,IACL,IAAK,KACDmD,EAAc3D,IAAQ7B,EAAML,EAC5B,MAEJ,KAAK,IACL,IAAK,KACD6F,EAAc1D,IAAU9B,EAAML,EAC9B,MAEJ,KAAK,IACL,IAAK,KACD6F,EAAczD,IAAU/B,EAAML,EAC9B,MAEJ,KAAK,IACL,IAAK,KACL,IAAK,MACL,IAAK,OACD6F,EAAcxD,IAAehC,EAAuB,KAAhB,KAAOL,GAC3C,MAEJ,KAAK,IACDtD,EAAOt5C,GAAK,GAAI5zB,MAAK6wE,EAAML,GAC3B,MAEJ,KAAK,IACDtD,EAAOt5C,GAAK,GAAI5zB,MAAyB,IAApB+gB,WAAWyvD,GAChC,MAEJ,KAAK,IACL,IAAK,KACDtD,EAAOwJ,SAAU,EACjBxJ,EAAO2B,KAAOkH,EAAoBvF,EAClC,MAEJ,KAAK,KACL,IAAK,MACL,IAAK,OACDvvE,EAAIisE,EAAOmB,QAAQsI,cAAcnG,GAExB,MAALvvE,GACAisE,EAAO0J,GAAK1J,EAAO0J,OACnB1J,EAAO0J,GAAM,EAAI31E,GAEjBisE,EAAO8B,IAAI6H,eAAiBrG,CAEhC,MAEJ,KAAK,IACL,IAAK,KACL,IAAK,IACL,IAAK,KACL,IAAK,IACL,IAAK,IACL,IAAK,IACDpT,EAAQA,EAAMp2D,OAAO,EAAG,EAE5B,KAAK,OACL,IAAK,OACL,IAAK,QACDo2D,EAAQA,EAAMp2D,OAAO,EAAG,GACpBwpE,IACAtD,EAAO0J,GAAK1J,EAAO0J,OACnB1J,EAAO0J,GAAGxZ,GAASyT,EAAML,GAE7B,MACJ,KAAK,KACL,IAAK,KACDtD,EAAO0J,GAAK1J,EAAO0J,OACnB1J,EAAO0J,GAAGxZ,GAAS79D,GAAOi3E,kBAAkBhG,IAIpD,QAASsG,GAAsB5J,GAC3B,GAAIniB,GAAGgsB,EAAU/I,EAAMzwC,EAAS00C,EAAKC,EAAK8E,CAE1CjsB,GAAImiB,EAAO0J,GACC,MAAR7rB,EAAEksB,IAAqB,MAAPlsB,EAAEmsB,GAAoB,MAAPnsB,EAAEosB,GACjClF,EAAM,EACNC,EAAM,EAMN6E,EAAWnM,EAAI7f,EAAEksB,GAAI/J,EAAOoF,GAAGG,IAAON,GAAW5yE,KAAU,EAAG,GAAG00B,MACjE+5C,EAAOpD,EAAI7f,EAAEmsB,EAAG,GAChB35C,EAAUqtC,EAAI7f,EAAEosB,EAAG,KAEnBlF,EAAM/E,EAAOmB,QAAQ+I,MAAMnF,IAC3BC,EAAMhF,EAAOmB,QAAQ+I,MAAMlF,IAE3B6E,EAAWnM,EAAI7f,EAAEssB,GAAInK,EAAOoF,GAAGG,IAAON,GAAW5yE,KAAU0yE,EAAKC,GAAKj+C,MACrE+5C,EAAOpD,EAAI7f,EAAEA,EAAG,GAEL,MAAPA,EAAE1iD,GAEFk1B,EAAUwtB,EAAE1iD,EACE4pE,EAAV10C,KACEywC,GAINzwC,EAFc,MAAPwtB,EAAEv4B,EAECu4B,EAAEv4B,EAAIy/C,EAGNA,GAGlB+E,EAAOM,GAAmBP,EAAU/I,EAAMzwC,EAAS20C,EAAKD,GAExD/E,EAAOoF,GAAGG,IAAQuE,EAAK/iD,KACvBi5C,EAAOqJ,WAAaS,EAAKhjD,UAO7B,QAASujD,GAAerK,GACpB,GAAIhsE,GAAGizB,EAAkBqjD,EAAaC,EAAzBjH,IAEb,KAAItD,EAAOt5C,GAAX,CA6BA,IAzBA4jD,EAAcE,GAAiBxK,GAG3BA,EAAO0J,IAAyB,MAAnB1J,EAAOoF,GAAGE,KAAqC,MAApBtF,EAAOoF,GAAGC,KAClDuE,EAAsB5J,GAItBA,EAAOqJ,aACPkB,EAAY7M,EAAIsC,EAAOoF,GAAGG,IAAO+E,EAAY/E,KAEzCvF,EAAOqJ,WAAanE,EAAWqF,KAC/BvK,EAAO8B,IAAI8D,oBAAqB,GAGpC3+C,EAAOwjD,GAAYF,EAAW,EAAGvK,EAAOqJ,YACxCrJ,EAAOoF,GAAGC,IAASp+C,EAAKyjD,cACxB1K,EAAOoF,GAAGE,IAAQr+C,EAAK49C,cAQtB7wE,EAAI,EAAO,EAAJA,GAAyB,MAAhBgsE,EAAOoF,GAAGpxE,KAAcA,EACzCgsE,EAAOoF,GAAGpxE,GAAKsvE,EAAMtvE,GAAKs2E,EAAYt2E,EAI1C,MAAW,EAAJA,EAAOA,IACVgsE,EAAOoF,GAAGpxE,GAAKsvE,EAAMtvE,GAAsB,MAAhBgsE,EAAOoF,GAAGpxE,GAAqB,IAANA,EAAU,EAAI,EAAKgsE,EAAOoF,GAAGpxE,EAI7D,MAApBgsE,EAAOoF,GAAGI,KACgB,IAAtBxF,EAAOoF,GAAGK,KACY,IAAtBzF,EAAOoF,GAAGM,KACiB,IAA3B1F,EAAOoF,GAAGO,MACd3F,EAAO2K,UAAW,EAClB3K,EAAOoF,GAAGI,IAAQ,GAGtBxF,EAAOt5C,IAAMs5C,EAAOwJ,QAAUiB,GAAcG,IAAUrkE,MAAM,KAAM+8D,GAG/C,MAAftD,EAAO2B,MACP3B,EAAOt5C,GAAGmkD,cAAc7K,EAAOt5C,GAAGokD,gBAAkB9K,EAAO2B,MAG3D3B,EAAO2K,WACP3K,EAAOoF,GAAGI,IAAQ,KAI1B,QAASuF,GAAe/K,GACpB,GAAIO,EAEAP,GAAOt5C,KAIX65C,EAAkBC,EAAqBR,EAAOuB,IAC9CvB,EAAOoF,IACH7E,EAAgBx5C,KAChBw5C,EAAgBr5C,MAChBq5C,EAAgB15C,KAAO05C,EAAgBt5C,KACvCs5C,EAAgBnwC,KAChBmwC,EAAgBpwC,OAChBowC,EAAgBrwC,OAChBqwC,EAAgBtwC,aAGpBo6C,EAAerK,IAGnB,QAASwK,IAAiBxK,GACtB,GAAIr0C,GAAM,GAAI74B,KACd,OAAIktE,GAAOwJ,SAEH79C,EAAIq/C,iBACJr/C,EAAI++C,cACJ/+C,EAAIk5C,eAGAl5C,EAAIgF,cAAehF,EAAI4F,WAAY5F,EAAI2F,WAKvD,QAAS25C,IAA4BjL,GACjC,GAAIA,EAAOwB,KAAOnvE,GAAO64E,SAErB,WADAC,IAASnL,EAIbA,GAAOoF,MACPpF,EAAO8B,IAAIjE,OAAQ,CAGnB,IACI7pE,GAAGo3E,EAAaC,EAAQnb,EAAOob,EAD/BxC,EAAS,GAAK9I,EAAOuB,GAErBgK,EAAezC,EAAO30E,OACtBq3E,EAAyB,CAI7B,KAFAH,EAAStE,EAAa/G,EAAOwB,GAAIxB,EAAOmB,SAASpuE,MAAM6zE,QAElD5yE,EAAI,EAAGA,EAAIq3E,EAAOl3E,OAAQH,IAC3Bk8D,EAAQmb,EAAOr3E,GACfo3E,GAAetC,EAAO/1E,MAAMu0E,EAAsBpX,EAAO8P,SAAgB,GACrEoL,IACAE,EAAUxC,EAAOhvE,OAAO,EAAGgvE,EAAO3zE,QAAQi2E,IACtCE,EAAQn3E,OAAS,GACjB6rE,EAAO8B,IAAI/D,YAAYvnE,KAAK80E,GAEhCxC,EAASA,EAAOrvE,MAAMqvE,EAAO3zE,QAAQi2E,GAAeA,EAAYj3E,QAChEq3E,GAA0BJ,EAAYj3E,QAGtC0yE,GAAqB3W,IACjBkb,EACApL,EAAO8B,IAAIjE,OAAQ,EAGnBmC,EAAO8B,IAAIhE,aAAatnE,KAAK05D,GAEjCgZ,EAAwBhZ,EAAOkb,EAAapL,IAEvCA,EAAO0B,UAAY0J,GACxBpL,EAAO8B,IAAIhE,aAAatnE,KAAK05D,EAKrC8P,GAAO8B,IAAI9D,cAAgBuN,EAAeC,EACtC1C,EAAO30E,OAAS,GAChB6rE,EAAO8B,IAAI/D,YAAYvnE,KAAKsyE,GAI5B9I,EAAO8B,IAAIkE,WAAY,GAAQhG,EAAOoF,GAAGI,KAAS,KAClDxF,EAAO8B,IAAIkE,QAAUhxE,GAGzBgrE,EAAOoF,GAAGI,IAAQ/F,EAAgBO,EAAOmB,QAASnB,EAAOoF,GAAGI,IACpDxF,EAAOuJ,WACfc,EAAerK,GACfE,EAAcF,GAGlB,QAAS4I,IAAexuE,GACpB,MAAOA,GAAEpB,QAAQ,sCAAuC,SAAUyyE,EAAS7S,EAAIC,EAAIC,EAAI4S,GACnF,MAAO9S,IAAMC,GAAMC,GAAM4S,IAKjC,QAAS/C,IAAavuE,GAClB,MAAOA,GAAEpB,QAAQ,yBAA0B,QAI/C,QAAS2yE,IAA2B3L,GAChC,GAAI4L,GACAC,EAEAC,EACA93E,EACA+3E,CAEJ,IAAyB,IAArB/L,EAAOwB,GAAGrtE,OAGV,MAFA6rE,GAAO8B,IAAI3D,eAAgB,OAC3B6B,EAAOt5C,GAAK,GAAI5zB,MAAKk5E,KAIzB,KAAKh4E,EAAI,EAAGA,EAAIgsE,EAAOwB,GAAGrtE,OAAQH,IAC9B+3E,EAAe,EACfH,EAAazL,KAAeH,GACN,MAAlBA,EAAOwJ,UACPoC,EAAWpC,QAAUxJ,EAAOwJ,SAEhCoC,EAAW9J,IAAMlE,IACjBgO,EAAWpK,GAAKxB,EAAOwB,GAAGxtE,GAC1Bi3E,GAA4BW,GAEvB/F,EAAQ+F,KAKbG,GAAgBH,EAAW9J,IAAI9D,cAG/B+N,GAAqD,GAArCH,EAAW9J,IAAIhE,aAAa3pE,OAE5Cy3E,EAAW9J,IAAImK,MAAQF,GAEJ,MAAfD,GAAsCA,EAAfC,KACvBD,EAAcC,EACdF,EAAaD,GAIrB93E,GAAOksE,EAAQ6L,GAAcD,GAIjC,QAAST,IAASnL,GACd,GAAIhsE,GAAGk4E,EACHpD,EAAS9I,EAAOuB,GAChBxuE,EAAQo5E,GAASl5E,KAAK61E,EAE1B,IAAI/1E,EAAO,CAEP,IADAitE,EAAO8B,IAAIzD,KAAM,EACZrqE,EAAI,EAAGk4E,EAAIE,GAASj4E,OAAY+3E,EAAJl4E,EAAOA,IACpC,GAAIo4E,GAASp4E,GAAG,GAAGf,KAAK61E,GAAS,CAE7B9I,EAAOwB,GAAK4K,GAASp4E,GAAG,IAAMjB,EAAM,IAAM,IAC1C,OAGR,IAAKiB,EAAI,EAAGk4E,EAAIG,GAASl4E,OAAY+3E,EAAJl4E,EAAOA,IACpC,GAAIq4E,GAASr4E,GAAG,GAAGf,KAAK61E,GAAS,CAC7B9I,EAAOwB,IAAM6K,GAASr4E,GAAG,EACzB,OAGJ80E,EAAO/1E,MAAMq1E,MACbpI,EAAOwB,IAAM,KAEjByJ,GAA4BjL,OAE5BA,GAAO8F,UAAW,EAK1B,QAASwG,IAAmBtM,GACxBmL,GAASnL,GACLA,EAAO8F,YAAa,UACb9F,GAAO8F,SACdzzE,GAAOk6E,wBAAwBvM,IAIvC,QAASnkE,IAAI+sC,EAAKlhC,GACd,GAAc1T,GAAVuuE,IACJ,KAAKvuE,EAAI,EAAGA,EAAI40C,EAAIz0C,SAAUH,EAC1BuuE,EAAI/rE,KAAKkR,EAAGkhC,EAAI50C,GAAIA,GAExB,OAAOuuE,GAGX,QAASiK,IAAkBxM,GACvB,GAAuByL,GAAnBnI,EAAQtD,EAAOuB,EACf+B,KAAUtuE,EACVgrE,EAAOt5C,GAAK,GAAI5zB,MACTD,EAAOywE,GACdtD,EAAOt5C,GAAK,GAAI5zB,OAAMwwE,GAC6B,QAA3CmI,EAAUgB,GAAgBx5E,KAAKqwE,IACvCtD,EAAOt5C,GAAK,GAAI5zB,OAAM24E,EAAQ,IACN,gBAAVnI,GACdgJ,GAAmBtM,GACZtrE,EAAQ4uE,IACftD,EAAOoF,GAAKvpE,GAAIynE,EAAM7pE,MAAM,GAAI,SAAU8X,GACtC,MAAOnY,UAASmY,EAAK,MAEzB84D,EAAerK,IACU,gBAAZ,GACb+K,EAAe/K,GACU,gBAAZ,GAEbA,EAAOt5C,GAAK,GAAI5zB,MAAKwwE,GAErBjxE,GAAOk6E,wBAAwBvM,GAIvC,QAAS4K,IAASpqE,EAAGxR,EAAGmM,EAAGhB,EAAGo/D,EAAGn/D,EAAGsyE,GAGhC,GAAIzlD,GAAO,GAAIn0B,MAAK0N,EAAGxR,EAAGmM,EAAGhB,EAAGo/D,EAAGn/D,EAAGsyE,EAMtC,OAHQ,MAAJlsE,GACAymB,EAAKyJ,YAAYlwB,GAEdymB,EAGX,QAASwjD,IAAYjqE,GACjB,GAAIymB,GAAO,GAAIn0B,MAAKA,KAAK8xE,IAAIr+D,MAAM,KAAMrS,WAIzC,OAHQ,MAAJsM,GACAymB,EAAK0lD,eAAensE,GAEjBymB,EAGX,QAAS2lD,IAAatJ,EAAOzwC,GACzB,GAAqB,gBAAVywC,GACP,GAAKpwE,MAAMowE,IAKP,GADAA,EAAQzwC,EAAO42C,cAAcnG,GACR,gBAAVA,GACP,MAAO,UALXA,GAAQlqE,SAASkqE,EAAO,GAShC,OAAOA,GASX,QAASuJ,IAAkB/D,EAAQ7G,EAAQ6K,EAAeC,EAAUl6C,GAChE,MAAOA,GAAOm6C,aAAa/K,GAAU,IAAK6K,EAAehE,EAAQiE,GAGrE,QAASC,IAAaC,EAAgBH,EAAej6C,GACjD,GAAIv0B,GAAWjM,GAAOiM,SAAS2uE,GAAgB5zD,MAC3CyS,EAAU5P,GAAM5d,EAASmf,GAAG,MAC5BoO,EAAU3P,GAAM5d,EAASmf,GAAG,MAC5BmO,EAAQ1P,GAAM5d,EAASmf,GAAG,MAC1BsjD,EAAO7kD,GAAM5d,EAASmf,GAAG,MACzBmjD,EAAS1kD,GAAM5d,EAASmf,GAAG,MAC3BgjD,EAAQvkD,GAAM5d,EAASmf,GAAG,MAE1BhW,EAAOqkB,EAAUohD,GAAuB9yE,IAAM,IAAK0xB,IACnC,IAAZD,IAAkB,MAClBA,EAAUqhD,GAAuBl+E,IAAM,KAAM68B,IACnC,IAAVD,IAAgB,MAChBA,EAAQshD,GAAuB/yE,IAAM,KAAMyxB,IAClC,IAATm1C,IAAe,MACfA,EAAOmM,GAAuB/xE,IAAM,KAAM4lE,IAC/B,IAAXH,IAAiB,MACjBA,EAASsM,GAAuB3T,IAAM,KAAMqH,IAClC,IAAVH,IAAgB,OAAS,KAAMA,EAKvC,OAHAh5D,GAAK,GAAKqlE,EACVrlE,EAAK,IAAMwlE,EAAiB,EAC5BxlE,EAAK,GAAKorB,EACHg6C,GAAkBtmE,SAAUkB,GAgBvC,QAASw9D,IAAWjC,EAAKmK,EAAgBC,GACrC,GAEIC,GAFAhvE,EAAM+uE,EAAuBD,EAC7BG,EAAkBF,EAAuBpK,EAAIn8C,KAajD,OATIymD,GAAkBjvE,IAClBivE,GAAmB,GAGDjvE,EAAM,EAAxBivE,IACAA,GAAmB,GAGvBD,EAAiBh7E,GAAO2wE,GAAKvhE,IAAI6rE,EAAiB,MAE9CxM,KAAMptE,KAAKwyC,KAAKmnC,EAAevmD,YAAc,GAC7CC,KAAMsmD,EAAetmD,QAK7B,QAASqjD,IAAmBrjD,EAAM+5C,EAAMzwC,EAAS+8C,EAAsBD,GACnE,GAA6CI,GAAWzmD,EAApD3rB,EAAIsvE,GAAY1jD,EAAM,EAAG,GAAGymD,WAOhC,OALAryE,GAAU,IAANA,EAAU,EAAIA,EAClBk1B,EAAqB,MAAXA,EAAkBA,EAAU88C,EACtCI,EAAYJ,EAAiBhyE,GAAKA,EAAIiyE,EAAuB,EAAI,IAAUD,EAAJhyE,EAAqB,EAAI,GAChG2rB,EAAY,GAAKg6C,EAAO,IAAMzwC,EAAU88C,GAAkBI,EAAY,GAGlExmD,KAAMD,EAAY,EAAIC,EAAOA,EAAO,EACpCD,UAAWA,EAAY,EAAKA,EAAYo+C,EAAWn+C,EAAO,GAAKD,GAQvE,QAAS2mD,IAAWzN,GAChB,GAEIuC,GAFAe,EAAQtD,EAAOuB,GACfzxC,EAASkwC,EAAOwB,EAKpB,OAFAxB,GAAOmB,QAAUnB,EAAOmB,SAAW9uE,GAAO8sE,WAAWa,EAAOyB,IAE9C,OAAV6B,GAAmBxzC,IAAW96B,GAAuB,KAAVsuE,EACpCjxE,GAAOq7E,SAASzP,WAAW,KAGjB,gBAAVqF,KACPtD,EAAOuB,GAAK+B,EAAQtD,EAAOmB,QAAQwM,SAASrK,IAG5CjxE,GAAOoD,SAAS6tE,GACT,GAAIvD,GAAOuD,GAAO,IAClBxzC,EACHp7B,EAAQo7B,GACR67C,GAA2B3L,GAE3BiL,GAA4BjL,GAGhCwM,GAAkBxM,GAGtBuC,EAAM,GAAIxC,GAAOC,GACbuC,EAAIoI,WAEJpI,EAAI9gE,IAAI,EAAG,KACX8gE,EAAIoI,SAAW31E,GAGZutE,IAyCX,QAASqL,IAAOlmE,EAAImmE,GAChB,GAAItL,GAAKvuE,CAIT,IAHuB,IAAnB65E,EAAQ15E,QAAgBO,EAAQm5E,EAAQ,MACxCA,EAAUA,EAAQ,KAEjBA,EAAQ15E,OACT,MAAO9B,KAGX,KADAkwE,EAAMsL,EAAQ,GACT75E,EAAI,EAAGA,EAAI65E,EAAQ15E,SAAUH,EAC1B65E,EAAQ75E,GAAG0T,GAAI66D,KACfA,EAAMsL,EAAQ75E,GAGtB,OAAOuuE,GAsvBX,QAASc,IAAeL,EAAKntE,GACzB,GAAIi4E,EAGJ,OAAqB,gBAAVj4E,KACPA,EAAQmtE,EAAI7D,aAAaiK,YAAYvzE,GAEhB,gBAAVA,IACAmtE,GAIf8K,EAAap6E,KAAK4G,IAAI0oE,EAAI/7C,OAClB09C,EAAY3B,EAAIj8C,OAAQlxB,IAChCmtE,EAAIt8C,GAAG,OAASs8C,EAAIpB,OAAS,MAAQ,IAAM,SAAS/rE,EAAOi4E,GACpD9K,GAGX,QAASI,IAAUJ,EAAK+K,GACpB,MAAO/K,GAAIt8C,GAAG,OAASs8C,EAAIpB,OAAS,MAAQ,IAAMmM,KAGtD,QAAS5K,IAAUH,EAAK+K,EAAMl4E,GAC1B,MAAa,UAATk4E,EACO1K,GAAeL,EAAKntE,GAEpBmtE,EAAIt8C,GAAG,OAASs8C,EAAIpB,OAAS,MAAQ,IAAMmM,GAAMl4E,GAIhE,QAASm4E,IAAaD,EAAME,GACxB,MAAO,UAAUp4E,GACb,MAAa,OAATA,GACAstE,GAAU30E,KAAMu/E,EAAMl4E,GACtBxD,GAAOguE,aAAa7xE,KAAMy/E,GACnBz/E,MAEA40E,GAAU50E,KAAMu/E,IAqCnC,QAASG,IAAanN,GAElB,MAAc,KAAPA,EAAa,OAGxB,QAASoN,IAAa1N,GAGlB,MAAe,QAARA,EAAiB,IAuL5B,QAAS2N,IAAmB3pE,GACxBpS,GAAOiM,SAASoJ,GAAGjD,GAAQ,WACvB,MAAOjW,MAAK4S,MAAMqD,IA2D1B,QAAS4pE,IAAWC,GAEK,mBAAVC,SAGXC,GAAkBC,GAAYp8E,OAE1Bo8E,GAAYp8E,OADZi8E,EACqB5P,EACb,uGAGArsE,IAEaA,IAplF7B,IA/WA,GAAIA,IAIAm8E,GAGAx6E,GANA06E,GAAU,QAEVD,GAAiC,mBAAXhR,IAA6C,mBAAX1lE,SAA0BA,SAAW0lE,EAAO1lE,OAAoBvJ,KAATivE,EAE/GvhD,GAAQxoB,KAAKwoB,MACb5nB,GAAiBS,OAAO4M,UAAUrN,eAGlCixE,GAAO,EACPF,GAAQ,EACRC,GAAO,EACPE,GAAO,EACPC,GAAS,EACTC,GAAS,EACTC,GAAc,EAGdtyC,MAGA0uC,MAGAuE,GAA+B,mBAAXj4E,IAA0BA,GAAUA,EAAOD,QAG/Dq+E,GAAkB,sBAClBkC,GAA0B,uDAI1BC,GAAmB,gIAGnBhI,GAAmB,qKACnBQ,GAAwB,6CAGxBmB,GAA2B,QAC3BR,GAA6B,UAC7BL,GAA4B,UAC5BG,GAA2B,gBAC3BS,GAAmB,MACnBN,GAAiB,mHACjBI,GAAqB,uBACrBC,GAAc,KACdH,GAAqB,aACrBC,GAAwB,yBAGxBZ,GAAqB,KACrBO,GAAsB,OACtBN,GAAwB,QACxBC,GAAuB,QACvBG,GAAsB,aACtBD,GAAyB,WAIzBwE,GAAW,4IAEX0C,GAAY,uBAEZzC,KACK,eAAgB,0BAChB,aAAc,sBACd,eAAgB,oBAChB,aAAc,iBACd,WAAY,gBAIjBC,KACK,gBAAiB,6BACjB,WAAY,wBACZ,QAAS,mBACT,KAAM,cAIXpD,GAAuB,kBAIvB6F,IADyB,0CAA0Cv4E,MAAM,MAErEw4E,aAAiB,EACjBC,QAAY,IACZC,QAAY,IACZC,MAAU,KACVC,KAAS,MACTC,OAAW,OACXC,MAAU,UAGdtL,IACI2I,GAAK,cACLtyE,EAAI,SACJpL,EAAI,SACJmL,EAAI,OACJgB,EAAI,MACJm0E,EAAI,OACJzxB,EAAI,OACJmsB,EAAI,UACJzQ,EAAI,QACJgW,EAAI,UACJ/uE,EAAI,OACJgvE,IAAM,YACNlqD,EAAI,UACJ2kD,EAAI,aACJE,GAAI,WACJJ,GAAI,eAGR/F,IACIyL,UAAY,YACZC,WAAa,aACbC,QAAU,UACVC,SAAW,WACXC,YAAc,eAIlB7I,MAGAkG,IACI9yE,EAAG,GACHpL,EAAG,GACHmL,EAAG,GACHgB,EAAG,GACHo+D,EAAG,IAIPuW,GAAmB,gBAAgBv5E,MAAM,KACzCw5E,GAAe,kBAAkBx5E,MAAM,KAEvCswE,IACItN,EAAO,WACH,MAAO/qE,MAAK04B,QAAU,GAE1B8oD,IAAO,SAAUlgD,GACb,MAAOthC,MAAK2wE,aAAa8Q,YAAYzhF,KAAMshC,IAE/CogD,KAAO,SAAUpgD,GACb,MAAOthC,MAAK2wE,aAAayB,OAAOpyE,KAAMshC,IAE1Cw/C,EAAO,WACH,MAAO9gF,MAAKy4B,QAEhBuoD,IAAO,WACH,MAAOhhF,MAAKs4B,aAEhB3rB,EAAO,WACH,MAAO3M,MAAKq4B,OAEhBspD,GAAO,SAAUrgD,GACb,MAAOthC,MAAK2wE,aAAaiR,YAAY5hF,KAAMshC,IAE/CugD,IAAO,SAAUvgD,GACb,MAAOthC,MAAK2wE,aAAamR,cAAc9hF,KAAMshC,IAEjDygD,KAAO,SAAUzgD,GACb,MAAOthC,MAAK2wE,aAAaqR,SAAShiF,KAAMshC,IAE5C+tB,EAAO,WACH,MAAOrvD,MAAKsyE,QAEhBkJ,EAAO,WACH,MAAOx7E,MAAKiiF,WAEhBC,GAAO,WACH,MAAO1R,GAAaxwE,KAAKu4B,OAAS,IAAK,IAE3C4pD,KAAO,WACH,MAAO3R,GAAaxwE,KAAKu4B,OAAQ,IAErC6pD,MAAQ,WACJ,MAAO5R,GAAaxwE,KAAKu4B,OAAQ,IAErC8pD,OAAS,WACL,GAAIrwE,GAAIhS,KAAKu4B,OAAQvJ,EAAOhd,GAAK,EAAI,IAAM,GAC3C,OAAOgd,GAAOwhD,EAAatrE,KAAK2lB,IAAI7Y,GAAI,IAE5C2pE,GAAO,WACH,MAAOnL,GAAaxwE,KAAKq7E,WAAa,IAAK,IAE/CiH,KAAO,WACH,MAAO9R,GAAaxwE,KAAKq7E,WAAY,IAEzCkH,MAAQ,WACJ,MAAO/R,GAAaxwE,KAAKq7E,WAAY,IAEzCE,GAAO,WACH,MAAO/K,GAAaxwE,KAAKwiF,cAAgB,IAAK,IAElDC,KAAO,WACH,MAAOjS,GAAaxwE,KAAKwiF,cAAe,IAE5CE,MAAQ,WACJ,MAAOlS,GAAaxwE,KAAKwiF,cAAe,IAE5C1rD,EAAI,WACA,MAAO92B,MAAK6hC,WAEhB45C,EAAI,WACA,MAAOz7E,MAAK2iF,cAEhBp9E,EAAO,WACH,MAAOvF,MAAK2wE,aAAaO,SAASlxE,KAAKo9B,QAASp9B,KAAKq9B,WAAW,IAEpEwtC,EAAO,WACH,MAAO7qE,MAAK2wE,aAAaO,SAASlxE,KAAKo9B,QAASp9B,KAAKq9B,WAAW,IAEpEjT,EAAO,WACH,MAAOpqB,MAAKo9B,SAEhBzxB,EAAO,WACH,MAAO3L,MAAKo9B,QAAU,IAAM,IAEhC58B,EAAO,WACH,MAAOR,MAAKq9B,WAEhBzxB,EAAO,WACH,MAAO5L,MAAKs9B,WAEhBjT,EAAO,WACH,MAAO8qD,GAAMn1E,KAAKu9B,eAAiB,MAEvCqlD,GAAO,WACH,MAAOpS,GAAa2E,EAAMn1E,KAAKu9B,eAAiB,IAAK,IAEzDslD,IAAO,WACH,MAAOrS,GAAaxwE,KAAKu9B,eAAgB,IAE7CulD,KAAO,WACH,MAAOtS,GAAaxwE,KAAKu9B,eAAgB,IAE7CwlD,EAAO,WACH,GAAIx9E,GAAIvF,KAAKgjF,YACT58E,EAAI,GAKR,OAJQ,GAAJb,IACAA,GAAKA,EACLa,EAAI,KAEDA,EAAIoqE,EAAa2E,EAAM5vE,EAAI,IAAK,GAAK,IAAMirE,EAAa2E,EAAM5vE,GAAK,GAAI,IAElF09E,GAAO,WACH,GAAI19E,GAAIvF,KAAKgjF,YACT58E,EAAI,GAKR,OAJQ,GAAJb,IACAA,GAAKA,EACLa,EAAI,KAEDA,EAAIoqE,EAAa2E,EAAM5vE,EAAI,IAAK,GAAKirE,EAAa2E,EAAM5vE,GAAK,GAAI,IAE5E2X,EAAI,WACA,MAAOld,MAAKkjF,YAEhBC,GAAK,WACD,MAAOnjF,MAAKojF,YAEhBrxE,EAAO,WACH,MAAO/R,MAAKgH,WAEhB4jB,EAAO,WACH,MAAO5qB,MAAKqjF,QAEhBtC,EAAI,WACA,MAAO/gF,MAAKmyE,YAIpB9B,MAEAiT,IAAS,SAAU,cAAe,WAAY,gBAAiB,eAE/D1R,IAAmB,EAyFhB0P,GAAiB37E,QACpBH,GAAI87E,GAAiBjnC,MACrBg+B,GAAqB7yE,GAAI,KAAOirE,EAAgB4H,GAAqB7yE,IAAIA,GAE7E,MAAO+7E,GAAa57E,QAChBH,GAAI+7E,GAAalnC,MACjBg+B,GAAqB7yE,GAAIA,IAAK8qE,EAAS+H,GAAqB7yE,IAAI,EAEpE6yE,IAAqBkL,KAAOjT,EAAS+H,GAAqB2I,IAAK,GA0d/D17E,EAAOgsE,EAAOn+D,WAEVg2C,IAAM,SAAUqoB,GACZ,GAAI3rE,GAAML,CACV,KAAKA,IAAKgsE,GACN3rE,EAAO2rE,EAAOhsE,GACM,kBAATK,GACP7F,KAAKwF,GAAKK,EAEV7F,KAAK,IAAMwF,GAAKK,CAKxB7F,MAAKi6E,qBAAuB,GAAIC,QAAOl6E,KAAKg6E,cAAcvV,OAAS,IAAM,UAAUA,SAGvFiO,QAAU,wFAAwF3qE,MAAM,KACxGqqE,OAAS,SAAU5xE,GACf,MAAOR,MAAK0yE,QAAQlyE,EAAEk4B,UAG1B8qD,aAAe,kDAAkDz7E,MAAM,KACvE05E,YAAc,SAAUjhF,GACpB,MAAOR,MAAKwjF,aAAahjF,EAAEk4B,UAG/BkiD,YAAc,SAAU6I,EAAWniD,EAAQ6gC,GACvC,GAAI38D,GAAGgvE,EAAKkP,CAQZ,KANK1jF,KAAK2jF,eACN3jF,KAAK2jF,gBACL3jF,KAAK4jF,oBACL5jF,KAAK6jF,sBAGJr+E,EAAI,EAAO,GAAJA,EAAQA,IAAK,CAYrB,GAVAgvE,EAAM3wE,GAAOkyE,KAAK,IAAMvwE,IACpB28D,IAAWniE,KAAK4jF,iBAAiBp+E,KACjCxF,KAAK4jF,iBAAiBp+E,GAAK,GAAI00E,QAAO,IAAMl6E,KAAKoyE,OAAOoC,EAAK,IAAIhqE,QAAQ,IAAK,IAAM,IAAK,KACzFxK,KAAK6jF,kBAAkBr+E,GAAK,GAAI00E,QAAO,IAAMl6E,KAAKyhF,YAAYjN,EAAK,IAAIhqE,QAAQ,IAAK,IAAM,IAAK,MAE9F23D,GAAWniE,KAAK2jF,aAAan+E,KAC9Bk+E,EAAQ,IAAM1jF,KAAKoyE,OAAOoC,EAAK,IAAM,KAAOx0E,KAAKyhF,YAAYjN,EAAK,IAClEx0E,KAAK2jF,aAAan+E,GAAK,GAAI00E,QAAOwJ,EAAMl5E,QAAQ,IAAK,IAAK,MAG1D23D,GAAqB,SAAX7gC,GAAqBthC,KAAK4jF,iBAAiBp+E,GAAGwI,KAAKy1E,GAC7D,MAAOj+E,EACJ,IAAI28D,GAAqB,QAAX7gC,GAAoBthC,KAAK6jF,kBAAkBr+E,GAAGwI,KAAKy1E,GACpE,MAAOj+E,EACJ,KAAK28D,GAAUniE,KAAK2jF,aAAan+E,GAAGwI,KAAKy1E,GAC5C,MAAOj+E,KAKnBs+E,UAAY,2DAA2D/7E,MAAM,KAC7Ei6E,SAAW,SAAUxhF,GACjB,MAAOR,MAAK8jF,UAAUtjF,EAAE63B,QAG5B0rD,eAAiB,8BAA8Bh8E,MAAM,KACrD+5E,cAAgB,SAAUthF,GACtB,MAAOR,MAAK+jF,eAAevjF,EAAE63B,QAGjC2rD,aAAe,uBAAuBj8E,MAAM,KAC5C65E,YAAc,SAAUphF,GACpB,MAAOR,MAAKgkF,aAAaxjF,EAAE63B,QAG/B4iD,cAAgB,SAAUgJ,GACtB,GAAIz+E,GAAGgvE,EAAKkP,CAMZ,KAJK1jF,KAAKkkF,iBACNlkF,KAAKkkF,mBAGJ1+E,EAAI,EAAO,EAAJA,EAAOA,IAQf,GANKxF,KAAKkkF,eAAe1+E,KACrBgvE,EAAM3wE,IAAQ,IAAM,IAAIw0B,IAAI7yB,GAC5Bk+E,EAAQ,IAAM1jF,KAAKgiF,SAASxN,EAAK,IAAM,KAAOx0E,KAAK8hF,cAActN,EAAK,IAAM,KAAOx0E,KAAK4hF,YAAYpN,EAAK,IACzGx0E,KAAKkkF,eAAe1+E,GAAK,GAAI00E,QAAOwJ,EAAMl5E,QAAQ,IAAK,IAAK,MAG5DxK,KAAKkkF,eAAe1+E,GAAGwI,KAAKi2E,GAC5B,MAAOz+E,IAKnB2+E,iBACIC,IAAM,YACNC,GAAK,SACLC,EAAI,aACJC,GAAK,eACLC,IAAM,kBACNC,KAAO,yBAEX9L,eAAiB,SAAUjwE,GACvB,GAAIkrE,GAAS5zE,KAAKmkF,gBAAgBz7E,EAOlC;OANKkrE,GAAU5zE,KAAKmkF,gBAAgBz7E,EAAIu8B,iBACpC2uC,EAAS5zE,KAAKmkF,gBAAgBz7E,EAAIu8B,eAAez6B,QAAQ,mBAAoB,SAAUqoE,GACnF,MAAOA,GAAI5nE,MAAM,KAErBjL,KAAKmkF,gBAAgBz7E,GAAOkrE,GAEzBA,GAGXvC,KAAO,SAAUyD,GAGb,MAAiD,OAAxCA,EAAQ,IAAIvwC,cAAcnf,OAAO,IAG9Cq0D,eAAiB,gBACjBvI,SAAW,SAAU9zC,EAAOC,EAASqnD,GACjC,MAAItnD,GAAQ,GACDsnD,EAAU,KAAO,KAEjBA,EAAU,KAAO,MAKhCC,WACIC,QAAU,gBACVC,QAAU,mBACVC,SAAW,eACXC,QAAU,oBACVC,SAAW,sBACXC,SAAW,KAEfC,SAAW,SAAUx8E,EAAK8rE,EAAKr3C,GAC3B,GAAIy2C,GAAS5zE,KAAK2kF,UAAUj8E,EAC5B,OAAyB,kBAAXkrE,GAAwBA,EAAO77D,MAAMy8D,GAAMr3C,IAAQy2C,GAGrEuR,eACIC,OAAS,QACTC,KAAO,SACPz5E,EAAI,gBACJpL,EAAI,WACJ8kF,GAAK,aACL35E,EAAI,UACJ45E,GAAK,WACL54E,EAAI,QACJg1E,GAAK,UACL5W,EAAI,UACJya,GAAK,YACLxzE,EAAI,SACJyzE,GAAK,YAGTjH,aAAe,SAAU/K,EAAQ6K,EAAehE,EAAQiE,GACpD,GAAI3K,GAAS5zE,KAAKmlF,cAAc7K,EAChC,OAA0B,kBAAX1G,GACXA,EAAOH,EAAQ6K,EAAehE,EAAQiE,GACtC3K,EAAOppE,QAAQ,MAAOipE,IAG9BiS,WAAa,SAAUr5D,EAAMunD,GACzB,GAAItyC,GAASthC,KAAKmlF,cAAc94D,EAAO,EAAI,SAAW,OACtD,OAAyB,kBAAXiV,GAAwBA,EAAOsyC,GAAUtyC,EAAO92B,QAAQ,MAAOopE,IAGjFhD,QAAU,SAAU6C,GAChB,MAAOzzE,MAAK2lF,SAASn7E,QAAQ,KAAMipE,IAEvCkS,SAAW,KACX3L,cAAgB,UAEhBmF,SAAW,SAAU7E,GACjB,MAAOA,IAGXsL,WAAa,SAAUtL,GACnB,MAAOA,IAGXhI,KAAO,SAAUkC,GACb,MAAOiC,IAAWjC,EAAKx0E,KAAK07E,MAAMnF,IAAKv2E,KAAK07E,MAAMlF,KAAKlE,MAG3DoJ,OACInF,IAAM,EACNC,IAAM,GAGVmI,eAAiB,WACb,MAAO3+E,MAAK07E,MAAMnF,KAGtBsP,eAAiB,WACb,MAAO7lF,MAAK07E,MAAMlF,KAGtBsP,aAAc,eACdrN,YAAa,WACT,MAAOz4E,MAAK8lF,gBA0yBpBjiF,GAAS,SAAUixE,EAAOxzC,EAAQ+C,EAAQ89B,GACtC,GAAI1hE,EAiBJ,OAfuB,iBAAb,KACN0hE,EAAS99B,EACTA,EAAS79B,GAIb/F,KACAA,EAAEqyE,kBAAmB,EACrBryE,EAAEsyE,GAAK+B,EACPr0E,EAAEuyE,GAAK1xC,EACP7gC,EAAEwyE,GAAK5uC,EACP5jC,EAAEyyE,QAAU/Q,EACZ1hE,EAAE2yE,QAAS,EACX3yE,EAAE6yE,IAAMlE,IAED6P,GAAWx+E,IAGtBoD,GAAOmsE,6BAA8B,EAErCnsE,GAAOk6E,wBAA0B7N,EAC7B,4LAIA,SAAUsB,GACNA,EAAOt5C,GAAK,GAAI5zB,MAAKktE,EAAOuB,IAAMvB,EAAOwJ,QAAU,OAAS,OA0BpEn3E,GAAOiI,IAAM,WACT,GAAImN,MAAUhO,MAAM1K,KAAKmF,UAAW,EAEpC,OAAO05E,IAAO,WAAYnmE,IAG9BpV,GAAO6I,IAAM,WACT,GAAIuM,MAAUhO,MAAM1K,KAAKmF,UAAW,EAEpC,OAAO05E,IAAO,UAAWnmE,IAI7BpV,GAAOkyE,IAAM,SAAUjB,EAAOxzC,EAAQ+C,EAAQ89B,GAC1C,GAAI1hE,EAkBJ,OAhBuB,iBAAb,KACN0hE,EAAS99B,EACTA,EAAS79B,GAIb/F,KACAA,EAAEqyE,kBAAmB,EACrBryE,EAAEu6E,SAAU,EACZv6E,EAAE2yE,QAAS,EACX3yE,EAAEwyE,GAAK5uC,EACP5jC,EAAEsyE,GAAK+B,EACPr0E,EAAEuyE,GAAK1xC,EACP7gC,EAAEyyE,QAAU/Q,EACZ1hE,EAAE6yE,IAAMlE,IAED6P,GAAWx+E,GAAGs1E,OAIzBlyE,GAAOw/E,KAAO,SAAUvO,GACpB,MAAOjxE,IAAe,IAARixE,IAIlBjxE,GAAOiM,SAAW,SAAUglE,EAAOpsE,GAC/B,GAGIsmB,GACA+2D,EACAC,EACAC,EANAn2E,EAAWglE,EAEXvwE,EAAQ,IAiEZ,OA3DIV,IAAOqiF,WAAWpR,GAClBhlE,GACIouE,GAAIpJ,EAAMtC,cACV7lE,EAAGmoE,EAAMrC,MACT1H,EAAG+J,EAAMpC,SAEW,gBAAVoC,IACdhlE,KACIpH,EACAoH,EAASpH,GAAOosE,EAEhBhlE,EAASytB,aAAeu3C,IAElBvwE,EAAQ47E,GAAwB17E,KAAKqwE,KAC/C9lD,EAAqB,MAAbzqB,EAAM,GAAc,GAAK,EACjCuL,GACIkC,EAAG,EACHrF,EAAGwoE,EAAM5wE,EAAMuyE,KAAS9nD,EACxBrjB,EAAGwpE,EAAM5wE,EAAMyyE,KAAShoD,EACxBxuB,EAAG20E,EAAM5wE,EAAM0yE,KAAWjoD,EAC1BpjB,EAAGupE,EAAM5wE,EAAM2yE,KAAWloD,EAC1BkvD,GAAI/I,EAAM5wE,EAAM4yE,KAAgBnoD,KAE1BzqB,EAAQ67E,GAAiB37E,KAAKqwE,KACxC9lD,EAAqB,MAAbzqB,EAAM,GAAc,GAAK,EACjCyhF,EAAW,SAAUG,GAIjB,GAAIpS,GAAMoS,GAAO9gE,WAAW8gE,EAAI37E,QAAQ,IAAK,KAE7C,QAAQ9F,MAAMqvE,GAAO,EAAIA,GAAO/kD,GAEpClf,GACIkC,EAAGg0E,EAASzhF,EAAM,IAClBwmE,EAAGib,EAASzhF,EAAM,IAClBoI,EAAGq5E,EAASzhF,EAAM,IAClBoH,EAAGq6E,EAASzhF,EAAM,IAClB/D,EAAGwlF,EAASzhF,EAAM,IAClBqH,EAAGo6E,EAASzhF,EAAM,IAClB8qD,EAAG22B,EAASzhF,EAAM,MAEH,MAAZuL,EACPA,KAC2B,gBAAbA,KACT,QAAUA,IAAY,MAAQA,MACnCm2E,EAAUhS,EAAkBpwE,GAAOiM,EAASsZ,MAAOvlB,GAAOiM,EAASuZ,KAEnEvZ,KACAA,EAASouE,GAAK+H,EAAQ1oD,aACtBztB,EAASi7D,EAAIkb,EAAQ7T,QAGzB2T,EAAM,GAAIjU,GAAShiE,GAEfjM,GAAOqiF,WAAWpR,IAAU3F,EAAW2F,EAAO,aAC9CiR,EAAIpT,QAAUmC,EAAMnC,SAGjBoT,GAIXliF,GAAOuiF,QAAUlG,GAGjBr8E,GAAOm+B,cAAgBq+C,GAGvBx8E,GAAO64E,SAAW,aAIlB74E,GAAO0vE,iBAAmBA,GAI1B1vE,GAAOguE,aAAe,aAGtBhuE,GAAOwiF,sBAAwB,SAAUruB,EAAWsuB,GAChD,MAAI5H,IAAuB1mB,KAAexxD,GAC/B,EAEP8/E,IAAU9/E,EACHk4E,GAAuB1mB,IAElC0mB,GAAuB1mB,GAAasuB,GAC7B,IAGXziF,GAAOygC,KAAO4rC,EACV,wDACA,SAAUxnE,EAAKrB,GACX,MAAOxD,IAAOwgC,OAAO37B,EAAKrB,KAOlCxD,GAAOwgC,OAAS,SAAU37B,EAAKoO,GAC3B,GAAIpE,EAcJ,OAbIhK,KAEIgK,EADmB,mBAAb,GACC7O,GAAO0iF,aAAa79E,EAAKoO,GAGzBjT,GAAO8sE,WAAWjoE,GAGzBgK,IACA7O,GAAOiM,SAAS6iE,QAAU9uE,GAAO8uE,QAAUjgE,IAI5C7O,GAAO8uE,QAAQ6T,OAG1B3iF,GAAO0iF,aAAe,SAAUtwE,EAAMa,GAClC,MAAe,QAAXA,GACAA,EAAO2vE,KAAOxwE,EACT4uB,GAAQ5uB,KACT4uB,GAAQ5uB,GAAQ,GAAIq7D,IAExBzsC,GAAQ5uB,GAAMkzC,IAAIryC,GAGlBjT,GAAOwgC,OAAOpuB,GAEP4uB,GAAQ5uB,WAGR4uB,IAAQ5uB,GACR,OAIfpS,GAAO6iF,SAAWxW,EACd,gEACA,SAAUxnE,GACN,MAAO7E,IAAO8sE,WAAWjoE,KAKjC7E,GAAO8sE,WAAa,SAAUjoE,GAC1B,GAAI27B,EAMJ,IAJI37B,GAAOA,EAAIiqE,SAAWjqE,EAAIiqE,QAAQ6T,QAClC99E,EAAMA,EAAIiqE,QAAQ6T,QAGjB99E,EACD,MAAO7E,IAAO8uE,OAGlB,KAAKzsE,EAAQwC,GAAM,CAGf,GADA27B,EAASuzC,EAAWlvE,GAEhB,MAAO27B,EAEX37B,IAAOA,GAGX,MAAOgvE,GAAahvE,IAIxB7E,GAAOoD,SAAW,SAAU8b,GACxB,MAAOA,aAAewuD,IACV,MAAPxuD,GAAeosD,EAAWpsD,EAAK,qBAIxClf,GAAOqiF,WAAa,SAAUnjE,GAC1B,MAAOA,aAAe+uD,GAG1B,KAAKtsE,GAAI89E,GAAM39E,OAAS,EAAGH,IAAK,IAAKA,GACjCmwE,EAAS2N,GAAM99E,IAGnB3B,IAAOuxE,eAAiB,SAAUC,GAC9B,MAAOD,GAAeC,IAG1BxxE,GAAOq7E,QAAU,SAAUyH,GACvB,GAAInmF,GAAIqD,GAAOkyE,IAAIyH,IAQnB,OAPa,OAATmJ,EACArhF,EAAO9E,EAAE8yE,IAAKqT,GAGdnmF,EAAE8yE,IAAI1D,iBAAkB,EAGrBpvE,GAGXqD,GAAO+iF,UAAY,WACf,MAAO/iF,IAAOkU,MAAM,KAAMrS,WAAWkhF,aAGzC/iF,GAAOi3E,kBAAoB,SAAUhG,GACjC,MAAOK,GAAML,IAAUK,EAAML,GAAS,GAAK,KAAO,MAGtDjxE,GAAOQ,OAASA,EAOhBiB,EAAOzB,GAAOqV,GAAKq4D,EAAOp+D,WAEtBilB,MAAQ,WACJ,MAAOv0B,IAAO7D,OAGlBgH,QAAU,WACN,OAAQhH,KAAKk4B,GAA4B,KAArBl4B,KAAKqzE,SAAW,IAGxCgQ,KAAO,WACH,MAAOn+E,MAAKC,OAAOnF,KAAO,MAG9BqF,SAAW,WACP,MAAOrF,MAAKo4B,QAAQiM,OAAO,MAAM/C,OAAO,qCAG5Cp6B,OAAS,WACL,MAAOlH,MAAKqzE,QAAU,GAAI/uE,OAAMtE,MAAQA,KAAKk4B,IAGjD9wB,YAAc,WACV,GAAI5G,GAAIqD,GAAO7D,MAAM+1E,KACrB,OAAI,GAAIv1E,EAAE+3B,QAAU/3B,EAAE+3B,QAAU,KACxB,kBAAsBj0B,MAAK6O,UAAU/L,YAE9BpH,KAAKkH,SAASE,cAEdkxE,EAAa93E,EAAG,gCAGpB83E,EAAa93E,EAAG,mCAI/B+H,QAAU,WACN,GAAI/H,GAAIR,IACR,QACIQ,EAAE+3B,OACF/3B,EAAEk4B,QACFl4B,EAAEi4B,OACFj4B,EAAE48B,QACF58B,EAAE68B,UACF78B,EAAE88B,UACF98B,EAAE+8B,iBAIV85C,QAAU,WACN,MAAOA,GAAQr3E,OAGnB6mF,aAAe,WACX,MAAI7mF,MAAK42E,GACE52E,KAAKq3E,WAAatC,EAAc/0E,KAAK42E,IAAK52E,KAAKozE,OAASvvE,GAAOkyE,IAAI/1E,KAAK42E,IAAM/yE,GAAO7D,KAAK42E,KAAKruE,WAAa,GAGhH,GAGXu+E,aAAe,WACX,MAAOxhF,MAAWtF,KAAKszE,MAG3ByT,UAAW,WACP,MAAO/mF,MAAKszE,IAAIzvD,UAGpBkyD,IAAM,SAAUiR,GACZ,MAAOhnF,MAAKgjF,UAAU,EAAGgE,IAG7B/O,MAAQ,SAAU+O,GASd,MARIhnF,MAAKozE,SACLpzE,KAAKgjF,UAAU,EAAGgE,GAClBhnF,KAAKozE,QAAS,EAEV4T,GACAhnF,KAAKqrB,SAASrrB,KAAKinF,iBAAkB,MAGtCjnF,MAGXshC,OAAS,SAAU4lD,GACf,GAAItT,GAAS0E,EAAat4E,KAAMknF,GAAerjF,GAAOm+B,cACtD,OAAOhiC,MAAK2wE,aAAaiV,WAAWhS,IAGxC3gE,IAAMmhE,EAAY,EAAG,OAErB/oD,SAAW+oD,EAAY,GAAI,YAE3B/nD,KAAO,SAAUyoD,EAAOO,EAAO8R,GAC3B,GAEY96D,GAAMunD,EAFdwT,EAAOlT,EAAOY,EAAO90E,MACrBqnF,EAAmD,KAAvCD,EAAKpE,YAAchjF,KAAKgjF,YAqBxC,OAlBA3N,GAAQD,EAAeC,GAET,SAAVA,GAA8B,UAAVA,GAA+B,YAAVA,GACzCzB,EAAS/C,EAAU7wE,KAAMonF,GACX,YAAV/R,EACAzB,GAAkB,EACD,SAAVyB,IACPzB,GAAkB,MAGtBvnD,EAAOrsB,KAAOonF,EACdxT,EAAmB,WAAVyB,EAAqBhpD,EAAO,IACvB,WAAVgpD,EAAqBhpD,EAAO,IAClB,SAAVgpD,EAAmBhpD,EAAO,KAChB,QAAVgpD,GAAmBhpD,EAAOg7D,GAAY,MAC5B,SAAVhS,GAAoBhpD,EAAOg7D,GAAY,OACvCh7D,GAED86D,EAAUvT,EAASJ,EAASI,IAGvCxqD,KAAO,SAAU+Q,EAAMmkD,GACnB,MAAOz6E,IAAOiM,UAAUuZ,GAAIrpB,KAAMopB,KAAM+Q,IAAOkK,OAAOrkC,KAAKqkC,UAAUijD,UAAUhJ,IAGnFiJ,QAAU,SAAUjJ,GAChB,MAAOt+E,MAAKopB,KAAKvlB,KAAUy6E,IAG/B4G,SAAW,SAAU/qD,GAIjB,GAAIgD,GAAMhD,GAAQt2B,KACd2jF,EAAMtT,EAAO/2C,EAAKn9B,MAAMynF,QAAQ,OAChCp7D,EAAOrsB,KAAKqsB,KAAKm7D,EAAK,QAAQ,GAC9BlmD,EAAgB,GAAPjV,EAAY,WACV,GAAPA,EAAY,WACL,EAAPA,EAAW,UACJ,EAAPA,EAAW,UACJ,EAAPA,EAAW,UACJ,EAAPA,EAAW,WAAa,UAChC,OAAOrsB,MAAKshC,OAAOthC,KAAK2wE,aAAauU,SAAS5jD,EAAQthC,KAAM6D,GAAOs5B,MAGvEw5C,WAAa,WACT,MAAOA,GAAW32E,KAAKu4B,SAG3BmvD,MAAQ,WACJ,MAAQ1nF,MAAKgjF,YAAchjF,KAAKo4B,QAAQM,MAAM,GAAGsqD,aAC7ChjF,KAAKgjF,YAAchjF,KAAKo4B,QAAQM,MAAM,GAAGsqD,aAGjD3qD,IAAM,SAAUy8C,GACZ,GAAIz8C,GAAMr4B,KAAKozE,OAASpzE,KAAKk4B,GAAG8mD,YAAch/E,KAAKk4B,GAAGyvD,QACtD,OAAa,OAAT7S,GACAA,EAAQsJ,GAAatJ,EAAO90E,KAAK2wE,cAC1B3wE,KAAKiT,IAAI6hE,EAAQz8C,EAAK,MAEtBA,GAIfK,MAAQ8mD,GAAa,SAAS,GAE9BiI,QAAU,SAAUpS,GAIhB,OAHAA,EAAQD,EAAeC,IAIvB,IAAK,OACDr1E,KAAK04B,MAAM,EAEf,KAAK,UACL,IAAK,QACD14B,KAAKy4B,KAAK,EAEd,KAAK,OACL,IAAK,UACL,IAAK,MACDz4B,KAAKo9B,MAAM,EAEf,KAAK,OACDp9B,KAAKq9B,QAAQ,EAEjB,KAAK,SACDr9B,KAAKs9B,QAAQ,EAEjB,KAAK,SACDt9B,KAAKu9B,aAAa,GAgBtB,MAXc,SAAV83C,EACAr1E,KAAK6hC,QAAQ,GACI,YAAVwzC,GACPr1E,KAAK2iF,WAAW,GAIN,YAAVtN,GACAr1E,KAAK04B,MAAqC,EAA/BxzB,KAAKC,MAAMnF,KAAK04B,QAAU,IAGlC14B,MAGX4nF,MAAO,SAAUvS,GAEb,MADAA,GAAQD,EAAeC,GACnBA,IAAU7uE,GAAuB,gBAAV6uE,EAChBr1E,KAEJA,KAAKynF,QAAQpS,GAAOpiE,IAAI,EAAc,YAAVoiE,EAAsB,OAASA,GAAQhqD,SAAS,EAAG,OAG1F2oD,QAAS,SAAUc,EAAOO,GACtB,GAAIwS,EAEJ,OADAxS,GAAQD,EAAgC,mBAAVC,GAAwBA,EAAQ,eAChD,gBAAVA,GACAP,EAAQjxE,GAAOoD,SAAS6tE,GAASA,EAAQjxE,GAAOixE,IACxC90E,MAAQ80E,IAEhB+S,EAAUhkF,GAAOoD,SAAS6tE,IAAUA,GAASjxE,GAAOixE,GAC7C+S,GAAW7nF,KAAKo4B,QAAQqvD,QAAQpS,KAI/ClB,SAAU,SAAUW,EAAOO,GACvB,GAAIwS,EAEJ,OADAxS,GAAQD,EAAgC,mBAAVC,GAAwBA,EAAQ,eAChD,gBAAVA,GACAP,EAAQjxE,GAAOoD,SAAS6tE,GAASA,EAAQjxE,GAAOixE,IAChCA,GAAR90E,OAER6nF,EAAUhkF,GAAOoD,SAAS6tE,IAAUA,GAASjxE,GAAOixE,IAC5C90E,KAAKo4B,QAAQwvD,MAAMvS,GAASwS,IAI5CC,UAAW,SAAU1+D,EAAMC,EAAIgsD,GAC3B,MAAOr1E,MAAKg0E,QAAQ5qD,EAAMisD,IAAUr1E,KAAKm0E,SAAS9qD,EAAIgsD,IAG1DpxC,OAAQ,SAAU6wC,EAAOO,GACrB,GAAIwS,EAEJ,OADAxS,GAAQD,EAAeC,GAAS,eAClB,gBAAVA,GACAP,EAAQjxE,GAAOoD,SAAS6tE,GAASA,EAAQjxE,GAAOixE,IACxC90E,QAAU80E,IAElB+S,GAAWhkF,GAAOixE,IACT90E,KAAKo4B,QAAQqvD,QAAQpS,IAAWwS,GAAWA,IAAa7nF,KAAKo4B,QAAQwvD,MAAMvS,KAI5FvpE,IAAKokE,EACI,mGACA,SAAUtqE,GAEN,MADAA,GAAQ/B,GAAOkU,MAAM,KAAMrS,WACZ1F,KAAR4F,EAAe5F,KAAO4F,IAI1C8G,IAAKwjE,EACG,mGACA,SAAUtqE,GAEN,MADAA,GAAQ/B,GAAOkU,MAAM,KAAMrS,WACpBE,EAAQ5F,KAAOA,KAAO4F,IAIzCmiF,KAAO7X,EACC,4GAEA,SAAU4E,EAAOkS,GACb,MAAa,OAATlS,GACqB,gBAAVA,KACPA,GAASA,GAGb90E,KAAKgjF,UAAUlO,EAAOkS,GAEfhnF,OAECA,KAAKgjF,cAe7BA,UAAY,SAAUlO,EAAOkS,GACzB,GACIgB,GADAr+D,EAAS3pB,KAAKqzE,SAAW,CAE7B,OAAa,OAATyB,GACqB,gBAAVA,KACPA,EAAQuF,EAAoBvF,IAE5B5vE,KAAK2lB,IAAIiqD,GAAS,KAClBA,EAAgB,GAARA,IAEP90E,KAAKozE,QAAU4T,IAChBgB,EAAchoF,KAAKinF,kBAEvBjnF,KAAKqzE,QAAUyB,EACf90E,KAAKozE,QAAS,EACK,MAAf4U,GACAhoF,KAAKiT,IAAI+0E,EAAa,KAEtBr+D,IAAWmrD,KACNkS,GAAiBhnF,KAAKioF,kBACvB1T,EAAgCv0E,KACxB6D,GAAOiM,SAASglE,EAAQnrD,EAAQ,KAAM,GAAG,GACzC3pB,KAAKioF,oBACbjoF,KAAKioF,mBAAoB,EACzBpkF,GAAOguE,aAAa7xE,MAAM,GAC1BA,KAAKioF,kBAAoB,OAI1BjoF,MAEAA,KAAKozE,OAASzpD,EAAS3pB,KAAKinF,kBAI3CiB,QAAU,WACN,OAAQloF,KAAKozE,QAGjB+U,YAAc,WACV,MAAOnoF,MAAKozE,QAGhBgV,MAAQ,WACJ,MAAOpoF,MAAKozE,QAA2B,IAAjBpzE,KAAKqzE,SAG/B6P,SAAW,WACP,MAAOljF,MAAKozE,OAAS,MAAQ,IAGjCgQ,SAAW,WACP,MAAOpjF,MAAKozE,OAAS,6BAA+B,IAGxDwT,UAAY,WAMR,MALI5mF,MAAKmzE,KACLnzE,KAAKgjF,UAAUhjF,KAAKmzE,MACM,gBAAZnzE,MAAK+yE,IACnB/yE,KAAKgjF,UAAU3I,EAAoBr6E,KAAK+yE,KAErC/yE,MAGXqoF,qBAAuB,SAAUvT,GAQ7B,MAHIA,GAJCA,EAIOjxE,GAAOixE,GAAOkO,YAHd,GAMJhjF,KAAKgjF,YAAclO,GAAS,KAAO,GAG/CqB,YAAc,WACV,MAAOA,GAAYn2E,KAAKu4B,OAAQv4B,KAAK04B,UAGzCJ,UAAY,SAAUw8C,GAClB,GAAIx8C,GAAY5K,IAAO7pB,GAAO7D,MAAMynF,QAAQ,OAAS5jF,GAAO7D,MAAMynF,QAAQ,SAAW,OAAS,CAC9F,OAAgB,OAAT3S,EAAgBx8C,EAAYt4B,KAAKiT,IAAK6hE,EAAQx8C,EAAY,MAGrE65C,QAAU,SAAU2C,GAChB,MAAgB,OAATA,EAAgB5vE,KAAKwyC,MAAM13C,KAAK04B,QAAU,GAAK,GAAK14B,KAAK04B,MAAoB,GAAbo8C,EAAQ,GAAS90E,KAAK04B,QAAU,IAG3G2iD,SAAW,SAAUvG,GACjB,GAAIv8C,GAAOk+C,GAAWz2E,KAAMA,KAAK2wE,aAAa+K,MAAMnF,IAAKv2E,KAAK2wE,aAAa+K,MAAMlF,KAAKj+C,IACtF,OAAgB,OAATu8C,EAAgBv8C,EAAOv4B,KAAKiT,IAAK6hE,EAAQv8C,EAAO,MAG3DiqD,YAAc,SAAU1N,GACpB,GAAIv8C,GAAOk+C,GAAWz2E,KAAM,EAAG,GAAGu4B,IAClC,OAAgB,OAATu8C,EAAgBv8C,EAAOv4B,KAAKiT,IAAK6hE,EAAQv8C,EAAO,MAG3D+5C,KAAO,SAAUwC,GACb,GAAIxC,GAAOtyE,KAAK2wE,aAAa2B,KAAKtyE,KAClC,OAAgB,OAAT80E,EAAgBxC,EAAOtyE,KAAKiT,IAAqB,GAAhB6hE,EAAQxC,GAAW,MAG/D2P,QAAU,SAAUnN,GAChB,GAAIxC,GAAOmE,GAAWz2E,KAAM,EAAG,GAAGsyE,IAClC,OAAgB,OAATwC,EAAgBxC,EAAOtyE,KAAKiT,IAAqB,GAAhB6hE,EAAQxC,GAAW,MAG/DzwC,QAAU,SAAUizC,GAChB,GAAIjzC,IAAW7hC,KAAKq4B,MAAQ,EAAIr4B,KAAK2wE,aAAa+K,MAAMnF,KAAO,CAC/D,OAAgB,OAATzB,EAAgBjzC,EAAU7hC,KAAKiT,IAAI6hE,EAAQjzC,EAAS,MAG/D8gD,WAAa,SAAU7N,GAInB,MAAgB,OAATA,EAAgB90E,KAAKq4B,OAAS,EAAIr4B,KAAKq4B,IAAIr4B,KAAKq4B,MAAQ,EAAIy8C,EAAQA,EAAQ,IAGvFwT,eAAiB,WACb,MAAOhS,GAAYt2E,KAAKu4B,OAAQ,EAAG,IAGvC+9C,YAAc,WACV,GAAIiS,GAAWvoF,KAAK2wE,aAAa+K,KACjC,OAAOpF,GAAYt2E,KAAKu4B,OAAQgwD,EAAShS,IAAKgS,EAAS/R,MAG3DthE,IAAM,SAAUmgE,GAEZ,MADAA,GAAQD,EAAeC,GAChBr1E,KAAKq1E,MAGhBlsB,IAAM,SAAUksB,EAAOhuE,GACnB,GAAIk4E,EACJ,IAAqB,gBAAVlK,GACP,IAAKkK,IAAQlK,GACTr1E,KAAKmpD,IAAIo2B,EAAMlK,EAAMkK,QAIzBlK,GAAQD,EAAeC,GACI,kBAAhBr1E,MAAKq1E,IACZr1E,KAAKq1E,GAAOhuE,EAGpB,OAAOrH,OAMXqkC,OAAS,SAAU37B,GACf,GAAI8/E,EAEJ,OAAI9/E,KAAQlC,EACDxG,KAAK2yE,QAAQ6T,OAEpBgC,EAAgB3kF,GAAO8sE,WAAWjoE,GACb,MAAjB8/E,IACAxoF,KAAK2yE,QAAU6V,GAEZxoF,OAIfskC,KAAO4rC,EACH,kJACA,SAAUxnE,GACN,MAAIA,KAAQlC,EACDxG,KAAK2wE,aAEL3wE,KAAKqkC,OAAO37B,KAK/BioE,WAAa,WACT,MAAO3wE,MAAK2yE,SAGhBsU,eAAiB,WAGb,MAAuD,KAA/C/hF,KAAKwoB,MAAM1tB,KAAKk4B,GAAGuwD,oBAAsB,OA+CzD5kF,GAAOqV,GAAGuoB,YAAc59B,GAAOqV,GAAGqkB,aAAeiiD,GAAa,gBAAgB,GAC9E37E,GAAOqV,GAAGwoB,OAAS79B,GAAOqV,GAAGokB,QAAUkiD,GAAa,WAAW,GAC/D37E,GAAOqV,GAAGyoB,OAAS99B,GAAOqV,GAAGmkB,QAAUmiD,GAAa,WAAW,GAK/D37E,GAAOqV,GAAG0oB,KAAO/9B,GAAOqV,GAAGkkB,MAAQoiD,GAAa,SAAS,GAEzD37E,GAAOqV,GAAGuf,KAAO+mD,GAAa,QAAQ,GACtC37E,GAAOqV,GAAGsgB,MAAQ02C,EAAU,kDAAmDsP,GAAa,QAAQ,IACpG37E,GAAOqV,GAAGqf,KAAOinD,GAAa,YAAY,GAC1C37E,GAAOqV,GAAG+4D,MAAQ/B,EAAU,kDAAmDsP,GAAa,YAAY,IAGxG37E,GAAOqV,GAAGq5D,KAAO1uE,GAAOqV,GAAGmf,IAC3Bx0B,GAAOqV,GAAGk5D,OAASvuE,GAAOqV,GAAGwf,MAC7B70B,GAAOqV,GAAGm5D,MAAQxuE,GAAOqV,GAAGo5D,KAC5BzuE,GAAOqV,GAAGwvE,SAAW7kF,GAAOqV,GAAG+oE,QAC/Bp+E,GAAOqV,GAAGg5D,SAAWruE,GAAOqV,GAAGi5D,QAG/BtuE,GAAOqV,GAAGyvE,OAAS9kF,GAAOqV,GAAG9R,YAG7BvD,GAAOqV,GAAG0vE,MAAQ/kF,GAAOqV,GAAGkvE,MAkB5B9iF,EAAOzB,GAAOiM,SAASoJ,GAAK44D,EAAS3+D,WAEjCy/D,QAAU,WACN,GAIIt1C,GAASD,EAASD,EAJlBG,EAAev9B,KAAKwyE,cACpBD,EAAOvyE,KAAKyyE,MACZL,EAASpyE,KAAK0yE,QACdhgE,EAAO1S,KAAK4S,MACaq/D,EAAQ,CAIrCv/D,GAAK6qB,aAAeA,EAAe,IAEnCD,EAAUk2C,EAASj2C,EAAe,KAClC7qB,EAAK4qB,QAAUA,EAAU,GAEzBD,EAAUm2C,EAASl2C,EAAU,IAC7B5qB,EAAK2qB,QAAUA,EAAU,GAEzBD,EAAQo2C,EAASn2C,EAAU,IAC3B3qB,EAAK0qB,MAAQA,EAAQ,GAErBm1C,GAAQiB,EAASp2C,EAAQ,IAGzB60C,EAAQuB,EAASkM,GAAYnN,IAC7BA,GAAQiB,EAASmM,GAAY1N,IAI7BG,GAAUoB,EAASjB,EAAO,IAC1BA,GAAQ,GAGRN,GAASuB,EAASpB,EAAS,IAC3BA,GAAU,GAEV1/D,EAAK6/D,KAAOA,EACZ7/D,EAAK0/D,OAASA,EACd1/D,EAAKu/D,MAAQA,GAGjBpnD,IAAM,WAYF,MAXA7qB,MAAKwyE,cAAgBttE,KAAK2lB,IAAI7qB,KAAKwyE,eACnCxyE,KAAKyyE,MAAQvtE,KAAK2lB,IAAI7qB,KAAKyyE,OAC3BzyE,KAAK0yE,QAAUxtE,KAAK2lB,IAAI7qB,KAAK0yE,SAE7B1yE,KAAK4S,MAAM2qB,aAAer4B,KAAK2lB,IAAI7qB,KAAK4S,MAAM2qB,cAC9Cv9B,KAAK4S,MAAM0qB,QAAUp4B,KAAK2lB,IAAI7qB,KAAK4S,MAAM0qB,SACzCt9B,KAAK4S,MAAMyqB,QAAUn4B,KAAK2lB,IAAI7qB,KAAK4S,MAAMyqB,SACzCr9B,KAAK4S,MAAMwqB,MAAQl4B,KAAK2lB,IAAI7qB,KAAK4S,MAAMwqB,OACvCp9B,KAAK4S,MAAMw/D,OAASltE,KAAK2lB,IAAI7qB,KAAK4S,MAAMw/D,QACxCpyE,KAAK4S,MAAMq/D,MAAQ/sE,KAAK2lB,IAAI7qB,KAAK4S,MAAMq/D,OAEhCjyE,MAGXqyE,MAAQ,WACJ,MAAOmB,GAASxzE,KAAKuyE,OAAS,IAGlCvrE,QAAU,WACN,MAAOhH,MAAKwyE,cACG,MAAbxyE,KAAKyyE,MACJzyE,KAAK0yE,QAAU,GAAM,OACK,QAA3ByC,EAAMn1E,KAAK0yE,QAAU,KAG3B4U,SAAW,SAAUuB,GACjB,GAAIjV,GAAS4K,GAAax+E,MAAO6oF,EAAY7oF,KAAK2wE,aAMlD,OAJIkY,KACAjV,EAAS5zE,KAAK2wE,aAAa+U,YAAY1lF,KAAM4zE,IAG1C5zE,KAAK2wE,aAAaiV,WAAWhS,IAGxC3gE,IAAM,SAAU6hE,EAAOjC,GAEnB,GAAIwB,GAAMxwE,GAAOiM,SAASglE,EAAOjC,EAQjC,OANA7yE,MAAKwyE,eAAiB6B,EAAI7B,cAC1BxyE,KAAKyyE,OAAS4B,EAAI5B,MAClBzyE,KAAK0yE,SAAW2B,EAAI3B,QAEpB1yE,KAAK4yE,UAEE5yE,MAGXqrB,SAAW,SAAUypD,EAAOjC,GACxB,GAAIwB,GAAMxwE,GAAOiM,SAASglE,EAAOjC,EAQjC,OANA7yE,MAAKwyE,eAAiB6B,EAAI7B,cAC1BxyE,KAAKyyE,OAAS4B,EAAI5B,MAClBzyE,KAAK0yE,SAAW2B,EAAI3B,QAEpB1yE,KAAK4yE,UAEE5yE,MAGXkV,IAAM,SAAUmgE,GAEZ,MADAA,GAAQD,EAAeC,GAChBr1E,KAAKq1E,EAAM9wC,cAAgB,QAGtCtV,GAAK,SAAUomD,GACX,GAAI9C,GAAMH,CAGV,IAFAiD,EAAQD,EAAeC,GAET,UAAVA,GAA+B,SAAVA,EAGrB,MAFA9C,GAAOvyE,KAAKyyE,MAAQzyE,KAAKwyE,cAAgB,MACzCJ,EAASpyE,KAAK0yE,QAA8B,GAApBgN,GAAYnN,GACnB,UAAV8C,EAAoBjD,EAASA,EAAS,EAI7C,QADAG,EAAOvyE,KAAKyyE,MAAQvtE,KAAKwoB,MAAMiyD,GAAY3/E,KAAK0yE,QAAU,KAClD2C,GACJ,IAAK,OAAQ,MAAO9C,GAAO,EAAIvyE,KAAKwyE,cAAgB,MACpD,KAAK,MAAO,MAAOD,GAAOvyE,KAAKwyE,cAAgB,KAC/C,KAAK,OAAQ,MAAc,IAAPD,EAAYvyE,KAAKwyE,cAAgB,IACrD,KAAK,SAAU,MAAc,IAAPD,EAAY,GAAKvyE,KAAKwyE,cAAgB,GAC5D,KAAK,SAAU,MAAc,IAAPD,EAAY,GAAK,GAAKvyE,KAAKwyE,cAAgB,GAEjE,KAAK,cAAe,MAAOttE,MAAKC,MAAa,GAAPotE,EAAY,GAAK,GAAK,KAAQvyE,KAAKwyE,aACzE,SAAS,KAAM,IAAI5uE,OAAM,gBAAkByxE,KAKvD/wC,KAAOzgC,GAAOqV,GAAGorB,KACjBD,OAASxgC,GAAOqV,GAAGmrB,OAEnBykD,YAAc5Y,EACV,sFAEA,WACI,MAAOlwE,MAAKoH,gBAIpBA,YAAc,WAEV,GAAI6qE,GAAQ/sE,KAAK2lB,IAAI7qB,KAAKiyE,SACtBG,EAASltE,KAAK2lB,IAAI7qB,KAAKoyE,UACvBG,EAAOrtE,KAAK2lB,IAAI7qB,KAAKuyE,QACrBn1C,EAAQl4B,KAAK2lB,IAAI7qB,KAAKo9B,SACtBC,EAAUn4B,KAAK2lB,IAAI7qB,KAAKq9B,WACxBC,EAAUp4B,KAAK2lB,IAAI7qB,KAAKs9B,UAAYt9B,KAAKu9B,eAAiB,IAE9D,OAAKv9B,MAAK+oF,aAMF/oF,KAAK+oF,YAAc,EAAI,IAAM,IACjC,KACC9W,EAAQA,EAAQ,IAAM,KACtBG,EAASA,EAAS,IAAM,KACxBG,EAAOA,EAAO,IAAM,KACnBn1C,GAASC,GAAWC,EAAW,IAAM,KACtCF,EAAQA,EAAQ,IAAM,KACtBC,EAAUA,EAAU,IAAM,KAC1BC,EAAUA,EAAU,IAAM,IAXpB,OAcfqzC,WAAa,WACT,MAAO3wE,MAAK2yE,SAGhBgW,OAAS,WACL,MAAO3oF,MAAKoH,iBAIpBvD,GAAOiM,SAASoJ,GAAG7T,SAAWxB,GAAOiM,SAASoJ,GAAG9R,WAQjD,KAAK5B,KAAK86E,IACFnR,EAAWmR,GAAwB96E,KACnCo6E,GAAmBp6E,GAAE++B,cAI7B1gC,IAAOiM,SAASoJ,GAAG8vE,eAAiB,WAChC,MAAOhpF,MAAKivB,GAAG,OAEnBprB,GAAOiM,SAASoJ,GAAG6vE,UAAY,WAC3B,MAAO/oF,MAAKivB,GAAG,MAEnBprB,GAAOiM,SAASoJ,GAAG+vE,UAAY,WAC3B,MAAOjpF,MAAKivB,GAAG,MAEnBprB,GAAOiM,SAASoJ,GAAGgwE,QAAU,WACzB,MAAOlpF,MAAKivB,GAAG,MAEnBprB,GAAOiM,SAASoJ,GAAGiwE,OAAS,WACxB,MAAOnpF,MAAKivB,GAAG,MAEnBprB,GAAOiM,SAASoJ,GAAGkwE,QAAU,WACzB,MAAOppF,MAAKivB,GAAG,UAEnBprB,GAAOiM,SAASoJ,GAAGmwE,SAAW,WAC1B,MAAOrpF,MAAKivB,GAAG,MAEnBprB,GAAOiM,SAASoJ,GAAGowE,QAAU,WACzB,MAAOtpF,MAAKivB,GAAG,MASnBprB,GAAOwgC,OAAO,MACVklD,aAAc,uBACd3Y,QAAU,SAAU6C,GAChB,GAAIrtE,GAAIqtE,EAAS,GACbG,EAAuC,IAA7BuB,EAAM1B,EAAS,IAAM,IAAa,KACrC,IAANrtE,EAAW,KACL,IAANA,EAAW,KACL,IAANA,EAAW,KAAO,IACvB,OAAOqtE,GAASG,KA4BpBkE,GACAj4E,EAAOD,QAAUiE,IAEfmrE,EAAgC,SAAUwa,EAAS5pF,EAASC,GAM1D,MALIA,GAAO2xE,QAAU3xE,EAAO2xE,UAAY3xE,EAAO2xE,SAASiY,YAAa,IAEjExJ,GAAYp8E,OAASm8E,IAGlBn8E,IACTtD,KAAKX,EAASM,EAAqBN,EAASC,KAASmvE,IAAkCxoE,IAAc3G,EAAOD,QAAUovE,IACxH6Q,IAAW,MAIhBt/E,KAAKP,QAEqBO,KAAKX,EAAU,WAAa,MAAOI,SAAYE,EAAoB,IAAIL,KAIhG,SAASA,EAAQD,GAErB,GAAI8pF,GAAgCC,EAA8B3a,GAOjE,SAAUtvE,EAAMC,GAGXgqF,KAAmCD,EAAiC,EAAW1a,EAA2E,kBAAnC0a,GAAiDA,EAA+B3xE,MAAMnY,EAAS+pF,GAAiCD,IAAmEljF,SAAlCwoE,IAAgDnvE,EAAOD,QAAUovE,KAU7VhvE,KAAM,WAEN,QAASmlD,GAAS12C,GAChB,GAOIjJ,GAPA6D,EAAiBoF,GAAWA,EAAQpF,iBAAkB,EAEtDkQ,EAAY9K,GAAWA,EAAQ8K,WAAahQ,OAE5CqgF,KACAC,GAAUC,WAAYC,UACtBC,IAIJ,KAAKxkF,EAAI,GAAS,KAALA,EAAUA,IAAMwkF,EAAM5lF,OAAO6lF,aAAazkF,KAAOuyE,KAAK,IAAMvyE,EAAI,IAAK8L,OAAO,EAEzF,KAAK9L,EAAI,GAAS,IAALA,EAASA,IAAMwkF,EAAM5lF,OAAO6lF,aAAazkF,KAAOuyE,KAAKvyE,EAAG8L,OAAO,EAE5E,KAAK9L,EAAI,EAAS,GAALA,EAAUA,IAAMwkF,EAAM,GAAKxkF,IAAMuyE,KAAK,GAAKvyE,EAAG8L,OAAO,EAElE,KAAK9L,EAAI,EAAS,IAALA,EAAWA,IAAMwkF,EAAM,IAAMxkF,IAAMuyE,KAAK,IAAMvyE,EAAG8L,OAAO,EAErE,KAAK9L,EAAI,EAAS,GAALA,EAAUA,IAAMwkF,EAAM,MAAQxkF,IAAMuyE,KAAK,GAAKvyE,EAAG8L,OAAO,EAGrE04E,GAAM,SAAWjS,KAAK,IAAKzmE,OAAO,GAClC04E,EAAM,SAAWjS,KAAK,IAAKzmE,OAAO,GAClC04E,EAAM,SAAWjS,KAAK,IAAKzmE,OAAO,GAClC04E,EAAM,SAAWjS,KAAK,IAAKzmE,OAAO,GAClC04E,EAAM,SAAWjS,KAAK,IAAKzmE,OAAO,GAElC04E,EAAY,MAAMjS,KAAK,GAAIzmE,OAAO,GAClC04E,EAAU,IAAQjS,KAAK,GAAIzmE,OAAO,GAClC04E,EAAa,OAAKjS,KAAK,GAAIzmE,OAAO,GAClC04E,EAAY,MAAMjS,KAAK,GAAIzmE,OAAO,GAElC04E,EAAa,OAAKjS,KAAK,GAAIzmE,OAAO,GAClC04E,EAAa,OAAKjS,KAAK,GAAIzmE,OAAO,GAClC04E,EAAa,OAAKjS,KAAK,GAAIzmE,MAAO9K,QAClCwjF,EAAW,KAAOjS,KAAK,GAAIzmE,OAAO,GAClC04E,EAAiB,WAAKjS,KAAK,EAAGzmE,OAAO,GACrC04E,EAAW,KAAWjS,KAAK,EAAGzmE,OAAO,GACrC04E,EAAY,MAAUjS,KAAK,GAAIzmE,OAAO,GACtC04E,EAAW,KAAWjS,KAAK,GAAIzmE,OAAO,GACtC04E,EAAM,WAAgBjS,KAAK,GAAIzmE,OAAO,GACtC04E,EAAc,QAAQjS,KAAK,GAAIzmE,OAAO,GACtC04E,EAAgB,UAAMjS,KAAK,GAAIzmE,OAAO,GAEtC04E,EAAM,MAAYjS,KAAK,IAAKzmE,OAAO,GACnC04E,EAAM,MAAYjS,KAAK,IAAKzmE,OAAO,GACnC04E,EAAM,MAAYjS,KAAK,IAAKzmE,OAAO,GACnC04E,EAAM,MAAYjS,KAAK,IAAKzmE,OAAO,EAInC,IAAI44E,GAAO,SAAS5gF,GAAQ6gF,EAAY7gF,EAAM,YAC1C8gF,EAAK,SAAS9gF,GAAQ6gF,EAAY7gF,EAAM,UAGxC6gF,EAAc,SAAS7gF,EAAMxC,GAC/B,GAAoCN,SAAhCqjF,EAAO/iF,GAAMwC,EAAM+gF,SAAwB,CAE7C,IAAK,GADDC,GAAQT,EAAO/iF,GAAMwC,EAAM+gF,SACtB7kF,EAAI,EAAGA,EAAI8kF,EAAM3kF,OAAQH,IACTgB,SAAnB8jF,EAAM9kF,GAAG8L,MACXg5E,EAAM9kF,GAAG0T,GAAG5P,GAEa,GAAlBghF,EAAM9kF,GAAG8L,OAAmC,GAAlBhI,EAAMyqC,SACvCu2C,EAAM9kF,GAAG0T,GAAG5P,GAEa,GAAlBghF,EAAM9kF,GAAG8L,OAAoC,GAAlBhI,EAAMyqC,UACxCu2C,EAAM9kF,GAAG0T,GAAG5P,EAIM,IAAlBD,GACFC,EAAMD,kBA4FZ,OAtFAugF,GAAiB90D,KAAO,SAASpsB,EAAKJ,EAAUxB,GAI9C,GAHaN,SAATM,IACFA,EAAO,WAEUN,SAAfwjF,EAAMthF,GACR,KAAM,IAAI9E,OAAM,oBAAsB8E,EAEFlC,UAAlCqjF,EAAO/iF,GAAMkjF,EAAMthF,GAAKqvE,QAC1B8R,EAAO/iF,GAAMkjF,EAAMthF,GAAKqvE,UAE1B8R,EAAO/iF,GAAMkjF,EAAMthF,GAAKqvE,MAAM/vE,MAAMkR,GAAG5Q,EAAUgJ,MAAM04E,EAAMthF,GAAK4I,SAKpEs4E,EAAiBW,QAAU,SAASjiF,EAAUxB,GAC/BN,SAATM,IACFA,EAAO,UAET,KAAK,GAAI4B,KAAOshF,GACVA,EAAMlkF,eAAe4C,IACvBkhF,EAAiB90D,KAAKpsB,EAAIJ,EAASxB,IAMzC8iF,EAAiBY,OAAS,SAASlhF,GACjC,IAAK,GAAIZ,KAAOshF,GACd,GAAIA,EAAMlkF,eAAe4C,GAAM,CAC7B,GAAsB,GAAlBY,EAAMyqC,UAAwC,GAApBi2C,EAAMthF,GAAK4I,OAAiBhI,EAAM+gF,SAAWL,EAAMthF,GAAKqvE,KACpF,MAAOrvE,EAEJ,IAAsB,GAAlBY,EAAMyqC,UAAyC,GAApBi2C,EAAMthF,GAAK4I,OAAkBhI,EAAM+gF,SAAWL,EAAMthF,GAAKqvE,KAC3F,MAAOrvE,EAEJ,IAAIY,EAAM+gF,SAAWL,EAAMthF,GAAKqvE,MAAe,SAAPrvE,EAC3C,MAAOA,GAIb,MAAO,wCAITkhF,EAAiBrb,OAAS,SAAS7lE,EAAKJ,EAAUxB,GAIhD,GAHaN,SAATM,IACFA,EAAO,WAEUN,SAAfwjF,EAAMthF,GACR,KAAM,IAAI9E,OAAM,oBAAsB8E,EAExC,IAAiBlC,SAAb8B,EAAwB,CAC1B,GAAImiF,MACAH,EAAQT,EAAO/iF,GAAMkjF,EAAMthF,GAAKqvE,KACpC,IAAcvxE,SAAV8jF,EACF,IAAK,GAAI9kF,GAAI,EAAGA,EAAI8kF,EAAM3kF,OAAQH,KAC1B8kF,EAAM9kF,GAAG0T,IAAM5Q,GAAYgiF,EAAM9kF,GAAG8L,OAAS04E,EAAMthF,GAAK4I,QAC5Dm5E,EAAYziF,KAAK6hF,EAAO/iF,GAAMkjF,EAAMthF,GAAKqvE,MAAMvyE,GAIrDqkF,GAAO/iF,GAAMkjF,EAAMthF,GAAKqvE,MAAQ0S,MAGhCZ,GAAO/iF,GAAMkjF,EAAMthF,GAAKqvE,UAK5B6R,EAAiBpgC,MAAQ,WACvBqgC,GAAUC,WAAYC,WAIxBH,EAAiBt2E,QAAU,WACzBu2E,GAAUC,WAAYC,UACtBxwE,EAAUpQ,oBAAoB,UAAW+gF,GAAM,GAC/C3wE,EAAUpQ,oBAAoB,QAASihF,GAAI,IAI7C7wE,EAAU5Q,iBAAiB,UAAUuhF,GAAK,GAC1C3wE,EAAU5Q,iBAAiB,QAAQyhF,GAAG,GAG/BR,EAGT,MAAOzkC,MAQL,SAAStlD,EAAQD,GAErB,GAAI8pF,GAAgCC,EAA8B3a,GAEjE,SAAUtvE,EAAMC,GAGXgqF,KAAmCD,EAAiC,EAAW1a,EAA2E,kBAAnC0a,GAAiDA,EAA+B3xE,MAAMnY,EAAS+pF,GAAiCD,IAAmEljF,SAAlCwoE,IAAgDnvE,EAAOD,QAAUovE,KAU7VhvE,KAAM,WAEN,GAAI0qF,GAAe,IAkBnB,OAAO,SAAS/lB,GAAY7gE,GAuG1B,QAASiE,GAAMy9D,GACb,MAAOA,GAAOjhE,MAAM,UAOtB,QAASomF,GAAkBrhF,GAEzB,GAAmB,iBAAfA,EAAMxC,KAAyB,CACjC,GAAIwC,EAAMwqC,SAAS82C,UAAYthF,EAAMwqC,SAAS82C,SAASthF,EAAMxC,MAC3D,MAIAwC,GAAMwqC,SAAS82C,YACfthF,EAAMwqC,SAAS82C,SAASthF,EAAMxC,OAAQ,EAK1C,GAAI+jF,IAAU,CACdvhF,GAAMq8B,gBAAkB,WACtBklD,GAAU,GAIZvhF,EAAMwhF,YAAcJ,CAIpB,KADA,GAAInjF,GAAOmjF,EACJnjF,IAASsjF,GAAS,CACvB,GAAIE,GAAYxjF,EAAKzD,QAAUyD,EAAKzD,OAAOinF,UAAUzhF,EAAMxC,KAC3D,IAAIikF,EACF,IAAK,GAAIvlF,GAAI,EAAGA,EAAIulF,EAAUplF,SAAWklF,EAASrlF,IAChDulF,EAAUvlF,GAAG8D,EAIjB/B,GAAOA,EAAKsC,YA9IhB,GAAI/F,EAAOknF,QAAS,CAGlB,GAAIjnF,GAASD,EAETmnF,EAAoB,SAASriF,EAAS6F,GACxC,MAAOk2D,GAAY,GAAI5gE,GAAO6E,EAAS6F,IAOzC,OALA1K,GAAOuB,OAAO2lF,EAAmBlnF,GACjCknF,EAAkBD,QAAU,SAAUpiF,EAAS6F,GAC7C,MAAOk2D,GAAY,GAAI5gE,GAAOinF,QAAQpiF,EAAS6F,KAG1Cw8E,EAIT,GAAIriF,GAAU9E,EAAO8E,OAiIrB,OAhIAA,GAAQ9E,OAASA,EAGjBA,EAAOonF,IAAMpnF,EAAOyP,GACpBzP,EAAOqnF,KAAOrnF,EAAO4P,IACrB5P,EAAOsnF,SAAWtnF,EAAOwP,QAGzBxP,EAAOinF,aAIPjnF,EAAOonF,IAAI,eAAgB,SAAU5hF,GAC/BA,EAAMqnC,UACR+5C,EAAephF,EAAMI,UAUzB5F,EAAOyP,GAAK,SAAUiyD,EAAQ6lB,GAa5B,MAXAtjF,GAAMy9D,GAAQn9D,QAAQ,SAAUiB,GAC9B,GAAIyhF,GAAYjnF,EAAOinF,UAAUzhF,EAC5ByhF,KACHjnF,EAAOinF,UAAUzhF,GAASyhF,KAG1BjnF,EAAOonF,IAAI5hF,EAAOqhF,IAEpBI,EAAU/iF,KAAKqjF,KAGVvnF,GAWTA,EAAO4P,IAAM,SAAU8xD,EAAQ6lB,GAoB7B,MAlBAtjF,GAAMy9D,GAAQn9D,QAAQ,SAAUiB,GAC9B,GAAIyhF,GAAYjnF,EAAOinF,UAAUzhF,EAC7ByhF,KACFA,EAAYM,EAAUN,EAAUp3E,OAAO,SAAUhI,GAC/C,MAAOA,KAAM0/E,OAGXN,EAAUplF,OAAS,EACrB7B,EAAOinF,UAAUzhF,GAASyhF,GAI1BjnF,EAAOqnF,KAAK7hF,EAAOqhF,SACZ7mF,GAAOinF,UAAUzhF,OAKvBxF,GAGTA,EAAOwP,QAAU,WAEf,GAAI1K,GAAU9E,EAAO8E,cACdA,GAAQ9E,OAGfA,EAAOinF,aAGPjnF,EAAOsnF,YAgDFtnF,MAOP,SAASjE,EAAQD,EAASM,GAE9B,GAAI8uE,IAKJ,SAAUzlE,EAAQgI,EAAU+5E,EAAY9kF,GAmBxC,QAAS+kF,GAAkBryE,EAAIsyE,EAASryE,GACpC,MAAOG,YAAWmyE,EAAOvyE,EAAIC,GAAUqyE,GAY3C,QAASE,GAAeC,EAAKzyE,EAAIC,GAC7B,MAAIlT,OAAMC,QAAQylF,IACdC,EAAKD,EAAKxyE,EAAQD,GAAKC,IAChB,IAEJ,EASX,QAASyyE,GAAK7oE,EAAK8oE,EAAU1yE,GACzB,GAAI3T,EAEJ,IAAKud,EAIL,GAAIA,EAAI1a,QACJ0a,EAAI1a,QAAQwjF,EAAU1yE,OACnB,IAAI4J,EAAIpd,SAAWa,EAEtB,IADAhB,EAAI,EACGA,EAAIud,EAAIpd,QACXkmF,EAAStrF,KAAK4Y,EAAS4J,EAAIvd,GAAIA,EAAGud,GAClCvd,QAGJ,KAAKA,IAAKud,GACNA,EAAIjd,eAAeN,IAAMqmF,EAAStrF,KAAK4Y,EAAS4J,EAAIvd,GAAIA,EAAGud,GAavE,QAASzd,GAAOwmF,EAAMlmC,EAAKmb,GAGvB,IAFA,GAAI3zD,GAAO7G,OAAO6G,KAAKw4C,GACnBpgD,EAAI,EACDA,EAAI4H,EAAKzH,UACPo7D,GAAUA,GAAS+qB,EAAK1+E,EAAK5H,MAAQgB,KACtCslF,EAAK1+E,EAAK5H,IAAMogD,EAAIx4C,EAAK5H,KAE7BA,GAEJ,OAAOsmF,GAUX,QAAS/qB,GAAM+qB,EAAMlmC,GACjB,MAAOtgD,GAAOwmF,EAAMlmC,GAAK,GAS7B,QAASmmC,GAAQC,EAAOlY,EAAMnlB,GAC1B,GACIs9B,GADAC,EAAQpY,EAAK3gE,SAGjB84E,GAASD,EAAM74E,UAAY5M,OAAO8H,OAAO69E,GACzCD,EAAO3lF,YAAc0lF,EACrBC,EAAOE,OAASD,EAEZv9B,GACArpD,EAAO2mF,EAAQt9B,GAUvB,QAAS88B,GAAOvyE,EAAIC,GAChB,MAAO,YACH,MAAOD,GAAGnB,MAAMoB,EAASzT,YAWjC,QAAS0mF,GAASvZ,EAAK55D,GACnB,aAAW45D,IAAOwZ,GACPxZ,EAAI96D,MAAMkB,EAAOA,EAAK,IAAMzS,EAAYA,EAAWyS,GAEvD45D,EASX,QAASyZ,GAAYC,EAAMC,GACvB,MAAQD,KAAS/lF,EAAagmF,EAAOD,EASzC,QAASE,GAAkB/iF,EAAQwN,EAAOm0E,GACtCO,EAAKc,EAASx1E,GAAQ,SAASpQ,GAC3B4C,EAAOf,iBAAiB7B,EAAMukF,GAAS,KAU/C,QAASsB,GAAqBjjF,EAAQwN,EAAOm0E,GACzCO,EAAKc,EAASx1E,GAAQ,SAASpQ,GAC3B4C,EAAOP,oBAAoBrC,EAAMukF,GAAS,KAWlD,QAASuB,GAAU9mC,EAAMhhB,GACrB,KAAOghB,GAAM,CACT,GAAIA,GAAQhhB,EACR,OAAO,CAEXghB,GAAOA,EAAKj8C,WAEhB,OAAO,EASX,QAASgjF,GAAMC,EAAKC,GAChB,MAAOD,GAAInmF,QAAQomF,GAAQ,GAQ/B,QAASL,GAASI,GACd,MAAOA,GAAI5/E,OAAOnF,MAAM,QAU5B,QAASilF,GAAQpnC,EAAKmnC,EAAME,GACxB,GAAIrnC,EAAIj/C,UAAYsmF,EAChB,MAAOrnC,GAAIj/C,QAAQomF,EAGnB,KADA,GAAIvnF,GAAI,EACDA,EAAIogD,EAAIjgD,QAAQ,CACnB,GAAKsnF,GAAarnC,EAAIpgD,GAAGynF,IAAcF,IAAWE,GAAarnC,EAAIpgD,KAAOunF,EACtE,MAAOvnF,EAEXA,KAEJ,MAAO,GASf,QAAS+C,GAAQwa,GACb,MAAO9c,OAAMkN,UAAUlI,MAAM1K,KAAKwiB,EAAK,GAU3C,QAASmqE,GAAYtnC,EAAKl9C,EAAKwN,GAK3B,IAJA,GAAI4/D,MACAh/D,KACAtR,EAAI,EAEDA,EAAIogD,EAAIjgD,QAAQ,CACnB,GAAIktE,GAAMnqE,EAAMk9C,EAAIpgD,GAAGkD,GAAOk9C,EAAIpgD,EAC9BwnF,GAAQl2E,EAAQ+7D,GAAO,GACvBiD,EAAQ9tE,KAAK49C,EAAIpgD,IAErBsR,EAAOtR,GAAKqtE,EACZrtE,IAaJ,MAVI0Q,KAII4/D,EAHCptE,EAGSotE,EAAQ5/D,KAAK,SAAyB3Q,EAAGa,GAC/C,MAAOb,GAAEmD,GAAOtC,EAAEsC,KAHZotE,EAAQ5/D,QAQnB4/D,EASX,QAASqX,GAASpqE,EAAKqqE,GAKnB,IAJA,GAAIC,GAAQxnF,EACRynF,EAAYF,EAAS,GAAGnoD,cAAgBmoD,EAASniF,MAAM,GAEvDzF,EAAI,EACDA,EAAI+nF,GAAgB5nF,QAAQ,CAI/B,GAHA0nF,EAASE,GAAgB/nF,GACzBK,EAAO,EAAWwnF,EAASC,EAAYF,EAEnCvnF,IAAQkd,GACR,MAAOld,EAEXL,KAEJ,MAAOgB,GAQX,QAASgnF,KACL,MAAOC,MAQX,QAASC,GAAoB9kF,GACzB,GAAI+kF,GAAM/kF,EAAQglF,aAClB,OAAQD,GAAIE,aAAeF,EAAIG,aAyCnC,QAASC,GAAMC,EAAS1lF,GACpB,GAAIomE,GAAO1uE,IACXA,MAAKguF,QAAUA,EACfhuF,KAAKsI,SAAWA,EAChBtI,KAAK4I,QAAUolF,EAAQplF,QACvB5I,KAAK0J,OAASskF,EAAQv/E,QAAQw/E,YAI9BjuF,KAAKkuF,WAAa,SAASC,GACnB/B,EAAS4B,EAAQv/E,QAAQ20B,QAAS4qD,KAClCtf,EAAK2c,QAAQ8C,IAIrBnuF,KAAKouF,OAoCT,QAASC,GAAoBL,GACzB,GAAIM,GACAC,EAAaP,EAAQv/E,QAAQ8/E,UAajC,OAAO,KAVHD,EADAC,EACOA,EACAC,GACAC,EACAC,GACAC,EACCC,GAGDC,EAFAC,GAIOd,EAAShnB,GAS/B,QAASA,GAAagnB,EAASe,EAAWja,GACtC,GAAIka,GAAcla,EAAMma,SAAStpF,OAC7BupF,EAAqBpa,EAAMqa,gBAAgBxpF,OAC3CgrC,EAAWo+C,EAAYK,IAAgBJ,EAAcE,IAAuB,EAC5EhoB,EAAW6nB,GAAaM,GAAYC,KAAkBN,EAAcE,IAAuB,CAE/Fpa,GAAMnkC,UAAYA,EAClBmkC,EAAM5N,UAAYA,EAEdv2B,IACAq9C,EAAQuB,YAKZza,EAAMia,UAAYA,EAGlBS,EAAiBxB,EAASlZ,GAG1BkZ,EAAQpgE,KAAK,eAAgBknD,GAE7BkZ,EAAQyB,UAAU3a,GAClBkZ,EAAQuB,QAAQG,UAAY5a,EAQhC,QAAS0a,GAAiBxB,EAASlZ,GAC/B,GAAIya,GAAUvB,EAAQuB,QAClBN,EAAWna,EAAMma,SACjBU,EAAiBV,EAAStpF,MAGzB4pF,GAAQK,aACTL,EAAQK,WAAaC,EAAqB/a,IAI1C6a,EAAiB,IAAMJ,EAAQO,cAC/BP,EAAQO,cAAgBD,EAAqB/a,GACnB,IAAnB6a,IACPJ,EAAQO,eAAgB,EAG5B,IAAIF,GAAaL,EAAQK,WACrBE,EAAgBP,EAAQO,cACxBC,EAAeD,EAAgBA,EAAc5jE,OAAS0jE,EAAW1jE,OAEjEA,EAAS4oD,EAAM5oD,OAAS8jE,EAAUf,EACtCna,GAAMmb,UAAY9yD,KAClB23C,EAAMob,UAAYpb,EAAMmb,UAAYL,EAAWK,UAE/Cnb,EAAM1mB,MAAQ+hC,EAASJ,EAAc7jE,GACrC4oD,EAAMnvD,SAAW84C,EAAYsxB,EAAc7jE,GAE3CkkE,EAAeb,EAASza,GACxBA,EAAMub,gBAAkBC,EAAaxb,EAAMt1C,OAAQs1C,EAAMr1C,QAEzDq1C,EAAM73D,MAAQ6yE,EAAgBv8B,EAASu8B,EAAcb,SAAUA,GAAY,EAC3Ena,EAAMyb,SAAWT,EAAgBU,EAAYV,EAAcb,SAAUA,GAAY,EAEjFwB,EAAyBlB,EAASza,EAGlC,IAAIprE,GAASskF,EAAQplF,OACjBgkF,GAAU9X,EAAMhhC,SAASpqC,OAAQA,KACjCA,EAASorE,EAAMhhC,SAASpqC,QAE5BorE,EAAMprE,OAASA,EAGnB,QAAS0mF,GAAeb,EAASza,GAC7B,GAAI5oD,GAAS4oD,EAAM5oD,OACfvC,EAAS4lE,EAAQmB,gBACjBC,EAAYpB,EAAQoB,cACpBjB,EAAYH,EAAQG,eAEpB5a,EAAMia,YAAcK,IAAeM,EAAUX,YAAcM,MAC3DsB,EAAYpB,EAAQoB,WAChB5+E,EAAG29E,EAAUlwD,QAAU,EACvBxtB,EAAG09E,EAAUjwD,QAAU,GAG3B9V,EAAS4lE,EAAQmB,aACb3+E,EAAGma,EAAOna,EACVC,EAAGka,EAAOla,IAIlB8iE,EAAMt1C,OAASmxD,EAAU5+E,GAAKma,EAAOna,EAAI4X,EAAO5X,GAChD+iE,EAAMr1C,OAASkxD,EAAU3+E,GAAKka,EAAOla,EAAI2X,EAAO3X,GAQpD,QAASy+E,GAAyBlB,EAASza,GACvC,GAEItW,GAAUoyB,EAAWC,EAAW31D,EAFhC41D,EAAOvB,EAAQwB,cAAgBjc,EAC/Bob,EAAYpb,EAAMmb,UAAYa,EAAKb,SAGvC,IAAInb,EAAMia,WAAaO,KAAiBY,EAAYc,IAAoBF,EAAKtyB,WAAah4D,GAAY,CAClG,GAAIg5B,GAASsxD,EAAKtxD,OAASs1C,EAAMt1C,OAC7BC,EAASqxD,EAAKrxD,OAASq1C,EAAMr1C,OAE7B5zB,EAAIolF,EAAYf,EAAW1wD,EAAQC,EACvCmxD,GAAY/kF,EAAEkG,EACd8+E,EAAYhlF,EAAEmG,EACdwsD,EAAY3zC,GAAIhf,EAAEkG,GAAK8Y,GAAIhf,EAAEmG,GAAMnG,EAAEkG,EAAIlG,EAAEmG,EAC3CkpB,EAAYo1D,EAAa9wD,EAAQC,GAEjC8vD,EAAQwB,aAAejc,MAGvBtW,GAAWsyB,EAAKtyB,SAChBoyB,EAAYE,EAAKF,UACjBC,EAAYC,EAAKD,UACjB31D,EAAY41D,EAAK51D,SAGrB45C,GAAMtW,SAAWA,EACjBsW,EAAM8b,UAAYA,EAClB9b,EAAM+b,UAAYA,EAClB/b,EAAM55C,UAAYA,EAQtB,QAAS20D,GAAqB/a,GAK1B,IAFA,GAAIma,MACAzpF,EAAI,EACDA,EAAIsvE,EAAMma,SAAStpF,QACtBspF,EAASzpF,IACLmX,QAAS+Q,GAAMonD,EAAMma,SAASzpF,GAAGmX,SACjCG,QAAS4Q,GAAMonD,EAAMma,SAASzpF,GAAGsX,UAErCtX,GAGJ,QACIyqF,UAAW9yD,KACX8xD,SAAUA,EACV/iE,OAAQ8jE,EAAUf,GAClBzvD,OAAQs1C,EAAMt1C,OACdC,OAAQq1C,EAAMr1C,QAStB,QAASuwD,GAAUf,GACf,GAAIU,GAAiBV,EAAStpF,MAG9B,IAAuB,IAAnBgqF,EACA,OACI59E,EAAG2b,GAAMuhE,EAAS,GAAGtyE,SACrB3K,EAAG0b,GAAMuhE,EAAS,GAAGnyE,SAK7B,KADA,GAAI/K,GAAI,EAAGC,EAAI,EAAGxM,EAAI,EACXmqF,EAAJnqF,GACHuM,GAAKk9E,EAASzpF,GAAGmX,QACjB3K,GAAKi9E,EAASzpF,GAAGsX,QACjBtX,GAGJ,QACIuM,EAAG2b,GAAM3b,EAAI49E,GACb39E,EAAG0b,GAAM1b,EAAI29E,IAWrB,QAASsB,GAAYf,EAAWn+E,EAAGC,GAC/B,OACID,EAAGA,EAAIm+E,GAAa,EACpBl+E,EAAGA,EAAIk+E,GAAa,GAU5B,QAASI,GAAav+E,EAAGC,GACrB,MAAID,KAAMC,EACCk/E,GAGPrmE,GAAI9Y,IAAM8Y,GAAI7Y,GACPD,EAAI,EAAIo/E,GAAiBC,GAE7Bp/E,EAAI,EAAIq/E,GAAeC,GAUlC,QAAS7yB,GAAY2L,EAAIC,EAAIrkE,GACpBA,IACDA,EAAQurF,GAEZ,IAAIx/E,GAAIs4D,EAAGrkE,EAAM,IAAMokE,EAAGpkE,EAAM,IAC5BgM,EAAIq4D,EAAGrkE,EAAM,IAAMokE,EAAGpkE,EAAM,GAEhC,OAAOd,MAAKyqB,KAAM5d,EAAIA,EAAMC,EAAIA,GAUpC,QAASm+E,GAAS/lB,EAAIC,EAAIrkE,GACjBA,IACDA,EAAQurF,GAEZ,IAAIx/E,GAAIs4D,EAAGrkE,EAAM,IAAMokE,EAAGpkE,EAAM,IAC5BgM,EAAIq4D,EAAGrkE,EAAM,IAAMokE,EAAGpkE,EAAM,GAChC,OAA0B,KAAnBd,KAAK4xD,MAAM9kD,EAAGD,GAAW7M,KAAKymB,GASzC,QAAS6kE,GAAY5gF,EAAOC,GACxB,MAAOsgF,GAAStgF,EAAI,GAAIA,EAAI,GAAI2hF,IAAmBrB,EAASvgF,EAAM,GAAIA,EAAM,GAAI4hF,IAUpF,QAASj+B,GAAS3jD,EAAOC,GACrB,MAAO4uD,GAAY5uD,EAAI,GAAIA,EAAI,GAAI2hF,IAAmB/yB,EAAY7uD,EAAM,GAAIA,EAAM,GAAI4hF,IAiB1F,QAAS1C,KACL9uF,KAAKyxF,KAAOC,GACZ1xF,KAAK2xF,MAAQC,GAEb5xF,KAAK6xF,OAAQ,EACb7xF,KAAK8xF,SAAU,EAEf/D,EAAMh2E,MAAM/X,KAAM0F,WAoEtB,QAAS+oF,KACLzuF,KAAKyxF,KAAOM,GACZ/xF,KAAK2xF,MAAQK,GAEbjE,EAAMh2E,MAAM/X,KAAM0F,WAElB1F,KAAKiyF,MAASjyF,KAAKguF,QAAQuB,QAAQ2C,iBAoEvC,QAASC,KACLnyF,KAAKoyF,SAAWC,GAChBryF,KAAK2xF,MAAQW,GACbtyF,KAAKuyF,SAAU,EAEfxE,EAAMh2E,MAAM/X,KAAM0F,WAsCtB,QAAS8sF,GAAuBrE,EAAIrnF,GAChC,GAAI2rF,GAAMlqF,EAAQ4lF,EAAGuE,SACjBzzD,EAAU12B,EAAQ4lF,EAAGwE,eAMzB,OAJI7rF,IAAQuoF,GAAYC,MACpBmD,EAAMvF,EAAYuF,EAAIz+E,OAAOirB,GAAU,cAAc,KAGjDwzD,EAAKxzD,GAiBjB,QAAS0vD,KACL3uF,KAAKoyF,SAAWQ,GAChB5yF,KAAK6yF,aAEL9E,EAAMh2E,MAAM/X,KAAM0F,WA0BtB,QAASotF,GAAW3E,EAAIrnF,GACpB,GAAIisF,GAAaxqF,EAAQ4lF,EAAGuE,SACxBG,EAAY7yF,KAAK6yF,SAGrB,IAAI/rF,GAAQsoF,GAAc4D,KAAqC,IAAtBD,EAAWptF,OAEhD,MADAktF,GAAUE,EAAW,GAAGE,aAAc,GAC9BF,EAAYA,EAGxB,IAAIvtF,GACAoX,EACA+1E,EAAiBpqF,EAAQ4lF,EAAGwE,gBAC5BO,KACAxpF,EAAS1J,KAAK0J,MAQlB,IALAkT,EAAgBm2E,EAAWp/E,OAAO,SAASmqB,GACvC,MAAO8uD,GAAU9uD,EAAMp0B,OAAQA,KAI/B5C,IAASsoF,GAET,IADA5pF,EAAI,EACGA,EAAIoX,EAAcjX,QACrBktF,EAAUj2E,EAAcpX,GAAGytF,aAAc,EACzCztF,GAMR,KADAA,EAAI,EACGA,EAAImtF,EAAehtF,QAClBktF,EAAUF,EAAentF,GAAGytF,aAC5BC,EAAqBlrF,KAAK2qF,EAAentF,IAIzCsB,GAAQuoF,GAAYC,WACbuD,GAAUF,EAAentF,GAAGytF,YAEvCztF,GAGJ,OAAK0tF,GAAqBvtF,QAMtBunF,EAAYtwE,EAAc5I,OAAOk/E,GAAuB,cAAc,GACtEA,GAPJ,OAoBJ,QAASrE,KACLd,EAAMh2E,MAAM/X,KAAM0F,UAElB,IAAI2lF,GAAUI,EAAOzrF,KAAKqrF,QAASrrF,KACnCA,MAAK89B,MAAQ,GAAI6wD,GAAW3uF,KAAKguF,QAAS3C,GAC1CrrF,KAAKmzF,MAAQ,GAAIrE,GAAW9uF,KAAKguF,QAAS3C,GAyD9C,QAAS+H,GAAYpF,EAAS3mF,GAC1BrH,KAAKguF,QAAUA,EACfhuF,KAAKmpD,IAAI9hD,GAuFb,QAASgsF,GAAkBC,GAEvB,GAAIzG,EAAMyG,EAASC,IACf,MAAOA,GAGX,IAAIC,GAAU3G,EAAMyG,EAASG,IACzBC,EAAU7G,EAAMyG,EAASK,GAG7B,OAAIH,IAAWE,EACJD,GAAqB,IAAME,GAIlCH,GAAWE,EACJF,EAAUC,GAAqBE,GAItC9G,EAAMyG,EAASM,IACRA,GAGJC,GA4CX,QAASC,GAAWrlF,GAChBzO,KAAKK,GAAKmtF,IAEVxtF,KAAKguF,QAAU,KACfhuF,KAAKyO,QAAUsyD,EAAMtyD,MAAezO,KAAK+zF,UAGzC/zF,KAAKyO,QAAQ20B,OAASkpD,EAAYtsF,KAAKyO,QAAQ20B,QAAQ,GAEvDpjC,KAAKg0F,MAAQC,GAEbj0F,KAAKk0F,gBACLl0F,KAAKm0F,eAiOT,QAASC,GAASJ,GACd,MAAIA,GAAQK,GACD,SACAL,EAAQM,GACR,MACAN,EAAQO,GACR,OACAP,EAAQQ,GACR,QAEJ,GAQX,QAASC,GAAav5D,GAClB,MAAIA,IAAao2D,GACN,OACAp2D,GAAam2D,GACb,KACAn2D,GAAai2D,GACb,OACAj2D,GAAak2D,GACb,QAEJ,GASX,QAASsD,IAA6BC,EAAiBC,GACnD,GAAI5G,GAAU4G,EAAW5G,OACzB,OAAIA,GACOA,EAAQ94E,IAAIy/E,GAEhBA,EAQX,QAASE,MACLf,EAAW/7E,MAAM/X,KAAM0F,WA6D3B,QAASovF,MACLD,GAAe98E,MAAM/X,KAAM0F,WAE3B1F,KAAK+0F,GAAK,KACV/0F,KAAKg1F,GAAK,KA2Ed,QAASC,MACLJ,GAAe98E,MAAM/X,KAAM0F,WAsC/B,QAASwvF,MACLpB,EAAW/7E,MAAM/X,KAAM0F,WAEvB1F,KAAKm1F,OAAS,KACdn1F,KAAKo1F,OAAS,KAmElB,QAASC,MACLR,GAAe98E,MAAM/X,KAAM0F,WA8B/B,QAAS4vF,MACLT,GAAe98E,MAAM/X,KAAM0F,WA0D/B,QAAS6vF,MACLzB,EAAW/7E,MAAM/X,KAAM0F,WAIvB1F,KAAKw1F,OAAQ,EACbx1F,KAAKy1F,SAAU,EAEfz1F,KAAKm1F,OAAS,KACdn1F,KAAKo1F,OAAS,KACdp1F,KAAKgX,MAAQ,EAqGjB,QAASjT,IAAO6E,EAAS6F,GAGrB,MAFAA,GAAUA,MACVA,EAAQinF,YAAcpJ,EAAY79E,EAAQinF,YAAa3xF,GAAOgwF,SAAS4B,QAChE,GAAI3K,IAAQpiF,EAAS6F,GAiIhC,QAASu8E,IAAQpiF,EAAS6F,GACtBA,EAAUA,MAEVzO,KAAKyO,QAAUsyD,EAAMtyD,EAAS1K,GAAOgwF,UACrC/zF,KAAKyO,QAAQw/E,YAAcjuF,KAAKyO,QAAQw/E,aAAerlF,EAEvD5I,KAAK41F,YACL51F,KAAKuvF,WACLvvF,KAAK01F,eAEL11F,KAAK4I,QAAUA,EACf5I,KAAK80E,MAAQuZ,EAAoBruF,MACjCA,KAAKslE,YAAc,GAAI8tB,GAAYpzF,KAAMA,KAAKyO,QAAQ62D,aAEtDuwB,GAAe71F,MAAM,GAErB4rF,EAAKn9E,EAAQinF,YAAa,SAASrmF,GAC/B,GAAIulF,GAAa50F,KAAKiT,IAAI,GAAK5D,GAAK,GAAIA,EAAK,IAC7CA,GAAK,IAAMulF,EAAWkB,cAAczmF,EAAK,IACzCA,EAAK,IAAMulF,EAAWmB,eAAe1mF,EAAK,KAC3CrP,MAyOP,QAAS61F,IAAe7H,EAAS/6E,GAC7B,GAAIrK,GAAUolF,EAAQplF,OACtBgjF,GAAKoC,EAAQv/E,QAAQunF,SAAU,SAAS3uF,EAAO4O,GAC3CrN,EAAQqE,MAAMkgF,EAASvkF,EAAQqE,MAAOgJ,IAAShD,EAAM5L,EAAQ,KASrE,QAAS4uF,IAAgB3sF,EAAOoJ,GAC5B,GAAIwjF,GAAe3kF,EAAS4kF,YAAY,QACxCD,GAAaE,UAAU9sF,GAAO,GAAM,GACpC4sF,EAAaG,QAAU3jF,EACvBA,EAAKhJ,OAAO4sF,cAAcJ,GAr1E9B,GAAI3I,KAAmB,GAAI,SAAU,MAAO,KAAM,KAAM,KACpDgJ,GAAehlF,EAASM,cAAc,OAEtCw6E,GAAgB,WAEhB3+D,GAAQxoB,KAAKwoB,MACb7C,GAAM3lB,KAAK2lB,IACXsS,GAAM74B,KAAK64B,IAwSXswD,GAAY,EAeZ+I,GAAe,wCAEf5H,GAAiB,gBAAkBrlF,GACnCilF,GAAyBrB,EAAS5jF,EAAQ,kBAAoB/C,EAC9DkoF,GAAqBE,IAAiB4H,GAAaxoF,KAAKhF,UAAUC,WAElEwtF,GAAmB,QACnBC,GAAiB,MACjBC,GAAmB,QACnBC,GAAoB,SAEpB5F,GAAmB,GAEnB5B,GAAc,EACd4D,GAAa,EACb3D,GAAY,EACZC,GAAe,EAEf4B,GAAiB,EACjBC,GAAiB,EACjBC,GAAkB,EAClBC,GAAe,EACfC,GAAiB,GAEjBuF,GAAuB1F,GAAiBC,GACxC0F,GAAqBzF,GAAeC,GACpCyF,GAAgBF,GAAuBC,GAEvCvF,IAAY,IAAK,KACjBC,IAAmB,UAAW,UA4BlCzD,GAAM56E,WAKFk4E,QAAS,aAKT+C,KAAM,WACFpuF,KAAKyxF,MAAQhF,EAAkBzsF,KAAK4I,QAAS5I,KAAKyxF,KAAMzxF,KAAKkuF,YAC7DluF,KAAKoyF,UAAY3F,EAAkBzsF,KAAK0J,OAAQ1J,KAAKoyF,SAAUpyF,KAAKkuF,YACpEluF,KAAK2xF,OAASlF,EAAkBiB,EAAoB1tF,KAAK4I,SAAU5I,KAAK2xF,MAAO3xF,KAAKkuF,aAMxF56E,QAAS,WACLtT,KAAKyxF,MAAQ9E,EAAqB3sF,KAAK4I,QAAS5I,KAAKyxF,KAAMzxF,KAAKkuF,YAChEluF,KAAKoyF,UAAYzF,EAAqB3sF,KAAK0J,OAAQ1J,KAAKoyF,SAAUpyF,KAAKkuF,YACvEluF,KAAK2xF,OAAShF,EAAqBe,EAAoB1tF,KAAK4I,SAAU5I,KAAK2xF,MAAO3xF,KAAKkuF,aAoT/F,IAAI8I,KACAC,UAAW7H,GACX8H,UAAWlE,GACXmE,QAAS9H,IAGTqC,GAAuB,YACvBE,GAAsB,mBAiB1B7F,GAAQ+C,EAAYf,GAKhB1C,QAAS,SAAmB8C,GACxB,GAAIY,GAAYiI,GAAgB7I,EAAGrnF,KAG/BioF,GAAYK,IAA6B,IAAdjB,EAAG1hE,SAC9BzsB,KAAK8xF,SAAU,GAGf/C,EAAYiE,IAA2B,IAAb7E,EAAG3hE,QAC7BuiE,EAAYM,IAIXrvF,KAAK8xF,SAAY9xF,KAAK6xF,QAIvB9C,EAAYM,KACZrvF,KAAK8xF,SAAU,GAGnB9xF,KAAKsI,SAAStI,KAAKguF,QAASe,GACxBE,UAAWd,GACXgB,iBAAkBhB,GAClBiJ,YAAaT,GACb7iD,SAAUq6C,OAKtB,IAAIkJ,KACAC,YAAalI,GACbmI,YAAavE,GACbwE,UAAWnI,GACXoI,cAAenI,GACfoI,WAAYpI,IAIZqI,IACAC,EAAGnB,GACHoB,EAAGnB,GACHoB,EAAGnB,GACHoB,EAAGnB,IAGH7E,GAAyB,cACzBC,GAAwB,qCAGxBzoF,GAAOyuF,iBACPjG,GAAyB,gBACzBC,GAAwB,6CAiB5BjG,EAAQ0C,EAAmBV,GAKvB1C,QAAS,SAAmB8C,GACxB,GAAI8D,GAAQjyF,KAAKiyF,MACbgG,GAAgB,EAEhBC,EAAsB/J,EAAGrnF,KAAKy9B,cAAc/5B,QAAQ,KAAM,IAC1DukF,EAAYsI,GAAkBa,GAC9Bd,EAAcO,GAAuBxJ,EAAGiJ,cAAgBjJ,EAAGiJ,YAE3De,EAAWf,GAAeX,GAG1B2B,EAAapL,EAAQiF,EAAO9D,EAAGkK,UAAW,YAG1CtJ,GAAYK,KAA8B,IAAdjB,EAAG1hE,QAAgB0rE,GAC9B,EAAbC,IACAnG,EAAMjqF,KAAKmmF,GACXiK,EAAanG,EAAMtsF,OAAS,GAEzBopF,GAAaM,GAAYC,MAChC2I,GAAgB,GAIH,EAAbG,IAKJnG,EAAMmG,GAAcjK,EAEpBnuF,KAAKsI,SAAStI,KAAKguF,QAASe,GACxBE,SAAUgD,EACV9C,iBAAkBhB,GAClBiJ,YAAaA,EACbtjD,SAAUq6C,IAGV8J,GAEAhG,EAAM7pF,OAAOgwF,EAAY,MAKrC,IAAIE,KACAC,WAAYnJ,GACZoJ,UAAWxF,GACXyF,SAAUpJ,GACVqJ,YAAapJ,IAGb+C,GAA6B,aAC7BC,GAA6B,2CAejCvG,GAAQoG,EAAkBpE,GACtB1C,QAAS,SAAmB8C,GACxB,GAAIrnF,GAAOwxF,GAAuBnK,EAAGrnF,KAOrC,IAJIA,IAASsoF,KACTpvF,KAAKuyF,SAAU,GAGdvyF,KAAKuyF,QAAV,CAIA,GAAIG,GAAUF,EAAuBjyF,KAAKP,KAAMmuF,EAAIrnF,EAGhDA,IAAQuoF,GAAYC,KAAiBoD,EAAQ,GAAG/sF,OAAS+sF,EAAQ,GAAG/sF,SAAW,IAC/E3F,KAAKuyF,SAAU,GAGnBvyF,KAAKsI,SAAStI,KAAKguF,QAASlnF,GACxBmoF,SAAUyD,EAAQ,GAClBvD,gBAAiBuD,EAAQ,GACzB0E,YAAaX,GACb3iD,SAAUq6C,OAsBtB,IAAIwK,KACAJ,WAAYnJ,GACZoJ,UAAWxF,GACXyF,SAAUpJ,GACVqJ,YAAapJ,IAGbsD,GAAsB,2CAc1B7G,GAAQ4C,EAAYZ,GAChB1C,QAAS,SAAoB8C,GACzB,GAAIrnF,GAAO6xF,GAAgBxK,EAAGrnF,MAC1B4rF,EAAUI,EAAWvyF,KAAKP,KAAMmuF,EAAIrnF,EACnC4rF,IAIL1yF,KAAKsI,SAAStI,KAAKguF,QAASlnF,GACxBmoF,SAAUyD,EAAQ,GAClBvD,gBAAiBuD,EAAQ,GACzB0E,YAAaX,GACb3iD,SAAUq6C,OAmFtBpC,EAAQ8C,EAAiBd,GAOrB1C,QAAS,SAAoB2C,EAAS4K,EAAYC,GAC9C,GAAIV,GAAWU,EAAUzB,aAAeX,GACpCqC,EAAWD,EAAUzB,aAAeT,EAIxC,IAAIwB,EACAn4F,KAAKmzF,MAAMtB,OAAQ,MAChB,IAAIiH,IAAY94F,KAAKmzF,MAAMtB,MAC9B,MAIA+G,IAAcvJ,GAAYC,MAC1BtvF,KAAKmzF,MAAMtB,OAAQ,GAGvB7xF,KAAKsI,SAAS0lF,EAAS4K,EAAYC,IAMvCvlF,QAAS,WACLtT,KAAK89B,MAAMxqB,UACXtT,KAAKmzF,MAAM7/E,YAInB,IAAIylF,IAAwB5L,EAASoJ,GAAatpF,MAAO,eACrD+rF,GAAsBD,KAA0BvyF,EAGhDyyF,GAAuB,UACvBpF,GAAoB,OACpBD,GAA4B,eAC5BL,GAAoB,OACpBE,GAAqB,QACrBE,GAAqB,OAczBP,GAAYjgF,WAKRg2C,IAAK,SAAS9hD,GAENA,GAAS4xF,KACT5xF,EAAQrH,KAAKk5F,WAGbF,KACAh5F,KAAKguF,QAAQplF,QAAQqE,MAAM8rF,IAAyB1xF,GAExDrH,KAAKszF,QAAUjsF,EAAMk9B,cAAcr3B,QAMvC2H,OAAQ,WACJ7U,KAAKmpD,IAAInpD,KAAKguF,QAAQv/E,QAAQ62D,cAOlC4zB,QAAS,WACL,GAAI5F,KAMJ,OALA1H,GAAK5rF,KAAKguF,QAAQ0H,YAAa,SAASd,GAChCxI,EAASwI,EAAWnmF,QAAQ20B,QAASwxD,MACrCtB,EAAUA,EAAQt/E,OAAO4gF,EAAWuE,qBAGrC9F,EAAkBC,EAAQrrF,KAAK,OAO1CmxF,gBAAiB,SAAStkB,GAEtB,IAAIkkB,GAAJ,CAIA,GAAIllD,GAAWghC,EAAMhhC,SACjB5Y,EAAY45C,EAAMub,eAGtB,IAAIrwF,KAAKguF,QAAQuB,QAAQ8J,UAErB,WADAvlD,GAASzqC,gBAIb,IAAIiqF,GAAUtzF,KAAKszF,QACfgG,EAAUzM,EAAMyG,EAASC,IACzBG,EAAU7G,EAAMyG,EAASK,IACzBH,EAAU3G,EAAMyG,EAASG,GAE7B,OAAI6F,IACC5F,GAAWx4D,EAAY27D,IACvBrD,GAAWt4D,EAAY47D,GACjB92F,KAAKu5F,WAAWzlD,GAH3B,SAWJylD,WAAY,SAASzlD,GACjB9zC,KAAKguF,QAAQuB,QAAQ8J,WAAY,EACjCvlD,EAASzqC,kBA+DjB,IAAI4qF,IAAiB,EACjBO,GAAc,EACdD,GAAgB,EAChBD,GAAc,EACdkF,GAAmBlF,GACnBD,GAAkB,GAClBoF,GAAe,EAuBnB3F,GAAW3gF,WAKP4gF,YAOA5qC,IAAK,SAAS16C,GAKV,MAJAnJ,GAAOtF,KAAKyO,QAASA,GAGrBzO,KAAKguF,SAAWhuF,KAAKguF,QAAQ1oB,YAAYzwD,SAClC7U,MAQX81F,cAAe,SAASnB,GACpB,GAAIjJ,EAAeiJ,EAAiB,gBAAiB30F,MACjD,MAAOA,KAGX,IAAIk0F,GAAel0F,KAAKk0F,YAMxB,OALAS,GAAkBD,GAA6BC,EAAiB30F,MAC3Dk0F,EAAaS,EAAgBt0F,MAC9B6zF,EAAaS,EAAgBt0F,IAAMs0F,EACnCA,EAAgBmB,cAAc91F,OAE3BA,MAQX05F,kBAAmB,SAAS/E,GACxB,MAAIjJ,GAAeiJ,EAAiB,oBAAqB30F,MAC9CA,MAGX20F,EAAkBD,GAA6BC,EAAiB30F,YACzDA,MAAKk0F,aAAaS,EAAgBt0F,IAClCL,OAQX+1F,eAAgB,SAASpB,GACrB,GAAIjJ,EAAeiJ,EAAiB,iBAAkB30F,MAClD,MAAOA,KAGX,IAAIm0F,GAAcn0F,KAAKm0F,WAMvB,OALAQ,GAAkBD,GAA6BC,EAAiB30F,MAClB,KAA1CgtF,EAAQmH,EAAaQ,KACrBR,EAAYnsF,KAAK2sF,GACjBA,EAAgBoB,eAAe/1F,OAE5BA,MAQX25F,mBAAoB,SAAShF,GACzB,GAAIjJ,EAAeiJ,EAAiB,qBAAsB30F,MACtD,MAAOA,KAGX20F,GAAkBD,GAA6BC,EAAiB30F,KAChE,IAAImI,GAAQ6kF,EAAQhtF,KAAKm0F,YAAaQ,EAItC,OAHIxsF,GAAQ,IACRnI,KAAKm0F,YAAY/rF,OAAOD,EAAO,GAE5BnI,MAOX45F,mBAAoB,WAChB,MAAO55F,MAAKm0F,YAAYxuF,OAAS,GAQrCk0F,iBAAkB,SAASlF,GACvB,QAAS30F,KAAKk0F,aAAaS,EAAgBt0F,KAQ/CutB,KAAM,SAASknD,GAIX,QAASlnD,GAAKksE,GACVprB,EAAKsf,QAAQpgE,KAAK8gD,EAAKjgE,QAAQnF,OAASwwF,EAAY1F,EAASJ,GAAS,IAAKlf,GAJ/E,GAAIpG,GAAO1uE,KACPg0F,EAAQh0F,KAAKg0F,KAOLM,IAARN,GACApmE,GAAK,GAGTA,IAGIomE,GAASM,IACT1mE,GAAK,IAUbmsE,QAAS,SAASjlB,GACd,MAAI90E,MAAKg6F,UACEh6F,KAAK4tB,KAAKknD,QAGrB90E,KAAKg0F,MAAQyF,KAOjBO,QAAS,WAEL,IADA,GAAIx0F,GAAI,EACDA,EAAIxF,KAAKm0F,YAAYxuF,QAAQ,CAChC,KAAM3F,KAAKm0F,YAAY3uF,GAAGwuF,OAASyF,GAAexF,KAC9C,OAAO,CAEXzuF,KAEJ,OAAO,GAOXiqF,UAAW,SAASoJ,GAGhB,GAAIoB,GAAiB30F,KAAWuzF,EAGhC,OAAKzM,GAASpsF,KAAKyO,QAAQ20B,QAASpjC,KAAMi6F,KAOtCj6F,KAAKg0F,OAASwF,GAAmBnF,GAAkBoF,MACnDz5F,KAAKg0F,MAAQC,IAGjBj0F,KAAKg0F,MAAQh0F,KAAKk6F,QAAQD,QAItBj6F,KAAKg0F,OAASQ,GAAcD,GAAgBD,GAAcD,KAC1Dr0F,KAAK+5F,QAAQE,MAfbj6F,KAAKwpD,aACLxpD,KAAKg0F,MAAQyF,MAyBrBS,QAAS,aAOTf,eAAgB,aAOhB3vC,MAAO,cA8DXuiC,EAAQ8I,GAAgBf,GAKpBC,UAKI9E,SAAU,GASdkL,SAAU,SAASrlB,GACf,GAAIslB,GAAiBp6F,KAAKyO,QAAQwgF,QAClC,OAA0B,KAAnBmL,GAAwBtlB,EAAMma,SAAStpF,SAAWy0F,GAS7DF,QAAS,SAASplB,GACd,GAAIkf,GAAQh0F,KAAKg0F,MACbjF,EAAYja,EAAMia,UAElBsL,EAAerG,GAASQ,GAAcD,IACtCld,EAAUr3E,KAAKm6F,SAASrlB,EAG5B,OAAIulB,KAAiBtL,EAAYO,KAAiBjY,GACvC2c,EAAQK,GACRgG,GAAgBhjB,EACnB0X,EAAYM,GACL2E,EAAQM,GACNN,EAAQQ,GAGdR,EAAQO,GAFJC,GAIRiF,MAiBf1N,EAAQ+I,GAAeD,IAKnBd,UACIzqF,MAAO,MACP0uD,UAAW,GACXi3B,SAAU,EACV/zD,UAAW67D,IAGfoC,eAAgB,WACZ,GAAIj+D,GAAYl7B,KAAKyO,QAAQysB,UACzBo4D,IAOJ,OANIp4D,GAAY27D,IACZvD,EAAQtrF,KAAK2rF,IAEbz4D,EAAY47D,IACZxD,EAAQtrF,KAAKyrF,IAEVH,GAGXgH,cAAe,SAASxlB,GACpB,GAAIrmE,GAAUzO,KAAKyO,QACf8rF,GAAW,EACX50E,EAAWmvD,EAAMnvD,SACjBuV,EAAY45C,EAAM55C,UAClBnpB,EAAI+iE,EAAMt1C,OACVxtB,EAAI8iE,EAAMr1C,MAed,OAZMvE,GAAYzsB,EAAQysB,YAClBzsB,EAAQysB,UAAY27D,IACpB37D,EAAmB,IAANnpB,EAAWm/E,GAAsB,EAAJn/E,EAASo/E,GAAiBC,GACpEmJ,EAAWxoF,GAAK/R,KAAK+0F,GACrBpvE,EAAWzgB,KAAK2lB,IAAIiqD,EAAMt1C,UAE1BtE,EAAmB,IAANlpB,EAAWk/E,GAAsB,EAAJl/E,EAASq/E,GAAeC,GAClEiJ,EAAWvoF,GAAKhS,KAAKg1F,GACrBrvE,EAAWzgB,KAAK2lB,IAAIiqD,EAAMr1C,UAGlCq1C,EAAM55C,UAAYA,EACXq/D,GAAY50E,EAAWlX,EAAQupD,WAAa98B,EAAYzsB,EAAQysB,WAG3Ei/D,SAAU,SAASrlB,GACf,MAAO+f,IAAe1hF,UAAUgnF,SAAS55F,KAAKP,KAAM80E,KAC/C90E,KAAKg0F,MAAQQ,MAAkBx0F,KAAKg0F,MAAQQ,KAAgBx0F,KAAKs6F,cAAcxlB,KAGxFlnD,KAAM,SAASknD,GACX90E,KAAK+0F,GAAKjgB,EAAMt1C,OAChBx/B,KAAKg1F,GAAKlgB,EAAMr1C,MAEhB,IAAIvE,GAAYu5D,EAAa3f,EAAM55C,UAC/BA,IACAl7B,KAAKguF,QAAQpgE,KAAK5tB,KAAKyO,QAAQnF,MAAQ4xB,EAAW45C,GAGtD90E,KAAKmsF,OAAOv+D,KAAKrtB,KAAKP,KAAM80E,MAcpCiX,EAAQkJ,GAAiBJ,IAKrBd,UACIzqF,MAAO,QACP0uD,UAAW,EACXi3B,SAAU,GAGdkK,eAAgB,WACZ,OAAQ5F,KAGZ4G,SAAU,SAASrlB,GACf,MAAO90E,MAAKmsF,OAAOgO,SAAS55F,KAAKP,KAAM80E,KAClC5vE,KAAK2lB,IAAIiqD,EAAM73D,MAAQ,GAAKjd,KAAKyO,QAAQupD,WAAah4D,KAAKg0F,MAAQQ,KAG5E5mE,KAAM,SAASknD,GAEX,GADA90E,KAAKmsF,OAAOv+D,KAAKrtB,KAAKP,KAAM80E,GACR,IAAhBA,EAAM73D,MAAa,CACnB,GAAIu9E,GAAQ1lB,EAAM73D,MAAQ,EAAI,KAAO,KACrCjd,MAAKguF,QAAQpgE,KAAK5tB,KAAKyO,QAAQnF,MAAQkxF,EAAO1lB,OAkB1DiX,EAAQmJ,GAAiBpB,GAKrBC,UACIzqF,MAAO,QACP2lF,SAAU,EACV90D,KAAM,IACN69B,UAAW,GAGfmhC,eAAgB,WACZ,OAAQtF,KAGZqG,QAAS,SAASplB,GACd,GAAIrmE,GAAUzO,KAAKyO,QACfgsF,EAAgB3lB,EAAMma,SAAStpF,SAAW8I,EAAQwgF,SAClDyL,EAAgB5lB,EAAMnvD,SAAWlX,EAAQupD,UACzC2iC,EAAY7lB,EAAMob,UAAYzhF,EAAQ0rB,IAM1C,IAJAn6B,KAAKo1F,OAAStgB,GAIT4lB,IAAkBD,GAAkB3lB,EAAMia,WAAaM,GAAYC,MAAkBqL,EACtF36F,KAAKwpD,YACF,IAAIsrB,EAAMia,UAAYK,GACzBpvF,KAAKwpD,QACLxpD,KAAKm1F,OAAS5J,EAAkB,WAC5BvrF,KAAKg0F,MAAQwF,GACbx5F,KAAK+5F;EACNtrF,EAAQ0rB,KAAMn6B,UACd,IAAI80E,EAAMia,UAAYM,GACzB,MAAOmK,GAEX,OAAOC,KAGXjwC,MAAO,WACHnwC,aAAarZ,KAAKm1F,SAGtBvnE,KAAM,SAASknD,GACP90E,KAAKg0F,QAAUwF,KAIf1kB,GAAUA,EAAMia,UAAYM,GAC5BrvF,KAAKguF,QAAQpgE,KAAK5tB,KAAKyO,QAAQnF,MAAQ,KAAMwrE,IAE7C90E,KAAKo1F,OAAOnF,UAAY9yD,KACxBn9B,KAAKguF,QAAQpgE,KAAK5tB,KAAKyO,QAAQnF,MAAOtJ,KAAKo1F,aAevDrJ,EAAQsJ,GAAkBR,IAKtBd,UACIzqF,MAAO,SACP0uD,UAAW,EACXi3B,SAAU,GAGdkK,eAAgB,WACZ,OAAQ5F,KAGZ4G,SAAU,SAASrlB,GACf,MAAO90E,MAAKmsF,OAAOgO,SAAS55F,KAAKP,KAAM80E,KAClC5vE,KAAK2lB,IAAIiqD,EAAMyb,UAAYvwF,KAAKyO,QAAQupD,WAAah4D,KAAKg0F,MAAQQ,OAc/EzI,EAAQuJ,GAAiBT,IAKrBd,UACIzqF,MAAO,QACP0uD,UAAW,GACXwG,SAAU,IACVtjC,UAAW27D,GAAuBC,GAClC7H,SAAU,GAGdkK,eAAgB,WACZ,MAAOrE,IAAc3hF,UAAUgmF,eAAe54F,KAAKP,OAGvDm6F,SAAU,SAASrlB,GACf,GACItW,GADAtjC,EAAYl7B,KAAKyO,QAAQysB,SAW7B,OARIA,IAAa27D,GAAuBC,IACpCt4B,EAAWsW,EAAMtW,SACVtjC,EAAY27D,GACnBr4B,EAAWsW,EAAM8b,UACV11D,EAAY47D,KACnBt4B,EAAWsW,EAAM+b,WAGd7wF,KAAKmsF,OAAOgO,SAAS55F,KAAKP,KAAM80E,IACnC55C,EAAY45C,EAAM55C,WAClB45C,EAAMnvD,SAAW3lB,KAAKyO,QAAQupD,WAC9BntC,GAAI2zC,GAAYx+D,KAAKyO,QAAQ+vD,UAAYsW,EAAMia,UAAYM,IAGnEzhE,KAAM,SAASknD,GACX,GAAI55C,GAAYu5D,EAAa3f,EAAM55C,UAC/BA,IACAl7B,KAAKguF,QAAQpgE,KAAK5tB,KAAKyO,QAAQnF,MAAQ4xB,EAAW45C,GAGtD90E,KAAKguF,QAAQpgE,KAAK5tB,KAAKyO,QAAQnF,MAAOwrE,MA2B9CiX,EAAQwJ,GAAezB,GAKnBC,UACIzqF,MAAO,MACP2lF,SAAU,EACV2L,KAAM,EACNpoE,SAAU,IACV2H,KAAM,IACN69B,UAAW,EACX6iC,aAAc,IAGlB1B,eAAgB,WACZ,OAAQvF,KAGZsG,QAAS,SAASplB,GACd,GAAIrmE,GAAUzO,KAAKyO,QAEfgsF,EAAgB3lB,EAAMma,SAAStpF,SAAW8I,EAAQwgF,SAClDyL,EAAgB5lB,EAAMnvD,SAAWlX,EAAQupD,UACzC8iC,EAAiBhmB,EAAMob,UAAYzhF,EAAQ0rB,IAI/C,IAFAn6B,KAAKwpD,QAEAsrB,EAAMia,UAAYK,IAAgC,IAAfpvF,KAAKgX,MACzC,MAAOhX,MAAK+6F,aAKhB,IAAIL,GAAiBI,GAAkBL,EAAe,CAClD,GAAI3lB,EAAMia,WAAaM,GACnB,MAAOrvF,MAAK+6F,aAGhB,IAAIC,GAAgBh7F,KAAKw1F,MAAS1gB,EAAMmb,UAAYjwF,KAAKw1F,MAAQ/mF,EAAQ+jB,UAAY,EACjFyoE,GAAiBj7F,KAAKy1F,SAAWh3B,EAAYz+D,KAAKy1F,QAAS3gB,EAAM5oD,QAAUzd,EAAQosF,YAEvF76F,MAAKw1F,MAAQ1gB,EAAMmb,UACnBjwF,KAAKy1F,QAAU3gB,EAAM5oD,OAEhB+uE,GAAkBD,EAGnBh7F,KAAKgX,OAAS,EAFdhX,KAAKgX,MAAQ,EAKjBhX,KAAKo1F,OAAStgB,CAId,IAAIomB,GAAWl7F,KAAKgX,MAAQvI,EAAQmsF,IACpC,IAAiB,IAAbM,EAGA,MAAKl7F,MAAK45F,sBAGN55F,KAAKm1F,OAAS5J,EAAkB,WAC5BvrF,KAAKg0F,MAAQwF,GACbx5F,KAAK+5F,WACNtrF,EAAQ+jB,SAAUxyB,MACdw0F,IANAgF,GAUnB,MAAOC,KAGXsB,YAAa,WAIT,MAHA/6F,MAAKm1F,OAAS5J,EAAkB,WAC5BvrF,KAAKg0F,MAAQyF,IACdz5F,KAAKyO,QAAQ+jB,SAAUxyB,MACnBy5F,IAGXjwC,MAAO,WACHnwC,aAAarZ,KAAKm1F,SAGtBvnE,KAAM,WACE5tB,KAAKg0F,OAASwF,KACdx5F,KAAKo1F,OAAO8F,SAAWl7F,KAAKgX,MAC5BhX,KAAKguF,QAAQpgE,KAAK5tB,KAAKyO,QAAQnF,MAAOtJ,KAAKo1F,YAoBvDrxF,GAAOm8E,QAAU,QAMjBn8E,GAAOgwF,UAOHoH,WAAW,EAQX71B,YAAa2zB,GAMb71D,QAAQ,EASR6qD,YAAa,KAObM,WAAY,KAOZoH,SAEKN,IAAoBjyD,QAAQ,KAC5B6xD,IAAmB7xD,QAAQ,IAAU,YACrCkyD,IAAkBp6D,UAAW27D,MAC7B/B,IAAiB55D,UAAW27D,KAAyB,WACrDtB,KACAA,IAAiBjsF,MAAO,YAAasxF,KAAM,IAAM,SACjD1F,KAQLc,UAMIoF,WAAY,OAOZC,YAAa,OASbC,aAAc,OAOdC,eAAgB,OAOhBC,SAAU,OAQVC,kBAAmB,iBAI3B,IAAIC,IAAO,EACPC,GAAc,CA+BlB3Q,IAAQ73E,WAMJg2C,IAAK,SAAS16C,GAaV,MAZAnJ,GAAOtF,KAAKyO,QAASA,GAGjBA,EAAQ62D,aACRtlE,KAAKslE,YAAYzwD,SAEjBpG,EAAQw/E,cAERjuF,KAAK80E,MAAMxhE,UACXtT,KAAK80E,MAAMprE,OAAS+E,EAAQw/E,YAC5BjuF,KAAK80E,MAAMsZ,QAERpuF,MASXklB,KAAM,SAAS0b,GACX5gC,KAAKuvF,QAAQ1E,QAAUjqD,EAAQ+6D,GAAcD,IASjDjM,UAAW,SAASoJ,GAChB,GAAItJ,GAAUvvF,KAAKuvF,OACnB,KAAIA,EAAQ1E,QAAZ,CAKA7qF,KAAKslE,YAAY8zB,gBAAgBP,EAEjC,IAAIjE,GACAc,EAAc11F,KAAK01F,YAKnBkG,EAAgBrM,EAAQqM,gBAIvBA,GAAkBA,GAAiBA,EAAc5H,MAAQwF,MAC1DoC,EAAgBrM,EAAQqM,cAAgB,KAI5C,KADA,GAAIp2F,GAAI,EACDA,EAAIkwF,EAAY/vF,QACnBivF,EAAac,EAAYlwF,GAQrB+pF,EAAQ1E,UAAY8Q,IACfC,GAAiBhH,GAAcgH,IAChChH,EAAWiF,iBAAiB+B,GAGhChH,EAAWprC,QAFXorC,EAAWnF,UAAUoJ,IAOpB+C,GAAiBhH,EAAWZ,OAASQ,GAAcD,GAAgBD,MACpEsH,EAAgBrM,EAAQqM,cAAgBhH,GAE5CpvF,MASR0P,IAAK,SAAS0/E,GACV,GAAIA,YAAsBd,GACtB,MAAOc,EAIX,KAAK,GADDc,GAAc11F,KAAK01F,YACdlwF,EAAI,EAAGA,EAAIkwF,EAAY/vF,OAAQH,IACpC,GAAIkwF,EAAYlwF,GAAGiJ,QAAQnF,OAASsrF,EAChC,MAAOc,GAAYlwF,EAG3B,OAAO,OASXyN,IAAK,SAAS2hF,GACV,GAAIlJ,EAAekJ,EAAY,MAAO50F,MAClC,MAAOA,KAIX,IAAI+hD,GAAW/hD,KAAKkV,IAAI0/E,EAAWnmF,QAAQnF,MAS3C,OARIy4C,IACA/hD,KAAKqW,OAAO0rC,GAGhB/hD,KAAK01F,YAAY1tF,KAAK4sF,GACtBA,EAAW5G,QAAUhuF,KAErBA,KAAKslE,YAAYzwD,SACV+/E,GAQXv+E,OAAQ,SAASu+E,GACb,GAAIlJ,EAAekJ,EAAY,SAAU50F,MACrC,MAAOA,KAGX,IAAI01F,GAAc11F,KAAK01F,WAKvB,OAJAd,GAAa50F,KAAKkV,IAAI0/E,GACtBc,EAAYttF,OAAO4kF,EAAQ0I,EAAad,GAAa,GAErD50F,KAAKslE,YAAYzwD,SACV7U,MASXuT,GAAI,SAASiyD,EAAQ6lB,GACjB,GAAIuK,GAAW51F,KAAK41F,QAKpB,OAJAhK,GAAKc,EAASlnB,GAAS,SAASl8D,GAC5BssF,EAAStsF,GAASssF,EAAStsF,OAC3BssF,EAAStsF,GAAOtB,KAAKqjF,KAElBrrF,MASX0T,IAAK,SAAS8xD,EAAQ6lB,GAClB,GAAIuK,GAAW51F,KAAK41F,QAQpB,OAPAhK,GAAKc,EAASlnB,GAAS,SAASl8D,GACvB+hF,EAGDuK,EAAStsF,GAAOlB,OAAO4kF,EAAQ4I,EAAStsF,GAAQ+hF,GAAU,SAFnDuK,GAAStsF,KAKjBtJ,MAQX4tB,KAAM,SAAStkB,EAAOoJ,GAEd1S,KAAKyO,QAAQ0sF,WACblF,GAAgB3sF,EAAOoJ,EAI3B,IAAIkjF,GAAW51F,KAAK41F,SAAStsF,IAAUtJ,KAAK41F,SAAStsF,GAAO2B,OAC5D,IAAK2qF,GAAaA,EAASjwF,OAA3B,CAIA+M,EAAK5L,KAAOwC,EACZoJ,EAAKrJ,eAAiB,WAClBqJ,EAAKohC,SAASzqC,iBAIlB,KADA,GAAI7D,GAAI,EACDA,EAAIowF,EAASjwF,QAChBiwF,EAASpwF,GAAGkN,GACZlN,MAQR8N,QAAS,WACLtT,KAAK4I,SAAWitF,GAAe71F,MAAM,GAErCA,KAAK41F,YACL51F,KAAKuvF,WACLvvF,KAAK80E,MAAMxhE,UACXtT,KAAK4I,QAAU,OA4BvBtD,EAAOvB,IACHqrF,YAAaA,GACb4D,WAAYA,GACZ3D,UAAWA,GACXC,aAAcA,GAEd2E,eAAgBA,GAChBO,YAAaA,GACbD,cAAeA,GACfD,YAAaA,GACbkF,iBAAkBA,GAClBnF,gBAAiBA,GACjBoF,aAAcA,GAEdvI,eAAgBA,GAChBC,eAAgBA,GAChBC,gBAAiBA,GACjBC,aAAcA,GACdC,eAAgBA,GAChBuF,qBAAsBA,GACtBC,mBAAoBA,GACpBC,cAAeA,GAEf/L,QAASA,GACT+C,MAAOA,EACPqF,YAAaA,EAEbzE,WAAYA,EACZG,WAAYA,EACZL,kBAAmBA,EACnBI,gBAAiBA,EACjBsD,iBAAkBA,EAElB2B,WAAYA,EACZe,eAAgBA,GAChBgH,IAAKtG,GACLuG,IAAKhH,GACLiH,MAAOzG,GACP0G,MAAO/G,GACPgH,OAAQ5G,GACR6G,MAAOhH,GAEP3hF,GAAIk5E,EACJ/4E,IAAKi5E,EACLf,KAAMA,EACN7qB,MAAOA,EACPz7D,OAAQA,EACRymF,QAASA,EACTN,OAAQA,EACR0B,SAAUA,IAGV,YAAcd,IAAiBnsF,EAAoB,KACjD8uE,EAAgC,WAC9B,MAAOjrE,KACTxD,KAAKX,EAASM,EAAqBN,EAASC,KAASmvE,IAAkCxoE,IAAc3G,EAAOD,QAAUovE,KAChG,mBAAVnvE,IAAyBA,EAAOD,QAC9CC,EAAOD,QAAUmE,GAEjBwF,EAAO+hF,GAAcvnF,IAGtBwF,OAAQgI,SAAU,WAKjB,SAAS1R,EAAQD,GAYrBA,EAAQslD,oBAAsB,WAE7BllD,KAAKm8F,aAAan8F,KAAK0hD,UAAUvC,WAAWC,iBAAiB,GAG7Dp/C,KAAKwuD,eAIDxuD,KAAKmhD,WACPnhD,KAAK6nD,aAEP7nD,KAAK4P,SASNhQ,EAAQu8F,aAAe,SAASC,EAAkBC,GAOhD,IANA,GAAI31C,GAAgB1mD,KAAK8jD,YAAYn+C,OAEjC22F,EAAY,GACZ1+C,EAAQ,EAGL8I,EAAgB01C,GAA4BE,EAAR1+C,GACrCA,EAAQ,GAAK,GACf59C,KAAKu8F,oBAAmB,GACxBv8F,KAAKw8F,0BAGLx8F,KAAKy8F,uBAGP/1C,EAAgB1mD,KAAK8jD,YAAYn+C,OACjCi4C,GAAS,CAIPA,GAAQ,GAAmB,GAAdy+C,GACfr8F,KAAK08F,kBAEP18F,KAAKquD,2BASPzuD,EAAQ+8F,YAAc,SAAS72C,GAC7B,GAAI82C,GAA2B58F,KAAK8kD,MACpC,IAAIgB,EAAKuW,YAAcr8D,KAAK0hD,UAAUvC,WAAWM,iBAAmBz/C,KAAK68F,kBAAkB/2C,KACrE,WAAlB9lD,KAAK88F,WAAqD,GAA3B98F,KAAK8jD,YAAYn+C,QAAc,CAEhE3F,KAAK+8F,WAAWj3C,EAIhB,KAHA,GAAIlI,GAAQ,EAGJ59C,KAAK8jD,YAAYn+C,OAAS3F,KAAK0hD,UAAUvC,WAAWC,iBAA6B,GAARxB,GAC/E59C,KAAKg9F,uBACLp/C,GAAS,MAKX59C,MAAKi9F,mBAAmBn3C,GAAK,GAAM,GAGnC9lD,KAAKgnD,uBACLhnD,KAAKk9F,sBACLl9F,KAAKquD,0BACLruD,KAAKwuD,cAIHxuD,MAAK8kD,QAAU83C,GACjB58F,KAAK4P,SAQThQ,EAAQ4sD,sBAAwB,WACW,GAArCxsD,KAAK0hD,UAAUvC,WAAWzwC,SAC5B1O,KAAKm9F,eAAe,GAAE,GAAM,IAUhCv9F,EAAQ68F,qBAAuB,WAC7Bz8F,KAAKm9F,eAAe,IAAG,GAAM,IAS/Bv9F,EAAQo9F,qBAAuB,WAC7Bh9F,KAAKm9F,eAAe,GAAE,GAAM,IAgB9Bv9F,EAAQu9F,eAAiB,SAASC,EAAcC,EAAUz8D,EAAM08D,GAC9D,GAAIV,GAA2B58F,KAAK8kD,OAChCy4C,EAAgBv9F,KAAK8jD,YAAYn+C,MAGjC3F,MAAKmkD,cAAgBnkD,KAAKid,OAA0B,GAAjBmgF,GACrCp9F,KAAKw9F,kBAIHx9F,KAAKmkD,cAAgBnkD,KAAKid,OAA0B,IAAjBmgF,EAGrCp9F,KAAKy9F,cAAc78D,IAEZ5gC,KAAKmkD,cAAgBnkD,KAAKid,OAA0B,GAAjBmgF,KAC7B,GAATx8D,EAGF5gC,KAAK09F,cAAcL,EAAUz8D,GAI7B5gC,KAAK29F,uBAGT39F,KAAKgnD,uBAGDhnD,KAAK8jD,YAAYn+C,QAAU43F,IAAkBv9F,KAAKmkD,cAAgBnkD,KAAKid,OAA0B,IAAjBmgF,KAClFp9F,KAAK49F,eAAeh9D,GACpB5gC,KAAKgnD,yBAIHhnD,KAAKmkD,cAAgBnkD,KAAKid,OAA0B,IAAjBmgF,KACrCp9F,KAAK69F,eACL79F,KAAKgnD,wBAGPhnD,KAAKmkD,cAAgBnkD,KAAKid,MAG1Bjd,KAAKk9F,sBACLl9F,KAAKwuD,eAGDxuD,KAAK8jD,YAAYn+C,OAAS43F,IAC5Bv9F,KAAK87D,gBAAkB,EAEvB97D,KAAKw8F,2BAGW,GAAdc,GAAsC92F,SAAf82F,IAErBt9F,KAAK8kD,QAAU83C,GACjB58F,KAAK4P,QAIT5P,KAAKquD,2BAMPzuD,EAAQi+F,aAAe,WAErB,GAAIC,GAAkB99F,KAAK+9F,mBACvBD,GAAkB99F,KAAK0hD,UAAUvC,WAAWI,gBAC9Cv/C,KAAKg+F,sBAAsB,EAAIh+F,KAAK0hD,UAAUvC,WAAWI,eAAiBu+C,IAW9El+F,EAAQg+F,eAAiB,SAASh9D,GAChC5gC,KAAKi+F,cACLj+F,KAAKk+F,mBAAmBt9D,GAAM,IAQhChhC,EAAQ28F,mBAAqB,SAASe,GACpC,GAAIV,GAA2B58F,KAAK8kD,OAChCy4C,EAAgBv9F,KAAK8jD,YAAYn+C,MAErC3F,MAAK49F,gBAAe,GAGpB59F,KAAKgnD,uBACLhnD,KAAKk9F,sBACLl9F,KAAKwuD,eAGDxuD,KAAK8jD,YAAYn+C,QAAU43F,IAC7Bv9F,KAAK87D,gBAAkB,IAGP,GAAdwhC,GAAsC92F,SAAf82F,IAErBt9F,KAAK8kD,QAAU83C,GACjB58F,KAAK4P,SAUXhQ,EAAQ+9F,oBAAsB,WAC5B,IAAK,GAAIx3C,KAAUnmD,MAAKg9C,MACtB,GAAIh9C,KAAKg9C,MAAMl3C,eAAeqgD,GAAS,CACrC,GAAIL,GAAO9lD,KAAKg9C,MAAMmJ,EACD,IAAjBL,EAAKma,WACFna,EAAKvzC,MAAMvS,KAAKid,MAAQjd,KAAK0hD,UAAUvC,WAAWO,oBAAsB1/C,KAAKsf,MAAMC,OAAOC,aAC1FsmC,EAAKtzC,OAAOxS,KAAKid,MAAQjd,KAAK0hD,UAAUvC,WAAWO,oBAAsB1/C,KAAKsf,MAAMC,OAAOsF,eAC9F7kB,KAAK28F,YAAY72C,KAc3BlmD,EAAQ89F,cAAgB,SAASL,EAAUz8D,GACzC,IAAK,GAAIp7B,GAAI,EAAGA,EAAIxF,KAAK8jD,YAAYn+C,OAAQH,IAAK,CAChD,GAAIsgD,GAAO9lD,KAAKg9C,MAAMh9C,KAAK8jD,YAAYt+C,GACvCxF,MAAKi9F,mBAAmBn3C,EAAKu3C,EAAUz8D,GACvC5gC,KAAKquD,4BAeTzuD,EAAQq9F,mBAAqB,SAASpzF,EAAYwzF,EAAWz8D,EAAOu9D,GAElE,GAAIt0F,EAAWwyD,YAAc,IAEvBxyD,EAAWwyD,YAAcr8D,KAAK0hD,UAAUvC,WAAWM,kBACrD0+C,GAAU,GAEZd,EAAYc,GAAU,EAAOd,EAGzBxzF,EAAWuyD,eAAiBp8D,KAAKid,OAAkB,GAAT2jB,GAE5C,IAAK,GAAIw9D,KAAmBv0F,GAAWyyD,eACrC,GAAIzyD,EAAWyyD,eAAex2D,eAAes4F,GAAkB,CAC7D,GAAIC,GAAYx0F,EAAWyyD,eAAe8hC,EAI7B,IAATx9D,GACEy9D,EAAUviC,gBAAkBjyD,EAAW2yD,gBAAgB3yD,EAAW2yD,gBAAgB72D,OAAO,IACtFw4F,IACLn+F,KAAKs+F,sBAAsBz0F,EAAWu0F,EAAgBf,EAAUz8D,EAAMu9D,GAIpEn+F,KAAK68F,kBAAkBhzF,IACzB7J,KAAKs+F,sBAAsBz0F,EAAWu0F,EAAgBf,EAAUz8D,EAAMu9D,KAwBpFv+F,EAAQ0+F,sBAAwB,SAASz0F,EAAYu0F,EAAiBf,EAAWz8D,EAAOu9D,GACtF,GAAIE,GAAYx0F,EAAWyyD,eAAe8hC,EAG1C,IAAIC,EAAUjiC,eAAiBp8D,KAAKid,OAAkB,GAAT2jB,EAAe,CAE1D5gC,KAAKu+F,eAGLv+F,KAAKg9C,MAAMohD,GAAmBC,EAG9Br+F,KAAKw+F,uBAAuB30F,EAAWw0F,GAGvCr+F,KAAKy+F,wBAAwB50F,EAAWw0F,GAGxCr+F,KAAK0+F,eAAe70F,GAGpBA,EAAW4E,QAAQwuC,MAAQohD,EAAU5vF,QAAQwuC,KAC7CpzC,EAAWwyD,aAAegiC,EAAUhiC,YACpCxyD,EAAW4E,QAAQ8uC,SAAWr4C,KAAK4G,IAAI9L,KAAK0hD,UAAUvC,WAAWS,YAAa5/C,KAAK0hD,UAAU1E,MAAMO,SAAWv9C,KAAK0hD,UAAUvC,WAAWQ,oBAAoB91C,EAAWwyD,YAAY,IACnLxyD,EAAWgyD,mBAAqBhyD,EAAWslD,aAAaxpD,OAGxD04F,EAAUtsF,EAAIlI,EAAWkI,EAAIlI,EAAWqyD,iBAAmB,GAAMh3D,KAAKE,UACtEi5F,EAAUrsF,EAAInI,EAAWmI,EAAInI,EAAWqyD,iBAAmB,GAAMh3D,KAAKE,gBAG/DyE,GAAWyyD,eAAe8hC,EAGjC,IAAIO,IAAgB,CACpB,KAAK,GAAIC,KAAe/0F,GAAWyyD,eACjC,GAAIzyD,EAAWyyD,eAAex2D,eAAe84F,IACvC/0F,EAAWyyD,eAAesiC,GAAa9iC,gBAAkBuiC,EAAUviC,eAAgB,CACrF6iC,GAAgB,CAChB,OAKe,GAAjBA,GACF90F,EAAW2yD,gBAAgBniB,MAG7Br6C,KAAK6+F,uBAAuBR,GAI5BA,EAAUviC,eAAiB,EAG3BjyD,EAAWs0D,iBAGXn+D,KAAK8kD,QAAS,EAIC,GAAbu4C,GACFr9F,KAAKi9F,mBAAmBoB,EAAUhB,EAAUz8D,EAAMu9D,IAWtDv+F,EAAQi/F,uBAAyB,SAAS/4C,GACxC,IAAK,GAAItgD,GAAI,EAAGA,EAAIsgD,EAAKqJ,aAAaxpD,OAAQH,IAC5CsgD,EAAKqJ,aAAa3pD,GAAG2sD,sBAczBvyD,EAAQ69F,cAAgB,SAAS78D,GAClB,GAATA,EACF5gC,KAAK8+F,sBAGL9+F,KAAK++F,wBAUTn/F,EAAQk/F,oBAAsB,WAC5B,GAAIlgF,GAAGC,EAAGlZ,EACNq5F,EAAYh/F,KAAK0hD,UAAUvC,WAAWK,qBAAqBx/C,KAAKid,KAIpE,KAAK,GAAI+vC,KAAUhtD,MAAK89C,MACtB,GAAI99C,KAAK89C,MAAMh4C,eAAeknD,GAAS,CACrC,GAAIU,GAAO1tD,KAAK89C,MAAMkP,EACtB,IAAIU,EAAKC,WACHD,EAAKkG,MAAQlG,EAAKiG,SACpB/0C,EAAM8uC,EAAKrkC,GAAGtX,EAAI27C,EAAKtkC,KAAKrX,EAC5B8M,EAAM6uC,EAAKrkC,GAAGrX,EAAI07C,EAAKtkC,KAAKpX,EAC5BrM,EAAST,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAGrBmgF,EAATr5F,GAAoB,CAEtB,GAAIkE,GAAa6jD,EAAKtkC,KAClBi1E,EAAY3wC,EAAKrkC,EACjBqkC,GAAKrkC,GAAG5a,QAAQwuC,KAAOyQ,EAAKtkC,KAAK3a,QAAQwuC,OAC3CpzC,EAAa6jD,EAAKrkC,GAClBg1E,EAAY3wC,EAAKtkC,MAGiB,GAAhCi1E,EAAUxiC,mBACZ77D,KAAKi/F,cAAcp1F,EAAWw0F,GAAU,GAEA,GAAjCx0F,EAAWgyD,oBAClB77D,KAAKi/F,cAAcZ,EAAUx0F,GAAW,MAetDjK,EAAQm/F,qBAAuB,WAC7B,IAAK,GAAI54C,KAAUnmD,MAAKg9C,MAEtB,GAAIh9C,KAAKg9C,MAAMl3C,eAAeqgD,GAAS,CACrC,GAAIk4C,GAAYr+F,KAAKg9C,MAAMmJ,EAG3B,IAAoC,GAAhCk4C,EAAUxiC,oBAA4D,GAAjCwiC,EAAUlvC,aAAaxpD,OAAa,CAC3E,GAAI+nD,GAAO2wC,EAAUlvC,aAAa,GAC9BtlD,EAAc6jD,EAAKkG,MAAQyqC,EAAUh+F,GAAML,KAAKg9C,MAAM0Q,EAAKiG,QAAU3zD,KAAKg9C,MAAM0Q,EAAKkG,KAGrFyqC,GAAUh+F,IAAMwJ,EAAWxJ,KACzBwJ,EAAW4E,QAAQwuC,KAAOohD,EAAU5vF,QAAQwuC,KAC9Cj9C,KAAKi/F,cAAcp1F,EAAWw0F,GAAU,GAGxCr+F,KAAKi/F,cAAcZ,EAAUx0F,GAAW,OAgBpDjK,EAAQs/F,4BAA8B,SAASp5C,GAG7C,IAAK,GAFDq5C,GAAoB,GACpBC,EAAwB,KACnB55F,EAAI,EAAGA,EAAIsgD,EAAKqJ,aAAaxpD,OAAQH,IAC5C,GAA6BgB,SAAzBs/C,EAAKqJ,aAAa3pD,GAAkB,CACtC,GAAI65F,GAAY,IACZv5C,GAAKqJ,aAAa3pD,GAAGmuD,QAAU7N,EAAKzlD,GACtCg/F,EAAYv5C,EAAKqJ,aAAa3pD,GAAG4jB,KAE1B08B,EAAKqJ,aAAa3pD,GAAGouD,MAAQ9N,EAAKzlD,KACzCg/F,EAAYv5C,EAAKqJ,aAAa3pD,GAAG6jB,IAIlB,MAAbg2E,GAAqBF,EAAoBE,EAAU7iC,gBAAgB72D,SACrEw5F,EAAoBE,EAAU7iC,gBAAgB72D,OAC9Cy5F,EAAwBC,GAKb,MAAbA,GAAkD74F,SAA7BxG,KAAKg9C,MAAMqiD,EAAUh/F,KAC5CL,KAAKi/F,cAAcI,EAAWv5C,GAAM,IAYxClmD,EAAQs+F,mBAAqB,SAASt9D,EAAO0+D,GAE3C,IAAK,GAAIn5C,KAAUnmD,MAAKg9C,MAElBh9C,KAAKg9C,MAAMl3C,eAAeqgD,IAC5BnmD,KAAKu/F,oBAAoBv/F,KAAKg9C,MAAMmJ,GAAQvlB,EAAM0+D,IAcxD1/F,EAAQ2/F,oBAAsB,SAASC,EAAS5+D,EAAO0+D,EAAWG,GAKhE,GAJ6Bj5F,SAAzBi5F,IACFA,EAAuB,GAGpBD,EAAQ3jC,oBAAsB77D,KAAKqtE,cAA6B,GAAbiyB,GACrDE,EAAQ3jC,oBAAsB77D,KAAKqtE,cAA6B,GAAbiyB,EAAoB,CASxE,IAAK,GAPD1gF,GAAGC,EAAGlZ,EACNq5F,EAAYh/F,KAAK0hD,UAAUvC,WAAWK,qBAAqBx/C,KAAKid,MAChEyiF,GAAe,EAGfC,KACAC,EAAuBJ,EAAQrwC,aAAaxpD,OACvCimB,EAAI,EAAOg0E,EAAJh0E,EAA0BA,IACxC+zE,EAAa33F,KAAKw3F,EAAQrwC,aAAavjC,GAAGvrB,GAK5C,IAAa,GAATugC,EAEF,IADA8+D,GAAe,EACV9zE,EAAI,EAAOg0E,EAAJh0E,EAA0BA,IAAK,CACzC,GAAI8hC,GAAO1tD,KAAK89C,MAAM6hD,EAAa/zE,GACnC,IAAaplB,SAATknD,GACEA,EAAKC,WACHD,EAAKkG,MAAQlG,EAAKiG,SACpB/0C,EAAM8uC,EAAKrkC,GAAGtX,EAAI27C,EAAKtkC,KAAKrX,EAC5B8M,EAAM6uC,EAAKrkC,GAAGrX,EAAI07C,EAAKtkC,KAAKpX,EAC5BrM,EAAST,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAErBmgF,EAATr5F,GAAoB,CACtB+5F,GAAe,CACf,QASZ,IAAM9+D,GAAS8+D,GAAiB9+D,EAE9B,IAAKhV,EAAI,EAAOg0E,EAAJh0E,EAA0BA,IAGpC,GAFA8hC,EAAO1tD,KAAK89C,MAAM6hD,EAAa/zE,IAElBplB,SAATknD,EAAoB,CACtB,GAAI2wC,GAAYr+F,KAAKg9C,MAAO0Q,EAAKiG,QAAU6rC,EAAQn/F,GAAMqtD,EAAKkG,KAAOlG,EAAKiG,OAErE0qC,GAAUlvC,aAAaxpD,QAAW3F,KAAKqtE,aAAeoyB,GACtDpB,EAAUh+F,IAAMm/F,EAAQn/F,IAC3BL,KAAKi/F,cAAcO,EAAQnB,EAAUz9D,MAkBjDhhC,EAAQq/F,cAAgB,SAASp1F,EAAYw0F,EAAWz9D,GAEtD/2B,EAAWyyD,eAAe+hC,EAAUh+F,IAAMg+F,CAG1C,KAAK,GAAI74F,GAAI,EAAGA,EAAI64F,EAAUlvC,aAAaxpD,OAAQH,IAAK,CACtD,GAAIkoD,GAAO2wC,EAAUlvC,aAAa3pD,EAC9BkoD,GAAKkG,MAAQ/pD,EAAWxJ,IAAMqtD,EAAKiG,QAAU9pD,EAAWxJ,GAC1DL,KAAK6/F,qBAAqBh2F,EAAWw0F,EAAU3wC,GAG/C1tD,KAAK8/F,sBAAsBj2F,EAAWw0F,EAAU3wC,GAIpD2wC,EAAUlvC,gBAGVnvD,KAAK+/F,8BAA8Bl2F,EAAWw0F,SAIvCr+F,MAAKg9C,MAAMqhD,EAAUh+F,GAG5B,IAAI2/F,GAAan2F,EAAW4E,QAAQwuC,IACpCohD,GAAUviC,eAAiB97D,KAAK87D,eAChCjyD,EAAW4E,QAAQwuC,MAAQohD,EAAU5vF,QAAQwuC,KAC7CpzC,EAAWwyD,aAAegiC,EAAUhiC,YACpCxyD,EAAW4E,QAAQ8uC,SAAWr4C,KAAK4G,IAAI9L,KAAK0hD,UAAUvC,WAAWS,YAAa5/C,KAAK0hD,UAAU1E,MAAMO,SAAWv9C,KAAK0hD,UAAUvC,WAAWQ,mBAAmB91C,EAAWwyD,aAGlKxyD,EAAW2yD,gBAAgB3yD,EAAW2yD,gBAAgB72D,OAAS,IAAM3F,KAAK87D,gBAC5EjyD,EAAW2yD,gBAAgBx0D,KAAKhI,KAAK87D,gBAMrCjyD,EAAWuyD,eAFA,GAATx7B,EAE0B,EAGA5gC,KAAKid,MAInCpT,EAAWs0D,iBAGXt0D,EAAWyyD,eAAe+hC,EAAUh+F,IAAI+7D,eAAiBvyD,EAAWuyD,eAGpEiiC,EAAUn+B,gBAGVr2D,EAAWs2D,eAAe6/B,GAG1BhgG,KAAK8kD,QAAS,GAUhBllD,EAAQs9F,oBAAsB,WAC5B,IAAK,GAAI13F,GAAI,EAAGA,EAAIxF,KAAK8jD,YAAYn+C,OAAQH,IAAK,CAChD,GAAIsgD,GAAO9lD,KAAKg9C,MAAMh9C,KAAK8jD,YAAYt+C,GACvCsgD,GAAK+V,mBAAqB/V,EAAKqJ,aAAaxpD,MAG5C,IAAIs6F,GAAa,CACjB,IAAIn6C,EAAK+V,mBAAqB,EAC5B,IAAK,GAAIjwC,GAAI,EAAGA,EAAIk6B,EAAK+V,mBAAqB,EAAGjwC,IAG/C,IAAK,GAFDs0E,GAAWp6C,EAAKqJ,aAAavjC,GAAGgoC,KAChCusC,EAAar6C,EAAKqJ,aAAavjC,GAAG+nC,OAC7BysC,EAAIx0E,EAAE,EAAGw0E,EAAIt6C,EAAK+V,mBAAoBukC,KACxCt6C,EAAKqJ,aAAaixC,GAAGxsC,MAAQssC,GAAYp6C,EAAKqJ,aAAaixC,GAAGzsC,QAAUwsC,GACxEr6C,EAAKqJ,aAAaixC,GAAGzsC,QAAUusC,GAAYp6C,EAAKqJ,aAAaixC,GAAGxsC,MAAQusC,KAC3EF,GAAc,EAKtBn6C,GAAK+V,oBAAsBokC,IAa/BrgG,EAAQigG,qBAAuB,SAASh2F,EAAYw0F,EAAW3wC,GAEvD7jD,EAAW0yD,eAAez2D,eAAeu4F,EAAUh+F,MACvDwJ,EAAW0yD,eAAe8hC,EAAUh+F,QAGtCwJ,EAAW0yD,eAAe8hC,EAAUh+F,IAAI2H,KAAK0lD,SAGtC1tD,MAAK89C,MAAM4P,EAAKrtD,GAGvB,KAAK,GAAImF,GAAI,EAAGA,EAAIqE,EAAWslD,aAAaxpD,OAAQH,IAClD,GAAIqE,EAAWslD,aAAa3pD,GAAGnF,IAAMqtD,EAAKrtD,GAAI,CAC5CwJ,EAAWslD,aAAa/mD,OAAO5C,EAAE,EACjC,SAcN5F,EAAQkgG,sBAAwB,SAASj2F,EAAYw0F,EAAW3wC,GAE1DA,EAAKkG,MAAQlG,EAAKiG,OACpB3zD,KAAK6/F,qBAAqBh2F,EAAYw0F,EAAW3wC,IAG7CA,EAAKkG,MAAQyqC,EAAUh+F,IACzBqtD,EAAK0G,aAAapsD,KAAKq2F,EAAUh+F,IACjCqtD,EAAKrkC,GAAKxf,EACV6jD,EAAKkG,KAAO/pD,EAAWxJ,KAIvBqtD,EAAKyG,eAAensD,KAAKq2F,EAAUh+F,IACnCqtD,EAAKtkC,KAAOvf,EACZ6jD,EAAKiG,OAAS9pD,EAAWxJ,IAG3BL,KAAKqgG,oBAAoBx2F,EAAWw0F,EAAU3wC,KAalD9tD,EAAQmgG,8BAAgC,SAASl2F,EAAYw0F,GAE3D,IAAK,GAAI74F,GAAI,EAAGA,EAAIqE,EAAWslD,aAAaxpD,OAAQH,IAAK,CACvD,GAAIkoD,GAAO7jD,EAAWslD,aAAa3pD,EAE/BkoD,GAAKkG,MAAQlG,EAAKiG,QACpB3zD,KAAK6/F,qBAAqBh2F,EAAYw0F,EAAW3wC,KAcvD9tD,EAAQygG,oBAAsB,SAASx2F,EAAYw0F,EAAW3wC,GAGtD7jD,EAAWkxD,cAAcj1D,eAAeu4F,EAAUh+F,MACtDwJ,EAAWkxD,cAAcsjC,EAAUh+F,QAErCwJ,EAAWkxD,cAAcsjC,EAAUh+F,IAAI2H,KAAK0lD,GAG5C7jD,EAAWslD,aAAannD,KAAK0lD,IAY/B9tD,EAAQ6+F,wBAA0B,SAAS50F,EAAYw0F,GACrD,GAAIx0F,EAAWkxD,cAAcj1D,eAAeu4F,EAAUh+F,IAAK,CACzD,IAAK,GAAImF,GAAI,EAAGA,EAAIqE,EAAWkxD,cAAcsjC,EAAUh+F,IAAIsF,OAAQH,IAAK,CACtE,GAAIkoD,GAAO7jD,EAAWkxD,cAAcsjC,EAAUh+F,IAAImF,EAC9CkoD,GAAKyG,eAAezG,EAAKyG,eAAexuD,OAAO,IAAM04F,EAAUh+F,IACjEqtD,EAAKyG,eAAe9Z,MACpBqT,EAAKiG,OAAS0qC,EAAUh+F,GACxBqtD,EAAKtkC,KAAOi1E,IAGZ3wC,EAAK0G,aAAa/Z,MAClBqT,EAAKkG,KAAOyqC,EAAUh+F,GACtBqtD,EAAKrkC,GAAKg1E,GAIZA,EAAUlvC,aAAannD,KAAK0lD,EAG5B,KAAK,GAAI9hC,GAAI,EAAGA,EAAI/hB,EAAWslD,aAAaxpD,OAAQimB,IAClD,GAAI/hB,EAAWslD,aAAavjC,GAAGvrB,IAAMqtD,EAAKrtD,GAAI,CAC5CwJ,EAAWslD,aAAa/mD,OAAOwjB,EAAE,EACjC,cAKC/hB,GAAWkxD,cAAcsjC,EAAUh+F,MAa9CT,EAAQ8+F,eAAiB,SAAS70F,GAChC,IAAK,GAAIrE,GAAI,EAAGA,EAAIqE,EAAWslD,aAAaxpD,OAAQH,IAAK,CACvD,GAAIkoD,GAAO7jD,EAAWslD,aAAa3pD,EAC/BqE,GAAWxJ,IAAMqtD,EAAKkG,MAAQ/pD,EAAWxJ,IAAMqtD,EAAKiG,QACtD9pD,EAAWslD,aAAa/mD,OAAO5C,EAAE,KAcvC5F,EAAQ4+F,uBAAyB,SAAS30F,EAAYw0F,GACpD,IAAK,GAAI74F,GAAI,EAAGA,EAAIqE,EAAW0yD,eAAe8hC,EAAUh+F,IAAIsF,OAAQH,IAAK,CACvE,GAAIkoD,GAAO7jD,EAAW0yD,eAAe8hC,EAAUh+F,IAAImF,EAGnDxF,MAAK89C,MAAM4P,EAAKrtD,IAAMqtD,EAGtB2wC,EAAUlvC,aAAannD,KAAK0lD,GAC5B7jD,EAAWslD,aAAannD,KAAK0lD,SAGxB7jD,GAAW0yD,eAAe8hC,EAAUh+F,KAa7CT,EAAQ4uD,aAAe,WACrB,GAAIrI,EAEJ,KAAKA,IAAUnmD,MAAKg9C,MAClB,GAAIh9C,KAAKg9C,MAAMl3C,eAAeqgD,GAAS,CACrC,GAAIL,GAAO9lD,KAAKg9C,MAAMmJ,EAClBL,GAAKuW,YAAc,IACrBvW,EAAKr9B,MAAQ,IAAIzU,OAAO5P,OAAO0hD,EAAKuW,aAAa,MAMvD,IAAKlW,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GACM,GAApBL,EAAKuW,cAELvW,EAAKr9B,MADoBjiB,SAAvBs/C,EAAK2W,cACM3W,EAAK2W,cAGLr4D,OAAO0hD,EAAKzlD,OAuBnCT,EAAQ48F,uBAAyB,WAC/B,GAGIr2C,GAHAm6C,EAAW,EACXC,EAAW,IACXC,EAAe,CAInB,KAAKr6C,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5Bq6C,EAAexgG,KAAKg9C,MAAMmJ,GAAQqW,gBAAgB72D,OACnC66F,EAAXF,IAA0BA,EAAWE,GACrCD,EAAWC,IAAeD,EAAWC,GAI7C,IAAIF,EAAWC,EAAWvgG,KAAK0hD,UAAUvC,WAAWgB,uBAAwB,CAC1E,GAAIo9C,GAAgBv9F,KAAK8jD,YAAYn+C,OACjC86F,EAAcH,EAAWtgG,KAAK0hD,UAAUvC,WAAWgB,sBAEvD,KAAKgG,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,IACxBnmD,KAAKg9C,MAAMmJ,GAAQqW,gBAAgB72D,OAAS86F,GAC9CzgG,KAAKk/F,4BAA4Bl/F,KAAKg9C,MAAMmJ,GAIlDnmD,MAAKgnD,uBACLhnD,KAAKk9F,sBAEDl9F,KAAK8jD,YAAYn+C,QAAU43F,IAC7Bv9F,KAAK87D,gBAAkB,KAe7Bl8D,EAAQi9F,kBAAoB,SAAS/2C,GACnC,MACE5gD,MAAK2lB,IAAIi7B,EAAK/zC,EAAI/R,KAAKkkD,WAAWnyC,IAAM/R,KAAK0hD,UAAUvC,WAAWe,kBAAkBlgD,KAAKid,OAEzF/X,KAAK2lB,IAAIi7B,EAAK9zC,EAAIhS,KAAKkkD,WAAWlyC,IAAMhS,KAAK0hD,UAAUvC,WAAWe,kBAAkBlgD,KAAKid,OAU7Frd,EAAQ88F,gBAAkB,WACxB,IAAK,GAAIl3F,GAAI,EAAGA,EAAIxF,KAAK8jD,YAAYn+C,OAAQH,IAAK,CAChD,GAAIsgD,GAAO9lD,KAAKg9C,MAAMh9C,KAAK8jD,YAAYt+C,GACvC,IAAoB,GAAfsgD,EAAKqF,QAAkC,GAAfrF,EAAKsF,OAAkB,CAClD,GAAI3/B,GAAS,EAASzrB,KAAK8jD,YAAYn+C,OAAST,KAAK4G,IAAI,IAAIg6C,EAAKr3C,QAAQwuC,MACtEmR,EAAQ,EAAIlpD,KAAKymB,GAAKzmB,KAAKE,QACZ,IAAf0gD,EAAKqF,SAAkBrF,EAAK/zC,EAAI0Z,EAASvmB,KAAKqZ,IAAI6vC,IACnC,GAAftI,EAAKsF,SAAkBtF,EAAK9zC,EAAIyZ,EAASvmB,KAAKkZ,IAAIgwC,IACtDpuD,KAAK6+F,uBAAuB/4C,MAYlClmD,EAAQq+F,YAAc,WAMpB,IAAK,GALDyC,GAAU,EACVC,EAAiB,EACjBC,EAAa,EACbC,EAAa,EAERr7F,EAAI,EAAGA,EAAIxF,KAAK8jD,YAAYn+C,OAAQH,IAAK,CAEhD,GAAIsgD,GAAO9lD,KAAKg9C,MAAMh9C,KAAK8jD,YAAYt+C,GACnCsgD,GAAK+V,mBAAqBglC,IAC5BA,EAAa/6C,EAAK+V,oBAEpB6kC,GAAW56C,EAAK+V,mBAChB8kC,GAAkBz7F,KAAK4uB,IAAIgyB,EAAK+V,mBAAmB,GACnD+kC,GAAc,EAEhBF,GAAoBE,EACpBD,GAAkCC,CAElC,IAAIE,GAAWH,EAAiBz7F,KAAK4uB,IAAI4sE,EAAQ,GAE7CK,EAAoB77F,KAAKyqB,KAAKmxE,EAElC9gG,MAAKqtE,aAAenoE,KAAKC,MAAMu7F,EAAU,EAAEK,GAGvC/gG,KAAKqtE,aAAewzB,IACtB7gG,KAAKqtE,aAAewzB,IAexBjhG,EAAQo+F,sBAAwB,SAASgD,GACvChhG,KAAKqtE,aAAe,CACpB,IAAI4zB,GAAe/7F,KAAKC,MAAMnF,KAAK8jD,YAAYn+C,OAASq7F,EACxD,KAAK,GAAI76C,KAAUnmD,MAAKg9C,MAClBh9C,KAAKg9C,MAAMl3C,eAAeqgD,IACiB,GAAzCnmD,KAAKg9C,MAAMmJ,GAAQ0V,oBAA2B77D,KAAKg9C,MAAMmJ,GAAQgJ,aAAaxpD,QAAU,GACtFs7F,EAAe,IACjBjhG,KAAKu/F,oBAAoBv/F,KAAKg9C,MAAMmJ,IAAQ,GAAK,EAAK,GACtD86C,GAAgB,IAa1BrhG,EAAQm+F,kBAAoB,WAC1B,GAAImD,GAAS,EACTC,EAAQ,CACZ,KAAK,GAAIh7C,KAAUnmD,MAAKg9C,MAClBh9C,KAAKg9C,MAAMl3C,eAAeqgD,KACiB,GAAzCnmD,KAAKg9C,MAAMmJ,GAAQ0V,oBAA2B77D,KAAKg9C,MAAMmJ,GAAQgJ,aAAaxpD,QAAU,IAC1Fu7F,GAAU,GAEZC,GAAS,EAGb,OAAOD,GAAOC,IAMZ,SAASthG,EAAQD,EAASM,GAE9B,GAAIS,GAAOT,EAAoB,GAC3BqD,EAAOrD,EAAoB,GAgB/BN,GAAQ+nD,iBAAmB,WACzB3nD,KAAKkvD,QAAgB,OAAElvD,KAAK88F,WAAW9/C,MAAQh9C,KAAKg9C,MACpDh9C,KAAKkvD,QAAgB,OAAElvD,KAAK88F,WAAWh/C,MAAQ99C,KAAK89C,MACpD99C,KAAKkvD,QAAgB,OAAElvD,KAAK88F,WAAWh5C,YAAc9jD,KAAK8jD,aAa5DlkD,EAAQwhG,gBAAkB,SAASC,EAAUC,GACxB96F,SAAf86F,GAA0C,UAAdA,EAC9BthG,KAAKuhG,sBAAsBF,GAG3BrhG,KAAKwhG,sBAAsBH,IAY/BzhG,EAAQ2hG,sBAAwB,SAASF,GACvCrhG,KAAK8jD,YAAc9jD,KAAKkvD,QAAgB,OAAEmyC,GAAuB,YACjErhG,KAAKg9C,MAAch9C,KAAKkvD,QAAgB,OAAEmyC,GAAiB,MAC3DrhG,KAAK89C,MAAc99C,KAAKkvD,QAAgB,OAAEmyC,GAAiB,OAU7DzhG,EAAQ6hG,uBAAyB,WAC/BzhG,KAAK8jD,YAAc9jD,KAAKkvD,QAAiB,QAAe,YACxDlvD,KAAKg9C,MAAch9C,KAAKkvD,QAAiB,QAAS,MAClDlvD,KAAK89C,MAAc99C,KAAKkvD,QAAiB,QAAS,OAWpDtvD,EAAQ4hG,sBAAwB,SAASH,GACvCrhG,KAAK8jD,YAAc9jD,KAAKkvD,QAAgB,OAAEmyC,GAAuB,YACjErhG,KAAKg9C,MAAch9C,KAAKkvD,QAAgB,OAAEmyC,GAAiB,MAC3DrhG,KAAK89C,MAAc99C,KAAKkvD,QAAgB,OAAEmyC,GAAiB,OAU7DzhG,EAAQ8hG,kBAAoB,WAC1B1hG,KAAKohG,gBAAgBphG,KAAK88F,YAU5Bl9F,EAAQk9F,QAAU,WAChB,MAAO98F,MAAKstE,aAAattE,KAAKstE,aAAa3nE,OAAO,IAUpD/F,EAAQ+hG,gBAAkB,WACxB,GAAI3hG,KAAKstE,aAAa3nE,OAAS,EAC7B,MAAO3F,MAAKstE,aAAattE,KAAKstE,aAAa3nE,OAAO,EAGlD,MAAM,IAAIU,WAAU,iEAaxBzG,EAAQgiG,iBAAmB,SAASC,GAClC7hG,KAAKstE,aAAatlE,KAAK65F,IAUzBjiG,EAAQkiG,kBAAoB,WAC1B9hG,KAAKstE,aAAajzB,OAWpBz6C,EAAQmiG,iBAAmB,SAASF,GAElC7hG,KAAKkvD,QAAgB,OAAE2yC,IAAU7kD,SACAc,SACAgG,eACAsY,eAAkBp8D,KAAKid,MACvBswD,YAAe/mE,QAGhDxG,KAAKkvD,QAAgB,OAAE2yC,GAAoB,YAAI,GAAIt+F,IAC9ClD,GAAGwhG,EACF12F,OACEgB,WAAY,UACZC,OAAQ,iBAEJpM,KAAK0hD,WACjB1hD,KAAKkvD,QAAgB,OAAE2yC,GAAoB,YAAExlC,YAAc,GAW7Dz8D,EAAQoiG,oBAAsB,SAASX,SAC9BrhG,MAAKkvD,QAAgB,OAAEmyC,IAWhCzhG,EAAQqiG,oBAAsB,SAASZ,SAC9BrhG,MAAKkvD,QAAgB,OAAEmyC,IAWhCzhG,EAAQsiG,cAAgB,SAASb,GAE/BrhG,KAAKkvD,QAAgB,OAAEmyC,GAAYrhG,KAAKkvD,QAAgB,OAAEmyC,GAG1DrhG,KAAKgiG,oBAAoBX,IAW3BzhG,EAAQuiG,gBAAkB,SAASd,GAEjCrhG,KAAKkvD,QAAgB,OAAEmyC,GAAYrhG,KAAKkvD,QAAgB,OAAEmyC,GAG1DrhG,KAAKiiG,oBAAoBZ,IAa3BzhG,EAAQwiG,qBAAuB,SAASf,GAEtC,IAAK,GAAIl7C,KAAUnmD,MAAKg9C,MAClBh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BnmD,KAAKkvD,QAAgB,OAAEmyC,GAAiB,MAAEl7C,GAAUnmD,KAAKg9C,MAAMmJ,GAKnE,KAAK,GAAI6G,KAAUhtD,MAAK89C,MAClB99C,KAAK89C,MAAMh4C,eAAeknD,KAC5BhtD,KAAKkvD,QAAgB,OAAEmyC,GAAiB,MAAEr0C,GAAUhtD,KAAK89C,MAAMkP,GAKnE,KAAK,GAAIxnD,GAAI,EAAGA,EAAIxF,KAAK8jD,YAAYn+C,OAAQH,IAC3CxF,KAAKkvD,QAAgB,OAAEmyC,GAAuB,YAAEr5F,KAAKhI,KAAK8jD,YAAYt+C,KAW1E5F,EAAQyiG,6BAA+B,WACrCriG,KAAKm8F,aAAa,GAAE,IAUtBv8F,EAAQm9F,WAAa,SAASj3C,GAE5B,GAAIw8C,GAAStiG,KAAK88F,gBAWX98F,MAAKg9C,MAAM8I,EAAKzlD,GAEvB,IAAIkiG,GAAmB5hG,EAAKqE,YAG5BhF,MAAKkiG,cAAcI,GAGnBtiG,KAAK+hG,iBAAiBQ,GAGtBviG,KAAK4hG,iBAAiBW,GAGtBviG,KAAKohG,gBAAgBphG,KAAK88F,WAG1B98F,KAAKg9C,MAAM8I,EAAKzlD,IAAMylD,GAUxBlmD,EAAQ49F,gBAAkB,WAExB,GAAI8E,GAAStiG,KAAK88F,SAGlB,IAAc,WAAVwF,IAC8B,GAA3BtiG,KAAK8jD,YAAYn+C,QACpB3F,KAAKkvD,QAAgB,OAAEozC,GAAqB,YAAE/vF,MAAMvS,KAAKid,MAAQjd,KAAK0hD,UAAUvC,WAAWO,oBAAsB1/C,KAAKsf,MAAMC,OAAOC,aACnIxf,KAAKkvD,QAAgB,OAAEozC,GAAqB,YAAE9vF,OAAOxS,KAAKid,MAAQjd,KAAK0hD,UAAUvC,WAAWO,oBAAsB1/C,KAAKsf,MAAMC,OAAOsF,cAAe,CACnJ,GAAI29E,GAAiBxiG,KAAK2hG,iBAG1B3hG,MAAKqiG,+BAILriG,KAAKoiG,qBAAqBI,GAI1BxiG,KAAKgiG,oBAAoBM,GAGzBtiG,KAAKmiG,gBAAgBK,GAGrBxiG,KAAKohG,gBAAgBoB,GAGrBxiG,KAAK8hG,oBAGL9hG,KAAKgnD,uBAGLhnD,KAAKquD,4BAeXzuD,EAAQsxD,sBAAwB,SAASuxC,EAAYC,GACnD,GAAIC,KACJ,IAAiBn8F,SAAbk8F,EACF,IAAK,GAAIJ,KAAUtiG,MAAKkvD,QAAgB,OAClClvD,KAAKkvD,QAAgB,OAAEppD,eAAew8F,KAExCtiG,KAAKuhG,sBAAsBe,GAC3BK,EAAa36F,KAAMhI,KAAKyiG,WAK5B,KAAK,GAAIH,KAAUtiG,MAAKkvD,QAAgB,OACtC,GAAIlvD,KAAKkvD,QAAgB,OAAEppD,eAAew8F,GAAS,CAEjDtiG,KAAKuhG,sBAAsBe,EAC3B,IAAIrpF,GAAOhT,MAAMkN,UAAU/K,OAAO7H,KAAKmF,UAAW,EAEhDi9F,GAAa36F,KADXiR,EAAKtT,OAAS,EACG3F,KAAKyiG,GAAaxpF,EAAK,GAAGA,EAAK,IAG/BjZ,KAAKyiG,GAAaC,IAO7C,MADA1iG,MAAK0hG,oBACEiB,GAaT/iG,EAAQuxD,mBAAqB,SAASsxC,EAAYC,GAChD,GAAIC,IAAe,CACnB,IAAiBn8F,SAAbk8F,EACF1iG,KAAKyhG,yBACLkB,EAAe3iG,KAAKyiG,SAEjB,CACHziG,KAAKyhG,wBACL,IAAIxoF,GAAOhT,MAAMkN,UAAU/K,OAAO7H,KAAKmF,UAAW,EAEhDi9F,GADE1pF,EAAKtT,OAAS,EACD3F,KAAKyiG,GAAaxpF,EAAK,GAAGA,EAAK,IAG/BjZ,KAAKyiG,GAAaC,GAKrC,MADA1iG,MAAK0hG,oBACEiB,GAaT/iG,EAAQgjG,sBAAwB,SAASH,EAAYC,GACnD,GAAiBl8F,SAAbk8F,EACF,IAAK,GAAIJ,KAAUtiG,MAAKkvD,QAAgB,OAClClvD,KAAKkvD,QAAgB,OAAEppD,eAAew8F,KAExCtiG,KAAKwhG,sBAAsBc,GAC3BtiG,KAAKyiG,UAKT,KAAK,GAAIH,KAAUtiG,MAAKkvD,QAAgB,OACtC,GAAIlvD,KAAKkvD,QAAgB,OAAEppD,eAAew8F,GAAS,CAEjDtiG,KAAKwhG,sBAAsBc,EAC3B,IAAIrpF,GAAOhT,MAAMkN,UAAU/K,OAAO7H,KAAKmF,UAAW,EAC9CuT,GAAKtT,OAAS,EAChB3F,KAAKyiG,GAAaxpF,EAAK,GAAGA,EAAK,IAG/BjZ,KAAKyiG,GAAaC,GAK1B1iG,KAAK0hG,qBAaP9hG,EAAQ4vD,gBAAkB,SAASizC,EAAYC,GAC7C,GAAIzpF,GAAOhT,MAAMkN,UAAU/K,OAAO7H,KAAKmF,UAAW,EACjCc,UAAbk8F,GACF1iG,KAAKkxD,sBAAsBuxC,GAC3BziG,KAAK4iG,sBAAsBH,IAGvBxpF,EAAKtT,OAAS,GAChB3F,KAAKkxD,sBAAsBuxC,EAAYxpF,EAAK,GAAGA,EAAK,IACpDjZ,KAAK4iG,sBAAsBH,EAAYxpF,EAAK,GAAGA,EAAK,MAGpDjZ,KAAKkxD,sBAAsBuxC,EAAYC,GACvC1iG,KAAK4iG,sBAAsBH,EAAYC,KAY7C9iG,EAAQqnD,oBAAsB,WAC5B,GAAIq7C,GAAStiG,KAAK88F,SAClB98F,MAAKkvD,QAAgB,OAAEozC,GAAqB,eAC5CtiG,KAAK8jD,YAAc9jD,KAAKkvD,QAAgB,OAAEozC,GAAqB,aAWjE1iG,EAAQijG,iBAAmB,SAAS97E,EAAIu6E,GACtC,GAAsDx7C,GAAlDC,EAAO,IAAKC,EAAO,KAAMC,EAAO,IAAKC,EAAO,IAChD,KAAK,GAAIo8C,KAAUtiG,MAAKkvD,QAAQoyC,GAC9B,GAAIthG,KAAKkvD,QAAQoyC,GAAYx7F,eAAew8F,IACc97F,SAApDxG,KAAKkvD,QAAQoyC,GAAYgB,GAAqB,YAAiB,CAEjEtiG,KAAKohG,gBAAgBkB,EAAOhB,GAE5Bv7C,EAAO,IAAKC,EAAO,KAAMC,EAAO,IAAKC,EAAO,IAC5C,KAAK,GAAIC,KAAUnmD,MAAKg9C,MAClBh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GAClBL,EAAKkQ,OAAOjvC,GACRk/B,EAAOH,EAAK/zC,EAAI,GAAM+zC,EAAKvzC,QAAQ0zC,EAAOH,EAAK/zC,EAAI,GAAM+zC,EAAKvzC,OAC9D2zC,EAAOJ,EAAK/zC,EAAI,GAAM+zC,EAAKvzC,QAAQ2zC,EAAOJ,EAAK/zC,EAAI,GAAM+zC,EAAKvzC,OAC9DwzC,EAAOD,EAAK9zC,EAAI,GAAM8zC,EAAKtzC,SAASuzC,EAAOD,EAAK9zC,EAAI,GAAM8zC,EAAKtzC,QAC/DwzC,EAAOF,EAAK9zC,EAAI,GAAM8zC,EAAKtzC,SAASwzC,EAAOF,EAAK9zC,EAAI,GAAM8zC,EAAKtzC,QAGvEszC,GAAO9lD,KAAKkvD,QAAQoyC,GAAYgB,GAAqB,YACrDx8C,EAAK/zC,EAAI,IAAOm0C,EAAOD,GACvBH,EAAK9zC,EAAI,IAAOg0C,EAAOD,GACvBD,EAAKvzC,MAAQ,GAAKuzC,EAAK/zC,EAAIk0C,GAC3BH,EAAKtzC,OAAS,GAAKszC,EAAK9zC,EAAI+zC,GAC5BD,EAAKr3C,QAAQgd,OAASvmB,KAAKyqB,KAAKzqB,KAAK4uB,IAAI,GAAIgyB,EAAKvzC,MAAM,GAAKrN,KAAK4uB,IAAI,GAAIgyB,EAAKtzC,OAAO,IACtFszC,EAAK9iB,SAAShjC,KAAKid,OACnB6oC,EAAKoX,YAAYn2C,KAMzBnnB,EAAQkjG,oBAAsB,SAAS/7E,GACrC/mB,KAAK6iG,iBAAiB97E,EAAI,UAC1B/mB,KAAK6iG,iBAAiB97E,EAAI,UAC1B/mB,KAAK0hG,sBAMH,SAAS7hG,EAAQD,EAASM,GAE9B,GAAIqD,GAAOrD,EAAoB,GAS/BN,GAAQmjG,yBAA2B,SAAS9+F,EAAQqpD,GAClD,GAAItQ,GAAQh9C,KAAKg9C,KACjB,KAAK,GAAImJ,KAAUnJ,GACbA,EAAMl3C,eAAeqgD,IACnBnJ,EAAMmJ,GAAQoH,kBAAkBtpD,IAClCqpD,EAAiBtlD,KAAKm+C,IAY9BvmD,EAAQojG,4BAA8B,SAAU/+F,GAC9C,GAAIqpD,KAEJ,OADAttD,MAAKkxD,sBAAsB,2BAA2BjtD,EAAOqpD,GACtDA,GAWT1tD,EAAQqjG,yBAA2B,SAASpjE,GAC1C,GAAI9tB,GAAI/R,KAAKurD,qBAAqB1rB,EAAQ9tB,GACtCC,EAAIhS,KAAKyrD,qBAAqB5rB,EAAQ7tB,EAE1C,QACEvK,KAAQsK,EACRpK,IAAQqK,EACRqV,MAAQtV,EACRuR,OAAQtR,IAYZpS,EAAQgrD,WAAa,SAAU/qB,GAE7B,GAAIqjE,GAAiBljG,KAAKijG,yBAAyBpjE,GAC/CytB,EAAmBttD,KAAKgjG,4BAA4BE,EAIxD,OAAI51C,GAAiB3nD,OAAS,EACpB3F,KAAKg9C,MAAMsQ,EAAiBA,EAAiB3nD,OAAS,IAGvD,MAWX/F,EAAQujG,yBAA2B,SAAUl/F,EAAQwpD,GACnD,GAAI3P,GAAQ99C,KAAK89C,KACjB,KAAK,GAAIkP,KAAUlP,GACbA,EAAMh4C,eAAeknD,IACnBlP,EAAMkP,GAAQO,kBAAkBtpD,IAClCwpD,EAAiBzlD,KAAKglD,IAa9BptD,EAAQwjG,4BAA8B,SAAUn/F,GAC9C,GAAIwpD,KAEJ,OADAztD,MAAKkxD,sBAAsB,2BAA2BjtD,EAAOwpD,GACtDA,GAWT7tD,EAAQqtD,WAAa,SAASptB,GAC5B,GAAIqjE,GAAiBljG,KAAKijG,yBAAyBpjE,GAC/C4tB,EAAmBztD,KAAKojG,4BAA4BF,EAExD,OAAIz1C,GAAiB9nD,OAAS,EACrB3F,KAAK89C,MAAM2P,EAAiBA,EAAiB9nD,OAAS,IAGtD,MAWX/F,EAAQyjG,gBAAkB,SAAStgF,GAC7BA,YAAexf,GACjBvD,KAAKkrD,aAAalO,MAAMj6B,EAAI1iB,IAAM0iB,EAGlC/iB,KAAKkrD,aAAapN,MAAM/6B,EAAI1iB,IAAM0iB,GAUtCnjB,EAAQ0jG,YAAc,SAASvgF,GACzBA,YAAexf,GACjBvD,KAAK4hD,SAAS5E,MAAMj6B,EAAI1iB,IAAM0iB,EAG9B/iB,KAAK4hD,SAAS9D,MAAM/6B,EAAI1iB,IAAM0iB,GAWlCnjB,EAAQ2jG,qBAAuB,SAASxgF,GAClCA,YAAexf,SACVvD,MAAKkrD,aAAalO,MAAMj6B,EAAI1iB,UAG5BL,MAAKkrD,aAAapN,MAAM/6B,EAAI1iB,KAUvCT,EAAQ2+F,aAAe,SAASiF,GACTh9F,SAAjBg9F,IACFA,GAAe,EAEjB,KAAI,GAAIr9C,KAAUnmD,MAAKkrD,aAAalO,MAC/Bh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,IACxCnmD,KAAKkrD,aAAalO,MAAMmJ,GAAQjV,UAGpC,KAAI,GAAI8b,KAAUhtD,MAAKkrD,aAAapN,MAC/B99C,KAAKkrD,aAAapN,MAAMh4C,eAAeknD,IACxChtD,KAAKkrD,aAAapN,MAAMkP,GAAQ9b,UAIpClxC,MAAKkrD,cAAgBlO,SAASc,UAEV,GAAhB0lD,GACFxjG,KAAK4tB,KAAK,SAAU5tB,KAAK42B,iBAU7Bh3B,EAAQ6jG,kBAAoB,SAASD,GACdh9F,SAAjBg9F,IACFA,GAAe,EAGjB,KAAK,GAAIr9C,KAAUnmD,MAAKkrD,aAAalO,MAC/Bh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,IACrCnmD,KAAKkrD,aAAalO,MAAMmJ,GAAQkW,YAAc,IAChDr8D,KAAKkrD,aAAalO,MAAMmJ,GAAQjV,WAChClxC,KAAKujG,qBAAqBvjG,KAAKkrD,aAAalO,MAAMmJ,IAKpC,IAAhBq9C,GACFxjG,KAAK4tB,KAAK,SAAU5tB,KAAK42B,iBAW7Bh3B,EAAQ8jG,sBAAwB,WAC9B,GAAI1sF,GAAQ,CACZ,KAAK,GAAImvC,KAAUnmD,MAAKkrD,aAAalO,MAC/Bh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,KACzCnvC,GAAS,EAGb,OAAOA,IASTpX,EAAQ+jG,iBAAmB,WACzB,IAAK,GAAIx9C,KAAUnmD,MAAKkrD,aAAalO,MACnC,GAAIh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,GACzC,MAAOnmD,MAAKkrD,aAAalO,MAAMmJ,EAGnC,OAAO,OASTvmD,EAAQgkG,iBAAmB,WACzB,IAAK,GAAI52C,KAAUhtD,MAAKkrD,aAAapN,MACnC,GAAI99C,KAAKkrD,aAAapN,MAAMh4C,eAAeknD,GACzC,MAAOhtD,MAAKkrD,aAAapN,MAAMkP,EAGnC,OAAO,OAUTptD,EAAQikG,sBAAwB,WAC9B,GAAI7sF,GAAQ,CACZ,KAAK,GAAIg2C,KAAUhtD,MAAKkrD,aAAapN,MAC/B99C,KAAKkrD,aAAapN,MAAMh4C,eAAeknD,KACzCh2C,GAAS,EAGb,OAAOA,IAUTpX,EAAQkkG,wBAA0B,WAChC,GAAI9sF,GAAQ,CACZ,KAAI,GAAImvC,KAAUnmD,MAAKkrD,aAAalO,MAC/Bh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,KACxCnvC,GAAS,EAGb,KAAI,GAAIg2C,KAAUhtD,MAAKkrD,aAAapN,MAC/B99C,KAAKkrD,aAAapN,MAAMh4C,eAAeknD,KACxCh2C,GAAS,EAGb,OAAOA,IASTpX,EAAQmkG,kBAAoB,WAC1B,IAAI,GAAI59C,KAAUnmD,MAAKkrD,aAAalO,MAClC,GAAGh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,GACxC,OAAO,CAGX,KAAI,GAAI6G,KAAUhtD,MAAKkrD,aAAapN,MAClC,GAAG99C,KAAKkrD,aAAapN,MAAMh4C,eAAeknD,GACxC,OAAO,CAGX,QAAO,GAUTptD,EAAQokG,oBAAsB,WAC5B,IAAI,GAAI79C,KAAUnmD,MAAKkrD,aAAalO,MAClC,GAAGh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,IACpCnmD,KAAKkrD,aAAalO,MAAMmJ,GAAQkW,YAAc,EAChD,OAAO,CAIb,QAAO,GASTz8D,EAAQqkG,sBAAwB,SAASn+C,GACvC,IAAK,GAAItgD,GAAI,EAAGA,EAAIsgD,EAAKqJ,aAAaxpD,OAAQH,IAAK,CACjD,GAAIkoD,GAAO5H,EAAKqJ,aAAa3pD,EAC7BkoD,GAAKvc,SACLnxC,KAAKqjG,gBAAgB31C,KAUzB9tD,EAAQskG,qBAAuB,SAASp+C,GACtC,IAAK,GAAItgD,GAAI,EAAGA,EAAIsgD,EAAKqJ,aAAaxpD,OAAQH,IAAK,CACjD,GAAIkoD,GAAO5H,EAAKqJ,aAAa3pD,EAC7BkoD,GAAKphD,OAAQ,EACbtM,KAAKsjG,YAAY51C,KAWrB9tD,EAAQukG,wBAA0B,SAASr+C,GACzC,IAAK,GAAItgD,GAAI,EAAGA,EAAIsgD,EAAKqJ,aAAaxpD,OAAQH,IAAK,CACjD,GAAIkoD,GAAO5H,EAAKqJ,aAAa3pD,EAC7BkoD,GAAKxc,WACLlxC,KAAKujG,qBAAqB71C,KAgB9B9tD,EAAQmrD,cAAgB,SAAS9mD,EAAQmgG,EAAQZ,EAAca,EAAgBC,GACxD99F,SAAjBg9F,IACFA,GAAe,GAEMh9F,SAAnB69F,IACFA,GAAiB,GAGa,GAA5BrkG,KAAK+jG,qBAA0C,GAAVK,GAAgD,GAA7BpkG,KAAKytE,sBAC/DztE,KAAKu+F,cAAa,GAIG,GAAnBt6F,EAAOovC,UAAmD,GAA7BrzC,KAAK0hD,UAAUzS,aAAsBq1D,EAQ1C,GAAnBrgG,EAAOovC,UACdrzC,KAAKqjG,gBAAgBp/F,GACrBu/F,GAAe,IAGfv/F,EAAOitC,WACPlxC,KAAKujG,qBAAqBt/F,KAb1BA,EAAOktC,SACPnxC,KAAKqjG,gBAAgBp/F,GACjBA,YAAkBV,IAA6C,GAArCvD,KAAKwtE,8BAA2D,GAAlB62B,GAC1ErkG,KAAKikG,sBAAsBhgG,IAaX,GAAhBu/F,GACFxjG,KAAK4tB,KAAK,SAAU5tB,KAAK42B,iBAY7Bh3B,EAAQutD,YAAc,SAASlpD,GACT,GAAhBA,EAAOqI,QACTrI,EAAOqI,OAAQ,EACftM,KAAK4tB,KAAK,YAAYk4B,KAAK7hD,EAAO5D,OAWtCT,EAAQstD,aAAe,SAASjpD,GACV,GAAhBA,EAAOqI,QACTrI,EAAOqI,OAAQ,EACftM,KAAKsjG,YAAYr/F,GACbA,YAAkBV,IACpBvD,KAAK4tB,KAAK,aAAak4B,KAAK7hD,EAAO5D,MAGnC4D,YAAkBV,IACpBvD,KAAKkkG,qBAAqBjgG,IAa9BrE,EAAQ8qD,aAAe,aAUvB9qD,EAAQgsD,WAAa,SAAS/rB,GAC5B,GAAIimB,GAAO9lD,KAAK4qD,WAAW/qB,EAC3B,IAAY,MAARimB,EACF9lD,KAAK+qD,cAAcjF,GAAM,OAEtB,CACH,GAAI4H,GAAO1tD,KAAKitD,WAAWptB,EACf,OAAR6tB,EACF1tD,KAAK+qD,cAAc2C,GAAM,GAGzB1tD,KAAKu+F,eAGT,GAAI5vC,GAAa3uD,KAAK42B,cACtB+3B,GAAoB,SAClB41C,KAAMxyF,EAAG8tB,EAAQ9tB,EAAGC,EAAG6tB,EAAQ7tB,GAC/BuN,QAASxN,EAAG/R,KAAKurD,qBAAqB1rB,EAAQ9tB,GAAIC,EAAGhS,KAAKyrD,qBAAqB5rB,EAAQ7tB,KAEzFhS,KAAK4tB,KAAK,QAAS+gC,GACnB3uD,KAAK6iD,WAUPjjD,EAAQisD,iBAAmB,SAAShsB,GAClC,GAAIimB,GAAO9lD,KAAK4qD,WAAW/qB,EACf,OAARimB,GAAyBt/C,SAATs/C,IAElB9lD,KAAKkkD,YAAenyC,EAAM/R,KAAKurD,qBAAqB1rB,EAAQ9tB,GACxCC,EAAMhS,KAAKyrD,qBAAqB5rB,EAAQ7tB,IAC5DhS,KAAK28F,YAAY72C,GAEnB,IAAI6I,GAAa3uD,KAAK42B,cACtB+3B,GAAoB,SAClB41C,KAAMxyF,EAAG8tB,EAAQ9tB,EAAGC,EAAG6tB,EAAQ7tB,GAC/BuN,QAASxN,EAAG/R,KAAKurD,qBAAqB1rB,EAAQ9tB,GAAIC,EAAGhS,KAAKyrD,qBAAqB5rB,EAAQ7tB,KAEzFhS,KAAK4tB,KAAK,cAAe+gC,IAU3B/uD,EAAQksD,cAAgB,SAASjsB,GAC/B,GAAIimB,GAAO9lD,KAAK4qD,WAAW/qB,EAC3B,IAAY,MAARimB,EACF9lD,KAAK+qD,cAAcjF,GAAK,OAErB,CACH,GAAI4H,GAAO1tD,KAAKitD,WAAWptB,EACf,OAAR6tB,GACF1tD,KAAK+qD,cAAc2C,GAAK,GAG5B1tD,KAAK6iD,WAUPjjD,EAAQosD,iBAAmB,SAASnsB,GAClC7/B,KAAKwkG,6BAA6B3kE,GAClC7/B,KAAKykG,2BAA2B5kE,IAGlCjgC,EAAQ4kG,6BAA+B,aACvC5kG,EAAQ6kG,2BAA6B,aAOrC7kG,EAAQg3B,aAAe,WACrB,GAAIo0B,GAAUhrD,KAAK0kG,mBACfC,EAAU3kG,KAAK4kG,kBACnB,QAAQ5nD,MAAMgO,EAASlN,MAAM6mD,IAS/B/kG,EAAQ8kG,iBAAmB,WACzB,GAAIG,KACJ,IAAiC,GAA7B7kG,KAAK0hD,UAAUzS,WACjB,IAAK,GAAIkX,KAAUnmD,MAAKkrD,aAAalO,MAC/Bh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,IACzC0+C,EAAQ78F,KAAKm+C,EAInB,OAAO0+C,IASTjlG,EAAQglG,iBAAmB,WACzB,GAAIC,KACJ,IAAiC,GAA7B7kG,KAAK0hD,UAAUzS,WACjB,IAAK,GAAI+d,KAAUhtD,MAAKkrD,aAAapN,MAC/B99C,KAAKkrD,aAAapN,MAAMh4C,eAAeknD,IACzC63C,EAAQ78F,KAAKglD,EAInB,OAAO63C,IASTjlG,EAAQ82B,aAAe,WACrBiC,QAAQhF,IAAI,gEAUd/zB,EAAQklG,YAAc,SAAS30D,EAAWk0D,GACxC,GAAI7+F,GAAGq7B,EAAMxgC,CAEb,KAAK8vC,GAAkC3pC,QAApB2pC,EAAUxqC,OAC3B,KAAM,qCAKR,KAFA3F,KAAKu+F,cAAa,GAEb/4F,EAAI,EAAGq7B,EAAOsP,EAAUxqC,OAAYk7B,EAAJr7B,EAAUA,IAAK,CAClDnF,EAAK8vC,EAAU3qC,EAEf,IAAIsgD,GAAO9lD,KAAKg9C,MAAM38C,EACtB,KAAKylD,EACH,KAAM,IAAIi/C,YAAW,iBAAmB1kG,EAAK,cAE/CL,MAAK+qD,cAAcjF,GAAK,GAAK,EAAKu+C,GAAe,GAEnDrkG,KAAKyhB,UASP7hB,EAAQolG,YAAc,SAAS70D,GAC7B,GAAI3qC,GAAGq7B,EAAMxgC,CAEb,KAAK8vC,GAAkC3pC,QAApB2pC,EAAUxqC,OAC3B,KAAM,qCAKR,KAFA3F,KAAKu+F,cAAa,GAEb/4F,EAAI,EAAGq7B,EAAOsP,EAAUxqC,OAAYk7B,EAAJr7B,EAAUA,IAAK,CAClDnF,EAAK8vC,EAAU3qC,EAEf,IAAIkoD,GAAO1tD,KAAK89C,MAAMz9C,EACtB,KAAKqtD,EACH,KAAM,IAAIq3C,YAAW,iBAAmB1kG,EAAK,cAE/CL,MAAK+qD,cAAc2C,GAAK,GAAK,GAAK,GAAM,GAE1C1tD,KAAKyhB,UAOP7hB,EAAQuuD,iBAAmB,WACzB,IAAI,GAAIhI,KAAUnmD,MAAKkrD,aAAalO,MAC/Bh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,KACnCnmD,KAAKg9C,MAAMl3C,eAAeqgD,UACtBnmD,MAAKkrD,aAAalO,MAAMmJ,GAIrC,KAAI,GAAI6G,KAAUhtD,MAAKkrD,aAAapN,MAC/B99C,KAAKkrD,aAAapN,MAAMh4C,eAAeknD,KACnChtD,KAAK89C,MAAMh4C,eAAeknD,UACtBhtD,MAAKkrD,aAAapN,MAAMkP,MASnC,SAASntD,EAAQD,EAASM,GAE9B,GAAIS,GAAOT,EAAoB,GAC3BqD,EAAOrD,EAAoB,IAC3BkD,EAAOlD,EAAoB,GAO/BN,GAAQqlG,qBAAuB,WAC7BjlG,KAAKqqD,oBAAoBrqD,KAAK0tE,iBAC9B1tE,KAAKklG,mBAELllG,KAAKwkG,6BAA+B,mBAC7BxkG,MAAKkvD,QAAiB,QAAS,MAAc,iBAC7ClvD,MAAKkvD,QAAiB,QAAS,MAAiB,cACvDlvD,KAAK6hD,oBAAqB,EAC1B7hD,KAAKujD,kBAAmB,GAU1B3jD,EAAQulG,4BAA8B,WACpC,IAAK,GAAIC,KAAgBplG,MAAKwjD,gBACxBxjD,KAAKwjD,gBAAgB19C,eAAes/F,KACtCplG,KAAKolG,GAAgBplG,KAAKwjD,gBAAgB4hD,SACnCplG,MAAKwjD,gBAAgB4hD,KAUlCxlG,EAAQylG,gBAAkB,WACxBrlG,KAAKkoD,UAAYloD,KAAKkoD,QACtB,IAAIo9C,GAAUtlG,KAAK0tE,gBACfE,EAAW5tE,KAAK4tE,SAChBD,EAAc3tE,KAAK2tE,WACF,IAAjB3tE,KAAKkoD,UACPo9C,EAAQr4F,MAAM26B,QAAQ,QACtBgmC,EAAS3gE,MAAM26B,QAAQ,QACvB+lC,EAAY1gE,MAAM26B,QAAQ,OAC1BgmC,EAAS57C,QAAUhyB,KAAKqlG,gBAAgBvwE,KAAK90B,QAG7CslG,EAAQr4F,MAAM26B,QAAQ,OACtBgmC,EAAS3gE,MAAM26B,QAAQ,OACvB+lC,EAAY1gE,MAAM26B,QAAQ,QAC1BgmC,EAAS57C,QAAU,MAErBhyB,KAAKmnD,yBAQPvnD,EAAQunD,sBAAwB,WAE1BnnD,KAAKulG,eACPvlG,KAAK0T,IAAI,SAAU1T,KAAKulG,cAG1B,IAAIlhE,GAASrkC,KAAK0hD,UAAU7c,QAAQ7kC,KAAK0hD,UAAUrd,OAqBnD,IAnB6B79B,SAAzBxG,KAAKwlG,kBACPxlG,KAAKwlG,gBAAgB/rC,uBACrBz5D,KAAKwlG,gBAAkBh/F,OACvBxG,KAAKylG,oBAAsB,KAC3BzlG,KAAK6hD,oBAAqB,EAC1B7hD,KAAK6iD,WAIP7iD,KAAKmlG,8BAGLnlG,KAAKujD,kBAAmB,EAGxBvjD,KAAKwtE,8BAA+B,EACpCxtE,KAAKytE,sBAAuB,EAC5BztE,KAAKklG,mBAEgB,GAAjBllG,KAAKkoD,SAAkB,CACzB,KAAOloD,KAAK0tE,gBAAgBhqD,iBAC1B1jB,KAAK0tE,gBAAgBv8D,YAAYnR,KAAK0tE,gBAAgB/pD,WAGxD3jB,MAAKklG,gBAA6B,YAAI3zF,SAASM,cAAc,QAC7D7R,KAAKklG,gBAA6B,YAAEr9F,UAAY,6BAChD7H,KAAKklG,gBAAkC,iBAAI3zF,SAASM,cAAc,QAClE7R,KAAKklG,gBAAkC,iBAAEr9F,UAAY,4BACrD7H,KAAKklG,gBAAkC,iBAAEjhF,UAAYogB,EAAgB,QACrErkC,KAAKklG,gBAA6B,YAAEzzF,YAAYzR,KAAKklG,gBAAkC,kBAEvFllG,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,OACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,wBAEtD7H,KAAKklG,gBAA6B,YAAI3zF,SAASM,cAAc,QAC7D7R,KAAKklG,gBAA6B,YAAEr9F,UAAY,iCAChD7H,KAAKklG,gBAAkC,iBAAI3zF,SAASM,cAAc,QAClE7R,KAAKklG,gBAAkC,iBAAEr9F,UAAY,4BACrD7H,KAAKklG,gBAAkC,iBAAEjhF,UAAYogB,EAAgB,QACrErkC,KAAKklG,gBAA6B,YAAEzzF,YAAYzR,KAAKklG,gBAAkC,kBAEvFllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAA6B,aACnEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAmC,mBACzEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAA6B,aAE/B,GAAhCllG,KAAK0jG,yBAAgC1jG,KAAK28C,iBAAiBC,MAC7D58C,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,OACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,wBAEtD7H,KAAKklG,gBAA8B,aAAI3zF,SAASM,cAAc,QAC9D7R,KAAKklG,gBAA8B,aAAEr9F,UAAY,8BACjD7H,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,QACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,4BACtD7H,KAAKklG,gBAAmC,kBAAEjhF,UAAYogB,EAAiB,SACvErkC,KAAKklG,gBAA8B,aAAEzzF,YAAYzR,KAAKklG,gBAAmC,mBAEzFllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAmC,mBACzEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAA8B,eAE7B,GAAhCllG,KAAK6jG,yBAAgE,GAAhC7jG,KAAK0jG,0BACjD1jG,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,OACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,wBAEtD7H,KAAKklG,gBAA8B,aAAI3zF,SAASM,cAAc,QAC9D7R,KAAKklG,gBAA8B,aAAEr9F,UAAY,8BACjD7H,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,QACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,4BACtD7H,KAAKklG,gBAAmC,kBAAEjhF,UAAYogB,EAAiB,SACvErkC,KAAKklG,gBAA8B,aAAEzzF,YAAYzR,KAAKklG,gBAAmC,mBAEzFllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAmC,mBACzEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAA8B,eAEtC,GAA5BllG,KAAK+jG,sBACP/jG,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,OACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,wBAEtD7H,KAAKklG,gBAA4B,WAAI3zF,SAASM,cAAc,QAC5D7R,KAAKklG,gBAA4B,WAAEr9F,UAAY,gCAC/C7H,KAAKklG,gBAAiC,gBAAI3zF,SAASM,cAAc,QACjE7R,KAAKklG,gBAAiC,gBAAEr9F,UAAY,4BACpD7H,KAAKklG,gBAAiC,gBAAEjhF,UAAYogB,EAAY,IAChErkC,KAAKklG,gBAA4B,WAAEzzF,YAAYzR,KAAKklG,gBAAiC,iBAErFllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAmC,mBACzEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAA4B,aAKpEllG,KAAKklG,gBAA6B,YAAElzE,QAAUhyB,KAAK0lG,sBAAsB5wE,KAAK90B,MAC9EA,KAAKklG,gBAA6B,YAAElzE,QAAUhyB,KAAK2lG,sBAAsB7wE,KAAK90B,MAC1C,GAAhCA,KAAK0jG,yBAAgC1jG,KAAK28C,iBAAiBC,KAC7D58C,KAAKklG,gBAA8B,aAAElzE,QAAUhyB,KAAK4lG,UAAU9wE,KAAK90B,MAE5B,GAAhCA,KAAK6jG,yBAAgE,GAAhC7jG,KAAK0jG,0BACjD1jG,KAAKklG,gBAA8B,aAAElzE,QAAUhyB,KAAK6lG,uBAAuB/wE,KAAK90B,OAElD,GAA5BA,KAAK+jG,sBACP/jG,KAAKklG,gBAA4B,WAAElzE,QAAUhyB,KAAKmqD,gBAAgBr1B,KAAK90B,OAEzEA,KAAK4tE,SAAS57C,QAAUhyB,KAAKqlG,gBAAgBvwE,KAAK90B,KAElD;GAAImU,GAAKnU,IACTA,MAAKulG,cAAgBpxF,EAAGgzC,sBACxBnnD,KAAKuT,GAAG,SAAUvT,KAAKulG,mBAEpB,CACH,KAAOvlG,KAAK2tE,YAAYjqD,iBACtB1jB,KAAK2tE,YAAYx8D,YAAYnR,KAAK2tE,YAAYhqD,WAGhD3jB,MAAKklG,gBAA8B,aAAI3zF,SAASM,cAAc,QAC9D7R,KAAKklG,gBAA8B,aAAEr9F,UAAY,uCACjD7H,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,QACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,4BACtD7H,KAAKklG,gBAAmC,kBAAEjhF,UAAYogB,EAAa,KACnErkC,KAAKklG,gBAA8B,aAAEzzF,YAAYzR,KAAKklG,gBAAmC,mBAEzFllG,KAAK2tE,YAAYl8D,YAAYzR,KAAKklG,gBAA8B,cAEhEllG,KAAKklG,gBAA8B,aAAElzE,QAAUhyB,KAAKqlG,gBAAgBvwE,KAAK90B,QAW7EJ,EAAQ8lG,sBAAwB,WAE9B1lG,KAAKilG,uBACDjlG,KAAKulG,eACPvlG,KAAK0T,IAAI,SAAU1T,KAAKulG,cAG1B,IAAIlhE,GAASrkC,KAAK0hD,UAAU7c,QAAQ7kC,KAAK0hD,UAAUrd,OAEnDrkC,MAAKklG,mBACLllG,KAAKklG,gBAA0B,SAAI3zF,SAASM,cAAc,QAC1D7R,KAAKklG,gBAA0B,SAAEr9F,UAAY,8BAC7C7H,KAAKklG,gBAA+B,cAAI3zF,SAASM,cAAc,QAC/D7R,KAAKklG,gBAA+B,cAAEr9F,UAAY,4BAClD7H,KAAKklG,gBAA+B,cAAEjhF,UAAYogB,EAAa,KAC/DrkC,KAAKklG,gBAA0B,SAAEzzF,YAAYzR,KAAKklG,gBAA+B,eAEjFllG,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,OACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,wBAEtD7H,KAAKklG,gBAAiC,gBAAI3zF,SAASM,cAAc,QACjE7R,KAAKklG,gBAAiC,gBAAEr9F,UAAY,8BACpD7H,KAAKklG,gBAAsC,qBAAI3zF,SAASM,cAAc,QACtE7R,KAAKklG,gBAAsC,qBAAEr9F,UAAY,4BACzD7H,KAAKklG,gBAAsC,qBAAEjhF,UAAYogB,EAAuB,eAChFrkC,KAAKklG,gBAAiC,gBAAEzzF,YAAYzR,KAAKklG,gBAAsC,sBAE/FllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAA0B,UAChEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAmC,mBACzEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAiC,iBAGvEllG,KAAKklG,gBAA0B,SAAElzE,QAAUhyB,KAAKmnD,sBAAsBryB,KAAK90B,KAG3E,IAAImU,GAAKnU,IACTA,MAAKulG,cAAgBpxF,EAAG2xF,SACxB9lG,KAAKuT,GAAG,SAAUvT,KAAKulG,gBASzB3lG,EAAQ+lG,sBAAwB,WAE9B3lG,KAAKilG,uBACLjlG,KAAKu+F,cAAa,GAClBv+F,KAAKujD,kBAAmB,EAEpBvjD,KAAKulG,eACPvlG,KAAK0T,IAAI,SAAU1T,KAAKulG,cAG1B,IAAIlhE,GAASrkC,KAAK0hD,UAAU7c,QAAQ7kC,KAAK0hD,UAAUrd,OAEnDrkC,MAAKu+F,eACLv+F,KAAKytE,sBAAuB,EAC5BztE,KAAKwtE,8BAA+B,EAEpCxtE,KAAKklG,mBACLllG,KAAKklG,gBAA0B,SAAI3zF,SAASM,cAAc,QAC1D7R,KAAKklG,gBAA0B,SAAEr9F,UAAY,8BAC7C7H,KAAKklG,gBAA+B,cAAI3zF,SAASM,cAAc,QAC/D7R,KAAKklG,gBAA+B,cAAEr9F,UAAY,4BAClD7H,KAAKklG,gBAA+B,cAAEjhF,UAAYogB,EAAa,KAC/DrkC,KAAKklG,gBAA0B,SAAEzzF,YAAYzR,KAAKklG,gBAA+B,eAEjFllG,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,OACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,wBAEtD7H,KAAKklG,gBAAiC,gBAAI3zF,SAASM,cAAc,QACjE7R,KAAKklG,gBAAiC,gBAAEr9F,UAAY,8BACpD7H,KAAKklG,gBAAsC,qBAAI3zF,SAASM,cAAc,QACtE7R,KAAKklG,gBAAsC,qBAAEr9F,UAAY,4BACzD7H,KAAKklG,gBAAsC,qBAAEjhF,UAAYogB,EAAwB,gBACjFrkC,KAAKklG,gBAAiC,gBAAEzzF,YAAYzR,KAAKklG,gBAAsC,sBAE/FllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAA0B,UAChEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAmC,mBACzEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAiC,iBAGvEllG,KAAKklG,gBAA0B,SAAElzE,QAAUhyB,KAAKmnD,sBAAsBryB,KAAK90B,KAG3E,IAAImU,GAAKnU,IACTA,MAAKulG,cAAgBpxF,EAAG4xF,eACxB/lG,KAAKuT,GAAG,SAAUvT,KAAKulG,eAGvBvlG,KAAKwjD,gBAA8B,aAAIxjD,KAAK0qD,aAC5C1qD,KAAKwjD,gBAA8C,6BAAIxjD,KAAKwkG,6BAC5DxkG,KAAKwjD,gBAAkC,iBAAIxjD,KAAK2qD,iBAChD3qD,KAAKwjD,gBAAgC,eAAIxjD,KAAK2rD,eAC9C3rD,KAAK0qD,aAAe1qD,KAAK+lG,eACzB/lG,KAAKwkG,6BAA+B,aACpCxkG,KAAK2qD,iBAAmB,aACxB3qD,KAAK2rD,eAAiB3rD,KAAKgmG,eAG3BhmG,KAAK6iD,WAQPjjD,EAAQimG,uBAAyB,WAE/B7lG,KAAKilG,uBACLjlG,KAAK6hD,oBAAqB,EAEtB7hD,KAAKulG,eACPvlG,KAAK0T,IAAI,SAAU1T,KAAKulG,eAG1BvlG,KAAKwlG,gBAAkBxlG,KAAK4jG,mBAC5B5jG,KAAKwlG,gBAAgBhsC,qBAErB,IAAIn1B,GAASrkC,KAAK0hD,UAAU7c,QAAQ7kC,KAAK0hD,UAAUrd,OAEnDrkC,MAAKklG,mBACLllG,KAAKklG,gBAA0B,SAAI3zF,SAASM,cAAc,QAC1D7R,KAAKklG,gBAA0B,SAAEr9F,UAAY,8BAC7C7H,KAAKklG,gBAA+B,cAAI3zF,SAASM,cAAc,QAC/D7R,KAAKklG,gBAA+B,cAAEr9F,UAAY,4BAClD7H,KAAKklG,gBAA+B,cAAEjhF,UAAYogB,EAAa,KAC/DrkC,KAAKklG,gBAA0B,SAAEzzF,YAAYzR,KAAKklG,gBAA+B,eAEjFllG,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,OACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,wBAEtD7H,KAAKklG,gBAAiC,gBAAI3zF,SAASM,cAAc,QACjE7R,KAAKklG,gBAAiC,gBAAEr9F,UAAY,8BACpD7H,KAAKklG,gBAAsC,qBAAI3zF,SAASM,cAAc,QACtE7R,KAAKklG,gBAAsC,qBAAEr9F,UAAY,4BACzD7H,KAAKklG,gBAAsC,qBAAEjhF,UAAYogB,EAA4B,oBACrFrkC,KAAKklG,gBAAiC,gBAAEzzF,YAAYzR,KAAKklG,gBAAsC,sBAE/FllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAA0B,UAChEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAmC,mBACzEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAiC,iBAGvEllG,KAAKklG,gBAA0B,SAAElzE,QAAUhyB,KAAKmnD,sBAAsBryB,KAAK90B,MAG3EA,KAAKwjD,gBAA8B,aAASxjD,KAAK0qD,aACjD1qD,KAAKwjD,gBAA8C,6BAAKxjD,KAAKwkG,6BAC7DxkG,KAAKwjD,gBAA4B,WAAWxjD,KAAK4rD,WACjD5rD,KAAKwjD,gBAAkC,iBAAKxjD,KAAK2qD,iBACjD3qD,KAAKwjD,gBAA+B,cAAQxjD,KAAKqrD,cACjDrrD,KAAK0qD,aAAmB1qD,KAAKimG,mBAC7BjmG,KAAK4rD,WAAmB,aACxB5rD,KAAKqrD,cAAmBrrD,KAAKkmG,iBAC7BlmG,KAAK2qD,iBAAmB,aACxB3qD,KAAKwkG,6BAA+BxkG,KAAKmmG,oBAGzCnmG,KAAK6iD,WAUPjjD,EAAQqmG,mBAAqB,SAASpmE,GACpC7/B,KAAKwlG,gBAAgBhxC,aAAaprC,KAAK8nB,WACvClxC,KAAKwlG,gBAAgBhxC,aAAanrC,GAAG6nB,WACrClxC,KAAKylG,oBAAsBzlG,KAAKwlG,gBAAgB9rC,wBAAwB15D,KAAKurD,qBAAqB1rB,EAAQ9tB,GAAG/R,KAAKyrD,qBAAqB5rB,EAAQ7tB,IAC9G,OAA7BhS,KAAKylG,sBACPzlG,KAAKylG,oBAAoBt0D,SACzBnxC,KAAKujD,kBAAmB,GAE1BvjD,KAAK6iD,WAUPjjD,EAAQsmG,iBAAmB,SAAS58F,GAClC,GAAIu2B,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,OACJ,QAA7BlsB,KAAKylG,qBAA6Dj/F,SAA7BxG,KAAKylG,sBAC5CzlG,KAAKylG,oBAAoB1zF,EAAI/R,KAAKurD,qBAAqB1rB,EAAQ9tB,GAC/D/R,KAAKylG,oBAAoBzzF,EAAIhS,KAAKyrD,qBAAqB5rB,EAAQ7tB,IAEjEhS,KAAK6iD,WASPjjD,EAAQumG,oBAAsB,SAAStmE,GACrC,GAAIumE,GAAUpmG,KAAK4qD,WAAW/qB,EACd,QAAZumE,GACqD,GAAnDpmG,KAAKwlG,gBAAgBhxC,aAAaprC,KAAKiqB,WACzCrzC,KAAKwlG,gBAAgB3rC,uBACrB75D,KAAKqmG,UAAUD,EAAQ/lG,GAAIL,KAAKwlG,gBAAgBn8E,GAAGhpB,IACnDL,KAAKwlG,gBAAgBhxC,aAAaprC,KAAK8nB,YAEY,GAAjDlxC,KAAKwlG,gBAAgBhxC,aAAanrC,GAAGgqB,WACvCrzC,KAAKwlG,gBAAgB3rC,uBACrB75D,KAAKqmG,UAAUrmG,KAAKwlG,gBAAgBp8E,KAAK/oB,GAAI+lG,EAAQ/lG,IACrDL,KAAKwlG,gBAAgBhxC,aAAanrC,GAAG6nB,aAIvClxC,KAAKwlG,gBAAgB3rC,uBAEvB75D,KAAKujD,kBAAmB,EACxBvjD,KAAK6iD,WASPjjD,EAAQmmG,eAAiB,SAASlmE,GAChC,GAAoC,GAAhC7/B,KAAK0jG,wBAA8B,CACrC,GAAI59C,GAAO9lD,KAAK4qD,WAAW/qB,EAE3B,IAAY,MAARimB,EACF,GAAIA,EAAKuW,YAAc,EACrBiqC,MAAMtmG,KAAK0hD,UAAU7c,QAAQ7kC,KAAK0hD,UAAUrd,QAAyB,qBAElE,CACHrkC,KAAK+qD,cAAcjF,GAAK,EACxB,IAAIygD,GAAevmG,KAAKkvD,QAAiB,QAAS,KAGlDq3C,GAAyB,WAAI,GAAIhjG,IAAMlD,GAAG,oBAAoBL,KAAK0hD,UACnE,IAAI8kD,GAAaD,EAAyB,UAC1CC,GAAWz0F,EAAI+zC,EAAK/zC,EACpBy0F,EAAWx0F,EAAI8zC,EAAK9zC,EAGpBhS,KAAK89C,MAAsB,eAAI,GAAI16C,IAAM/C,GAAG,iBAAiB+oB,KAAK08B,EAAKzlD,GAAGgpB,GAAGm9E,EAAWnmG,IAAKL,KAAMA,KAAK0hD,UACxG,IAAI+kD,GAAiBzmG,KAAK89C,MAAsB,cAChD2oD,GAAer9E,KAAO08B,EACtB2gD,EAAe94C,WAAY,EAC3B84C,EAAeh4F,QAAQqyC,cAAgBpyC,SAAS,EAC5CqyC,SAAS,EACTj6C,KAAM,aACNk6C,UAAW,IAEfylD,EAAepzD,UAAW,EAC1BozD,EAAep9E,GAAKm9E,EAEpBxmG,KAAKwjD,gBAA+B,cAAIxjD,KAAKqrD,cAC7CrrD,KAAKqrD,cAAgB,SAAS/hD,GAC5B,GAAIu2B,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,QACjCu6E,EAAiBzmG,KAAK89C,MAAsB,cAChD2oD,GAAep9E,GAAGtX,EAAI/R,KAAKurD,qBAAqB1rB,EAAQ9tB,GACxD00F,EAAep9E,GAAGrX,EAAIhS,KAAKyrD,qBAAqB5rB,EAAQ7tB,IAG1DhS,KAAK8kD,QAAS,EACd9kD,KAAK4P,WAMbhQ,EAAQomG,eAAiB,SAAS18F,GAChC,GAAoC,GAAhCtJ,KAAK0jG,wBAA8B,CACrC,GAAI7jE,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,OAErClsB,MAAKqrD,cAAgBrrD,KAAKwjD,gBAA+B,oBAClDxjD,MAAKwjD,gBAA+B,aAG3C,IAAIkjD,GAAgB1mG,KAAK89C,MAAsB,eAAE6V,aAG1C3zD,MAAK89C,MAAsB,qBAC3B99C,MAAKkvD,QAAiB,QAAS,MAAc,iBAC7ClvD,MAAKkvD,QAAiB,QAAS,MAAiB,aAEvD,IAAIpJ,GAAO9lD,KAAK4qD,WAAW/qB,EACf,OAARimB,IACEA,EAAKuW,YAAc,EACrBiqC,MAAMtmG,KAAK0hD,UAAU7c,QAAQ7kC,KAAK0hD,UAAUrd,QAAyB,kBAGrErkC,KAAK2mG,YAAYD,EAAc5gD,EAAKzlD,IACpCL,KAAKmnD,0BAGTnnD,KAAKu+F,iBAQT3+F,EAAQkmG,SAAW,WACjB,GAAI9lG,KAAK+jG,qBAAwC,GAAjB/jG,KAAKkoD,SAAkB,CACrD,GAAIg7C,GAAiBljG,KAAKijG,yBAAyBjjG,KAAKikD,iBACpD2iD,GAAevmG,GAAGM,EAAKqE,aAAa+M,EAAEmxF,EAAez7F,KAAKuK,EAAEkxF,EAAev7F,IAAI8gB,MAAM,MAAM+pC,gBAAe,EAAKC,gBAAe,EAClI,IAAIzyD,KAAK28C,iBAAiB1pC,IAAK,CAC7B,GAAwC,GAApCjT,KAAK28C,iBAAiB1pC,IAAItN,OAU5B,KAAM,IAAI/B,OAAM,sEAThB,IAAIuQ,GAAKnU,IACTA,MAAK28C,iBAAiB1pC,IAAI2zF,EAAa,SAASC,GAC9C1yF,EAAGiwC,UAAUnxC,IAAI4zF,GACjB1yF,EAAGgzC,wBACHhzC,EAAG2wC,QAAS,EACZ3wC,EAAGvE,cAWP5P,MAAKokD,UAAUnxC,IAAI2zF,GACnB5mG,KAAKmnD,wBACLnnD,KAAK8kD,QAAS,EACd9kD,KAAK4P,UAWXhQ,EAAQ+mG,YAAc,SAASG,EAAaC,GAC1C,GAAqB,GAAjB/mG,KAAKkoD,SAAkB,CACzB,GAAI0+C,IAAex9E,KAAK09E,EAAcz9E,GAAG09E,EACzC,IAAI/mG,KAAK28C,iBAAiBG,QAAS,CACjC,GAA4C,GAAxC98C,KAAK28C,iBAAiBG,QAAQn3C,OAShC,KAAM,IAAI/B,OAAM,0EARhB,IAAIuQ,GAAKnU,IACTA,MAAK28C,iBAAiBG,QAAQ8pD,EAAa,SAASC,GAClD1yF,EAAGkwC,UAAUpxC,IAAI4zF,GACjB1yF,EAAG2wC,QAAS,EACZ3wC,EAAGvE,cAUP5P,MAAKqkD,UAAUpxC,IAAI2zF,GACnB5mG,KAAK8kD,QAAS,EACd9kD,KAAK4P,UAUXhQ,EAAQymG,UAAY,SAASS,EAAaC,GACxC,GAAqB,GAAjB/mG,KAAKkoD,SAAkB,CACzB,GAAI0+C,IAAevmG,GAAIL,KAAKwlG,gBAAgBnlG,GAAI+oB,KAAK09E,EAAcz9E,GAAG09E,EACtE,IAAI/mG,KAAK28C,iBAAiBE,SAAU,CAClC,GAA6C,GAAzC78C,KAAK28C,iBAAiBE,SAASl3C,OASjC,KAAM,IAAI/B,OAAM,wEARhB,IAAIuQ,GAAKnU,IACTA,MAAK28C,iBAAiBE,SAAS+pD,EAAa,SAASC,GACnD1yF,EAAGkwC,UAAUxvC,OAAOgyF,GACpB1yF,EAAG2wC,QAAS,EACZ3wC,EAAGvE,cAUP5P,MAAKqkD,UAAUxvC,OAAO+xF,GACtB5mG,KAAK8kD,QAAS,EACd9kD,KAAK4P,UAUXhQ,EAAQgmG,UAAY,WAClB,IAAI5lG,KAAK28C,iBAAiBC,MAAyB,GAAjB58C,KAAKkoD,SA4BrC,KAAM,IAAItkD,OAAM,iDA3BhB,IAAIkiD,GAAO9lD,KAAK2jG,mBACZjxF,GAAQrS,GAAGylD,EAAKzlD,GAClBooB,MAAOq9B,EAAKr9B,MACZxW,MAAO6zC,EAAKr3C,QAAQwD,MACpBmrC,MAAO0I,EAAKr3C,QAAQ2uC,MACpBjyC,OACEgB,WAAW25C,EAAKr3C,QAAQtD,MAAMgB,WAC9BC,OAAO05C,EAAKr3C,QAAQtD,MAAMiB,OAC1BC,WACEF,WAAW25C,EAAKr3C,QAAQtD,MAAMkB,UAAUF,WACxCC,OAAO05C,EAAKr3C,QAAQtD,MAAMkB,UAAUD,SAG1C,IAAyC,GAArCpM,KAAK28C,iBAAiBC,KAAKj3C,OAU7B,KAAM,IAAI/B,OAAM,wEAThB,IAAIuQ,GAAKnU,IACTA,MAAK28C,iBAAiBC,KAAKlqC,EAAM,SAAUm0F,GACzC1yF,EAAGiwC,UAAUvvC,OAAOgyF,GACpB1yF,EAAGgzC,wBACHhzC,EAAG2wC,QAAS,EACZ3wC,EAAGvE,WAoBXhQ,EAAQuqD,gBAAkB,WACxB,IAAKnqD,KAAK+jG,qBAAwC,GAAjB/jG,KAAKkoD,SACpC,GAAKloD,KAAKgkG,sBA4BRsC,MAAMtmG,KAAK0hD,UAAU7c,QAAQ7kC,KAAK0hD,UAAUrd,QAA4B,wBA5BzC,CAC/B,GAAI2iE,GAAgBhnG,KAAK0kG,mBACrBuC,EAAgBjnG,KAAK4kG,kBACzB,IAAI5kG,KAAK28C,iBAAiBI,IAAK,CAC7B,GAAI5oC,GAAKnU,KACL0S,GAAQsqC,MAAOgqD,EAAelpD,MAAOmpD,EACzC,IAAwC,GAApCjnG,KAAK28C,iBAAiBI,IAAIp3C,OAU5B,KAAM,IAAI/B,OAAM,0EAThB5D,MAAK28C,iBAAiBI,IAAIrqC,EAAM,SAAUm0F,GACxC1yF,EAAGkwC,UAAUhuC,OAAOwwF,EAAc/oD,OAClC3pC,EAAGiwC,UAAU/tC,OAAOwwF,EAAc7pD,OAClC7oC,EAAGoqF,eACHpqF,EAAG2wC,QAAS,EACZ3wC,EAAGvE,cAQP5P,MAAKqkD,UAAUhuC,OAAO4wF,GACtBjnG,KAAKokD,UAAU/tC,OAAO2wF,GACtBhnG,KAAKu+F,eACLv+F,KAAK8kD,QAAS,EACd9kD,KAAK4P,WAYT,SAAS/P,EAAQD,EAASM,GAE9B,GACIklD,IADOllD,EAAoB,GACdA,EAAoB,KACjC6D,EAAS7D,EAAoB,GAEjCN,GAAQiuE,iBAAmB,WAEzB,GAA8C,GAA1C7tE,KAAK8hD,kBAAkBC,SAASp8C,OAAa,CAC/C,IAAK,GAAIH,GAAI,EAAGA,EAAIxF,KAAK8hD,kBAAkBC,SAASp8C,OAAQH,IAC1DxF,KAAK8hD,kBAAkBC,SAASv8C,GAAG8N,SAErCtT,MAAK8hD,kBAAkBC,YAGzB/hD,KAAKykG,2BAA6B,aAG9BzkG,KAAKknG,gBAAkBlnG,KAAKknG,eAAwB,SAAKlnG,KAAKknG,eAAwB,QAAEr9F,YAC1F7J,KAAKknG,eAAwB,QAAEr9F,WAAWsH,YAAYnR,KAAKknG,eAAwB,UAYvFtnG,EAAQkuE,wBAA0B,WAChC9tE,KAAK6tE,mBAEL7tE,KAAKknG,iBACL,IAAIA,IAAkB,KAAK,OAAO,OAAO,QAAQ,SAAS,UAAU,eAChEC,GAAwB,UAAU,YAAY,YAAY,aAAa,UAAU,WAAW,cAEhGnnG,MAAKknG,eAAwB,QAAI31F,SAASM,cAAc,OACxD7R,KAAKsf,MAAM7N,YAAYzR,KAAKknG,eAAwB,QAEpD,KAAK,GAAI1hG,GAAI,EAAGA,EAAI0hG,EAAevhG,OAAQH,IAAK,CAC9CxF,KAAKknG,eAAeA,EAAe1hG,IAAM+L,SAASM,cAAc,OAChE7R,KAAKknG,eAAeA,EAAe1hG,IAAIqC,UAAY,sBAAwBq/F,EAAe1hG,GAC1FxF,KAAKknG,eAAwB,QAAEz1F,YAAYzR,KAAKknG,eAAeA,EAAe1hG,IAE9E,IAAI1B,GAAS,GAAIC,GAAO/D,KAAKknG,eAAeA,EAAe1hG,KAAMwoE,iBAAiB,GAClF5oB,GAAWkE,QAAQxlD,EAAQ9D,KAAKmnG,EAAqB3hG,IAAIsvB,KAAK90B,OAC9DolD,EAAW6hB,UAAUnjE,EAAQ9D,KAAK+rD,WAAWj3B,KAAK90B,OAElDA,KAAK8hD,kBAAkBE,KAAKh6C,KAAKlE,GAGnC9D,KAAKykG,2BAA6BzkG,KAAKonG,cAEvCpnG,KAAK8hD,kBAAkBC,SAAW/hD,KAAK8hD,kBAAkBE,MAS3DpiD,EAAQynG,YAAc,SAAS/9F,GAC7BtJ,KAAKilD,YAAYn1C,SAAS,MAC1BxG,EAAMq8B,mBAQR/lC,EAAQwnG,cAAgB,WACtBpnG,KAAK8pD,eACL9pD,KAAK2pD,eACL3pD,KAAKiqD,aAYPrqD,EAAQ8pD,QAAU,SAASpgD,GACzBtJ,KAAK+iD,WAAa/iD,KAAK0hD,UAAUrB,SAASC,MAAMtuC,EAChDhS,KAAK4P,QACLtG,EAAMD,kBAQRzJ,EAAQgqD,UAAY,SAAStgD,GAC3BtJ,KAAK+iD,YAAc/iD,KAAK0hD,UAAUrB,SAASC,MAAMtuC,EACjDhS,KAAK4P,QACLtG,EAAMD,kBAQRzJ,EAAQiqD,UAAY,SAASvgD,GAC3BtJ,KAAK8iD,WAAa9iD,KAAK0hD,UAAUrB,SAASC,MAAMvuC,EAChD/R,KAAK4P,QACLtG,EAAMD,kBAQRzJ,EAAQmqD,WAAa,SAASzgD,GAC5BtJ,KAAK8iD,YAAc9iD,KAAK0hD,UAAUrB,SAASC,MAAMtuC,EACjDhS,KAAK4P,QACLtG,EAAMD,kBAQRzJ,EAAQoqD,QAAU,SAAS1gD,GACzBtJ,KAAKgjD,cAAgBhjD,KAAK0hD,UAAUrB,SAASC,MAAMpgB,KACnDlgC,KAAK4P,QACLtG,EAAMD,kBAQRzJ,EAAQsqD,SAAW,SAAS5gD,GAC1BtJ,KAAKgjD,eAAiBhjD,KAAK0hD,UAAUrB,SAASC,MAAMpgB,KACpDlgC,KAAK4P,QACLtG,EAAMD,kBAQRzJ,EAAQqqD,UAAY,SAAS3gD,GAC3BtJ,KAAKgjD,cAAgB,EACrB15C,GAASA,EAAMD,kBAQjBzJ,EAAQ+pD,aAAe,SAASrgD,GAC9BtJ,KAAK+iD,WAAa,EAClBz5C,GAASA,EAAMD,kBAQjBzJ,EAAQkqD,aAAe,SAASxgD,GAC9BtJ,KAAK8iD,WAAa,EAClBx5C,GAASA,EAAMD,mBAMb,SAASxJ,EAAQD,GAErBA,EAAQgoD,aAAe,WACrB,IAAK,GAAIzB,KAAUnmD,MAAKg9C,MACtB,GAAIh9C,KAAKg9C,MAAMl3C,eAAeqgD,GAAS,CACrC,GAAIL,GAAO9lD,KAAKg9C,MAAMmJ,EACO,IAAzBL,EAAKuV,mBACPvV,EAAKlI,MAAQ,GACbkI,EAAKwV,qBAAsB,KAYnC17D,EAAQolD,yBAA2B,WACjC,GAAiD,GAA7ChlD,KAAK0hD,UAAUjB,mBAAmB/xC,SAAmB1O,KAAK8jD,YAAYn+C,OAAS,EAAG,CAEpF,GACImgD,GAAMK,EADNmhD,EAAU,EAEVC,GAAe,EACfC,GAAiB,CAErB,KAAKrhD,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GACA,IAAdL,EAAKlI,MACP2pD,GAAe,EAGfC,GAAiB,EAEfF,EAAUxhD,EAAKhI,MAAMn4C,SACvB2hG,EAAUxhD,EAAKhI,MAAMn4C,QAM3B,IAAsB,GAAlB6hG,GAA0C,GAAhBD,EAC5B,KAAM,IAAI3jG,OAAM,wHAQhB5D,MAAKynG,mBAGiB,GAAlBD,IAC8C,WAA5CxnG,KAAK0hD,UAAUjB,mBAAmBG,OACpC5gD,KAAK0nG,iBAAiBJ,GAGtBtnG,KAAK2nG,0BAAyB,GAKlC,IAAIC,GAAe5nG,KAAK6nG,kBAGxB7nG,MAAK8nG,uBAAuBF,GAG5B5nG,KAAK4P,UAYXhQ,EAAQkoG,uBAAyB,SAASF,GACxC,GAAIzhD,GAAQL,CAGZ,KAAK,GAAIlI,KAASgqD,GAChB,GAAIA,EAAa9hG,eAAe83C,GAE9B,IAAKuI,IAAUyhD,GAAahqD,GAAOZ,MAC7B4qD,EAAahqD,GAAOZ,MAAMl3C,eAAeqgD,KAC3CL,EAAO8hD,EAAahqD,GAAOZ,MAAMmJ,GACkB,MAA/CnmD,KAAK0hD,UAAUjB,mBAAmBvlB,WAAoE,MAA/Cl7B,KAAK0hD,UAAUjB,mBAAmBvlB,UACvF4qB,EAAKqF,SACPrF,EAAK/zC,EAAI61F,EAAahqD,GAAOmqD,OAC7BjiD,EAAKqF,QAAS,EAEdy8C,EAAahqD,GAAOmqD,QAAUH,EAAahqD,GAAO+C,aAIhDmF,EAAKsF,SACPtF,EAAK9zC,EAAI41F,EAAahqD,GAAOmqD,OAC7BjiD,EAAKsF,QAAS,EAEdw8C,EAAahqD,GAAOmqD,QAAUH,EAAahqD,GAAO+C,aAGtD3gD,KAAKgoG,kBAAkBliD,EAAKhI,MAAMgI,EAAKzlD,GAAGunG,EAAa9hD,EAAKlI,OAOpE59C,MAAK6nD,cAUPjoD,EAAQioG,iBAAmB,WACzB,GACI1hD,GAAQL,EAAMlI,EADdgqD,IAKJ,KAAKzhD,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GAClBL,EAAKqF,QAAS,EACdrF,EAAKsF,QAAS,EACqC,MAA/CprD,KAAK0hD,UAAUjB,mBAAmBvlB,WAAoE,MAA/Cl7B,KAAK0hD,UAAUjB,mBAAmBvlB,UAC3F4qB,EAAK9zC,EAAIhS,KAAK0hD,UAAUjB,mBAAmBC,gBAAgBoF,EAAKlI,MAGhEkI,EAAK/zC,EAAI/R,KAAK0hD,UAAUjB,mBAAmBC,gBAAgBoF,EAAKlI,MAEjCp3C,SAA7BohG,EAAa9hD,EAAKlI,SACpBgqD,EAAa9hD,EAAKlI,QAAUquB,OAAQ,EAAGjvB,SAAW+qD,OAAO,EAAGpnD,YAAY,IAE1EinD,EAAa9hD,EAAKlI,OAAOquB,QAAU,EACnC27B,EAAa9hD,EAAKlI,OAAOZ,MAAMmJ,GAAUL,EAK7C,IAAImiD,GAAW,CACf,KAAKrqD,IAASgqD,GACRA,EAAa9hG,eAAe83C,IAC1BqqD,EAAWL,EAAahqD,GAAOquB,SACjCg8B,EAAWL,EAAahqD,GAAOquB,OAMrC,KAAKruB,IAASgqD,GACRA,EAAa9hG,eAAe83C,KAC9BgqD,EAAahqD,GAAO+C,aAAesnD,EAAW,GAAKjoG,KAAK0hD,UAAUjB,mBAAmBE,YACrFinD,EAAahqD,GAAO+C,aAAgBinD,EAAahqD,GAAOquB,OAAS,EACjE27B,EAAahqD,GAAOmqD,OAASH,EAAahqD,GAAO+C,YAAe,IAAOinD,EAAahqD,GAAOquB,OAAS,GAAK27B,EAAahqD,GAAO+C,YAIjI,OAAOinD,IAUThoG,EAAQ8nG,iBAAmB,SAASJ,GAClC,GAAInhD,GAAQL,CAGZ,KAAKK,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GACdL,EAAKhI,MAAMn4C,QAAU2hG,IACvBxhD,EAAKlI,MAAQ,GAMnB,KAAKuI,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GACA,GAAdL,EAAKlI,OACP59C,KAAKkoG,UAAU,EAAEpiD,EAAKhI,MAAMgI,EAAKzlD,MAczCT,EAAQ+nG,yBAA2B,WACjC,GAAIxhD,GAAQL,EAAMqiD,EACd5H,EAAW,GAGf4H,GAAYnoG,KAAKg9C,MAAMh9C,KAAK8jD,YAAY,IACxCqkD,EAAUvqD,MAAQ2iD,EAClBvgG,KAAKooG,kBAAkB7H,EAAS4H,EAAUrqD,MAAMqqD,EAAU9nG,GAG1D,KAAK8lD,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GAClBo6C,EAAWz6C,EAAKlI,MAAQ2iD,EAAWz6C,EAAKlI,MAAQ2iD,EAKpD,KAAKp6C,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GAClBL,EAAKlI,OAAS2iD,IAepB3gG,EAAQ6nG,iBAAmB,WACzBznG,KAAK0hD,UAAUvC,WAAWzwC,SAAU,EACpC1O,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,SAAU,EAC3C1O,KAAK0hD,UAAUlD,QAAQU,sBAAsBxwC,SAAU,EACvD1O,KAAKmtE,2BACsC,GAAvCntE,KAAK0hD,UAAUZ,aAAapyC,UAC9B1O,KAAK0hD,UAAUZ,aAAaC,SAAU,GAExC/gD,KAAK0oD,wBAEL,IAAI8oB,GAASxxE,KAAK0hD,UAAUjB,kBAC5B+wB,GAAO9wB,gBAAkBx7C,KAAK2lB,IAAI2mD,EAAO9wB,kBACjB,MAApB8wB,EAAOt2C,WAAyC,MAApBs2C,EAAOt2C,aACrCs2C,EAAO9wB,iBAAmB,IAGJ,MAApB8wB,EAAOt2C,WAAyC,MAApBs2C,EAAOt2C,UACM,GAAvCl7B,KAAK0hD,UAAUZ,aAAapyC,UAC9B1O,KAAK0hD,UAAUZ,aAAah6C,KAAO,YAIM,GAAvC9G,KAAK0hD,UAAUZ,aAAapyC,UAC9B1O,KAAK0hD,UAAUZ,aAAah6C,KAAO,eAgBzClH,EAAQooG,kBAAoB,SAASlqD,EAAOuqD,EAAUT,EAAcU,GAClE,IAAK,GAAI9iG,GAAI,EAAGA,EAAIs4C,EAAMn4C,OAAQH,IAAK,CACrC,GAAI64F,GAAY,IAEdA,GADEvgD,EAAMt4C,GAAGouD,MAAQy0C,EACPvqD,EAAMt4C,GAAG4jB,KAGT00B,EAAMt4C,GAAG6jB,EAIvB,IAAIk/E,IAAY,CACmC,OAA/CvoG,KAAK0hD,UAAUjB,mBAAmBvlB,WAAoE,MAA/Cl7B,KAAK0hD,UAAUjB,mBAAmBvlB,UACvFmjE,EAAUlzC,QAAUkzC,EAAUzgD,MAAQ0qD,IACxCjK,EAAUlzC,QAAS,EACnBkzC,EAAUtsF,EAAI61F,EAAavJ,EAAUzgD,OAAOmqD,OAC5CQ,GAAY,GAIVlK,EAAUjzC,QAAUizC,EAAUzgD,MAAQ0qD,IACxCjK,EAAUjzC,QAAS,EACnBizC,EAAUrsF,EAAI41F,EAAavJ,EAAUzgD,OAAOmqD,OAC5CQ,GAAY,GAIC,GAAbA,IACFX,EAAavJ,EAAUzgD,OAAOmqD,QAAUH,EAAavJ,EAAUzgD,OAAO+C,YAClE09C,EAAUvgD,MAAMn4C,OAAS,GAC3B3F,KAAKgoG,kBAAkB3J,EAAUvgD,MAAMugD,EAAUh+F,GAAGunG,EAAavJ,EAAUzgD,UAenFh+C,EAAQsoG,UAAY,SAAStqD,EAAOE,EAAOuqD,GACzC,IAAK,GAAI7iG,GAAI,EAAGA,EAAIs4C,EAAMn4C,OAAQH,IAAK,CACrC,GAAI64F,GAAY,IAEdA,GADEvgD,EAAMt4C,GAAGouD,MAAQy0C,EACPvqD,EAAMt4C,GAAG4jB,KAGT00B,EAAMt4C,GAAG6jB,IAEA,IAAnBg1E,EAAUzgD,OAAeygD,EAAUzgD,MAAQA,KAC7CygD,EAAUzgD,MAAQA,EACdygD,EAAUvgD,MAAMn4C,OAAS,GAC3B3F,KAAKkoG,UAAUtqD,EAAM,EAAGygD,EAAUvgD,MAAOugD,EAAUh+F,OAe3DT,EAAQwoG,kBAAoB,SAASxqD,EAAOE,EAAOuqD,GACjDroG,KAAKg9C,MAAMqrD,GAAU/sC,qBAAsB,CAE3C,KAAK,GADD+iC,GAAWnjE,EACN11B,EAAI,EAAGA,EAAIs4C,EAAMn4C,OAAQH,IAChC01B,EAAY,EACR4iB,EAAMt4C,GAAGouD,MAAQy0C,GACnBhK,EAAYvgD,EAAMt4C,GAAG4jB,KACrB8R,EAAY,IAGZmjE,EAAYvgD,EAAMt4C,GAAG6jB,GAEA,IAAnBg1E,EAAUzgD,QACZygD,EAAUzgD,MAAQA,EAAQ1iB,EAI9B,KAAK,GAAI11B,GAAI,EAAGA,EAAIs4C,EAAMn4C,OAAQH,IACA64F,EAA5BvgD,EAAMt4C,GAAGouD,MAAQy0C,EAAuBvqD,EAAMt4C,GAAG4jB,KACnC00B,EAAMt4C,GAAG6jB,GAEvBg1E,EAAUvgD,MAAMn4C,OAAS,GAAK04F,EAAU/iC,uBAAwB,GAClEt7D,KAAKooG,kBAAkB/J,EAAUzgD,MAAOygD,EAAUvgD,MAAOugD,EAAUh+F,KAWzET,EAAQ4oG,cAAgB,WACtB,IAAK,GAAIriD,KAAUnmD,MAAKg9C,MAClBh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BnmD,KAAKg9C,MAAMmJ,GAAQgF,QAAS,EAC5BnrD,KAAKg9C,MAAMmJ,GAAQiF,QAAS,KAQ9B,SAASvrD,EAAQD,EAASM,GAkgB9B,QAASuoG,KACPzoG,KAAK0hD,UAAUZ,aAAapyC,SAAW1O,KAAK0hD,UAAUZ,aAAapyC,OACnE,IAAIg6F,GAAqBn3F,SAASo3F,eAAe,qBACCD,GAAmBz7F,MAAMd,WAAhC,GAAvCnM,KAAK0hD,UAAUZ,aAAapyC,QAAwD,UACR,UAEhF1O,KAAK0oD,wBAAuB,GAO9B,QAASkgD,KACP,IAAK,GAAIziD,KAAUnmD,MAAK4jD,iBAClB5jD,KAAK4jD,iBAAiB99C,eAAeqgD,KACvCnmD,KAAK4jD,iBAAiBuC,GAAQsV,GAAK,EAAIz7D,KAAK4jD,iBAAiBuC,GAAQuV,GAAK,EAC1E17D,KAAK4jD,iBAAiBuC,GAAQoV,GAAK,EAAIv7D,KAAK4jD,iBAAiBuC,GAAQqV,GAAK,EAG7B,IAA7Cx7D,KAAK0hD,UAAUjB,mBAAmB/xC,SACpC1O,KAAKglD,2BACL6jD,EAAiBtoG,KAAKP,KAAM,aAAc,EAAG,8CAC7C6oG,EAAiBtoG,KAAKP,KAAM,aAAc,EAAG,0BAC7C6oG,EAAiBtoG,KAAKP,KAAM,aAAc,EAAG,0BAC7C6oG,EAAiBtoG,KAAKP,KAAM,aAAc,EAAG,wBAC7C6oG,EAAiBtoG,KAAKP,KAAM,eAAgB,EAAG,oBAG/CA,KAAK08F,kBAEP18F,KAAK8kD,QAAS,EACd9kD,KAAK4P,QAMP,QAASk5F,KACP,GAAIr6F,GAAU,gDACVs6F,KACAC,EAAez3F,SAASo3F,eAAe,wBACvCM,EAAe13F,SAASo3F,eAAe,uBAC3C,IAA4B,GAAxBK,EAAaE,QAAiB,CAMhC,GALIlpG,KAAK0hD,UAAUlD,QAAQC,UAAUE,uBAAyB3+C,KAAKmpG,gBAAgB3qD,QAAQC,UAAUE,uBAAwBoqD,EAAgB/gG,KAAK,0BAA4BhI,KAAK0hD,UAAUlD,QAAQC,UAAUE,uBAC3M3+C,KAAK0hD,UAAUlD,QAAQI,gBAAkB5+C,KAAKmpG,gBAAgB3qD,QAAQC,UAAUG,gBAAyCmqD,EAAgB/gG,KAAK,mBAAqBhI,KAAK0hD,UAAUlD,QAAQI,gBAC1L5+C,KAAK0hD,UAAUlD,QAAQK,cAAgB7+C,KAAKmpG,gBAAgB3qD,QAAQC,UAAUI,cAA2CkqD,EAAgB/gG,KAAK,iBAAmBhI,KAAK0hD,UAAUlD,QAAQK,cACxL7+C,KAAK0hD,UAAUlD,QAAQM,gBAAkB9+C,KAAKmpG,gBAAgB3qD,QAAQC,UAAUK,gBAAyCiqD,EAAgB/gG,KAAK,mBAAqBhI,KAAK0hD,UAAUlD,QAAQM,gBAC1L9+C,KAAK0hD,UAAUlD,QAAQO,SAAW/+C,KAAKmpG,gBAAgB3qD,QAAQC,UAAUM,SAAgDgqD,EAAgB/gG,KAAK,YAAchI,KAAK0hD,UAAUlD,QAAQO,SACzJ,GAA1BgqD,EAAgBpjG,OAAa,CAC/B8I,EAAU,kBACVA,GAAW,wBACX,KAAK,GAAIjJ,GAAI,EAAGA,EAAIujG,EAAgBpjG,OAAQH,IAC1CiJ,GAAWs6F,EAAgBvjG,GACvBA,EAAIujG,EAAgBpjG,OAAS,IAC/B8I,GAAW,KAGfA,IAAW,KAETzO,KAAK0hD,UAAUZ,aAAapyC,SAAW1O,KAAKmpG,gBAAgBroD,aAAapyC,UAC7C,GAA1Bq6F,EAAgBpjG,OAAc8I,EAAU,kBACtCA,GAAW,KACjBA,GAAW,iBAAmBzO,KAAK0hD,UAAUZ,aAAapyC,SAE7C,iDAAXD,IACFA,GAAW,UAGV,IAA4B,GAAxBw6F,EAAaC,QAAiB,CAQrC,GAPAz6F,EAAU,kBACVA,GAAW,wCACPzO,KAAK0hD,UAAUlD,QAAQQ,UAAUC,cAAgBj/C,KAAKmpG,gBAAgB3qD,QAAQQ,UAAUC,cAAgB8pD,EAAgB/gG,KAAK,iBAAmBhI,KAAK0hD,UAAUlD,QAAQQ,UAAUC,cACjLj/C,KAAK0hD,UAAUlD,QAAQI,gBAAkB5+C,KAAKmpG,gBAAgB3qD,QAAQQ,UAAUJ,gBAAwBmqD,EAAgB/gG,KAAK,mBAAqBhI,KAAK0hD,UAAUlD,QAAQI,gBACzK5+C,KAAK0hD,UAAUlD,QAAQK,cAAgB7+C,KAAKmpG,gBAAgB3qD,QAAQQ,UAAUH,cAA0BkqD,EAAgB/gG,KAAK,iBAAmBhI,KAAK0hD,UAAUlD,QAAQK,cACvK7+C,KAAK0hD,UAAUlD,QAAQM,gBAAkB9+C,KAAKmpG,gBAAgB3qD,QAAQQ,UAAUF,gBAAwBiqD,EAAgB/gG,KAAK,mBAAqBhI,KAAK0hD,UAAUlD,QAAQM,gBACzK9+C,KAAK0hD,UAAUlD,QAAQO,SAAW/+C,KAAKmpG,gBAAgB3qD,QAAQQ,UAAUD,SAA+BgqD,EAAgB/gG,KAAK,YAAchI,KAAK0hD,UAAUlD,QAAQO,SACxI,GAA1BgqD,EAAgBpjG,OAAa,CAC/B8I,GAAW,gBACX,KAAK,GAAIjJ,GAAI,EAAGA,EAAIujG,EAAgBpjG,OAAQH,IAC1CiJ,GAAWs6F,EAAgBvjG,GACvBA,EAAIujG,EAAgBpjG,OAAS,IAC/B8I,GAAW,KAGfA,IAAW,KAEiB,GAA1Bs6F,EAAgBpjG,SAAc8I,GAAW,KACzCzO,KAAK0hD,UAAUZ,cAAgB9gD,KAAKmpG,gBAAgBroD,eACtDryC,GAAW,mBAAqBzO,KAAK0hD,UAAUZ,cAEjDryC,GAAW,SAER,CAOH,GANAA,EAAU,kBACNzO,KAAK0hD,UAAUlD,QAAQU,sBAAsBD,cAAgBj/C,KAAKmpG,gBAAgB3qD,QAAQU,sBAAsBD,cAAgB8pD,EAAgB/gG,KAAK,iBAAmBhI,KAAK0hD,UAAUlD,QAAQU,sBAAsBD,cACrNj/C,KAAK0hD,UAAUlD,QAAQI,gBAAkB5+C,KAAKmpG,gBAAgB3qD,QAAQU,sBAAsBN,gBAAwBmqD,EAAgB/gG,KAAK,mBAAqBhI,KAAK0hD,UAAUlD,QAAQI,gBACrL5+C,KAAK0hD,UAAUlD,QAAQK,cAAgB7+C,KAAKmpG,gBAAgB3qD,QAAQU,sBAAsBL,cAA0BkqD,EAAgB/gG,KAAK,iBAAmBhI,KAAK0hD,UAAUlD,QAAQK,cACnL7+C,KAAK0hD,UAAUlD,QAAQM,gBAAkB9+C,KAAKmpG,gBAAgB3qD,QAAQU,sBAAsBJ,gBAAwBiqD,EAAgB/gG,KAAK,mBAAqBhI,KAAK0hD,UAAUlD,QAAQM,gBACrL9+C,KAAK0hD,UAAUlD,QAAQO,SAAW/+C,KAAKmpG,gBAAgB3qD,QAAQU,sBAAsBH,SAA+BgqD,EAAgB/gG,KAAK,YAAchI,KAAK0hD,UAAUlD,QAAQO,SACpJ,GAA1BgqD,EAAgBpjG,OAAa,CAC/B8I,GAAW,oCACX,KAAK,GAAIjJ,GAAI,EAAGA,EAAIujG,EAAgBpjG,OAAQH,IAC1CiJ,GAAWs6F,EAAgBvjG,GACvBA,EAAIujG,EAAgBpjG,OAAS,IAC/B8I,GAAW,KAGfA,IAAW,MAOb,GALAA,GAAW,wBACXs6F,KACI/oG,KAAK0hD,UAAUjB,mBAAmBvlB,WAAal7B,KAAKmpG,gBAAgB1oD,mBAAmBvlB,WAAkC6tE,EAAgB/gG,KAAK,cAAgBhI,KAAK0hD,UAAUjB,mBAAmBvlB,WAChMh2B,KAAK2lB,IAAI7qB,KAAK0hD,UAAUjB,mBAAmBC,kBAAoB1gD,KAAKmpG,gBAAgB1oD,mBAAmBC,iBAAkBqoD,EAAgB/gG,KAAK,oBAAsBhI,KAAK0hD,UAAUjB,mBAAmBC,iBACtM1gD,KAAK0hD,UAAUjB,mBAAmBE,aAAe3gD,KAAKmpG,gBAAgB1oD,mBAAmBE,aAAgCooD,EAAgB/gG,KAAK,gBAAkBhI,KAAK0hD,UAAUjB,mBAAmBE,aACxK,GAA1BooD,EAAgBpjG,OAAa,CAC/B,IAAK,GAAIH,GAAI,EAAGA,EAAIujG,EAAgBpjG,OAAQH,IAC1CiJ,GAAWs6F,EAAgBvjG,GACvBA,EAAIujG,EAAgBpjG,OAAS,IAC/B8I,GAAW,KAGfA,IAAW,QAGXA,IAAW,eAEbA,IAAW,KAIbzO,KAAKopG,WAAWnlF,UAAYxV,EAO9B,QAAS46F,KACP,GAAIl0F,IAAO,iBAAkB,gBAAiB,iBAC1Cm0F,EAAc/3F,SAASg4F,cAAc,6CAA6CliG,MAClFmiG,EAAU,SAAWF,EAAc,SACnCG,EAAQl4F,SAASo3F,eAAea,EACpCC,GAAMx8F,MAAM26B,QAAU,OACtB,KAAK,GAAIpiC,GAAI,EAAGA,EAAI2P,EAAIxP,OAAQH,IAC1B2P,EAAI3P,IAAMgkG,IACZC,EAAQl4F,SAASo3F,eAAexzF,EAAI3P,IACpCikG,EAAMx8F,MAAM26B,QAAU,OAG1B5nC,MAAKwoG,gBACc,KAAfc,GACFtpG,KAAK0hD,UAAUjB,mBAAmB/xC,SAAU,EAC5C1O,KAAK0hD,UAAUlD,QAAQU,sBAAsBxwC,SAAU,EACvD1O,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,SAAU,GAErB,KAAf46F,EAC0C,GAA7CtpG,KAAK0hD,UAAUjB,mBAAmB/xC,UACpC1O,KAAK0hD,UAAUjB,mBAAmB/xC,SAAU,EAC5C1O,KAAK0hD,UAAUlD,QAAQU,sBAAsBxwC,SAAU,EACvD1O,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,SAAU,EAC3C1O,KAAK0hD,UAAUZ,aAAapyC,SAAU,EACtC1O,KAAKglD,6BAIPhlD,KAAK0hD,UAAUjB,mBAAmB/xC,SAAU,EAC5C1O,KAAK0hD,UAAUlD,QAAQU,sBAAsBxwC,SAAU,EACvD1O,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,SAAU,GAE7C1O,KAAKmtE,0BACL,IAAIu7B,GAAqBn3F,SAASo3F,eAAe,qBACCD,GAAmBz7F,MAAMd,WAAhC,GAAvCnM,KAAK0hD,UAAUZ,aAAapyC,QAAwD,UACR,UAChF1O,KAAK8kD,QAAS,EACd9kD,KAAK4P,QAWP,QAASi5F,GAAkBxoG,EAAGgN,EAAIq8F,GAChC,GAAIC,GAAUtpG,EAAK,SACfupG,EAAar4F,SAASo3F,eAAetoG,GAAIgH,KAEzCpB,OAAMC,QAAQmH,IAChBkE,SAASo3F,eAAegB,GAAStiG,MAAQgG,EAAIzC,SAASg/F,IACtD5pG,KAAK6pG,yBAAyBH,EAAsBr8F,EAAIzC,SAASg/F,OAGjEr4F,SAASo3F,eAAegB,GAAStiG,MAAQuD,SAASyC,GAAOgY,WAAWukF,GACpE5pG,KAAK6pG,yBAAyBH,EAAuB9+F,SAASyC,GAAOgY,WAAWukF,MAGrD,gCAAzBF,GACuB,sCAAzBA,GACyB,kCAAzBA,IACA1pG,KAAKglD,2BAEPhlD,KAAK8kD,QAAS,EACd9kD,KAAK4P,QA7sBP,GAAIjP,GAAOT,EAAoB,GAC3B4pG,EAAiB5pG,EAAoB,IACrC6pG,EAA4B7pG,EAAoB,IAChD8pG,EAAiB9pG,EAAoB,GAOzCN,GAAQqqG,iBAAmB,WACzBjqG,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,SAAW1O,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,QAC7E1O,KAAKmtE,2BACLntE,KAAK8kD,QAAS,EACd9kD,KAAK4P,SASPhQ,EAAQutE,yBAA2B,WAEe,GAA5CntE,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,SACnC1O,KAAKktE,YAAY48B,GACjB9pG,KAAKktE,YAAY68B,GAEjB/pG,KAAK0hD,UAAUlD,QAAQI,eAAiB5+C,KAAK0hD,UAAUlD,QAAQC,UAAUG,eACzE5+C,KAAK0hD,UAAUlD,QAAQK,aAAe7+C,KAAK0hD,UAAUlD,QAAQC,UAAUI,aACvE7+C,KAAK0hD,UAAUlD,QAAQM,eAAiB9+C,KAAK0hD,UAAUlD,QAAQC,UAAUK,eACzE9+C,KAAK0hD,UAAUlD,QAAQO,QAAU/+C,KAAK0hD,UAAUlD,QAAQC,UAAUM,QAElE/+C,KAAK+sE,WAAWi9B,IAE+C,GAAxDhqG,KAAK0hD,UAAUlD,QAAQU,sBAAsBxwC,SACpD1O,KAAKktE,YAAY88B,GACjBhqG,KAAKktE,YAAY48B,GAEjB9pG,KAAK0hD,UAAUlD,QAAQI,eAAiB5+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBN,eACrF5+C,KAAK0hD,UAAUlD,QAAQK,aAAe7+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBL,aACnF7+C,KAAK0hD,UAAUlD,QAAQM,eAAiB9+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBJ,eACrF9+C,KAAK0hD,UAAUlD,QAAQO,QAAU/+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBH,QAE9E/+C,KAAK+sE,WAAWg9B,KAGhB/pG,KAAKktE,YAAY88B,GACjBhqG,KAAKktE,YAAY68B,GACjB/pG,KAAKkqG,cAAgB1jG,OAErBxG,KAAK0hD,UAAUlD,QAAQI,eAAiB5+C,KAAK0hD,UAAUlD,QAAQQ,UAAUJ,eACzE5+C,KAAK0hD,UAAUlD,QAAQK,aAAe7+C,KAAK0hD,UAAUlD,QAAQQ,UAAUH,aACvE7+C,KAAK0hD,UAAUlD,QAAQM,eAAiB9+C,KAAK0hD,UAAUlD,QAAQQ,UAAUF,eACzE9+C,KAAK0hD,UAAUlD,QAAQO,QAAU/+C,KAAK0hD,UAAUlD,QAAQQ,UAAUD,QAElE/+C,KAAK+sE,WAAW+8B,KAUpBlqG,EAAQuqG,4BAA8B,WAEL,GAA3BnqG,KAAK8jD,YAAYn+C,OACnB3F,KAAKg9C,MAAMh9C,KAAK8jD,YAAY,IAAIsa,UAAU,EAAG,IAIzCp+D,KAAK8jD,YAAYn+C,OAAS3F,KAAK0hD,UAAUvC,WAAWE,kBAAyD,GAArCr/C,KAAK0hD,UAAUvC,WAAWzwC,SACpG1O,KAAKm8F,aAAan8F,KAAK0hD,UAAUvC,WAAWG,eAAe,GAI7Dt/C,KAAKoqG,qBAUTxqG,EAAQwqG,iBAAmB,WAKzBpqG,KAAKqqG,gCACLrqG,KAAKsqG,uBAEDtqG,KAAK0hD,UAAUlD,QAAQM,eAAiB,IACC,GAAvC9+C,KAAK0hD,UAAUZ,aAAapyC,SAA0D,GAAvC1O,KAAK0hD,UAAUZ,aAAaC,QAC7E/gD,KAAKuqG,oCAGuD,GAAxDvqG,KAAK0hD,UAAUlD,QAAQU,sBAAsBxwC,QAC/C1O,KAAKwqG,qCAGLxqG,KAAKyqG,2BAeb7qG,EAAQyuD,wBAA0B,WAChC,GAA2C,GAAvCruD,KAAK0hD,UAAUZ,aAAapyC,SAA0D,GAAvC1O,KAAK0hD,UAAUZ,aAAaC,QAAiB,CAC9F/gD,KAAK4jD,oBACL5jD,KAAK6jD,yBAEL,KAAK,GAAIsC,KAAUnmD,MAAKg9C,MAClBh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BnmD,KAAK4jD,iBAAiBuC,GAAUnmD,KAAKg9C,MAAMmJ,GAG/C,IAAIogD,GAAevmG,KAAKkvD,QAAiB,QAAS,KAClD,KAAK,GAAIw7C,KAAiBnE,GACpBA,EAAazgG,eAAe4kG,KAC1B1qG,KAAK89C,MAAMh4C,eAAeygG,EAAamE,GAAez4C,cACxDjyD,KAAK4jD,iBAAiB8mD,GAAiBnE,EAAamE,GAGpDnE,EAAamE,GAAetsC,UAAU,EAAG,GAK/C,KAAK,GAAIlX,KAAOlnD,MAAK4jD,iBACf5jD,KAAK4jD,iBAAiB99C,eAAeohD,IACvClnD,KAAK6jD,uBAAuB77C,KAAKk/C,OAKrClnD,MAAK4jD,iBAAmB5jD,KAAKg9C,MAC7Bh9C,KAAK6jD,uBAAyB7jD,KAAK8jD,aAUvClkD,EAAQyqG,8BAAgC,WACtC,GAAIzrF,GAAIC,EAAI8G,EAAUmgC,EAAMtgD,EACxBw3C,EAAQh9C,KAAK4jD,iBACb+mD,EAAU3qG,KAAK0hD,UAAUlD,QAAQI,eACjCgsD,EAAe,CAEnB,KAAKplG,EAAI,EAAGA,EAAIxF,KAAK6jD,uBAAuBl+C,OAAQH,IAClDsgD,EAAO9I,EAAMh9C,KAAK6jD,uBAAuBr+C,IACzCsgD,EAAK/G,QAAU/+C,KAAK0hD,UAAUlD,QAAQO,QAEhB,WAAlB/+C,KAAK88F,WAAqC,GAAX6N,GACjC/rF,GAAMknC,EAAK/zC,EACX8M,GAAMinC,EAAK9zC,EACX2T,EAAWzgB,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAEpC+rF,EAA4B,GAAZjlF,EAAiB,EAAKglF,EAAUhlF,EAChDmgC,EAAKyV,GAAK38C,EAAKgsF,EACf9kD,EAAK0V,GAAK38C,EAAK+rF,IAGf9kD,EAAKyV,GAAK,EACVzV,EAAK0V,GAAK,IAahB57D,EAAQ6qG,uBAAyB,WAC/B,GAAII,GAAYn9C,EAAMV,EAClBpuC,EAAIC,EAAI08C,EAAIC,EAAIsvC,EAAanlF,EAC7Bm4B,EAAQ99C,KAAK89C,KAGjB,KAAKkP,IAAUlP,GACTA,EAAMh4C,eAAeknD,KACvBU,EAAO5P,EAAMkP,GACTU,EAAKC,WAEH3tD,KAAKg9C,MAAMl3C,eAAe4nD,EAAKkG,OAAS5zD,KAAKg9C,MAAMl3C,eAAe4nD,EAAKiG,UACzEk3C,EAAan9C,EAAKlP,QAAQK,aAE1BgsD,IAAen9C,EAAKrkC,GAAGgzC,YAAc3O,EAAKtkC,KAAKizC,YAAc,GAAKr8D,KAAK0hD,UAAUvC,WAAWY,WAE5FnhC,EAAM8uC,EAAKtkC,KAAKrX,EAAI27C,EAAKrkC,GAAGtX,EAC5B8M,EAAM6uC,EAAKtkC,KAAKpX,EAAI07C,EAAKrkC,GAAGrX,EAC5B2T,EAAWzgB,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAEpB,GAAZ8G,IACFA,EAAW,KAIbmlF,EAAc9qG,KAAK0hD,UAAUlD,QAAQM,gBAAkB+rD,EAAallF,GAAYA,EAEhF41C,EAAK38C,EAAKksF,EACVtvC,EAAK38C,EAAKisF,EAEVp9C,EAAKtkC,KAAKmyC,IAAMA,EAChB7N,EAAKtkC,KAAKoyC,IAAMA,EAChB9N,EAAKrkC,GAAGkyC,IAAMA,EACd7N,EAAKrkC,GAAGmyC,IAAMA,KAexB57D,EAAQ2qG,kCAAoC,WAC1C,GAAIM,GAAYn9C,EAAMV,EAAQ+9C,EAC1BjtD,EAAQ99C,KAAK89C,KAGjB,KAAKkP,IAAUlP,GACb,GAAIA,EAAMh4C,eAAeknD,KACvBU,EAAO5P,EAAMkP,GACTU,EAAKC,WAEH3tD,KAAKg9C,MAAMl3C,eAAe4nD,EAAKkG,OAAS5zD,KAAKg9C,MAAMl3C,eAAe4nD,EAAKiG,SACzD,MAAZjG,EAAKuB,KAAa,CACpB,GAAI+7C,GAAQt9C,EAAKrkC,GACb4hF,EAAQv9C,EAAKuB,IACbi8C,EAAQx9C,EAAKtkC,IAEjByhF,GAAan9C,EAAKlP,QAAQK,aAE1BksD,EAAsBC,EAAM3uC,YAAc6uC,EAAM7uC,YAAc,EAG9DwuC,GAAcE,EAAsB/qG,KAAK0hD,UAAUvC,WAAWY,WAC9D//C,KAAKmrG,sBAAsBH,EAAOC,EAAO,GAAMJ,GAC/C7qG,KAAKmrG,sBAAsBF,EAAOC,EAAO,GAAML,KAiB3DjrG,EAAQurG,sBAAwB,SAAUH,EAAOC,EAAOJ,GACtD,GAAIjsF,GAAIC,EAAI08C,EAAIC,EAAIsvC,EAAanlF,CAEjC/G,GAAMosF,EAAMj5F,EAAIk5F,EAAMl5F,EACtB8M,EAAMmsF,EAAMh5F,EAAIi5F,EAAMj5F,EACtB2T,EAAWzgB,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAEpB,GAAZ8G,IACFA,EAAW,KAIbmlF,EAAc9qG,KAAK0hD,UAAUlD,QAAQM,gBAAkB+rD,EAAallF,GAAYA,EAEhF41C,EAAK38C,EAAKksF,EACVtvC,EAAK38C,EAAKisF,EAEVE,EAAMzvC,IAAMA,EACZyvC,EAAMxvC,IAAMA,EACZyvC,EAAM1vC,IAAMA,EACZ0vC,EAAMzvC,IAAMA,GAId57D,EAAQwqD,6BAA+B,WACrC,GAAkC5jD,SAA9BxG,KAAKorG,qBAAoC,CAC3C,KAAOprG,KAAKorG,qBAAqB1nF,iBAC/B1jB,KAAKorG,qBAAqBj6F,YAAYnR,KAAKorG,qBAAqBznF,WAGlE3jB,MAAKorG,qBAAqBvhG,WAAWsH,YAAYnR,KAAKorG,sBACtDprG,KAAKorG,qBAAuB5kG,SAQhC5G,EAAQwtE,0BAA4B,WAClC,GAAkC5mE,SAA9BxG,KAAKorG,qBAAoC,CAC3CprG,KAAKmpG,mBACLxoG,EAAK8F,WAAWzG,KAAKmpG,gBAAgBnpG,KAAK0hD,UAE1C,IAAI2pD,IAAgC,KAAM,KAAM,KAAM,KACtDrrG,MAAKorG,qBAAuB75F,SAASM,cAAc,OACnD7R,KAAKorG,qBAAqBvjG,UAAY,uBACtC7H,KAAKorG,qBAAqBnnF,UAAY,onBAW2E,GAAKjkB,KAAK0hD,UAAUlD,QAAQC,UAAUE,sBAAyB,wGAA2G,GAAK3+C,KAAK0hD,UAAUlD,QAAQC,UAAUE,sBAAyB,4JAGpP3+C,KAAK0hD,UAAUlD,QAAQC,UAAUG,eAAiB,wFAA0F5+C,KAAK0hD,UAAUlD,QAAQC,UAAUG,eAAiB,2JAG/L5+C,KAAK0hD,UAAUlD,QAAQC,UAAUI,aAAe,sFAAwF7+C,KAAK0hD,UAAUlD,QAAQC,UAAUI,aAAe,6JAGtL7+C,KAAK0hD,UAAUlD,QAAQC,UAAUK,eAAiB,0FAA4F9+C,KAAK0hD,UAAUlD,QAAQC,UAAUK,eAAiB,sJAGvM9+C,KAAK0hD,UAAUlD,QAAQC,UAAUM,QAAU,4FAA8F/+C,KAAK0hD,UAAUlD,QAAQC,UAAUM,QAAU,sPAM/K/+C,KAAK0hD,UAAUlD,QAAQQ,UAAUC,aAAe,kGAAoGj/C,KAAK0hD,UAAUlD,QAAQQ,UAAUC,aAAe,2JAGnMj/C,KAAK0hD,UAAUlD,QAAQQ,UAAUJ,eAAiB,uFAAyF5+C,KAAK0hD,UAAUlD,QAAQQ,UAAUJ,eAAiB,0JAG9L5+C,KAAK0hD,UAAUlD,QAAQQ,UAAUH,aAAe,qFAAuF7+C,KAAK0hD,UAAUlD,QAAQQ,UAAUH,aAAe,4JAGrL7+C,KAAK0hD,UAAUlD,QAAQQ,UAAUF,eAAiB,yFAA2F9+C,KAAK0hD,UAAUlD,QAAQQ,UAAUF,eAAiB,qJAGtM9+C,KAAK0hD,UAAUlD,QAAQQ,UAAUD,QAAU,2FAA6F/+C,KAAK0hD,UAAUlD,QAAQQ,UAAUD,QAAU,oQAM9K/+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBD,aAAe,kGAAoGj/C,KAAK0hD,UAAUlD,QAAQU,sBAAsBD,aAAe,2JAG3Nj/C,KAAK0hD,UAAUlD,QAAQU,sBAAsBN,eAAiB,uFAAyF5+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBN,eAAiB,0JAGtN5+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBL,aAAe,qFAAuF7+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBL,aAAe,4JAG7M7+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBJ,eAAiB,yFAA2F9+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBJ,eAAiB,qJAG9N9+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBH,QAAU,2FAA6F/+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBH,QAAU,uJAG3MssD,EAA6B1kG,QAAQ3G,KAAK0hD,UAAUjB,mBAAmBvlB,WAAa,0FAA4Fl7B,KAAK0hD,UAAUjB,mBAAmBvlB,UAAY,oKAGtNl7B,KAAK0hD,UAAUjB,mBAAmBC,gBAAkB,yFAA2F1gD,KAAK0hD,UAAUjB,mBAAmBC,gBAAkB,6JAGvM1gD,KAAK0hD,UAAUjB,mBAAmBE,YAAc,wFAA0F3gD,KAAK0hD,UAAUjB,mBAAmBE,YAAc,odAU9R3gD,KAAKyZ,iBAAiB6xF,cAAc15F,aAAa5R,KAAKorG,qBAAsBprG,KAAKyZ,kBACjFzZ,KAAKopG,WAAa73F,SAASM,cAAc,OACzC7R,KAAKopG,WAAWn8F,MAAMswC,SAAW,OACjCv9C,KAAKopG,WAAWn8F,MAAMszD,WAAa,UACnCvgE,KAAKyZ,iBAAiB6xF,cAAc15F,aAAa5R,KAAKopG,WAAYppG,KAAKyZ,iBAEvE;GAAI8xF,EACJA,GAAeh6F,SAASo3F,eAAe,eACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,cAAe,GAAI,2CACvEurG,EAAeh6F,SAASo3F,eAAe,eACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,cAAe,EAAG,0BACtEurG,EAAeh6F,SAASo3F,eAAe,eACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,cAAe,EAAG,0BACtEurG,EAAeh6F,SAASo3F,eAAe,eACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,cAAe,EAAG,wBACtEurG,EAAeh6F,SAASo3F,eAAe,iBACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,gBAAiB,EAAG,mBAExEurG,EAAeh6F,SAASo3F,eAAe,cACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,aAAc,EAAG,kCACrEurG,EAAeh6F,SAASo3F,eAAe,cACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,aAAc,EAAG,0BACrEurG,EAAeh6F,SAASo3F,eAAe,cACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,aAAc,EAAG,0BACrEurG,EAAeh6F,SAASo3F,eAAe,cACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,aAAc,EAAG,wBACrEurG,EAAeh6F,SAASo3F,eAAe,gBACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,eAAgB,EAAG,mBAEvEurG,EAAeh6F,SAASo3F,eAAe,cACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,aAAc,EAAG,8CACrEurG,EAAeh6F,SAASo3F,eAAe,cACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,aAAc,EAAG,0BACrEurG,EAAeh6F,SAASo3F,eAAe,cACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,aAAc,EAAG,0BACrEurG,EAAeh6F,SAASo3F,eAAe,cACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,aAAc,EAAG,wBACrEurG,EAAeh6F,SAASo3F,eAAe,gBACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,eAAgB,EAAG,mBACvEurG,EAAeh6F,SAASo3F,eAAe,qBACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,oBAAqBqrG,EAA8B,gCACvGE,EAAeh6F,SAASo3F,eAAe,kBACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,iBAAkB,EAAG,sCACzEurG,EAAeh6F,SAASo3F,eAAe,iBACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,gBAAiB,EAAG,iCAExE,IAAIgpG,GAAez3F,SAASo3F,eAAe,wBACvCM,EAAe13F,SAASo3F,eAAe,wBACvC6C,EAAej6F,SAASo3F,eAAe,uBAC3CM,GAAaC,SAAU,EACnBlpG,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,UACnCs6F,EAAaE,SAAU,GAErBlpG,KAAK0hD,UAAUjB,mBAAmB/xC,UACpC88F,EAAatC,SAAU,EAGzB,IAAIR,GAAqBn3F,SAASo3F,eAAe,sBAC7C8C,EAAwBl6F,SAASo3F,eAAe,yBAChD+C,EAAwBn6F,SAASo3F,eAAe,wBAEpDD,GAAmB12E,QAAUy2E,EAAwB3zE,KAAK90B,MAC1DyrG,EAAsBz5E,QAAU42E,EAAqB9zE,KAAK90B,MAC1D0rG,EAAsB15E,QAAU82E,EAAqBh0E,KAAK90B,MAExD0oG,EAAmBz7F,MAAMd,WADQ,GAA/BnM,KAAK0hD,UAAUZ,cAA8D,GAAtC9gD,KAAK0hD,UAAUiqD,oBAClB,UAGA,UAIxCtC,EAAqBtxF,MAAM/X,MAE3BgpG,EAAangF,SAAWwgF,EAAqBv0E,KAAK90B,MAClDipG,EAAapgF,SAAWwgF,EAAqBv0E,KAAK90B,MAClDwrG,EAAa3iF,SAAWwgF,EAAqBv0E,KAAK90B,QAWtDJ,EAAQiqG,yBAA2B,SAAUH,EAAuBriG,GAClE,GAAIukG,GAAYlC,EAAsB3hG,MAAM,IACpB,IAApB6jG,EAAUjmG,OACZ3F,KAAK0hD,UAAUkqD,EAAU,IAAMvkG,EAEJ,GAApBukG,EAAUjmG,OACjB3F,KAAK0hD,UAAUkqD,EAAU,IAAIA,EAAU,IAAMvkG,EAElB,GAApBukG,EAAUjmG,SACjB3F,KAAK0hD,UAAUkqD,EAAU,IAAIA,EAAU,IAAIA,EAAU,IAAMvkG,KA6N3D,SAASxH,GAEb,QAASgsG,GAAeC,GACvB,KAAM,IAAIloG,OAAM,uBAAyBkoG,EAAM,MAEhDD,EAAez+F,KAAO,WAAa,UACnCy+F,EAAeE,QAAUF,EACzBhsG,EAAOD,QAAUisG,EACjBA,EAAexrG,GAAK,IAKhB,SAASR,EAAQD,GAQrBA,EAAQ0qG,qBAAuB,WAC7B,GAAI1rF,GAAIC,EAAW8G,EAAU41C,EAAIC,EAAIuvC,EACnCiB,EAAgBhB,EAAOC,EAAOzlG,EAAGomB,EAE/BoxB,EAAQh9C,KAAK4jD,iBACbE,EAAc9jD,KAAK6jD,uBAGnBooD,EAAS,GAAK,EACd7lG,EAAI,EAAI,EAGR64C,EAAej/C,KAAK0hD,UAAUlD,QAAQQ,UAAUC,aAChDitD,EAAkBjtD,CAItB,KAAKz5C,EAAI,EAAGA,EAAIs+C,EAAYn+C,OAAS,EAAGH,IAEtC,IADAwlG,EAAQhuD,EAAM8G,EAAYt+C,IACrBomB,EAAIpmB,EAAI,EAAGomB,EAAIk4B,EAAYn+C,OAAQimB,IAAK,CAC3Cq/E,EAAQjuD,EAAM8G,EAAYl4B,IAC1Bm/E,EAAsBC,EAAM3uC,YAAc4uC,EAAM5uC,YAAc,EAE9Dz9C,EAAKqsF,EAAMl5F,EAAIi5F,EAAMj5F,EACrB8M,EAAKosF,EAAMj5F,EAAIg5F,EAAMh5F,EACrB2T,EAAWzgB,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAGpB,GAAZ8G,IACFA,EAAW,GAAIzgB,KAAKE,SACpBwZ,EAAK+G,GAGPumF,EAA0C,GAAvBnB,EAA4B9rD,EAAgBA,GAAgB,EAAI8rD,EAAsB/qG,KAAK0hD,UAAUvC,WAAWW,sBACnI,IAAIv6C,GAAI0mG,EAASC,CACF,GAAIA,EAAfvmF,IAEAqmF,EADa,GAAME,EAAjBvmF,EACe,EAGApgB,EAAIogB,EAAWvf,EAIlC4lG,GAA0C,GAAvBjB,EAA4B,EAAI,EAAIA,EAAsB/qG,KAAK0hD,UAAUvC,WAAWU,mBACvGmsD,GAAkC9mG,KAAKwH,IAAIiZ,EAAS,IAAKumF,GAEzD3wC,EAAK38C,EAAKotF,EACVxwC,EAAK38C,EAAKmtF,EACVhB,EAAMzvC,IAAMA,EACZyvC,EAAMxvC,IAAMA,EACZyvC,EAAM1vC,IAAMA,EACZ0vC,EAAMzvC,IAAMA,MAUhB,SAAS37D,EAAQD,GAQrBA,EAAQ0qG,qBAAuB,WAC7B,GAAI1rF,GAAIC,EAAI8G,EAAU41C,EAAIC,EACxBwwC,EAAgBhB,EAAOC,EAAOzlG,EAAGomB,EAE/BoxB,EAAQh9C,KAAK4jD,iBACbE,EAAc9jD,KAAK6jD,uBAGnB5E,EAAej/C,KAAK0hD,UAAUlD,QAAQU,sBAAsBD,YAIhE,KAAKz5C,EAAI,EAAGA,EAAIs+C,EAAYn+C,OAAS,EAAGH,IAEtC,IADAwlG,EAAQhuD,EAAM8G,EAAYt+C,IACrBomB,EAAIpmB,EAAI,EAAGomB,EAAIk4B,EAAYn+C,OAAQimB,IAItC,GAHAq/E,EAAQjuD,EAAM8G,EAAYl4B,IAGtBo/E,EAAMptD,OAASqtD,EAAMrtD,MAAO,CAE9Bh/B,EAAKqsF,EAAMl5F,EAAIi5F,EAAMj5F,EACrB8M,EAAKosF,EAAMj5F,EAAIg5F,EAAMh5F,EACrB2T,EAAWzgB,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,EAGpC,IAAIstF,GAAY,GAEdH,GADa/sD,EAAXt5B,GACgBzgB,KAAK4uB,IAAIq4E,EAAUxmF,EAAS,GAAKzgB,KAAK4uB,IAAIq4E,EAAUltD,EAAa,GAGlE,EAGD,GAAZt5B,EACFA,EAAW,IAGXqmF,GAAkCrmF,EAEpC41C,EAAK38C,EAAKotF,EACVxwC,EAAK38C,EAAKmtF,EAEVhB,EAAMzvC,IAAMA,EACZyvC,EAAMxvC,IAAMA,EACZyvC,EAAM1vC,IAAMA,EACZ0vC,EAAMzvC,IAAMA,IAYtB57D,EAAQ4qG,mCAAqC,WAS3C,IAAK,GARDK,GAAYn9C,EAAMV,EAClBpuC,EAAIC,EAAI08C,EAAIC,EAAIsvC,EAAanlF,EAC7Bm4B,EAAQ99C,KAAK89C,MAEbd,EAAQh9C,KAAK4jD,iBACbE,EAAc9jD,KAAK6jD,uBAGdr+C,EAAI,EAAGA,EAAIs+C,EAAYn+C,OAAQH,IAAK,CAC3C,GAAIwlG,GAAQhuD,EAAM8G,EAAYt+C,GAC9BwlG,GAAMoB,SAAW,EACjBpB,EAAMqB,SAAW,EAKnB,IAAKr/C,IAAUlP,GACb,GAAIA,EAAMh4C,eAAeknD,KACvBU,EAAO5P,EAAMkP,GACTU,EAAKC,WAEH3tD,KAAKg9C,MAAMl3C,eAAe4nD,EAAKkG,OAAS5zD,KAAKg9C,MAAMl3C,eAAe4nD,EAAKiG,SAqBzE,GApBAk3C,EAAan9C,EAAKlP,QAAQK,aAE1BgsD,IAAen9C,EAAKrkC,GAAGgzC,YAAc3O,EAAKtkC,KAAKizC,YAAc,GAAKr8D,KAAK0hD,UAAUvC,WAAWY,WAE5FnhC,EAAM8uC,EAAKtkC,KAAKrX,EAAI27C,EAAKrkC,GAAGtX,EAC5B8M,EAAM6uC,EAAKtkC,KAAKpX,EAAI07C,EAAKrkC,GAAGrX,EAC5B2T,EAAWzgB,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAEpB,GAAZ8G,IACFA,EAAW,KAIbmlF,EAAc9qG,KAAK0hD,UAAUlD,QAAQM,gBAAkB+rD,EAAallF,GAAYA,EAEhF41C,EAAK38C,EAAKksF,EACVtvC,EAAK38C,EAAKisF,EAINp9C,EAAKrkC,GAAGu0B,OAAS8P,EAAKtkC,KAAKw0B,MAC7B8P,EAAKrkC,GAAG+iF,UAAY7wC,EACpB7N,EAAKrkC,GAAGgjF,UAAY7wC,EACpB9N,EAAKtkC,KAAKgjF,UAAY7wC,EACtB7N,EAAKtkC,KAAKijF,UAAY7wC,MAEnB,CACH,GAAI7U,GAAS,EACb+G,GAAKrkC,GAAGkyC,IAAM5U,EAAO4U,EACrB7N,EAAKrkC,GAAGmyC,IAAM7U,EAAO6U,EACrB9N,EAAKtkC,KAAKmyC,IAAM5U,EAAO4U,EACvB7N,EAAKtkC,KAAKoyC,IAAM7U,EAAO6U,EAQjC,GACI4wC,GAAUC,EADVvB,EAAc,CAElB,KAAKtlG,EAAI,EAAGA,EAAIs+C,EAAYn+C,OAAQH,IAAK,CACvC,GAAIsgD,GAAO9I,EAAM8G,EAAYt+C,GAC7B4mG,GAAWlnG,KAAK4G,IAAIg/F,EAAY5lG,KAAKwH,KAAKo+F,EAAYhlD,EAAKsmD,WAC3DC,EAAWnnG,KAAK4G,IAAIg/F,EAAY5lG,KAAKwH,KAAKo+F,EAAYhlD,EAAKumD,WAE3DvmD,EAAKyV,IAAM6wC,EACXtmD,EAAK0V,IAAM6wC,EAIb,GAAIC,GAAU,EACVC,EAAU,CACd,KAAK/mG,EAAI,EAAGA,EAAIs+C,EAAYn+C,OAAQH,IAAK,CACvC,GAAIsgD,GAAO9I,EAAM8G,EAAYt+C,GAC7B8mG,IAAWxmD,EAAKyV,GAChBgxC,GAAWzmD,EAAK0V,GAElB,GAAIgxC,GAAeF,EAAUxoD,EAAYn+C,OACrC8mG,EAAeF,EAAUzoD,EAAYn+C,MAEzC,KAAKH,EAAI,EAAGA,EAAIs+C,EAAYn+C,OAAQH,IAAK,CACvC,GAAIsgD,GAAO9I,EAAM8G,EAAYt+C,GAC7BsgD,GAAKyV,IAAMixC,EACX1mD,EAAK0V,IAAMixC,KAOX,SAAS5sG,EAAQD,GAQrBA,EAAQ0qG,qBAAuB,WAC7B,GAA8D,GAA1DtqG,KAAK0hD,UAAUlD,QAAQC,UAAUE,sBAA4B,CAC/D,GAAImH,GACA9I,EAAQh9C,KAAK4jD,iBACbE,EAAc9jD,KAAK6jD,uBACnB6oD,EAAY5oD,EAAYn+C,MAE5B3F,MAAK2sG,mBAAmB3vD,EAAM8G,EAK9B,KAAK,GAHDomD,GAAgBlqG,KAAKkqG,cAGhB1kG,EAAI,EAAOknG,EAAJlnG,EAAeA,IAC7BsgD,EAAO9I,EAAM8G,EAAYt+C,IACrBsgD,EAAKr3C,QAAQwuC,KAAO,IAEtBj9C,KAAK4sG,sBAAsB1C,EAAcxqG,KAAKmtG,SAASC,GAAGhnD,GAC1D9lD,KAAK4sG,sBAAsB1C,EAAcxqG,KAAKmtG,SAASE,GAAGjnD,GAC1D9lD,KAAK4sG,sBAAsB1C,EAAcxqG,KAAKmtG,SAASG,GAAGlnD,GAC1D9lD,KAAK4sG,sBAAsB1C,EAAcxqG,KAAKmtG,SAASI,GAAGnnD,MAelElmD,EAAQgtG,sBAAwB,SAASM,EAAapnD,GAEpD,GAAIonD,EAAaC,cAAgB,EAAG,CAClC,GAAIvuF,GAAGC,EAAG8G,CAUV,IAPA/G,EAAKsuF,EAAaE,aAAar7F,EAAI+zC,EAAK/zC,EACxC8M,EAAKquF,EAAaE,aAAap7F,EAAI8zC,EAAK9zC,EACxC2T,EAAWzgB,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAKhC8G,EAAWunF,EAAaG,SAAWrtG,KAAK0hD,UAAUlD,QAAQC,UAAUC,cAAe,CAErE,GAAZ/4B,IACFA,EAAW,GAAIzgB,KAAKE,SACpBwZ,EAAK+G,EAEP,IAAIilF,GAAe5qG,KAAK0hD,UAAUlD,QAAQC,UAAUE,sBAAwBuuD,EAAajwD,KAAO6I,EAAKr3C,QAAQwuC,MAAQt3B,EAAWA,EAAWA,GACvI41C,EAAK38C,EAAKgsF,EACVpvC,EAAK38C,EAAK+rF,CACd9kD,GAAKyV,IAAMA,EACXzV,EAAK0V,IAAMA,MAIX,IAAkC,GAA9B0xC,EAAaC,cACfntG,KAAK4sG,sBAAsBM,EAAaL,SAASC,GAAGhnD,GACpD9lD,KAAK4sG,sBAAsBM,EAAaL,SAASE,GAAGjnD,GACpD9lD,KAAK4sG,sBAAsBM,EAAaL,SAASG,GAAGlnD,GACpD9lD,KAAK4sG,sBAAsBM,EAAaL,SAASI,GAAGnnD,OAGpD,IAAIonD,EAAaL,SAASn6F,KAAKrS,IAAMylD,EAAKzlD,GAAI,CAE5B,GAAZslB,IACFA,EAAW,GAAIzgB,KAAKE,SACpBwZ,EAAK+G,EAEP,IAAIilF,GAAe5qG,KAAK0hD,UAAUlD,QAAQC,UAAUE,sBAAwBuuD,EAAajwD,KAAO6I,EAAKr3C,QAAQwuC,MAAQt3B,EAAWA,EAAWA,GACvI41C,EAAK38C,EAAKgsF,EACVpvC,EAAK38C,EAAK+rF,CACd9kD,GAAKyV,IAAMA,EACXzV,EAAK0V,IAAMA,KAcrB57D,EAAQ+sG,mBAAqB,SAAS3vD,EAAM8G,GAU1C,IAAK,GATDgC,GACA4mD,EAAY5oD,EAAYn+C,OAExBsgD,EAAO/hD,OAAOopG,UAChBvnD,EAAO7hD,OAAOopG,UACdpnD,GAAOhiD,OAAOopG,UACdtnD,GAAO9hD,OAAOopG,UAGP9nG,EAAI,EAAOknG,EAAJlnG,EAAeA,IAAK,CAClC,GAAIuM,GAAIirC,EAAM8G,EAAYt+C,IAAIuM,EAC1BC,EAAIgrC,EAAM8G,EAAYt+C,IAAIwM,CAC1BgrC,GAAM8G,EAAYt+C,IAAIiJ,QAAQwuC,KAAO,IAC/BgJ,EAAJl0C,IAAYk0C,EAAOl0C,GACnBA,EAAIm0C,IAAQA,EAAOn0C,GACfg0C,EAAJ/zC,IAAY+zC,EAAO/zC,GACnBA,EAAIg0C,IAAQA,EAAOh0C,IAI3B,GAAIu7F,GAAWroG,KAAK2lB,IAAIq7B,EAAOD,GAAQ/gD,KAAK2lB,IAAIm7B,EAAOD,EACnDwnD,GAAW,GAAIxnD,GAAQ,GAAMwnD,EAAUvnD,GAAQ,GAAMunD,IACtCtnD,GAAQ,GAAMsnD,EAAUrnD,GAAQ,GAAMqnD,EAGzD,IAAIC,GAAkB,KAClBC,EAAWvoG,KAAKwH,IAAI8gG,EAAgBtoG,KAAK2lB,IAAIq7B,EAAOD,IACpDynD,EAAe,GAAMD,EACrBxuC,EAAU,IAAOhZ,EAAOC,GAAOgZ,EAAU,IAAOnZ,EAAOC,GAGvDkkD,GACFxqG,MACE0tG,cAAer7F,EAAE,EAAGC,EAAE,GACtBirC,KAAK,EACLxnB,OACEwwB,KAAMgZ,EAAQyuC,EAAaxnD,KAAK+Y,EAAQyuC,EACxC3nD,KAAMmZ,EAAQwuC,EAAa1nD,KAAKkZ,EAAQwuC,GAE1Cr7F,KAAMo7F,EACNJ,SAAU,EAAII,EACdZ,UAAYn6F,KAAK,MACjBgpC,SAAU,EACVkC,MAAO,EACPuvD,cAAe,GAMnB,KAHAntG,KAAK2tG,aAAazD,EAAcxqG,MAG3B8F,EAAI,EAAOknG,EAAJlnG,EAAeA,IACzBsgD,EAAO9I,EAAM8G,EAAYt+C,IACrBsgD,EAAKr3C,QAAQwuC,KAAO,GACtBj9C,KAAK4tG,aAAa1D,EAAcxqG,KAAKomD,EAKzC9lD,MAAKkqG,cAAgBA,GAWvBtqG,EAAQiuG,kBAAoB,SAASX,EAAcpnD,GACjD,GAAIgoD,GAAYZ,EAAajwD,KAAO6I,EAAKr3C,QAAQwuC,KAC7C8wD,EAAe,EAAED,CAErBZ,GAAaE,aAAar7F,EAAIm7F,EAAaE,aAAar7F,EAAIm7F,EAAajwD,KAAO6I,EAAK/zC,EAAI+zC,EAAKr3C,QAAQwuC,KACtGiwD,EAAaE,aAAar7F,GAAKg8F,EAE/Bb,EAAaE,aAAap7F,EAAIk7F,EAAaE,aAAap7F,EAAIk7F,EAAajwD,KAAO6I,EAAK9zC,EAAI8zC,EAAKr3C,QAAQwuC,KACtGiwD,EAAaE,aAAap7F,GAAK+7F,EAE/Bb,EAAajwD,KAAO6wD,CACpB,IAAIE,GAAc9oG,KAAKwH,IAAIxH,KAAKwH,IAAIo5C,EAAKtzC,OAAOszC,EAAKr6B,QAAQq6B,EAAKvzC,MAClE26F,GAAaxxD,SAAYwxD,EAAaxxD,SAAWsyD,EAAeA,EAAcd,EAAaxxD,UAa7F97C,EAAQguG,aAAe,SAASV,EAAapnD,EAAKmoD,IAC1B,GAAlBA,GAA6CznG,SAAnBynG,IAE5BjuG,KAAK6tG,kBAAkBX,EAAapnD,GAGlConD,EAAaL,SAASC,GAAGr3E,MAAMywB,KAAOJ,EAAK/zC,EACzCm7F,EAAaL,SAASC,GAAGr3E,MAAMuwB,KAAOF,EAAK9zC,EAC7ChS,KAAKkuG,eAAehB,EAAapnD,EAAK,MAGtC9lD,KAAKkuG,eAAehB,EAAapnD,EAAK,MAIpConD,EAAaL,SAASC,GAAGr3E,MAAMuwB,KAAOF,EAAK9zC,EAC7ChS,KAAKkuG,eAAehB,EAAapnD,EAAK,MAGtC9lD,KAAKkuG,eAAehB,EAAapnD,EAAK,OAc5ClmD,EAAQsuG,eAAiB,SAAShB,EAAapnD,EAAKqoD,GAClD,OAAQjB,EAAaL,SAASsB,GAAQhB,eACpC,IAAK,GACHD,EAAaL,SAASsB,GAAQtB,SAASn6F,KAAOozC,EAC9ConD,EAAaL,SAASsB,GAAQhB,cAAgB,EAC9CntG,KAAK6tG,kBAAkBX,EAAaL,SAASsB,GAAQroD,EACrD,MACF,KAAK,GAGConD,EAAaL,SAASsB,GAAQtB,SAASn6F,KAAKX,GAAK+zC,EAAK/zC,GACtDm7F,EAAaL,SAASsB,GAAQtB,SAASn6F,KAAKV,GAAK8zC,EAAK9zC,GACxD8zC,EAAK/zC,GAAK7M,KAAKE,SACf0gD,EAAK9zC,GAAK9M,KAAKE,WAGfpF,KAAK2tG,aAAaT,EAAaL,SAASsB,IACxCnuG,KAAK4tG,aAAaV,EAAaL,SAASsB,GAAQroD,GAElD,MACF,KAAK,GACH9lD,KAAK4tG,aAAaV,EAAaL,SAASsB,GAAQroD,KAatDlmD,EAAQ+tG,aAAe,SAAST,GAE9B,GAAIkB,GAAgB,IACc,IAA9BlB,EAAaC,gBACfiB,EAAgBlB,EAAaL,SAASn6F,KACtCw6F,EAAajwD,KAAO,EAAGiwD,EAAaE,aAAar7F,EAAI,EAAGm7F,EAAaE,aAAap7F,EAAI,GAExFk7F,EAAaC,cAAgB,EAC7BD,EAAaL,SAASn6F,KAAO,KAC7B1S,KAAKquG,cAAcnB,EAAa,MAChCltG,KAAKquG,cAAcnB,EAAa,MAChCltG,KAAKquG,cAAcnB,EAAa,MAChCltG,KAAKquG,cAAcnB,EAAa,MAEX,MAAjBkB,GACFpuG,KAAK4tG,aAAaV,EAAakB,IAenCxuG,EAAQyuG,cAAgB,SAASnB,EAAciB,GAC7C,GAAIloD,GAAKC,EAAKH,EAAKC,EACfsoD,EAAY,GAAMpB,EAAa76F,IACnC,QAAQ87F,GACN,IAAK,KACHloD,EAAOinD,EAAaz3E,MAAMwwB,KAC1BC,EAAOgnD,EAAaz3E,MAAMwwB,KAAOqoD,EACjCvoD,EAAOmnD,EAAaz3E,MAAMswB,KAC1BC,EAAOknD,EAAaz3E,MAAMswB,KAAOuoD,CACjC,MACF,KAAK,KACHroD,EAAOinD,EAAaz3E,MAAMwwB,KAAOqoD,EACjCpoD,EAAOgnD,EAAaz3E,MAAMywB,KAC1BH,EAAOmnD,EAAaz3E,MAAMswB,KAC1BC,EAAOknD,EAAaz3E,MAAMswB,KAAOuoD,CACjC,MACF,KAAK,KACHroD,EAAOinD,EAAaz3E,MAAMwwB,KAC1BC,EAAOgnD,EAAaz3E,MAAMwwB,KAAOqoD,EACjCvoD,EAAOmnD,EAAaz3E,MAAMswB,KAAOuoD,EACjCtoD,EAAOknD,EAAaz3E,MAAMuwB,IAC1B,MACF,KAAK,KACHC,EAAOinD,EAAaz3E,MAAMwwB,KAAOqoD,EACjCpoD,EAAOgnD,EAAaz3E,MAAMywB,KAC1BH,EAAOmnD,EAAaz3E,MAAMswB,KAAOuoD,EACjCtoD,EAAOknD,EAAaz3E,MAAMuwB,KAK9BknD,EAAaL,SAASsB,IACpBf,cAAcr7F,EAAE,EAAEC,EAAE,GACpBirC,KAAK,EACLxnB,OAAOwwB,KAAKA,EAAKC,KAAKA,EAAKH,KAAKA,EAAKC,KAAKA,GAC1C3zC,KAAM,GAAM66F,EAAa76F,KACzBg7F,SAAU,EAAIH,EAAaG,SAC3BR,UAAWn6F,KAAK,MAChBgpC,SAAU,EACVkC,MAAOsvD,EAAatvD,MAAM,EAC1BuvD,cAAe,IAYnBvtG,EAAQ2uG,UAAY,SAASxnF,EAAI5b,GACJ3E,SAAvBxG,KAAKkqG,gBAEPnjF,EAAIO,UAAY,EAEhBtnB,KAAKwuG,YAAYxuG,KAAKkqG,cAAcxqG,KAAKqnB,EAAI5b,KAajDvL,EAAQ4uG,YAAc,SAASC,EAAO1nF,EAAI5b,GAC1B3E,SAAV2E,IACFA,EAAQ,WAGkB,GAAxBsjG,EAAOtB,gBACTntG,KAAKwuG,YAAYC,EAAO5B,SAASC,GAAG/lF,GACpC/mB,KAAKwuG,YAAYC,EAAO5B,SAASE,GAAGhmF,GACpC/mB,KAAKwuG,YAAYC,EAAO5B,SAASI,GAAGlmF,GACpC/mB,KAAKwuG,YAAYC,EAAO5B,SAASG,GAAGjmF,IAEtCA,EAAIY,YAAcxc,EAClB4b,EAAIa,YACJb,EAAIc,OAAO4mF,EAAOh5E,MAAMwwB,KAAKwoD,EAAOh5E,MAAMswB,MAC1Ch/B,EAAIe,OAAO2mF,EAAOh5E,MAAMywB,KAAKuoD,EAAOh5E,MAAMswB,MAC1Ch/B,EAAIlH,SAEJkH,EAAIa,YACJb,EAAIc,OAAO4mF,EAAOh5E,MAAMywB,KAAKuoD,EAAOh5E,MAAMswB,MAC1Ch/B,EAAIe,OAAO2mF,EAAOh5E,MAAMywB,KAAKuoD,EAAOh5E,MAAMuwB,MAC1Cj/B,EAAIlH,SAEJkH,EAAIa,YACJb,EAAIc,OAAO4mF,EAAOh5E,MAAMywB,KAAKuoD,EAAOh5E,MAAMuwB,MAC1Cj/B,EAAIe,OAAO2mF,EAAOh5E,MAAMwwB,KAAKwoD,EAAOh5E,MAAMuwB,MAC1Cj/B,EAAIlH,SAEJkH,EAAIa,YACJb,EAAIc,OAAO4mF,EAAOh5E,MAAMwwB,KAAKwoD,EAAOh5E,MAAMuwB,MAC1Cj/B,EAAIe,OAAO2mF,EAAOh5E,MAAMwwB,KAAKwoD,EAAOh5E,MAAMswB,MAC1Ch/B,EAAIlH,WAaF,SAAShgB,GAEbA,EAAOD,QAAU,SAASC,GAQzB,MAPIA,GAAO6uG,kBACV7uG,EAAOqwE,UAAY,aACnBrwE,EAAO8uG,SAEP9uG,EAAOgtG,YACPhtG,EAAO6uG,gBAAkB,GAEnB7uG,IAMJ,SAASA,EAAQD,IAEO,SAASgvG,GAA0B/uG,EAAOD,QAAUgvG,IAEnDruG,KAAKX"} \ No newline at end of file diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis.min.js b/core/src/main/resources/org/apache/spark/ui/static/vis.min.js index 4af2c818c27a6..2b3b1d60463f7 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/vis.min.js +++ b/core/src/main/resources/org/apache/spark/ui/static/vis.min.js @@ -36,4 +36,3 @@ return!e&&this._longDateFormat[t.toUpperCase()]&&(e=this._longDateFormat[t.toUpp },e.time,this);else if(t.eventType&ke)return pi;return mi},reset:function(){clearTimeout(this._timer)},emit:function(t){this.state===pi&&(t&&t.eventType&ke?this.manager.emit(this.options.event+"up",t):(this._input.timeStamp=ve(),this.manager.emit(this.options.event,this._input)))}}),u(ne,ee,{defaults:{event:"rotate",threshold:0,pointers:2},getTouchAction:function(){return[ni]},attrTest:function(t){return this._super.attrTest.call(this,t)&&(Math.abs(t.rotation)>this.options.threshold||this.state&di)}}),u(re,ee,{defaults:{event:"swipe",threshold:10,velocity:.65,direction:Re|Fe,pointers:1},getTouchAction:function(){return ie.prototype.getTouchAction.call(this)},attrTest:function(t){var e,i=this.options.direction;return i&(Re|Fe)?e=t.velocity:i&Re?e=t.velocityX:i&Fe&&(e=t.velocityY),this._super.attrTest.call(this,t)&&i&t.direction&&t.distance>this.options.threshold&&ge(e)>this.options.velocity&&t.eventType&ke},emit:function(t){var e=J(t.direction);e&&this.manager.emit(this.options.event+e,t),this.manager.emit(this.options.event,t)}}),u(ae,K,{defaults:{event:"tap",pointers:1,taps:1,interval:300,time:250,threshold:2,posThreshold:10},getTouchAction:function(){return[oi]},process:function(t){var e=this.options,i=t.pointers.length===e.pointers,s=t.distancet&&s>o;)o%3==0?(this.forceAggregateHubs(!0),this.normalizeClusterLevels()):this.increaseClusterLevel(),i=this.nodeIndices.length,o+=1;o>0&&1==e&&this.repositionNodes(),this._updateCalculationNodes()},e.openCluster=function(t){var e=this.moving;if(t.clusterSize>this.constants.clustering.sectorThreshold&&this._nodeInActiveArea(t)&&("default"!=this._sector()||1!=this.nodeIndices.length)){this._addSector(t);for(var i=0;this.nodeIndices.lengthi;)this.decreaseClusterLevel(),i+=1}else this._expandClusterNode(t,!1,!0),this._updateNodeIndexList(),this._updateDynamicEdges(),this._updateCalculationNodes(),this.updateLabels();this.moving!=e&&this.start()},e.updateClustersDefault=function(){1==this.constants.clustering.enabled&&this.updateClusters(0,!1,!1)},e.increaseClusterLevel=function(){this.updateClusters(-1,!1,!0)},e.decreaseClusterLevel=function(){this.updateClusters(1,!1,!0)},e.updateClusters=function(t,e,i,s){var o=this.moving,n=this.nodeIndices.length;this.previousScale>this.scale&&0==t&&this._collapseSector(),this.previousScale>this.scale||-1==t?this._formClusters(i):(this.previousScalethis.scale||-1==t)&&(this._aggregateHubs(i),this._updateNodeIndexList()),(this.previousScale>this.scale||-1==t)&&(this.handleChains(),this._updateNodeIndexList()),this.previousScale=this.scale,this._updateDynamicEdges(),this.updateLabels(),this.nodeIndices.lengththis.constants.clustering.chainThreshold&&this._reduceAmountOfChains(1-this.constants.clustering.chainThreshold/t)},e._aggregateHubs=function(t){this._getHubSize(),this._formClustersByHub(t,!1)},e.forceAggregateHubs=function(t){var e=this.moving,i=this.nodeIndices.length;this._aggregateHubs(!0),this._updateNodeIndexList(),this._updateDynamicEdges(),this.updateLabels(),this.nodeIndices.length!=i&&(this.clusterSession+=1),(0==t||void 0===t)&&this.moving!=e&&this.start()},e._openClustersBySize=function(){for(var t in this.nodes)if(this.nodes.hasOwnProperty(t)){var e=this.nodes[t];1==e.inView()&&(e.width*this.scale>this.constants.clustering.screenSizeThreshold*this.frame.canvas.clientWidth||e.height*this.scale>this.constants.clustering.screenSizeThreshold*this.frame.canvas.clientHeight)&&this.openCluster(e)}},e._openClusters=function(t,e){for(var i=0;i1&&(t.clusterSizei)){var r=n.from,a=n.to;n.to.options.mass>n.from.options.mass&&(r=n.to,a=n.from),1==a.dynamicEdgesLength?this._addToCluster(r,a,!1):1==r.dynamicEdgesLength&&this._addToCluster(a,r,!1)}}},e._forceClustersByZoom=function(){for(var t in this.nodes)if(this.nodes.hasOwnProperty(t)){var e=this.nodes[t];if(1==e.dynamicEdgesLength&&0!=e.dynamicEdges.length){var i=e.dynamicEdges[0],s=i.toId==e.id?this.nodes[i.fromId]:this.nodes[i.toId];e.id!=s.id&&(s.options.mass>e.options.mass?this._addToCluster(s,e,!0):this._addToCluster(e,s,!0))}}},e._clusterToSmallestNeighbour=function(t){for(var e=-1,i=null,s=0;so.clusterSessions.length&&(e=o.clusterSessions.length,i=o)}null!=o&&void 0!==this.nodes[o.id]&&this._addToCluster(o,t,!0)},e._formClustersByHub=function(t,e){for(var i in this.nodes)this.nodes.hasOwnProperty(i)&&this._formClusterFromHub(this.nodes[i],t,e)},e._formClusterFromHub=function(t,e,i,s){if(void 0===s&&(s=0),t.dynamicEdgesLength>=this.hubThreshold&&0==i||t.dynamicEdgesLength==this.hubThreshold&&1==i){for(var o,n,r,a=this.constants.clustering.clusterEdgeThreshold/this.scale,h=!1,d=[],l=t.dynamicEdges.length,c=0;l>c;c++)d.push(t.dynamicEdges[c].id);if(0==e)for(h=!1,c=0;l>c;c++){var p=this.edges[d[c]];if(void 0!==p&&p.connected&&p.toId!=p.fromId&&(o=p.to.x-p.from.x,n=p.to.y-p.from.y,r=Math.sqrt(o*o+n*n),a>r)){h=!0;break}}if(!e&&h||e)for(c=0;l>c;c++)if(p=this.edges[d[c]],void 0!==p){var u=this.nodes[p.fromId==t.id?p.toId:p.fromId];u.dynamicEdges.length<=this.hubThreshold+s&&u.id!=t.id&&this._addToCluster(t,u,e)}}},e._addToCluster=function(t,e,i){t.containedNodes[e.id]=e;for(var s=0;s1)for(var s=0;s1&&(e.label="[".concat(String(e.clusterSize),"]"))}for(t in this.nodes)this.nodes.hasOwnProperty(t)&&(e=this.nodes[t],1==e.clusterSize&&(e.label=void 0!==e.originalLabel?e.originalLabel:String(e.id)))},e.normalizeClusterLevels=function(){var t,e=0,i=1e9,s=0;for(t in this.nodes)this.nodes.hasOwnProperty(t)&&(s=this.nodes[t].clusterSessions.length,s>e&&(e=s),i>s&&(i=s));if(e-i>this.constants.clustering.clusterLevelDifference){var o=this.nodeIndices.length,n=e-this.constants.clustering.clusterLevelDifference;for(t in this.nodes)this.nodes.hasOwnProperty(t)&&this.nodes[t].clusterSessions.lengths&&(s=n.dynamicEdgesLength),t+=n.dynamicEdgesLength,e+=Math.pow(n.dynamicEdgesLength,2),i+=1}t/=i,e/=i;var r=e-Math.pow(t,2),a=Math.sqrt(r);this.hubThreshold=Math.floor(t+2*a),this.hubThreshold>s&&(this.hubThreshold=s)},e._reduceAmountOfChains=function(t){this.hubThreshold=2;var e=Math.floor(this.nodeIndices.length*t);for(var i in this.nodes)this.nodes.hasOwnProperty(i)&&2==this.nodes[i].dynamicEdgesLength&&this.nodes[i].dynamicEdges.length>=2&&e>0&&(this._formClusterFromHub(this.nodes[i],!0,!0,1),e-=1)},e._getChainFraction=function(){var t=0,e=0;for(var i in this.nodes)this.nodes.hasOwnProperty(i)&&(2==this.nodes[i].dynamicEdgesLength&&this.nodes[i].dynamicEdges.length>=2&&(t+=1),e+=1);return t/e}},function(t,e,i){var s=i(1),o=i(40);e._putDataInSector=function(){this.sectors.active[this._sector()].nodes=this.nodes,this.sectors.active[this._sector()].edges=this.edges,this.sectors.active[this._sector()].nodeIndices=this.nodeIndices},e._switchToSector=function(t,e){void 0===e||"active"==e?this._switchToActiveSector(t):this._switchToFrozenSector(t)},e._switchToActiveSector=function(t){this.nodeIndices=this.sectors.active[t].nodeIndices,this.nodes=this.sectors.active[t].nodes,this.edges=this.sectors.active[t].edges},e._switchToSupportSector=function(){this.nodeIndices=this.sectors.support.nodeIndices,this.nodes=this.sectors.support.nodes,this.edges=this.sectors.support.edges},e._switchToFrozenSector=function(t){this.nodeIndices=this.sectors.frozen[t].nodeIndices,this.nodes=this.sectors.frozen[t].nodes,this.edges=this.sectors.frozen[t].edges},e._loadLatestSector=function(){this._switchToSector(this._sector())},e._sector=function(){return this.activeSector[this.activeSector.length-1]},e._previousSector=function(){if(this.activeSector.length>1)return this.activeSector[this.activeSector.length-2];throw new TypeError("there are not enough sectors in the this.activeSector array.")},e._setActiveSector=function(t){this.activeSector.push(t)},e._forgetLastSector=function(){this.activeSector.pop()},e._createNewSector=function(t){this.sectors.active[t]={nodes:{},edges:{},nodeIndices:[],formationScale:this.scale,drawingNode:void 0},this.sectors.active[t].drawingNode=new o({id:t,color:{background:"#eaefef",border:"495c5e"}},{},{},this.constants),this.sectors.active[t].drawingNode.clusterSize=2},e._deleteActiveSector=function(t){delete this.sectors.active[t]},e._deleteFrozenSector=function(t){delete this.sectors.frozen[t]},e._freezeSector=function(t){this.sectors.frozen[t]=this.sectors.active[t],this._deleteActiveSector(t)},e._activateSector=function(t){this.sectors.active[t]=this.sectors.frozen[t],this._deleteFrozenSector(t)},e._mergeThisWithFrozen=function(t){for(var e in this.nodes)this.nodes.hasOwnProperty(e)&&(this.sectors.frozen[t].nodes[e]=this.nodes[e]);for(var i in this.edges)this.edges.hasOwnProperty(i)&&(this.sectors.frozen[t].edges[i]=this.edges[i]);for(var s=0;s1?this[t](o[0],o[1]):this[t](e))}return this._loadLatestSector(),i},e._doInSupportSector=function(t,e){var i=!1;if(void 0===e)this._switchToSupportSector(),i=this[t]();else{this._switchToSupportSector();var s=Array.prototype.splice.call(arguments,1);i=s.length>1?this[t](s[0],s[1]):this[t](e)}return this._loadLatestSector(),i},e._doInAllFrozenSectors=function(t,e){if(void 0===e)for(var i in this.sectors.frozen)this.sectors.frozen.hasOwnProperty(i)&&(this._switchToFrozenSector(i),this[t]());else for(var i in this.sectors.frozen)if(this.sectors.frozen.hasOwnProperty(i)){this._switchToFrozenSector(i);var s=Array.prototype.splice.call(arguments,1);s.length>1?this[t](s[0],s[1]):this[t](e)}this._loadLatestSector()},e._doInAllSectors=function(t,e){var i=Array.prototype.splice.call(arguments,1);void 0===e?(this._doInAllActiveSectors(t),this._doInAllFrozenSectors(t)):i.length>1?(this._doInAllActiveSectors(t,i[0],i[1]),this._doInAllFrozenSectors(t,i[0],i[1])):(this._doInAllActiveSectors(t,e),this._doInAllFrozenSectors(t,e))},e._clearNodeIndexList=function(){var t=this._sector();this.sectors.active[t].nodeIndices=[],this.nodeIndices=this.sectors.active[t].nodeIndices},e._drawSectorNodes=function(t,e){var i,s=1e9,o=-1e9,n=1e9,r=-1e9;for(var a in this.sectors[e])if(this.sectors[e].hasOwnProperty(a)&&void 0!==this.sectors[e][a].drawingNode){this._switchToSector(a,e),s=1e9,o=-1e9,n=1e9,r=-1e9;for(var h in this.nodes)this.nodes.hasOwnProperty(h)&&(i=this.nodes[h],i.resize(t),n>i.x-.5*i.width&&(n=i.x-.5*i.width),ri.y-.5*i.height&&(s=i.y-.5*i.height),o0?this.nodes[i[i.length-1]]:null},e._getEdgesOverlappingWith=function(t,e){var i=this.edges;for(var s in i)i.hasOwnProperty(s)&&i[s].isOverlappingWith(t)&&e.push(s)},e._getAllEdgesOverlappingWith=function(t){var e=[];return this._doInAllActiveSectors("_getEdgesOverlappingWith",t,e),e},e._getEdgeAt=function(t){var e=this._pointerToPositionObject(t),i=this._getAllEdgesOverlappingWith(e);return i.length>0?this.edges[i[i.length-1]]:null},e._addToSelection=function(t){t instanceof s?this.selectionObj.nodes[t.id]=t:this.selectionObj.edges[t.id]=t},e._addToHover=function(t){t instanceof s?this.hoverObj.nodes[t.id]=t:this.hoverObj.edges[t.id]=t},e._removeFromSelection=function(t){t instanceof s?delete this.selectionObj.nodes[t.id]:delete this.selectionObj.edges[t.id]},e._unselectAll=function(t){void 0===t&&(t=!1);for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&this.selectionObj.nodes[e].unselect();for(var i in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(i)&&this.selectionObj.edges[i].unselect();this.selectionObj={nodes:{},edges:{}},0==t&&this.emit("select",this.getSelection())},e._unselectClusters=function(t){void 0===t&&(t=!1);for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&this.selectionObj.nodes[e].clusterSize>1&&(this.selectionObj.nodes[e].unselect(),this._removeFromSelection(this.selectionObj.nodes[e]));0==t&&this.emit("select",this.getSelection())},e._getSelectedNodeCount=function(){var t=0;for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&(t+=1);return t},e._getSelectedNode=function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t))return this.selectionObj.nodes[t];return null},e._getSelectedEdge=function(){for(var t in this.selectionObj.edges)if(this.selectionObj.edges.hasOwnProperty(t))return this.selectionObj.edges[t];return null},e._getSelectedEdgeCount=function(){var t=0;for(var e in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(e)&&(t+=1);return t},e._getSelectedObjectCount=function(){var t=0;for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&(t+=1);for(var i in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(i)&&(t+=1);return t},e._selectionIsEmpty=function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t))return!1;for(var e in this.selectionObj.edges)if(this.selectionObj.edges.hasOwnProperty(e))return!1;return!0},e._clusterInSelection=function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t)&&this.selectionObj.nodes[t].clusterSize>1)return!0;return!1},e._selectConnectedEdges=function(t){for(var e=0;ei;i++){o=t[i];var n=this.nodes[o];if(!n)throw new RangeError('Node with id "'+o+'" not found');this._selectObject(n,!0,!0,e,!0)}this.redraw()},e.selectEdges=function(t){var e,i,s;if(!t||void 0==t.length)throw"Selection must be an array with ids";for(this._unselectAll(!0),e=0,i=t.length;i>e;e++){s=t[e];var o=this.edges[s];if(!o)throw new RangeError('Edge with id "'+s+'" not found');this._selectObject(o,!0,!0,!1,!0)}this.redraw()},e._updateSelection=function(){for(var t in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(t)&&(this.nodes.hasOwnProperty(t)||delete this.selectionObj.nodes[t]);for(var e in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(e)&&(this.edges.hasOwnProperty(e)||delete this.selectionObj.edges[e])}},function(t,e,i){var s=i(1),o=i(40),n=i(37);e._clearManipulatorBar=function(){this._recursiveDOMDelete(this.manipulationDiv),this.manipulationDOM={},this._manipulationReleaseOverload=function(){},delete this.sectors.support.nodes.targetNode,delete this.sectors.support.nodes.targetViaNode,this.controlNodesActive=!1,this.freezeSimulation=!1},e._restoreOverloadedFunctions=function(){for(var t in this.cachedFunctions)this.cachedFunctions.hasOwnProperty(t)&&(this[t]=this.cachedFunctions[t],delete this.cachedFunctions[t])},e._toggleEditMode=function(){this.editMode=!this.editMode;var t=this.manipulationDiv,e=this.closeDiv,i=this.editModeDiv;1==this.editMode?(t.style.display="block",e.style.display="block",i.style.display="none",e.onclick=this._toggleEditMode.bind(this)):(t.style.display="none",e.style.display="none",i.style.display="block",e.onclick=null),this._createManipulatorBar()},e._createManipulatorBar=function(){this.boundFunction&&this.off("select",this.boundFunction);var t=this.constants.locales[this.constants.locale];if(void 0!==this.edgeBeingEdited&&(this.edgeBeingEdited._disableControlNodes(),this.edgeBeingEdited=void 0,this.selectedControlNode=null,this.controlNodesActive=!1,this._redraw()),this._restoreOverloadedFunctions(),this.freezeSimulation=!1,this.blockConnectingEdgeSelection=!1,this.forceAppendSelection=!1,this.manipulationDOM={},1==this.editMode){for(;this.manipulationDiv.hasChildNodes();)this.manipulationDiv.removeChild(this.manipulationDiv.firstChild);this.manipulationDOM.addNodeSpan=document.createElement("span"),this.manipulationDOM.addNodeSpan.className="network-manipulationUI add",this.manipulationDOM.addNodeLabelSpan=document.createElement("span"),this.manipulationDOM.addNodeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.addNodeLabelSpan.innerHTML=t.addNode,this.manipulationDOM.addNodeSpan.appendChild(this.manipulationDOM.addNodeLabelSpan),this.manipulationDOM.seperatorLineDiv1=document.createElement("div"),this.manipulationDOM.seperatorLineDiv1.className="network-seperatorLine",this.manipulationDOM.addEdgeSpan=document.createElement("span"),this.manipulationDOM.addEdgeSpan.className="network-manipulationUI connect",this.manipulationDOM.addEdgeLabelSpan=document.createElement("span"),this.manipulationDOM.addEdgeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.addEdgeLabelSpan.innerHTML=t.addEdge,this.manipulationDOM.addEdgeSpan.appendChild(this.manipulationDOM.addEdgeLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.addNodeSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv1),this.manipulationDiv.appendChild(this.manipulationDOM.addEdgeSpan),1==this._getSelectedNodeCount()&&this.triggerFunctions.edit?(this.manipulationDOM.seperatorLineDiv2=document.createElement("div"),this.manipulationDOM.seperatorLineDiv2.className="network-seperatorLine",this.manipulationDOM.editNodeSpan=document.createElement("span"),this.manipulationDOM.editNodeSpan.className="network-manipulationUI edit",this.manipulationDOM.editNodeLabelSpan=document.createElement("span"),this.manipulationDOM.editNodeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.editNodeLabelSpan.innerHTML=t.editNode,this.manipulationDOM.editNodeSpan.appendChild(this.manipulationDOM.editNodeLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv2),this.manipulationDiv.appendChild(this.manipulationDOM.editNodeSpan)):1==this._getSelectedEdgeCount()&&0==this._getSelectedNodeCount()&&(this.manipulationDOM.seperatorLineDiv3=document.createElement("div"),this.manipulationDOM.seperatorLineDiv3.className="network-seperatorLine",this.manipulationDOM.editEdgeSpan=document.createElement("span"),this.manipulationDOM.editEdgeSpan.className="network-manipulationUI edit",this.manipulationDOM.editEdgeLabelSpan=document.createElement("span"),this.manipulationDOM.editEdgeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.editEdgeLabelSpan.innerHTML=t.editEdge,this.manipulationDOM.editEdgeSpan.appendChild(this.manipulationDOM.editEdgeLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv3),this.manipulationDiv.appendChild(this.manipulationDOM.editEdgeSpan)),0==this._selectionIsEmpty()&&(this.manipulationDOM.seperatorLineDiv4=document.createElement("div"),this.manipulationDOM.seperatorLineDiv4.className="network-seperatorLine",this.manipulationDOM.deleteSpan=document.createElement("span"),this.manipulationDOM.deleteSpan.className="network-manipulationUI delete",this.manipulationDOM.deleteLabelSpan=document.createElement("span"),this.manipulationDOM.deleteLabelSpan.className="network-manipulationLabel",this.manipulationDOM.deleteLabelSpan.innerHTML=t.del,this.manipulationDOM.deleteSpan.appendChild(this.manipulationDOM.deleteLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv4),this.manipulationDiv.appendChild(this.manipulationDOM.deleteSpan)),this.manipulationDOM.addNodeSpan.onclick=this._createAddNodeToolbar.bind(this),this.manipulationDOM.addEdgeSpan.onclick=this._createAddEdgeToolbar.bind(this),1==this._getSelectedNodeCount()&&this.triggerFunctions.edit?this.manipulationDOM.editNodeSpan.onclick=this._editNode.bind(this):1==this._getSelectedEdgeCount()&&0==this._getSelectedNodeCount()&&(this.manipulationDOM.editEdgeSpan.onclick=this._createEditEdgeToolbar.bind(this)),0==this._selectionIsEmpty()&&(this.manipulationDOM.deleteSpan.onclick=this._deleteSelected.bind(this)),this.closeDiv.onclick=this._toggleEditMode.bind(this); var e=this;this.boundFunction=e._createManipulatorBar,this.on("select",this.boundFunction)}else{for(;this.editModeDiv.hasChildNodes();)this.editModeDiv.removeChild(this.editModeDiv.firstChild);this.manipulationDOM.editModeSpan=document.createElement("span"),this.manipulationDOM.editModeSpan.className="network-manipulationUI edit editmode",this.manipulationDOM.editModeLabelSpan=document.createElement("span"),this.manipulationDOM.editModeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.editModeLabelSpan.innerHTML=t.edit,this.manipulationDOM.editModeSpan.appendChild(this.manipulationDOM.editModeLabelSpan),this.editModeDiv.appendChild(this.manipulationDOM.editModeSpan),this.manipulationDOM.editModeSpan.onclick=this._toggleEditMode.bind(this)}},e._createAddNodeToolbar=function(){this._clearManipulatorBar(),this.boundFunction&&this.off("select",this.boundFunction);var t=this.constants.locales[this.constants.locale];this.manipulationDOM={},this.manipulationDOM.backSpan=document.createElement("span"),this.manipulationDOM.backSpan.className="network-manipulationUI back",this.manipulationDOM.backLabelSpan=document.createElement("span"),this.manipulationDOM.backLabelSpan.className="network-manipulationLabel",this.manipulationDOM.backLabelSpan.innerHTML=t.back,this.manipulationDOM.backSpan.appendChild(this.manipulationDOM.backLabelSpan),this.manipulationDOM.seperatorLineDiv1=document.createElement("div"),this.manipulationDOM.seperatorLineDiv1.className="network-seperatorLine",this.manipulationDOM.descriptionSpan=document.createElement("span"),this.manipulationDOM.descriptionSpan.className="network-manipulationUI none",this.manipulationDOM.descriptionLabelSpan=document.createElement("span"),this.manipulationDOM.descriptionLabelSpan.className="network-manipulationLabel",this.manipulationDOM.descriptionLabelSpan.innerHTML=t.addDescription,this.manipulationDOM.descriptionSpan.appendChild(this.manipulationDOM.descriptionLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.backSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv1),this.manipulationDiv.appendChild(this.manipulationDOM.descriptionSpan),this.manipulationDOM.backSpan.onclick=this._createManipulatorBar.bind(this);var e=this;this.boundFunction=e._addNode,this.on("select",this.boundFunction)},e._createAddEdgeToolbar=function(){this._clearManipulatorBar(),this._unselectAll(!0),this.freezeSimulation=!0,this.boundFunction&&this.off("select",this.boundFunction);var t=this.constants.locales[this.constants.locale];this._unselectAll(),this.forceAppendSelection=!1,this.blockConnectingEdgeSelection=!0,this.manipulationDOM={},this.manipulationDOM.backSpan=document.createElement("span"),this.manipulationDOM.backSpan.className="network-manipulationUI back",this.manipulationDOM.backLabelSpan=document.createElement("span"),this.manipulationDOM.backLabelSpan.className="network-manipulationLabel",this.manipulationDOM.backLabelSpan.innerHTML=t.back,this.manipulationDOM.backSpan.appendChild(this.manipulationDOM.backLabelSpan),this.manipulationDOM.seperatorLineDiv1=document.createElement("div"),this.manipulationDOM.seperatorLineDiv1.className="network-seperatorLine",this.manipulationDOM.descriptionSpan=document.createElement("span"),this.manipulationDOM.descriptionSpan.className="network-manipulationUI none",this.manipulationDOM.descriptionLabelSpan=document.createElement("span"),this.manipulationDOM.descriptionLabelSpan.className="network-manipulationLabel",this.manipulationDOM.descriptionLabelSpan.innerHTML=t.edgeDescription,this.manipulationDOM.descriptionSpan.appendChild(this.manipulationDOM.descriptionLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.backSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv1),this.manipulationDiv.appendChild(this.manipulationDOM.descriptionSpan),this.manipulationDOM.backSpan.onclick=this._createManipulatorBar.bind(this);var e=this;this.boundFunction=e._handleConnect,this.on("select",this.boundFunction),this.cachedFunctions._handleTouch=this._handleTouch,this.cachedFunctions._manipulationReleaseOverload=this._manipulationReleaseOverload,this.cachedFunctions._handleDragStart=this._handleDragStart,this.cachedFunctions._handleDragEnd=this._handleDragEnd,this._handleTouch=this._handleConnect,this._manipulationReleaseOverload=function(){},this._handleDragStart=function(){},this._handleDragEnd=this._finishConnect,this._redraw()},e._createEditEdgeToolbar=function(){this._clearManipulatorBar(),this.controlNodesActive=!0,this.boundFunction&&this.off("select",this.boundFunction),this.edgeBeingEdited=this._getSelectedEdge(),this.edgeBeingEdited._enableControlNodes();var t=this.constants.locales[this.constants.locale];this.manipulationDOM={},this.manipulationDOM.backSpan=document.createElement("span"),this.manipulationDOM.backSpan.className="network-manipulationUI back",this.manipulationDOM.backLabelSpan=document.createElement("span"),this.manipulationDOM.backLabelSpan.className="network-manipulationLabel",this.manipulationDOM.backLabelSpan.innerHTML=t.back,this.manipulationDOM.backSpan.appendChild(this.manipulationDOM.backLabelSpan),this.manipulationDOM.seperatorLineDiv1=document.createElement("div"),this.manipulationDOM.seperatorLineDiv1.className="network-seperatorLine",this.manipulationDOM.descriptionSpan=document.createElement("span"),this.manipulationDOM.descriptionSpan.className="network-manipulationUI none",this.manipulationDOM.descriptionLabelSpan=document.createElement("span"),this.manipulationDOM.descriptionLabelSpan.className="network-manipulationLabel",this.manipulationDOM.descriptionLabelSpan.innerHTML=t.editEdgeDescription,this.manipulationDOM.descriptionSpan.appendChild(this.manipulationDOM.descriptionLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.backSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv1),this.manipulationDiv.appendChild(this.manipulationDOM.descriptionSpan),this.manipulationDOM.backSpan.onclick=this._createManipulatorBar.bind(this),this.cachedFunctions._handleTouch=this._handleTouch,this.cachedFunctions._manipulationReleaseOverload=this._manipulationReleaseOverload,this.cachedFunctions._handleTap=this._handleTap,this.cachedFunctions._handleDragStart=this._handleDragStart,this.cachedFunctions._handleOnDrag=this._handleOnDrag,this._handleTouch=this._selectControlNode,this._handleTap=function(){},this._handleOnDrag=this._controlNodeDrag,this._handleDragStart=function(){},this._manipulationReleaseOverload=this._releaseControlNode,this._redraw()},e._selectControlNode=function(t){this.edgeBeingEdited.controlNodes.from.unselect(),this.edgeBeingEdited.controlNodes.to.unselect(),this.selectedControlNode=this.edgeBeingEdited._getSelectedControlNode(this._XconvertDOMtoCanvas(t.x),this._YconvertDOMtoCanvas(t.y)),null!==this.selectedControlNode&&(this.selectedControlNode.select(),this.freezeSimulation=!0),this._redraw()},e._controlNodeDrag=function(t){var e=this._getPointer(t.center);null!==this.selectedControlNode&&void 0!==this.selectedControlNode&&(this.selectedControlNode.x=this._XconvertDOMtoCanvas(e.x),this.selectedControlNode.y=this._YconvertDOMtoCanvas(e.y)),this._redraw()},e._releaseControlNode=function(t){var e=this._getNodeAt(t);null!==e?(1==this.edgeBeingEdited.controlNodes.from.selected&&(this.edgeBeingEdited._restoreControlNodes(),this._editEdge(e.id,this.edgeBeingEdited.to.id),this.edgeBeingEdited.controlNodes.from.unselect()),1==this.edgeBeingEdited.controlNodes.to.selected&&(this.edgeBeingEdited._restoreControlNodes(),this._editEdge(this.edgeBeingEdited.from.id,e.id),this.edgeBeingEdited.controlNodes.to.unselect())):this.edgeBeingEdited._restoreControlNodes(),this.freezeSimulation=!1,this._redraw()},e._handleConnect=function(t){if(0==this._getSelectedNodeCount()){var e=this._getNodeAt(t);if(null!=e)if(e.clusterSize>1)alert(this.constants.locales[this.constants.locale].createEdgeError);else{this._selectObject(e,!1);var i=this.sectors.support.nodes;i.targetNode=new o({id:"targetNode"},{},{},this.constants);var s=i.targetNode;s.x=e.x,s.y=e.y,this.edges.connectionEdge=new n({id:"connectionEdge",from:e.id,to:s.id},this,this.constants);var r=this.edges.connectionEdge;r.from=e,r.connected=!0,r.options.smoothCurves={enabled:!0,dynamic:!1,type:"continuous",roundness:.5},r.selected=!0,r.to=s,this.cachedFunctions._handleOnDrag=this._handleOnDrag,this._handleOnDrag=function(t){var e=this._getPointer(t.center),i=this.edges.connectionEdge;i.to.x=this._XconvertDOMtoCanvas(e.x),i.to.y=this._YconvertDOMtoCanvas(e.y)},this.moving=!0,this.start()}}},e._finishConnect=function(t){if(1==this._getSelectedNodeCount()){var e=this._getPointer(t.center);this._handleOnDrag=this.cachedFunctions._handleOnDrag,delete this.cachedFunctions._handleOnDrag;var i=this.edges.connectionEdge.fromId;delete this.edges.connectionEdge,delete this.sectors.support.nodes.targetNode,delete this.sectors.support.nodes.targetViaNode;var s=this._getNodeAt(e);null!=s&&(s.clusterSize>1?alert(this.constants.locales[this.constants.locale].createEdgeError):(this._createEdge(i,s.id),this._createManipulatorBar())),this._unselectAll()}},e._addNode=function(){if(this._selectionIsEmpty()&&1==this.editMode){var t=this._pointerToPositionObject(this.pointerPosition),e={id:s.randomUUID(),x:t.left,y:t.top,label:"new",allowedToMoveX:!0,allowedToMoveY:!0};if(this.triggerFunctions.add){if(2!=this.triggerFunctions.add.length)throw new Error("The function for add does not support two arguments (data,callback)");var i=this;this.triggerFunctions.add(e,function(t){i.nodesData.add(t),i._createManipulatorBar(),i.moving=!0,i.start()})}else this.nodesData.add(e),this._createManipulatorBar(),this.moving=!0,this.start()}},e._createEdge=function(t,e){if(1==this.editMode){var i={from:t,to:e};if(this.triggerFunctions.connect){if(2!=this.triggerFunctions.connect.length)throw new Error("The function for connect does not support two arguments (data,callback)");var s=this;this.triggerFunctions.connect(i,function(t){s.edgesData.add(t),s.moving=!0,s.start()})}else this.edgesData.add(i),this.moving=!0,this.start()}},e._editEdge=function(t,e){if(1==this.editMode){var i={id:this.edgeBeingEdited.id,from:t,to:e};if(this.triggerFunctions.editEdge){if(2!=this.triggerFunctions.editEdge.length)throw new Error("The function for edit does not support two arguments (data, callback)");var s=this;this.triggerFunctions.editEdge(i,function(t){s.edgesData.update(t),s.moving=!0,s.start()})}else this.edgesData.update(i),this.moving=!0,this.start()}},e._editNode=function(){if(!this.triggerFunctions.edit||1!=this.editMode)throw new Error("No edit function has been bound to this button");var t=this._getSelectedNode(),e={id:t.id,label:t.label,group:t.options.group,shape:t.options.shape,color:{background:t.options.color.background,border:t.options.color.border,highlight:{background:t.options.color.highlight.background,border:t.options.color.highlight.border}}};if(2!=this.triggerFunctions.edit.length)throw new Error("The function for edit does not support two arguments (data, callback)");var i=this;this.triggerFunctions.edit(e,function(t){i.nodesData.update(t),i._createManipulatorBar(),i.moving=!0,i.start()})},e._deleteSelected=function(){if(!this._selectionIsEmpty()&&1==this.editMode)if(this._clusterInSelection())alert(this.constants.locales[this.constants.locale].deleteClusterError);else{var t=this.getSelectedNodes(),e=this.getSelectedEdges();if(this.triggerFunctions.del){var i=this,s={nodes:t,edges:e};if(2!=this.triggerFunctions.del.length)throw new Error("The function for delete does not support two arguments (data, callback)");this.triggerFunctions.del(s,function(t){i.edgesData.remove(t.edges),i.nodesData.remove(t.nodes),i._unselectAll(),i.moving=!0,i.start()})}else this.edgesData.remove(e),this.nodesData.remove(t),this._unselectAll(),this.moving=!0,this.start()}}},function(t,e,i){var s=(i(1),i(47)),o=i(45);e._cleanNavigation=function(){if(0!=this.navigationHammers.existing.length){for(var t=0;t0){var t,e,i=0,s=!1,o=!1;for(e in this.nodes)this.nodes.hasOwnProperty(e)&&(t=this.nodes[e],-1!=t.level?s=!0:o=!0,is&&(n.xFixed=!1,n.x=i[n.level].minPos,r=!0):n.yFixed&&n.level>s&&(n.yFixed=!1,n.y=i[n.level].minPos,r=!0),1==r&&(i[n.level].minPos+=i[n.level].nodeSpacing,n.edges.length>1&&this._placeBranchNodes(n.edges,n.id,i,n.level))}},e._setLevel=function(t,e,i){for(var s=0;st)&&(o.level=t,o.edges.length>1&&this._setLevel(t+1,o.edges,o.id))}},e._setLevelDirected=function(t,e,i){this.nodes[i].hierarchyEnumerated=!0;for(var s,o,n=0;n1&&s.hierarchyEnumerated===!1&&this._setLevelDirected(s.level,s.edges,s.id)},e._restoreNodes=function(){for(var t in this.nodes)this.nodes.hasOwnProperty(t)&&(this.nodes[t].xFixed=!1,this.nodes[t].yFixed=!1)}},function(t,e,i){function s(){this.constants.smoothCurves.enabled=!this.constants.smoothCurves.enabled;var t=document.getElementById("graph_toggleSmooth");t.style.background=1==this.constants.smoothCurves.enabled?"#A4FF56":"#FF8532",this._configureSmoothCurves(!1)}function o(){for(var t in this.calculationNodes)this.calculationNodes.hasOwnProperty(t)&&(this.calculationNodes[t].vx=0,this.calculationNodes[t].vy=0,this.calculationNodes[t].fx=0,this.calculationNodes[t].fy=0);1==this.constants.hierarchicalLayout.enabled?(this._setupHierarchicalLayout(),a.call(this,"graph_H_nd",1,"physics_hierarchicalRepulsion_nodeDistance"),a.call(this,"graph_H_cg",1,"physics_centralGravity"),a.call(this,"graph_H_sc",1,"physics_springConstant"),a.call(this,"graph_H_sl",1,"physics_springLength"),a.call(this,"graph_H_damp",1,"physics_damping")):this.repositionNodes(),this.moving=!0,this.start()}function n(){var t="No options are required, default values used.",e=[],i=document.getElementById("graph_physicsMethod1"),s=document.getElementById("graph_physicsMethod2");if(1==i.checked){if(this.constants.physics.barnesHut.gravitationalConstant!=this.backupConstants.physics.barnesHut.gravitationalConstant&&e.push("gravitationalConstant: "+this.constants.physics.barnesHut.gravitationalConstant),this.constants.physics.centralGravity!=this.backupConstants.physics.barnesHut.centralGravity&&e.push("centralGravity: "+this.constants.physics.centralGravity),this.constants.physics.springLength!=this.backupConstants.physics.barnesHut.springLength&&e.push("springLength: "+this.constants.physics.springLength),this.constants.physics.springConstant!=this.backupConstants.physics.barnesHut.springConstant&&e.push("springConstant: "+this.constants.physics.springConstant),this.constants.physics.damping!=this.backupConstants.physics.barnesHut.damping&&e.push("damping: "+this.constants.physics.damping),0!=e.length){t="var options = {",t+="physics: {barnesHut: {";for(var o=0;othis.constants.clustering.clusterThreshold&&1==this.constants.clustering.enabled&&this.clusterToFit(this.constants.clustering.reduceToNodes,!1),this._calculateForces())},e._calculateForces=function(){this._calculateGravitationalForces(),this._calculateNodeForces(),this.constants.physics.springConstant>0&&(1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic?this._calculateSpringForcesWithSupport():1==this.constants.physics.hierarchicalRepulsion.enabled?this._calculateHierarchicalSpringForces():this._calculateSpringForces())},e._updateCalculationNodes=function(){if(1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic){this.calculationNodes={},this.calculationNodeIndices=[];for(var t in this.nodes)this.nodes.hasOwnProperty(t)&&(this.calculationNodes[t]=this.nodes[t]);var e=this.sectors.support.nodes;for(var i in e)e.hasOwnProperty(i)&&(this.edges.hasOwnProperty(e[i].parentEdgeId)?this.calculationNodes[i]=e[i]:e[i]._setForce(0,0));for(var s in this.calculationNodes)this.calculationNodes.hasOwnProperty(s)&&this.calculationNodeIndices.push(s)}else this.calculationNodes=this.nodes,this.calculationNodeIndices=this.nodeIndices},e._calculateGravitationalForces=function(){var t,e,i,s,o,n=this.calculationNodes,r=this.constants.physics.centralGravity,a=0;for(o=0;oSimulation Mode:Barnes HutRepulsionHierarchical
Options:
',this.containerElement.parentElement.insertBefore(this.physicsConfiguration,this.containerElement),this.optionsDiv=document.createElement("div"),this.optionsDiv.style.fontSize="14px",this.optionsDiv.style.fontFamily="verdana",this.containerElement.parentElement.insertBefore(this.optionsDiv,this.containerElement); var e;e=document.getElementById("graph_BH_gc"),e.onchange=a.bind(this,"graph_BH_gc",-1,"physics_barnesHut_gravitationalConstant"),e=document.getElementById("graph_BH_cg"),e.onchange=a.bind(this,"graph_BH_cg",1,"physics_centralGravity"),e=document.getElementById("graph_BH_sc"),e.onchange=a.bind(this,"graph_BH_sc",1,"physics_springConstant"),e=document.getElementById("graph_BH_sl"),e.onchange=a.bind(this,"graph_BH_sl",1,"physics_springLength"),e=document.getElementById("graph_BH_damp"),e.onchange=a.bind(this,"graph_BH_damp",1,"physics_damping"),e=document.getElementById("graph_R_nd"),e.onchange=a.bind(this,"graph_R_nd",1,"physics_repulsion_nodeDistance"),e=document.getElementById("graph_R_cg"),e.onchange=a.bind(this,"graph_R_cg",1,"physics_centralGravity"),e=document.getElementById("graph_R_sc"),e.onchange=a.bind(this,"graph_R_sc",1,"physics_springConstant"),e=document.getElementById("graph_R_sl"),e.onchange=a.bind(this,"graph_R_sl",1,"physics_springLength"),e=document.getElementById("graph_R_damp"),e.onchange=a.bind(this,"graph_R_damp",1,"physics_damping"),e=document.getElementById("graph_H_nd"),e.onchange=a.bind(this,"graph_H_nd",1,"physics_hierarchicalRepulsion_nodeDistance"),e=document.getElementById("graph_H_cg"),e.onchange=a.bind(this,"graph_H_cg",1,"physics_centralGravity"),e=document.getElementById("graph_H_sc"),e.onchange=a.bind(this,"graph_H_sc",1,"physics_springConstant"),e=document.getElementById("graph_H_sl"),e.onchange=a.bind(this,"graph_H_sl",1,"physics_springLength"),e=document.getElementById("graph_H_damp"),e.onchange=a.bind(this,"graph_H_damp",1,"physics_damping"),e=document.getElementById("graph_H_direction"),e.onchange=a.bind(this,"graph_H_direction",t,"hierarchicalLayout_direction"),e=document.getElementById("graph_H_levsep"),e.onchange=a.bind(this,"graph_H_levsep",1,"hierarchicalLayout_levelSeparation"),e=document.getElementById("graph_H_nspac"),e.onchange=a.bind(this,"graph_H_nspac",1,"hierarchicalLayout_nodeSpacing");var i=document.getElementById("graph_physicsMethod1"),d=document.getElementById("graph_physicsMethod2"),l=document.getElementById("graph_physicsMethod3");d.checked=!0,this.constants.physics.barnesHut.enabled&&(i.checked=!0),this.constants.hierarchicalLayout.enabled&&(l.checked=!0);var c=document.getElementById("graph_toggleSmooth"),p=document.getElementById("graph_repositionNodes"),u=document.getElementById("graph_generateOptions");c.onclick=s.bind(this),p.onclick=o.bind(this),u.onclick=n.bind(this),c.style.background=1==this.constants.smoothCurves&&0==this.constants.dynamicSmoothCurves?"#A4FF56":"#FF8532",r.apply(this),i.onchange=r.bind(this),d.onchange=r.bind(this),l.onchange=r.bind(this)}},e._overWriteGraphConstants=function(t,e){var i=t.split("_");1==i.length?this.constants[i[0]]=e:2==i.length?this.constants[i[0]][i[1]]=e:3==i.length&&(this.constants[i[0]][i[1]][i[2]]=e)}},function(t){function e(t){throw new Error("Cannot find module '"+t+"'.")}e.keys=function(){return[]},e.resolve=e,t.exports=e,e.id=68},function(t,e){e._calculateNodeForces=function(){var t,e,i,s,o,n,r,a,h,d,l,c=this.calculationNodes,p=this.calculationNodeIndices,u=-2/3,m=4/3,f=this.constants.physics.repulsion.nodeDistance,g=f;for(d=0;di&&(r=.5*g>i?1:v*i+m,r*=0==n?1:1+n*this.constants.clustering.forceAmplification,r/=Math.max(i,.01*g),s=t*r,o=e*r,a.fx-=s,a.fy-=o,h.fx+=s,h.fy+=o)}}},function(t,e){e._calculateNodeForces=function(){var t,e,i,s,o,n,r,a,h,d,l=this.calculationNodes,c=this.calculationNodeIndices,p=this.constants.physics.hierarchicalRepulsion.nodeDistance;for(h=0;hi?-Math.pow(u*i,2)+Math.pow(u*p,2):0,0==i?i=.01:n/=i,s=t*n,o=e*n,r.fx-=s,r.fy-=o,a.fx+=s,a.fy+=o}},e._calculateHierarchicalSpringForces=function(){for(var t,e,i,s,o,n,r,a,h,d=this.edges,l=this.calculationNodes,c=this.calculationNodeIndices,p=0;pn;n++)t=e[i[n]],t.options.mass>0&&(this._getForceContribution(o.root.children.NW,t),this._getForceContribution(o.root.children.NE,t),this._getForceContribution(o.root.children.SW,t),this._getForceContribution(o.root.children.SE,t))}},e._getForceContribution=function(t,e){if(t.childrenCount>0){var i,s,o;if(i=t.centerOfMass.x-e.x,s=t.centerOfMass.y-e.y,o=Math.sqrt(i*i+s*s),o*t.calcSize>this.constants.physics.barnesHut.thetaInverted){0==o&&(o=.1*Math.random(),i=o);var n=this.constants.physics.barnesHut.gravitationalConstant*t.mass*e.options.mass/(o*o*o),r=i*n,a=s*n;e.fx+=r,e.fy+=a}else if(4==t.childrenCount)this._getForceContribution(t.children.NW,e),this._getForceContribution(t.children.NE,e),this._getForceContribution(t.children.SW,e),this._getForceContribution(t.children.SE,e);else if(t.children.data.id!=e.id){0==o&&(o=.5*Math.random(),i=o);var n=this.constants.physics.barnesHut.gravitationalConstant*t.mass*e.options.mass/(o*o*o),r=i*n,a=s*n;e.fx+=r,e.fy+=a}}},e._formBarnesHutTree=function(t,e){for(var i,s=e.length,o=Number.MAX_VALUE,n=Number.MAX_VALUE,r=-Number.MAX_VALUE,a=-Number.MAX_VALUE,h=0;s>h;h++){var d=t[e[h]].x,l=t[e[h]].y;t[e[h]].options.mass>0&&(o>d&&(o=d),d>r&&(r=d),n>l&&(n=l),l>a&&(a=l))}var c=Math.abs(r-o)-Math.abs(a-n);c>0?(n-=.5*c,a+=.5*c):(o+=.5*c,r-=.5*c);var p=1e-5,u=Math.max(p,Math.abs(r-o)),m=.5*u,f=.5*(o+r),g=.5*(n+a),v={root:{centerOfMass:{x:0,y:0},mass:0,range:{minX:f-m,maxX:f+m,minY:g-m,maxY:g+m},size:u,calcSize:1/u,children:{data:null},maxWidth:0,level:0,childrenCount:4}};for(this._splitBranch(v.root),h=0;s>h;h++)i=t[e[h]],i.options.mass>0&&this._placeInTree(v.root,i);this.barnesHutTree=v},e._updateBranchMass=function(t,e){var i=t.mass+e.options.mass,s=1/i;t.centerOfMass.x=t.centerOfMass.x*t.mass+e.x*e.options.mass,t.centerOfMass.x*=s,t.centerOfMass.y=t.centerOfMass.y*t.mass+e.y*e.options.mass,t.centerOfMass.y*=s,t.mass=i;var o=Math.max(Math.max(e.height,e.radius),e.width);t.maxWidth=t.maxWidthe.x?t.children.NW.range.maxY>e.y?this._placeInRegion(t,e,"NW"):this._placeInRegion(t,e,"SW"):t.children.NW.range.maxY>e.y?this._placeInRegion(t,e,"NE"):this._placeInRegion(t,e,"SE")},e._placeInRegion=function(t,e,i){switch(t.children[i].childrenCount){case 0:t.children[i].children.data=e,t.children[i].childrenCount=1,this._updateBranchMass(t.children[i],e);break;case 1:t.children[i].children.data.x==e.x&&t.children[i].children.data.y==e.y?(e.x+=Math.random(),e.y+=Math.random()):(this._splitBranch(t.children[i]),this._placeInTree(t.children[i],e));break;case 4:this._placeInTree(t.children[i],e)}},e._splitBranch=function(t){var e=null;1==t.childrenCount&&(e=t.children.data,t.mass=0,t.centerOfMass.x=0,t.centerOfMass.y=0),t.childrenCount=4,t.children.data=null,this._insertRegion(t,"NW"),this._insertRegion(t,"NE"),this._insertRegion(t,"SW"),this._insertRegion(t,"SE"),null!=e&&this._placeInTree(t,e)},e._insertRegion=function(t,e){var i,s,o,n,r=.5*t.size;switch(e){case"NW":i=t.range.minX,s=t.range.minX+r,o=t.range.minY,n=t.range.minY+r;break;case"NE":i=t.range.minX+r,s=t.range.maxX,o=t.range.minY,n=t.range.minY+r;break;case"SW":i=t.range.minX,s=t.range.minX+r,o=t.range.minY+r,n=t.range.maxY;break;case"SE":i=t.range.minX+r,s=t.range.maxX,o=t.range.minY+r,n=t.range.maxY}t.children[e]={centerOfMass:{x:0,y:0},mass:0,range:{minX:i,maxX:s,minY:o,maxY:n},size:.5*t.size,calcSize:2*t.calcSize,children:{data:null},maxWidth:0,level:t.level+1,childrenCount:0}},e._drawTree=function(t,e){void 0!==this.barnesHutTree&&(t.lineWidth=1,this._drawBranch(this.barnesHutTree.root,t,e))},e._drawBranch=function(t,e,i){void 0===i&&(i="#FF0000"),4==t.childrenCount&&(this._drawBranch(t.children.NW,e),this._drawBranch(t.children.NE,e),this._drawBranch(t.children.SE,e),this._drawBranch(t.children.SW,e)),e.strokeStyle=i,e.beginPath(),e.moveTo(t.range.minX,t.range.minY),e.lineTo(t.range.maxX,t.range.minY),e.stroke(),e.beginPath(),e.moveTo(t.range.maxX,t.range.minY),e.lineTo(t.range.maxX,t.range.maxY),e.stroke(),e.beginPath(),e.moveTo(t.range.maxX,t.range.maxY),e.lineTo(t.range.minX,t.range.maxY),e.stroke(),e.beginPath(),e.moveTo(t.range.minX,t.range.maxY),e.lineTo(t.range.minX,t.range.minY),e.stroke()}},function(t){t.exports=function(t){return t.webpackPolyfill||(t.deprecate=function(){},t.paths=[],t.children=[],t.webpackPolyfill=1),t}},function(t,e){(function(e){t.exports=e}).call(e,{})}])}); -//# sourceMappingURL=vis.map diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 669ad48937c05..e7c1d475d4e52 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -106,14 +106,18 @@ span.rest-uri { } pre { - font-size: 0.8em; + font-size: 12px; + line-height: 18px; + padding: 6px; + margin: 0; + border-radius: 3px; } .stage-details { max-height: 100px; overflow-y: auto; margin: 0; - transition: max-height 0.5s ease-out, padding 0.5s ease-out; + transition: max-height 0.25s ease-out, padding 0.25s ease-out; } .stage-details.collapsed { @@ -135,7 +139,7 @@ pre { max-height: 300px; overflow-y: auto; margin: 0; - transition: max-height 0.5s ease-out, padding 0.5s ease-out; + transition: max-height 0.25s ease-out, padding 0.25s ease-out; } .stacktrace-details.collapsed { @@ -158,7 +162,7 @@ span.additional-metric-title { } .tooltip { - font-weight: normal; + font-weight: normal; } .arrow-open { @@ -166,9 +170,9 @@ span.additional-metric-title { height: 0; border-left: 5px solid transparent; border-right: 5px solid transparent; - border-top: 5px solid black; - float: left; - margin-top: 6px; + border-top: 5px solid #08c; + display: inline-block; + margin-bottom: 2px; } .arrow-closed { @@ -176,8 +180,10 @@ span.additional-metric-title { height: 0; border-top: 5px solid transparent; border-bottom: 5px solid transparent; - border-left: 5px solid black; + border-left: 5px solid #08c; display: inline-block; + margin-left: 2px; + margin-right: 3px; } .version { @@ -196,3 +202,17 @@ span.additional-metric-title { .serialization_time, .getting_result_time { display: none; } + +.accordion-inner { + background: #f5f5f5; +} + +.accordion-inner pre { + border: 0; + padding: 0; + background: none; +} + +a.expandbutton { + cursor: pointer; +} diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 16072283edbe9..018422827e1c8 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -367,7 +367,11 @@ private[spark] object MapOutputTracker extends Logging { // Opposite of serializeMapStatuses. def deserializeMapStatuses(bytes: Array[Byte]): Array[MapStatus] = { val objIn = new ObjectInputStream(new GZIPInputStream(new ByteArrayInputStream(bytes))) - objIn.readObject().asInstanceOf[Array[MapStatus]] + Utils.tryWithSafeFinally { + objIn.readObject().asInstanceOf[Array[MapStatus]] + } { + objIn.close() + } } // Convert an array of MapStatuses to locations and sizes for a given reduce ID. If diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 2ca6882c8d890..b59f562d05ead 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -407,15 +407,17 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") + // "_jobProgressListener" should be set up before creating SparkEnv because when creating + // "SparkEnv", some messages will be posted to "listenerBus" and we should not miss them. + _jobProgressListener = new JobProgressListener(_conf) + listenerBus.addListener(jobProgressListener) + // Create the Spark execution environment (cache, map output tracker, etc) _env = createSparkEnv(_conf, isLocal, listenerBus) SparkEnv.set(_env) _metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, _conf) - _jobProgressListener = new JobProgressListener(_conf) - listenerBus.addListener(jobProgressListener) - _statusTracker = new SparkStatusTracker(this) _progressBar = diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 8a0327984e195..329fa06ba8ba5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -332,6 +332,47 @@ object SparkSubmit { } } + // In yarn mode for a python app, add pyspark archives to files + // that can be distributed with the job + if (args.isPython && clusterManager == YARN) { + var pyArchives: String = null + val pyArchivesEnvOpt = sys.env.get("PYSPARK_ARCHIVES_PATH") + if (pyArchivesEnvOpt.isDefined) { + pyArchives = pyArchivesEnvOpt.get + } else { + if (!sys.env.contains("SPARK_HOME")) { + printErrorAndExit("SPARK_HOME does not exist for python application in yarn mode.") + } + val pythonPath = new ArrayBuffer[String] + for (sparkHome <- sys.env.get("SPARK_HOME")) { + val pyLibPath = Seq(sparkHome, "python", "lib").mkString(File.separator) + val pyArchivesFile = new File(pyLibPath, "pyspark.zip") + if (!pyArchivesFile.exists()) { + printErrorAndExit("pyspark.zip does not exist for python application in yarn mode.") + } + val py4jFile = new File(pyLibPath, "py4j-0.8.2.1-src.zip") + if (!py4jFile.exists()) { + printErrorAndExit("py4j-0.8.2.1-src.zip does not exist for python application " + + "in yarn mode.") + } + pythonPath += pyArchivesFile.getAbsolutePath() + pythonPath += py4jFile.getAbsolutePath() + } + pyArchives = pythonPath.mkString(",") + } + + pyArchives = pyArchives.split(",").map { localPath=> + val localURI = Utils.resolveURI(localPath) + if (localURI.getScheme != "local") { + args.files = mergeFileLists(args.files, localURI.toString) + new Path(localPath).getName + } else { + localURI.getPath + } + }.mkString(File.pathSeparator) + sysProps("spark.submit.pyArchives") = pyArchives + } + // If we're running a R app, set the main class to our specific R runner if (args.isR && deployMode == CLIENT) { if (args.primaryResource == SPARKR_SHELL) { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 50522e69dc519..517cbe5176241 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -83,31 +83,27 @@ class HistoryServer( return } - val appKey = - if (parts.length == 3) { - s"${parts(1)}/${parts(2)}" - } else { - parts(1) + val appId = parts(1) + val attemptId = if (parts.length >= 3) Some(parts(2)) else None + + // Since we may have applications with multiple attempts mixed with applications with a + // single attempt, we need to try both. Try the single-attempt route first, and if an + // error is raised, then try the multiple attempt route. + if (!loadAppUi(appId, None) && (!attemptId.isDefined || !loadAppUi(appId, attemptId))) { + val msg =
Application {appId} not found.
+ res.setStatus(HttpServletResponse.SC_NOT_FOUND) + UIUtils.basicSparkPage(msg, "Not Found").foreach { n => + res.getWriter().write(n.toString) } + return + } // Note we don't use the UI retrieved from the cache; the cache loader above will register // the app's UI, and all we need to do is redirect the user to the same URI that was // requested, and the proper data should be served at that point. - try { - appCache.get(appKey) - res.sendRedirect(res.encodeRedirectURL(req.getRequestURI())) - } catch { - case e: Exception => e.getCause() match { - case nsee: NoSuchElementException => - val msg =
Application {appKey} not found.
- res.setStatus(HttpServletResponse.SC_NOT_FOUND) - UIUtils.basicSparkPage(msg, "Not Found").foreach( - n => res.getWriter().write(n.toString)) - - case cause: Exception => throw cause - } - } + res.sendRedirect(res.encodeRedirectURL(req.getRequestURI())) } + // SPARK-5983 ensure TRACE is not supported protected override def doTrace(req: HttpServletRequest, res: HttpServletResponse): Unit = { res.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED) @@ -183,6 +179,20 @@ class HistoryServer( */ def getProviderConfig(): Map[String, String] = provider.getConfig() + private def loadAppUi(appId: String, attemptId: Option[String]): Boolean = { + try { + appCache.get(appId + attemptId.map { id => s"/$id" }.getOrElse("")) + true + } catch { + case e: Exception => e.getCause() match { + case nsee: NoSuchElementException => + false + + case cause: Exception => throw cause + } + } + } + } /** @@ -203,8 +213,8 @@ object HistoryServer extends Logging { def main(argStrings: Array[String]) { SignalLogger.register(log) - initSecurity() new HistoryServerArguments(conf, argStrings) + initSecurity() val securityManager = new SecurityManager(conf) val providerName = conf.getOption("spark.history.provider") diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 53e1903a3d125..fccceb3ea528b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -165,7 +165,7 @@ private[master] class Master( (fsFactory.createPersistenceEngine(), fsFactory.createLeaderElectionAgent(this)) case "CUSTOM" => val clazz = Class.forName(conf.get("spark.deploy.recoveryMode.factory")) - val factory = clazz.getConstructor(conf.getClass, Serialization.getClass) + val factory = clazz.getConstructor(classOf[SparkConf], classOf[Serialization]) .newInstance(conf, SerializationExtension(context.system)) .asInstanceOf[StandaloneRecoveryModeFactory] (factory.createPersistenceEngine(), factory.createLeaderElectionAgent(this)) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala b/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala index 537b56b49f866..9440d456edf15 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala @@ -111,7 +111,7 @@ private[spark] object RDDOperationScope { private[spark] def withScope[T]( sc: SparkContext, name: String, - allowNesting: Boolean = false)(body: => T): T = { + allowNesting: Boolean)(body: => T): T = { // Save the old scope to restore it later val scopeKey = SparkContext.RDD_SCOPE_KEY val noOverrideKey = SparkContext.RDD_SCOPE_NO_OVERRIDE_KEY diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 06f0e2881c344..1067a7f1caf4c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -370,16 +370,21 @@ private[spark] class MesosClusterScheduler( val executorOpts = desc.schedulerProperties.map { case (k, v) => s"-D$k=$v" }.mkString(" ") envBuilder.addVariables( Variable.newBuilder().setName("SPARK_EXECUTOR_OPTS").setValue(executorOpts)) - val cmdOptions = generateCmdOption(desc) + val cmdOptions = generateCmdOption(desc).mkString(" ") + val dockerDefined = desc.schedulerProperties.contains("spark.mesos.executor.docker.image") val executorUri = desc.schedulerProperties.get("spark.executor.uri") .orElse(desc.command.environment.get("SPARK_EXECUTOR_URI")) val appArguments = desc.command.arguments.mkString(" ") - val cmd = if (executorUri.isDefined) { + val (executable, jar) = if (dockerDefined) { + // Application jar is automatically downloaded in the mounted sandbox by Mesos, + // and the path to the mounted volume is stored in $MESOS_SANDBOX env variable. + ("./bin/spark-submit", s"$$MESOS_SANDBOX/${desc.jarUrl.split("/").last}") + } else if (executorUri.isDefined) { builder.addUris(CommandInfo.URI.newBuilder().setValue(executorUri.get).build()) val folderBasename = executorUri.get.split('/').last.split('.').head val cmdExecutable = s"cd $folderBasename*; $prefixEnv bin/spark-submit" val cmdJar = s"../${desc.jarUrl.split("/").last}" - s"$cmdExecutable ${cmdOptions.mkString(" ")} $cmdJar $appArguments" + (cmdExecutable, cmdJar) } else { val executorSparkHome = desc.schedulerProperties.get("spark.mesos.executor.home") .orElse(conf.getOption("spark.home")) @@ -389,9 +394,9 @@ private[spark] class MesosClusterScheduler( } val cmdExecutable = new File(executorSparkHome, "./bin/spark-submit").getCanonicalPath val cmdJar = desc.jarUrl.split("/").last - s"$cmdExecutable ${cmdOptions.mkString(" ")} $cmdJar $appArguments" + (cmdExecutable, cmdJar) } - builder.setValue(cmd) + builder.setValue(s"$executable $cmdOptions $jar $appArguments") builder.setEnvironment(envBuilder.build()) builder.build() } @@ -458,9 +463,20 @@ private[spark] class MesosClusterScheduler( .setCommand(commandInfo) .addResources(cpuResource) .addResources(memResource) - .build() + submission.schedulerProperties.get("spark.mesos.executor.docker.image").foreach { image => + val container = taskInfo.getContainerBuilder() + val volumes = submission.schedulerProperties + .get("spark.mesos.executor.docker.volumes") + .map(MesosSchedulerBackendUtil.parseVolumesSpec) + val portmaps = submission.schedulerProperties + .get("spark.mesos.executor.docker.portmaps") + .map(MesosSchedulerBackendUtil.parsePortMappingsSpec) + MesosSchedulerBackendUtil.addDockerInfo( + container, image, volumes = volumes, portmaps = portmaps) + taskInfo.setContainer(container.build()) + } val queuedTasks = tasks.getOrElseUpdate(offer.offer.getId, new ArrayBuffer[TaskInfo]) - queuedTasks += taskInfo + queuedTasks += taskInfo.build() logTrace(s"Using offer ${offer.offer.getId.getValue} to launch driver " + submission.submissionId) val newState = new MesosClusterSubmissionState(submission, taskId, offer.offer.getSlaveId, diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index b7bc087855b9f..64ba27f34d2f1 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -32,6 +32,7 @@ import org.apache.spark._ import org.apache.spark.api.python.PythonBroadcast import org.apache.spark.broadcast.HttpBroadcast import org.apache.spark.network.nio.{GetBlock, GotBlock, PutBlock} +import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.{CompressedMapStatus, HighlyCompressedMapStatus} import org.apache.spark.storage._ import org.apache.spark.util.BoundedPriorityQueue @@ -51,18 +52,18 @@ class KryoSerializer(conf: SparkConf) private val bufferSizeKb = conf.getSizeAsKb("spark.kryoserializer.buffer", "64k") - if (bufferSizeKb >= 2048) { + if (bufferSizeKb >= ByteUnit.GiB.toKiB(2)) { throw new IllegalArgumentException("spark.kryoserializer.buffer must be less than " + - s"2048 mb, got: + $bufferSizeKb mb.") + s"2048 mb, got: + ${ByteUnit.KiB.toMiB(bufferSizeKb)} mb.") } - private val bufferSize = (bufferSizeKb * 1024).toInt + private val bufferSize = ByteUnit.KiB.toBytes(bufferSizeKb).toInt val maxBufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer.max", "64m").toInt - if (maxBufferSizeMb >= 2048) { + if (maxBufferSizeMb >= ByteUnit.GiB.toMiB(2)) { throw new IllegalArgumentException("spark.kryoserializer.buffer.max must be less than " + s"2048 mb, got: + $maxBufferSizeMb mb.") } - private val maxBufferSize = maxBufferSizeMb * 1024 * 1024 + private val maxBufferSize = ByteUnit.MiB.toBytes(maxBufferSizeMb).toInt private val referenceTracking = conf.getBoolean("spark.kryo.referenceTracking", true) private val registrationRequired = conf.getBoolean("spark.kryo.registrationRequired", false) @@ -125,6 +126,13 @@ class KryoSerializer(conf: SparkConf) override def newInstance(): SerializerInstance = { new KryoSerializerInstance(this) } + + private[spark] override lazy val supportsRelocationOfSerializedObjects: Boolean = { + // If auto-reset is disabled, then Kryo may store references to duplicate occurrences of objects + // in the stream rather than writing those objects' serialized bytes, breaking relocation. See + // https://groups.google.com/d/msg/kryo-users/6ZUSyfjjtdo/FhGG1KHDXPgJ for more details. + newInstance().asInstanceOf[KryoSerializerInstance].getAutoReset() + } } private[spark] diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index c381672a4f588..6078c9d433ebf 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -23,7 +23,7 @@ import java.nio.ByteBuffer import scala.reflect.ClassTag import org.apache.spark.{SparkConf, SparkEnv} -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Private} import org.apache.spark.util.{Utils, ByteBufferInputStream, NextIterator} /** @@ -63,6 +63,39 @@ abstract class Serializer { /** Creates a new [[SerializerInstance]]. */ def newInstance(): SerializerInstance + + /** + * :: Private :: + * Returns true if this serializer supports relocation of its serialized objects and false + * otherwise. This should return true if and only if reordering the bytes of serialized objects + * in serialization stream output is equivalent to having re-ordered those elements prior to + * serializing them. More specifically, the following should hold if a serializer supports + * relocation: + * + * {{{ + * serOut.open() + * position = 0 + * serOut.write(obj1) + * serOut.flush() + * position = # of bytes writen to stream so far + * obj1Bytes = output[0:position-1] + * serOut.write(obj2) + * serOut.flush() + * position2 = # of bytes written to stream so far + * obj2Bytes = output[position:position2-1] + * serIn.open([obj2bytes] concatenate [obj1bytes]) should return (obj2, obj1) + * }}} + * + * In general, this property should hold for serializers that are stateless and that do not + * write special metadata at the beginning or end of the serialization stream. + * + * This API is private to Spark; this method should not be overridden in third-party subclasses + * or called in user code and is subject to removal in future Spark releases. + * + * See SPARK-7311 for more details. + */ + @Private + private[spark] def supportsRelocationOfSerializedObjects: Boolean = false } diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala rename to core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala index e9b4e2b955dc8..6ad427bcac7f9 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -18,7 +18,6 @@ package org.apache.spark.shuffle import java.io.File -import java.nio.ByteBuffer import java.util.concurrent.ConcurrentLinkedQueue import java.util.concurrent.atomic.AtomicInteger @@ -29,7 +28,7 @@ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.serializer.Serializer -import org.apache.spark.shuffle.FileShuffleBlockManager.ShuffleFileGroup +import org.apache.spark.shuffle.FileShuffleBlockResolver.ShuffleFileGroup import org.apache.spark.storage._ import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} @@ -64,9 +63,8 @@ private[spark] trait ShuffleWriterGroup { * files within a ShuffleFileGroups associated with the block's reducer. */ // Note: Changes to the format in this file should be kept in sync with -// org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getHashBasedShuffleBlockData(). -private[spark] -class FileShuffleBlockManager(conf: SparkConf) +// org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getHashBasedShuffleBlockData(). +private[spark] class FileShuffleBlockResolver(conf: SparkConf) extends ShuffleBlockResolver with Logging { private val transportConf = SparkTransportConf.fromSparkConf(conf) @@ -242,8 +240,7 @@ class FileShuffleBlockManager(conf: SparkConf) } } -private[spark] -object FileShuffleBlockManager { +private[spark] object FileShuffleBlockResolver { /** * A group of shuffle files, one per reducer. * A particular mapper will be assigned a single ShuffleFileGroup to write its output to. diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala similarity index 93% rename from core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala rename to core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index a1741e2875c16..d9c63b6e7bbb9 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -18,7 +18,6 @@ package org.apache.spark.shuffle import java.io._ -import java.nio.ByteBuffer import com.google.common.io.ByteStreams @@ -28,7 +27,7 @@ import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.storage._ import org.apache.spark.util.Utils -import IndexShuffleBlockManager.NOOP_REDUCE_ID +import IndexShuffleBlockResolver.NOOP_REDUCE_ID /** * Create and maintain the shuffle blocks' mapping between logic block and physical file location. @@ -40,9 +39,8 @@ import IndexShuffleBlockManager.NOOP_REDUCE_ID * */ // Note: Changes to the format in this file should be kept in sync with -// org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getSortBasedShuffleBlockData(). -private[spark] -class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockResolver { +// org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getSortBasedShuffleBlockData(). +private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleBlockResolver { private lazy val blockManager = SparkEnv.get.blockManager @@ -115,7 +113,7 @@ class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockResolver { override def stop(): Unit = {} } -private[spark] object IndexShuffleBlockManager { +private[spark] object IndexShuffleBlockResolver { // No-op reduce ID used in interactions with disk store and BlockObjectWriter. // The disk store currently expects puts to relate to a (map, reduce) pair, but in the sort // shuffle outputs for several reduces are glommed into a single file. diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala index 2a7df8dd5bd83..c089088f409dd 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala @@ -26,7 +26,7 @@ import org.apache.spark.shuffle._ */ private[spark] class HashShuffleManager(conf: SparkConf) extends ShuffleManager { - private val fileShuffleBlockManager = new FileShuffleBlockManager(conf) + private val fileShuffleBlockResolver = new FileShuffleBlockResolver(conf) /* Register a shuffle with the manager and obtain a handle for it to pass to tasks. */ override def registerShuffle[K, V, C]( @@ -61,8 +61,8 @@ private[spark] class HashShuffleManager(conf: SparkConf) extends ShuffleManager shuffleBlockResolver.removeShuffle(shuffleId) } - override def shuffleBlockResolver: FileShuffleBlockManager = { - fileShuffleBlockManager + override def shuffleBlockResolver: FileShuffleBlockResolver = { + fileShuffleBlockResolver } /** Shut down this ShuffleManager. */ diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index cd27c9e07a3cd..897f0a5dc5bcc 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -25,7 +25,7 @@ import org.apache.spark.shuffle._ import org.apache.spark.storage.BlockObjectWriter private[spark] class HashShuffleWriter[K, V]( - shuffleBlockManager: FileShuffleBlockManager, + shuffleBlockResolver: FileShuffleBlockResolver, handle: BaseShuffleHandle[K, V, _], mapId: Int, context: TaskContext) @@ -45,7 +45,7 @@ private[spark] class HashShuffleWriter[K, V]( private val blockManager = SparkEnv.get.blockManager private val ser = Serializer.getSerializer(dep.serializer.getOrElse(null)) - private val shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, mapId, numOutputSplits, ser, + private val shuffle = shuffleBlockResolver.forMapTask(dep.shuffleId, mapId, numOutputSplits, ser, writeMetrics) /** Write a bunch of records to this task's output */ diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 0497036192154..15842941daaab 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -25,7 +25,7 @@ import org.apache.spark.shuffle.hash.HashShuffleReader private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager { - private val indexShuffleBlockManager = new IndexShuffleBlockManager(conf) + private val indexShuffleBlockResolver = new IndexShuffleBlockResolver(conf) private val shuffleMapNumber = new ConcurrentHashMap[Int, Int]() /** @@ -72,8 +72,8 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager true } - override def shuffleBlockResolver: IndexShuffleBlockManager = { - indexShuffleBlockManager + override def shuffleBlockResolver: IndexShuffleBlockResolver = { + indexShuffleBlockResolver } /** Shut down this ShuffleManager. */ diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index a066435df6fb0..add2656294ca2 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -20,12 +20,12 @@ package org.apache.spark.shuffle.sort import org.apache.spark.{MapOutputTracker, SparkEnv, Logging, TaskContext} import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.scheduler.MapStatus -import org.apache.spark.shuffle.{IndexShuffleBlockManager, ShuffleWriter, BaseShuffleHandle} +import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleWriter, BaseShuffleHandle} import org.apache.spark.storage.ShuffleBlockId import org.apache.spark.util.collection.ExternalSorter private[spark] class SortShuffleWriter[K, V, C]( - shuffleBlockManager: IndexShuffleBlockManager, + shuffleBlockResolver: IndexShuffleBlockResolver, handle: BaseShuffleHandle[K, V, C], mapId: Int, context: TaskContext) @@ -65,10 +65,10 @@ private[spark] class SortShuffleWriter[K, V, C]( // Don't bother including the time to open the merged output file in the shuffle write time, // because it just opens a single file, so is typically too fast to measure accurately // (see SPARK-3570). - val outputFile = shuffleBlockManager.getDataFile(dep.shuffleId, mapId) - val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockManager.NOOP_REDUCE_ID) + val outputFile = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId) + val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) val partitionLengths = sorter.writePartitionedFile(blockId, context, outputFile) - shuffleBlockManager.writeIndexFile(dep.shuffleId, mapId, partitionLengths) + shuffleBlockResolver.writeIndexFile(dep.shuffleId, mapId, partitionLengths) mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) } @@ -84,7 +84,7 @@ private[spark] class SortShuffleWriter[K, V, C]( return Option(mapStatus) } else { // The map task failed, so delete our output data. - shuffleBlockManager.removeDataByMap(dep.shuffleId, mapId) + shuffleBlockResolver.removeDataByMap(dep.shuffleId, mapId) return None } } finally { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index c186fd360fef6..524f6970992a5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -54,7 +54,7 @@ case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { } // Format of the shuffle block ids (including data and index) should be kept in sync with -// org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getBlockData(). +// org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getBlockData(). @DeveloperApi case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a46fecd2274ef..cc794e5c90ffa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -431,10 +431,11 @@ private[spark] class BlockManager( // As an optimization for map output fetches, if the block is for a shuffle, return it // without acquiring a lock; the disk store never deletes (recent) items so this should work if (blockId.isShuffle) { - val shuffleBlockManager = shuffleManager.shuffleBlockResolver + val shuffleBlockResolver = shuffleManager.shuffleBlockResolver // TODO: This should gracefully handle case where local block is not available. Currently // downstream code will throw an exception. - Option(shuffleBlockManager.getBlockData(blockId.asInstanceOf[ShuffleBlockId]).nioByteBuffer()) + Option( + shuffleBlockResolver.getBlockData(blockId.asInstanceOf[ShuffleBlockId]).nioByteBuffer()) } else { doGetLocal(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index 499dd97c0656a..8bc4e205bc3c6 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -59,7 +59,7 @@ private[spark] abstract class BlockObjectWriter(val blockId: BlockId) extends Ou def write(key: Any, value: Any) /** - * Notify the writer that a record worth of bytes has been written with writeBytes. + * Notify the writer that a record worth of bytes has been written with OutputStream#write. */ def recordWritten() @@ -215,12 +215,7 @@ private[spark] class DiskBlockObjectWriter( objOut.writeKey(key) objOut.writeValue(value) - numRecordsWritten += 1 - writeMetrics.incShuffleRecordsWritten(1) - - if (numRecordsWritten % 32 == 0) { - updateBytesWritten() - } + recordWritten() } override def write(b: Int): Unit = throw new UnsupportedOperationException() diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 5764c16902c66..2a4447705fa65 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -55,7 +55,7 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon /** Looks up a file by hashing it into one of our local subdirectories. */ // This method should be kept in sync with - // org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getFile(). + // org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getFile(). def getFile(filename: String): File = { // Figure out which local directory it hashes to, and which subdirectory in that val hash = Utils.nonNegativeHash(filename) diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala index 24f3236456248..063e2a1f8b18e 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -57,4 +57,23 @@ private[spark] object ToolTips { val GC_TIME = """Time that the executor spent paused for Java garbage collection while the task was running.""" + + val JOB_TIMELINE = + """Shows when jobs started and ended and when executors joined or left. Drag to scroll. + Click Enable Zooming and use mouse wheel to zoom in/out.""" + + val STAGE_TIMELINE = + """Shows when stages started and ended and when executors joined or left. Drag to scroll. + Click Enable Zooming and use mouse wheel to zoom in/out.""" + + val JOB_DAG = + """Shows a graph of stages executed for this job, each of which can contain + multiple RDD operations (e.g. map() and filter()), and of RDDs inside each operation + (shown as dots).""" + + val STAGE_DAG = + """Shows a graph of RDD operations in this stage, and RDDs inside each one. A stage can run + multiple operations (e.g. two map() functions) if they can be pipelined. Some operations + also create multiple RDDs internally. Cached RDDs are shown in green. + """ } 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 2f3fb181e4026..6a0f5c5d16daa 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui import java.text.SimpleDateFormat import java.util.{Locale, Date} -import scala.xml.{Node, Text} +import scala.xml.{Node, Text, Unparsed} import org.apache.spark.Logging import org.apache.spark.ui.scope.RDDOperationGraph @@ -156,13 +156,10 @@ private[spark] object UIUtils extends Logging { def commonHeaderNodes: Seq[Node] = { - - - - + + + + @@ -174,6 +171,7 @@ private[spark] object UIUtils extends Logging { } def vizHeaderNodes: Seq[Node] = { + @@ -252,7 +250,7 @@ private[spark] object UIUtils extends Logging {

- {org.apache.spark.SPARK_VERSION} {title} @@ -352,13 +350,16 @@ private[spark] object UIUtils extends Logging {
- DAG visualization + + DAG Visualization +
{ graphs.map { g => - } + + /** Return a script element that automatically expands the DAG visualization on page load. */ + def expandDagVizOnLoad(forJob: Boolean): Seq[Node] = { + + } + } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 09323d1d80ad6..e010ebef3b34a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -18,12 +18,12 @@ package org.apache.spark.ui.jobs import scala.collection.mutable.{HashMap, ListBuffer} -import scala.xml.{Node, NodeSeq, Unparsed} +import scala.xml.{Node, NodeSeq, Unparsed, Utility} import java.util.Date import javax.servlet.http.HttpServletRequest -import org.apache.spark.ui.{UIUtils, WebUIPage} +import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} import org.apache.spark.ui.jobs.UIData.{ExecutorUIData, JobUIData} import org.apache.spark.JobExecutionStatus @@ -81,6 +81,9 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { case JobExecutionStatus.RUNNING => "running" } + // The timeline library treats contents as HTML, so we have to escape them; for the + // data-title attribute string we have to escape them twice since that's in a string. + val escapedDesc = Utility.escape(displayJobDescription) val jobEventJsonAsStr = s""" |{ @@ -90,16 +93,17 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { | 'end': new Date(${completionTime}), | 'content': '
' + + | 'Status: ${status}
' + + | 'Submitted: ${UIUtils.formatDate(new Date(submissionTime))}' + | '${ if (status != JobExecutionStatus.RUNNING) { - s"""
Completion Time: ${UIUtils.formatDate(new Date(completionTime))}""" + s"""
Completed: ${UIUtils.formatDate(new Date(completionTime))}""" } else { "" } }">' + - | '${displayJobDescription} (Job ${jobId})
' + | '${escapedDesc} (Job ${jobId})
' |} """.stripMargin jobEventJsonAsStr @@ -179,13 +183,15 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { - Event timeline + + Event Timeline + ++ ++ @@ -283,7 +289,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { {if (parent.sc.isDefined) { // Total duration is not meaningful unless the UI is live
  • - Total Duration: + Total Uptime: {UIUtils.formatDuration(System.currentTimeMillis() - startTime)}
  • }} @@ -336,9 +342,8 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { failedJobsTable } - val helpText = """A job is triggered by an action, like "count()" or "saveAsTextFile()".""" + - " Click on a job's title to see information about the stages of tasks associated with" + - " the job." + val helpText = """A job is triggered by an action, like count() or saveAsTextFile().""" + + " Click on a job to see information about the stages of tasks inside it." UIUtils.headerSparkPage("Spark Jobs", content, parent, helpText = Some(helpText)) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala index a37f739ab9c66..5e52942b64f3f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala @@ -73,19 +73,6 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { val summary: NodeSeq =
      - { - if (sc.isDefined) { - // Total duration is not meaningful unless the UI is live -
    • - Total Duration: - {UIUtils.formatDuration(now - sc.get.startTime)} -
    • - } - } -
    • - Scheduling Mode: - {listener.schedulingMode.map(_.toString).getOrElse("Unknown")} -
    • { if (shouldShowActiveStages) {
    • @@ -145,7 +132,7 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { content ++=

      Failed Stages ({numFailedStages})

      ++ failedStagesTable.toNodeSeq } - UIUtils.headerSparkPage("Spark Stages (for all jobs)", content, parent) + UIUtils.headerSparkPage("Stages for All Jobs", content, parent) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 7163217e1fed0..2cad0a796913e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -20,13 +20,13 @@ package org.apache.spark.ui.jobs import java.util.Date import scala.collection.mutable.{Buffer, HashMap, ListBuffer} -import scala.xml.{NodeSeq, Node, Unparsed} +import scala.xml.{NodeSeq, Node, Unparsed, Utility} import javax.servlet.http.HttpServletRequest import org.apache.spark.JobExecutionStatus import org.apache.spark.scheduler.StageInfo -import org.apache.spark.ui.{UIUtils, WebUIPage} +import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} import org.apache.spark.ui.jobs.UIData.ExecutorUIData /** Page showing statistics and stage list for a given job */ @@ -64,6 +64,9 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val submissionTime = stage.submissionTime.get val completionTime = stage.completionTime.getOrElse(System.currentTimeMillis()) + // The timeline library treats contents as HTML, so we have to escape them; for the + // data-title attribute string we have to escape them twice since that's in a string. + val escapedName = Utility.escape(name) s""" |{ | 'className': 'stage job-timeline-object ${status}', @@ -72,17 +75,17 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { | 'end': new Date(${completionTime}), | 'content': '
      ' + | 'Status: ${status.toUpperCase}
      ' + - | 'Submission Time: ${UIUtils.formatDate(new Date(submissionTime))}' + + | 'Submitted: ${UIUtils.formatDate(new Date(submissionTime))}' + | '${ if (status != "running") { - s"""
      Completion Time: ${UIUtils.formatDate(new Date(completionTime))}""" + s"""
      Completed: ${UIUtils.formatDate(new Date(completionTime))}""" } else { "" } }">' + - | '${name} (Stage ${stageId}.${attemptId})
      ', + | '${escapedName} (Stage ${stageId}.${attemptId})
    ', |} """.stripMargin } @@ -161,13 +164,15 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { - Event timeline + + Event Timeline + ++ ++ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 89d175b06b947..8f7b1c2f09665 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -44,6 +44,10 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val parameterAttempt = request.getParameter("attempt") require(parameterAttempt != null && parameterAttempt.nonEmpty, "Missing attempt parameter") + // If this is set, expand the dag visualization by default + val expandDagVizParam = request.getParameter("expandDagViz") + val expandDagViz = expandDagVizParam != null && expandDagVizParam.toBoolean + val stageId = parameterId.toInt val stageAttemptId = parameterAttempt.toInt val stageDataOption = progressListener.stageIdToData.get((stageId, stageAttemptId)) @@ -77,7 +81,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
    • - Total task time across all tasks: + Total Time Across All Tasks: {UIUtils.formatDuration(stageData.executorRunTime)}
    • {if (stageData.hasInput) { @@ -94,25 +98,25 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { }} {if (stageData.hasShuffleRead) {
    • - Shuffle read: + Shuffle Read: {s"${Utils.bytesToString(stageData.shuffleReadTotalBytes)} / " + s"${stageData.shuffleReadRecords}"}
    • }} {if (stageData.hasShuffleWrite) {
    • - Shuffle write: + Shuffle Write: {s"${Utils.bytesToString(stageData.shuffleWriteBytes)} / " + s"${stageData.shuffleWriteRecords}"}
    • }} {if (stageData.hasBytesSpilled) {
    • - Shuffle spill (memory): + Shuffle Spill (Memory): {Utils.bytesToString(stageData.memoryBytesSpilled)}
    • - Shuffle spill (disk): + Shuffle Spill (Disk): {Utils.bytesToString(stageData.diskBytesSpilled)}
    • }} @@ -123,10 +127,10 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
      - Show additional metrics + Show Additional Metrics
      +## ElementwiseProduct + +ElementwiseProduct multiplies each input vector by a provided "weight" vector, using element-wise multiplication. In other words, it scales each column of the dataset by a scalar multiplier. This represents the [Hadamard product](https://en.wikipedia.org/wiki/Hadamard_product_%28matrices%29) between the input vector, `v` and transforming vector, `w`, to yield a result vector. + +`\[ \begin{pmatrix} +v_1 \\ +\vdots \\ +v_N +\end{pmatrix} \circ \begin{pmatrix} + w_1 \\ + \vdots \\ + w_N + \end{pmatrix} += \begin{pmatrix} + v_1 w_1 \\ + \vdots \\ + v_N w_N + \end{pmatrix} +\]` + +[`ElementwiseProduct`](api/scala/index.html#org.apache.spark.mllib.feature.ElementwiseProduct) has the following parameter in the constructor: + +* `w`: the transforming vector. + +`ElementwiseProduct` implements [`VectorTransformer`](api/scala/index.html#org.apache.spark.mllib.feature.VectorTransformer) which can apply the weighting on a `Vector` to produce a transformed `Vector` or on an `RDD[Vector]` to produce a transformed `RDD[Vector]`. + +### Example + +This example below demonstrates how to load a simple vectors file, extract a set of vectors, then transform those vectors using a transforming vector value. + + +
      +
      +{% highlight scala %} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.feature.ElementwiseProduct +import org.apache.spark.mllib.linalg.Vectors + +// Load and parse the data: +val data = sc.textFile("data/mllib/kmeans_data.txt") +val parsedData = data.map(s => Vectors.dense(s.split(' ').map(_.toDouble))) + +val transformingVector = Vectors.dense(0.0, 1.0, 2.0) +val transformer = new ElementwiseProduct(transformingVector) + +// Batch transform and per-row transform give the same results: +val transformedData = transformer.transform(parsedData) +val transformedData2 = parsedData.map(x => transformer.transform(x)) + +{% endhighlight %} +
      +
      + + diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index b8233ae06fdf3..df4c123bdd86c 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -139,7 +139,6 @@ DataFrames provide a domain-specific language for structured data manipulation i Here we include some basic examples of structured data processing using DataFrames: -
      {% highlight scala %} @@ -242,6 +241,12 @@ df.groupBy("age").count().show();
      +In Python it's possible to access a DataFrame's columns either by attribute +(`df.age`) or by indexing (`df['age']`). While the former is convenient for +interactive data exploration, users are highly encouraged to use the +latter form, which is future proof and won't break with column names that +are also attributes on the DataFrame class. + {% highlight python %} from pyspark.sql import SQLContext sqlContext = SQLContext(sc) @@ -270,14 +275,14 @@ df.select("name").show() ## Justin # Select everybody, but increment the age by 1 -df.select(df.name, df.age + 1).show() +df.select(df['name'], df['age'] + 1).show() ## name (age + 1) ## Michael null ## Andy 31 ## Justin 20 # Select people older than 21 -df.filter(df.age > 21).show() +df.filter(df['age'] > 21).show() ## age name ## 30 Andy diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index 387c0e421334b..f407367a54f6c 100644 --- a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala +++ b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -17,9 +17,9 @@ package org.apache.spark.examples.streaming -import java.util.Properties +import java.util.HashMap -import kafka.producer._ +import org.apache.kafka.clients.producer.{ProducerConfig, KafkaProducer, ProducerRecord} import org.apache.spark.streaming._ import org.apache.spark.streaming.kafka._ @@ -77,23 +77,25 @@ object KafkaWordCountProducer { val Array(brokers, topic, messagesPerSec, wordsPerMessage) = args // Zookeeper connection properties - val props = new Properties() - props.put("metadata.broker.list", brokers) - props.put("serializer.class", "kafka.serializer.StringEncoder") + val props = new HashMap[String, Object]() + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers) + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer") + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer") - val config = new ProducerConfig(props) - val producer = new Producer[String, String](config) + val producer = new KafkaProducer[String, String](props) // Send some messages while(true) { - val messages = (1 to messagesPerSec.toInt).map { messageNum => + (1 to messagesPerSec.toInt).foreach { messageNum => val str = (1 to wordsPerMessage.toInt).map(x => scala.util.Random.nextInt(10).toString) .mkString(" ") - new KeyedMessage[String, String](topic, str) - }.toArray + val message = new ProducerRecord[String, String](topic, null, str) + producer.send(message) + } - producer.send(messages: _*) Thread.sleep(100) } } diff --git a/examples/src/main/r/dataframe.R b/examples/src/main/r/dataframe.R new file mode 100644 index 0000000000000..53b817144f6ac --- /dev/null +++ b/examples/src/main/r/dataframe.R @@ -0,0 +1,54 @@ +# +# 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. +# + +library(SparkR) + +# Initialize SparkContext and SQLContext +sc <- sparkR.init(appName="SparkR-DataFrame-example") +sqlContext <- sparkRSQL.init(sc) + +# Create a simple local data.frame +localDF <- data.frame(name=c("John", "Smith", "Sarah"), age=c(19, 23, 18)) + +# Convert local data frame to a SparkR DataFrame +df <- createDataFrame(sqlContext, localDF) + +# Print its schema +printSchema(df) +# root +# |-- name: string (nullable = true) +# |-- age: double (nullable = true) + +# Create a DataFrame from a JSON file +path <- file.path(Sys.getenv("SPARK_HOME"), "examples/src/main/resources/people.json") +peopleDF <- jsonFile(sqlContext, path) +printSchema(peopleDF) + +# Register this DataFrame as a table. +registerTempTable(peopleDF, "people") + +# SQL statements can be run by using the sql methods provided by sqlContext +teenagers <- sql(sqlContext, "SELECT name FROM people WHERE age >= 13 AND age <= 19") + +# Call collect to get a local data.frame +teenagersLocalDF <- collect(teenagers) + +# Print the teenagers in our dataset +print(teenagersLocalDF) + +# Stop the SparkContext now +sparkR.stop() diff --git a/examples/src/main/r/kmeans.R b/examples/src/main/r/kmeans.R deleted file mode 100644 index 6e6b5cb93789c..0000000000000 --- a/examples/src/main/r/kmeans.R +++ /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. -# - -library(SparkR) - -# Logistic regression in Spark. -# Note: unlike the example in Scala, a point here is represented as a vector of -# doubles. - -parseVectors <- function(lines) { - lines <- strsplit(as.character(lines) , " ", fixed = TRUE) - list(matrix(as.numeric(unlist(lines)), ncol = length(lines[[1]]))) -} - -dist.fun <- function(P, C) { - apply( - C, - 1, - function(x) { - colSums((t(P) - x)^2) - } - ) -} - -closestPoint <- function(P, C) { - max.col(-dist.fun(P, C)) -} -# Main program - -args <- commandArgs(trailing = TRUE) - -if (length(args) != 3) { - print("Usage: kmeans ") - q("no") -} - -sc <- sparkR.init(appName = "RKMeans") -K <- as.integer(args[[2]]) -convergeDist <- as.double(args[[3]]) - -lines <- textFile(sc, args[[1]]) -points <- cache(lapplyPartition(lines, parseVectors)) -# kPoints <- take(points, K) -kPoints <- do.call(rbind, takeSample(points, FALSE, K, 16189L)) -tempDist <- 1.0 - -while (tempDist > convergeDist) { - closest <- lapplyPartition( - lapply(points, - function(p) { - cp <- closestPoint(p, kPoints); - mapply(list, unique(cp), split.data.frame(cbind(1, p), cp), SIMPLIFY=FALSE) - }), - function(x) {do.call(c, x) - }) - - pointStats <- reduceByKey(closest, - function(p1, p2) { - t(colSums(rbind(p1, p2))) - }, - 2L) - - newPoints <- do.call( - rbind, - collect(lapply(pointStats, - function(tup) { - point.sum <- tup[[2]][, -1] - point.count <- tup[[2]][, 1] - point.sum/point.count - }))) - - D <- dist.fun(kPoints, newPoints) - tempDist <- sum(D[cbind(1:3, max.col(-D))]) - kPoints <- newPoints - cat("Finished iteration (delta = ", tempDist, ")\n") -} - -cat("Final centers:\n") -writeLines(unlist(lapply(kPoints, paste, collapse = " "))) diff --git a/examples/src/main/r/linear_solver_mnist.R b/examples/src/main/r/linear_solver_mnist.R deleted file mode 100644 index c864a4232d010..0000000000000 --- a/examples/src/main/r/linear_solver_mnist.R +++ /dev/null @@ -1,107 +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. -# - -# Instructions: https://github.com/amplab-extras/SparkR-pkg/wiki/SparkR-Example:-Digit-Recognition-on-EC2 - -library(SparkR) -library(Matrix) - -args <- commandArgs(trailing = TRUE) - -# number of random features; default to 1100 -D <- ifelse(length(args) > 0, as.integer(args[[1]]), 1100) -# number of partitions for training dataset -trainParts <- 12 -# dimension of digits -d <- 784 -# number of test examples -NTrain <- 60000 -# number of training examples -NTest <- 10000 -# scale of features -gamma <- 4e-4 - -sc <- sparkR.init(appName = "SparkR-LinearSolver") - -# You can also use HDFS path to speed things up: -# hdfs:///train-mnist-dense-with-labels.data -file <- textFile(sc, "/data/train-mnist-dense-with-labels.data", trainParts) - -W <- gamma * matrix(nrow=D, ncol=d, data=rnorm(D*d)) -b <- 2 * pi * matrix(nrow=D, ncol=1, data=runif(D)) -broadcastW <- broadcast(sc, W) -broadcastB <- broadcast(sc, b) - -includePackage(sc, Matrix) -numericLines <- lapplyPartitionsWithIndex(file, - function(split, part) { - matList <- sapply(part, function(line) { - as.numeric(strsplit(line, ",", fixed=TRUE)[[1]]) - }, simplify=FALSE) - mat <- Matrix(ncol=d+1, data=unlist(matList, F, F), - sparse=T, byrow=T) - mat - }) - -featureLabels <- cache(lapplyPartition( - numericLines, - function(part) { - label <- part[,1] - mat <- part[,-1] - ones <- rep(1, nrow(mat)) - features <- cos( - mat %*% t(value(broadcastW)) + (matrix(ncol=1, data=ones) %*% t(value(broadcastB)))) - onesMat <- Matrix(ones) - featuresPlus <- cBind(features, onesMat) - labels <- matrix(nrow=nrow(mat), ncol=10, data=-1) - for (i in 1:nrow(mat)) { - labels[i, label[i]] <- 1 - } - list(label=labels, features=featuresPlus) - })) - -FTF <- Reduce("+", collect(lapplyPartition(featureLabels, - function(part) { - t(part$features) %*% part$features - }), flatten=F)) - -FTY <- Reduce("+", collect(lapplyPartition(featureLabels, - function(part) { - t(part$features) %*% part$label - }), flatten=F)) - -# solve for the coefficient matrix -C <- solve(FTF, FTY) - -test <- Matrix(as.matrix(read.csv("/data/test-mnist-dense-with-labels.data", - header=F), sparse=T)) -testData <- test[,-1] -testLabels <- matrix(ncol=1, test[,1]) - -err <- 0 - -# contstruct the feature maps for all examples from this digit -featuresTest <- cos(testData %*% t(value(broadcastW)) + - (matrix(ncol=1, data=rep(1, NTest)) %*% t(value(broadcastB)))) -featuresTest <- cBind(featuresTest, Matrix(rep(1, NTest))) - -# extract the one vs. all assignment -results <- featuresTest %*% C -labelsGot <- apply(results, 1, which.max) -err <- sum(testLabels != labelsGot) / nrow(testLabels) - -cat("\nFinished running. The error rate is: ", err, ".\n") diff --git a/examples/src/main/r/logistic_regression.R b/examples/src/main/r/logistic_regression.R deleted file mode 100644 index 2a86aa98160d3..0000000000000 --- a/examples/src/main/r/logistic_regression.R +++ /dev/null @@ -1,62 +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. -# - -library(SparkR) - -args <- commandArgs(trailing = TRUE) - -if (length(args) != 3) { - print("Usage: logistic_regression ") - q("no") -} - -# Initialize Spark context -sc <- sparkR.init(appName = "LogisticRegressionR") -iterations <- as.integer(args[[2]]) -D <- as.integer(args[[3]]) - -readPartition <- function(part){ - part = strsplit(part, " ", fixed = T) - list(matrix(as.numeric(unlist(part)), ncol = length(part[[1]]))) -} - -# Read data points and convert each partition to a matrix -points <- cache(lapplyPartition(textFile(sc, args[[1]]), readPartition)) - -# Initialize w to a random value -w <- runif(n=D, min = -1, max = 1) -cat("Initial w: ", w, "\n") - -# Compute logistic regression gradient for a matrix of data points -gradient <- function(partition) { - partition = partition[[1]] - Y <- partition[, 1] # point labels (first column of input file) - X <- partition[, -1] # point coordinates - - # For each point (x, y), compute gradient function - dot <- X %*% w - logit <- 1 / (1 + exp(-Y * dot)) - grad <- t(X) %*% ((logit - 1) * Y) - list(grad) -} - -for (i in 1:iterations) { - cat("On iteration ", i, "\n") - w <- w - reduce(lapplyPartition(points, gradient), "+") -} - -cat("Final w: ", w, "\n") diff --git a/examples/src/main/r/pi.R b/examples/src/main/r/pi.R deleted file mode 100644 index aa7a833e147a0..0000000000000 --- a/examples/src/main/r/pi.R +++ /dev/null @@ -1,46 +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. -# - -library(SparkR) - -args <- commandArgs(trailing = TRUE) - -sc <- sparkR.init(appName = "PiR") - -slices <- ifelse(length(args) > 1, as.integer(args[[2]]), 2) - -n <- 100000 * slices - -piFunc <- function(elem) { - rands <- runif(n = 2, min = -1, max = 1) - val <- ifelse((rands[1]^2 + rands[2]^2) < 1, 1.0, 0.0) - val -} - - -piFuncVec <- function(elems) { - message(length(elems)) - rands1 <- runif(n = length(elems), min = -1, max = 1) - rands2 <- runif(n = length(elems), min = -1, max = 1) - val <- ifelse((rands1^2 + rands2^2) < 1, 1.0, 0.0) - sum(val) -} - -rdd <- parallelize(sc, 1:n, slices) -count <- reduce(lapplyPartition(rdd, piFuncVec), sum) -cat("Pi is roughly", 4.0 * count / n, "\n") -cat("Num elements in RDD ", count(rdd), "\n") diff --git a/examples/src/main/r/wordcount.R b/examples/src/main/r/wordcount.R deleted file mode 100644 index b734cb0ecf55b..0000000000000 --- a/examples/src/main/r/wordcount.R +++ /dev/null @@ -1,42 +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. -# - -library(SparkR) - -args <- commandArgs(trailing = TRUE) - -if (length(args) != 1) { - print("Usage: wordcount ") - q("no") -} - -# Initialize Spark context -sc <- sparkR.init(appName = "RwordCount") -lines <- textFile(sc, args[[1]]) - -words <- flatMap(lines, - function(line) { - strsplit(line, " ")[[1]] - }) -wordCount <- lapply(words, function(word) { list(word, 1L) }) - -counts <- reduceByKey(wordCount, "+", 2L) -output <- collect(counts) - -for (wordcount in output) { - cat(wordcount[[1]], ": ", wordcount[[2]], "\n") -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala similarity index 86% rename from mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala rename to mllib/src/main/scala/org/apache/spark/ml/Predictor.scala index e8b3628140e99..0e53877de92db 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -15,29 +15,23 @@ * limitations under the License. */ -package org.apache.spark.ml.impl.estimator +package org.apache.spark.ml -import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} -import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} +import org.apache.spark.sql.{DataFrame, Row} /** - * :: DeveloperApi :: - * - * Trait for parameters for prediction (regression and classification). - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + * (private[ml]) Trait for parameters for prediction (regression and classification). */ -@DeveloperApi -private[spark] trait PredictorParams extends Params +private[ml] trait PredictorParams extends Params with HasLabelCol with HasFeaturesCol with HasPredictionCol { /** @@ -63,7 +57,7 @@ private[spark] trait PredictorParams extends Params } /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Abstraction for prediction problems (regression and classification). * @@ -73,11 +67,9 @@ private[spark] trait PredictorParams extends Params * parameter to specify the concrete type. * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type * parameter to specify the concrete type for the corresponding model. - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent -private[spark] abstract class Predictor[ +@DeveloperApi +abstract class Predictor[ FeaturesType, Learner <: Predictor[FeaturesType, Learner, M], M <: PredictionModel[FeaturesType, M]] @@ -104,8 +96,6 @@ private[spark] abstract class Predictor[ } /** - * :: DeveloperApi :: - * * Train a model using the given dataset and parameters. * Developers can implement this instead of [[fit()]] to avoid dealing with schema validation * and copying parameters into the model. @@ -113,12 +103,9 @@ private[spark] abstract class Predictor[ * @param dataset Training dataset * @return Fitted model */ - @DeveloperApi protected def train(dataset: DataFrame): M /** - * :: DeveloperApi :: - * * Returns the SQL DataType corresponding to the FeaturesType type parameter. * * This is used by [[validateAndTransformSchema()]]. @@ -126,7 +113,6 @@ private[spark] abstract class Predictor[ * * The default value is VectorUDT, but it may be overridden if FeaturesType is not Vector. */ - @DeveloperApi protected def featuresDataType: DataType = new VectorUDT override def transformSchema(schema: StructType): StructType = { @@ -146,7 +132,7 @@ private[spark] abstract class Predictor[ } /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Abstraction for a model for prediction tasks (regression and classification). * @@ -154,11 +140,9 @@ private[spark] abstract class Predictor[ * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type * parameter to specify the concrete type for the corresponding model. - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent -private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, M]] +@DeveloperApi +abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, M]] extends Model[M] with PredictorParams { /** @group setParam */ @@ -168,8 +152,6 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel def setPredictionCol(value: String): M = set(predictionCol, value).asInstanceOf[M] /** - * :: DeveloperApi :: - * * Returns the SQL DataType corresponding to the FeaturesType type parameter. * * This is used by [[validateAndTransformSchema()]]. @@ -177,7 +159,6 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel * * The default value is VectorUDT, but it may be overridden if FeaturesType is not Vector. */ - @DeveloperApi protected def featuresDataType: DataType = new VectorUDT override def transformSchema(schema: StructType): StructType = { @@ -192,12 +173,8 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel * @return transformed dataset with [[predictionCol]] of type [[Double]] */ override def transform(dataset: DataFrame): DataFrame = { - // This default implementation should be overridden as needed. - - // Check schema transformSchema(dataset.schema, logging = true) - - if ($(predictionCol) != "") { + if ($(predictionCol).nonEmpty) { dataset.withColumn($(predictionCol), callUDF(predict _, DoubleType, col($(featuresCol)))) } else { this.logWarning(s"$uid: Predictor.transform() was called as NOOP" + @@ -207,11 +184,8 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel } /** - * :: DeveloperApi :: - * * Predict label for the given features. * This internal method is used to implement [[transform()]] and output [[predictionCol]]. */ - @DeveloperApi protected def predict(features: FeaturesType): Double } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index d3361e24705c8..263d580fe2dd3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -17,8 +17,8 @@ package org.apache.spark.ml.classification -import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.{PredictionModel, PredictorParams, Predictor} import org.apache.spark.ml.param.shared.HasRawPredictionCol import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vector, VectorUDT} @@ -26,15 +26,12 @@ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} + /** - * :: DeveloperApi :: - * Params for classification. - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + * (private[spark]) Params for classification. */ -@DeveloperApi -private[spark] trait ClassifierParams extends PredictorParams - with HasRawPredictionCol { +private[spark] trait ClassifierParams + extends PredictorParams with HasRawPredictionCol { override protected def validateAndTransformSchema( schema: StructType, @@ -46,23 +43,21 @@ private[spark] trait ClassifierParams extends PredictorParams } /** - * :: AlphaComponent :: + * :: DeveloperApi :: + * * Single-label binary or multiclass classification. * Classes are indexed {0, 1, ..., numClasses - 1}. * * @tparam FeaturesType Type of input features. E.g., [[Vector]] * @tparam E Concrete Estimator type * @tparam M Concrete Model type - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent -private[spark] abstract class Classifier[ +@DeveloperApi +abstract class Classifier[ FeaturesType, E <: Classifier[FeaturesType, E, M], M <: ClassificationModel[FeaturesType, M]] - extends Predictor[FeaturesType, E, M] - with ClassifierParams { + extends Predictor[FeaturesType, E, M] with ClassifierParams { /** @group setParam */ def setRawPredictionCol(value: String): E = set(rawPredictionCol, value).asInstanceOf[E] @@ -71,17 +66,15 @@ private[spark] abstract class Classifier[ } /** - * :: AlphaComponent :: + * :: DeveloperApi :: + * * Model produced by a [[Classifier]]. * Classes are indexed {0, 1, ..., numClasses - 1}. * * @tparam FeaturesType Type of input features. E.g., [[Vector]] * @tparam M Concrete Model type - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent -private[spark] +@DeveloperApi abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[FeaturesType, M]] extends PredictionModel[FeaturesType, M] with ClassifierParams { @@ -101,13 +94,27 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur * @return transformed dataset */ override def transform(dataset: DataFrame): DataFrame = { - // This default implementation should be overridden as needed. - - // Check schema transformSchema(dataset.schema, logging = true) - val (numColsOutput, outputData) = - ClassificationModel.transformColumnsImpl[FeaturesType](dataset, this) + // Output selected columns only. + // This is a bit complicated since it tries to avoid repeated computation. + var outputData = dataset + var numColsOutput = 0 + if (getRawPredictionCol != "") { + outputData = outputData.withColumn(getRawPredictionCol, + callUDF(predictRaw _, new VectorUDT, col(getFeaturesCol))) + numColsOutput += 1 + } + if (getPredictionCol != "") { + val predUDF = if (getRawPredictionCol != "") { + callUDF(raw2prediction _, DoubleType, col(getRawPredictionCol)) + } else { + callUDF(predict _, DoubleType, col(getFeaturesCol)) + } + outputData = outputData.withColumn(getPredictionCol, predUDF) + numColsOutput += 1 + } + if (numColsOutput == 0) { logWarning(s"$uid: ClassificationModel.transform() was called as NOOP" + " since no output columns were set.") @@ -116,22 +123,17 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur } /** - * :: DeveloperApi :: - * * Predict label for the given features. * This internal method is used to implement [[transform()]] and output [[predictionCol]]. * * This default implementation for classification predicts the index of the maximum value * from [[predictRaw()]]. */ - @DeveloperApi override protected def predict(features: FeaturesType): Double = { - predictRaw(features).toArray.zipWithIndex.maxBy(_._1)._2 + raw2prediction(predictRaw(features)) } /** - * :: DeveloperApi :: - * * Raw prediction for each possible label. * The meaning of a "raw" prediction may vary between algorithms, but it intuitively gives * a measure of confidence in each possible label (where larger = more confident). @@ -141,48 +143,12 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur * This raw prediction may be any real number, where a larger value indicates greater * confidence for that label. */ - @DeveloperApi protected def predictRaw(features: FeaturesType): Vector -} - -private[ml] object ClassificationModel { /** - * Added prediction column(s). This is separated from [[ClassificationModel.transform()]] - * since it is used by [[org.apache.spark.ml.classification.ProbabilisticClassificationModel]]. - * @param dataset Input dataset - * @return (number of columns added, transformed dataset) + * Given a vector of raw predictions, select the predicted label. + * This may be overridden to support thresholds which favor particular labels. + * @return predicted label */ - def transformColumnsImpl[FeaturesType]( - dataset: DataFrame, - model: ClassificationModel[FeaturesType, _]): (Int, DataFrame) = { - - // Output selected columns only. - // This is a bit complicated since it tries to avoid repeated computation. - var tmpData = dataset - var numColsOutput = 0 - if (model.getRawPredictionCol != "") { - // output raw prediction - val features2raw: FeaturesType => Vector = model.predictRaw - tmpData = tmpData.withColumn(model.getRawPredictionCol, - callUDF(features2raw, new VectorUDT, col(model.getFeaturesCol))) - numColsOutput += 1 - if (model.getPredictionCol != "") { - val raw2pred: Vector => Double = (rawPred) => { - rawPred.toArray.zipWithIndex.maxBy(_._1)._2 - } - tmpData = tmpData.withColumn(model.getPredictionCol, - callUDF(raw2pred, DoubleType, col(model.getRawPredictionCol))) - numColsOutput += 1 - } - } else if (model.getPredictionCol != "") { - // output prediction - val features2pred: FeaturesType => Double = model.predict - tmpData = tmpData.withColumn(model.getPredictionCol, - callUDF(features2pred, DoubleType, col(model.getFeaturesCol))) - numColsOutput += 1 - } - (numColsOutput, tmpData) - } - + protected def raw2prediction(rawPrediction: Vector): Double = rawPrediction.toDense.argmax } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index 419e5ba05d38a..dcebea1d4b015 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -18,10 +18,9 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.tree.{DecisionTreeModel, Node} +import org.apache.spark.ml.tree.{TreeClassifierParams, DecisionTreeParams, DecisionTreeModel, Node} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index 534ea95b1c538..ae51b05a0c42d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -21,11 +21,10 @@ import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.{Param, ParamMap} import org.apache.spark.ml.regression.DecisionTreeRegressionModel -import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel} +import org.apache.spark.ml.tree.{GBTParams, TreeClassifierParams, DecisionTreeModel, TreeEnsembleModel} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index b73be035e29b5..550369d18cfec 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -21,9 +21,8 @@ import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS -import org.apache.spark.mllib.linalg.{BLAS, Vector, VectorUDT, Vectors} +import org.apache.spark.mllib.linalg._ import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.functions._ import org.apache.spark.storage.StorageLevel /** @@ -99,76 +98,17 @@ class LogisticRegressionModel private[ml] ( /** @group setParam */ def setThreshold(value: Double): this.type = set(threshold, value) + /** Margin (rawPrediction) for class label 1. For binary classification only. */ private val margin: Vector => Double = (features) => { BLAS.dot(features, weights) + intercept } + /** Score (probability) for class label 1. For binary classification only. */ private val score: Vector => Double = (features) => { val m = margin(features) 1.0 / (1.0 + math.exp(-m)) } - override def transform(dataset: DataFrame): DataFrame = { - // This is overridden (a) to be more efficient (avoiding re-computing values when creating - // multiple output columns) and (b) to handle threshold, which the abstractions do not use. - // TODO: We should abstract away the steps defined by UDFs below so that the abstractions - // can call whichever UDFs are needed to create the output columns. - - // Check schema - transformSchema(dataset.schema, logging = true) - - // Output selected columns only. - // This is a bit complicated since it tries to avoid repeated computation. - // rawPrediction (-margin, margin) - // probability (1.0-score, score) - // prediction (max margin) - var tmpData = dataset - var numColsOutput = 0 - if ($(rawPredictionCol) != "") { - val features2raw: Vector => Vector = (features) => predictRaw(features) - tmpData = tmpData.withColumn($(rawPredictionCol), - callUDF(features2raw, new VectorUDT, col($(featuresCol)))) - numColsOutput += 1 - } - if ($(probabilityCol) != "") { - if ($(rawPredictionCol) != "") { - val raw2prob = udf { (rawPreds: Vector) => - val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) - Vectors.dense(1.0 - prob1, prob1): Vector - } - tmpData = tmpData.withColumn($(probabilityCol), raw2prob(col($(rawPredictionCol)))) - } else { - val features2prob = udf { (features: Vector) => predictProbabilities(features) : Vector } - tmpData = tmpData.withColumn($(probabilityCol), features2prob(col($(featuresCol)))) - } - numColsOutput += 1 - } - if ($(predictionCol) != "") { - val t = $(threshold) - if ($(probabilityCol) != "") { - val predict = udf { probs: Vector => - if (probs(1) > t) 1.0 else 0.0 - } - tmpData = tmpData.withColumn($(predictionCol), predict(col($(probabilityCol)))) - } else if ($(rawPredictionCol) != "") { - val predict = udf { rawPreds: Vector => - val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) - if (prob1 > t) 1.0 else 0.0 - } - tmpData = tmpData.withColumn($(predictionCol), predict(col($(rawPredictionCol)))) - } else { - val predict = udf { features: Vector => this.predict(features) } - tmpData = tmpData.withColumn($(predictionCol), predict(col($(featuresCol)))) - } - numColsOutput += 1 - } - if (numColsOutput == 0) { - this.logWarning(s"$uid: LogisticRegressionModel.transform() was called as NOOP" + - " since no output columns were set.") - } - tmpData - } - override val numClasses: Int = 2 /** @@ -179,17 +119,43 @@ class LogisticRegressionModel private[ml] ( if (score(features) > getThreshold) 1 else 0 } - override protected def predictProbabilities(features: Vector): Vector = { - val s = score(features) - Vectors.dense(1.0 - s, s) + override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = { + rawPrediction match { + case dv: DenseVector => + var i = 0 + while (i < dv.size) { + dv.values(i) = 1.0 / (1.0 + math.exp(-dv.values(i))) + i += 1 + } + dv + case sv: SparseVector => + throw new RuntimeException("Unexpected error in LogisticRegressionModel:" + + " raw2probabilitiesInPlace encountered SparseVector") + } } override protected def predictRaw(features: Vector): Vector = { val m = margin(features) - Vectors.dense(0.0, m) + Vectors.dense(-m, m) } override def copy(extra: ParamMap): LogisticRegressionModel = { copyValues(new LogisticRegressionModel(parent, weights, intercept), extra) } + + override protected def raw2prediction(rawPrediction: Vector): Double = { + val t = getThreshold + val rawThreshold = if (t == 0.0) { + Double.NegativeInfinity + } else if (t == 1.0) { + Double.PositiveInfinity + } else { + Math.log(t / (1.0 - t)) + } + if (rawPrediction(1) > rawThreshold) 1 else 0 + } + + override protected def probability2prediction(probability: Vector): Double = { + if (probability(1) > getThreshold) 1 else 0 + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index 8519841c5c26c..330ae2938f4e0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -17,16 +17,16 @@ package org.apache.spark.ml.classification -import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.{DoubleType, DataType, StructType} /** - * Params for probabilistic classification. + * (private[classification]) Params for probabilistic classification. */ private[classification] trait ProbabilisticClassifierParams extends ClassifierParams with HasProbabilityCol { @@ -42,17 +42,15 @@ private[classification] trait ProbabilisticClassifierParams /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Single-label binary or multiclass classifier which can output class conditional probabilities. * * @tparam FeaturesType Type of input features. E.g., [[Vector]] * @tparam E Concrete Estimator type * @tparam M Concrete Model type - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent +@DeveloperApi private[spark] abstract class ProbabilisticClassifier[ FeaturesType, E <: ProbabilisticClassifier[FeaturesType, E, M], @@ -65,17 +63,15 @@ private[spark] abstract class ProbabilisticClassifier[ /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Model produced by a [[ProbabilisticClassifier]]. * Classes are indexed {0, 1, ..., numClasses - 1}. * * @tparam FeaturesType Type of input features. E.g., [[Vector]] * @tparam M Concrete Model type - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent +@DeveloperApi private[spark] abstract class ProbabilisticClassificationModel[ FeaturesType, M <: ProbabilisticClassificationModel[FeaturesType, M]] @@ -95,39 +91,79 @@ private[spark] abstract class ProbabilisticClassificationModel[ * @return transformed dataset */ override def transform(dataset: DataFrame): DataFrame = { - // This default implementation should be overridden as needed. - - // Check schema transformSchema(dataset.schema, logging = true) - val (numColsOutput, outputData) = - ClassificationModel.transformColumnsImpl[FeaturesType](dataset, this) - // Output selected columns only. - if ($(probabilityCol) != "") { - // output probabilities - outputData.withColumn($(probabilityCol), - callUDF(predictProbabilities _, new VectorUDT, col($(featuresCol)))) - } else { - if (numColsOutput == 0) { - this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() was called as NOOP" + - " since no output columns were set.") + // This is a bit complicated since it tries to avoid repeated computation. + var outputData = dataset + var numColsOutput = 0 + if ($(rawPredictionCol).nonEmpty) { + outputData = outputData.withColumn(getRawPredictionCol, + callUDF(predictRaw _, new VectorUDT, col(getFeaturesCol))) + numColsOutput += 1 + } + if ($(probabilityCol).nonEmpty) { + val probUDF = if ($(rawPredictionCol).nonEmpty) { + callUDF(raw2probability _, new VectorUDT, col($(rawPredictionCol))) + } else { + callUDF(predictProbability _, new VectorUDT, col($(featuresCol))) + } + outputData = outputData.withColumn($(probabilityCol), probUDF) + numColsOutput += 1 + } + if ($(predictionCol).nonEmpty) { + val predUDF = if ($(rawPredictionCol).nonEmpty) { + callUDF(raw2prediction _, DoubleType, col($(rawPredictionCol))) + } else if ($(probabilityCol).nonEmpty) { + callUDF(probability2prediction _, DoubleType, col($(probabilityCol))) + } else { + callUDF(predict _, DoubleType, col($(featuresCol))) } - outputData + outputData = outputData.withColumn($(predictionCol), predUDF) + numColsOutput += 1 + } + + if (numColsOutput == 0) { + this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() was called as NOOP" + + " since no output columns were set.") } + outputData } /** - * :: DeveloperApi :: + * Estimate the probability of each class given the raw prediction, + * doing the computation in-place. + * These predictions are also called class conditional probabilities. + * + * This internal method is used to implement [[transform()]] and output [[probabilityCol]]. * + * @return Estimated class conditional probabilities (modified input vector) + */ + protected def raw2probabilityInPlace(rawPrediction: Vector): Vector + + /** Non-in-place version of [[raw2probabilityInPlace()]] */ + protected def raw2probability(rawPrediction: Vector): Vector = { + val probs = rawPrediction.copy + raw2probabilityInPlace(probs) + } + + /** * Predict the probability of each class given the features. * These predictions are also called class conditional probabilities. * - * WARNING: Not all models output well-calibrated probability estimates! These probabilities - * should be treated as confidences, not precise probabilities. - * * This internal method is used to implement [[transform()]] and output [[probabilityCol]]. + * + * @return Estimated class conditional probabilities + */ + protected def predictProbability(features: FeaturesType): Vector = { + val rawPreds = predictRaw(features) + raw2probabilityInPlace(rawPreds) + } + + /** + * Given a vector of class conditional probabilities, select the predicted label. + * This may be overridden to support thresholds which favor particular labels. + * @return predicted label */ - @DeveloperApi - protected def predictProbabilities(features: FeaturesType): Vector + protected def probability2prediction(probability: Vector): Double = probability.toDense.argmax } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index 17f59bb42e129..9954893f14359 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -20,10 +20,9 @@ package org.apache.spark.ml.classification import scala.collection.mutable import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel} +import org.apache.spark.ml.tree.{RandomForestParams, TreeClassifierParams, DecisionTreeModel, TreeEnsembleModel} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala new file mode 100644 index 0000000000000..f8b56293e3ccc --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -0,0 +1,55 @@ +/* + * 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.ml.feature + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.param.Param +import org.apache.spark.mllib.feature +import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.sql.types.DataType + +/** + * :: AlphaComponent :: + * Outputs the Hadamard product (i.e., the element-wise product) of each input vector with a + * provided "weight" vector. In other words, it scales each column of the dataset by a scalar + * multiplier. + */ +@AlphaComponent +class ElementwiseProduct extends UnaryTransformer[Vector, Vector, ElementwiseProduct] { + + /** + * the vector to multiply with input vectors + * @group param + */ + val scalingVec: Param[Vector] = new Param(this, "scalingVector", "vector for hadamard product") + + /** @group setParam */ + def setScalingVec(value: Vector): this.type = set(scalingVec, value) + + /** @group getParam */ + def getScalingVec: Vector = getOrDefault(scalingVec) + + override protected def createTransformFunc: Vector => Vector = { + require(params.contains(scalingVec), s"transformation requires a weight vector") + val elemScaler = new feature.ElementwiseProduct($(scalingVec)) + elemScaler.transform + } + + override protected def outputDataType: DataType = new VectorUDT() +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala index 63e190c8aae53..9e6177ca27e4a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.types.DataType * which is available at [[http://en.wikipedia.org/wiki/Polynomial_expansion]], "In mathematics, an * expansion of a product of sums expresses it as a sum of products by using the fact that * multiplication distributes over addition". Take a 2-variable feature vector as an example: - * `(x, y)`, if we want to expand it with degree 2, then we get `(x, y, x * x, x * y, y * y)`. + * `(x, y)`, if we want to expand it with degree 2, then we get `(x, x * x, y, x * y, y * y)`. */ @AlphaComponent class PolynomialExpansion extends UnaryTransformer[Vector, Vector, PolynomialExpansion] { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala index 2863b7621526e..649c217b16590 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala @@ -42,7 +42,7 @@ class Tokenizer extends UnaryTransformer[String, Seq[String], Tokenizer] { /** * :: AlphaComponent :: - * A regex based tokenizer that extracts tokens either by repeatedly matching the regex(default) + * A regex based tokenizer that extracts tokens either by repeatedly matching the regex(default) * or using it to split the text (set matching to false). Optional parameters also allow filtering * tokens using a minimal length. * It returns an array of strings that can be empty. diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala index 8f2e62a8e2081..796758a70ef18 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.types._ /** * :: AlphaComponent :: - * A feature transformer than merge multiple columns into a vector column. + * A feature transformer that merges multiple columns into a vector column. */ @AlphaComponent class VectorAssembler extends Transformer with HasInputCols with HasOutputCol { @@ -102,6 +102,6 @@ object VectorAssembler { case o => throw new SparkException(s"$o of type ${o.getClass.getName} is not supported.") } - Vectors.sparse(cur, indices.result(), values.result()) + Vectors.sparse(cur, indices.result(), values.result()).compressed } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 51ce19d29cd29..0e1b60d172e3b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -22,6 +22,7 @@ import java.util.NoSuchElementException import scala.annotation.varargs import scala.collection.mutable +import scala.collection.JavaConverters._ import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.util.Identifiable @@ -218,6 +219,19 @@ class BooleanParam(parent: Params, name: String, doc: String) // No need for isV override def w(value: Boolean): ParamPair[Boolean] = super.w(value) } +/** Specialized version of [[Param[Array[T]]]] for Java. */ +class StringArrayParam(parent: Params, name: String, doc: String, isValid: Array[String] => Boolean) + extends Param[Array[String]](parent, name, doc, isValid) { + + def this(parent: Params, name: String, doc: String) = + this(parent, name, doc, ParamValidators.alwaysTrue) + + override def w(value: Array[String]): ParamPair[Array[String]] = super.w(value) + + /** Creates a param pair with a [[java.util.List]] of values (for Java and Python). */ + def w(value: java.util.List[String]): ParamPair[Array[String]] = w(value.asScala.toArray) +} + /** * A param amd its value. */ @@ -310,9 +324,7 @@ trait Params extends Identifiable with Serializable { * Sets a parameter in the embedded param map. */ protected final def set[T](param: Param[T], value: T): this.type = { - shouldOwn(param) - paramMap.put(param.asInstanceOf[Param[Any]], value) - this + set(param -> value) } /** @@ -322,6 +334,15 @@ trait Params extends Identifiable with Serializable { set(getParam(param), value) } + /** + * Sets a parameter in the embedded param map. + */ + protected final def set(paramPair: ParamPair[_]): this.type = { + shouldOwn(paramPair.param) + paramMap.put(paramPair) + this + } + /** * Optionally returns the user-supplied value of a param. */ @@ -366,13 +387,11 @@ trait Params extends Identifiable with Serializable { /** * Sets default values for a list of params. * - * Note: Java developers should use the single-parameter [[setDefault()]]. - * Annotating this with varargs causes compilation failures. - * * @param paramPairs a list of param pairs that specify params and their default values to set * respectively. Make sure that the params are initialized before this method * gets called. */ + @varargs protected final def setDefault(paramPairs: ParamPair[_]*): this.type = { paramPairs.foreach { p => setDefault(p.param.asInstanceOf[Param[Any]], p.value) diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala index d379172e0bf53..5085b798daa17 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -40,8 +40,10 @@ private[shared] object SharedParamsCodeGen { ParamDesc[String]("predictionCol", "prediction column name", Some("\"prediction\"")), ParamDesc[String]("rawPredictionCol", "raw prediction (a.k.a. confidence) column name", Some("\"rawPrediction\"")), - ParamDesc[String]("probabilityCol", - "column name for predicted class conditional probabilities", Some("\"probability\"")), + ParamDesc[String]("probabilityCol", "Column name for predicted class conditional" + + " probabilities. Note: Not all models output well-calibrated probability estimates!" + + " These probabilities should be treated as confidences, not precise probabilities.", + Some("\"probability\"")), ParamDesc[Double]("threshold", "threshold in binary classification prediction, in range [0, 1]", isValid = "ParamValidators.inRange(0, 1)"), @@ -83,6 +85,7 @@ private[shared] object SharedParamsCodeGen { case _ if c == classOf[Float] => "FloatParam" case _ if c == classOf[Double] => "DoubleParam" case _ if c == classOf[Boolean] => "BooleanParam" + case _ if c.isArray && c.getComponentType == classOf[String] => s"StringArrayParam" case _ => s"Param[${getTypeString(c)}]" } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala index fb1874ccfc8dc..7525d37007377 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -128,10 +128,10 @@ private[ml] trait HasRawPredictionCol extends Params { private[ml] trait HasProbabilityCol extends Params { /** - * Param for column name for predicted class conditional probabilities. + * Param for Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities.. * @group param */ - final val probabilityCol: Param[String] = new Param[String](this, "probabilityCol", "column name for predicted class conditional probabilities") + final val probabilityCol: Param[String] = new Param[String](this, "probabilityCol", "Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities.") setDefault(probabilityCol, "probability") @@ -178,7 +178,7 @@ private[ml] trait HasInputCols extends Params { * Param for input column names. * @group param */ - final val inputCols: Param[Array[String]] = new Param[Array[String]](this, "inputCols", "input column names") + final val inputCols: StringArrayParam = new StringArrayParam(this, "inputCols", "input column names") /** @group getParam */ final def getInputCols: Array[String] = $(inputCols) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index b07c26fe79b36..f8f0b161a4812 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -18,10 +18,9 @@ package org.apache.spark.ml.regression import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.tree.{DecisionTreeModel, Node} +import org.apache.spark.ml.tree.{TreeRegressorParams, DecisionTreeParams, DecisionTreeModel, Node} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index bc796958e4545..461905c12701a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -21,10 +21,9 @@ import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.{Param, ParamMap} -import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel} +import org.apache.spark.ml.tree.{GBTParams, TreeRegressorParams, DecisionTreeModel, TreeEnsembleModel} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 66c475f2d9840..e63c9a3eead52 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -25,6 +25,7 @@ import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS, import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared.{HasElasticNetParam, HasMaxIter, HasRegParam, HasTol} import org.apache.spark.mllib.linalg.{Vector, Vectors} @@ -39,7 +40,7 @@ import org.apache.spark.util.StatCounter /** * Params for linear regression. */ -private[regression] trait LinearRegressionParams extends RegressorParams +private[regression] trait LinearRegressionParams extends PredictorParams with HasRegParam with HasElasticNetParam with HasMaxIter with HasTol /** @@ -240,7 +241,7 @@ class LinearRegressionModel private[ml] ( * + \bar{y} / \hat{y}||^2 * = 1/2n ||\sum_i w_i^\prime x_i - y / \hat{y} + offset||^2 = 1/2n diff^2 * }}} - * where w_i^\prime is the effective weights defined by w_i/\hat{x_i}, offset is + * where w_i^\prime^ is the effective weights defined by w_i/\hat{x_i}, offset is * {{{ * - \sum_i (w_i/\hat{x_i})\bar{x_i} + \bar{y} / \hat{y}. * }}}, and diff is diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index 0468a1be1ba74..dbc628927433d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -18,10 +18,9 @@ package org.apache.spark.ml.regression import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree.{RandomForestParams, TreeRegressorParams} +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel} +import org.apache.spark.ml.tree.{RandomForestParams, TreeRegressorParams, DecisionTreeModel, TreeEnsembleModel} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala index c6b3327db6ad3..c72ef29680329 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala @@ -17,62 +17,40 @@ package org.apache.spark.ml.regression -import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.{PredictionModel, PredictorParams, Predictor} -/** - * :: DeveloperApi :: - * Params for regression. - * Currently empty, but may add functionality later. - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. - */ -@DeveloperApi -private[spark] trait RegressorParams extends PredictorParams /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Single-label regression * * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] * @tparam Learner Concrete Estimator type * @tparam M Concrete Model type - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent +@DeveloperApi private[spark] abstract class Regressor[ FeaturesType, Learner <: Regressor[FeaturesType, Learner, M], M <: RegressionModel[FeaturesType, M]] - extends Predictor[FeaturesType, Learner, M] - with RegressorParams { + extends Predictor[FeaturesType, Learner, M] with PredictorParams { // TODO: defaultEvaluator (follow-up PR) } /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Model produced by a [[Regressor]]. * * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] * @tparam M Concrete Model type. - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent -private[spark] abstract class RegressionModel[FeaturesType, M <: RegressionModel[FeaturesType, M]] - extends PredictionModel[FeaturesType, M] with RegressorParams { - - /** - * :: DeveloperApi :: - * - * Predict real-valued label for the given features. - * This internal method is used to implement [[transform()]] and output [[predictionCol]]. - */ - @DeveloperApi - protected def predict(features: FeaturesType): Double +@DeveloperApi +abstract class RegressionModel[FeaturesType, M <: RegressionModel[FeaturesType, M]] + extends PredictionModel[FeaturesType, M] with PredictorParams { + // TODO: defaultEvaluator (follow-up PR) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala similarity index 99% rename from mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala rename to mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala index 0e225627d4ee3..816fcedf2efb3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.spark.ml.impl.tree +package org.apache.spark.ml.tree import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.ml.impl.estimator.PredictorParams +import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasMaxIter, HasSeed} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, BoostingStrategy => OldBoostingStrategy, Strategy => OldStrategy} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index cee2aa6e85523..ac0d1fed84b2e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -52,10 +52,12 @@ private[ml] trait CrossValidatorParams extends Params { def getEstimatorParamMaps: Array[ParamMap] = $(estimatorParamMaps) /** - * param for the evaluator for selection + * param for the evaluator used to select hyper-parameters that maximize the cross-validated + * metric * @group param */ - val evaluator: Param[Evaluator] = new Param(this, "evaluator", "evaluator for selection") + val evaluator: Param[Evaluator] = new Param(this, "evaluator", + "evaluator used to select hyper-parameters that maximize the cross-validated metric") /** @group getParam */ def getEvaluator: Evaluator = $(evaluator) @@ -103,7 +105,7 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP override def fit(dataset: DataFrame): CrossValidatorModel = { val schema = dataset.schema - transformSchema(dataset.schema, logging = true) + transformSchema(schema, logging = true) val sqlCtx = dataset.sqlContext val est = $(estimator) val eval = $(evaluator) @@ -120,6 +122,7 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP trainingDataset.unpersist() var i = 0 while (i < numModels) { + // TODO: duplicate evaluator to take extra params from input val metric = eval.evaluate(models(i).transform(validationDataset, epm(i))) logDebug(s"Got metric $metric for model trained with ${epm(i)}.") metrics(i) += metric @@ -156,6 +159,7 @@ class CrossValidatorModel private[ml] ( } override def transform(dataset: DataFrame): DataFrame = { + transformSchema(dataset.schema, logging = true) bestModel.transform(dataset) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index b086cec083381..426306d78c1c3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -282,6 +282,24 @@ private[python] class PythonMLLibAPI extends Serializable { map(_.asInstanceOf[Object]).asJava } + /** + * Java stub for Python mllib IsotonicRegression.run() + */ + def trainIsotonicRegressionModel( + data: JavaRDD[Vector], + isotonic: Boolean): JList[Object] = { + val isotonicRegressionAlg = new IsotonicRegression().setIsotonic(isotonic) + val input = data.rdd.map { x => + (x(0), x(1), x(2)) + }.persist(StorageLevel.MEMORY_AND_DISK) + try { + val model = isotonicRegressionAlg.run(input) + List[AnyRef](model.boundaryVector, model.predictionVector).asJava + } finally { + data.rdd.unpersist(blocking = false) + } + } + /** * Java stub for Python mllib KMeans.run() */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index c8daa2388e868..a410547a72fda 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -18,8 +18,9 @@ package org.apache.spark.mllib.clustering import breeze.linalg.{DenseVector => BDV} + import org.apache.spark.Logging -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaPairRDD import org.apache.spark.graphx._ import org.apache.spark.mllib.linalg.Vector @@ -197,12 +198,20 @@ class LDA private ( } - /** LDAOptimizer used to perform the actual calculation */ + /** + * :: DeveloperApi :: + * + * LDAOptimizer used to perform the actual calculation + */ + @DeveloperApi def getOptimizer: LDAOptimizer = ldaOptimizer /** + * :: DeveloperApi :: + * * LDAOptimizer used to perform the actual calculation (default = EMLDAOptimizer) */ + @DeveloperApi def setOptimizer(optimizer: LDAOptimizer): this.type = { this.ldaOptimizer = optimizer this @@ -210,7 +219,7 @@ class LDA private ( /** * Set the LDAOptimizer used to perform the actual calculation by algorithm name. - * Currently "em", "online" is supported. + * Currently "em", "online" are supported. */ def setOptimizer(optimizerName: String): this.type = { this.ldaOptimizer = diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala index 093aa0f315ab2..6fa2fe053c6a4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala @@ -23,7 +23,7 @@ import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, sum, normalize, kr import breeze.numerics.{digamma, exp, abs} import breeze.stats.distributions.{Gamma, RandBasis} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.graphx._ import org.apache.spark.graphx.impl.GraphImpl import org.apache.spark.mllib.impl.PeriodicGraphCheckpointer @@ -31,13 +31,13 @@ import org.apache.spark.mllib.linalg.{Matrices, SparseVector, DenseVector, Vecto import org.apache.spark.rdd.RDD /** - * :: Experimental :: + * :: DeveloperApi :: * * An LDAOptimizer specifies which optimization/learning/inference algorithm to use, and it can * hold optimizer-specific parameters for users to set. */ -@Experimental -trait LDAOptimizer { +@DeveloperApi +sealed trait LDAOptimizer { /* DEVELOPERS NOTE: @@ -59,7 +59,7 @@ trait LDAOptimizer { } /** - * :: Experimental :: + * :: DeveloperApi :: * * Optimizer for EM algorithm which stores data + parameter graph, plus algorithm parameters. * @@ -75,8 +75,8 @@ trait LDAOptimizer { * "On Smoothing and Inference for Topic Models." UAI, 2009. * */ -@Experimental -class EMLDAOptimizer extends LDAOptimizer { +@DeveloperApi +final class EMLDAOptimizer extends LDAOptimizer { import LDA._ @@ -211,7 +211,7 @@ class EMLDAOptimizer extends LDAOptimizer { /** - * :: Experimental :: + * :: DeveloperApi :: * * An online optimizer for LDA. The Optimizer implements the Online variational Bayes LDA * algorithm, which processes a subset of the corpus on each iteration, and updates the term-topic @@ -220,8 +220,8 @@ class EMLDAOptimizer extends LDAOptimizer { * Original Online LDA paper: * Hoffman, Blei and Bach, "Online Learning for Latent Dirichlet Allocation." NIPS, 2010. */ -@Experimental -class OnlineLDAOptimizer extends LDAOptimizer { +@DeveloperApi +final class OnlineLDAOptimizer extends LDAOptimizer { // LDA common parameters private var k: Int = 0 @@ -243,8 +243,8 @@ class OnlineLDAOptimizer extends LDAOptimizer { private var randomGenerator: java.util.Random = null // Online LDA specific parameters - // Learning rate is: (tau_0 + t)^{-kappa} - private var tau_0: Double = 1024 + // Learning rate is: (tau0 + t)^{-kappa} + private var tau0: Double = 1024 private var kappa: Double = 0.51 private var miniBatchFraction: Double = 0.05 @@ -265,16 +265,16 @@ class OnlineLDAOptimizer extends LDAOptimizer { * A (positive) learning parameter that downweights early iterations. Larger values make early * iterations count less. */ - def getTau_0: Double = this.tau_0 + def getTau0: Double = this.tau0 /** * A (positive) learning parameter that downweights early iterations. Larger values make early * iterations count less. * Default: 1024, following the original Online LDA paper. */ - def setTau_0(tau_0: Double): this.type = { - require(tau_0 > 0, s"LDA tau_0 must be positive, but was set to $tau_0") - this.tau_0 = tau_0 + def setTau0(tau0: Double): this.type = { + require(tau0 > 0, s"LDA tau0 must be positive, but was set to $tau0") + this.tau0 = tau0 this } @@ -434,11 +434,8 @@ class OnlineLDAOptimizer extends LDAOptimizer { * Update lambda based on the batch submitted. batchSize can be different for each iteration. */ private[clustering] def update(stat: BDM[Double], iter: Int, batchSize: Int): Unit = { - val tau_0 = this.getTau_0 - val kappa = this.getKappa - // weight of the mini-batch. - val weight = math.pow(tau_0 + iter, -kappa) + val weight = math.pow(getTau0 + iter, -getKappa) // Update lambda based on documents. lambda = lambda * (1 - weight) + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala index 693117d820580..e577bf87f885e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala @@ -22,6 +22,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.Logging import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, MultivariateOnlineSummarizer} +import org.apache.spark.sql.DataFrame /** * :: Experimental :: @@ -32,6 +33,14 @@ import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Multivariate @Experimental class RegressionMetrics(predictionAndObservations: RDD[(Double, Double)]) extends Logging { + /** + * An auxiliary constructor taking a DataFrame. + * @param predictionAndObservations a DataFrame with two double columns: + * prediction and observation + */ + private[mllib] def this(predictionAndObservations: DataFrame) = + this(predictionAndObservations.map(r => (r.getDouble(0), r.getDouble(1)))) + /** * Use MultivariateOnlineSummarizer to calculate summary statistics of observations and errors. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala new file mode 100644 index 0000000000000..b0985baf9b278 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.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.mllib.feature + +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg._ + +/** + * :: Experimental :: + * Outputs the Hadamard product (i.e., the element-wise product) of each input vector with a + * provided "weight" vector. In other words, it scales each column of the dataset by a scalar + * multiplier. + * @param scalingVector The values used to scale the reference vector's individual components. + */ +@Experimental +class ElementwiseProduct(val scalingVector: Vector) extends VectorTransformer { + + /** + * Does the hadamard product transformation. + * + * @param vector vector to be transformed. + * @return transformed vector. + */ + override def transform(vector: Vector): Vector = { + require(vector.size == scalingVector.size, + s"vector sizes do not match: Expected ${scalingVector.size} but found ${vector.size}") + vector match { + case dv: DenseVector => + val values: Array[Double] = dv.values.clone() + val dim = scalingVector.size + var i = 0 + while (i < dim) { + values(i) *= scalingVector(i) + i += 1 + } + Vectors.dense(values) + case SparseVector(size, indices, vs) => + val values = vs.clone() + val dim = values.length + var i = 0 + while (i < dim) { + values(i) *= scalingVector(indices(i)) + i += 1 + } + Vectors.sparse(size, indices, values) + case v => throw new IllegalArgumentException("Does not support vector type " + v.getClass) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 188d1e542b5b5..f6bcdf83cd337 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -587,6 +587,28 @@ class DenseVector(val values: Array[Double]) extends Vector { } new SparseVector(size, ii, vv) } + + /** + * Find the index of a maximal element. Returns the first maximal element in case of a tie. + * Returns -1 if vector has length 0. + */ + private[spark] def argmax: Int = { + if (size == 0) { + -1 + } else { + var maxIdx = 0 + var maxValue = values(0) + var i = 1 + while (i < size) { + if (values(i) > maxValue) { + maxIdx = i + maxValue = values(i) + } + i += 1 + } + maxIdx + } + } } object DenseVector { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala index 5af55aaf84802..1b93e2d764c69 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala @@ -46,7 +46,7 @@ class MLPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) extends Se combOp = (queue1, queue2) => { queue1 ++= queue2 } - ).mapValues(_.toArray.reverse) // This is an min-heap, so we reverse the order. + ).mapValues(_.toArray.sorted(ord.reverse)) // This is an min-heap, so we reverse the order. } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala index 1d7617046b6c7..be2a00c2dfea4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala @@ -21,18 +21,20 @@ import java.io.Serializable import java.lang.{Double => JDouble} import java.util.Arrays.binarySearch +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ +import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} +import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.SQLContext /** * :: Experimental :: @@ -57,6 +59,13 @@ class IsotonicRegressionModel ( assertOrdered(boundaries) assertOrdered(predictions)(predictionOrd) + /** A Java-friendly constructor that takes two Iterable parameters and one Boolean parameter. */ + def this(boundaries: java.lang.Iterable[Double], + predictions: java.lang.Iterable[Double], + isotonic: java.lang.Boolean) = { + this(boundaries.asScala.toArray, predictions.asScala.toArray, isotonic) + } + /** Asserts the input array is monotone with the given ordering. */ private def assertOrdered(xs: Array[Double])(implicit ord: Ordering[Double]): Unit = { var i = 1 @@ -132,6 +141,12 @@ class IsotonicRegressionModel ( } } + /** A convenient method for boundaries called by the Python API. */ + private[mllib] def boundaryVector: Vector = Vectors.dense(boundaries) + + /** A convenient method for boundaries called by the Python API. */ + private[mllib] def predictionVector: Vector = Vectors.dense(predictions) + override def save(sc: SparkContext, path: String): Unit = { IsotonicRegressionModel.SaveLoadV1_0.save(sc, path, boundaries, predictions, isotonic) } diff --git a/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java b/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java index 8abe575610d19..532eca47918fc 100644 --- a/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java +++ b/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java @@ -59,5 +59,6 @@ public JavaTestParams() { ParamValidators.inArray(validStrings)); setDefault(myIntParam, 1); setDefault(myDoubleParam, 0.5); + setDefault(myIntParam.w(1), myDoubleParam.w(0.5)); } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java index f394d903966de..96c2da169961f 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java @@ -117,7 +117,7 @@ public void OnlineOptimizerCompatibility() { // Train a model OnlineLDAOptimizer op = new OnlineLDAOptimizer() - .setTau_0(1024) + .setTau0(1024) .setKappa(0.51) .setGammaShape(1e40) .setMiniBatchFraction(0.5); diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala index 57d0278e03639..0db27607bc274 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature import org.scalatest.FunSuite import org.apache.spark.SparkException -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{Row, SQLContext} @@ -48,6 +48,14 @@ class VectorAssemblerSuite extends FunSuite with MLlibTestSparkContext { } } + test("assemble should compress vectors") { + import org.apache.spark.ml.feature.VectorAssembler.assemble + val v1 = assemble(0.0, 0.0, 0.0, Vectors.dense(4.0)) + assert(v1.isInstanceOf[SparseVector]) + val v2 = assemble(1.0, 2.0, 3.0, Vectors.sparse(1, Array(0), Array(4.0))) + assert(v2.isInstanceOf[DenseVector]) + } + test("VectorAssembler") { val df = sqlContext.createDataFrame(Seq( (0, 0.0, Vectors.dense(1.0, 2.0), "a", Vectors.sparse(2, Array(1), Array(3.0)), 10L) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index 2dcc881f5abd2..d5b7d96335744 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -138,12 +138,12 @@ class LDASuite extends FunSuite with MLlibTestSparkContext { val lda = new LDA().setK(2) val corpus = sc.parallelize(tinyCorpus, 2) val op = new OnlineLDAOptimizer().initialize(corpus, lda) - op.setKappa(0.9876).setMiniBatchFraction(0.123).setTau_0(567) + op.setKappa(0.9876).setMiniBatchFraction(0.123).setTau0(567) assert(op.getAlpha == 0.5) // default 1.0 / k assert(op.getEta == 0.5) // default 1.0 / k assert(op.getKappa == 0.9876) assert(op.getMiniBatchFraction == 0.123) - assert(op.getTau_0 == 567) + assert(op.getTau0 == 567) } test("OnlineLDAOptimizer one iteration") { @@ -159,7 +159,7 @@ class LDASuite extends FunSuite with MLlibTestSparkContext { val corpus = sc.parallelize(docs, 2) // Set GammaShape large to avoid the stochastic impact. - val op = new OnlineLDAOptimizer().setTau_0(1024).setKappa(0.51).setGammaShape(1e40) + val op = new OnlineLDAOptimizer().setTau0(1024).setKappa(0.51).setGammaShape(1e40) .setMiniBatchFraction(1) val lda = new LDA().setK(k).setMaxIterations(1).setOptimizer(op).setSeed(12345) @@ -192,7 +192,7 @@ class LDASuite extends FunSuite with MLlibTestSparkContext { ).zipWithIndex.map { case (wordCounts, docId) => (docId.toLong, wordCounts) } val docs = sc.parallelize(toydata) - val op = new OnlineLDAOptimizer().setMiniBatchFraction(1).setTau_0(1024).setKappa(0.51) + val op = new OnlineLDAOptimizer().setMiniBatchFraction(1).setTau0(1024).setKappa(0.51) .setGammaShape(1e10) val lda = new LDA().setK(2) .setDocConcentration(0.01) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/ElementwiseProductSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/ElementwiseProductSuite.scala new file mode 100644 index 0000000000000..f3a482abda873 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/ElementwiseProductSuite.scala @@ -0,0 +1,61 @@ +/* + * 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.mllib.feature + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ + +class ElementwiseProductSuite extends FunSuite with MLlibTestSparkContext { + + test("elementwise (hadamard) product should properly apply vector to dense data set") { + val denseData = Array( + Vectors.dense(1.0, 4.0, 1.9, -9.0) + ) + val scalingVec = Vectors.dense(2.0, 0.5, 0.0, 0.25) + val transformer = new ElementwiseProduct(scalingVec) + val transformedData = transformer.transform(sc.makeRDD(denseData)) + val transformedVecs = transformedData.collect() + val transformedVec = transformedVecs(0) + val expectedVec = Vectors.dense(2.0, 2.0, 0.0, -2.25) + assert(transformedVec ~== expectedVec absTol 1E-5, + s"Expected transformed vector $expectedVec but found $transformedVec") + } + + test("elementwise (hadamard) product should properly apply vector to sparse data set") { + val sparseData = Array( + Vectors.sparse(3, Seq((1, -1.0), (2, -3.0))) + ) + val dataRDD = sc.parallelize(sparseData, 3) + val scalingVec = Vectors.dense(1.0, 0.0, 0.5) + val transformer = new ElementwiseProduct(scalingVec) + val data2 = sparseData.map(transformer.transform) + val data2RDD = transformer.transform(dataRDD) + + assert((sparseData, data2, data2RDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after hadamard product") + + assert((data2, data2RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + assert(data2(0) ~== Vectors.sparse(3, Seq((1, 0.0), (2, -1.5))) absTol 1E-5) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala index cb8fe4dba96f5..57216e8eb4a55 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala @@ -24,13 +24,14 @@ import org.apache.spark.mllib.rdd.MLPairRDDFunctions._ class MLPairRDDFunctionsSuite extends FunSuite with MLlibTestSparkContext { test("topByKey") { - val topMap = sc.parallelize(Array((1, 1), (1, 2), (3, 2), (3, 7), (5, 1), (3, 5)), 2) - .topByKey(2) + val topMap = sc.parallelize(Array((1, 7), (1, 3), (1, 6), (1, 1), (1, 2), (3, 2), (3, 7), (5, + 1), (3, 5)), 2) + .topByKey(5) .collectAsMap() assert(topMap.size === 3) - assert(topMap(1) === Array(2, 1)) - assert(topMap(3) === Array(7, 5)) + assert(topMap(1) === Array(7, 6, 3, 2, 1)) + assert(topMap(3) === Array(7, 5, 2)) assert(topMap(5) === Array(1)) } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index 46ca9708621b9..e4faaf8854fc7 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -46,18 +46,18 @@ public class ExternalShuffleBlockHandler extends RpcHandler { private final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockHandler.class); - private final ExternalShuffleBlockManager blockManager; + private final ExternalShuffleBlockResolver blockManager; private final OneForOneStreamManager streamManager; public ExternalShuffleBlockHandler(TransportConf conf) { - this(new OneForOneStreamManager(), new ExternalShuffleBlockManager(conf)); + this(new OneForOneStreamManager(), new ExternalShuffleBlockResolver(conf)); } /** Enables mocking out the StreamManager and BlockManager. */ @VisibleForTesting ExternalShuffleBlockHandler( OneForOneStreamManager streamManager, - ExternalShuffleBlockManager blockManager) { + ExternalShuffleBlockResolver blockManager) { this.streamManager = streamManager; this.blockManager = blockManager; } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java similarity index 95% rename from network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java rename to network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 93e6fdd7161fa..dd08e24cade23 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -44,13 +44,13 @@ * Manages converting shuffle BlockIds into physical segments of local files, from a process outside * of Executors. Each Executor must register its own configuration about where it stores its files * (local dirs) and how (shuffle manager). The logic for retrieval of individual files is replicated - * from Spark's FileShuffleBlockManager and IndexShuffleBlockManager. + * from Spark's FileShuffleBlockResolver and IndexShuffleBlockResolver. * * Executors with shuffle file consolidation are not currently supported, as the index is stored in - * the Executor's memory, unlike the IndexShuffleBlockManager. + * the Executor's memory, unlike the IndexShuffleBlockResolver. */ -public class ExternalShuffleBlockManager { - private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockManager.class); +public class ExternalShuffleBlockResolver { + private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockResolver.class); // Map containing all registered executors' metadata. private final ConcurrentMap executors; @@ -60,7 +60,7 @@ public class ExternalShuffleBlockManager { private final TransportConf conf; - public ExternalShuffleBlockManager(TransportConf conf) { + public ExternalShuffleBlockResolver(TransportConf conf) { this(conf, Executors.newSingleThreadExecutor( // Add `spark` prefix because it will run in NM in Yarn mode. NettyUtils.createThreadFactory("spark-shuffle-directory-cleaner"))); @@ -68,7 +68,7 @@ public ExternalShuffleBlockManager(TransportConf conf) { // Allows tests to have more control over when directories are cleaned up. @VisibleForTesting - ExternalShuffleBlockManager(TransportConf conf, Executor directoryCleaner) { + ExternalShuffleBlockResolver(TransportConf conf, Executor directoryCleaner) { this.conf = conf; this.executors = Maps.newConcurrentMap(); this.directoryCleaner = directoryCleaner; @@ -168,7 +168,7 @@ private void deleteExecutorDirs(String[] dirs) { /** * Hash-based shuffle data is simply stored as one file per block. - * This logic is from FileShuffleBlockManager. + * This logic is from FileShuffleBlockResolver. */ // TODO: Support consolidated hash shuffle files private ManagedBuffer getHashBasedShuffleBlockData(ExecutorShuffleInfo executor, String blockId) { @@ -178,7 +178,7 @@ private ManagedBuffer getHashBasedShuffleBlockData(ExecutorShuffleInfo executor, /** * Sort-based shuffle data uses an index called "shuffle_ShuffleId_MapId_0.index" into a data file - * called "shuffle_ShuffleId_MapId_0.data". This logic is from IndexShuffleBlockManager, + * called "shuffle_ShuffleId_MapId_0.data". This logic is from IndexShuffleBlockResolver, * and the block id format is from ShuffleDataBlockId and ShuffleIndexBlockId. */ private ManagedBuffer getSortBasedShuffleBlockData( diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java index 3f9fe1681cf27..73374cdc77a23 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -45,14 +45,14 @@ public class ExternalShuffleBlockHandlerSuite { TransportClient client = mock(TransportClient.class); OneForOneStreamManager streamManager; - ExternalShuffleBlockManager blockManager; + ExternalShuffleBlockResolver blockResolver; RpcHandler handler; @Before public void beforeEach() { streamManager = mock(OneForOneStreamManager.class); - blockManager = mock(ExternalShuffleBlockManager.class); - handler = new ExternalShuffleBlockHandler(streamManager, blockManager); + blockResolver = mock(ExternalShuffleBlockResolver.class); + handler = new ExternalShuffleBlockHandler(streamManager, blockResolver); } @Test @@ -62,7 +62,7 @@ public void testRegisterExecutor() { ExecutorShuffleInfo config = new ExecutorShuffleInfo(new String[] {"/a", "/b"}, 16, "sort"); byte[] registerMessage = new RegisterExecutor("app0", "exec1", config).toByteArray(); handler.receive(client, registerMessage, callback); - verify(blockManager, times(1)).registerExecutor("app0", "exec1", config); + verify(blockResolver, times(1)).registerExecutor("app0", "exec1", config); verify(callback, times(1)).onSuccess((byte[]) any()); verify(callback, never()).onFailure((Throwable) any()); @@ -75,12 +75,12 @@ public void testOpenShuffleBlocks() { ManagedBuffer block0Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[3])); ManagedBuffer block1Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[7])); - when(blockManager.getBlockData("app0", "exec1", "b0")).thenReturn(block0Marker); - when(blockManager.getBlockData("app0", "exec1", "b1")).thenReturn(block1Marker); + when(blockResolver.getBlockData("app0", "exec1", "b0")).thenReturn(block0Marker); + when(blockResolver.getBlockData("app0", "exec1", "b1")).thenReturn(block1Marker); byte[] openBlocks = new OpenBlocks("app0", "exec1", new String[] { "b0", "b1" }).toByteArray(); handler.receive(client, openBlocks, callback); - verify(blockManager, times(1)).getBlockData("app0", "exec1", "b0"); - verify(blockManager, times(1)).getBlockData("app0", "exec1", "b1"); + verify(blockResolver, times(1)).getBlockData("app0", "exec1", "b0"); + verify(blockResolver, times(1)).getBlockData("app0", "exec1", "b1"); ArgumentCaptor response = ArgumentCaptor.forClass(byte[].class); verify(callback, times(1)).onSuccess(response.capture()); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManagerSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java similarity index 77% rename from network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManagerSuite.java rename to network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index dad6428a836fc..d02f4f0fdb682 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManagerSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -30,7 +30,7 @@ import static org.junit.Assert.*; -public class ExternalShuffleBlockManagerSuite { +public class ExternalShuffleBlockResolverSuite { static String sortBlock0 = "Hello!"; static String sortBlock1 = "World!"; @@ -60,29 +60,29 @@ public static void afterAll() { @Test public void testBadRequests() { - ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(conf); + ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf); // Unregistered executor try { - manager.getBlockData("app0", "exec1", "shuffle_1_1_0"); + resolver.getBlockData("app0", "exec1", "shuffle_1_1_0"); fail("Should have failed"); } catch (RuntimeException e) { assertTrue("Bad error message: " + e, e.getMessage().contains("not registered")); } // Invalid shuffle manager - manager.registerExecutor("app0", "exec2", dataContext.createExecutorInfo("foobar")); + resolver.registerExecutor("app0", "exec2", dataContext.createExecutorInfo("foobar")); try { - manager.getBlockData("app0", "exec2", "shuffle_1_1_0"); + resolver.getBlockData("app0", "exec2", "shuffle_1_1_0"); fail("Should have failed"); } catch (UnsupportedOperationException e) { // pass } // Nonexistent shuffle block - manager.registerExecutor("app0", "exec3", + resolver.registerExecutor("app0", "exec3", dataContext.createExecutorInfo("org.apache.spark.shuffle.sort.SortShuffleManager")); try { - manager.getBlockData("app0", "exec3", "shuffle_1_1_0"); + resolver.getBlockData("app0", "exec3", "shuffle_1_1_0"); fail("Should have failed"); } catch (Exception e) { // pass @@ -91,18 +91,18 @@ public void testBadRequests() { @Test public void testSortShuffleBlocks() throws IOException { - ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(conf); - manager.registerExecutor("app0", "exec0", + ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf); + resolver.registerExecutor("app0", "exec0", dataContext.createExecutorInfo("org.apache.spark.shuffle.sort.SortShuffleManager")); InputStream block0Stream = - manager.getBlockData("app0", "exec0", "shuffle_0_0_0").createInputStream(); + resolver.getBlockData("app0", "exec0", "shuffle_0_0_0").createInputStream(); String block0 = CharStreams.toString(new InputStreamReader(block0Stream)); block0Stream.close(); assertEquals(sortBlock0, block0); InputStream block1Stream = - manager.getBlockData("app0", "exec0", "shuffle_0_0_1").createInputStream(); + resolver.getBlockData("app0", "exec0", "shuffle_0_0_1").createInputStream(); String block1 = CharStreams.toString(new InputStreamReader(block1Stream)); block1Stream.close(); assertEquals(sortBlock1, block1); @@ -110,18 +110,18 @@ public void testSortShuffleBlocks() throws IOException { @Test public void testHashShuffleBlocks() throws IOException { - ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(conf); - manager.registerExecutor("app0", "exec0", + ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf); + resolver.registerExecutor("app0", "exec0", dataContext.createExecutorInfo("org.apache.spark.shuffle.hash.HashShuffleManager")); InputStream block0Stream = - manager.getBlockData("app0", "exec0", "shuffle_1_0_0").createInputStream(); + resolver.getBlockData("app0", "exec0", "shuffle_1_0_0").createInputStream(); String block0 = CharStreams.toString(new InputStreamReader(block0Stream)); block0Stream.close(); assertEquals(hashBlock0, block0); InputStream block1Stream = - manager.getBlockData("app0", "exec0", "shuffle_1_0_1").createInputStream(); + resolver.getBlockData("app0", "exec0", "shuffle_1_0_1").createInputStream(); String block1 = CharStreams.toString(new InputStreamReader(block1Stream)); block1Stream.close(); assertEquals(hashBlock1, block1); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java index 254e3a7a32b98..d9d9c1bf2f17a 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java @@ -41,14 +41,15 @@ public class ExternalShuffleCleanupSuite { public void noCleanupAndCleanup() throws IOException { TestShuffleDataContext dataContext = createSomeData(); - ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(conf, sameThreadExecutor); - manager.registerExecutor("app", "exec0", dataContext.createExecutorInfo("shuffleMgr")); - manager.applicationRemoved("app", false /* cleanup */); + ExternalShuffleBlockResolver resolver = + new ExternalShuffleBlockResolver(conf, sameThreadExecutor); + resolver.registerExecutor("app", "exec0", dataContext.createExecutorInfo("shuffleMgr")); + resolver.applicationRemoved("app", false /* cleanup */); assertStillThere(dataContext); - manager.registerExecutor("app", "exec1", dataContext.createExecutorInfo("shuffleMgr")); - manager.applicationRemoved("app", true /* cleanup */); + resolver.registerExecutor("app", "exec1", dataContext.createExecutorInfo("shuffleMgr")); + resolver.applicationRemoved("app", true /* cleanup */); assertCleanedUp(dataContext); } @@ -64,7 +65,7 @@ public void cleanupUsesExecutor() throws IOException { @Override public void execute(Runnable runnable) { cleanupCalled.set(true); } }; - ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(conf, noThreadExecutor); + ExternalShuffleBlockResolver manager = new ExternalShuffleBlockResolver(conf, noThreadExecutor); manager.registerExecutor("app", "exec0", dataContext.createExecutorInfo("shuffleMgr")); manager.applicationRemoved("app", true); @@ -81,11 +82,12 @@ public void cleanupMultipleExecutors() throws IOException { TestShuffleDataContext dataContext0 = createSomeData(); TestShuffleDataContext dataContext1 = createSomeData(); - ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(conf, sameThreadExecutor); + ExternalShuffleBlockResolver resolver = + new ExternalShuffleBlockResolver(conf, sameThreadExecutor); - manager.registerExecutor("app", "exec0", dataContext0.createExecutorInfo("shuffleMgr")); - manager.registerExecutor("app", "exec1", dataContext1.createExecutorInfo("shuffleMgr")); - manager.applicationRemoved("app", true); + resolver.registerExecutor("app", "exec0", dataContext0.createExecutorInfo("shuffleMgr")); + resolver.registerExecutor("app", "exec1", dataContext1.createExecutorInfo("shuffleMgr")); + resolver.applicationRemoved("app", true); assertCleanedUp(dataContext0); assertCleanedUp(dataContext1); @@ -96,25 +98,26 @@ public void cleanupOnlyRemovedApp() throws IOException { TestShuffleDataContext dataContext0 = createSomeData(); TestShuffleDataContext dataContext1 = createSomeData(); - ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(conf, sameThreadExecutor); + ExternalShuffleBlockResolver resolver = + new ExternalShuffleBlockResolver(conf, sameThreadExecutor); - manager.registerExecutor("app-0", "exec0", dataContext0.createExecutorInfo("shuffleMgr")); - manager.registerExecutor("app-1", "exec0", dataContext1.createExecutorInfo("shuffleMgr")); + resolver.registerExecutor("app-0", "exec0", dataContext0.createExecutorInfo("shuffleMgr")); + resolver.registerExecutor("app-1", "exec0", dataContext1.createExecutorInfo("shuffleMgr")); - manager.applicationRemoved("app-nonexistent", true); + resolver.applicationRemoved("app-nonexistent", true); assertStillThere(dataContext0); assertStillThere(dataContext1); - manager.applicationRemoved("app-0", true); + resolver.applicationRemoved("app-0", true); assertCleanedUp(dataContext0); assertStillThere(dataContext1); - manager.applicationRemoved("app-1", true); + resolver.applicationRemoved("app-1", true); assertCleanedUp(dataContext0); assertCleanedUp(dataContext1); // Make sure it's not an error to cleanup multiple times - manager.applicationRemoved("app-1", true); + resolver.applicationRemoved("app-1", true); assertCleanedUp(dataContext0); assertCleanedUp(dataContext1); } diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java index 76639114df5d9..3fdde054ab6c7 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -29,7 +29,7 @@ /** * Manages some sort- and hash-based shuffle data, including the creation - * and cleanup of directories that can be read by the {@link ExternalShuffleBlockManager}. + * and cleanup of directories that can be read by the {@link ExternalShuffleBlockResolver}. */ public class TestShuffleDataContext { public final String[] localDirs; @@ -61,9 +61,9 @@ public void insertSortShuffleData(int shuffleId, int mapId, byte[][] blocks) thr String blockId = "shuffle_" + shuffleId + "_" + mapId + "_0"; OutputStream dataStream = new FileOutputStream( - ExternalShuffleBlockManager.getFile(localDirs, subDirsPerLocalDir, blockId + ".data")); + ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, blockId + ".data")); DataOutputStream indexStream = new DataOutputStream(new FileOutputStream( - ExternalShuffleBlockManager.getFile(localDirs, subDirsPerLocalDir, blockId + ".index"))); + ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, blockId + ".index"))); long offset = 0; indexStream.writeLong(offset); @@ -82,7 +82,7 @@ public void insertHashShuffleData(int shuffleId, int mapId, byte[][] blocks) thr for (int i = 0; i < blocks.length; i ++) { String blockId = "shuffle_" + shuffleId + "_" + mapId + "_" + i; Files.write(blocks[i], - ExternalShuffleBlockManager.getFile(localDirs, subDirsPerLocalDir, blockId)); + ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, blockId)); } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index bf343d4b7e40b..cfe387faec14b 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -89,6 +89,8 @@ object MimaExcludes { ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.linalg.Vector.numActives") ) ++ Seq( + // Execution should never be included as its always internal. + MimaBuild.excludeSparkPackage("sql.execution"), // This `protected[sql]` method was removed in 1.3.1 ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.sql.SQLContext.checkAnalysis"), diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b4431c7ee05b6..186345af0e60e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -193,6 +193,7 @@ object SparkBuild extends PomBuild { * Usage: `build/sbt sparkShell` */ val sparkShell = taskKey[Unit]("start a spark-shell.") + val sparkSql = taskKey[Unit]("starts the spark sql CLI.") enable(Seq( connectInput in run := true, @@ -203,6 +204,12 @@ object SparkBuild extends PomBuild { sparkShell := { (runMain in Compile).toTask(" org.apache.spark.repl.Main -usejavacp").value + }, + + javaOptions in Compile += "-Dspark.master=local", + + sparkSql := { + (runMain in Compile).toTask(" org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver").value } ))(assembly) @@ -363,6 +370,7 @@ object Assembly { object PySparkAssembly { import sbtassembly.Plugin._ import AssemblyKeys._ + import java.util.zip.{ZipOutputStream, ZipEntry} lazy val settings = Seq( unmanagedJars in Compile += { BuildCommons.sparkHome / "python/lib/py4j-0.8.2.1-src.zip" }, @@ -370,16 +378,48 @@ object PySparkAssembly { // to be included in the assembly. We can't just add "python/" to the assembly's resource dir // list since that will copy unneeded / unwanted files. resourceGenerators in Compile <+= resourceManaged in Compile map { outDir: File => + val src = new File(BuildCommons.sparkHome, "python/pyspark") + + val zipFile = new File(BuildCommons.sparkHome , "python/lib/pyspark.zip") + zipFile.delete() + zipRecursive(src, zipFile) + val dst = new File(outDir, "pyspark") if (!dst.isDirectory()) { require(dst.mkdirs()) } - - val src = new File(BuildCommons.sparkHome, "python/pyspark") copy(src, dst) } ) + private def zipRecursive(source: File, destZipFile: File) = { + val destOutput = new ZipOutputStream(new FileOutputStream(destZipFile)) + addFilesToZipStream("", source, destOutput) + destOutput.flush() + destOutput.close() + } + + private def addFilesToZipStream(parent: String, source: File, output: ZipOutputStream): Unit = { + if (source.isDirectory()) { + output.putNextEntry(new ZipEntry(parent + source.getName())) + for (file <- source.listFiles()) { + addFilesToZipStream(parent + source.getName() + File.separator, file, output) + } + } else { + val in = new FileInputStream(source) + output.putNextEntry(new ZipEntry(parent + source.getName())) + val buf = new Array[Byte](8192) + var n = 0 + while (n != -1) { + n = in.read(buf) + if (n != -1) { + output.write(buf, 0, n) + } + } + in.close() + } + } + private def copy(src: File, dst: File): Seq[File] = { src.listFiles().flatMap { f => val child = new File(dst, f.getName()) @@ -497,7 +537,7 @@ object TestSettings { // Setting SPARK_DIST_CLASSPATH is a simple way to make sure any child processes // launched by the tests have access to the correct test-time classpath. envVars in Test ++= Map( - "SPARK_DIST_CLASSPATH" -> + "SPARK_DIST_CLASSPATH" -> (fullClasspath in Test).value.files.map(_.getAbsolutePath).mkString(":").stripSuffix(":"), "JAVA_HOME" -> sys.env.get("JAVA_HOME").getOrElse(sys.props("java.home"))), javaOptions in Test += "-Dspark.test.home=" + sparkHome, diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 4e4614b859ac6..f35bc1463d51b 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -16,12 +16,571 @@ # from pyspark.rdd import ignore_unicode_prefix -from pyspark.ml.param.shared import HasInputCol, HasOutputCol, HasNumFeatures +from pyspark.ml.param.shared import * from pyspark.ml.util import keyword_only -from pyspark.ml.wrapper import JavaTransformer +from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaTransformer from pyspark.mllib.common import inherit_doc -__all__ = ['Tokenizer', 'HashingTF'] +__all__ = ['Binarizer', 'HashingTF', 'IDF', 'IDFModel', 'Normalizer', 'OneHotEncoder', + 'PolynomialExpansion', 'RegexTokenizer', 'StandardScaler', 'StandardScalerModel', + 'StringIndexer', 'StringIndexerModel', 'Tokenizer', 'VectorAssembler', 'VectorIndexer', + 'Word2Vec', 'Word2VecModel'] + + +@inherit_doc +class Binarizer(JavaTransformer, HasInputCol, HasOutputCol): + """ + Binarize a column of continuous features given a threshold. + + >>> df = sqlContext.createDataFrame([(0.5,)], ["values"]) + >>> binarizer = Binarizer(threshold=1.0, inputCol="values", outputCol="features") + >>> binarizer.transform(df).head().features + 0.0 + >>> binarizer.setParams(outputCol="freqs").transform(df).head().freqs + 0.0 + >>> params = {binarizer.threshold: -0.5, binarizer.outputCol: "vector"} + >>> binarizer.transform(df, params).head().vector + 1.0 + """ + + _java_class = "org.apache.spark.ml.feature.Binarizer" + # a placeholder to make it appear in the generated doc + threshold = Param(Params._dummy(), "threshold", + "threshold in binary classification prediction, in range [0, 1]") + + @keyword_only + def __init__(self, threshold=0.0, inputCol=None, outputCol=None): + """ + __init__(self, threshold=0.0, inputCol=None, outputCol=None) + """ + super(Binarizer, self).__init__() + self.threshold = Param(self, "threshold", + "threshold in binary classification prediction, in range [0, 1]") + self._setDefault(threshold=0.0) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, threshold=0.0, inputCol=None, outputCol=None): + """ + setParams(self, threshold=0.0, inputCol=None, outputCol=None) + Sets params for this Binarizer. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setThreshold(self, value): + """ + Sets the value of :py:attr:`threshold`. + """ + self.paramMap[self.threshold] = value + return self + + def getThreshold(self): + """ + Gets the value of threshold or its default value. + """ + return self.getOrDefault(self.threshold) + + +@inherit_doc +class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): + """ + Maps a sequence of terms to their term frequencies using the + hashing trick. + + >>> df = sqlContext.createDataFrame([(["a", "b", "c"],)], ["words"]) + >>> hashingTF = HashingTF(numFeatures=10, inputCol="words", outputCol="features") + >>> hashingTF.transform(df).head().features + SparseVector(10, {7: 1.0, 8: 1.0, 9: 1.0}) + >>> hashingTF.setParams(outputCol="freqs").transform(df).head().freqs + SparseVector(10, {7: 1.0, 8: 1.0, 9: 1.0}) + >>> params = {hashingTF.numFeatures: 5, hashingTF.outputCol: "vector"} + >>> hashingTF.transform(df, params).head().vector + SparseVector(5, {2: 1.0, 3: 1.0, 4: 1.0}) + """ + + _java_class = "org.apache.spark.ml.feature.HashingTF" + + @keyword_only + def __init__(self, numFeatures=1 << 18, inputCol=None, outputCol=None): + """ + __init__(self, numFeatures=1 << 18, inputCol=None, outputCol=None) + """ + super(HashingTF, self).__init__() + self._setDefault(numFeatures=1 << 18) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, numFeatures=1 << 18, inputCol=None, outputCol=None): + """ + setParams(self, numFeatures=1 << 18, inputCol=None, outputCol=None) + Sets params for this HashingTF. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + +@inherit_doc +class IDF(JavaEstimator, HasInputCol, HasOutputCol): + """ + Compute the Inverse Document Frequency (IDF) given a collection of documents. + + >>> from pyspark.mllib.linalg import DenseVector + >>> df = sqlContext.createDataFrame([(DenseVector([1.0, 2.0]),), + ... (DenseVector([0.0, 1.0]),), (DenseVector([3.0, 0.2]),)], ["tf"]) + >>> idf = IDF(minDocFreq=3, inputCol="tf", outputCol="idf") + >>> idf.fit(df).transform(df).head().idf + DenseVector([0.0, 0.0]) + >>> idf.setParams(outputCol="freqs").fit(df).transform(df).collect()[1].freqs + DenseVector([0.0, 0.0]) + >>> params = {idf.minDocFreq: 1, idf.outputCol: "vector"} + >>> idf.fit(df, params).transform(df).head().vector + DenseVector([0.2877, 0.0]) + """ + + _java_class = "org.apache.spark.ml.feature.IDF" + + # a placeholder to make it appear in the generated doc + minDocFreq = Param(Params._dummy(), "minDocFreq", + "minimum of documents in which a term should appear for filtering") + + @keyword_only + def __init__(self, minDocFreq=0, inputCol=None, outputCol=None): + """ + __init__(self, minDocFreq=0, inputCol=None, outputCol=None) + """ + super(IDF, self).__init__() + self.minDocFreq = Param(self, "minDocFreq", + "minimum of documents in which a term should appear for filtering") + self._setDefault(minDocFreq=0) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, minDocFreq=0, inputCol=None, outputCol=None): + """ + setParams(self, minDocFreq=0, inputCol=None, outputCol=None) + Sets params for this IDF. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setMinDocFreq(self, value): + """ + Sets the value of :py:attr:`minDocFreq`. + """ + self.paramMap[self.minDocFreq] = value + return self + + def getMinDocFreq(self): + """ + Gets the value of minDocFreq or its default value. + """ + return self.getOrDefault(self.minDocFreq) + + +class IDFModel(JavaModel): + """ + Model fitted by IDF. + """ + + +@inherit_doc +class Normalizer(JavaTransformer, HasInputCol, HasOutputCol): + """ + Normalize a vector to have unit norm using the given p-norm. + + >>> from pyspark.mllib.linalg import Vectors + >>> svec = Vectors.sparse(4, {1: 4.0, 3: 3.0}) + >>> df = sqlContext.createDataFrame([(Vectors.dense([3.0, -4.0]), svec)], ["dense", "sparse"]) + >>> normalizer = Normalizer(p=2.0, inputCol="dense", outputCol="features") + >>> normalizer.transform(df).head().features + DenseVector([0.6, -0.8]) + >>> normalizer.setParams(inputCol="sparse", outputCol="freqs").transform(df).head().freqs + SparseVector(4, {1: 0.8, 3: 0.6}) + >>> params = {normalizer.p: 1.0, normalizer.inputCol: "dense", normalizer.outputCol: "vector"} + >>> normalizer.transform(df, params).head().vector + DenseVector([0.4286, -0.5714]) + """ + + # a placeholder to make it appear in the generated doc + p = Param(Params._dummy(), "p", "the p norm value.") + + _java_class = "org.apache.spark.ml.feature.Normalizer" + + @keyword_only + def __init__(self, p=2.0, inputCol=None, outputCol=None): + """ + __init__(self, p=2.0, inputCol=None, outputCol=None) + """ + super(Normalizer, self).__init__() + self.p = Param(self, "p", "the p norm value.") + self._setDefault(p=2.0) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, p=2.0, inputCol=None, outputCol=None): + """ + setParams(self, p=2.0, inputCol=None, outputCol=None) + Sets params for this Normalizer. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setP(self, value): + """ + Sets the value of :py:attr:`p`. + """ + self.paramMap[self.p] = value + return self + + def getP(self): + """ + Gets the value of p or its default value. + """ + return self.getOrDefault(self.p) + + +@inherit_doc +class OneHotEncoder(JavaTransformer, HasInputCol, HasOutputCol): + """ + A one-hot encoder that maps a column of label indices to a column of binary vectors, with + at most a single one-value. By default, the binary vector has an element for each category, so + with 5 categories, an input value of 2.0 would map to an output vector of + (0.0, 0.0, 1.0, 0.0, 0.0). If includeFirst is set to false, the first category is omitted, so + the output vector for the previous example would be (0.0, 1.0, 0.0, 0.0) and an input value + of 0.0 would map to a vector of all zeros. Including the first category makes the vector columns + linearly dependent because they sum up to one. + + TODO: This method requires the use of StringIndexer first. Decouple them. + + >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed") + >>> model = stringIndexer.fit(stringIndDf) + >>> td = model.transform(stringIndDf) + >>> encoder = OneHotEncoder(includeFirst=False, inputCol="indexed", outputCol="features") + >>> encoder.transform(td).head().features + SparseVector(2, {}) + >>> encoder.setParams(outputCol="freqs").transform(td).head().freqs + SparseVector(2, {}) + >>> params = {encoder.includeFirst: True, encoder.outputCol: "test"} + >>> encoder.transform(td, params).head().test + SparseVector(3, {0: 1.0}) + """ + + _java_class = "org.apache.spark.ml.feature.OneHotEncoder" + + # a placeholder to make it appear in the generated doc + includeFirst = Param(Params._dummy(), "includeFirst", "include first category") + + @keyword_only + def __init__(self, includeFirst=True, inputCol=None, outputCol=None): + """ + __init__(self, includeFirst=True, inputCol=None, outputCol=None) + """ + super(OneHotEncoder, self).__init__() + self.includeFirst = Param(self, "includeFirst", "include first category") + self._setDefault(includeFirst=True) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, includeFirst=True, inputCol=None, outputCol=None): + """ + setParams(self, includeFirst=True, inputCol=None, outputCol=None) + Sets params for this OneHotEncoder. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setIncludeFirst(self, value): + """ + Sets the value of :py:attr:`includeFirst`. + """ + self.paramMap[self.includeFirst] = value + return self + + def getIncludeFirst(self): + """ + Gets the value of includeFirst or its default value. + """ + return self.getOrDefault(self.includeFirst) + + +@inherit_doc +class PolynomialExpansion(JavaTransformer, HasInputCol, HasOutputCol): + """ + Perform feature expansion in a polynomial space. As said in wikipedia of Polynomial Expansion, + which is available at `http://en.wikipedia.org/wiki/Polynomial_expansion`, "In mathematics, an + expansion of a product of sums expresses it as a sum of products by using the fact that + multiplication distributes over addition". Take a 2-variable feature vector as an example: + `(x, y)`, if we want to expand it with degree 2, then we get `(x, x * x, y, x * y, y * y)`. + + >>> from pyspark.mllib.linalg import Vectors + >>> df = sqlContext.createDataFrame([(Vectors.dense([0.5, 2.0]),)], ["dense"]) + >>> px = PolynomialExpansion(degree=2, inputCol="dense", outputCol="expanded") + >>> px.transform(df).head().expanded + DenseVector([0.5, 0.25, 2.0, 1.0, 4.0]) + >>> px.setParams(outputCol="test").transform(df).head().test + DenseVector([0.5, 0.25, 2.0, 1.0, 4.0]) + """ + + _java_class = "org.apache.spark.ml.feature.PolynomialExpansion" + + # a placeholder to make it appear in the generated doc + degree = Param(Params._dummy(), "degree", "the polynomial degree to expand (>= 1)") + + @keyword_only + def __init__(self, degree=2, inputCol=None, outputCol=None): + """ + __init__(self, degree=2, inputCol=None, outputCol=None) + """ + super(PolynomialExpansion, self).__init__() + self.degree = Param(self, "degree", "the polynomial degree to expand (>= 1)") + self._setDefault(degree=2) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, degree=2, inputCol=None, outputCol=None): + """ + setParams(self, degree=2, inputCol=None, outputCol=None) + Sets params for this PolynomialExpansion. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setDegree(self, value): + """ + Sets the value of :py:attr:`degree`. + """ + self.paramMap[self.degree] = value + return self + + def getDegree(self): + """ + Gets the value of degree or its default value. + """ + return self.getOrDefault(self.degree) + + +@inherit_doc +@ignore_unicode_prefix +class RegexTokenizer(JavaTransformer, HasInputCol, HasOutputCol): + """ + A regex based tokenizer that extracts tokens either by repeatedly matching the regex(default) + or using it to split the text (set matching to false). Optional parameters also allow filtering + tokens using a minimal length. + It returns an array of strings that can be empty. + + >>> df = sqlContext.createDataFrame([("a b c",)], ["text"]) + >>> reTokenizer = RegexTokenizer(inputCol="text", outputCol="words") + >>> reTokenizer.transform(df).head() + Row(text=u'a b c', words=[u'a', u'b', u'c']) + >>> # Change a parameter. + >>> reTokenizer.setParams(outputCol="tokens").transform(df).head() + Row(text=u'a b c', tokens=[u'a', u'b', u'c']) + >>> # Temporarily modify a parameter. + >>> reTokenizer.transform(df, {reTokenizer.outputCol: "words"}).head() + Row(text=u'a b c', words=[u'a', u'b', u'c']) + >>> reTokenizer.transform(df).head() + Row(text=u'a b c', tokens=[u'a', u'b', u'c']) + >>> # Must use keyword arguments to specify params. + >>> reTokenizer.setParams("text") + Traceback (most recent call last): + ... + TypeError: Method setParams forces keyword arguments. + """ + + _java_class = "org.apache.spark.ml.feature.RegexTokenizer" + # a placeholder to make it appear in the generated doc + minTokenLength = Param(Params._dummy(), "minTokenLength", "minimum token length (>= 0)") + gaps = Param(Params._dummy(), "gaps", "Set regex to match gaps or tokens") + pattern = Param(Params._dummy(), "pattern", "regex pattern used for tokenizing") + + @keyword_only + def __init__(self, minTokenLength=1, gaps=False, pattern="\\p{L}+|[^\\p{L}\\s]+", + inputCol=None, outputCol=None): + """ + __init__(self, minTokenLength=1, gaps=False, pattern="\\p{L}+|[^\\p{L}\\s]+", + inputCol=None, outputCol=None) + """ + super(RegexTokenizer, self).__init__() + self.minTokenLength = Param(self, "minLength", "minimum token length (>= 0)") + self.gaps = Param(self, "gaps", "Set regex to match gaps or tokens") + self.pattern = Param(self, "pattern", "regex pattern used for tokenizing") + self._setDefault(minTokenLength=1, gaps=False, pattern="\\p{L}+|[^\\p{L}\\s]+") + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, minTokenLength=1, gaps=False, pattern="\\p{L}+|[^\\p{L}\\s]+", + inputCol=None, outputCol=None): + """ + setParams(self, minTokenLength=1, gaps=False, pattern="\\p{L}+|[^\\p{L}\\s]+", + inputCol="input", outputCol="output") + Sets params for this RegexTokenizer. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setMinTokenLength(self, value): + """ + Sets the value of :py:attr:`minTokenLength`. + """ + self.paramMap[self.minTokenLength] = value + return self + + def getMinTokenLength(self): + """ + Gets the value of minTokenLength or its default value. + """ + return self.getOrDefault(self.minTokenLength) + + def setGaps(self, value): + """ + Sets the value of :py:attr:`gaps`. + """ + self.paramMap[self.gaps] = value + return self + + def getGaps(self): + """ + Gets the value of gaps or its default value. + """ + return self.getOrDefault(self.gaps) + + def setPattern(self, value): + """ + Sets the value of :py:attr:`pattern`. + """ + self.paramMap[self.pattern] = value + return self + + def getPattern(self): + """ + Gets the value of pattern or its default value. + """ + return self.getOrDefault(self.pattern) + + +@inherit_doc +class StandardScaler(JavaEstimator, HasInputCol, HasOutputCol): + """ + Standardizes features by removing the mean and scaling to unit variance using column summary + statistics on the samples in the training set. + + >>> from pyspark.mllib.linalg import Vectors + >>> df = sqlContext.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) + >>> standardScaler = StandardScaler(inputCol="a", outputCol="scaled") + >>> model = standardScaler.fit(df) + >>> model.transform(df).collect()[1].scaled + DenseVector([1.4142]) + """ + + _java_class = "org.apache.spark.ml.feature.StandardScaler" + + # a placeholder to make it appear in the generated doc + withMean = Param(Params._dummy(), "withMean", "Center data with mean") + withStd = Param(Params._dummy(), "withStd", "Scale to unit standard deviation") + + @keyword_only + def __init__(self, withMean=False, withStd=True, inputCol=None, outputCol=None): + """ + __init__(self, withMean=False, withStd=True, inputCol=None, outputCol=None) + """ + super(StandardScaler, self).__init__() + self.withMean = Param(self, "withMean", "Center data with mean") + self.withStd = Param(self, "withStd", "Scale to unit standard deviation") + self._setDefault(withMean=False, withStd=True) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, withMean=False, withStd=True, inputCol=None, outputCol=None): + """ + setParams(self, withMean=False, withStd=True, inputCol=None, outputCol=None) + Sets params for this StandardScaler. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setWithMean(self, value): + """ + Sets the value of :py:attr:`withMean`. + """ + self.paramMap[self.withMean] = value + return self + + def getWithMean(self): + """ + Gets the value of withMean or its default value. + """ + return self.getOrDefault(self.withMean) + + def setWithStd(self, value): + """ + Sets the value of :py:attr:`withStd`. + """ + self.paramMap[self.withStd] = value + return self + + def getWithStd(self): + """ + Gets the value of withStd or its default value. + """ + return self.getOrDefault(self.withStd) + + +class StandardScalerModel(JavaModel): + """ + Model fitted by StandardScaler. + """ + + +@inherit_doc +class StringIndexer(JavaEstimator, HasInputCol, HasOutputCol): + """ + A label indexer that maps a string column of labels to an ML column of label indices. + If the input column is numeric, we cast it to string and index the string values. + The indices are in [0, numLabels), ordered by label frequencies. + So the most frequent label gets index 0. + + >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed") + >>> model = stringIndexer.fit(stringIndDf) + >>> td = model.transform(stringIndDf) + >>> sorted(set([(i[0], i[1]) for i in td.select(td.id, td.indexed).collect()]), + ... key=lambda x: x[0]) + [(0, 0.0), (1, 2.0), (2, 1.0), (3, 0.0), (4, 0.0), (5, 1.0)] + """ + + _java_class = "org.apache.spark.ml.feature.StringIndexer" + + @keyword_only + def __init__(self, inputCol=None, outputCol=None): + """ + __init__(self, inputCol=None, outputCol=None) + """ + super(StringIndexer, self).__init__() + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, inputCol=None, outputCol=None): + """ + setParams(self, inputCol=None, outputCol=None) + Sets params for this StringIndexer. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + +class StringIndexerModel(JavaModel): + """ + Model fitted by StringIndexer. + """ @inherit_doc @@ -31,8 +590,7 @@ class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): A tokenizer that converts the input string to lowercase and then splits it by white spaces. - >>> from pyspark.sql import Row - >>> df = sc.parallelize([Row(text="a b c")]).toDF() + >>> df = sqlContext.createDataFrame([("a b c",)], ["text"]) >>> tokenizer = Tokenizer(inputCol="text", outputCol="words") >>> tokenizer.transform(df).head() Row(text=u'a b c', words=[u'a', u'b', u'c']) @@ -73,49 +631,244 @@ def setParams(self, inputCol=None, outputCol=None): @inherit_doc -class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): +class VectorAssembler(JavaTransformer, HasInputCols, HasOutputCol): """ - Maps a sequence of terms to their term frequencies using the - hashing trick. + A feature transformer that merges multiple columns into a vector column. - >>> from pyspark.sql import Row - >>> df = sc.parallelize([Row(words=["a", "b", "c"])]).toDF() - >>> hashingTF = HashingTF(numFeatures=10, inputCol="words", outputCol="features") - >>> hashingTF.transform(df).head().features - SparseVector(10, {7: 1.0, 8: 1.0, 9: 1.0}) - >>> hashingTF.setParams(outputCol="freqs").transform(df).head().freqs - SparseVector(10, {7: 1.0, 8: 1.0, 9: 1.0}) - >>> params = {hashingTF.numFeatures: 5, hashingTF.outputCol: "vector"} - >>> hashingTF.transform(df, params).head().vector - SparseVector(5, {2: 1.0, 3: 1.0, 4: 1.0}) + >>> df = sqlContext.createDataFrame([(1, 0, 3)], ["a", "b", "c"]) + >>> vecAssembler = VectorAssembler(inputCols=["a", "b", "c"], outputCol="features") + >>> vecAssembler.transform(df).head().features + DenseVector([1.0, 0.0, 3.0]) + >>> vecAssembler.setParams(outputCol="freqs").transform(df).head().freqs + DenseVector([1.0, 0.0, 3.0]) + >>> params = {vecAssembler.inputCols: ["b", "a"], vecAssembler.outputCol: "vector"} + >>> vecAssembler.transform(df, params).head().vector + DenseVector([0.0, 1.0]) """ - _java_class = "org.apache.spark.ml.feature.HashingTF" + _java_class = "org.apache.spark.ml.feature.VectorAssembler" @keyword_only - def __init__(self, numFeatures=1 << 18, inputCol=None, outputCol=None): + def __init__(self, inputCols=None, outputCol=None): """ - __init__(self, numFeatures=1 << 18, inputCol=None, outputCol=None) + __init__(self, inputCols=None, outputCol=None) """ - super(HashingTF, self).__init__() - self._setDefault(numFeatures=1 << 18) + super(VectorAssembler, self).__init__() kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only - def setParams(self, numFeatures=1 << 18, inputCol=None, outputCol=None): + def setParams(self, inputCols=None, outputCol=None): """ - setParams(self, numFeatures=1 << 18, inputCol=None, outputCol=None) - Sets params for this HashingTF. + setParams(self, inputCols=None, outputCol=None) + Sets params for this VectorAssembler. """ kwargs = self.setParams._input_kwargs return self._set(**kwargs) +@inherit_doc +class VectorIndexer(JavaEstimator, HasInputCol, HasOutputCol): + """ + Class for indexing categorical feature columns in a dataset of [[Vector]]. + + This has 2 usage modes: + - Automatically identify categorical features (default behavior) + - This helps process a dataset of unknown vectors into a dataset with some continuous + features and some categorical features. The choice between continuous and categorical + is based upon a maxCategories parameter. + - Set maxCategories to the maximum number of categorical any categorical feature should + have. + - E.g.: Feature 0 has unique values {-1.0, 0.0}, and feature 1 values {1.0, 3.0, 5.0}. + If maxCategories = 2, then feature 0 will be declared categorical and use indices {0, 1}, + and feature 1 will be declared continuous. + - Index all features, if all features are categorical + - If maxCategories is set to be very large, then this will build an index of unique + values for all features. + - Warning: This can cause problems if features are continuous since this will collect ALL + unique values to the driver. + - E.g.: Feature 0 has unique values {-1.0, 0.0}, and feature 1 values {1.0, 3.0, 5.0}. + If maxCategories >= 3, then both features will be declared categorical. + + This returns a model which can transform categorical features to use 0-based indices. + + Index stability: + - This is not guaranteed to choose the same category index across multiple runs. + - If a categorical feature includes value 0, then this is guaranteed to map value 0 to + index 0. This maintains vector sparsity. + - More stability may be added in the future. + + TODO: Future extensions: The following functionality is planned for the future: + - Preserve metadata in transform; if a feature's metadata is already present, + do not recompute. + - Specify certain features to not index, either via a parameter or via existing metadata. + - Add warning if a categorical feature has only 1 category. + - Add option for allowing unknown categories. + + >>> from pyspark.mllib.linalg import Vectors + >>> df = sqlContext.createDataFrame([(Vectors.dense([-1.0, 0.0]),), + ... (Vectors.dense([0.0, 1.0]),), (Vectors.dense([0.0, 2.0]),)], ["a"]) + >>> indexer = VectorIndexer(maxCategories=2, inputCol="a", outputCol="indexed") + >>> model = indexer.fit(df) + >>> model.transform(df).head().indexed + DenseVector([1.0, 0.0]) + >>> indexer.setParams(outputCol="test").fit(df).transform(df).collect()[1].test + DenseVector([0.0, 1.0]) + >>> params = {indexer.maxCategories: 3, indexer.outputCol: "vector"} + >>> model2 = indexer.fit(df, params) + >>> model2.transform(df).head().vector + DenseVector([1.0, 0.0]) + """ + + _java_class = "org.apache.spark.ml.feature.VectorIndexer" + # a placeholder to make it appear in the generated doc + maxCategories = Param(Params._dummy(), "maxCategories", + "Threshold for the number of values a categorical feature can take " + + "(>= 2). If a feature is found to have > maxCategories values, then " + + "it is declared continuous.") + + @keyword_only + def __init__(self, maxCategories=20, inputCol=None, outputCol=None): + """ + __init__(self, maxCategories=20, inputCol=None, outputCol=None) + """ + super(VectorIndexer, self).__init__() + self.maxCategories = Param(self, "maxCategories", + "Threshold for the number of values a categorical feature " + + "can take (>= 2). If a feature is found to have " + + "> maxCategories values, then it is declared continuous.") + self._setDefault(maxCategories=20) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, maxCategories=20, inputCol=None, outputCol=None): + """ + setParams(self, maxCategories=20, inputCol=None, outputCol=None) + Sets params for this VectorIndexer. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setMaxCategories(self, value): + """ + Sets the value of :py:attr:`maxCategories`. + """ + self.paramMap[self.maxCategories] = value + return self + + def getMaxCategories(self): + """ + Gets the value of maxCategories or its default value. + """ + return self.getOrDefault(self.maxCategories) + + +@inherit_doc +@ignore_unicode_prefix +class Word2Vec(JavaEstimator, HasStepSize, HasMaxIter, HasSeed, HasInputCol, HasOutputCol): + """ + Word2Vec trains a model of `Map(String, Vector)`, i.e. transforms a word into a code for further + natural language processing or machine learning process. + + >>> sent = ("a b " * 100 + "a c " * 10).split(" ") + >>> doc = sqlContext.createDataFrame([(sent,), (sent,)], ["sentence"]) + >>> model = Word2Vec(vectorSize=5, seed=42, inputCol="sentence", outputCol="model").fit(doc) + >>> model.transform(doc).head().model + DenseVector([-0.0422, -0.5138, -0.2546, 0.6885, 0.276]) + """ + + _java_class = "org.apache.spark.ml.feature.Word2Vec" + # a placeholder to make it appear in the generated doc + vectorSize = Param(Params._dummy(), "vectorSize", + "the dimension of codes after transforming from words") + numPartitions = Param(Params._dummy(), "numPartitions", + "number of partitions for sentences of words") + minCount = Param(Params._dummy(), "minCount", + "the minimum number of times a token must appear to be included in the " + + "word2vec model's vocabulary") + + @keyword_only + def __init__(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, + seed=42, inputCol=None, outputCol=None): + """ + __init__(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, + seed=42, inputCol=None, outputCol=None) + """ + super(Word2Vec, self).__init__() + self.vectorSize = Param(self, "vectorSize", + "the dimension of codes after transforming from words") + self.numPartitions = Param(self, "numPartitions", + "number of partitions for sentences of words") + self.minCount = Param(self, "minCount", + "the minimum number of times a token must appear to be included " + + "in the word2vec model's vocabulary") + self._setDefault(vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, + seed=42) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, + seed=42, inputCol=None, outputCol=None): + """ + setParams(self, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, seed=42, + inputCol=None, outputCol=None) + Sets params for this Word2Vec. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setVectorSize(self, value): + """ + Sets the value of :py:attr:`vectorSize`. + """ + self.paramMap[self.vectorSize] = value + return self + + def getVectorSize(self): + """ + Gets the value of vectorSize or its default value. + """ + return self.getOrDefault(self.vectorSize) + + def setNumPartitions(self, value): + """ + Sets the value of :py:attr:`numPartitions`. + """ + self.paramMap[self.numPartitions] = value + return self + + def getNumPartitions(self): + """ + Gets the value of numPartitions or its default value. + """ + return self.getOrDefault(self.numPartitions) + + def setMinCount(self, value): + """ + Sets the value of :py:attr:`minCount`. + """ + self.paramMap[self.minCount] = value + return self + + def getMinCount(self): + """ + Gets the value of minCount or its default value. + """ + return self.getOrDefault(self.minCount) + + +class Word2VecModel(JavaModel): + """ + Model fitted by Word2Vec. + """ + + if __name__ == "__main__": import doctest from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import Row, SQLContext globs = globals().copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: @@ -123,8 +876,11 @@ def setParams(self, numFeatures=1 << 18, inputCol=None, outputCol=None): sqlContext = SQLContext(sc) globs['sc'] = sc globs['sqlContext'] = sqlContext - (failure_count, test_count) = doctest.testmod( - globs=globs, optionflags=doctest.ELLIPSIS) + testData = sc.parallelize([Row(id=0, label="a"), Row(id=1, label="b"), + Row(id=2, label="c"), Row(id=3, label="a"), + Row(id=4, label="a"), Row(id=5, label="c")], 2) + globs['stringIndDf'] = sqlContext.createDataFrame(testData) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) sc.stop() if failure_count: exit(-1) diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index c71c823db2c81..ee901f25847bc 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -95,8 +95,11 @@ def get$Name(self): ("predictionCol", "prediction column name", "'prediction'"), ("rawPredictionCol", "raw prediction column name", "'rawPrediction'"), ("inputCol", "input column name", None), + ("inputCols", "input column names", None), ("outputCol", "output column name", None), - ("numFeatures", "number of features", None)] + ("seed", "random seed", None), + ("tol", "the convergence tolerance for iterative algorithms", None), + ("stepSize", "Step size to be used for each iteration of optimization.", None)] code = [] for name, doc, defaultValueStr in shared: code.append(_gen_param_code(name, doc, defaultValueStr)) diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 4f243844f8caa..5e7529c1dcf1e 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -223,6 +223,35 @@ def getInputCol(self): return self.getOrDefault(self.inputCol) +class HasInputCols(Params): + """ + Mixin for param inputCols: input column names. + """ + + # a placeholder to make it appear in the generated doc + inputCols = Param(Params._dummy(), "inputCols", "input column names") + + def __init__(self): + super(HasInputCols, self).__init__() + #: param for input column names + self.inputCols = Param(self, "inputCols", "input column names") + if None is not None: + self._setDefault(inputCols=None) + + def setInputCols(self, value): + """ + Sets the value of :py:attr:`inputCols`. + """ + self.paramMap[self.inputCols] = value + return self + + def getInputCols(self): + """ + Gets the value of inputCols or its default value. + """ + return self.getOrDefault(self.inputCols) + + class HasOutputCol(Params): """ Mixin for param outputCol: output column name. @@ -279,3 +308,92 @@ def getNumFeatures(self): Gets the value of numFeatures or its default value. """ return self.getOrDefault(self.numFeatures) + + +class HasSeed(Params): + """ + Mixin for param seed: random seed. + """ + + # a placeholder to make it appear in the generated doc + seed = Param(Params._dummy(), "seed", "random seed") + + def __init__(self): + super(HasSeed, self).__init__() + #: param for random seed + self.seed = Param(self, "seed", "random seed") + if None is not None: + self._setDefault(seed=None) + + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + self.paramMap[self.seed] = value + return self + + def getSeed(self): + """ + Gets the value of seed or its default value. + """ + return self.getOrDefault(self.seed) + + +class HasTol(Params): + """ + Mixin for param tol: the convergence tolerance for iterative algorithms. + """ + + # a placeholder to make it appear in the generated doc + tol = Param(Params._dummy(), "tol", "the convergence tolerance for iterative algorithms") + + def __init__(self): + super(HasTol, self).__init__() + #: param for the convergence tolerance for iterative algorithms + self.tol = Param(self, "tol", "the convergence tolerance for iterative algorithms") + if None is not None: + self._setDefault(tol=None) + + def setTol(self, value): + """ + Sets the value of :py:attr:`tol`. + """ + self.paramMap[self.tol] = value + return self + + def getTol(self): + """ + Gets the value of tol or its default value. + """ + return self.getOrDefault(self.tol) + + +class HasStepSize(Params): + """ + Mixin for param stepSize: Step size to be used for each iteration of optimization.. + """ + + # a placeholder to make it appear in the generated doc + stepSize = Param(Params._dummy(), "stepSize", + "Step size to be used for each iteration of optimization.") + + def __init__(self): + super(HasStepSize, self).__init__() + #: param for Step size to be used for each iteration of optimization. + self.stepSize = Param(self, "stepSize", + "Step size to be used for each iteration of optimization.") + if None is not None: + self._setDefault(stepSize=None) + + def setStepSize(self, value): + """ + Sets the value of :py:attr:`stepSize`. + """ + self.paramMap[self.stepSize] = value + return self + + def getStepSize(self): + """ + Gets the value of stepSize or its default value. + """ + return self.getOrDefault(self.stepSize) diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 7b875e4b71254..c1b2077c985cf 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -22,7 +22,7 @@ from pyspark.mllib.common import inherit_doc -__all__ = ['Estimator', 'Transformer', 'Pipeline', 'PipelineModel', 'Evaluator'] +__all__ = ['Estimator', 'Transformer', 'Pipeline', 'PipelineModel', 'Evaluator', 'Model'] @inherit_doc @@ -70,6 +70,15 @@ def transform(self, dataset, params={}): raise NotImplementedError() +@inherit_doc +class Model(Transformer): + """ + Abstract class for models that are fitted by estimators. + """ + + __metaclass__ = ABCMeta + + @inherit_doc class Pipeline(Estimator): """ @@ -154,7 +163,7 @@ def fit(self, dataset, params={}): @inherit_doc -class PipelineModel(Transformer): +class PipelineModel(Model): """ Represents a compiled pipeline with transformers and fitted models. """ diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 1773ab5bdcdb1..28e3727f2c064 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -16,29 +16,33 @@ # import itertools +import numpy as np -__all__ = ['ParamGridBuilder'] +from pyspark.ml.param import Params, Param +from pyspark.ml import Estimator, Model +from pyspark.ml.util import keyword_only +from pyspark.sql.functions import rand + +__all__ = ['ParamGridBuilder', 'CrossValidator', 'CrossValidatorModel'] class ParamGridBuilder(object): - """ + r""" Builder for a param grid used in grid search-based model selection. - >>> from classification import LogisticRegression + >>> from pyspark.ml.classification import LogisticRegression >>> lr = LogisticRegression() - >>> output = ParamGridBuilder().baseOn({lr.labelCol: 'l'}) \ - .baseOn([lr.predictionCol, 'p']) \ - .addGrid(lr.regParam, [1.0, 2.0, 3.0]) \ - .addGrid(lr.maxIter, [1, 5]) \ - .addGrid(lr.featuresCol, ['f']) \ - .build() - >>> expected = [ \ -{lr.regParam: 1.0, lr.featuresCol: 'f', lr.maxIter: 1, lr.labelCol: 'l', lr.predictionCol: 'p'}, \ -{lr.regParam: 2.0, lr.featuresCol: 'f', lr.maxIter: 1, lr.labelCol: 'l', lr.predictionCol: 'p'}, \ -{lr.regParam: 3.0, lr.featuresCol: 'f', lr.maxIter: 1, lr.labelCol: 'l', lr.predictionCol: 'p'}, \ -{lr.regParam: 1.0, lr.featuresCol: 'f', lr.maxIter: 5, lr.labelCol: 'l', lr.predictionCol: 'p'}, \ -{lr.regParam: 2.0, lr.featuresCol: 'f', lr.maxIter: 5, lr.labelCol: 'l', lr.predictionCol: 'p'}, \ -{lr.regParam: 3.0, lr.featuresCol: 'f', lr.maxIter: 5, lr.labelCol: 'l', lr.predictionCol: 'p'}] + >>> output = ParamGridBuilder() \ + ... .baseOn({lr.labelCol: 'l'}) \ + ... .baseOn([lr.predictionCol, 'p']) \ + ... .addGrid(lr.regParam, [1.0, 2.0]) \ + ... .addGrid(lr.maxIter, [1, 5]) \ + ... .build() + >>> expected = [ + ... {lr.regParam: 1.0, lr.maxIter: 1, lr.labelCol: 'l', lr.predictionCol: 'p'}, + ... {lr.regParam: 2.0, lr.maxIter: 1, lr.labelCol: 'l', lr.predictionCol: 'p'}, + ... {lr.regParam: 1.0, lr.maxIter: 5, lr.labelCol: 'l', lr.predictionCol: 'p'}, + ... {lr.regParam: 2.0, lr.maxIter: 5, lr.labelCol: 'l', lr.predictionCol: 'p'}] >>> len(output) == len(expected) True >>> all([m in expected for m in output]) @@ -79,6 +83,179 @@ def build(self): return [dict(zip(keys, prod)) for prod in itertools.product(*grid_values)] +class CrossValidator(Estimator): + """ + K-fold cross validation. + + >>> from pyspark.ml.classification import LogisticRegression + >>> from pyspark.ml.evaluation import BinaryClassificationEvaluator + >>> from pyspark.mllib.linalg import Vectors + >>> dataset = sqlContext.createDataFrame( + ... [(Vectors.dense([0.0, 1.0]), 0.0), + ... (Vectors.dense([1.0, 2.0]), 1.0), + ... (Vectors.dense([0.55, 3.0]), 0.0), + ... (Vectors.dense([0.45, 4.0]), 1.0), + ... (Vectors.dense([0.51, 5.0]), 1.0)] * 10, + ... ["features", "label"]) + >>> lr = LogisticRegression() + >>> grid = ParamGridBuilder().addGrid(lr.maxIter, [0, 1, 5]).build() + >>> evaluator = BinaryClassificationEvaluator() + >>> cv = CrossValidator(estimator=lr, estimatorParamMaps=grid, evaluator=evaluator) + >>> # SPARK-7432: The following test is flaky. + >>> # cvModel = cv.fit(dataset) + >>> # expected = lr.fit(dataset, {lr.maxIter: 5}).transform(dataset) + >>> # cvModel.transform(dataset).collect() == expected.collect() + """ + + # a placeholder to make it appear in the generated doc + estimator = Param(Params._dummy(), "estimator", "estimator to be cross-validated") + + # a placeholder to make it appear in the generated doc + estimatorParamMaps = Param(Params._dummy(), "estimatorParamMaps", "estimator param maps") + + # a placeholder to make it appear in the generated doc + evaluator = Param( + Params._dummy(), "evaluator", + "evaluator used to select hyper-parameters that maximize the cross-validated metric") + + # a placeholder to make it appear in the generated doc + numFolds = Param(Params._dummy(), "numFolds", "number of folds for cross validation") + + @keyword_only + def __init__(self, estimator=None, estimatorParamMaps=None, evaluator=None, numFolds=3): + """ + __init__(self, estimator=None, estimatorParamMaps=None, evaluator=None, numFolds=3) + """ + super(CrossValidator, self).__init__() + #: param for estimator to be cross-validated + self.estimator = Param(self, "estimator", "estimator to be cross-validated") + #: param for estimator param maps + self.estimatorParamMaps = Param(self, "estimatorParamMaps", "estimator param maps") + #: param for the evaluator used to select hyper-parameters that + #: maximize the cross-validated metric + self.evaluator = Param( + self, "evaluator", + "evaluator used to select hyper-parameters that maximize the cross-validated metric") + #: param for number of folds for cross validation + self.numFolds = Param(self, "numFolds", "number of folds for cross validation") + self._setDefault(numFolds=3) + kwargs = self.__init__._input_kwargs + self._set(**kwargs) + + @keyword_only + def setParams(self, estimator=None, estimatorParamMaps=None, evaluator=None, numFolds=3): + """ + setParams(self, estimator=None, estimatorParamMaps=None, evaluator=None, numFolds=3): + Sets params for cross validator. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setEstimator(self, value): + """ + Sets the value of :py:attr:`estimator`. + """ + self.paramMap[self.estimator] = value + return self + + def getEstimator(self): + """ + Gets the value of estimator or its default value. + """ + return self.getOrDefault(self.estimator) + + def setEstimatorParamMaps(self, value): + """ + Sets the value of :py:attr:`estimatorParamMaps`. + """ + self.paramMap[self.estimatorParamMaps] = value + return self + + def getEstimatorParamMaps(self): + """ + Gets the value of estimatorParamMaps or its default value. + """ + return self.getOrDefault(self.estimatorParamMaps) + + def setEvaluator(self, value): + """ + Sets the value of :py:attr:`evaluator`. + """ + self.paramMap[self.evaluator] = value + return self + + def getEvaluator(self): + """ + Gets the value of evaluator or its default value. + """ + return self.getOrDefault(self.evaluator) + + def setNumFolds(self, value): + """ + Sets the value of :py:attr:`numFolds`. + """ + self.paramMap[self.numFolds] = value + return self + + def getNumFolds(self): + """ + Gets the value of numFolds or its default value. + """ + return self.getOrDefault(self.numFolds) + + def fit(self, dataset, params={}): + paramMap = self.extractParamMap(params) + est = paramMap[self.estimator] + epm = paramMap[self.estimatorParamMaps] + numModels = len(epm) + eva = paramMap[self.evaluator] + nFolds = paramMap[self.numFolds] + h = 1.0 / nFolds + randCol = self.uid + "_rand" + df = dataset.select("*", rand(0).alias(randCol)) + metrics = np.zeros(numModels) + for i in range(nFolds): + validateLB = i * h + validateUB = (i + 1) * h + condition = (df[randCol] >= validateLB) & (df[randCol] < validateUB) + validation = df.filter(condition) + train = df.filter(~condition) + for j in range(numModels): + model = est.fit(train, epm[j]) + # TODO: duplicate evaluator to take extra params from input + metric = eva.evaluate(model.transform(validation, epm[j])) + metrics[j] += metric + bestIndex = np.argmax(metrics) + bestModel = est.fit(dataset, epm[bestIndex]) + return CrossValidatorModel(bestModel) + + +class CrossValidatorModel(Model): + """ + Model from k-fold cross validation. + """ + + def __init__(self, bestModel): + #: best model from cross validation + self.bestModel = bestModel + + def transform(self, dataset, params={}): + return self.bestModel.transform(dataset, params) + + if __name__ == "__main__": import doctest - doctest.testmod() + from pyspark.context import SparkContext + from pyspark.sql import SQLContext + globs = globals().copy() + # The small batch size here ensures that we see multiple batches, + # even in these small test examples: + sc = SparkContext("local[2]", "ml.tuning tests") + sqlContext = SQLContext(sc) + globs['sc'] = sc + globs['sqlContext'] = sqlContext + (failure_count, test_count) = doctest.testmod( + globs=globs, optionflags=doctest.ELLIPSIS) + sc.stop() + if failure_count: + exit(-1) diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index 73741c4b40dfb..f5ac2a398642a 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -20,7 +20,7 @@ from pyspark import SparkContext from pyspark.sql import DataFrame from pyspark.ml.param import Params -from pyspark.ml.pipeline import Estimator, Transformer, Evaluator +from pyspark.ml.pipeline import Estimator, Transformer, Evaluator, Model from pyspark.mllib.common import inherit_doc @@ -67,7 +67,9 @@ def _transfer_params_to_java(self, params, java_obj): paramMap = self.extractParamMap(params) for param in self.params: if param in paramMap: - java_obj.set(param.name, paramMap[param]) + value = paramMap[param] + java_param = java_obj.getParam(param.name) + java_obj.set(java_param.w(value)) def _empty_java_param_map(self): """ @@ -79,7 +81,8 @@ def _create_java_param_map(self, params, java_obj): paramMap = self._empty_java_param_map() for param, value in params.items(): if param.parent is self: - paramMap.put(java_obj.getParam(param.name), value) + java_param = java_obj.getParam(param.name) + paramMap.put(java_param.w(value)) return paramMap @@ -126,14 +129,12 @@ class JavaTransformer(Transformer, JavaWrapper): def transform(self, dataset, params={}): java_obj = self._java_obj() - self._transfer_params_to_java({}, java_obj) - java_param_map = self._create_java_param_map(params, java_obj) - return DataFrame(java_obj.transform(dataset._jdf, java_param_map), - dataset.sql_ctx) + self._transfer_params_to_java(params, java_obj) + return DataFrame(java_obj.transform(dataset._jdf), dataset.sql_ctx) @inherit_doc -class JavaModel(JavaTransformer): +class JavaModel(Model, JavaTransformer): """ Base class for :py:class:`Model`s that wrap Java/Scala implementations. diff --git a/python/pyspark/mllib/evaluation.py b/python/pyspark/mllib/evaluation.py index 16cb49cc0cfff..3e11df09da6b1 100644 --- a/python/pyspark/mllib/evaluation.py +++ b/python/pyspark/mllib/evaluation.py @@ -27,9 +27,9 @@ class BinaryClassificationMetrics(JavaModelWrapper): >>> scoreAndLabels = sc.parallelize([ ... (0.1, 0.0), (0.1, 1.0), (0.4, 0.0), (0.6, 0.0), (0.6, 1.0), (0.6, 1.0), (0.8, 1.0)], 2) >>> metrics = BinaryClassificationMetrics(scoreAndLabels) - >>> metrics.areaUnderROC() + >>> metrics.areaUnderROC 0.70... - >>> metrics.areaUnderPR() + >>> metrics.areaUnderPR 0.83... >>> metrics.unpersist() """ @@ -47,6 +47,7 @@ def __init__(self, scoreAndLabels): java_model = java_class(df._jdf) super(BinaryClassificationMetrics, self).__init__(java_model) + @property def areaUnderROC(self): """ Computes the area under the receiver operating characteristic @@ -54,6 +55,7 @@ def areaUnderROC(self): """ return self.call("areaUnderROC") + @property def areaUnderPR(self): """ Computes the area under the precision-recall curve. @@ -67,6 +69,78 @@ def unpersist(self): self.call("unpersist") +class RegressionMetrics(JavaModelWrapper): + """ + Evaluator for regression. + + >>> predictionAndObservations = sc.parallelize([ + ... (2.5, 3.0), (0.0, -0.5), (2.0, 2.0), (8.0, 7.0)]) + >>> metrics = RegressionMetrics(predictionAndObservations) + >>> metrics.explainedVariance + 0.95... + >>> metrics.meanAbsoluteError + 0.5... + >>> metrics.meanSquaredError + 0.37... + >>> metrics.rootMeanSquaredError + 0.61... + >>> metrics.r2 + 0.94... + """ + + def __init__(self, predictionAndObservations): + """ + :param predictionAndObservations: an RDD of (prediction, observation) pairs. + """ + sc = predictionAndObservations.ctx + sql_ctx = SQLContext(sc) + df = sql_ctx.createDataFrame(predictionAndObservations, schema=StructType([ + StructField("prediction", DoubleType(), nullable=False), + StructField("observation", DoubleType(), nullable=False)])) + java_class = sc._jvm.org.apache.spark.mllib.evaluation.RegressionMetrics + java_model = java_class(df._jdf) + super(RegressionMetrics, self).__init__(java_model) + + @property + def explainedVariance(self): + """ + Returns the explained variance regression score. + explainedVariance = 1 - variance(y - \hat{y}) / variance(y) + """ + return self.call("explainedVariance") + + @property + def meanAbsoluteError(self): + """ + Returns the mean absolute error, which is a risk function corresponding to the + expected value of the absolute error loss or l1-norm loss. + """ + return self.call("meanAbsoluteError") + + @property + def meanSquaredError(self): + """ + Returns the mean squared error, which is a risk function corresponding to the + expected value of the squared error loss or quadratic loss. + """ + return self.call("meanSquaredError") + + @property + def rootMeanSquaredError(self): + """ + Returns the root mean squared error, which is defined as the square root of + the mean squared error. + """ + return self.call("rootMeanSquaredError") + + @property + def r2(self): + """ + Returns R^2^, the coefficient of determination. + """ + return self.call("r2") + + def _test(): import doctest from pyspark import SparkContext diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 9f3b0baf9f19f..23d1a79ffe511 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -208,9 +208,46 @@ def __init__(self, ar): ar = ar.astype(np.float64) self.array = ar + @staticmethod + def parse(s): + """ + Parse string representation back into the DenseVector. + + >>> DenseVector.parse(' [ 0.0,1.0,2.0, 3.0]') + DenseVector([0.0, 1.0, 2.0, 3.0]) + """ + start = s.find('[') + if start == -1: + raise ValueError("Array should start with '['.") + end = s.find(']') + if end == -1: + raise ValueError("Array should end with ']'.") + s = s[start + 1: end] + + try: + values = [float(val) for val in s.split(',')] + except ValueError: + raise ValueError("Unable to parse values from %s" % s) + return DenseVector(values) + def __reduce__(self): return DenseVector, (self.array.tostring(),) + def numNonzeros(self): + return np.count_nonzero(self.array) + + def norm(self, p): + """ + Calculte the norm of a DenseVector. + + >>> a = DenseVector([0, -1, 2, -3]) + >>> a.norm(2) + 3.7... + >>> a.norm(1) + 6.0 + """ + return np.linalg.norm(self.array, p) + def dot(self, other): """ Compute the dot product of two Vectors. We support @@ -387,8 +424,74 @@ def __init__(self, size, *args): if self.indices[i] >= self.indices[i + 1]: raise TypeError("indices array must be sorted") + def numNonzeros(self): + return np.count_nonzero(self.values) + + def norm(self, p): + """ + Calculte the norm of a SparseVector. + + >>> a = SparseVector(4, [0, 1], [3., -4.]) + >>> a.norm(1) + 7.0 + >>> a.norm(2) + 5.0 + """ + return np.linalg.norm(self.values, p) + def __reduce__(self): - return (SparseVector, (self.size, self.indices.tostring(), self.values.tostring())) + return ( + SparseVector, + (self.size, self.indices.tostring(), self.values.tostring())) + + @staticmethod + def parse(s): + """ + Parse string representation back into the DenseVector. + + >>> SparseVector.parse(' (4, [0,1 ],[ 4.0,5.0] )') + SparseVector(4, {0: 4.0, 1: 5.0}) + """ + start = s.find('(') + if start == -1: + raise ValueError("Tuple should start with '('") + end = s.find(')') + if start == -1: + raise ValueError("Tuple should end with ')'") + s = s[start + 1: end].strip() + + size = s[: s.find(',')] + try: + size = int(size) + except ValueError: + raise ValueError("Cannot parse size %s." % size) + + ind_start = s.find('[') + if ind_start == -1: + raise ValueError("Indices array should start with '['.") + ind_end = s.find(']') + if ind_end == -1: + raise ValueError("Indices array should end with ']'") + new_s = s[ind_start + 1: ind_end] + ind_list = new_s.split(',') + try: + indices = [int(ind) for ind in ind_list] + except ValueError: + raise ValueError("Unable to parse indices from %s." % new_s) + s = s[ind_end + 1:].strip() + + val_start = s.find('[') + if val_start == -1: + raise ValueError("Values array should start with '['.") + val_end = s.find(']') + if val_end == -1: + raise ValueError("Values array should end with ']'.") + val_list = s[val_start + 1: val_end].split(',') + try: + values = [float(val) for val in val_list] + except ValueError: + raise ValueError("Unable to parse values from %s." % s) + return SparseVector(size, indices, values) def dot(self, other): """ @@ -633,6 +736,49 @@ def stringify(vector): """ return str(vector) + @staticmethod + def squared_distance(v1, v2): + """ + Squared distance between two vectors. + a and b can be of type SparseVector, DenseVector, np.ndarray + or array.array. + + >>> a = Vectors.sparse(4, [(0, 1), (3, 4)]) + >>> b = Vectors.dense([2, 5, 4, 1]) + >>> a.squared_distance(b) + 51.0 + """ + v1, v2 = _convert_to_vector(v1), _convert_to_vector(v2) + return v1.squared_distance(v2) + + @staticmethod + def norm(vector, p): + """ + Find norm of the given vector. + """ + return _convert_to_vector(vector).norm(p) + + @staticmethod + def parse(s): + """Parse a string representation back into the Vector. + + >>> Vectors.parse('[2,1,2 ]') + DenseVector([2.0, 1.0, 2.0]) + >>> Vectors.parse(' ( 100, [0], [2])') + SparseVector(100, {0: 2.0}) + """ + if s.find('(') == -1 and s.find('[') != -1: + return DenseVector.parse(s) + elif s.find('(') != -1: + return SparseVector.parse(s) + else: + raise ValueError( + "Cannot find tokens '[' or '(' from the input string.") + + @staticmethod + def zeros(size): + return DenseVector(np.zeros(size)) + class Matrix(object): """ diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 4bc6351bdf02f..41bde2ce3e60b 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -18,14 +18,16 @@ import numpy as np from numpy import array +from pyspark import RDD from pyspark.mllib.common import callMLlibFunc, _py2java, _java2py, inherit_doc -from pyspark.mllib.linalg import SparseVector, _convert_to_vector +from pyspark.mllib.linalg import SparseVector, Vectors, _convert_to_vector from pyspark.mllib.util import Saveable, Loader __all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'LinearRegressionWithSGD', 'RidgeRegressionModel', 'RidgeRegressionWithSGD', - 'LassoModel', 'LassoWithSGD'] + 'LassoModel', 'LassoWithSGD', 'IsotonicRegressionModel', + 'IsotonicRegression'] class LabeledPoint(object): @@ -396,6 +398,73 @@ def train(rdd, i): return _regression_train_wrapper(train, RidgeRegressionModel, data, initialWeights) +class IsotonicRegressionModel(Saveable, Loader): + + """Regression model for isotonic regression. + + >>> data = [(1, 0, 1), (2, 1, 1), (3, 2, 1), (1, 3, 1), (6, 4, 1), (17, 5, 1), (16, 6, 1)] + >>> irm = IsotonicRegression.train(sc.parallelize(data)) + >>> irm.predict(3) + 2.0 + >>> irm.predict(5) + 16.5 + >>> irm.predict(sc.parallelize([3, 5])).collect() + [2.0, 16.5] + >>> import os, tempfile + >>> path = tempfile.mkdtemp() + >>> irm.save(sc, path) + >>> sameModel = IsotonicRegressionModel.load(sc, path) + >>> sameModel.predict(3) + 2.0 + >>> sameModel.predict(5) + 16.5 + >>> try: + ... os.removedirs(path) + ... except OSError: + ... pass + """ + + def __init__(self, boundaries, predictions, isotonic): + self.boundaries = boundaries + self.predictions = predictions + self.isotonic = isotonic + + def predict(self, x): + if isinstance(x, RDD): + return x.map(lambda v: self.predict(v)) + return np.interp(x, self.boundaries, self.predictions) + + def save(self, sc, path): + java_boundaries = _py2java(sc, self.boundaries.tolist()) + java_predictions = _py2java(sc, self.predictions.tolist()) + java_model = sc._jvm.org.apache.spark.mllib.regression.IsotonicRegressionModel( + java_boundaries, java_predictions, self.isotonic) + java_model.save(sc._jsc.sc(), path) + + @classmethod + def load(cls, sc, path): + java_model = sc._jvm.org.apache.spark.mllib.regression.IsotonicRegressionModel.load( + sc._jsc.sc(), path) + py_boundaries = _java2py(sc, java_model.boundaryVector()).toArray() + py_predictions = _java2py(sc, java_model.predictionVector()).toArray() + return IsotonicRegressionModel(py_boundaries, py_predictions, java_model.isotonic) + + +class IsotonicRegression(object): + """ + Run IsotonicRegression algorithm to obtain isotonic regression model. + + :param data: RDD of (label, feature, weight) tuples. + :param isotonic: Whether this is isotonic or antitonic. + """ + @classmethod + def train(cls, data, isotonic=True): + """Train a isotonic regression model on the given data.""" + boundaries, predictions = callMLlibFunc("trainIsotonicRegressionModel", + data.map(_convert_to_vector), bool(isotonic)) + return IsotonicRegressionModel(boundaries.toArray(), predictions.toArray(), isotonic) + + def _test(): import doctest from pyspark import SparkContext diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index d05cfe2af04b2..36a4c7a5408c6 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -24,7 +24,7 @@ import tempfile import array as pyarray -from numpy import array, array_equal, zeros +from numpy import array, array_equal, zeros, inf from py4j.protocol import Py4JJavaError if sys.version_info[:2] <= (2, 6): @@ -220,6 +220,29 @@ def test_dense_matrix_is_transposed(self): self.assertTrue(array_equal(sm.colPtrs, [0, 2, 5])) self.assertTrue(array_equal(sm.values, [1, 3, 4, 6, 9])) + def test_parse_vector(self): + a = DenseVector([3, 4, 6, 7]) + self.assertTrue(str(a), '[3.0,4.0,6.0,7.0]') + self.assertTrue(Vectors.parse(str(a)), a) + a = SparseVector(4, [0, 2], [3, 4]) + self.assertTrue(str(a), '(4,[0,2],[3.0,4.0])') + self.assertTrue(Vectors.parse(str(a)), a) + a = SparseVector(10, [0, 1], [4, 5]) + self.assertTrue(SparseVector.parse(' (10, [0,1 ],[ 4.0,5.0] )'), a) + + def test_norms(self): + a = DenseVector([0, 2, 3, -1]) + self.assertAlmostEqual(a.norm(2), 3.742, 3) + self.assertTrue(a.norm(1), 6) + self.assertTrue(a.norm(inf), 3) + a = SparseVector(4, [0, 2], [3, -4]) + self.assertAlmostEqual(a.norm(2), 5) + self.assertTrue(a.norm(1), 7) + self.assertTrue(a.norm(inf), 4) + + tmp = SparseVector(4, [0, 2], [3, 0]) + self.assertEqual(tmp.numNonzeros(), 1) + class ListTests(MLlibTestCase): diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 24f370543def4..a9697999e82cb 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -1275,7 +1275,12 @@ def __init__(self, jc): # container operators __contains__ = _bin_op("contains") - __getitem__ = _bin_op("getItem") + __getitem__ = _bin_op("apply") + + # bitwise operators + bitwiseOR = _bin_op("bitwiseOR") + bitwiseAND = _bin_op("bitwiseAND") + bitwiseXOR = _bin_op("bitwiseXOR") def getItem(self, key): """An expression that gets an item at position `ordinal` out of a list, @@ -1303,19 +1308,19 @@ def getField(self, name): >>> from pyspark.sql import Row >>> df = sc.parallelize([Row(r=Row(a=1, b="b"))]).toDF() >>> df.select(df.r.getField("b")).show() - +---+ - |r.b| - +---+ - | b| - +---+ + +----+ + |r[b]| + +----+ + | b| + +----+ >>> df.select(df.r.a).show() - +---+ - |r.a| - +---+ - | 1| - +---+ + +----+ + |r[a]| + +----+ + | 1| + +----+ """ - return Column(self._jc.getField(name)) + return self[name] def __getattr__(self, item): if item.startswith("__"): diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 641220a264295..38a043a3c59d7 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -37,6 +37,7 @@ 'rand', 'randn', 'sparkPartitionId', + 'coalesce', 'udf'] @@ -51,6 +52,19 @@ def _(col): return _ +def _create_binary_mathfunction(name, doc=""): + """ Create a binary mathfunction by name""" + def _(col1, col2): + sc = SparkContext._active_spark_context + # users might write ints for simplicity. This would throw an error on the JVM side. + jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), + col2._jc if isinstance(col2, Column) else float(col2)) + return Column(jc) + _.__name__ = name + _.__doc__ = doc + return _ + + _functions = { 'lit': 'Creates a :class:`Column` of literal value.', 'col': 'Returns a :class:`Column` based on the given column name.', @@ -63,6 +77,36 @@ def _(col): 'sqrt': 'Computes the square root of the specified float value.', 'abs': 'Computes the absolute value.', + # unary math functions + 'acos': 'Computes the cosine inverse of the given value; the returned angle is in the range' + + '0.0 through pi.', + 'asin': 'Computes the sine inverse of the given value; the returned angle is in the range' + + '-pi/2 through pi/2.', + 'atan': 'Computes the tangent inverse of the given value.', + 'cbrt': 'Computes the cube-root of the given value.', + 'ceil': 'Computes the ceiling of the given value.', + 'cos': 'Computes the cosine of the given value.', + 'cosh': 'Computes the hyperbolic cosine of the given value.', + 'exp': 'Computes the exponential of the given value.', + 'expm1': 'Computes the exponential of the given value minus one.', + 'floor': 'Computes the floor of the given value.', + 'log': 'Computes the natural logarithm of the given value.', + 'log10': 'Computes the logarithm of the given value in Base 10.', + 'log1p': 'Computes the natural logarithm of the given value plus one.', + 'rint': 'Returns the double value that is closest in value to the argument and' + + ' is equal to a mathematical integer.', + 'signum': 'Computes the signum of the given value.', + 'sin': 'Computes the sine of the given value.', + 'sinh': 'Computes the hyperbolic sine of the given value.', + 'tan': 'Computes the tangent of the given value.', + 'tanh': 'Computes the hyperbolic tangent of the given value.', + 'toDegrees': 'Converts an angle measured in radians to an approximately equivalent angle ' + + 'measured in degrees.', + 'toRadians': 'Converts an angle measured in degrees to an approximately equivalent angle ' + + 'measured in radians.', + + 'bitwiseNOT': 'Computes bitwise not.', + 'max': 'Aggregate function: returns the maximum value of the expression in a group.', 'min': 'Aggregate function: returns the minimum value of the expression in a group.', 'first': 'Aggregate function: returns the first value in a group.', @@ -74,10 +118,21 @@ def _(col): 'sumDistinct': 'Aggregate function: returns the sum of distinct values in the expression.', } +# math functions that take two arguments as input +_binary_mathfunctions = { + 'atan2': 'Returns the angle theta from the conversion of rectangular coordinates (x, y) to' + + 'polar coordinates (r, theta).', + 'hypot': 'Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow.', + 'pow': 'Returns the value of the first argument raised to the power of the second argument.' +} + for _name, _doc in _functions.items(): globals()[_name] = _create_function(_name, _doc) +for _name, _doc in _binary_mathfunctions.items(): + globals()[_name] = _create_binary_mathfunction(_name, _doc) del _name, _doc __all__ += _functions.keys() +__all__ += _binary_mathfunctions.keys() __all__.sort() @@ -113,6 +168,42 @@ def approxCountDistinct(col, rsd=None): return Column(jc) +def coalesce(*cols): + """Returns the first column that is not null. + + >>> cDf = sqlContext.createDataFrame([(None, None), (1, None), (None, 2)], ("a", "b")) + >>> cDf.show() + +----+----+ + | a| b| + +----+----+ + |null|null| + | 1|null| + |null| 2| + +----+----+ + + >>> cDf.select(coalesce(cDf["a"], cDf["b"])).show() + +-------------+ + |Coalesce(a,b)| + +-------------+ + | null| + | 1| + | 2| + +-------------+ + + >>> cDf.select('*', coalesce(cDf["a"], lit(0.0))).show() + +----+----+---------------+ + | a| b|Coalesce(a,0.0)| + +----+----+---------------+ + |null|null| 0.0| + | 1|null| 1.0| + |null| 2| 0.0| + +----+----+---------------+ + """ + sc = SparkContext._active_spark_context + jc = sc._jvm.functions.coalesce(_to_seq(sc, cols, _to_java_column)) + return Column(jc) + + def countDistinct(col, *cols): """Returns a new :class:`Column` for distinct count of ``col`` or ``cols``. diff --git a/python/pyspark/sql/mathfunctions.py b/python/pyspark/sql/mathfunctions.py deleted file mode 100644 index 7dbcab8694293..0000000000000 --- a/python/pyspark/sql/mathfunctions.py +++ /dev/null @@ -1,101 +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. -# - -""" -A collection of builtin math functions -""" - -from pyspark import SparkContext -from pyspark.sql.dataframe import Column - -__all__ = [] - - -def _create_unary_mathfunction(name, doc=""): - """ Create a unary mathfunction by name""" - def _(col): - sc = SparkContext._active_spark_context - jc = getattr(sc._jvm.mathfunctions, name)(col._jc if isinstance(col, Column) else col) - return Column(jc) - _.__name__ = name - _.__doc__ = doc - return _ - - -def _create_binary_mathfunction(name, doc=""): - """ Create a binary mathfunction by name""" - def _(col1, col2): - sc = SparkContext._active_spark_context - # users might write ints for simplicity. This would throw an error on the JVM side. - if type(col1) is int: - col1 = col1 * 1.0 - if type(col2) is int: - col2 = col2 * 1.0 - jc = getattr(sc._jvm.mathfunctions, name)(col1._jc if isinstance(col1, Column) else col1, - col2._jc if isinstance(col2, Column) else col2) - return Column(jc) - _.__name__ = name - _.__doc__ = doc - return _ - - -# math functions are found under another object therefore, they need to be handled separately -_mathfunctions = { - 'acos': 'Computes the cosine inverse of the given value; the returned angle is in the range' + - '0.0 through pi.', - 'asin': 'Computes the sine inverse of the given value; the returned angle is in the range' + - '-pi/2 through pi/2.', - 'atan': 'Computes the tangent inverse of the given value.', - 'cbrt': 'Computes the cube-root of the given value.', - 'ceil': 'Computes the ceiling of the given value.', - 'cos': 'Computes the cosine of the given value.', - 'cosh': 'Computes the hyperbolic cosine of the given value.', - 'exp': 'Computes the exponential of the given value.', - 'expm1': 'Computes the exponential of the given value minus one.', - 'floor': 'Computes the floor of the given value.', - 'log': 'Computes the natural logarithm of the given value.', - 'log10': 'Computes the logarithm of the given value in Base 10.', - 'log1p': 'Computes the natural logarithm of the given value plus one.', - 'rint': 'Returns the double value that is closest in value to the argument and' + - ' is equal to a mathematical integer.', - 'signum': 'Computes the signum of the given value.', - 'sin': 'Computes the sine of the given value.', - 'sinh': 'Computes the hyperbolic sine of the given value.', - 'tan': 'Computes the tangent of the given value.', - 'tanh': 'Computes the hyperbolic tangent of the given value.', - 'toDeg': 'Converts an angle measured in radians to an approximately equivalent angle ' + - 'measured in degrees.', - 'toRad': 'Converts an angle measured in degrees to an approximately equivalent angle ' + - 'measured in radians.' -} - -# math functions that take two arguments as input -_binary_mathfunctions = { - 'atan2': 'Returns the angle theta from the conversion of rectangular coordinates (x, y) to' + - 'polar coordinates (r, theta).', - 'hypot': 'Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow.', - 'pow': 'Returns the value of the first argument raised to the power of the second argument.' -} - -for _name, _doc in _mathfunctions.items(): - globals()[_name] = _create_unary_mathfunction(_name, _doc) -for _name, _doc in _binary_mathfunctions.items(): - globals()[_name] = _create_binary_mathfunction(_name, _doc) -del _name, _doc -__all__ += _mathfunctions.keys() -__all__ += _binary_mathfunctions.keys() -__all__.sort() diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 46c4c88e98eb1..7e63f4d6461f6 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -416,7 +416,7 @@ def test_crosstab(self): def test_math_functions(self): df = self.sc.parallelize([Row(a=i, b=2 * i) for i in range(10)]).toDF() - from pyspark.sql import mathfunctions as functions + from pyspark.sql import functions import math def get_values(l): @@ -519,6 +519,13 @@ def test_access_nested_types(self): self.assertEqual("v", df.select(df.d["k"]).first()[0]) self.assertEqual("v", df.select(df.d.getItem("k")).first()[0]) + def test_field_accessor(self): + df = self.sc.parallelize([Row(l=[1], r=Row(a=1, b="b"), d={"k": "v"})]).toDF() + self.assertEqual(1, df.select(df.l[0]).first()[0]) + self.assertEqual(1, df.select(df.r["a"]).first()[0]) + self.assertEqual("b", df.select(df.r["b"]).first()[0]) + self.assertEqual("v", df.select(df.d["k"]).first()[0]) + def test_infer_long_type(self): longrow = [Row(f1='a', f2=100000000000000)] df = self.sc.parallelize(longrow).toDF() @@ -645,6 +652,19 @@ def test_fillna(self): self.assertEqual(row.age, None) self.assertEqual(row.height, None) + def test_bitwise_operations(self): + from pyspark.sql import functions + row = Row(a=170, b=75) + df = self.sqlCtx.createDataFrame([row]) + result = df.select(df.a.bitwiseAND(df.b)).collect()[0].asDict() + self.assertEqual(170 & 75, result['(a & b)']) + result = df.select(df.a.bitwiseOR(df.b)).collect()[0].asDict() + self.assertEqual(170 | 75, result['(a | b)']) + result = df.select(df.a.bitwiseXOR(df.b)).collect()[0].asDict() + self.assertEqual(170 ^ 75, result['(a ^ b)']) + result = df.select(functions.bitwiseNOT(df.b)).collect()[0].asDict() + self.assertEqual(~75, result['~b']) + class HiveContextSQLTests(ReusedPySparkTestCase): diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 8dc0e0c965923..488f3a9f33256 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -1028,7 +1028,7 @@ class SparkILoop( logInfo("Created sql context (with Hive support)..") } catch { - case cnf: java.lang.ClassNotFoundException => + case _: java.lang.ClassNotFoundException | _: java.lang.NoClassDefFoundError => sqlContext = new SQLContext(sparkContext) logInfo("Created sql context..") } diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index 2210fbaafeadb..f4f4b626988e9 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -88,7 +88,7 @@ object Main extends Logging { logInfo("Created sql context (with Hive support)..") } catch { - case cnf: java.lang.ClassNotFoundException => + case _: java.lang.ClassNotFoundException | _: java.lang.NoClassDefFoundError => sqlContext = new SQLContext(sparkContext) logInfo("Created sql context..") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index c52965507c715..6998cc8d9666d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.types._ */ object ScalaReflection extends ScalaReflection { val universe: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe + val mirror: universe.Mirror = universe.runtimeMirror(Thread.currentThread().getContextClassLoader) } /** @@ -36,6 +37,9 @@ trait ScalaReflection { /** The universe we work in (runtime or macro) */ val universe: scala.reflect.api.Universe + /** The mirror used to access types in the universe */ + val mirror: universe.Mirror + import universe._ // The Predef.Map is scala.collection.immutable.Map. @@ -52,7 +56,19 @@ trait ScalaReflection { /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ def schemaFor[T: TypeTag]: Schema = - ScalaReflectionLock.synchronized { schemaFor(typeOf[T]) } + ScalaReflectionLock.synchronized { schemaFor(localTypeOf[T]) } + + /** + * Return the Scala Type for `T` in the current classloader mirror. + * + * Use this method instead of the convenience method `universe.typeOf`, which + * assumes that all types can be found in the classloader that loaded scala-reflect classes. + * That's not necessarily the case when running using Eclipse launchers or even + * Sbt console or test (without `fork := true`). + * + * @see SPARK-5281 + */ + private def localTypeOf[T: TypeTag]: `Type` = typeTag[T].in(mirror).tpe /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ def schemaFor(tpe: `Type`): Schema = ScalaReflectionLock.synchronized { @@ -67,25 +83,25 @@ trait ScalaReflection { val udt = Utils.classForName(className) .getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() Schema(udt, nullable = true) - case t if t <:< typeOf[Option[_]] => + case t if t <:< localTypeOf[Option[_]] => val TypeRef(_, _, Seq(optType)) = t Schema(schemaFor(optType).dataType, nullable = true) // Need to decide if we actually need a special type here. - case t if t <:< typeOf[Array[Byte]] => Schema(BinaryType, nullable = true) - case t if t <:< typeOf[Array[_]] => + case t if t <:< localTypeOf[Array[Byte]] => Schema(BinaryType, nullable = true) + case t if t <:< localTypeOf[Array[_]] => val TypeRef(_, _, Seq(elementType)) = t val Schema(dataType, nullable) = schemaFor(elementType) Schema(ArrayType(dataType, containsNull = nullable), nullable = true) - case t if t <:< typeOf[Seq[_]] => + case t if t <:< localTypeOf[Seq[_]] => val TypeRef(_, _, Seq(elementType)) = t val Schema(dataType, nullable) = schemaFor(elementType) Schema(ArrayType(dataType, containsNull = nullable), nullable = true) - case t if t <:< typeOf[Map[_, _]] => + case t if t <:< localTypeOf[Map[_, _]] => val TypeRef(_, _, Seq(keyType, valueType)) = t val Schema(valueDataType, valueNullable) = schemaFor(valueType) Schema(MapType(schemaFor(keyType).dataType, valueDataType, valueContainsNull = valueNullable), nullable = true) - case t if t <:< typeOf[Product] => + case t if t <:< localTypeOf[Product] => val formalTypeArgs = t.typeSymbol.asClass.typeParams val TypeRef(_, _, actualTypeArgs) = t val constructorSymbol = t.member(nme.CONSTRUCTOR) @@ -107,19 +123,20 @@ trait ScalaReflection { schemaFor(p.typeSignature.substituteTypes(formalTypeArgs, actualTypeArgs)) StructField(p.name.toString, dataType, nullable) }), nullable = true) - case t if t <:< typeOf[String] => Schema(StringType, nullable = true) - case t if t <:< typeOf[java.sql.Timestamp] => Schema(TimestampType, nullable = true) - case t if t <:< typeOf[java.sql.Date] => Schema(DateType, nullable = true) - case t if t <:< typeOf[BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) - case t if t <:< typeOf[java.math.BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) - case t if t <:< typeOf[Decimal] => Schema(DecimalType.Unlimited, nullable = true) - case t if t <:< typeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) - case t if t <:< typeOf[java.lang.Long] => Schema(LongType, nullable = true) - case t if t <:< typeOf[java.lang.Double] => Schema(DoubleType, nullable = true) - case t if t <:< typeOf[java.lang.Float] => Schema(FloatType, nullable = true) - case t if t <:< typeOf[java.lang.Short] => Schema(ShortType, nullable = true) - case t if t <:< typeOf[java.lang.Byte] => Schema(ByteType, nullable = true) - case t if t <:< typeOf[java.lang.Boolean] => Schema(BooleanType, nullable = true) + case t if t <:< localTypeOf[String] => Schema(StringType, nullable = true) + case t if t <:< localTypeOf[java.sql.Timestamp] => Schema(TimestampType, nullable = true) + case t if t <:< localTypeOf[java.sql.Date] => Schema(DateType, nullable = true) + case t if t <:< localTypeOf[BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) + case t if t <:< localTypeOf[java.math.BigDecimal] => + Schema(DecimalType.Unlimited, nullable = true) + case t if t <:< localTypeOf[Decimal] => Schema(DecimalType.Unlimited, nullable = true) + case t if t <:< localTypeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) + case t if t <:< localTypeOf[java.lang.Long] => Schema(LongType, nullable = true) + case t if t <:< localTypeOf[java.lang.Double] => Schema(DoubleType, nullable = true) + case t if t <:< localTypeOf[java.lang.Float] => Schema(FloatType, nullable = true) + case t if t <:< localTypeOf[java.lang.Short] => Schema(ShortType, nullable = true) + case t if t <:< localTypeOf[java.lang.Byte] => Schema(ByteType, nullable = true) + case t if t <:< localTypeOf[java.lang.Boolean] => Schema(BooleanType, nullable = true) case t if t <:< definitions.IntTpe => Schema(IntegerType, nullable = false) case t if t <:< definitions.LongTpe => Schema(LongType, nullable = false) case t if t <:< definitions.DoubleTpe => Schema(DoubleType, nullable = false) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 1d3a2dc0d9bb0..fc36b9f1f20d2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -296,13 +296,13 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { | LOWER ~ "(" ~> expression <~ ")" ^^ { case exp => Lower(exp) } | IF ~ "(" ~> expression ~ ("," ~> expression) ~ ("," ~> expression) <~ ")" ^^ { case c ~ t ~ f => If(c, t, f) } - | CASE ~> expression.? ~ (WHEN ~> expression ~ (THEN ~> expression)).* ~ + | CASE ~> expression.? ~ rep1(WHEN ~> expression ~ (THEN ~> expression)) ~ (ELSE ~> expression).? <~ END ^^ { case casePart ~ altPart ~ elsePart => - val altExprs = altPart.flatMap { case whenExpr ~ thenExpr => - Seq(casePart.fold(whenExpr)(EqualTo(_, whenExpr)), thenExpr) - } - CaseWhen(altExprs ++ elsePart.toList) + val branches = altPart.flatMap { case whenExpr ~ thenExpr => + Seq(whenExpr, thenExpr) + } ++ elsePart + casePart.map(CaseKeyWhen(_, branches)).getOrElse(CaseWhen(branches)) } | (SUBSTR | SUBSTRING) ~ "(" ~> expression ~ ("," ~> expression) <~ ")" ^^ { case s ~ p => Substring(s, p, Literal(Integer.MAX_VALUE)) } @@ -375,9 +375,9 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { protected lazy val primary: PackratParser[Expression] = ( literal | expression ~ ("[" ~> expression <~ "]") ^^ - { case base ~ ordinal => GetItem(base, ordinal) } + { case base ~ ordinal => UnresolvedExtractValue(base, ordinal) } | (expression <~ ".") ~ ident ^^ - { case base ~ fieldName => UnresolvedGetField(base, fieldName) } + { case base ~ fieldName => UnresolvedExtractValue(base, Literal(fieldName)) } | cast | "(" ~> expression <~ ")" | function diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 5e42b409dcc59..ecbac57ea4d62 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.analysis +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ @@ -53,6 +55,10 @@ class Analyzer( val extendedResolutionRules: Seq[Rule[LogicalPlan]] = Nil lazy val batches: Seq[Batch] = Seq( + Batch("Substitution", fixedPoint, + CTESubstitution :: + WindowsSubstitution :: + Nil : _*), Batch("Resolution", fixedPoint, ResolveRelations :: ResolveReferences :: @@ -61,6 +67,7 @@ class Analyzer( ResolveGenerate :: ImplicitGenerate :: ResolveFunctions :: + ExtractWindowExpressions :: GlobalAggregates :: UnresolvedHavingClauseAttributes :: TrimGroupingAliases :: @@ -68,6 +75,55 @@ class Analyzer( extendedResolutionRules : _*) ) + /** + * Substitute child plan with cte definitions + */ + object CTESubstitution extends Rule[LogicalPlan] { + // TODO allow subquery to define CTE + def apply(plan: LogicalPlan): LogicalPlan = plan match { + case With(child, relations) => substituteCTE(child, relations) + case other => other + } + + def substituteCTE(plan: LogicalPlan, cteRelations: Map[String, LogicalPlan]): LogicalPlan = { + plan transform { + // In hive, if there is same table name in database and CTE definition, + // hive will use the table in database, not the CTE one. + // Taking into account the reasonableness and the implementation complexity, + // here use the CTE definition first, check table name only and ignore database name + // see https://github.com/apache/spark/pull/4929#discussion_r27186638 for more info + case u : UnresolvedRelation => + val substituted = cteRelations.get(u.tableIdentifier.last).map { relation => + val withAlias = u.alias.map(Subquery(_, relation)) + withAlias.getOrElse(relation) + } + substituted.getOrElse(u) + } + } + } + + /** + * Substitute child plan with WindowSpecDefinitions. + */ + object WindowsSubstitution extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // Lookup WindowSpecDefinitions. This rule works with unresolved children. + case WithWindowDefinition(windowDefinitions, child) => + child.transform { + case plan => plan.transformExpressions { + case UnresolvedWindowExpression(c, WindowSpecReference(windowName)) => + val errorMessage = + s"Window specification $windowName is not defined in the WINDOW clause." + val windowSpecDefinition = + windowDefinitions + .get(windowName) + .getOrElse(failAnalysis(errorMessage)) + WindowExpression(c, windowSpecDefinition) + } + } + } + } + /** * Removes no-op Alias expressions from the plan. */ @@ -169,36 +225,20 @@ class Analyzer( * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ object ResolveRelations extends Rule[LogicalPlan] { - def getTable(u: UnresolvedRelation, cteRelations: Map[String, LogicalPlan]): LogicalPlan = { + def getTable(u: UnresolvedRelation): LogicalPlan = { try { - // In hive, if there is same table name in database and CTE definition, - // hive will use the table in database, not the CTE one. - // Taking into account the reasonableness and the implementation complexity, - // here use the CTE definition first, check table name only and ignore database name - cteRelations.get(u.tableIdentifier.last) - .map(relation => u.alias.map(Subquery(_, relation)).getOrElse(relation)) - .getOrElse(catalog.lookupRelation(u.tableIdentifier, u.alias)) + catalog.lookupRelation(u.tableIdentifier, u.alias) } catch { case _: NoSuchTableException => u.failAnalysis(s"no such table ${u.tableName}") } } - def apply(plan: LogicalPlan): LogicalPlan = { - val (realPlan, cteRelations) = plan match { - // TODO allow subquery to define CTE - // Add cte table to a temp relation map,drop `with` plan and keep its child - case With(child, relations) => (child, relations) - case other => (other, Map.empty[String, LogicalPlan]) - } - - realPlan transform { - case i@InsertIntoTable(u: UnresolvedRelation, _, _, _, _) => - i.copy( - table = EliminateSubQueries(getTable(u, cteRelations))) - case u: UnresolvedRelation => - getTable(u, cteRelations) - } + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case i@InsertIntoTable(u: UnresolvedRelation, _, _, _, _) => + i.copy(table = EliminateSubQueries(getTable(u))) + case u: UnresolvedRelation => + getTable(u) } } @@ -308,8 +348,8 @@ class Analyzer( withPosition(u) { q.resolveChildren(nameParts, resolver).getOrElse(u) } logDebug(s"Resolving $u to $result") result - case UnresolvedGetField(child, fieldName) if child.resolved => - GetField(child, fieldName, resolver) + case UnresolvedExtractValue(child, fieldExpr) if child.resolved => + ExtractValue(child, fieldExpr, resolver) } } @@ -529,6 +569,187 @@ class Analyzer( makeGeneratorOutput(p.generator, p.generatorOutput), p.child) } } + + /** + * Extracts [[WindowExpression]]s from the projectList of a [[Project]] operator and + * aggregateExpressions of an [[Aggregate]] operator and creates individual [[Window]] + * operators for every distinct [[WindowSpecDefinition]]. + * + * This rule handles three cases: + * - A [[Project]] having [[WindowExpression]]s in its projectList; + * - An [[Aggregate]] having [[WindowExpression]]s in its aggregateExpressions. + * - An [[Filter]]->[[Aggregate]] pattern representing GROUP BY with a HAVING + * clause and the [[Aggregate]] has [[WindowExpression]]s in its aggregateExpressions. + * Note: If there is a GROUP BY clause in the query, aggregations and corresponding + * filters (expressions in the HAVING clause) should be evaluated before any + * [[WindowExpression]]. If a query has SELECT DISTINCT, the DISTINCT part should be + * evaluated after all [[WindowExpression]]s. + * + * For every case, the transformation works as follows: + * 1. For a list of [[Expression]]s (a projectList or an aggregateExpressions), partitions + * it two lists of [[Expression]]s, one for all [[WindowExpression]]s and another for + * all regular expressions. + * 2. For all [[WindowExpression]]s, groups them based on their [[WindowSpecDefinition]]s. + * 3. For every distinct [[WindowSpecDefinition]], creates a [[Window]] operator and inserts + * it into the plan tree. + */ + object ExtractWindowExpressions extends Rule[LogicalPlan] { + def hasWindowFunction(projectList: Seq[NamedExpression]): Boolean = + projectList.exists(hasWindowFunction) + + def hasWindowFunction(expr: NamedExpression): Boolean = { + expr.find { + case window: WindowExpression => true + case _ => false + }.isDefined + } + + /** + * From a Seq of [[NamedExpression]]s, extract window expressions and + * other regular expressions. + */ + def extract( + expressions: Seq[NamedExpression]): (Seq[NamedExpression], Seq[NamedExpression]) = { + // First, we simple partition the input expressions to two part, one having + // WindowExpressions and another one without WindowExpressions. + val (windowExpressions, regularExpressions) = expressions.partition(hasWindowFunction) + + // Then, we need to extract those regular expressions used in the WindowExpression. + // For example, when we have col1 - Sum(col2 + col3) OVER (PARTITION BY col4 ORDER BY col5), + // we need to make sure that col1 to col5 are all projected from the child of the Window + // operator. + val extractedExprBuffer = new ArrayBuffer[NamedExpression]() + def extractExpr(expr: Expression): Expression = expr match { + case ne: NamedExpression => + // If a named expression is not in regularExpressions, add extract it and replace it + // with an AttributeReference. + val missingExpr = + AttributeSet(Seq(expr)) -- (regularExpressions ++ extractedExprBuffer) + if (missingExpr.nonEmpty) { + extractedExprBuffer += ne + } + ne.toAttribute + case e: Expression if e.foldable => + e // No need to create an attribute reference if it will be evaluated as a Literal. + case e: Expression => + // For other expressions, we extract it and replace it with an AttributeReference (with + // an interal column name, e.g. "_w0"). + val withName = Alias(e, s"_w${extractedExprBuffer.length}")() + extractedExprBuffer += withName + withName.toAttribute + } + + // Now, we extract expressions from windowExpressions by using extractExpr. + val newWindowExpressions = windowExpressions.map { + _.transform { + // Extracts children expressions of a WindowFunction (input parameters of + // a WindowFunction). + case wf : WindowFunction => + val newChildren = wf.children.map(extractExpr(_)) + wf.withNewChildren(newChildren) + + // Extracts expressions from the partition spec and order spec. + case wsc @ WindowSpecDefinition(partitionSpec, orderSpec, _) => + val newPartitionSpec = partitionSpec.map(extractExpr(_)) + val newOrderSpec = orderSpec.map { so => + val newChild = extractExpr(so.child) + so.copy(child = newChild) + } + wsc.copy(partitionSpec = newPartitionSpec, orderSpec = newOrderSpec) + + // Extracts AggregateExpression. For example, for SUM(x) - Sum(y) OVER (...), + // we need to extract SUM(x). + case agg: AggregateExpression => + val withName = Alias(agg, s"_w${extractedExprBuffer.length}")() + extractedExprBuffer += withName + withName.toAttribute + }.asInstanceOf[NamedExpression] + } + + (newWindowExpressions, regularExpressions ++ extractedExprBuffer) + } + + /** + * Adds operators for Window Expressions. Every Window operator handles a single Window Spec. + */ + def addWindow(windowExpressions: Seq[NamedExpression], child: LogicalPlan): LogicalPlan = { + // First, we group window expressions based on their Window Spec. + val groupedWindowExpression = windowExpressions.groupBy { expr => + val windowSpec = expr.collectFirst { + case window: WindowExpression => window.windowSpec + } + windowSpec.getOrElse( + failAnalysis(s"$windowExpressions does not have any WindowExpression.")) + }.toSeq + + // For every Window Spec, we add a Window operator and set currentChild as the child of it. + var currentChild = child + var i = 0 + while (i < groupedWindowExpression.size) { + val (windowSpec, windowExpressions) = groupedWindowExpression(i) + // Set currentChild to the newly created Window operator. + currentChild = Window(currentChild.output, windowExpressions, windowSpec, currentChild) + + // Move to next WindowExpression. + i += 1 + } + + // We return the top operator. + currentChild + } + + // We have to use transformDown at here to make sure the rule of + // "Aggregate with Having clause" will be triggered. + def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + // Aggregate with Having clause. This rule works with an unresolved Aggregate because + // a resolved Aggregate will not have Window Functions. + case f @ Filter(condition, a @ Aggregate(groupingExprs, aggregateExprs, child)) + if child.resolved && + hasWindowFunction(aggregateExprs) && + a.expressions.forall(_.resolved) => + val (windowExpressions, aggregateExpressions) = extract(aggregateExprs) + // Create an Aggregate operator to evaluate aggregation functions. + val withAggregate = Aggregate(groupingExprs, aggregateExpressions, child) + // Add a Filter operator for conditions in the Having clause. + val withFilter = Filter(condition, withAggregate) + val withWindow = addWindow(windowExpressions, withFilter) + + // Finally, generate output columns according to the original projectList. + val finalProjectList = aggregateExprs.map (_.toAttribute) + Project(finalProjectList, withWindow) + + case p: LogicalPlan if !p.childrenResolved => p + + // Aggregate without Having clause. + case a @ Aggregate(groupingExprs, aggregateExprs, child) + if hasWindowFunction(aggregateExprs) && + a.expressions.forall(_.resolved) => + val (windowExpressions, aggregateExpressions) = extract(aggregateExprs) + // Create an Aggregate operator to evaluate aggregation functions. + val withAggregate = Aggregate(groupingExprs, aggregateExpressions, child) + // Add Window operators. + val withWindow = addWindow(windowExpressions, withAggregate) + + // Finally, generate output columns according to the original projectList. + val finalProjectList = aggregateExprs.map (_.toAttribute) + Project(finalProjectList, withWindow) + + // We only extract Window Expressions after all expressions of the Project + // have been resolved. + case p @ Project(projectList, child) + if hasWindowFunction(projectList) && !p.expressions.exists(!_.resolved) => + val (windowExpressions, regularExpressions) = extract(projectList) + // We add a project to get all needed expressions for window expressions from the child + // of the original Project operator. + val withProject = Project(regularExpressions, child) + // Add Window operators. + val withWindow = addWindow(windowExpressions, withProject) + + // Finally, generate output columns according to the original projectList. + val finalProjectList = projectList.map (_.toAttribute) + Project(finalProjectList, withWindow) + } + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 2381689e17525..c8288c6767004 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -70,6 +70,11 @@ trait CheckAnalysis { failAnalysis( s"invalid expression ${b.prettyString} " + s"between ${b.left.simpleString} and ${b.right.simpleString}") + + case w @ WindowExpression(windowFunction, windowSpec) if windowSpec.validate.nonEmpty => + // The window spec is not valid. + val reason = windowSpec.validate.get + failAnalysis(s"Window specification $windowSpec is not valid because $reason") } operator match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 831fb4fe95fe7..168a4e30eab86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -26,7 +26,14 @@ object HiveTypeCoercion { // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types. // The conversion for integral and floating point types have a linear widening hierarchy: private val numericPrecedence = - Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.Unlimited) + IndexedSeq( + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + DecimalType.Unlimited) /** * Find the tightest common type of two types that might be used in a binary expression. @@ -34,25 +41,21 @@ object HiveTypeCoercion { * with primitive types, because in that case the precision and scale of the result depends on * the operation. Those rules are implemented in [[HiveTypeCoercion.DecimalPrecision]]. */ - def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = { - val valueTypes = Seq(t1, t2).filter(t => t != NullType) - if (valueTypes.distinct.size > 1) { - // Promote numeric types to the highest of the two and all numeric types to unlimited decimal - if (numericPrecedence.contains(t1) && numericPrecedence.contains(t2)) { - Some(numericPrecedence.filter(t => t == t1 || t == t2).last) - } else if (t1.isInstanceOf[DecimalType] && t2.isInstanceOf[DecimalType]) { - // Fixed-precision decimals can up-cast into unlimited - if (t1 == DecimalType.Unlimited || t2 == DecimalType.Unlimited) { - Some(DecimalType.Unlimited) - } else { - None - } - } else { - None - } - } else { - Some(if (valueTypes.size == 0) NullType else valueTypes.head) - } + val findTightestCommonType: (DataType, DataType) => Option[DataType] = { + case (t1, t2) if t1 == t2 => Some(t1) + case (NullType, t1) => Some(t1) + case (t1, NullType) => Some(t1) + + // Promote numeric types to the highest of the two and all numeric types to unlimited decimal + case (t1, t2) if Seq(t1, t2).forall(numericPrecedence.contains) => + val index = numericPrecedence.lastIndexWhere(t => t == t1 || t == t2) + Some(numericPrecedence(index)) + + // Fixed-precision decimals can up-cast into unlimited + case (DecimalType.Unlimited, _: DecimalType) => Some(DecimalType.Unlimited) + case (_: DecimalType, DecimalType.Unlimited) => Some(DecimalType.Unlimited) + + case _ => None } } @@ -69,6 +72,7 @@ trait HiveTypeCoercion { val typeCoercionRules = PropagateTypes :: ConvertNaNs :: + InConversion :: WidenTypes :: PromoteStrings :: DecimalPrecision :: @@ -287,6 +291,16 @@ trait HiveTypeCoercion { } } + /** + * Convert all expressions in in() list to the left operator type + */ + object InConversion extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case i @ In(a, b) if b.exists(_.dataType != a.dataType) => + i.makeCopy(Array(a, b.map(Cast(_, a.dataType)))) + } + } + // scalastyle:off /** * Calculates and propagates precision for fixed-precision decimals. Hive has a number of @@ -617,31 +631,24 @@ trait HiveTypeCoercion { import HiveTypeCoercion._ def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case cw @ CaseWhen(branches) if !cw.resolved && !branches.exists(!_.resolved) => - val valueTypes = branches.sliding(2, 2).map { - case Seq(_, value) => value.dataType - case Seq(elseVal) => elseVal.dataType - }.toSeq - - logDebug(s"Input values for null casting ${valueTypes.mkString(",")}") - - if (valueTypes.distinct.size > 1) { - val commonType = valueTypes.reduce { (v1, v2) => - findTightestCommonType(v1, v2) - .getOrElse(sys.error( - s"Types in CASE WHEN must be the same or coercible to a common type: $v1 != $v2")) - } - val transformedBranches = branches.sliding(2, 2).map { - case Seq(cond, value) if value.dataType != commonType => - Seq(cond, Cast(value, commonType)) - case Seq(elseVal) if elseVal.dataType != commonType => - Seq(Cast(elseVal, commonType)) - case s => s - }.reduce(_ ++ _) - CaseWhen(transformedBranches) - } else { - // Types match up. Hopefully some other rule fixes whatever is wrong with resolution. - cw + case cw: CaseWhenLike if !cw.resolved && cw.childrenResolved && !cw.valueTypesEqual => + logDebug(s"Input values for null casting ${cw.valueTypes.mkString(",")}") + val commonType = cw.valueTypes.reduce { (v1, v2) => + findTightestCommonType(v1, v2).getOrElse(sys.error( + s"Types in CASE WHEN must be the same or coercible to a common type: $v1 != $v2")) + } + val transformedBranches = cw.branches.sliding(2, 2).map { + case Seq(when, value) if value.dataType != commonType => + Seq(when, Cast(value, commonType)) + case Seq(elseVal) if elseVal.dataType != commonType => + Seq(Cast(elseVal, commonType)) + case s => s + }.reduce(_ ++ _) + cw match { + case _: CaseWhen => + CaseWhen(transformedBranches) + case CaseKeyWhen(key, _) => + CaseKeyWhen(key, transformedBranches) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 3f567e3e8b2a6..2999c2ef3efe1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -95,7 +95,7 @@ case class UnresolvedFunction(name: String, children: Seq[Expression]) extends E * Represents all of the input attributes to a given relational operator, for example in * "SELECT * FROM ...". A [[Star]] gets automatically expanded during analysis. */ -trait Star extends Attribute with trees.LeafNode[Expression] { +trait Star extends NamedExpression with trees.LeafNode[Expression] { self: Product => override def name: String = throw new UnresolvedException(this, "name") @@ -103,13 +103,9 @@ trait Star extends Attribute with trees.LeafNode[Expression] { override def dataType: DataType = throw new UnresolvedException(this, "dataType") override def nullable: Boolean = throw new UnresolvedException(this, "nullable") override def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers") + override def toAttribute: Attribute = throw new UnresolvedException(this, "toAttribute") override lazy val resolved = false - override def newInstance(): Star = this - override def withNullability(newNullability: Boolean): Star = this - override def withQualifiers(newQualifiers: Seq[String]): Star = this - override def withName(newName: String): Star = this - // Star gets expanded at runtime so we never evaluate a Star. override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") @@ -154,7 +150,7 @@ case class UnresolvedStar(table: Option[String]) extends Star { * @param names the names to be associated with each output of computing [[child]]. */ case class MultiAlias(child: Expression, names: Seq[String]) - extends Attribute with trees.UnaryNode[Expression] { + extends NamedExpression with trees.UnaryNode[Expression] { override def name: String = throw new UnresolvedException(this, "name") @@ -166,15 +162,9 @@ case class MultiAlias(child: Expression, names: Seq[String]) override def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers") - override lazy val resolved = false - - override def newInstance(): MultiAlias = this - - override def withNullability(newNullability: Boolean): MultiAlias = this - - override def withQualifiers(newQualifiers: Seq[String]): MultiAlias = this + override def toAttribute: Attribute = throw new UnresolvedException(this, "toAttribute") - override def withName(newName: String): MultiAlias = this + override lazy val resolved = false override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") @@ -194,7 +184,17 @@ case class ResolvedStar(expressions: Seq[NamedExpression]) extends Star { override def toString: String = expressions.mkString("ResolvedStar(", ", ", ")") } -case class UnresolvedGetField(child: Expression, fieldName: String) extends UnaryExpression { +/** + * Extracts a value or values from an Expression + * + * @param child The expression to extract value from, + * can be Map, Array, Struct or array of Structs. + * @param extraction The expression to describe the extraction, + * can be key of Map, index of Array, field name of Struct. + */ +case class UnresolvedExtractValue(child: Expression, extraction: Expression) + extends UnaryExpression { + override def dataType: DataType = throw new UnresolvedException(this, "dataType") override def foldable: Boolean = throw new UnresolvedException(this, "foldable") override def nullable: Boolean = throw new UnresolvedException(this, "nullable") @@ -203,5 +203,5 @@ case class UnresolvedGetField(child: Expression, fieldName: String) extends Unar override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") - override def toString: String = s"$child.$fieldName" + override def toString: String = s"$child[$extraction]" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index fa6cc7a1a36cf..4c0d70203c6f5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -22,7 +22,7 @@ import java.sql.{Date, Timestamp} import scala.language.implicitConversions import scala.reflect.runtime.universe.{TypeTag, typeTag} -import org.apache.spark.sql.catalyst.analysis.{EliminateSubQueries, UnresolvedGetField, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.analysis.{EliminateSubQueries, UnresolvedExtractValue, UnresolvedAttribute} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} @@ -100,8 +100,9 @@ package object dsl { def isNull: Predicate = IsNull(expr) def isNotNull: Predicate = IsNotNull(expr) - def getItem(ordinal: Expression): Expression = GetItem(expr, ordinal) - def getField(fieldName: String): UnresolvedGetField = UnresolvedGetField(expr, fieldName) + def getItem(ordinal: Expression): UnresolvedExtractValue = UnresolvedExtractValue(expr, ordinal) + def getField(fieldName: String): UnresolvedExtractValue = + UnresolvedExtractValue(expr, Literal(fieldName)) def cast(to: DataType): Expression = Cast(expr, to) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 4fd1bc4dd642d..0837a3179d897 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -64,7 +64,7 @@ abstract class Expression extends TreeNode[Expression] { * Returns true if all the children of this expression have been resolved to a specific schema * and false if any still contains any unresolved placeholders. */ - def childrenResolved: Boolean = !children.exists(!_.resolved) + def childrenResolved: Boolean = children.forall(_.resolved) /** * Returns a string representation of this expression that does not have developer centric diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExtractValue.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExtractValue.scala new file mode 100644 index 0000000000000..e05926cbfe74b --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExtractValue.scala @@ -0,0 +1,206 @@ +/* + * 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.catalyst.expressions + +import scala.collection.Map + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.types._ + +object ExtractValue { + /** + * Returns the resolved `ExtractValue`. It will return one kind of concrete `ExtractValue`, + * depend on the type of `child` and `extraction`. + * + * `child` | `extraction` | concrete `ExtractValue` + * ---------------------------------------------------------------- + * Struct | Literal String | GetStructField + * Array[Struct] | Literal String | GetArrayStructFields + * Array | Integral type | GetArrayItem + * Map | Any type | GetMapValue + */ + def apply( + child: Expression, + extraction: Expression, + resolver: Resolver): ExtractValue = { + + (child.dataType, extraction) match { + case (StructType(fields), Literal(fieldName, StringType)) => + val ordinal = findField(fields, fieldName.toString, resolver) + GetStructField(child, fields(ordinal), ordinal) + case (ArrayType(StructType(fields), containsNull), Literal(fieldName, StringType)) => + val ordinal = findField(fields, fieldName.toString, resolver) + GetArrayStructFields(child, fields(ordinal), ordinal, containsNull) + case (_: ArrayType, _) if extraction.dataType.isInstanceOf[IntegralType] => + GetArrayItem(child, extraction) + case (_: MapType, _) => + GetMapValue(child, extraction) + case (otherType, _) => + val errorMsg = otherType match { + case StructType(_) | ArrayType(StructType(_), _) => + s"Field name should be String Literal, but it's $extraction" + case _: ArrayType => + s"Array index should be integral type, but it's ${extraction.dataType}" + case other => + s"Can't extract value from $child" + } + throw new AnalysisException(errorMsg) + } + } + + def unapply(g: ExtractValue): Option[(Expression, Expression)] = { + g match { + case o: ExtractValueWithOrdinal => Some((o.child, o.ordinal)) + case _ => Some((g.child, null)) + } + } + + /** + * Find the ordinal of StructField, report error if no desired field or over one + * desired fields are found. + */ + private def findField(fields: Array[StructField], fieldName: String, resolver: Resolver): Int = { + val checkField = (f: StructField) => resolver(f.name, fieldName) + val ordinal = fields.indexWhere(checkField) + if (ordinal == -1) { + throw new AnalysisException( + s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}") + } else if (fields.indexWhere(checkField, ordinal + 1) != -1) { + throw new AnalysisException( + s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}") + } else { + ordinal + } + } +} + +trait ExtractValue extends UnaryExpression { + self: Product => + + type EvaluatedType = Any +} + +/** + * Returns the value of fields in the Struct `child`. + */ +case class GetStructField(child: Expression, field: StructField, ordinal: Int) + extends ExtractValue { + + override def dataType: DataType = field.dataType + override def nullable: Boolean = child.nullable || field.nullable + override def foldable: Boolean = child.foldable + override def toString: String = s"$child.${field.name}" + + override def eval(input: Row): Any = { + val baseValue = child.eval(input).asInstanceOf[Row] + if (baseValue == null) null else baseValue(ordinal) + } +} + +/** + * Returns the array of value of fields in the Array of Struct `child`. + */ +case class GetArrayStructFields( + child: Expression, + field: StructField, + ordinal: Int, + containsNull: Boolean) extends ExtractValue { + + override def dataType: DataType = ArrayType(field.dataType, containsNull) + override def nullable: Boolean = child.nullable + override def foldable: Boolean = child.foldable + override def toString: String = s"$child.${field.name}" + + override def eval(input: Row): Any = { + val baseValue = child.eval(input).asInstanceOf[Seq[Row]] + if (baseValue == null) null else { + baseValue.map { row => + if (row == null) null else row(ordinal) + } + } + } +} + +abstract class ExtractValueWithOrdinal extends ExtractValue { + self: Product => + + def ordinal: Expression + + /** `Null` is returned for invalid ordinals. */ + override def nullable: Boolean = true + override def foldable: Boolean = child.foldable && ordinal.foldable + override def toString: String = s"$child[$ordinal]" + override def children: Seq[Expression] = child :: ordinal :: Nil + + override def eval(input: Row): Any = { + val value = child.eval(input) + if (value == null) { + null + } else { + val o = ordinal.eval(input) + if (o == null) { + null + } else { + evalNotNull(value, o) + } + } + } + + protected def evalNotNull(value: Any, ordinal: Any): Any +} + +/** + * Returns the field at `ordinal` in the Array `child` + */ +case class GetArrayItem(child: Expression, ordinal: Expression) + extends ExtractValueWithOrdinal { + + override def dataType: DataType = child.dataType.asInstanceOf[ArrayType].elementType + + override lazy val resolved = childrenResolved && + child.dataType.isInstanceOf[ArrayType] && ordinal.dataType.isInstanceOf[IntegralType] + + protected def evalNotNull(value: Any, ordinal: Any) = { + // TODO: consider using Array[_] for ArrayType child to avoid + // boxing of primitives + val baseValue = value.asInstanceOf[Seq[_]] + val index = ordinal.asInstanceOf[Int] + if (index >= baseValue.size || index < 0) { + null + } else { + baseValue(index) + } + } +} + +/** + * Returns the value of key `ordinal` in Map `child` + */ +case class GetMapValue(child: Expression, ordinal: Expression) + extends ExtractValueWithOrdinal { + + override def dataType: DataType = child.dataType.asInstanceOf[MapType].valueType + + override lazy val resolved = childrenResolved && child.dataType.isInstanceOf[MapType] + + protected def evalNotNull(value: Any, ordinal: Any) = { + val baseValue = value.asInstanceOf[Map[Any, _]] + baseValue.get(ordinal).orNull + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index c2866cd955409..8cae548279eb1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -548,3 +548,97 @@ class JoinedRow5 extends Row { } } } + +/** + * JIT HACK: Replace with macros + */ +class JoinedRow6 extends Row { + private[this] var row1: Row = _ + private[this] var row2: Row = _ + + def this(left: Row, right: Row) = { + this() + row1 = left + row2 = right + } + + /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ + def apply(r1: Row, r2: Row): Row = { + row1 = r1 + row2 = r2 + this + } + + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + + override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq + + override def length: Int = row1.length + row2.length + + override def apply(i: Int): Any = + if (i < row1.length) row1(i) else row2(i - row1.length) + + override def isNullAt(i: Int): Boolean = + if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length) + + override def getInt(i: Int): Int = + if (i < row1.length) row1.getInt(i) else row2.getInt(i - row1.length) + + override def getLong(i: Int): Long = + if (i < row1.length) row1.getLong(i) else row2.getLong(i - row1.length) + + override def getDouble(i: Int): Double = + if (i < row1.length) row1.getDouble(i) else row2.getDouble(i - row1.length) + + override def getBoolean(i: Int): Boolean = + if (i < row1.length) row1.getBoolean(i) else row2.getBoolean(i - row1.length) + + override def getShort(i: Int): Short = + if (i < row1.length) row1.getShort(i) else row2.getShort(i - row1.length) + + override def getByte(i: Int): Byte = + if (i < row1.length) row1.getByte(i) else row2.getByte(i - row1.length) + + override def getFloat(i: Int): Float = + if (i < row1.length) row1.getFloat(i) else row2.getFloat(i - row1.length) + + override def getString(i: Int): String = + if (i < row1.length) row1.getString(i) else row2.getString(i - row1.length) + + override def getAs[T](i: Int): T = + if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) + + override def copy(): Row = { + val totalSize = row1.length + row2.length + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } + + override def toString: String = { + // Make sure toString never throws NullPointerException. + if ((row1 eq null) && (row2 eq null)) { + "[ empty row ]" + } else if (row1 eq null) { + row2.mkString("[", ",", "]") + } else if (row2 eq null) { + row1.mkString("[", ",", "]") + } else { + mkString("[", ",", "]") + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index fc1f69655963d..956a2429b0b61 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -17,139 +17,8 @@ package org.apache.spark.sql.catalyst.expressions -import scala.collection.Map - -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.types._ -/** - * Returns the item at `ordinal` in the Array `child` or the Key `ordinal` in Map `child`. - */ -case class GetItem(child: Expression, ordinal: Expression) extends Expression { - type EvaluatedType = Any - - val children: Seq[Expression] = child :: ordinal :: Nil - /** `Null` is returned for invalid ordinals. */ - override def nullable: Boolean = true - override def foldable: Boolean = child.foldable && ordinal.foldable - - override def dataType: DataType = child.dataType match { - case ArrayType(dt, _) => dt - case MapType(_, vt, _) => vt - } - override lazy val resolved = - childrenResolved && - (child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType]) - - override def toString: String = s"$child[$ordinal]" - - override def eval(input: Row): Any = { - val value = child.eval(input) - if (value == null) { - null - } else { - val key = ordinal.eval(input) - if (key == null) { - null - } else { - if (child.dataType.isInstanceOf[ArrayType]) { - // TODO: consider using Array[_] for ArrayType child to avoid - // boxing of primitives - val baseValue = value.asInstanceOf[Seq[_]] - val o = key.asInstanceOf[Int] - if (o >= baseValue.size || o < 0) { - null - } else { - baseValue(o) - } - } else { - val baseValue = value.asInstanceOf[Map[Any, _]] - baseValue.get(key).orNull - } - } - } - } -} - - -trait GetField extends UnaryExpression { - self: Product => - - type EvaluatedType = Any - override def foldable: Boolean = child.foldable - override def toString: String = s"$child.${field.name}" - - def field: StructField -} - -object GetField { - /** - * Returns the resolved `GetField`, and report error if no desired field or over one - * desired fields are found. - */ - def apply( - expr: Expression, - fieldName: String, - resolver: Resolver): GetField = { - def findField(fields: Array[StructField]): Int = { - val checkField = (f: StructField) => resolver(f.name, fieldName) - val ordinal = fields.indexWhere(checkField) - if (ordinal == -1) { - throw new AnalysisException( - s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}") - } else if (fields.indexWhere(checkField, ordinal + 1) != -1) { - throw new AnalysisException( - s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}") - } else { - ordinal - } - } - expr.dataType match { - case StructType(fields) => - val ordinal = findField(fields) - StructGetField(expr, fields(ordinal), ordinal) - case ArrayType(StructType(fields), containsNull) => - val ordinal = findField(fields) - ArrayGetField(expr, fields(ordinal), ordinal, containsNull) - case otherType => - throw new AnalysisException(s"GetField is not valid on fields of type $otherType") - } - } -} - -/** - * Returns the value of fields in the Struct `child`. - */ -case class StructGetField(child: Expression, field: StructField, ordinal: Int) extends GetField { - - override def dataType: DataType = field.dataType - override def nullable: Boolean = child.nullable || field.nullable - - override def eval(input: Row): Any = { - val baseValue = child.eval(input).asInstanceOf[Row] - if (baseValue == null) null else baseValue(ordinal) - } -} - -/** - * Returns the array of value of fields in the Array of Struct `child`. - */ -case class ArrayGetField(child: Expression, field: StructField, ordinal: Int, containsNull: Boolean) - extends GetField { - - override def dataType: DataType = ArrayType(field.dataType, containsNull) - override def nullable: Boolean = child.nullable - - override def eval(input: Row): Any = { - val baseValue = child.eval(input).asInstanceOf[Seq[Row]] - if (baseValue == null) null else { - baseValue.map { row => - if (row == null) null else row(ordinal) - } - } - } -} /** * Returns an Array containing the evaluation of all children expressions. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index afcb2ce8b9cb4..a9170589f8c6c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -86,7 +86,7 @@ abstract class Attribute extends NamedExpression { def withQualifiers(newQualifiers: Seq[String]): Attribute def withName(newName: String): Attribute - def toAttribute: Attribute = this + override def toAttribute: Attribute = this def newInstance(): Attribute } @@ -171,6 +171,11 @@ case class AttributeReference( val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) extends Attribute with trees.LeafNode[Expression] { + /** + * Returns true iff the expression id is the same for both attributes. + */ + def sameRef(other: AttributeReference): Boolean = this.exprId == other.exprId + override def equals(other: Any): Boolean = other match { case ar: AttributeReference => name == ar.name && exprId == ar.exprId && dataType == ar.dataType case _ => false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 26c38c56c04f5..50b0f3ee5f93f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -353,79 +353,134 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi override def toString: String = s"if ($predicate) $trueValue else $falseValue" } +trait CaseWhenLike extends Expression { + self: Product => + + type EvaluatedType = Any + + // Note that `branches` are considered in consecutive pairs (cond, val), and the optional last + // element is the value for the default catch-all case (if provided). + // Hence, `branches` consists of at least two elements, and can have an odd or even length. + def branches: Seq[Expression] + + @transient lazy val whenList = + branches.sliding(2, 2).collect { case Seq(whenExpr, _) => whenExpr }.toSeq + @transient lazy val thenList = + branches.sliding(2, 2).collect { case Seq(_, thenExpr) => thenExpr }.toSeq + val elseValue = if (branches.length % 2 == 0) None else Option(branches.last) + + // both then and else val should be considered. + def valueTypes: Seq[DataType] = (thenList ++ elseValue).map(_.dataType) + def valueTypesEqual: Boolean = valueTypes.distinct.size <= 1 + + override def dataType: DataType = { + if (!resolved) { + throw new UnresolvedException(this, "cannot resolve due to differing types in some branches") + } + valueTypes.head + } + + override def nullable: Boolean = { + // If no value is nullable and no elseValue is provided, the whole statement defaults to null. + thenList.exists(_.nullable) || (elseValue.map(_.nullable).getOrElse(true)) + } +} + // scalastyle:off /** * Case statements of the form "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END". * Refer to this link for the corresponding semantics: * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions - * - * The other form of case statements "CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END" gets - * translated to this form at parsing time. Namely, such a statement gets translated to - * "CASE WHEN a=b THEN c [WHEN a=d THEN e]* [ELSE f] END". - * - * Note that `branches` are considered in consecutive pairs (cond, val), and the optional last - * element is the value for the default catch-all case (if provided). Hence, `branches` consists of - * at least two elements, and can have an odd or even length. */ // scalastyle:on -case class CaseWhen(branches: Seq[Expression]) extends Expression { - type EvaluatedType = Any +case class CaseWhen(branches: Seq[Expression]) extends CaseWhenLike { + + // Use private[this] Array to speed up evaluation. + @transient private[this] lazy val branchesArr = branches.toArray override def children: Seq[Expression] = branches - override def dataType: DataType = { - if (!resolved) { - throw new UnresolvedException(this, "cannot resolve due to differing types in some branches") + override lazy val resolved: Boolean = + childrenResolved && + whenList.forall(_.dataType == BooleanType) && + valueTypesEqual + + /** Written in imperative fashion for performance considerations. */ + override def eval(input: Row): Any = { + val len = branchesArr.length + var i = 0 + // If all branches fail and an elseVal is not provided, the whole statement + // defaults to null, according to Hive's semantics. + while (i < len - 1) { + if (branchesArr(i).eval(input) == true) { + return branchesArr(i + 1).eval(input) + } + i += 2 + } + var res: Any = null + if (i == len - 1) { + res = branchesArr(i).eval(input) } - branches(1).dataType + return res } + override def toString: String = { + "CASE" + branches.sliding(2, 2).map { + case Seq(cond, value) => s" WHEN $cond THEN $value" + case Seq(elseValue) => s" ELSE $elseValue" + }.mkString + } +} + +// scalastyle:off +/** + * Case statements of the form "CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END". + * Refer to this link for the corresponding semantics: + * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions + */ +// scalastyle:on +case class CaseKeyWhen(key: Expression, branches: Seq[Expression]) extends CaseWhenLike { + + // Use private[this] Array to speed up evaluation. @transient private[this] lazy val branchesArr = branches.toArray - @transient private[this] lazy val predicates = - branches.sliding(2, 2).collect { case Seq(cond, _) => cond }.toSeq - @transient private[this] lazy val values = - branches.sliding(2, 2).collect { case Seq(_, value) => value }.toSeq - @transient private[this] lazy val elseValue = - if (branches.length % 2 == 0) None else Option(branches.last) - override def nullable: Boolean = { - // If no value is nullable and no elseValue is provided, the whole statement defaults to null. - values.exists(_.nullable) || (elseValue.map(_.nullable).getOrElse(true)) - } + override def children: Seq[Expression] = key +: branches - override lazy val resolved: Boolean = { - if (!childrenResolved) { - false - } else { - val allCondBooleans = predicates.forall(_.dataType == BooleanType) - // both then and else val should be considered. - val dataTypesEqual = (values ++ elseValue).map(_.dataType).distinct.size <= 1 - allCondBooleans && dataTypesEqual - } - } + override lazy val resolved: Boolean = + childrenResolved && valueTypesEqual /** Written in imperative fashion for performance considerations. */ override def eval(input: Row): Any = { + val evaluatedKey = key.eval(input) val len = branchesArr.length var i = 0 // If all branches fail and an elseVal is not provided, the whole statement // defaults to null, according to Hive's semantics. - var res: Any = null while (i < len - 1) { - if (branchesArr(i).eval(input) == true) { - res = branchesArr(i + 1).eval(input) - return res + if (equalNullSafe(evaluatedKey, branchesArr(i).eval(input))) { + return branchesArr(i + 1).eval(input) } i += 2 } + var res: Any = null if (i == len - 1) { res = branchesArr(i).eval(input) } - res + return res + } + + private def equalNullSafe(l: Any, r: Any) = { + if (l == null && r == null) { + true + } else if (l == null || r == null) { + false + } else { + l == r + } } override def toString: String = { - "CASE" + branches.sliding(2, 2).map { + s"CASE $key" + branches.sliding(2, 2).map { case Seq(cond, value) => s" WHEN $cond THEN $value" case Seq(elseValue) => s" ELSE $elseValue" }.mkString diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala new file mode 100644 index 0000000000000..099d67ca7fee3 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -0,0 +1,340 @@ +/* + * 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.catalyst.expressions + +import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types.{NumericType, DataType} + +/** + * The trait of the Window Specification (specified in the OVER clause or WINDOW clause) for + * Window Functions. + */ +sealed trait WindowSpec + +/** + * The specification for a window function. + * @param partitionSpec It defines the way that input rows are partitioned. + * @param orderSpec It defines the ordering of rows in a partition. + * @param frameSpecification It defines the window frame in a partition. + */ +case class WindowSpecDefinition( + partitionSpec: Seq[Expression], + orderSpec: Seq[SortOrder], + frameSpecification: WindowFrame) extends Expression with WindowSpec { + + def validate: Option[String] = frameSpecification match { + case UnspecifiedFrame => + Some("Found a UnspecifiedFrame. It should be converted to a SpecifiedWindowFrame " + + "during analysis. Please file a bug report.") + case frame: SpecifiedWindowFrame => frame.validate.orElse { + def checkValueBasedBoundaryForRangeFrame(): Option[String] = { + if (orderSpec.length > 1) { + // It is not allowed to have a value-based PRECEDING and FOLLOWING + // as the boundary of a Range Window Frame. + Some("This Range Window Frame only accepts at most one ORDER BY expression.") + } else if (orderSpec.nonEmpty && !orderSpec.head.dataType.isInstanceOf[NumericType]) { + Some("The data type of the expression in the ORDER BY clause should be a numeric type.") + } else { + None + } + } + + (frame.frameType, frame.frameStart, frame.frameEnd) match { + case (RangeFrame, vp: ValuePreceding, _) => checkValueBasedBoundaryForRangeFrame() + case (RangeFrame, vf: ValueFollowing, _) => checkValueBasedBoundaryForRangeFrame() + case (RangeFrame, _, vp: ValuePreceding) => checkValueBasedBoundaryForRangeFrame() + case (RangeFrame, _, vf: ValueFollowing) => checkValueBasedBoundaryForRangeFrame() + case (_, _, _) => None + } + } + } + + type EvaluatedType = Any + + override def children: Seq[Expression] = partitionSpec ++ orderSpec + + override lazy val resolved: Boolean = + childrenResolved && frameSpecification.isInstanceOf[SpecifiedWindowFrame] + + + override def toString: String = simpleString + + override def eval(input: Row): EvaluatedType = throw new UnsupportedOperationException + override def nullable: Boolean = true + override def foldable: Boolean = false + override def dataType: DataType = throw new UnsupportedOperationException +} + +/** + * A Window specification reference that refers to the [[WindowSpecDefinition]] defined + * under the name `name`. + */ +case class WindowSpecReference(name: String) extends WindowSpec + +/** + * The trait used to represent the type of a Window Frame. + */ +sealed trait FrameType + +/** + * RowFrame treats rows in a partition individually. When a [[ValuePreceding]] + * or a [[ValueFollowing]] is used as its [[FrameBoundary]], the value is considered + * as a physical offset. + * For example, `ROW BETWEEN 1 PRECEDING AND 1 FOLLOWING` represents a 3-row frame, + * from the row precedes the current row to the row follows the current row. + */ +case object RowFrame extends FrameType + +/** + * RangeFrame treats rows in a partition as groups of peers. + * All rows having the same `ORDER BY` ordering are considered as peers. + * When a [[ValuePreceding]] or a [[ValueFollowing]] is used as its [[FrameBoundary]], + * the value is considered as a logical offset. + * For example, assuming the value of the current row's `ORDER BY` expression `expr` is `v`, + * `RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING` represents a frame containing rows whose values + * `expr` are in the range of [v-1, v+1]. + * + * If `ORDER BY` clause is not defined, all rows in the partition is considered as peers + * of the current row. + */ +case object RangeFrame extends FrameType + +/** + * The trait used to represent the type of a Window Frame Boundary. + */ +sealed trait FrameBoundary { + def notFollows(other: FrameBoundary): Boolean +} + +/** UNBOUNDED PRECEDING boundary. */ +case object UnboundedPreceding extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => true + case vp: ValuePreceding => true + case CurrentRow => true + case vf: ValueFollowing => true + case UnboundedFollowing => true + } + + override def toString: String = "UNBOUNDED PRECEDING" +} + +/** PRECEDING boundary. */ +case class ValuePreceding(value: Int) extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => false + case ValuePreceding(anotherValue) => value >= anotherValue + case CurrentRow => true + case vf: ValueFollowing => true + case UnboundedFollowing => true + } + + override def toString: String = s"$value PRECEDING" +} + +/** CURRENT ROW boundary. */ +case object CurrentRow extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => false + case vp: ValuePreceding => false + case CurrentRow => true + case vf: ValueFollowing => true + case UnboundedFollowing => true + } + + override def toString: String = "CURRENT ROW" +} + +/** FOLLOWING boundary. */ +case class ValueFollowing(value: Int) extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => false + case vp: ValuePreceding => false + case CurrentRow => false + case ValueFollowing(anotherValue) => value <= anotherValue + case UnboundedFollowing => true + } + + override def toString: String = s"$value FOLLOWING" +} + +/** UNBOUNDED FOLLOWING boundary. */ +case object UnboundedFollowing extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => false + case vp: ValuePreceding => false + case CurrentRow => false + case vf: ValueFollowing => false + case UnboundedFollowing => true + } + + override def toString: String = "UNBOUNDED FOLLOWING" +} + +/** + * The trait used to represent the a Window Frame. + */ +sealed trait WindowFrame + +/** Used as a place holder when a frame specification is not defined. */ +case object UnspecifiedFrame extends WindowFrame + +/** A specified Window Frame. */ +case class SpecifiedWindowFrame( + frameType: FrameType, + frameStart: FrameBoundary, + frameEnd: FrameBoundary) extends WindowFrame { + + /** If this WindowFrame is valid or not. */ + def validate: Option[String] = (frameType, frameStart, frameEnd) match { + case (_, UnboundedFollowing, _) => + Some(s"$UnboundedFollowing is not allowed as the start of a Window Frame.") + case (_, _, UnboundedPreceding) => + Some(s"$UnboundedPreceding is not allowed as the end of a Window Frame.") + // case (RowFrame, start, end) => ??? RowFrame specific rule + // case (RangeFrame, start, end) => ??? RangeFrame specific rule + case (_, start, end) => + if (start.notFollows(end)) { + None + } else { + val reason = + s"The end of this Window Frame $end is smaller than the start of " + + s"this Window Frame $start." + Some(reason) + } + } + + override def toString: String = frameType match { + case RowFrame => s"ROWS BETWEEN $frameStart AND $frameEnd" + case RangeFrame => s"RANGE BETWEEN $frameStart AND $frameEnd" + } +} + +object SpecifiedWindowFrame { + /** + * + * @param hasOrderSpecification If the window spec has order by expressions. + * @param acceptWindowFrame If the window function accepts user-specified frame. + * @return + */ + def defaultWindowFrame( + hasOrderSpecification: Boolean, + acceptWindowFrame: Boolean): SpecifiedWindowFrame = { + if (hasOrderSpecification && acceptWindowFrame) { + // If order spec is defined and the window function supports user specified window frames, + // the default frame is RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW. + SpecifiedWindowFrame(RangeFrame, UnboundedPreceding, CurrentRow) + } else { + // Otherwise, the default frame is + // ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING. + SpecifiedWindowFrame(RowFrame, UnboundedPreceding, UnboundedFollowing) + } + } +} + +/** + * Every window function needs to maintain a output buffer for its output. + * It should expect that for a n-row window frame, it will be called n times + * to retrieve value corresponding with these n rows. + */ +trait WindowFunction extends Expression { + self: Product => + + def init(): Unit + + def reset(): Unit + + def prepareInputParameters(input: Row): AnyRef + + def update(input: AnyRef): Unit + + def batchUpdate(inputs: Array[AnyRef]): Unit + + def evaluate(): Unit + + def get(index: Int): Any + + def newInstance(): WindowFunction +} + +case class UnresolvedWindowFunction( + name: String, + children: Seq[Expression]) + extends Expression with WindowFunction { + + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def foldable: Boolean = throw new UnresolvedException(this, "foldable") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") + override lazy val resolved = false + + override def init(): Unit = + throw new UnresolvedException(this, "init") + override def reset(): Unit = + throw new UnresolvedException(this, "reset") + override def prepareInputParameters(input: Row): AnyRef = + throw new UnresolvedException(this, "prepareInputParameters") + override def update(input: AnyRef): Unit = + throw new UnresolvedException(this, "update") + override def batchUpdate(inputs: Array[AnyRef]): Unit = + throw new UnresolvedException(this, "batchUpdate") + override def evaluate(): Unit = + throw new UnresolvedException(this, "evaluate") + override def get(index: Int): Any = + throw new UnresolvedException(this, "get") + // Unresolved functions are transient at compile time and don't get evaluated during execution. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + + override def toString: String = s"'$name(${children.mkString(",")})" + + override def newInstance(): WindowFunction = + throw new UnresolvedException(this, "newInstance") +} + +case class UnresolvedWindowExpression( + child: UnresolvedWindowFunction, + windowSpec: WindowSpecReference) extends UnaryExpression { + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def foldable: Boolean = throw new UnresolvedException(this, "foldable") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") + override lazy val resolved = false + + // Unresolved functions are transient at compile time and don't get evaluated during execution. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") +} + +case class WindowExpression( + windowFunction: WindowFunction, + windowSpec: WindowSpecDefinition) extends Expression { + override type EvaluatedType = Any + + override def children: Seq[Expression] = + windowFunction :: windowSpec :: Nil + + override def eval(input: Row): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + + override def dataType: DataType = windowFunction.dataType + override def foldable: Boolean = windowFunction.foldable + override def nullable: Boolean = windowFunction.nullable + + override def toString: String = s"$windowFunction $windowSpec" +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 709f7d672d931..d7b2f203a6934 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -227,10 +227,8 @@ object NullPropagation extends Rule[LogicalPlan] { case e @ Count(Literal(null, _)) => Cast(Literal(0L), e.dataType) case e @ IsNull(c) if !c.nullable => Literal.create(false, BooleanType) case e @ IsNotNull(c) if !c.nullable => Literal.create(true, BooleanType) - case e @ GetItem(Literal(null, _), _) => Literal.create(null, e.dataType) - case e @ GetItem(_, Literal(null, _)) => Literal.create(null, e.dataType) - case e @ StructGetField(Literal(null, _), _, _) => Literal.create(null, e.dataType) - case e @ ArrayGetField(Literal(null, _), _, _, _) => Literal.create(null, e.dataType) + case e @ ExtractValue(Literal(null, _), _) => Literal.create(null, e.dataType) + case e @ ExtractValue(_, Literal(null, _)) => Literal.create(null, e.dataType) case e @ EqualNullSafe(Literal(null, _), r) => IsNull(r) case e @ EqualNullSafe(l, Literal(null, _)) => IsNull(l) case e @ Count(expr) if !expr.nullable => Count(Literal(1)) @@ -310,8 +308,8 @@ object OptimizeIn extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsDown { case In(v, list) if !list.exists(!_.isInstanceOf[Literal]) => - val hSet = list.map(e => e.eval(null)) - InSet(v, HashSet() ++ hSet) + val hSet = list.map(e => e.eval(null)) + InSet(v, HashSet() ++ hSet) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 4574934d910db..cd54d04814ea4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -160,7 +160,7 @@ object PartialAggregation { // resolving struct field accesses, because `GetField` is not a `NamedExpression`. // (Should we just turn `GetField` into a `NamedExpression`?) namedGroupingExpressions - .get(e.transform { case Alias(g: GetField, _) => g }) + .get(e.transform { case Alias(g: ExtractValue, _) => g }) .map(_.toAttribute) .getOrElse(e) }).asInstanceOf[Seq[NamedExpression]] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index ae4620a4e5abf..dbb12d56f9497 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -209,7 +209,8 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // For example, consider "a.b.c", where "a" is resolved to an existing attribute. // Then this will add GetField("c", GetField("b", a)), and alias // the final expression as "c". - val fieldExprs = nestedFields.foldLeft(a: Expression)(GetField(_, _, resolver)) + val fieldExprs = nestedFields.foldLeft(a: Expression)((expr, fieldName) => + ExtractValue(expr, Literal(fieldName), resolver)) val aliasName = nestedFields.last Some(Alias(fieldExprs, aliasName)()) } catch { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 21208c8a5c281..0f349f9d11415 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -25,13 +25,14 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extend override def output: Seq[Attribute] = projectList.map(_.toAttribute) override lazy val resolved: Boolean = { - val containsAggregatesOrGenerators = projectList.exists ( _.collect { + val hasSpecialExpressions = projectList.exists ( _.collect { case agg: AggregateExpression => agg case generator: Generator => generator + case window: WindowExpression => window }.nonEmpty ) - !expressions.exists(!_.resolved) && childrenResolved && !containsAggregatesOrGenerators + !expressions.exists(!_.resolved) && childrenResolved && !hasSpecialExpressions } } @@ -148,16 +149,6 @@ case class InsertIntoTable( } } -case class CreateTableAsSelect[T]( - databaseName: Option[String], - tableName: String, - child: LogicalPlan, - allowExisting: Boolean, - desc: Option[T] = None) extends UnaryNode { - override def output: Seq[Attribute] = Seq.empty[Attribute] - override lazy val resolved: Boolean = databaseName != None && childrenResolved -} - /** * A container for holding named common table expressions (CTEs) and a query plan. * This operator will be removed during analysis and the relations will be substituted into child. @@ -170,6 +161,12 @@ case class With(child: LogicalPlan, cteRelations: Map[String, Subquery]) extends override def output: Seq[Attribute] = child.output } +case class WithWindowDefinition( + windowDefinitions: Map[String, WindowSpecDefinition], + child: LogicalPlan) extends UnaryNode { + override def output: Seq[Attribute] = child.output +} + case class WriteToFile( path: String, child: LogicalPlan) extends UnaryNode { @@ -177,10 +174,10 @@ case class WriteToFile( } /** - * @param order The ordering expressions - * @param global True means global sorting apply for entire data set, + * @param order The ordering expressions + * @param global True means global sorting apply for entire data set, * False means sorting only apply within the partition. - * @param child Child logical plan + * @param child Child logical plan */ case class Sort( order: Seq[SortOrder], @@ -195,9 +192,28 @@ case class Aggregate( child: LogicalPlan) extends UnaryNode { + override lazy val resolved: Boolean = { + val hasWindowExpressions = aggregateExpressions.exists ( _.collect { + case window: WindowExpression => window + }.nonEmpty + ) + + !expressions.exists(!_.resolved) && childrenResolved && !hasWindowExpressions + } + override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute) } +case class Window( + projectList: Seq[Attribute], + windowExpressions: Seq[NamedExpression], + windowSpec: WindowSpecDefinition, + child: LogicalPlan) extends UnaryNode { + + override def output: Seq[Attribute] = + (projectList ++ windowExpressions).map(_.toAttribute) +} + /** * Apply the all of the GroupExpressions to every input row, hence we will get * multiple output rows for a input row. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index 45905f8ef98c5..246f4d7e34d3d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -21,9 +21,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute /** * A logical node that represents a non-query command to be executed by the system. For example, - * commands can be used by parsers to represent DDL operations. + * commands can be used by parsers to represent DDL operations. Commands, unlike queries, are + * eagerly executed. */ -abstract class Command extends LeafNode { - self: Product => - def output: Seq[Attribute] = Seq.empty -} +trait Command diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 97502ed3afe72..bc2ad34523d2c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -71,6 +71,15 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { this.eq(other) || this == other } + /** + * Find the first [[TreeNode]] that satisfies the condition specified by `f`. + * The condition is recursively applied to this node and all of its children (pre-order). + */ + def find(f: BaseType => Boolean): Option[BaseType] = f(this) match { + case true => Some(this) + case false => children.foldLeft(None: Option[BaseType]) { (l, r) => l.orElse(r.find(f)) } + } + /** * Runs the given function on this node and then recursively on [[children]]. * @param f the function to be applied to each node in the tree. @@ -121,6 +130,17 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { ret } + /** + * Finds and returns the first [[TreeNode]] of the tree for which the given partial function + * is defined (pre-order), and applies the partial function to it. + */ + def collectFirst[B](pf: PartialFunction[BaseType, B]): Option[B] = { + val lifted = pf.lift + lifted(this).orElse { + children.foldLeft(None: Option[B]) { (l, r) => l.orElse(r.collectFirst(pf)) } + } + } + /** * Returns a copy of this node where `f` has been applied to all the nodes children. */ @@ -151,6 +171,20 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { val remainingNewChildren = newChildren.toBuffer val remainingOldChildren = children.toBuffer val newArgs = productIterator.map { + // Handle Seq[TreeNode] in TreeNode parameters. + case s: Seq[_] => s.map { + case arg: TreeNode[_] if children contains arg => + val newChild = remainingNewChildren.remove(0) + val oldChild = remainingOldChildren.remove(0) + if (newChild fastEquals oldChild) { + oldChild + } else { + changed = true + newChild + } + case nonChild: AnyRef => nonChild + case null => null + } case arg: TreeNode[_] if children contains arg => val newChild = remainingNewChildren.remove(0) val oldChild = remainingOldChildren.remove(0) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index d80ffca18ec9a..7e00a27dfe724 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -134,6 +134,10 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru throw new IllegalArgumentException(s"""Field "$name" does not exist.""")) } + private[sql] def getFieldIndex(name: String): Option[Int] = { + nameToIndex.get(name) + } + protected[sql] def toAttributes: Seq[AttributeReference] = map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala index a652c70560990..890ea2a84b82e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala @@ -17,11 +17,15 @@ package org.apache.spark.sql.catalyst +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.Command import org.scalatest.FunSuite -private[sql] case class TestCommand(cmd: String) extends Command +private[sql] case class TestCommand(cmd: String) extends LogicalPlan with Command { + override def output: Seq[Attribute] = Seq.empty + override def children: Seq[LogicalPlan] = Seq.empty +} private[sql] class SuperLongKeywordTestParser extends AbstractSparkSQLParser { protected val EXECUTE = Keyword("THISISASUPERLONGKEYWORDTEST") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index fa71001c9336e..04fd261d16aa3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -26,7 +26,7 @@ import org.scalatest.FunSuite import org.scalatest.Matchers._ import org.apache.spark.sql.catalyst.CatalystTypeConverters -import org.apache.spark.sql.catalyst.analysis.UnresolvedGetField +import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.mathfuncs._ import org.apache.spark.sql.types._ @@ -850,11 +850,37 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { assert(CaseWhen(Seq(c2, c4_notNull, c3, c5)).nullable === true) } + test("case key when") { + val row = create_row(null, 1, 2, "a", "b", "c") + val c1 = 'a.int.at(0) + val c2 = 'a.int.at(1) + val c3 = 'a.int.at(2) + val c4 = 'a.string.at(3) + val c5 = 'a.string.at(4) + val c6 = 'a.string.at(5) + + val literalNull = Literal.create(null, BooleanType) + val literalInt = Literal(1) + val literalString = Literal("a") + + checkEvaluation(CaseKeyWhen(c1, Seq(c2, c4, c5)), "b", row) + checkEvaluation(CaseKeyWhen(c1, Seq(c2, c4, literalNull, c5, c6)), "b", row) + checkEvaluation(CaseKeyWhen(c2, Seq(literalInt, c4, c5)), "a", row) + checkEvaluation(CaseKeyWhen(c2, Seq(c1, c4, c5)), "b", row) + checkEvaluation(CaseKeyWhen(c4, Seq(literalString, c2, c3)), 1, row) + checkEvaluation(CaseKeyWhen(c4, Seq(c1, c3, c5, c2, Literal(3))), 3, row) + + checkEvaluation(CaseKeyWhen(literalInt, Seq(c2, c4, c5)), "a", row) + checkEvaluation(CaseKeyWhen(literalString, Seq(c5, c2, c4, c3)), 2, row) + checkEvaluation(CaseKeyWhen(literalInt, Seq(c5, c2, c4, c3)), null, row) + checkEvaluation(CaseKeyWhen(literalNull, Seq(c5, c2, c1, c3)), 2, row) + } + test("complex type") { val row = create_row( "^Ba*n", // 0 null.asInstanceOf[UTF8String], // 1 - create_row("aa", "bb"), // 2 + create_row("aa", "bb"), // 2 Map("aa"->"bb"), // 3 Seq("aa", "bb") // 4 ) @@ -865,54 +891,79 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { val typeMap = MapType(StringType, StringType) val typeArray = ArrayType(StringType) - checkEvaluation(GetItem(BoundReference(3, typeMap, true), + checkEvaluation(GetMapValue(BoundReference(3, typeMap, true), Literal("aa")), "bb", row) - checkEvaluation(GetItem(Literal.create(null, typeMap), Literal("aa")), null, row) + checkEvaluation(GetMapValue(Literal.create(null, typeMap), Literal("aa")), null, row) checkEvaluation( - GetItem(Literal.create(null, typeMap), Literal.create(null, StringType)), null, row) - checkEvaluation(GetItem(BoundReference(3, typeMap, true), + GetMapValue(Literal.create(null, typeMap), Literal.create(null, StringType)), null, row) + checkEvaluation(GetMapValue(BoundReference(3, typeMap, true), Literal.create(null, StringType)), null, row) - checkEvaluation(GetItem(BoundReference(4, typeArray, true), + checkEvaluation(GetArrayItem(BoundReference(4, typeArray, true), Literal(1)), "bb", row) - checkEvaluation(GetItem(Literal.create(null, typeArray), Literal(1)), null, row) + checkEvaluation(GetArrayItem(Literal.create(null, typeArray), Literal(1)), null, row) checkEvaluation( - GetItem(Literal.create(null, typeArray), Literal.create(null, IntegerType)), null, row) - checkEvaluation(GetItem(BoundReference(4, typeArray, true), + GetArrayItem(Literal.create(null, typeArray), Literal.create(null, IntegerType)), null, row) + checkEvaluation(GetArrayItem(BoundReference(4, typeArray, true), Literal.create(null, IntegerType)), null, row) - def quickBuildGetField(expr: Expression, fieldName: String): StructGetField = { + def getStructField(expr: Expression, fieldName: String): ExtractValue = { expr.dataType match { case StructType(fields) => val field = fields.find(_.name == fieldName).get - StructGetField(expr, field, fields.indexOf(field)) + GetStructField(expr, field, fields.indexOf(field)) } } - def quickResolve(u: UnresolvedGetField): StructGetField = { - quickBuildGetField(u.child, u.fieldName) + def quickResolve(u: UnresolvedExtractValue): ExtractValue = { + ExtractValue(u.child, u.extraction, _ == _) } - checkEvaluation(quickBuildGetField(BoundReference(2, typeS, nullable = true), "a"), "aa", row) - checkEvaluation(quickBuildGetField(Literal.create(null, typeS), "a"), null, row) + checkEvaluation(getStructField(BoundReference(2, typeS, nullable = true), "a"), "aa", row) + checkEvaluation(getStructField(Literal.create(null, typeS), "a"), null, row) val typeS_notNullable = StructType( StructField("a", StringType, nullable = false) :: StructField("b", StringType, nullable = false) :: Nil ) - assert(quickBuildGetField(BoundReference(2,typeS, nullable = true), "a").nullable === true) - assert(quickBuildGetField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable + assert(getStructField(BoundReference(2,typeS, nullable = true), "a").nullable === true) + assert(getStructField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable === false) - assert(quickBuildGetField(Literal.create(null, typeS), "a").nullable === true) - assert(quickBuildGetField(Literal.create(null, typeS_notNullable), "a").nullable === true) + assert(getStructField(Literal.create(null, typeS), "a").nullable === true) + assert(getStructField(Literal.create(null, typeS_notNullable), "a").nullable === true) - checkEvaluation('c.map(typeMap).at(3).getItem("aa"), "bb", row) - checkEvaluation('c.array(typeArray.elementType).at(4).getItem(1), "bb", row) + checkEvaluation(quickResolve('c.map(typeMap).at(3).getItem("aa")), "bb", row) + checkEvaluation(quickResolve('c.array(typeArray.elementType).at(4).getItem(1)), "bb", row) checkEvaluation(quickResolve('c.struct(typeS).at(2).getField("a")), "aa", row) } + test("error message of ExtractValue") { + val structType = StructType(StructField("a", StringType, true) :: Nil) + val arrayStructType = ArrayType(structType) + val arrayType = ArrayType(StringType) + val otherType = StringType + + def checkErrorMessage( + childDataType: DataType, + fieldDataType: DataType, + errorMesage: String): Unit = { + val e = intercept[org.apache.spark.sql.AnalysisException] { + ExtractValue( + Literal.create(null, childDataType), + Literal.create(null, fieldDataType), + _ == _) + } + assert(e.getMessage().contains(errorMesage)) + } + + checkErrorMessage(structType, IntegerType, "Field name should be String Literal") + checkErrorMessage(arrayStructType, BooleanType, "Field name should be String Literal") + checkErrorMessage(arrayType, StringType, "Array index should be integral type") + checkErrorMessage(otherType, StringType, "Can't extract value from") + } + test("arithmetic") { val row = create_row(1, 2, 3, null) val c1 = 'a.int.at(0) @@ -1217,11 +1268,11 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { unaryMathFunctionEvaluation(Tanh, math.tanh) } - test("toDeg") { + test("toDegrees") { unaryMathFunctionEvaluation(ToDegrees, math.toDegrees) } - test("toRad") { + test("toRadians") { unaryMathFunctionEvaluation(ToRadians, math.toRadians) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index 18f92150b0966..6b7d9a85c341b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.analysis.{UnresolvedGetField, EliminateSubQueries} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedExtractValue, EliminateSubQueries} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.plans.PlanTest @@ -180,10 +180,10 @@ class ConstantFoldingSuite extends PlanTest { IsNull(Literal(null)) as 'c1, IsNotNull(Literal(null)) as 'c2, - GetItem(Literal.create(null, ArrayType(IntegerType)), 1) as 'c3, - GetItem( + UnresolvedExtractValue(Literal.create(null, ArrayType(IntegerType)), 1) as 'c3, + UnresolvedExtractValue( Literal.create(Seq(1), ArrayType(IntegerType)), Literal.create(null, IntegerType)) as 'c4, - UnresolvedGetField( + UnresolvedExtractValue( Literal.create(null, StructType(Seq(StructField("a", IntegerType, true)))), "a") as 'c5, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 6b393327cc97a..3d10dab5ba34c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{StringType, NullType} +import org.apache.spark.sql.types.{IntegerType, StringType, NullType} case class Dummy(optKey: Option[Expression]) extends Expression { def children: Seq[Expression] = optKey.toSeq @@ -129,5 +129,97 @@ class TreeNodeSuite extends FunSuite { assert(expected === actual) } + test("find") { + val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) + // Find the top node. + var actual: Option[Expression] = expression.find { + case add: Add => true + case other => false + } + var expected: Option[Expression] = + Some(Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4))))) + assert(expected === actual) + + // Find the first children. + actual = expression.find { + case Literal(1, IntegerType) => true + case other => false + } + expected = Some(Literal(1)) + assert(expected === actual) + + // Find an internal node (Subtract). + actual = expression.find { + case sub: Subtract => true + case other => false + } + expected = Some(Subtract(Literal(3), Literal(4))) + assert(expected === actual) + + // Find a leaf node. + actual = expression.find { + case Literal(3, IntegerType) => true + case other => false + } + expected = Some(Literal(3)) + assert(expected === actual) + + // Find nothing. + actual = expression.find { + case Literal(100, IntegerType) => true + case other => false + } + expected = None + assert(expected === actual) + } + + test("collectFirst") { + val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) + + // Collect the top node. + { + val actual = expression.collectFirst { + case add: Add => add + } + val expected = + Some(Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4))))) + assert(expected === actual) + } + + // Collect the first children. + { + val actual = expression.collectFirst { + case l @ Literal(1, IntegerType) => l + } + val expected = Some(Literal(1)) + assert(expected === actual) + } + + // Collect an internal node (Subtract). + { + val actual = expression.collectFirst { + case sub: Subtract => sub + } + val expected = Some(Subtract(Literal(3), Literal(4))) + assert(expected === actual) + } + // Collect a leaf node. + { + val actual = expression.collectFirst { + case l @ Literal(3, IntegerType) => l + } + val expected = Some(Literal(3)) + assert(expected === actual) + } + + // Collect nothing. + { + val actual = expression.collectFirst { + case l @ Literal(100, IntegerType) => l + } + val expected = None + assert(expected === actual) + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index c0503bf047052..e6e475bb82f82 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -23,7 +23,7 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.Logging import org.apache.spark.sql.functions.lit import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedStar, UnresolvedGetField} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedStar, UnresolvedExtractValue} import org.apache.spark.sql.types._ @@ -67,6 +67,19 @@ class Column(protected[sql] val expr: Expression) extends Logging { override def hashCode: Int = this.expr.hashCode + /** + * Extracts a value or values from a complex type. + * The following types of extraction are supported: + * - Given an Array, an integer ordinal can be used to retrieve a single value. + * - Given a Map, a key of the correct type can be used to retrieve an individual value. + * - Given a Struct, a string fieldName can be used to extract that field. + * - Given an Array of Structs, a string fieldName can be used to extract filed + * of every struct in that array, and return an Array of fields + * + * @group expr_ops + */ + def apply(field: Any): Column = UnresolvedExtractValue(expr, Literal(field)) + /** * Unary minus, i.e. negate the expression. * {{{ @@ -84,14 +97,14 @@ class Column(protected[sql] val expr: Expression) extends Logging { /** * Inversion of boolean expression, i.e. NOT. - * {{ + * {{{ * // Scala: select rows that are not active (isActive === false) * df.filter( !df("isActive") ) * * // Java: * import static org.apache.spark.sql.functions.*; * df.filter( not(df.col("isActive")) ); - * }} + * }}} * * @group expr_ops */ @@ -529,14 +542,14 @@ class Column(protected[sql] val expr: Expression) extends Logging { * * @group expr_ops */ - def getItem(key: Any): Column = GetItem(expr, Literal(key)) + def getItem(key: Any): Column = UnresolvedExtractValue(expr, Literal(key)) /** * An expression that gets a field by name in a [[StructType]]. * * @group expr_ops */ - def getField(fieldName: String): Column = UnresolvedGetField(expr, fieldName) + def getField(fieldName: String): Column = UnresolvedExtractValue(expr, Literal(fieldName)) /** * An expression that returns a substring. @@ -698,6 +711,37 @@ class Column(protected[sql] val expr: Expression) extends Logging { println(expr.prettyString) } } + + /** + * Compute bitwise OR of this expression with another expression. + * {{{ + * df.select($"colA".bitwiseOR($"colB")) + * }}} + * + * @group expr_ops + */ + def bitwiseOR(other: Any): Column = BitwiseOr(expr, lit(other).expr) + + /** + * Compute bitwise AND of this expression with another expression. + * {{{ + * df.select($"colA".bitwiseAND($"colB")) + * }}} + * + * @group expr_ops + */ + def bitwiseAND(other: Any): Column = BitwiseAnd(expr, lit(other).expr) + + /** + * Compute bitwise XOR of this expression with another expression. + * {{{ + * df.select($"colA".bitwiseXOR($"colB")) + * }}} + * + * @group expr_ops + */ + def bitwiseXOR(other: Any): Column = BitwiseXor(expr, lit(other).expr) + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index cf344710ff8b4..7947042c14299 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -42,7 +42,7 @@ import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, LogicalRDD} import org.apache.spark.sql.jdbc.JDBCWriteDetails -import org.apache.spark.sql.json.JsonRDD +import org.apache.spark.sql.json.{JacksonGenerator, JsonRDD} import org.apache.spark.sql.types._ import org.apache.spark.sql.sources.{ResolvedDataSource, CreateTableUsingAsSelect} import org.apache.spark.util.Utils @@ -143,7 +143,6 @@ class DataFrame private[sql]( // happen right away to let these side effects take place eagerly. case _: Command | _: InsertIntoTable | - _: CreateTableAsSelect[_] | _: CreateTableUsingAsSelect | _: WriteToFile => LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext) @@ -416,9 +415,7 @@ class DataFrame private[sql]( * }}} * @group dfops */ - def join(right: DataFrame, joinExprs: Column): DataFrame = { - Join(logicalPlan, right.logicalPlan, joinType = Inner, Some(joinExprs.expr)) - } + def join(right: DataFrame, joinExprs: Column): DataFrame = join(right, joinExprs, "inner") /** * Join with another [[DataFrame]], using the given join expression. The following performs @@ -440,7 +437,39 @@ class DataFrame private[sql]( * @group dfops */ def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame = { - Join(logicalPlan, right.logicalPlan, JoinType(joinType), Some(joinExprs.expr)) + // Note that in this function, we introduce a hack in the case of self-join to automatically + // resolve ambiguous join conditions into ones that might make sense [SPARK-6231]. + // Consider this case: df.join(df, df("key") === df("key")) + // Since df("key") === df("key") is a trivially true condition, this actually becomes a + // cartesian join. However, most likely users expect to perform a self join using "key". + // With that assumption, this hack turns the trivially true condition into equality on join + // keys that are resolved to both sides. + + // Trigger analysis so in the case of self-join, the analyzer will clone the plan. + // After the cloning, left and right side will have distinct expression ids. + val plan = Join(logicalPlan, right.logicalPlan, JoinType(joinType), Some(joinExprs.expr)) + .queryExecution.analyzed.asInstanceOf[Join] + + // If auto self join alias is disabled, return the plan. + if (!sqlContext.conf.dataFrameSelfJoinAutoResolveAmbiguity) { + return plan + } + + // If left/right have no output set intersection, return the plan. + val lanalyzed = this.logicalPlan.queryExecution.analyzed + val ranalyzed = right.logicalPlan.queryExecution.analyzed + if (lanalyzed.outputSet.intersect(ranalyzed.outputSet).isEmpty) { + return plan + } + + // Otherwise, find the trivially true predicates and automatically resolves them to both sides. + // By the time we get here, since we have already run analysis, all attributes should've been + // resolved and become AttributeReference. + val cond = plan.condition.map { _.transform { + case EqualTo(a: AttributeReference, b: AttributeReference) if a.sameRef(b) => + EqualTo(plan.left.resolve(a.name), plan.right.resolve(b.name)) + }} + plan.copy(condition = cond) } /** @@ -651,11 +680,11 @@ class DataFrame private[sql]( /** * (Scala-specific) Aggregates on the entire [[DataFrame]] without groups. - * {{ + * {{{ * // df.agg(...) is a shorthand for df.groupBy().agg(...) * df.agg("age" -> "max", "salary" -> "avg") * df.groupBy().agg("age" -> "max", "salary" -> "avg") - * }} + * }}} * @group dfops */ def agg(aggExpr: (String, String), aggExprs: (String, String)*): DataFrame = { @@ -664,33 +693,33 @@ class DataFrame private[sql]( /** * (Scala-specific) Aggregates on the entire [[DataFrame]] without groups. - * {{ + * {{{ * // df.agg(...) is a shorthand for df.groupBy().agg(...) * df.agg(Map("age" -> "max", "salary" -> "avg")) * df.groupBy().agg(Map("age" -> "max", "salary" -> "avg")) - * }} + * }}} * @group dfops */ def agg(exprs: Map[String, String]): DataFrame = groupBy().agg(exprs) /** * (Java-specific) Aggregates on the entire [[DataFrame]] without groups. - * {{ + * {{{ * // df.agg(...) is a shorthand for df.groupBy().agg(...) * df.agg(Map("age" -> "max", "salary" -> "avg")) * df.groupBy().agg(Map("age" -> "max", "salary" -> "avg")) - * }} + * }}} * @group dfops */ def agg(exprs: java.util.Map[String, String]): DataFrame = groupBy().agg(exprs) /** * Aggregates on the entire [[DataFrame]] without groups. - * {{ + * {{{ * // df.agg(...) is a shorthand for df.groupBy().agg(...) * df.agg(max($"age"), avg($"salary")) * df.groupBy().agg(max($"age"), avg($"salary")) - * }} + * }}} * @group dfops */ @scala.annotation.varargs @@ -1385,7 +1414,7 @@ class DataFrame private[sql]( new Iterator[String] { override def hasNext: Boolean = iter.hasNext override def next(): String = { - JsonRDD.rowToJSON(rowSchema, gen)(iter.next()) + JacksonGenerator(rowSchema, gen)(iter.next()) gen.flush() val json = writer.toString diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index 481ed4924857e..4a54120ba86f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -357,11 +357,12 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * TODO: This can be optimized to use broadcast join when replacementMap is large. */ private def replaceCol(col: StructField, replacementMap: Map[_, _]): Column = { - val branches: Seq[Expression] = replacementMap.flatMap { case (source, target) => - df.col(col.name).equalTo(lit(source).cast(col.dataType)).expr :: - lit(target).cast(col.dataType).expr :: Nil + val keyExpr = df.col(col.name).expr + def buildExpr(v: Any) = Cast(Literal(v), keyExpr.dataType) + val branches = replacementMap.flatMap { case (source, target) => + Seq(buildExpr(source), buildExpr(target)) }.toSeq - new Column(CaseWhen(branches ++ Seq(df.col(col.name).expr))).as(col.name) + new Column(CaseKeyWhen(keyExpr, branches :+ keyExpr)).as(col.name) } private def convertToDouble(v: Any): Double = v match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala index cb88deab35968..a1e74470afc89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala @@ -37,7 +37,7 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { StatFunctions.calculateCov(df, Seq(col1, col2)) } - /* + /** * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in * MLlib's Statistics. @@ -75,7 +75,7 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * each row. * @param col2 The name of the second column. Distinct items will make the column names * of the DataFrame. - * @return A Local DataFrame containing the table + * @return A DataFrame containing for the contingency table. */ def crosstab(col1: String, col2: String): DataFrame = { StatFunctions.crossTabulate(df, col1, col2) @@ -110,14 +110,25 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { } /** - * Python friendly implementation for `freqItems` + * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the + * frequent element count algorithm described in + * [[http://dx.doi.org/10.1145/762471.762473, proposed by Karp, Schenker, and Papadimitriou]]. + * + * @param cols the names of the columns to search frequent items in. + * @return A Local DataFrame with the Array of frequent items for each column. */ def freqItems(cols: Seq[String], support: Double): DataFrame = { FrequentItems.singlePassFreqItems(df, cols, support) } /** - * Python friendly implementation for `freqItems` with a default `support` of 1%. + * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the + * frequent element count algorithm described in + * [[http://dx.doi.org/10.1145/762471.762473, proposed by Karp, Schenker, and Papadimitriou]]. + * Uses a `default` support of 1%. + * + * @param cols the names of the columns to search frequent items in. + * @return A Local DataFrame with the Array of frequent items for each column. */ def freqItems(cols: Seq[String]): DataFrame = { FrequentItems.singlePassFreqItems(df, cols, 0.01) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 99db959a8741c..bfaddd0f2ce1b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -67,8 +67,14 @@ private[spark] object SQLConf { // Set to false when debugging requires the ability to look at invalid query plans. val DATAFRAME_EAGER_ANALYSIS = "spark.sql.eagerAnalysis" + // Whether to automatically resolve ambiguity in join conditions for self-joins. + // See SPARK-6231. + val DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY = "spark.sql.selfJoinAutoResolveAmbiguity" + val USE_SQL_SERIALIZER2 = "spark.sql.useSerializer2" + val USE_JACKSON_STREAMING_API = "spark.sql.json.useJacksonStreamingAPI" + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } @@ -162,6 +168,12 @@ private[sql] class SQLConf extends Serializable { private[spark] def useSqlSerializer2: Boolean = getConf(USE_SQL_SERIALIZER2, "true").toBoolean + /** + * Selects between the new (true) and old (false) JSON handlers, to be removed in Spark 1.5.0 + */ + private[spark] def useJacksonStreamingAPI: Boolean = + getConf(USE_JACKSON_STREAMING_API, "true").toBoolean + /** * Upper bound on the sizes (in bytes) of the tables qualified for the auto conversion to * a broadcast value during the physical executions of join operations. Setting this to -1 @@ -219,6 +231,9 @@ private[sql] class SQLConf extends Serializable { private[spark] def dataFrameEagerAnalysis: Boolean = getConf(DATAFRAME_EAGER_ANALYSIS, "true").toBoolean + private[spark] def dataFrameSelfJoinAutoResolveAmbiguity: Boolean = + getConf(DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY, "true").toBoolean + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 7eabb93c1e3d6..0ac0936f0f592 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -70,7 +70,7 @@ import org.apache.spark.{Partition, SparkContext} * spark-sql> SELECT * FROM src LIMIT 1; * *-- Exception will be thrown and switch to dialect - *-- "sql" (for SQLContext) or + *-- "sql" (for SQLContext) or *-- "hiveql" (for HiveContext) * }}} */ @@ -107,7 +107,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * @return Spark SQL configuration */ - protected[sql] def conf = tlSession.get().conf + protected[sql] def conf = currentSession().conf /** * Set Spark SQL configuration properties. @@ -659,13 +659,17 @@ class SQLContext(@transient val sparkContext: SparkContext) */ @Experimental def jsonRDD(json: RDD[String], schema: StructType): DataFrame = { - val columnNameOfCorruptJsonRecord = conf.columnNameOfCorruptRecord - val appliedSchema = - Option(schema).getOrElse( - JsonRDD.nullTypeToStringType( - JsonRDD.inferSchema(json, 1.0, columnNameOfCorruptJsonRecord))) - val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema, columnNameOfCorruptJsonRecord) - createDataFrame(rowRDD, appliedSchema, needsConversion = false) + if (conf.useJacksonStreamingAPI) { + baseRelationToDataFrame(new JSONRelation(() => json, None, 1.0, Some(schema))(this)) + } else { + val columnNameOfCorruptJsonRecord = conf.columnNameOfCorruptRecord + val appliedSchema = + Option(schema).getOrElse( + JsonRDD.nullTypeToStringType( + JsonRDD.inferSchema(json, 1.0, columnNameOfCorruptJsonRecord))) + val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema, columnNameOfCorruptJsonRecord) + createDataFrame(rowRDD, appliedSchema, needsConversion = false) + } } /** @@ -689,12 +693,16 @@ class SQLContext(@transient val sparkContext: SparkContext) */ @Experimental def jsonRDD(json: RDD[String], samplingRatio: Double): DataFrame = { - val columnNameOfCorruptJsonRecord = conf.columnNameOfCorruptRecord - val appliedSchema = - JsonRDD.nullTypeToStringType( - JsonRDD.inferSchema(json, samplingRatio, columnNameOfCorruptJsonRecord)) - val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema, columnNameOfCorruptJsonRecord) - createDataFrame(rowRDD, appliedSchema, needsConversion = false) + if (conf.useJacksonStreamingAPI) { + baseRelationToDataFrame(new JSONRelation(() => json, None, samplingRatio, None)(this)) + } else { + val columnNameOfCorruptJsonRecord = conf.columnNameOfCorruptRecord + val appliedSchema = + JsonRDD.nullTypeToStringType( + JsonRDD.inferSchema(json, samplingRatio, columnNameOfCorruptJsonRecord)) + val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema, columnNameOfCorruptJsonRecord) + createDataFrame(rowRDD, appliedSchema, needsConversion = false) + } } /** @@ -1189,13 +1197,17 @@ class SQLContext(@transient val sparkContext: SparkContext) |${stringOrError(executedPlan)} """.stripMargin.trim - override def toString: String = + override def toString: String = { + def output = + analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}").mkString(", ") + // TODO previously will output RDD details by run (${stringOrError(toRdd.toDebugString)}) // however, the `toRdd` will cause the real execution, which is not what we want. // We need to think about how to avoid the side effect. s"""== Parsed Logical Plan == |${stringOrError(logical)} |== Analyzed Logical Plan == + |${stringOrError(output)} |${stringOrError(analyzed)} |== Optimized Logical Plan == |${stringOrError(optimizedPlan)} @@ -1204,6 +1216,7 @@ class SQLContext(@transient val sparkContext: SparkContext) |Code Generation: ${stringOrError(executedPlan.codegenEnabled)} |== RDD == """.stripMargin.trim + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 5b2e46962cd3b..f0d54cd6cd94f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -84,18 +84,8 @@ case class Exchange( def serializer( keySchema: Array[DataType], valueSchema: Array[DataType], + hasKeyOrdering: Boolean, numPartitions: Int): Serializer = { - // In ExternalSorter's spillToMergeableFile function, key-value pairs are written out - // through write(key) and then write(value) instead of write((key, value)). Because - // SparkSqlSerializer2 assumes that objects passed in are Product2, we cannot safely use - // it when spillToMergeableFile in ExternalSorter will be used. - // So, we will not use SparkSqlSerializer2 when - // - Sort-based shuffle is enabled and the number of reducers (numPartitions) is greater - // then the bypassMergeThreshold; or - // - newOrdering is defined. - val cannotUseSqlSerializer2 = - (sortBasedShuffleOn && numPartitions > bypassMergeThreshold) || newOrdering.nonEmpty - // It is true when there is no field that needs to be write out. // For now, we will not use SparkSqlSerializer2 when noField is true. val noField = @@ -104,14 +94,13 @@ case class Exchange( val useSqlSerializer2 = child.sqlContext.conf.useSqlSerializer2 && // SparkSqlSerializer2 is enabled. - !cannotUseSqlSerializer2 && // Safe to use Serializer2. SparkSqlSerializer2.support(keySchema) && // The schema of key is supported. SparkSqlSerializer2.support(valueSchema) && // The schema of value is supported. !noField val serializer = if (useSqlSerializer2) { logInfo("Using SparkSqlSerializer2.") - new SparkSqlSerializer2(keySchema, valueSchema) + new SparkSqlSerializer2(keySchema, valueSchema, hasKeyOrdering) } else { logInfo("Using SparkSqlSerializer.") new SparkSqlSerializer(sparkConf) @@ -154,7 +143,8 @@ case class Exchange( } val keySchema = expressions.map(_.dataType).toArray val valueSchema = child.output.map(_.dataType).toArray - shuffled.setSerializer(serializer(keySchema, valueSchema, numPartitions)) + shuffled.setSerializer( + serializer(keySchema, valueSchema, newOrdering.nonEmpty, numPartitions)) shuffled.map(_._2) @@ -179,7 +169,8 @@ case class Exchange( new ShuffledRDD[Row, Null, Null](rdd, part) } val keySchema = child.output.map(_.dataType).toArray - shuffled.setSerializer(serializer(keySchema, null, numPartitions)) + shuffled.setSerializer( + serializer(keySchema, null, newOrdering.nonEmpty, numPartitions)) shuffled.map(_._1) @@ -199,7 +190,7 @@ case class Exchange( val partitioner = new HashPartitioner(1) val shuffled = new ShuffledRDD[Null, Row, Row](rdd, partitioner) val valueSchema = child.output.map(_.dataType).toArray - shuffled.setSerializer(serializer(null, valueSchema, 1)) + shuffled.setSerializer(serializer(null, valueSchema, false, 1)) shuffled.map(_._2) case _ => sys.error(s"Exchange not implemented for $newPartitioning") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala index 9552f41115866..256d527d7b636 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala @@ -27,7 +27,7 @@ import scala.reflect.ClassTag import org.apache.spark.serializer._ import org.apache.spark.Logging import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.expressions.SpecificMutableRow +import org.apache.spark.sql.catalyst.expressions.{SpecificMutableRow, MutableRow, GenericMutableRow} import org.apache.spark.sql.types._ /** @@ -49,9 +49,9 @@ private[sql] class Serializer2SerializationStream( out: OutputStream) extends SerializationStream with Logging { - val rowOut = new DataOutputStream(out) - val writeKeyFunc = SparkSqlSerializer2.createSerializationFunction(keySchema, rowOut) - val writeValueFunc = SparkSqlSerializer2.createSerializationFunction(valueSchema, rowOut) + private val rowOut = new DataOutputStream(new BufferedOutputStream(out)) + private val writeKeyFunc = SparkSqlSerializer2.createSerializationFunction(keySchema, rowOut) + private val writeValueFunc = SparkSqlSerializer2.createSerializationFunction(valueSchema, rowOut) override def writeObject[T: ClassTag](t: T): SerializationStream = { val kv = t.asInstanceOf[Product2[Row, Row]] @@ -86,31 +86,44 @@ private[sql] class Serializer2SerializationStream( private[sql] class Serializer2DeserializationStream( keySchema: Array[DataType], valueSchema: Array[DataType], + hasKeyOrdering: Boolean, in: InputStream) extends DeserializationStream with Logging { - val rowIn = new DataInputStream(new BufferedInputStream(in)) + private val rowIn = new DataInputStream(new BufferedInputStream(in)) + + private def rowGenerator(schema: Array[DataType]): () => (MutableRow) = { + if (schema == null) { + () => null + } else { + if (hasKeyOrdering) { + // We have key ordering specified in a ShuffledRDD, it is not safe to reuse a mutable row. + () => new GenericMutableRow(schema.length) + } else { + // It is safe to reuse the mutable row. + val mutableRow = new SpecificMutableRow(schema) + () => mutableRow + } + } + } - val key = if (keySchema != null) new SpecificMutableRow(keySchema) else null - val value = if (valueSchema != null) new SpecificMutableRow(valueSchema) else null - val readKeyFunc = SparkSqlSerializer2.createDeserializationFunction(keySchema, rowIn, key) - val readValueFunc = SparkSqlSerializer2.createDeserializationFunction(valueSchema, rowIn, value) + // Functions used to return rows for key and value. + private val getKey = rowGenerator(keySchema) + private val getValue = rowGenerator(valueSchema) + // Functions used to read a serialized row from the InputStream and deserialize it. + private val readKeyFunc = SparkSqlSerializer2.createDeserializationFunction(keySchema, rowIn) + private val readValueFunc = SparkSqlSerializer2.createDeserializationFunction(valueSchema, rowIn) override def readObject[T: ClassTag](): T = { - readKeyFunc() - readValueFunc() - - (key, value).asInstanceOf[T] + (readKeyFunc(getKey()), readValueFunc(getValue())).asInstanceOf[T] } override def readKey[T: ClassTag](): T = { - readKeyFunc() - key.asInstanceOf[T] + readKeyFunc(getKey()).asInstanceOf[T] } override def readValue[T: ClassTag](): T = { - readValueFunc() - value.asInstanceOf[T] + readValueFunc(getValue()).asInstanceOf[T] } override def close(): Unit = { @@ -118,9 +131,10 @@ private[sql] class Serializer2DeserializationStream( } } -private[sql] class ShuffleSerializerInstance( +private[sql] class SparkSqlSerializer2Instance( keySchema: Array[DataType], - valueSchema: Array[DataType]) + valueSchema: Array[DataType], + hasKeyOrdering: Boolean) extends SerializerInstance { def serialize[T: ClassTag](t: T): ByteBuffer = @@ -137,7 +151,7 @@ private[sql] class ShuffleSerializerInstance( } def deserializeStream(s: InputStream): DeserializationStream = { - new Serializer2DeserializationStream(keySchema, valueSchema, s) + new Serializer2DeserializationStream(keySchema, valueSchema, hasKeyOrdering, s) } } @@ -148,12 +162,21 @@ private[sql] class ShuffleSerializerInstance( * The schema of keys is represented by `keySchema` and that of values is represented by * `valueSchema`. */ -private[sql] class SparkSqlSerializer2(keySchema: Array[DataType], valueSchema: Array[DataType]) +private[sql] class SparkSqlSerializer2( + keySchema: Array[DataType], + valueSchema: Array[DataType], + hasKeyOrdering: Boolean) extends Serializer with Logging with Serializable{ - def newInstance(): SerializerInstance = new ShuffleSerializerInstance(keySchema, valueSchema) + def newInstance(): SerializerInstance = + new SparkSqlSerializer2Instance(keySchema, valueSchema, hasKeyOrdering) + + override def supportsRelocationOfSerializedObjects: Boolean = { + // SparkSqlSerializer2 is stateless and writes no stream headers + true + } } private[sql] object SparkSqlSerializer2 { @@ -318,11 +341,11 @@ private[sql] object SparkSqlSerializer2 { */ def createDeserializationFunction( schema: Array[DataType], - in: DataInputStream, - mutableRow: SpecificMutableRow): () => Unit = { - () => { - // If the schema is null, the returned function does nothing when it get called. - if (schema != null) { + in: DataInputStream): (MutableRow) => Row = { + if (schema == null) { + (mutableRow: MutableRow) => null + } else { + (mutableRow: MutableRow) => { var i = 0 while (i < schema.length) { schema(i) match { @@ -435,6 +458,8 @@ private[sql] object SparkSqlSerializer2 { } i += 1 } + + mutableRow } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 326e8ce4ca524..56a4689eb58f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -303,6 +303,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Expand(projections, output, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => execution.Aggregate(partial = false, group, agg, planLater(child)) :: Nil + case logical.Window(projectList, windowExpressions, spec, child) => + execution.Window(projectList, windowExpressions, spec, planLater(child)) :: Nil case logical.Sample(lb, ub, withReplacement, seed, child) => execution.Sample(lb, ub, withReplacement, seed, planLater(child)) :: Nil case logical.LocalRelation(output, data) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala new file mode 100644 index 0000000000000..217b559def512 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala @@ -0,0 +1,480 @@ +/* + * 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 + +import java.util + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, Distribution, ClusteredDistribution, Partitioning} +import org.apache.spark.util.collection.CompactBuffer + +/** + * :: DeveloperApi :: + * For every row, evaluates `windowExpression` containing Window Functions and attaches + * the results with other regular expressions (presented by `projectList`). + * Evert operator handles a single Window Specification, `windowSpec`. + */ +case class Window( + projectList: Seq[Attribute], + windowExpression: Seq[NamedExpression], + windowSpec: WindowSpecDefinition, + child: SparkPlan) + extends UnaryNode { + + override def output: Seq[Attribute] = + (projectList ++ windowExpression).map(_.toAttribute) + + override def requiredChildDistribution: Seq[Distribution] = + if (windowSpec.partitionSpec.isEmpty) { + // This operator will be very expensive. + AllTuples :: Nil + } else { + ClusteredDistribution(windowSpec.partitionSpec) :: Nil + } + + // Since window functions are adding columns to the input rows, the child's outputPartitioning + // is preserved. + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = { + // The required child ordering has two parts. + // The first part is the expressions in the partition specification. + // We add these expressions to the required ordering to make sure input rows are grouped + // based on the partition specification. So, we only need to process a single partition + // at a time. + // The second part is the expressions specified in the ORDER BY cluase. + // Basically, we first use sort to group rows based on partition specifications and then sort + // Rows in a group based on the order specification. + (windowSpec.partitionSpec.map(SortOrder(_, Ascending)) ++ windowSpec.orderSpec) :: Nil + } + + // Since window functions basically add columns to input rows, this operator + // will not change the ordering of input rows. + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + case class ComputedWindow( + unbound: WindowExpression, + windowFunction: WindowFunction, + resultAttribute: AttributeReference) + + // A list of window functions that need to be computed for each group. + private[this] val computedWindowExpressions = windowExpression.flatMap { window => + window.collect { + case w: WindowExpression => + ComputedWindow( + w, + BindReferences.bindReference(w.windowFunction, child.output), + AttributeReference(s"windowResult:$w", w.dataType, w.nullable)()) + } + }.toArray + + private[this] val windowFrame = + windowSpec.frameSpecification.asInstanceOf[SpecifiedWindowFrame] + + // Create window functions. + private[this] def windowFunctions(): Array[WindowFunction] = { + val functions = new Array[WindowFunction](computedWindowExpressions.length) + var i = 0 + while (i < computedWindowExpressions.length) { + functions(i) = computedWindowExpressions(i).windowFunction.newInstance() + functions(i).init() + i += 1 + } + functions + } + + // The schema of the result of all window function evaluations + private[this] val computedSchema = computedWindowExpressions.map(_.resultAttribute) + + private[this] val computedResultMap = + computedWindowExpressions.map { w => w.unbound -> w.resultAttribute }.toMap + + private[this] val windowExpressionResult = windowExpression.map { window => + window.transform { + case w: WindowExpression if computedResultMap.contains(w) => computedResultMap(w) + } + } + + def execute(): RDD[Row] = { + child.execute().mapPartitions { iter => + new Iterator[Row] { + + // Although input rows are grouped based on windowSpec.partitionSpec, we need to + // know when we have a new partition. + // This is to manually construct an ordering that can be used to compare rows. + // TODO: We may want to have a newOrdering that takes BoundReferences. + // So, we can take advantave of code gen. + private val partitionOrdering: Ordering[Row] = + RowOrdering.forSchema(windowSpec.partitionSpec.map(_.dataType)) + + // This is used to project expressions for the partition specification. + protected val partitionGenerator = + newMutableProjection(windowSpec.partitionSpec, child.output)() + + // This is ued to project expressions for the order specification. + protected val rowOrderGenerator = + newMutableProjection(windowSpec.orderSpec.map(_.child), child.output)() + + // The position of next output row in the inputRowBuffer. + var rowPosition: Int = 0 + // The number of buffered rows in the inputRowBuffer (the size of the current partition). + var partitionSize: Int = 0 + // The buffer used to buffer rows in a partition. + var inputRowBuffer: CompactBuffer[Row] = _ + // The partition key of the current partition. + var currentPartitionKey: Row = _ + // The partition key of next partition. + var nextPartitionKey: Row = _ + // The first row of next partition. + var firstRowInNextPartition: Row = _ + // Indicates if this partition is the last one in the iter. + var lastPartition: Boolean = false + + def createBoundaryEvaluator(): () => Unit = { + def findPhysicalBoundary( + boundary: FrameBoundary): () => Int = boundary match { + case UnboundedPreceding => () => 0 + case UnboundedFollowing => () => partitionSize - 1 + case CurrentRow => () => rowPosition + case ValuePreceding(value) => + () => + val newPosition = rowPosition - value + if (newPosition > 0) newPosition else 0 + case ValueFollowing(value) => + () => + val newPosition = rowPosition + value + if (newPosition < partitionSize) newPosition else partitionSize - 1 + } + + def findLogicalBoundary( + boundary: FrameBoundary, + searchDirection: Int, + evaluator: Expression, + joinedRow: JoinedRow): () => Int = boundary match { + case UnboundedPreceding => () => 0 + case UnboundedFollowing => () => partitionSize - 1 + case other => + () => { + // CurrentRow, ValuePreceding, or ValueFollowing. + var newPosition = rowPosition + searchDirection + var stopSearch = false + // rowOrderGenerator is a mutable projection. + // We need to make a copy of the returned by rowOrderGenerator since we will + // compare searched row with this currentOrderByValue. + val currentOrderByValue = rowOrderGenerator(inputRowBuffer(rowPosition)).copy() + while (newPosition >= 0 && newPosition < partitionSize && !stopSearch) { + val r = rowOrderGenerator(inputRowBuffer(newPosition)) + stopSearch = + !(evaluator.eval(joinedRow(currentOrderByValue, r)).asInstanceOf[Boolean]) + if (!stopSearch) { + newPosition += searchDirection + } + } + newPosition -= searchDirection + + if (newPosition < 0) { + 0 + } else if (newPosition >= partitionSize) { + partitionSize - 1 + } else { + newPosition + } + } + } + + windowFrame.frameType match { + case RowFrame => + val findStart = findPhysicalBoundary(windowFrame.frameStart) + val findEnd = findPhysicalBoundary(windowFrame.frameEnd) + () => { + frameStart = findStart() + frameEnd = findEnd() + } + case RangeFrame => + val joinedRowForBoundaryEvaluation: JoinedRow = new JoinedRow() + val orderByExpr = windowSpec.orderSpec.head + val currentRowExpr = + BoundReference(0, orderByExpr.dataType, orderByExpr.nullable) + val examedRowExpr = + BoundReference(1, orderByExpr.dataType, orderByExpr.nullable) + val differenceExpr = Abs(Subtract(currentRowExpr, examedRowExpr)) + + val frameStartEvaluator = windowFrame.frameStart match { + case CurrentRow => EqualTo(currentRowExpr, examedRowExpr) + case ValuePreceding(value) => + LessThanOrEqual(differenceExpr, Cast(Literal(value), orderByExpr.dataType)) + case ValueFollowing(value) => + GreaterThanOrEqual(differenceExpr, Cast(Literal(value), orderByExpr.dataType)) + case o => Literal(true) // This is just a dummy expression, we will not use it. + } + + val frameEndEvaluator = windowFrame.frameEnd match { + case CurrentRow => EqualTo(currentRowExpr, examedRowExpr) + case ValuePreceding(value) => + GreaterThanOrEqual(differenceExpr, Cast(Literal(value), orderByExpr.dataType)) + case ValueFollowing(value) => + LessThanOrEqual(differenceExpr, Cast(Literal(value), orderByExpr.dataType)) + case o => Literal(true) // This is just a dummy expression, we will not use it. + } + + val findStart = + findLogicalBoundary( + boundary = windowFrame.frameStart, + searchDirection = -1, + evaluator = frameStartEvaluator, + joinedRow = joinedRowForBoundaryEvaluation) + val findEnd = + findLogicalBoundary( + boundary = windowFrame.frameEnd, + searchDirection = 1, + evaluator = frameEndEvaluator, + joinedRow = joinedRowForBoundaryEvaluation) + () => { + frameStart = findStart() + frameEnd = findEnd() + } + } + } + + val boundaryEvaluator = createBoundaryEvaluator() + // Indicates if we the specified window frame requires us to maintain a sliding frame + // (e.g. RANGES BETWEEN 1 PRECEDING AND CURRENT ROW) or the window frame + // is the entire partition (e.g. ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING). + val requireUpdateFrame: Boolean = { + def requireUpdateBoundary(boundary: FrameBoundary): Boolean = boundary match { + case UnboundedPreceding => false + case UnboundedFollowing => false + case _ => true + } + + requireUpdateBoundary(windowFrame.frameStart) || + requireUpdateBoundary(windowFrame.frameEnd) + } + // The start position of the current frame in the partition. + var frameStart: Int = 0 + // The end position of the current frame in the partition. + var frameEnd: Int = -1 + // Window functions. + val functions: Array[WindowFunction] = windowFunctions() + // Buffers used to store input parameters for window functions. Because we may need to + // maintain a sliding frame, we use this buffer to avoid evaluate the parameters from + // the same row multiple times. + val windowFunctionParameterBuffers: Array[util.LinkedList[AnyRef]] = + functions.map(_ => new util.LinkedList[AnyRef]()) + + // The projection used to generate the final result rows of this operator. + private[this] val resultProjection = + newMutableProjection( + projectList ++ windowExpressionResult, + projectList ++ computedSchema)() + + // The row used to hold results of window functions. + private[this] val windowExpressionResultRow = + new GenericMutableRow(computedSchema.length) + + private[this] val joinedRow = new JoinedRow6 + + // Initialize this iterator. + initialize() + + private def initialize(): Unit = { + if (iter.hasNext) { + val currentRow = iter.next().copy() + // partitionGenerator is a mutable projection. Since we need to track nextPartitionKey, + // we are making a copy of the returned partitionKey at here. + nextPartitionKey = partitionGenerator(currentRow).copy() + firstRowInNextPartition = currentRow + fetchNextPartition() + } else { + // The iter is an empty one. So, we set all of the following variables + // to make sure hasNext will return false. + lastPartition = true + rowPosition = 0 + partitionSize = 0 + } + } + + // Indicates if we will have new output row. + override final def hasNext: Boolean = { + !lastPartition || (rowPosition < partitionSize) + } + + override final def next(): Row = { + if (hasNext) { + if (rowPosition == partitionSize) { + // All rows of this buffer have been consumed. + // We will move to next partition. + fetchNextPartition() + } + // Get the input row for the current output row. + val inputRow = inputRowBuffer(rowPosition) + // Get all results of the window functions for this output row. + var i = 0 + while (i < functions.length) { + windowExpressionResultRow.update(i, functions(i).get(rowPosition)) + i += 1 + } + + // Construct the output row. + val outputRow = resultProjection(joinedRow(inputRow, windowExpressionResultRow)) + // We will move to the next one. + rowPosition += 1 + if (requireUpdateFrame && rowPosition < partitionSize) { + // If we need to maintain a sliding frame and + // we will still work on this partition when next is called next time, do the update. + updateFrame() + } + + // Return the output row. + outputRow + } else { + // no more result + throw new NoSuchElementException + } + } + + // Fetch the next partition. + private def fetchNextPartition(): Unit = { + // Create a new buffer for input rows. + inputRowBuffer = new CompactBuffer[Row]() + // We already have the first row for this partition + // (recorded in firstRowInNextPartition). Add it back. + inputRowBuffer += firstRowInNextPartition + // Set the current partition key. + currentPartitionKey = nextPartitionKey + // Now, we will start to find all rows belonging to this partition. + // Create a variable to track if we see the next partition. + var findNextPartition = false + // The search will stop when we see the next partition or there is no + // input row left in the iter. + while (iter.hasNext && !findNextPartition) { + // Make a copy of the input row since we will put it in the buffer. + val currentRow = iter.next().copy() + // Get the partition key based on the partition specification. + // For the below compare method, we do not need to make a copy of partitionKey. + val partitionKey = partitionGenerator(currentRow) + // Check if the current row belongs the current input row. + val comparing = partitionOrdering.compare(currentPartitionKey, partitionKey) + if (comparing == 0) { + // This row is still in the current partition. + inputRowBuffer += currentRow + } else { + // The current input row is in a different partition. + findNextPartition = true + // partitionGenerator is a mutable projection. + // Since we need to track nextPartitionKey and we determine that it should be set + // as partitionKey, we are making a copy of the partitionKey at here. + nextPartitionKey = partitionKey.copy() + firstRowInNextPartition = currentRow + } + } + + // We have not seen a new partition. It means that there is no new row in the + // iter. The current partition is the last partition of the iter. + if (!findNextPartition) { + lastPartition = true + } + + // We have got all rows for the current partition. + // Set rowPosition to 0 (the next output row will be based on the first + // input row of this partition). + rowPosition = 0 + // The size of this partition. + partitionSize = inputRowBuffer.size + // Reset all parameter buffers of window functions. + var i = 0 + while (i < windowFunctionParameterBuffers.length) { + windowFunctionParameterBuffers(i).clear() + i += 1 + } + frameStart = 0 + frameEnd = -1 + // Create the first window frame for this partition. + // If we do not need to maintain a sliding frame, this frame will + // have the entire partition. + updateFrame() + } + + /** The function used to maintain the sliding frame. */ + private def updateFrame(): Unit = { + // Based on the difference between the new frame and old frame, + // updates the buffers holding input parameters of window functions. + // We will start to prepare input parameters starting from the row + // indicated by offset in the input row buffer. + def updateWindowFunctionParameterBuffers( + numToRemove: Int, + numToAdd: Int, + offset: Int): Unit = { + // First, remove unneeded entries from the head of every buffer. + var i = 0 + while (i < numToRemove) { + var j = 0 + while (j < windowFunctionParameterBuffers.length) { + windowFunctionParameterBuffers(j).remove() + j += 1 + } + i += 1 + } + // Then, add needed entries to the tail of every buffer. + i = 0 + while (i < numToAdd) { + var j = 0 + while (j < windowFunctionParameterBuffers.length) { + // Ask the function to prepare the input parameters. + val parameters = functions(j).prepareInputParameters(inputRowBuffer(i + offset)) + windowFunctionParameterBuffers(j).add(parameters) + j += 1 + } + i += 1 + } + } + + // Record the current frame start point and end point before + // we update them. + val previousFrameStart = frameStart + val previousFrameEnd = frameEnd + boundaryEvaluator() + updateWindowFunctionParameterBuffers( + frameStart - previousFrameStart, + frameEnd - previousFrameEnd, + previousFrameEnd + 1) + // Evaluate the current frame. + evaluateCurrentFrame() + } + + /** Evaluate the current window frame. */ + private def evaluateCurrentFrame(): Unit = { + var i = 0 + while (i < functions.length) { + // Reset the state of the window function. + functions(i).reset() + // Get all buffered input parameters based on rows of this window frame. + val inputParameters = windowFunctionParameterBuffers(i).toArray() + // Send these input parameters to the window function. + functions(i).batchUpdate(inputParameters) + // Ask the function to evaluate based on this window frame. + functions(i).evaluate() + i += 1 + } + } + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 98df5bef34efa..388a8184e4cfe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -32,9 +32,11 @@ import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext} * A logical command that is executed for its side-effects. `RunnableCommand`s are * wrapped in `ExecutedCommand` during execution. */ -trait RunnableCommand extends logical.Command { +private[sql] trait RunnableCommand extends LogicalPlan with logical.Command { self: Product => + override def output: Seq[Attribute] = Seq.empty + override def children: Seq[LogicalPlan] = Seq.empty def run(sqlContext: SQLContext): Seq[Row] } @@ -84,8 +86,14 @@ case class SetCommand( logWarning( s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") - sqlContext.setConf(SQLConf.SHUFFLE_PARTITIONS, value) - Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$value")) + if (value.toInt < 1) { + val msg = s"Setting negative ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} for automatically " + + "determining the number of reducers is not supported." + throw new IllegalArgumentException(msg) + } else { + sqlContext.setConf(SQLConf.SHUFFLE_PARTITIONS, value) + Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$value")) + } // Configures a single property. case Some((key, Some(value))) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index 7a43bfd8bc8d9..58cb1980f217e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -219,8 +219,8 @@ case class EvaluatePython( /** * :: DeveloperApi :: - * Uses PythonRDD to evaluate a [[PythonUDF]], one partition of tuples at a time. The input - * data is cached and zipped with the result of the udf evaluation. + * Uses PythonRDD to evaluate a [[PythonUDF]], one partition of tuples at a time. + * The input data is zipped with the result of the udf evaluation. */ @DeveloperApi case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: SparkPlan) @@ -229,8 +229,7 @@ case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: def children: Seq[SparkPlan] = child :: Nil def execute(): RDD[Row] = { - // TODO: Clean up after ourselves? - val childResults = child.execute().map(_.copy()).cache() + val childResults = child.execute().map(_.copy()) val parent = childResults.mapPartitions { iter => val pickle = new Pickler diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala index 386ac969f1e7d..71b7f6c2a6756 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala @@ -38,7 +38,7 @@ private[sql] object StatFunctions extends Logging { var yAvg = 0.0 // the mean of all examples seen so far in col2 var Ck = 0.0 // the co-moment after k examples var MkX = 0.0 // sum of squares of differences from the (current) mean for col1 - var MkY = 0.0 // sum of squares of differences from the (current) mean for col1 + var MkY = 0.0 // sum of squares of differences from the (current) mean for col2 var count = 0L // count of observed examples // add an example to the calculation def add(x: Double, y: Double): this.type = { @@ -55,15 +55,17 @@ private[sql] object StatFunctions extends Logging { // merge counters from other partitions. Formula can be found at: // http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance def merge(other: CovarianceCounter): this.type = { - val totalCount = count + other.count - val deltaX = xAvg - other.xAvg - val deltaY = yAvg - other.yAvg - Ck += other.Ck + deltaX * deltaY * count / totalCount * other.count - xAvg = (xAvg * count + other.xAvg * other.count) / totalCount - yAvg = (yAvg * count + other.yAvg * other.count) / totalCount - MkX += other.MkX + deltaX * deltaX * count / totalCount * other.count - MkY += other.MkY + deltaY * deltaY * count / totalCount * other.count - count = totalCount + if (other.count > 0) { + val totalCount = count + other.count + val deltaX = xAvg - other.xAvg + val deltaY = yAvg - other.yAvg + Ck += other.Ck + deltaX * deltaY * count / totalCount * other.count + xAvg = (xAvg * count + other.xAvg * other.count) / totalCount + yAvg = (yAvg * count + other.yAvg * other.count) / totalCount + MkX += other.MkX + deltaX * deltaX * count / totalCount * other.count + MkY += other.MkY + deltaY * deltaY * count / totalCount * other.count + count = totalCount + } this } // return the sample covariance for the observed examples 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 7e283393d0563..1728b0b8c910e 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 @@ -22,8 +22,9 @@ import scala.reflect.runtime.universe.{TypeTag, typeTag} import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction, Star} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedFunction, Star} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.mathfuncs._ import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -35,6 +36,7 @@ import org.apache.spark.util.Utils * @groupname agg_funcs Aggregate functions * @groupname sort_funcs Sorting functions * @groupname normal_funcs Non-aggregate functions + * @groupname math_funcs Math functions * @groupname Ungrouped Support functions for DataFrames. */ @Experimental @@ -84,10 +86,10 @@ object functions { /** * Returns a sort expression based on ascending order of the column. - * {{ + * {{{ * // Sort by dept in ascending order, and then age in descending order. * df.sort(asc("dept"), desc("age")) - * }} + * }}} * * @group sort_funcs */ @@ -95,10 +97,10 @@ object functions { /** * Returns a sort expression based on the descending order of the column. - * {{ + * {{{ * // Sort by dept in ascending order, and then age in descending order. * df.sort(asc("dept"), desc("age")) - * }} + * }}} * * @group sort_funcs */ @@ -351,13 +353,13 @@ object functions { /** * Inversion of boolean expression, i.e. NOT. - * {{ + * {{{ * // Scala: select rows that are not active (isActive === false) * df.filter( !df("isActive") ) * * // Java: * df.filter( not(df.col("isActive")) ); - * }} + * }}} * * @group normal_funcs */ @@ -436,6 +438,497 @@ object functions { */ def upper(e: Column): Column = Upper(e.expr) + + /** + * Computes bitwise NOT. + * + * @group normal_funcs + */ + def bitwiseNOT(e: Column): Column = BitwiseNot(e.expr) + + ////////////////////////////////////////////////////////////////////////////////////////////// + // Math Functions + ////////////////////////////////////////////////////////////////////////////////////////////// + + /** + * Computes the cosine inverse of the given value; the returned angle is in the range + * 0.0 through pi. + * + * @group math_funcs + */ + def acos(e: Column): Column = Acos(e.expr) + + /** + * Computes the cosine inverse of the given column; the returned angle is in the range + * 0.0 through pi. + * + * @group math_funcs + */ + def acos(columnName: String): Column = acos(Column(columnName)) + + /** + * Computes the sine inverse of the given value; the returned angle is in the range + * -pi/2 through pi/2. + * + * @group math_funcs + */ + def asin(e: Column): Column = Asin(e.expr) + + /** + * Computes the sine inverse of the given column; the returned angle is in the range + * -pi/2 through pi/2. + * + * @group math_funcs + */ + def asin(columnName: String): Column = asin(Column(columnName)) + + /** + * Computes the tangent inverse of the given value. + * + * @group math_funcs + */ + def atan(e: Column): Column = Atan(e.expr) + + /** + * Computes the tangent inverse of the given column. + * + * @group math_funcs + */ + def atan(columnName: String): Column = atan(Column(columnName)) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * @group math_funcs + */ + def atan2(l: Column, r: Column): Column = Atan2(l.expr, r.expr) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * @group math_funcs + */ + def atan2(l: Column, rightName: String): Column = atan2(l, Column(rightName)) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * @group math_funcs + */ + def atan2(leftName: String, r: Column): Column = atan2(Column(leftName), r) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * @group math_funcs + */ + def atan2(leftName: String, rightName: String): Column = + atan2(Column(leftName), Column(rightName)) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * @group math_funcs + */ + def atan2(l: Column, r: Double): Column = atan2(l, lit(r).expr) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * @group math_funcs + */ + def atan2(leftName: String, r: Double): Column = atan2(Column(leftName), r) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * @group math_funcs + */ + def atan2(l: Double, r: Column): Column = atan2(lit(l).expr, r) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * @group math_funcs + */ + def atan2(l: Double, rightName: String): Column = atan2(l, Column(rightName)) + + /** + * Computes the cube-root of the given value. + * + * @group math_funcs + */ + def cbrt(e: Column): Column = Cbrt(e.expr) + + /** + * Computes the cube-root of the given column. + * + * @group math_funcs + */ + def cbrt(columnName: String): Column = cbrt(Column(columnName)) + + /** + * Computes the ceiling of the given value. + * + * @group math_funcs + */ + def ceil(e: Column): Column = Ceil(e.expr) + + /** + * Computes the ceiling of the given column. + * + * @group math_funcs + */ + def ceil(columnName: String): Column = ceil(Column(columnName)) + + /** + * Computes the cosine of the given value. + * + * @group math_funcs + */ + def cos(e: Column): Column = Cos(e.expr) + + /** + * Computes the cosine of the given column. + * + * @group math_funcs + */ + def cos(columnName: String): Column = cos(Column(columnName)) + + /** + * Computes the hyperbolic cosine of the given value. + * + * @group math_funcs + */ + def cosh(e: Column): Column = Cosh(e.expr) + + /** + * Computes the hyperbolic cosine of the given column. + * + * @group math_funcs + */ + def cosh(columnName: String): Column = cosh(Column(columnName)) + + /** + * Computes the exponential of the given value. + * + * @group math_funcs + */ + def exp(e: Column): Column = Exp(e.expr) + + /** + * Computes the exponential of the given column. + * + * @group math_funcs + */ + def exp(columnName: String): Column = exp(Column(columnName)) + + /** + * Computes the exponential of the given value minus one. + * + * @group math_funcs + */ + def expm1(e: Column): Column = Expm1(e.expr) + + /** + * Computes the exponential of the given column. + * + * @group math_funcs + */ + def expm1(columnName: String): Column = expm1(Column(columnName)) + + /** + * Computes the floor of the given value. + * + * @group math_funcs + */ + def floor(e: Column): Column = Floor(e.expr) + + /** + * Computes the floor of the given column. + * + * @group math_funcs + */ + def floor(columnName: String): Column = floor(Column(columnName)) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * @group math_funcs + */ + def hypot(l: Column, r: Column): Column = Hypot(l.expr, r.expr) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * @group math_funcs + */ + def hypot(l: Column, rightName: String): Column = hypot(l, Column(rightName)) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * @group math_funcs + */ + def hypot(leftName: String, r: Column): Column = hypot(Column(leftName), r) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * @group math_funcs + */ + def hypot(leftName: String, rightName: String): Column = + hypot(Column(leftName), Column(rightName)) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * @group math_funcs + */ + def hypot(l: Column, r: Double): Column = hypot(l, lit(r).expr) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * @group math_funcs + */ + def hypot(leftName: String, r: Double): Column = hypot(Column(leftName), r) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * @group math_funcs + */ + def hypot(l: Double, r: Column): Column = hypot(lit(l).expr, r) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * @group math_funcs + */ + def hypot(l: Double, rightName: String): Column = hypot(l, Column(rightName)) + + /** + * Computes the natural logarithm of the given value. + * + * @group math_funcs + */ + def log(e: Column): Column = Log(e.expr) + + /** + * Computes the natural logarithm of the given column. + * + * @group math_funcs + */ + def log(columnName: String): Column = log(Column(columnName)) + + /** + * Computes the logarithm of the given value in Base 10. + * + * @group math_funcs + */ + def log10(e: Column): Column = Log10(e.expr) + + /** + * Computes the logarithm of the given value in Base 10. + * + * @group math_funcs + */ + def log10(columnName: String): Column = log10(Column(columnName)) + + /** + * Computes the natural logarithm of the given value plus one. + * + * @group math_funcs + */ + def log1p(e: Column): Column = Log1p(e.expr) + + /** + * Computes the natural logarithm of the given column plus one. + * + * @group math_funcs + */ + def log1p(columnName: String): Column = log1p(Column(columnName)) + + /** + * Returns the value of the first argument raised to the power of the second argument. + * + * @group math_funcs + */ + def pow(l: Column, r: Column): Column = Pow(l.expr, r.expr) + + /** + * Returns the value of the first argument raised to the power of the second argument. + * + * @group math_funcs + */ + def pow(l: Column, rightName: String): Column = pow(l, Column(rightName)) + + /** + * Returns the value of the first argument raised to the power of the second argument. + * + * @group math_funcs + */ + def pow(leftName: String, r: Column): Column = pow(Column(leftName), r) + + /** + * Returns the value of the first argument raised to the power of the second argument. + * + * @group math_funcs + */ + def pow(leftName: String, rightName: String): Column = pow(Column(leftName), Column(rightName)) + + /** + * Returns the value of the first argument raised to the power of the second argument. + * + * @group math_funcs + */ + def pow(l: Column, r: Double): Column = pow(l, lit(r).expr) + + /** + * Returns the value of the first argument raised to the power of the second argument. + * + * @group math_funcs + */ + def pow(leftName: String, r: Double): Column = pow(Column(leftName), r) + + /** + * Returns the value of the first argument raised to the power of the second argument. + * + * @group math_funcs + */ + def pow(l: Double, r: Column): Column = pow(lit(l).expr, r) + + /** + * Returns the value of the first argument raised to the power of the second argument. + * + * @group math_funcs + */ + def pow(l: Double, rightName: String): Column = pow(l, Column(rightName)) + + /** + * Returns the double value that is closest in value to the argument and + * is equal to a mathematical integer. + * + * @group math_funcs + */ + def rint(e: Column): Column = Rint(e.expr) + + /** + * Returns the double value that is closest in value to the argument and + * is equal to a mathematical integer. + * + * @group math_funcs + */ + def rint(columnName: String): Column = rint(Column(columnName)) + + /** + * Computes the signum of the given value. + * + * @group math_funcs + */ + def signum(e: Column): Column = Signum(e.expr) + + /** + * Computes the signum of the given column. + * + * @group math_funcs + */ + def signum(columnName: String): Column = signum(Column(columnName)) + + /** + * Computes the sine of the given value. + * + * @group math_funcs + */ + def sin(e: Column): Column = Sin(e.expr) + + /** + * Computes the sine of the given column. + * + * @group math_funcs + */ + def sin(columnName: String): Column = sin(Column(columnName)) + + /** + * Computes the hyperbolic sine of the given value. + * + * @group math_funcs + */ + def sinh(e: Column): Column = Sinh(e.expr) + + /** + * Computes the hyperbolic sine of the given column. + * + * @group math_funcs + */ + def sinh(columnName: String): Column = sinh(Column(columnName)) + + /** + * Computes the tangent of the given value. + * + * @group math_funcs + */ + def tan(e: Column): Column = Tan(e.expr) + + /** + * Computes the tangent of the given column. + * + * @group math_funcs + */ + def tan(columnName: String): Column = tan(Column(columnName)) + + /** + * Computes the hyperbolic tangent of the given value. + * + * @group math_funcs + */ + def tanh(e: Column): Column = Tanh(e.expr) + + /** + * Computes the hyperbolic tangent of the given column. + * + * @group math_funcs + */ + def tanh(columnName: String): Column = tanh(Column(columnName)) + + /** + * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. + * + * @group math_funcs + */ + def toDegrees(e: Column): Column = ToDegrees(e.expr) + + /** + * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. + * + * @group math_funcs + */ + def toDegrees(columnName: String): Column = toDegrees(Column(columnName)) + + /** + * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. + * + * @group math_funcs + */ + def toRadians(e: Column): Column = ToRadians(e.expr) + + /** + * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. + * + * @group math_funcs + */ + def toRadians(columnName: String): Column = toRadians(Column(columnName)) + + ////////////////////////////////////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index 2f6ba48dbc3d9..1a5083dbe0f61 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -362,8 +362,20 @@ private[sql] class JDBCRDD( conversions(i) match { case BooleanConversion => mutableRow.setBoolean(i, rs.getBoolean(pos)) case DateConversion => - mutableRow.update(i, DateUtils.fromJavaDate(rs.getDate(pos))) - case DecimalConversion => mutableRow.update(i, rs.getBigDecimal(pos)) + // DateUtils.fromJavaDate does not handle null value, so we need to check it. + val dateVal = rs.getDate(pos) + if (dateVal != null) { + mutableRow.update(i, DateUtils.fromJavaDate(dateVal)) + } else { + mutableRow.update(i, null) + } + case DecimalConversion => + val decimalVal = rs.getBigDecimal(pos) + if (decimalVal == null) { + mutableRow.update(i, null) + } else { + mutableRow.update(i, Decimal(decimalVal)) + } case DoubleConversion => mutableRow.setDouble(i, rs.getDouble(pos)) case FloatConversion => mutableRow.setFloat(i, rs.getFloat(pos)) case IntegerConversion => mutableRow.setInt(i, rs.getInt(pos)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/InferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/InferSchema.scala new file mode 100644 index 0000000000000..9c58b8e4bb16a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/InferSchema.scala @@ -0,0 +1,171 @@ +/* + * 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.json + +import com.fasterxml.jackson.core._ + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion +import org.apache.spark.sql.json.JacksonUtils.nextUntil +import org.apache.spark.sql.types._ + +private[sql] object InferSchema { + /** + * Infer the type of a collection of json records in three stages: + * 1. Infer the type of each record + * 2. Merge types by choosing the lowest type necessary to cover equal keys + * 3. Replace any remaining null fields with string, the top type + */ + def apply( + json: RDD[String], + samplingRatio: Double = 1.0, + columnNameOfCorruptRecords: String): StructType = { + require(samplingRatio > 0, s"samplingRatio ($samplingRatio) should be greater than 0") + val schemaData = if (samplingRatio > 0.99) { + json + } else { + json.sample(withReplacement = false, samplingRatio, 1) + } + + // perform schema inference on each row and merge afterwards + schemaData.mapPartitions { iter => + val factory = new JsonFactory() + iter.map { row => + try { + val parser = factory.createParser(row) + parser.nextToken() + inferField(parser) + } catch { + case _: JsonParseException => + StructType(Seq(StructField(columnNameOfCorruptRecords, StringType))) + } + } + }.treeAggregate[DataType](StructType(Seq()))(compatibleRootType, compatibleRootType) match { + case st: StructType => nullTypeToStringType(st) + } + } + + /** + * Infer the type of a json document from the parser's token stream + */ + private def inferField(parser: JsonParser): DataType = { + import com.fasterxml.jackson.core.JsonToken._ + parser.getCurrentToken match { + case null | VALUE_NULL => NullType + + case FIELD_NAME => + parser.nextToken() + inferField(parser) + + case VALUE_STRING if parser.getTextLength < 1 => + // Zero length strings and nulls have special handling to deal + // with JSON generators that do not distinguish between the two. + // To accurately infer types for empty strings that are really + // meant to represent nulls we assume that the two are isomorphic + // but will defer treating null fields as strings until all the + // record fields' types have been combined. + NullType + + case VALUE_STRING => StringType + case START_OBJECT => + val builder = Seq.newBuilder[StructField] + while (nextUntil(parser, END_OBJECT)) { + builder += StructField(parser.getCurrentName, inferField(parser), nullable = true) + } + + StructType(builder.result().sortBy(_.name)) + + case START_ARRAY => + // If this JSON array is empty, we use NullType as a placeholder. + // If this array is not empty in other JSON objects, we can resolve + // the type as we pass through all JSON objects. + var elementType: DataType = NullType + while (nextUntil(parser, END_ARRAY)) { + elementType = compatibleType(elementType, inferField(parser)) + } + + ArrayType(elementType) + + case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT => + import JsonParser.NumberType._ + parser.getNumberType match { + // For Integer values, use LongType by default. + case INT | LONG => LongType + // Since we do not have a data type backed by BigInteger, + // when we see a Java BigInteger, we use DecimalType. + case BIG_INTEGER | BIG_DECIMAL => DecimalType.Unlimited + case FLOAT | DOUBLE => DoubleType + } + + case VALUE_TRUE | VALUE_FALSE => BooleanType + } + } + + private def nullTypeToStringType(struct: StructType): StructType = { + val fields = struct.fields.map { + case StructField(fieldName, dataType, nullable, _) => + val newType = dataType match { + case NullType => StringType + case ArrayType(NullType, containsNull) => ArrayType(StringType, containsNull) + case ArrayType(struct: StructType, containsNull) => + ArrayType(nullTypeToStringType(struct), containsNull) + case struct: StructType =>nullTypeToStringType(struct) + case other: DataType => other + } + + StructField(fieldName, newType, nullable) + } + + StructType(fields) + } + + /** + * Remove top-level ArrayType wrappers and merge the remaining schemas + */ + private def compatibleRootType: (DataType, DataType) => DataType = { + case (ArrayType(ty1, _), ty2) => compatibleRootType(ty1, ty2) + case (ty1, ArrayType(ty2, _)) => compatibleRootType(ty1, ty2) + case (ty1, ty2) => compatibleType(ty1, ty2) + } + + /** + * Returns the most general data type for two given data types. + */ + private[json] def compatibleType(t1: DataType, t2: DataType): DataType = { + HiveTypeCoercion.findTightestCommonType(t1, t2).getOrElse { + // t1 or t2 is a StructType, ArrayType, or an unexpected type. + (t1, t2) match { + case (other: DataType, NullType) => other + case (NullType, other: DataType) => other + case (StructType(fields1), StructType(fields2)) => + val newFields = (fields1 ++ fields2).groupBy(field => field.name).map { + case (name, fieldTypes) => + val dataType = fieldTypes.view.map(_.dataType).reduce(compatibleType) + StructField(name, dataType, nullable = true) + } + StructType(newFields.toSeq.sortBy(_.name)) + + case (ArrayType(elementType1, containsNull1), ArrayType(elementType2, containsNull2)) => + ArrayType(compatibleType(elementType1, elementType2), containsNull1 || containsNull2) + + // strings and every string is a Json object. + case (_, _) => StringType + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index e3352d02787fd..c772cd1f53e53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -22,14 +22,16 @@ import java.io.IOException import org.apache.hadoop.fs.Path import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute, Row} import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} private[sql] class DefaultSource - extends RelationProvider with SchemaRelationProvider with CreatableRelationProvider { + extends RelationProvider + with SchemaRelationProvider + with CreatableRelationProvider { private def checkPath(parameters: Map[String, String]): String = { parameters.getOrElse("path", sys.error("'path' must be specified for json data.")) @@ -42,7 +44,7 @@ private[sql] class DefaultSource val path = checkPath(parameters) val samplingRatio = parameters.get("samplingRatio").map(_.toDouble).getOrElse(1.0) - JSONRelation(path, samplingRatio, None)(sqlContext) + new JSONRelation(path, samplingRatio, None, sqlContext) } /** Returns a new base relation with the given schema and parameters. */ @@ -53,7 +55,7 @@ private[sql] class DefaultSource val path = checkPath(parameters) val samplingRatio = parameters.get("samplingRatio").map(_.toDouble).getOrElse(1.0) - JSONRelation(path, samplingRatio, Some(schema))(sqlContext) + new JSONRelation(path, samplingRatio, Some(schema), sqlContext) } override def createRelation( @@ -101,32 +103,87 @@ private[sql] class DefaultSource } } -private[sql] case class JSONRelation( - path: String, - samplingRatio: Double, +private[sql] class JSONRelation( + // baseRDD is not immutable with respect to INSERT OVERWRITE + // and so it must be recreated at least as often as the + // underlying inputs are modified. To be safe, a function is + // used instead of a regular RDD value to ensure a fresh RDD is + // recreated for each and every operation. + baseRDD: () => RDD[String], + val path: Option[String], + val samplingRatio: Double, userSpecifiedSchema: Option[StructType])( @transient val sqlContext: SQLContext) extends BaseRelation with TableScan - with InsertableRelation { - - // TODO: Support partitioned JSON relation. - private def baseRDD = sqlContext.sparkContext.textFile(path) + with InsertableRelation + with CatalystScan { + + def this( + path: String, + samplingRatio: Double, + userSpecifiedSchema: Option[StructType], + sqlContext: SQLContext) = + this( + () => sqlContext.sparkContext.textFile(path), + Some(path), + samplingRatio, + userSpecifiedSchema)(sqlContext) + + private val useJacksonStreamingAPI: Boolean = sqlContext.conf.useJacksonStreamingAPI override val needConversion: Boolean = false - override val schema = userSpecifiedSchema.getOrElse( - JsonRDD.nullTypeToStringType( - JsonRDD.inferSchema( - baseRDD, + override lazy val schema = userSpecifiedSchema.getOrElse { + if (useJacksonStreamingAPI) { + InferSchema( + baseRDD(), samplingRatio, - sqlContext.conf.columnNameOfCorruptRecord))) + sqlContext.conf.columnNameOfCorruptRecord) + } else { + JsonRDD.nullTypeToStringType( + JsonRDD.inferSchema( + baseRDD(), + samplingRatio, + sqlContext.conf.columnNameOfCorruptRecord)) + } + } - override def buildScan(): RDD[Row] = - JsonRDD.jsonStringToRow(baseRDD, schema, sqlContext.conf.columnNameOfCorruptRecord) + override def buildScan(): RDD[Row] = { + if (useJacksonStreamingAPI) { + JacksonParser( + baseRDD(), + schema, + sqlContext.conf.columnNameOfCorruptRecord) + } else { + JsonRDD.jsonStringToRow( + baseRDD(), + schema, + sqlContext.conf.columnNameOfCorruptRecord) + } + } + + override def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] = { + if (useJacksonStreamingAPI) { + JacksonParser( + baseRDD(), + StructType.fromAttributes(requiredColumns), + sqlContext.conf.columnNameOfCorruptRecord) + } else { + JsonRDD.jsonStringToRow( + baseRDD(), + StructType.fromAttributes(requiredColumns), + sqlContext.conf.columnNameOfCorruptRecord) + } + } override def insert(data: DataFrame, overwrite: Boolean): Unit = { - val filesystemPath = new Path(path) + val filesystemPath = path match { + case Some(p) => new Path(p) + case None => + throw new IOException(s"Cannot INSERT into table with no path defined") + } + val fs = filesystemPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) if (overwrite) { @@ -147,7 +204,7 @@ private[sql] case class JSONRelation( } } // Write the data. - data.toJSON.saveAsTextFile(path) + data.toJSON.saveAsTextFile(filesystemPath.toString) // Right now, we assume that the schema is not changed. We will not update the schema. // schema = data.schema } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonGenerator.scala new file mode 100644 index 0000000000000..80bf74aa02602 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonGenerator.scala @@ -0,0 +1,77 @@ +/* + * 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.json + +import scala.collection.Map + +import com.fasterxml.jackson.core._ + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types._ + +private[sql] object JacksonGenerator { + /** Transforms a single Row to JSON using Jackson + * + * @param rowSchema the schema object used for conversion + * @param gen a JsonGenerator object + * @param row The row to convert + */ + def apply(rowSchema: StructType, gen: JsonGenerator)(row: Row): Unit = { + def valWriter: (DataType, Any) => Unit = { + case (_, null) | (NullType, _) => gen.writeNull() + case (StringType, v: String) => gen.writeString(v) + case (TimestampType, v: java.sql.Timestamp) => gen.writeString(v.toString) + case (IntegerType, v: Int) => gen.writeNumber(v) + case (ShortType, v: Short) => gen.writeNumber(v) + case (FloatType, v: Float) => gen.writeNumber(v) + case (DoubleType, v: Double) => gen.writeNumber(v) + case (LongType, v: Long) => gen.writeNumber(v) + case (DecimalType(), v: java.math.BigDecimal) => gen.writeNumber(v) + case (ByteType, v: Byte) => gen.writeNumber(v.toInt) + case (BinaryType, v: Array[Byte]) => gen.writeBinary(v) + case (BooleanType, v: Boolean) => gen.writeBoolean(v) + case (DateType, v) => gen.writeString(v.toString) + case (udt: UserDefinedType[_], v) => valWriter(udt.sqlType, udt.serialize(v)) + + case (ArrayType(ty, _), v: Seq[_] ) => + gen.writeStartArray() + v.foreach(valWriter(ty,_)) + gen.writeEndArray() + + case (MapType(kv,vv, _), v: Map[_,_]) => + gen.writeStartObject() + v.foreach { p => + gen.writeFieldName(p._1.toString) + valWriter(vv,p._2) + } + gen.writeEndObject() + + case (StructType(ty), v: Row) => + gen.writeStartObject() + ty.zip(v.toSeq).foreach { + case (_, null) => + case (field, v) => + gen.writeFieldName(field.name) + valWriter(field.dataType, v) + } + gen.writeEndObject() + } + + valWriter(rowSchema, row) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala new file mode 100644 index 0000000000000..a8e69ae61174f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala @@ -0,0 +1,215 @@ +/* + * 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.json + +import java.io.ByteArrayOutputStream +import java.sql.Timestamp + +import scala.collection.Map + +import com.fasterxml.jackson.core._ + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.json.JacksonUtils.nextUntil +import org.apache.spark.sql.types._ + +private[sql] object JacksonParser { + def apply( + json: RDD[String], + schema: StructType, + columnNameOfCorruptRecords: String): RDD[Row] = { + parseJson(json, schema, columnNameOfCorruptRecords) + } + + /** + * Parse the current token (and related children) according to a desired schema + */ + private[sql] def convertField( + factory: JsonFactory, + parser: JsonParser, + schema: DataType): Any = { + import com.fasterxml.jackson.core.JsonToken._ + (parser.getCurrentToken, schema) match { + case (null | VALUE_NULL, _) => + null + + case (FIELD_NAME, _) => + parser.nextToken() + convertField(factory, parser, schema) + + case (VALUE_STRING, StringType) => + UTF8String(parser.getText) + + case (VALUE_STRING, _) if parser.getTextLength < 1 => + // guard the non string type + null + + case (VALUE_STRING, DateType) => + DateUtils.millisToDays(DateUtils.stringToTime(parser.getText).getTime) + + case (VALUE_STRING, TimestampType) => + new Timestamp(DateUtils.stringToTime(parser.getText).getTime) + + case (VALUE_NUMBER_INT, TimestampType) => + new Timestamp(parser.getLongValue) + + case (_, StringType) => + val writer = new ByteArrayOutputStream() + val generator = factory.createGenerator(writer, JsonEncoding.UTF8) + generator.copyCurrentStructure(parser) + generator.close() + UTF8String(writer.toByteArray) + + case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) => + parser.getFloatValue + + case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) => + parser.getDoubleValue + + case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DecimalType()) => + // TODO: add fixed precision and scale handling + Decimal(parser.getDecimalValue) + + case (VALUE_NUMBER_INT, ByteType) => + parser.getByteValue + + case (VALUE_NUMBER_INT, ShortType) => + parser.getShortValue + + case (VALUE_NUMBER_INT, IntegerType) => + parser.getIntValue + + case (VALUE_NUMBER_INT, LongType) => + parser.getLongValue + + case (VALUE_TRUE, BooleanType) => + true + + case (VALUE_FALSE, BooleanType) => + false + + case (START_OBJECT, st: StructType) => + convertObject(factory, parser, st) + + case (START_ARRAY, ArrayType(st, _)) => + convertList(factory, parser, st) + + case (START_OBJECT, ArrayType(st, _)) => + // the business end of SPARK-3308: + // when an object is found but an array is requested just wrap it in a list + convertField(factory, parser, st) :: Nil + + case (START_OBJECT, MapType(StringType, kt, _)) => + convertMap(factory, parser, kt) + + case (_, udt: UserDefinedType[_]) => + udt.deserialize(convertField(factory, parser, udt.sqlType)) + } + } + + /** + * Parse an object from the token stream into a new Row representing the schema. + * + * Fields in the json that are not defined in the requested schema will be dropped. + */ + private def convertObject(factory: JsonFactory, parser: JsonParser, schema: StructType): Row = { + val row = new GenericMutableRow(schema.length) + while (nextUntil(parser, JsonToken.END_OBJECT)) { + schema.getFieldIndex(parser.getCurrentName) match { + case Some(index) => + row.update(index, convertField(factory, parser, schema(index).dataType)) + + case None => + parser.skipChildren() + } + } + + row + } + + /** + * Parse an object as a Map, preserving all fields + */ + private def convertMap( + factory: JsonFactory, + parser: JsonParser, + valueType: DataType): Map[String, Any] = { + val builder = Map.newBuilder[String, Any] + while (nextUntil(parser, JsonToken.END_OBJECT)) { + builder += parser.getCurrentName -> convertField(factory, parser, valueType) + } + + builder.result() + } + + private def convertList( + factory: JsonFactory, + parser: JsonParser, + schema: DataType): Seq[Any] = { + val builder = Seq.newBuilder[Any] + while (nextUntil(parser, JsonToken.END_ARRAY)) { + builder += convertField(factory, parser, schema) + } + + builder.result() + } + + private def parseJson( + json: RDD[String], + schema: StructType, + columnNameOfCorruptRecords: String): RDD[Row] = { + + def failedRecord(record: String): Seq[Row] = { + // create a row even if no corrupt record column is present + val row = new GenericMutableRow(schema.length) + for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecords)) { + require(schema(corruptIndex).dataType == StringType) + row.update(corruptIndex, record) + } + + Seq(row) + } + + json.mapPartitions { iter => + val factory = new JsonFactory() + + iter.flatMap { record => + try { + val parser = factory.createParser(record) + parser.nextToken() + + // to support both object and arrays (see SPARK-3308) we'll start + // by converting the StructType schema to an ArrayType and let + // convertField wrap an object into a single value array when necessary. + convertField(factory, parser, ArrayType(schema)) match { + case null => failedRecord(record) + case list: Seq[Row @unchecked] => list + case _ => + sys.error( + s"Failed to parse record $record. Please make sure that each line of the file " + + "(or each string in the RDD) is a valid JSON object or an array of JSON objects.") + } + } catch { + case _: JsonProcessingException => + failedRecord(record) + } + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonUtils.scala new file mode 100644 index 0000000000000..fde96852ce68e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonUtils.scala @@ -0,0 +1,32 @@ +/* + * 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.json + +import com.fasterxml.jackson.core.{JsonParser, JsonToken} + +private object JacksonUtils { + /** + * Advance the parser until a null or a specific token is found + */ + def nextUntil(parser: JsonParser, stopOn: JsonToken): Boolean = { + parser.nextToken() match { + case null => false + case x => x != stopOn + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 6e94e7056eb0b..f62973d5fcfab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -440,54 +440,4 @@ private[sql] object JsonRDD extends Logging { row } - - /** Transforms a single Row to JSON using Jackson - * - * @param rowSchema the schema object used for conversion - * @param gen a JsonGenerator object - * @param row The row to convert - */ - private[sql] def rowToJSON(rowSchema: StructType, gen: JsonGenerator)(row: Row) = { - def valWriter: (DataType, Any) => Unit = { - case (_, null) | (NullType, _) => gen.writeNull() - case (StringType, v: String) => gen.writeString(v) - case (TimestampType, v: java.sql.Timestamp) => gen.writeString(v.toString) - case (IntegerType, v: Int) => gen.writeNumber(v) - case (ShortType, v: Short) => gen.writeNumber(v) - case (FloatType, v: Float) => gen.writeNumber(v) - case (DoubleType, v: Double) => gen.writeNumber(v) - case (LongType, v: Long) => gen.writeNumber(v) - case (DecimalType(), v: java.math.BigDecimal) => gen.writeNumber(v) - case (ByteType, v: Byte) => gen.writeNumber(v.toInt) - case (BinaryType, v: Array[Byte]) => gen.writeBinary(v) - case (BooleanType, v: Boolean) => gen.writeBoolean(v) - case (DateType, v) => gen.writeString(v.toString) - case (udt: UserDefinedType[_], v) => valWriter(udt.sqlType, v) - - case (ArrayType(ty, _), v: Seq[_] ) => - gen.writeStartArray() - v.foreach(valWriter(ty,_)) - gen.writeEndArray() - - case (MapType(kv,vv, _), v: Map[_,_]) => - gen.writeStartObject() - v.foreach { p => - gen.writeFieldName(p._1.toString) - valWriter(vv,p._2) - } - gen.writeEndObject() - - case (StructType(ty), v: Row) => - gen.writeStartObject() - ty.zip(v.toSeq).foreach { - case (_, null) => - case (field, v) => - gen.writeFieldName(field.name) - valWriter(field.dataType, v) - } - gen.writeEndObject() - } - - valWriter(rowSchema, row) - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/mathfunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/mathfunctions.scala deleted file mode 100644 index db47480c3864d..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/mathfunctions.scala +++ /dev/null @@ -1,383 +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 - -import scala.language.implicitConversions - -import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.mathfuncs._ -import org.apache.spark.sql.functions.lit - -/** - * :: Experimental :: - * Mathematical Functions available for [[DataFrame]]. - */ -@Experimental -// scalastyle:off -object mathfunctions { -// scalastyle:on - - private[this] implicit def toColumn(expr: Expression): Column = Column(expr) - - /** - * Computes the cosine inverse of the given value; the returned angle is in the range - * 0.0 through pi. - */ - def acos(e: Column): Column = Acos(e.expr) - - /** - * Computes the cosine inverse of the given column; the returned angle is in the range - * 0.0 through pi. - */ - def acos(columnName: String): Column = acos(Column(columnName)) - - /** - * Computes the sine inverse of the given value; the returned angle is in the range - * -pi/2 through pi/2. - */ - def asin(e: Column): Column = Asin(e.expr) - - /** - * Computes the sine inverse of the given column; the returned angle is in the range - * -pi/2 through pi/2. - */ - def asin(columnName: String): Column = asin(Column(columnName)) - - /** - * Computes the tangent inverse of the given value. - */ - def atan(e: Column): Column = Atan(e.expr) - - /** - * Computes the tangent inverse of the given column. - */ - def atan(columnName: String): Column = atan(Column(columnName)) - - /** - * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). - */ - def atan2(l: Column, r: Column): Column = Atan2(l.expr, r.expr) - - /** - * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). - */ - def atan2(l: Column, rightName: String): Column = atan2(l, Column(rightName)) - - /** - * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). - */ - def atan2(leftName: String, r: Column): Column = atan2(Column(leftName), r) - - /** - * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). - */ - def atan2(leftName: String, rightName: String): Column = - atan2(Column(leftName), Column(rightName)) - - /** - * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). - */ - def atan2(l: Column, r: Double): Column = atan2(l, lit(r).expr) - - /** - * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta).= - */ - def atan2(leftName: String, r: Double): Column = atan2(Column(leftName), r) - - /** - * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). - */ - def atan2(l: Double, r: Column): Column = atan2(lit(l).expr, r) - - /** - * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). - */ - def atan2(l: Double, rightName: String): Column = atan2(l, Column(rightName)) - - /** - * Computes the cube-root of the given value. - */ - def cbrt(e: Column): Column = Cbrt(e.expr) - - /** - * Computes the cube-root of the given column. - */ - def cbrt(columnName: String): Column = cbrt(Column(columnName)) - - /** - * Computes the ceiling of the given value. - */ - def ceil(e: Column): Column = Ceil(e.expr) - - /** - * Computes the ceiling of the given column. - */ - def ceil(columnName: String): Column = ceil(Column(columnName)) - - /** - * Computes the cosine of the given value. - */ - def cos(e: Column): Column = Cos(e.expr) - - /** - * Computes the cosine of the given column. - */ - def cos(columnName: String): Column = cos(Column(columnName)) - - /** - * Computes the hyperbolic cosine of the given value. - */ - def cosh(e: Column): Column = Cosh(e.expr) - - /** - * Computes the hyperbolic cosine of the given column. - */ - def cosh(columnName: String): Column = cosh(Column(columnName)) - - /** - * Computes the exponential of the given value. - */ - def exp(e: Column): Column = Exp(e.expr) - - /** - * Computes the exponential of the given column. - */ - def exp(columnName: String): Column = exp(Column(columnName)) - - /** - * Computes the exponential of the given value minus one. - */ - def expm1(e: Column): Column = Expm1(e.expr) - - /** - * Computes the exponential of the given column. - */ - def expm1(columnName: String): Column = expm1(Column(columnName)) - - /** - * Computes the floor of the given value. - */ - def floor(e: Column): Column = Floor(e.expr) - - /** - * Computes the floor of the given column. - */ - def floor(columnName: String): Column = floor(Column(columnName)) - - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - */ - def hypot(l: Column, r: Column): Column = Hypot(l.expr, r.expr) - - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - */ - def hypot(l: Column, rightName: String): Column = hypot(l, Column(rightName)) - - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - */ - def hypot(leftName: String, r: Column): Column = hypot(Column(leftName), r) - - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - */ - def hypot(leftName: String, rightName: String): Column = - hypot(Column(leftName), Column(rightName)) - - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - */ - def hypot(l: Column, r: Double): Column = hypot(l, lit(r).expr) - - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - */ - def hypot(leftName: String, r: Double): Column = hypot(Column(leftName), r) - - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - */ - def hypot(l: Double, r: Column): Column = hypot(lit(l).expr, r) - - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - */ - def hypot(l: Double, rightName: String): Column = hypot(l, Column(rightName)) - - /** - * Computes the natural logarithm of the given value. - */ - def log(e: Column): Column = Log(e.expr) - - /** - * Computes the natural logarithm of the given column. - */ - def log(columnName: String): Column = log(Column(columnName)) - - /** - * Computes the logarithm of the given value in Base 10. - */ - def log10(e: Column): Column = Log10(e.expr) - - /** - * Computes the logarithm of the given value in Base 10. - */ - def log10(columnName: String): Column = log10(Column(columnName)) - - /** - * Computes the natural logarithm of the given value plus one. - */ - def log1p(e: Column): Column = Log1p(e.expr) - - /** - * Computes the natural logarithm of the given column plus one. - */ - def log1p(columnName: String): Column = log1p(Column(columnName)) - - /** - * Returns the value of the first argument raised to the power of the second argument. - */ - def pow(l: Column, r: Column): Column = Pow(l.expr, r.expr) - - /** - * Returns the value of the first argument raised to the power of the second argument. - */ - def pow(l: Column, rightName: String): Column = pow(l, Column(rightName)) - - /** - * Returns the value of the first argument raised to the power of the second argument. - */ - def pow(leftName: String, r: Column): Column = pow(Column(leftName), r) - - /** - * Returns the value of the first argument raised to the power of the second argument. - */ - def pow(leftName: String, rightName: String): Column = pow(Column(leftName), Column(rightName)) - - /** - * Returns the value of the first argument raised to the power of the second argument. - */ - def pow(l: Column, r: Double): Column = pow(l, lit(r).expr) - - /** - * Returns the value of the first argument raised to the power of the second argument. - */ - def pow(leftName: String, r: Double): Column = pow(Column(leftName), r) - - /** - * Returns the value of the first argument raised to the power of the second argument. - */ - def pow(l: Double, r: Column): Column = pow(lit(l).expr, r) - - /** - * Returns the value of the first argument raised to the power of the second argument. - */ - def pow(l: Double, rightName: String): Column = pow(l, Column(rightName)) - - /** - * Returns the double value that is closest in value to the argument and - * is equal to a mathematical integer. - */ - def rint(e: Column): Column = Rint(e.expr) - - /** - * Returns the double value that is closest in value to the argument and - * is equal to a mathematical integer. - */ - def rint(columnName: String): Column = rint(Column(columnName)) - - /** - * Computes the signum of the given value. - */ - def signum(e: Column): Column = Signum(e.expr) - - /** - * Computes the signum of the given column. - */ - def signum(columnName: String): Column = signum(Column(columnName)) - - /** - * Computes the sine of the given value. - */ - def sin(e: Column): Column = Sin(e.expr) - - /** - * Computes the sine of the given column. - */ - def sin(columnName: String): Column = sin(Column(columnName)) - - /** - * Computes the hyperbolic sine of the given value. - */ - def sinh(e: Column): Column = Sinh(e.expr) - - /** - * Computes the hyperbolic sine of the given column. - */ - def sinh(columnName: String): Column = sinh(Column(columnName)) - - /** - * Computes the tangent of the given value. - */ - def tan(e: Column): Column = Tan(e.expr) - - /** - * Computes the tangent of the given column. - */ - def tan(columnName: String): Column = tan(Column(columnName)) - - /** - * Computes the hyperbolic tangent of the given value. - */ - def tanh(e: Column): Column = Tanh(e.expr) - - /** - * Computes the hyperbolic tangent of the given column. - */ - def tanh(columnName: String): Column = tanh(Column(columnName)) - - /** - * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. - */ - def toDeg(e: Column): Column = ToDegrees(e.expr) - - /** - * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. - */ - def toDeg(columnName: String): Column = toDeg(Column(columnName)) - - /** - * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. - */ - def toRad(e: Column): Column = ToRadians(e.expr) - - /** - * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. - */ - def toRad(columnName: String): Column = toRad(Column(columnName)) -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 1abf3aa51cb25..06c64f2bdd59e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -269,8 +269,10 @@ private[sql] case class ResolvedDataSource(provider: Class[_], relation: BaseRel */ private[sql] case class DescribeCommand( table: LogicalPlan, - isExtended: Boolean) extends Command { - override val output = Seq( + isExtended: Boolean) extends LogicalPlan with Command { + + override def children: Seq[LogicalPlan] = Seq.empty + override val output: Seq[Attribute] = Seq( // Column names are based on Hive. AttributeReference("col_name", StringType, nullable = false, new MetadataBuilder().putString("comment", "name of the column").build())(), @@ -292,7 +294,11 @@ private[sql] case class CreateTableUsing( temporary: Boolean, options: Map[String, String], allowExisting: Boolean, - managedIfNoPath: Boolean) extends Command + managedIfNoPath: Boolean) extends LogicalPlan with Command { + + override def output: Seq[Attribute] = Seq.empty + override def children: Seq[LogicalPlan] = Seq.empty +} /** * A node used to support CTAS statements and saveAsTable for the data source API. @@ -318,7 +324,7 @@ private[sql] case class CreateTempTableUsing( provider: String, options: Map[String, String]) extends RunnableCommand { - def run(sqlContext: SQLContext): Seq[Row] = { + override def run(sqlContext: SQLContext): Seq[Row] = { val resolved = ResolvedDataSource(sqlContext, userSpecifiedSchema, provider, options) sqlContext.registerDataFrameAsTable( DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) @@ -333,7 +339,7 @@ private[sql] case class CreateTempTableUsingAsSelect( options: Map[String, String], query: LogicalPlan) extends RunnableCommand { - def run(sqlContext: SQLContext): Seq[Row] = { + override def run(sqlContext: SQLContext): Seq[Row] = { val df = DataFrame(sqlContext, query) val resolved = ResolvedDataSource(sqlContext, provider, mode, options, df) sqlContext.registerDataFrameAsTable( diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java index 58cc8e5be6075..72c42f4fe376b 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -39,7 +39,6 @@ import java.util.Map; import static org.apache.spark.sql.functions.*; -import static org.apache.spark.sql.mathfunctions.*; public class JavaDataFrameSuite { private transient JavaSparkContext jsc; 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 3c1ad656fc855..d96186c268720 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 @@ -27,8 +27,6 @@ import org.apache.spark.sql.types._ class ColumnExpressionSuite extends QueryTest { import org.apache.spark.sql.TestData._ - // TODO: Add test cases for bitwise operations. - test("collect on column produced by a binary operator") { val df = Seq((1, 2, 3)).toDF("a", "b", "c") checkAnswer(df.select(df("a") + df("b")), Seq(Row(3))) @@ -385,4 +383,35 @@ class ColumnExpressionSuite extends QueryTest { assert(row.getDouble(1) >= -4.0) } } + + test("bitwiseAND") { + checkAnswer( + testData2.select($"a".bitwiseAND(75)), + testData2.collect().toSeq.map(r => Row(r.getInt(0) & 75))) + + checkAnswer( + testData2.select($"a".bitwiseAND($"b").bitwiseAND(22)), + testData2.collect().toSeq.map(r => Row(r.getInt(0) & r.getInt(1) & 22))) + } + + test("bitwiseOR") { + checkAnswer( + testData2.select($"a".bitwiseOR(170)), + testData2.collect().toSeq.map(r => Row(r.getInt(0) | 170))) + + checkAnswer( + testData2.select($"a".bitwiseOR($"b").bitwiseOR(42)), + testData2.collect().toSeq.map(r => Row(r.getInt(0) | r.getInt(1) | 42))) + } + + test("bitwiseXOR") { + checkAnswer( + testData2.select($"a".bitwiseXOR(112)), + testData2.collect().toSeq.map(r => Row(r.getInt(0) ^ 112))) + + checkAnswer( + testData2.select($"a".bitwiseXOR($"b").bitwiseXOR(39)), + testData2.collect().toSeq.map(r => Row(r.getInt(0) ^ r.getInt(1) ^ 39))) + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index ca03713ef4658..b1e0faa310b68 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.sql.TestData._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.types._ @@ -81,4 +82,10 @@ class DataFrameFunctionsSuite extends QueryTest { struct(col("a") * 2) } } + + test("bitwiseNOT") { + checkAnswer( + testData2.select(bitwiseNOT($"a")), + testData2.collect().toSeq.map(r => Row(~r.getInt(0)))) + } } 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 new file mode 100644 index 0000000000000..f005f55b6432e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -0,0 +1,86 @@ +/* + * 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.TestData._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.test.TestSQLContext.implicits._ + + +class DataFrameJoinSuite extends QueryTest { + + test("join - join using") { + val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") + val df2 = Seq(1, 2, 3).map(i => (i, (i + 1).toString)).toDF("int", "str") + + checkAnswer( + df.join(df2, "int"), + Row(1, "1", "2") :: Row(2, "2", "3") :: Row(3, "3", "4") :: Nil) + } + + test("join - join using self join") { + val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") + + // self join + checkAnswer( + df.join(df, "int"), + Row(1, "1", "1") :: Row(2, "2", "2") :: Row(3, "3", "3") :: Nil) + } + + test("join - self join") { + val df1 = testData.select(testData("key")).as('df1) + val df2 = testData.select(testData("key")).as('df2) + + checkAnswer( + df1.join(df2, $"df1.key" === $"df2.key"), + sql("SELECT a.key, b.key FROM testData a JOIN testData b ON a.key = b.key").collect().toSeq) + } + + test("join - using aliases after self join") { + val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") + checkAnswer( + df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("x.str").count(), + Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) + + checkAnswer( + df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("y.str").count(), + Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) + } + + test("[SPARK-6231] join - self join auto resolve ambiguity") { + val df = Seq((1, "1"), (2, "2")).toDF("key", "value") + checkAnswer( + df.join(df, df("key") === df("key")), + Row(1, "1", 1, "1") :: Row(2, "2", 2, "2") :: Nil) + + checkAnswer( + df.join(df.filter($"value" === "2"), df("key") === df("key")), + Row(2, "2", 2, "2") :: Nil) + + checkAnswer( + df.join(df, df("key") === df("key") && df("value") === 1), + Row(1, "1", 1, "1") :: Nil) + + val left = df.groupBy("key").agg($"key", count("*")) + val right = df.groupBy("key").agg($"key", sum("key")) + checkAnswer( + left.join(right, left("key") === right("key")), + Row(1, 1, 1, 1) :: Row(2, 1, 2, 2) :: Nil) + } +} 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 ff31e15e2d472..d2ca8dccae574 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 @@ -24,7 +24,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.test.{ExamplePointUDT, ExamplePoint, TestSQLContext} import org.apache.spark.sql.test.TestSQLContext.logicalPlanToSparkQuery import org.apache.spark.sql.test.TestSQLContext.implicits._ -import org.apache.spark.sql.test.TestSQLContext.sql class DataFrameSuite extends QueryTest { @@ -118,44 +117,6 @@ class DataFrameSuite extends QueryTest { ) } - test("join - join using") { - val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") - val df2 = Seq(1, 2, 3).map(i => (i, (i + 1).toString)).toDF("int", "str") - - checkAnswer( - df.join(df2, "int"), - Row(1, "1", "2") :: Row(2, "2", "3") :: Row(3, "3", "4") :: Nil) - } - - test("join - join using self join") { - val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") - - // self join - checkAnswer( - df.join(df, "int"), - Row(1, "1", "1") :: Row(2, "2", "2") :: Row(3, "3", "3") :: Nil) - } - - test("join - self join") { - val df1 = testData.select(testData("key")).as('df1) - val df2 = testData.select(testData("key")).as('df2) - - checkAnswer( - df1.join(df2, $"df1.key" === $"df2.key"), - sql("SELECT a.key, b.key FROM testData a JOIN testData b ON a.key = b.key").collect().toSeq) - } - - test("join - using aliases after self join") { - val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") - checkAnswer( - df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("x.str").count(), - Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) - - checkAnswer( - df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("y.str").count(), - Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) - } - test("explode") { val df = Seq((1, "a b c"), (2, "a b"), (3, "a")).toDF("number", "letters") val df2 = @@ -488,7 +449,7 @@ class DataFrameSuite extends QueryTest { testData.collect().map { case Row(key: Int, value: String) => Row(key, value, key + 1) }.toSeq) - assert(df.schema.map(_.name).toSeq === Seq("key", "value", "newCol")) + assert(df.schema.map(_.name) === Seq("key", "value", "newCol")) } test("replace column using withColumn") { @@ -523,7 +484,7 @@ class DataFrameSuite extends QueryTest { testData.collect().map { case Row(key: Int, value: String) => Row(key, value, key + 1) }.toSeq) - assert(df.schema.map(_.name).toSeq === Seq("key", "valueRenamed", "newCol")) + assert(df.schema.map(_.name) === Seq("key", "valueRenamed", "newCol")) } test("randomSplit") { @@ -632,4 +593,10 @@ class DataFrameSuite extends QueryTest { Row(new java.math.BigDecimal(2.0))) TestSQLContext.setConf(SQLConf.CODEGEN_ENABLED, originalValue.toString) } + + test("SPARK-7133: Implement struct, array, and map field accessor") { + assert(complexData.filter(complexData("a")(0) === 2).count() == 1) + assert(complexData.filter(complexData("m")("1") === 1).count() == 1) + assert(complexData.filter(complexData("s")("key") === 1).count() == 1) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala index 9e19bb7482e9b..c4281c4b55c02 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala @@ -19,8 +19,7 @@ package org.apache.spark.sql import java.lang.{Double => JavaDouble} -import org.apache.spark.sql.functions.lit -import org.apache.spark.sql.mathfunctions._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.implicits._ @@ -159,11 +158,11 @@ class MathExpressionsSuite extends QueryTest { } test("toDeg") { - testOneToOneMathFunction(toDeg, math.toDegrees) + testOneToOneMathFunction(toDegrees, math.toDegrees) } test("toRad") { - testOneToOneMathFunction(toRad, math.toRadians) + testOneToOneMathFunction(toRadians, math.toRadians) } test("cbrt") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 0ab8558c1db13..77be3b8b206c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -120,6 +120,15 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Row(1, 1) :: Nil) } + test("SPARK-6201 IN type conversion") { + jsonRDD(sparkContext.parallelize(Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}"))) + .registerTempTable("d") + + checkAnswer( + sql("select * from d where d.a in (1,2)"), + Seq(Row("1"), Row("2"))) + } + test("SPARK-3176 Added Parser of SQL ABS()") { checkAnswer( sql("SELECT ABS(-1.3)"), @@ -862,6 +871,16 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { conf.clear() } + test("SET commands with illegal or inappropriate argument") { + conf.clear() + // Set negative mapred.reduce.tasks for automatically determing + // the number of reducers is not supported + intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-1")) + intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-01")) + intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-2")) + conf.clear() + } + test("apply schema") { val schema1 = StructType( StructField("f1", IntegerType, false) :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala index 27f063d73a9a9..15337c4045436 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala @@ -148,6 +148,15 @@ abstract class SparkSqlSerializer2Suite extends QueryTest with BeforeAndAfterAll table("shuffle").collect()) } + test("key schema is null") { + val aggregations = allColumns.split(",").map(c => s"COUNT($c)").mkString(",") + val df = sql(s"SELECT $aggregations FROM shuffle") + checkSerializer(df.queryExecution.executedPlan, serializerClass) + checkAnswer( + df, + Row(1000, 1000, 0, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000)) + } + test("value schema is null") { val df = sql(s"SELECT col0 FROM shuffle ORDER BY col0") checkSerializer(df.queryExecution.executedPlan, serializerClass) @@ -167,29 +176,20 @@ class SparkSqlSerializer2SortShuffleSuite extends SparkSqlSerializer2Suite { override def beforeAll(): Unit = { super.beforeAll() // Sort merge will not be triggered. - sql("set spark.sql.shuffle.partitions = 200") - } - - test("key schema is null") { - val aggregations = allColumns.split(",").map(c => s"COUNT($c)").mkString(",") - val df = sql(s"SELECT $aggregations FROM shuffle") - checkSerializer(df.queryExecution.executedPlan, serializerClass) - checkAnswer( - df, - Row(1000, 1000, 0, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000)) + val bypassMergeThreshold = + sparkContext.conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) + sql(s"set spark.sql.shuffle.partitions=${bypassMergeThreshold-1}") } } /** For now, we will use SparkSqlSerializer for sort based shuffle with sort merge. */ class SparkSqlSerializer2SortMergeShuffleSuite extends SparkSqlSerializer2Suite { - // We are expecting SparkSqlSerializer. - override val serializerClass: Class[Serializer] = - classOf[SparkSqlSerializer].asInstanceOf[Class[Serializer]] - override def beforeAll(): Unit = { super.beforeAll() // To trigger the sort merge. - sql("set spark.sql.shuffle.partitions = 201") + val bypassMergeThreshold = + sparkContext.conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) + sql(s"set spark.sql.shuffle.partitions=${bypassMergeThreshold + 1}") } } 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 b165ab2b1deb3..021affafe36a6 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 @@ -104,6 +104,8 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { ).executeUpdate() conn.prepareStatement("insert into test.timetypes values ('12:34:56', " + "'1996-01-01', '2002-02-20 11:22:33.543543543')").executeUpdate() + conn.prepareStatement("insert into test.timetypes values ('12:34:56', " + + "null, '2002-02-20 11:22:33.543543543')").executeUpdate() conn.commit() sql( s""" @@ -127,6 +129,23 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { |OPTIONS (url '$url', dbtable 'TEST.FLTTYPES', user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) + conn.prepareStatement( + s""" + |create table test.nulltypes (a INT, b BOOLEAN, c TINYINT, d BINARY(20), e VARCHAR(20), + |f VARCHAR_IGNORECASE(20), g CHAR(20), h BLOB, i CLOB, j TIME, k DATE, l TIMESTAMP, + |m DOUBLE, n REAL, o DECIMAL(40, 20)) + """.stripMargin.replaceAll("\n", " ")).executeUpdate() + conn.prepareStatement("insert into test.nulltypes values (" + + "null, null, null, null, null, null, null, null, null, " + + "null, null, null, null, null, null)").executeUpdate() + conn.commit() + sql( + s""" + |CREATE TEMPORARY TABLE nulltypes + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.NULLTYPES', user 'testUser', password 'testPass') + """.stripMargin.replaceAll("\n", " ")) + // Untested: IDENTITY, OTHER, UUID, ARRAY, and GEOMETRY types. } @@ -254,6 +273,7 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { val rows = TestSQLContext.jdbc(urlWithUserAndPass, "TEST.TIMETYPES").collect() val cachedRows = TestSQLContext.jdbc(urlWithUserAndPass, "TEST.TIMETYPES").cache().collect() assert(rows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) + assert(rows(1).getAs[java.sql.Date](1) === null) assert(cachedRows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) } @@ -266,13 +286,21 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { assert(cachedRows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) } + test("test types for null value") { + val rows = TestSQLContext.jdbc(urlWithUserAndPass, "TEST.NULLTYPES").collect() + assert((0 to 14).forall(i => rows(0).isNullAt(i))) + } + test("H2 floating-point types") { val rows = sql("SELECT * FROM flttypes").collect() assert(rows(0).getDouble(0) === 1.00000000000000022) // Yes, I meant ==. assert(rows(0).getDouble(1) === 1.00000011920928955) // Yes, I meant ==. assert(rows(0).getAs[BigDecimal](2) - .equals(new BigDecimal("123456789012345.54321543215432100000"))) + .equals(new BigDecimal("123456789012345.54321543215432100000"))) assert(rows(0).schema.fields(2).dataType === DecimalType(40, 20)) + val compareDecimal = sql("SELECT C FROM flttypes where C > C - 1").collect() + assert(compareDecimal(0).getAs[BigDecimal](0) + .equals(new BigDecimal("123456789012345.54321543215432100000"))) } test("SQL query as table name") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index fd0e2746dc045..263fafba930ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -17,13 +17,15 @@ package org.apache.spark.sql.json +import java.io.StringWriter import java.sql.{Date, Timestamp} +import com.fasterxml.jackson.core.JsonFactory import org.scalactic.Tolerance._ import org.apache.spark.sql.TestData._ import org.apache.spark.sql.functions._ -import org.apache.spark.sql.json.JsonRDD.{compatibleType, enforceCorrectType} +import org.apache.spark.sql.json.InferSchema.compatibleType import org.apache.spark.sql.sources.LogicalRelation import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ @@ -46,6 +48,18 @@ class JsonSuite extends QueryTest { s"${expected}(${expected.getClass}).") } + val factory = new JsonFactory() + def enforceCorrectType(value: Any, dataType: DataType): Any = { + val writer = new StringWriter() + val generator = factory.createGenerator(writer) + generator.writeObject(value) + generator.flush() + + val parser = factory.createParser(writer.toString) + parser.nextToken() + JacksonParser.convertField(factory, parser, dataType) + } + val intNumber: Int = 2147483647 checkTypePromotion(intNumber, enforceCorrectType(intNumber, IntegerType)) checkTypePromotion(intNumber.toLong, enforceCorrectType(intNumber, LongType)) @@ -439,7 +453,7 @@ class JsonSuite extends QueryTest { val jsonDF = jsonRDD(primitiveFieldValueTypeConflict) jsonDF.registerTempTable("jsonTable") - // Right now, the analyzer does not promote strings in a boolean expreesion. + // Right now, the analyzer does not promote strings in a boolean expression. // Number and Boolean conflict: resolve the type as boolean in this query. checkAnswer( sql("select num_bool from jsonTable where NOT num_bool"), @@ -508,7 +522,7 @@ class JsonSuite extends QueryTest { Row(Seq(), "11", "[1,2,3]", Row(null), "[]") :: Row(null, """{"field":false}""", null, null, "{}") :: Row(Seq(4, 5, 6), null, "str", Row(null), "[7,8,9]") :: - Row(Seq(7), "{}","[str1,str2,33]", Row("str"), """{"field":true}""") :: Nil + Row(Seq(7), "{}","""["str1","str2",33]""", Row("str"), """{"field":true}""") :: Nil ) } @@ -566,19 +580,19 @@ class JsonSuite extends QueryTest { val analyzed = jsonDF.queryExecution.analyzed assert( analyzed.isInstanceOf[LogicalRelation], - "The DataFrame returned by jsonFile should be based on JSONRelation.") + "The DataFrame returned by jsonFile should be based on LogicalRelation.") val relation = analyzed.asInstanceOf[LogicalRelation].relation assert( relation.isInstanceOf[JSONRelation], "The DataFrame returned by jsonFile should be based on JSONRelation.") - assert(relation.asInstanceOf[JSONRelation].path === path) + assert(relation.asInstanceOf[JSONRelation].path === Some(path)) assert(relation.asInstanceOf[JSONRelation].samplingRatio === (0.49 +- 0.001)) val schema = StructType(StructField("a", LongType, true) :: Nil) val logicalRelation = jsonFile(path, schema).queryExecution.analyzed.asInstanceOf[LogicalRelation] val relationWithSchema = logicalRelation.relation.asInstanceOf[JSONRelation] - assert(relationWithSchema.path === path) + assert(relationWithSchema.path === Some(path)) assert(relationWithSchema.schema === schema) assert(relationWithSchema.samplingRatio > 0.99) } @@ -1020,15 +1034,24 @@ class JsonSuite extends QueryTest { } test("JSONRelation equality test") { - val relation1 = - JSONRelation("path", 1.0, Some(StructType(StructField("a", IntegerType, true) :: Nil)))(null) + val context = org.apache.spark.sql.test.TestSQLContext + val relation1 = new JSONRelation( + "path", + 1.0, + Some(StructType(StructField("a", IntegerType, true) :: Nil)), + context) val logicalRelation1 = LogicalRelation(relation1) - val relation2 = - JSONRelation("path", 0.5, Some(StructType(StructField("a", IntegerType, true) :: Nil)))( - org.apache.spark.sql.test.TestSQLContext) + val relation2 = new JSONRelation( + "path", + 0.5, + Some(StructType(StructField("a", IntegerType, true) :: Nil)), + context) val logicalRelation2 = LogicalRelation(relation2) - val relation3 = - JSONRelation("path", 1.0, Some(StructType(StructField("b", StringType, true) :: Nil)))(null) + val relation3 = new JSONRelation( + "path", + 1.0, + Some(StructType(StructField("b", StringType, true) :: Nil)), + context) val logicalRelation3 = LogicalRelation(relation3) assert(relation1 === relation2) @@ -1046,7 +1069,7 @@ class JsonSuite extends QueryTest { test("SPARK-6245 JsonRDD.inferSchema on empty RDD") { // This is really a test that it doesn't throw an exception - val emptySchema = JsonRDD.inferSchema(empty, 1.0, "") + val emptySchema = InferSchema(empty, 1.0, "") assert(StructType(Seq()) === emptySchema) } 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 80efe9728fbc2..50629ea4dc066 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 @@ -21,7 +21,7 @@ import java.io.File import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.{AnalysisException, Row} +import org.apache.spark.sql.{SaveMode, AnalysisException, Row} import org.apache.spark.util.Utils class InsertSuite extends DataSourceTest with BeforeAndAfterAll { @@ -100,23 +100,48 @@ class InsertSuite extends DataSourceTest with BeforeAndAfterAll { test("INSERT OVERWRITE a JSONRelation multiple times") { sql( s""" - |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt - """.stripMargin) + |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt + """.stripMargin) + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + (1 to 10).map(i => Row(i, s"str$i")) + ) + // Writing the table to less part files. + val rdd1 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}"""), 5) + jsonRDD(rdd1).registerTempTable("jt1") sql( s""" - |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt - """.stripMargin) + |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt1 + """.stripMargin) + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + (1 to 10).map(i => Row(i, s"str$i")) + ) + // Writing the table to more part files. + val rdd2 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}"""), 10) + jsonRDD(rdd2).registerTempTable("jt2") sql( s""" - |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt - """.stripMargin) - + |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt2 + """.stripMargin) checkAnswer( sql("SELECT a, b FROM jsonTable"), (1 to 10).map(i => Row(i, s"str$i")) ) + + sql( + s""" + |INSERT OVERWRITE TABLE jsonTable SELECT a * 10, b FROM jt1 + """.stripMargin) + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + (1 to 10).map(i => Row(i * 10, s"str$i")) + ) + + dropTempTable("jt1") + dropTempTable("jt2") } test("INSERT INTO not supported for JSONRelation for now") { @@ -128,6 +153,20 @@ class InsertSuite extends DataSourceTest with BeforeAndAfterAll { } } + test("save directly to the path of a JSON table") { + table("jt").selectExpr("a * 5 as a", "b").save(path.toString, "json", SaveMode.Overwrite) + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + (1 to 10).map(i => Row(i * 5, s"str$i")) + ) + + table("jt").save(path.toString, "json", SaveMode.Overwrite) + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + (1 to 10).map(i => Row(i, s"str$i")) + ) + } + test("it is not allowed to write to a table while querying it.") { val message = intercept[AnalysisException] { sql( 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 b7b6925aa87f7..deb1008c468bf 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 @@ -37,7 +37,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.thrift.transport.TSocket import org.apache.spark.Logging -import org.apache.spark.sql.hive.HiveShim +import org.apache.spark.sql.hive.{HiveContext, HiveShim} import org.apache.spark.util.Utils private[hive] object SparkSQLCLIDriver { @@ -74,7 +74,12 @@ private[hive] object SparkSQLCLIDriver { System.exit(1) } - val sessionState = new CliSessionState(new HiveConf(classOf[SessionState])) + val cliConf = new HiveConf(classOf[SessionState]) + // Override the location of the metastore since this is only used for local execution. + HiveContext.newTemporaryConfiguration().foreach { + case (key, value) => cliConf.set(key, value) + } + val sessionState = new CliSessionState(cliConf) sessionState.in = System.in try { @@ -91,10 +96,14 @@ private[hive] object SparkSQLCLIDriver { // Set all properties specified via command line. val conf: HiveConf = sessionState.getConf - sessionState.cmdProperties.entrySet().foreach { item: java.util.Map.Entry[Object, Object] => - conf.set(item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) - sessionState.getOverriddenConfigurations.put( - item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) + sessionState.cmdProperties.entrySet().foreach { item => + val key = item.getKey.asInstanceOf[String] + val value = item.getValue.asInstanceOf[String] + // We do not propagate metastore options to the execution copy of hive. + if (key != "javax.jdo.option.ConnectionURL") { + conf.set(key, value) + sessionState.getOverriddenConfigurations.put(key, value) + } } SessionState.start(sessionState) @@ -138,8 +147,9 @@ private[hive] object SparkSQLCLIDriver { case e: UnsupportedEncodingException => System.exit(3) } - // use the specified database if specified - cli.processSelectDatabase(sessionState); + if (sessionState.database != null) { + SparkSQLEnv.hiveContext.runSqlHive(s"USE ${sessionState.database}") + } // Execute -i init files (always in silent mode) cli.processInitFiles(sessionState) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 97b46a01ba5b4..7c0c505e2d61e 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.thriftserver +import java.io.PrintStream + import scala.collection.JavaConversions._ import org.apache.spark.scheduler.StatsReportListener @@ -39,7 +41,6 @@ private[hive] object SparkSQLEnv extends Logging { sparkConf .setAppName(s"SparkSQL::${Utils.localHostName()}") - .set("spark.sql.hive.version", HiveShim.version) .set( "spark.serializer", maybeSerializer.getOrElse("org.apache.spark.serializer.KryoSerializer")) @@ -51,6 +52,12 @@ private[hive] object SparkSQLEnv extends Logging { sparkContext.addSparkListener(new StatsReportListener()) hiveContext = new HiveContext(sparkContext) + hiveContext.metadataHive.setOut(new PrintStream(System.out, true, "UTF-8")) + hiveContext.metadataHive.setInfo(new PrintStream(System.err, true, "UTF-8")) + hiveContext.metadataHive.setError(new PrintStream(System.err, true, "UTF-8")) + + hiveContext.setConf("spark.sql.hive.version", HiveShim.version) + if (log.isDebugEnabled) { hiveContext.hiveconf.getAllProperties.toSeq.sorted.foreach { case (k, v) => logDebug(s"HiveConf var: $k=$v") 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 81ee48ef4152f..b6245a57074c8 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 @@ -185,7 +185,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Hive does not support buckets. ".*bucket.*", - // No window support yet + // We have our own tests based on these query files. ".*window.*", // Fails in hive with authorization errors. @@ -240,7 +240,17 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // It has a bug and it has been fixed by // https://issues.apache.org/jira/browse/HIVE-7673 (in Hive 0.14 and trunk). - "input46" + "input46", + + // These tests were broken by the hive client isolation PR. + "part_inherit_tbl_props", + "part_inherit_tbl_props_with_star", + + "nullformatCTAS", // SPARK-7411: need to finish CTAS parser + + // The isolated classloader seemed to make some of our test reset mechanisms less robust. + "combine1", // This test changes compression settings in a way that breaks all subsequent tests. + "load_dyn_part14.*" // These work alone but fail when run with other tests... ) ++ HiveShim.compatibilityBlackList /** diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala new file mode 100644 index 0000000000000..934452fe579a1 --- /dev/null +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala @@ -0,0 +1,845 @@ +/* + * 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.execution + +import java.io.File +import java.util.{Locale, TimeZone} + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.util.Utils + +/** + * The test suite for window functions. To actually compare results with Hive, + * every test should be created by `createQueryTest`. Because we are reusing tables + * for different tests and there are a few properties needed to let Hive generate golden + * files, every `createQueryTest` calls should explicitly set `reset` to `false`. + */ +abstract class HiveWindowFunctionQueryBaseSuite extends HiveComparisonTest with BeforeAndAfter { + private val originalTimeZone = TimeZone.getDefault + private val originalLocale = Locale.getDefault + private val testTempDir = Utils.createTempDir() + + override def beforeAll() { + TestHive.cacheTables = true + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // Add Locale setting + Locale.setDefault(Locale.US) + + // Create the table used in windowing.q + sql("DROP TABLE IF EXISTS part") + sql( + """ + |CREATE TABLE part( + | p_partkey INT, + | p_name STRING, + | p_mfgr STRING, + | p_brand STRING, + | p_type STRING, + | p_size INT, + | p_container STRING, + | p_retailprice DOUBLE, + | p_comment STRING) + """.stripMargin) + val testData1 = TestHive.getHiveFile("data/files/part_tiny.txt").getCanonicalPath + sql( + s""" + |LOAD DATA LOCAL INPATH '$testData1' overwrite into table part + """.stripMargin) + + sql("DROP TABLE IF EXISTS over1k") + sql( + """ + |create table over1k( + | t tinyint, + | si smallint, + | i int, + | b bigint, + | f float, + | d double, + | bo boolean, + | s string, + | ts timestamp, + | dec decimal(4,2), + | bin binary) + |row format delimited + |fields terminated by '|' + """.stripMargin) + val testData2 = TestHive.getHiveFile("data/files/over1k").getCanonicalPath + sql( + s""" + |LOAD DATA LOCAL INPATH '$testData2' overwrite into table over1k + """.stripMargin) + + // The following settings are used for generating golden files with Hive. + // We have to use kryo to correctly let Hive serialize plans with window functions. + // This is used to generate golden files. + sql("set hive.plan.serialization.format=kryo") + // Explicitly set fs to local fs. + sql(s"set fs.default.name=file://$testTempDir/") + // Ask Hive to run jobs in-process as a single map and reduce task. + sql("set mapred.job.tracker=local") + } + + override def afterAll() { + TestHive.cacheTables = false + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + TestHive.reset() + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_multipartitioning.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_multipartitioning.q (deterministic) 1", + s""" + |select s, + |rank() over (partition by s order by si) r, + |sum(b) over (partition by s order by si) sum + |from over1k + |order by s, r, sum; + """.stripMargin, reset = false) + + /* timestamp comparison issue with Hive? + createQueryTest("windowing_multipartitioning.q (deterministic) 2", + s""" + |select s, + |rank() over (partition by s order by dec desc) r, + |sum(b) over (partition by s order by ts desc) as sum + |from over1k + |where s = 'tom allen' or s = 'bob steinbeck' + |order by s, r, sum; + """.stripMargin, reset = false) + */ + + createQueryTest("windowing_multipartitioning.q (deterministic) 3", + s""" + |select s, sum(i) over (partition by s), sum(f) over (partition by si) + |from over1k where s = 'tom allen' or s = 'bob steinbeck'; + """.stripMargin, reset = false) + + createQueryTest("windowing_multipartitioning.q (deterministic) 4", + s""" + |select s, rank() over (partition by s order by bo), + |rank() over (partition by si order by bin desc) from over1k + |where s = 'tom allen' or s = 'bob steinbeck'; + """.stripMargin, reset = false) + + createQueryTest("windowing_multipartitioning.q (deterministic) 5", + s""" + |select s, sum(f) over (partition by i), row_number() over (order by f) + |from over1k where s = 'tom allen' or s = 'bob steinbeck'; + """.stripMargin, reset = false) + + createQueryTest("windowing_multipartitioning.q (deterministic) 6", + s""" + |select s, rank() over w1, + |rank() over w2 + |from over1k + |where s = 'tom allen' or s = 'bob steinbeck' + |window + |w1 as (partition by s order by dec), + |w2 as (partition by si order by f) ; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_navfn.q + // Results of the original query file are not deterministic. + // Also, the original query of + // select i, lead(s) over (partition by bin order by d,i desc) from over1k ; + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_navfn.q (deterministic)", + s""" + |select s, row_number() over (partition by d order by dec) rn from over1k + |order by s, rn desc; + |select i, lead(s) over (partition by cast(bin as string) order by d,i desc) as l + |from over1k + |order by i desc, l; + |select i, lag(dec) over (partition by i order by s,i,dec) l from over1k + |order by i, l; + |select s, last_value(t) over (partition by d order by f) l from over1k + |order by s, l; + |select s, first_value(s) over (partition by bo order by s) f from over1k + |order by s, f; + |select t, s, i, last_value(i) over (partition by t order by s) + |from over1k where (s = 'oscar allen' or s = 'oscar carson') and t = 10; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_ntile.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_ntile.q (deterministic)", + s""" + |select i, ntile(10) over (partition by s order by i) n from over1k + |order by i, n; + |select s, ntile(100) over (partition by i order by s) n from over1k + |order by s, n; + |select f, ntile(4) over (partition by d order by f) n from over1k + |order by f, n; + |select d, ntile(1000) over (partition by dec order by d) n from over1k + |order by d, n; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_udaf.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_udaf.q (deterministic)", + s""" + |select s, min(i) over (partition by s) m from over1k + |order by s, m; + |select s, avg(f) over (partition by si order by s) a from over1k + |order by s, a; + |select s, avg(i) over (partition by t, b order by s) a from over1k + |order by s, a; + |select max(i) over w m from over1k + |order by m window w as (partition by f) ; + |select s, avg(d) over (partition by t order by f) a from over1k + |order by s, a; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_windowspec.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_windowspec.q (deterministic)", + s""" + |select s, sum(b) over (partition by i order by s,b rows unbounded preceding) as sum + |from over1k order by s, sum; + |select s, sum(f) over (partition by d order by s,f rows unbounded preceding) as sum + |from over1k order by s, sum; + |select s, sum(f) over + |(partition by ts order by f range between current row and unbounded following) as sum + |from over1k order by s, sum; + |select s, avg(f) + |over (partition by ts order by s,f rows between current row and 5 following) avg + |from over1k order by s, avg; + |select s, avg(d) over + |(partition by t order by s,d desc rows between 5 preceding and 5 following) avg + |from over1k order by s, avg; + |select s, sum(i) over(partition by ts order by s) sum from over1k + |order by s, sum; + |select f, sum(f) over + |(partition by ts order by f range between unbounded preceding and current row) sum + |from over1k order by f, sum; + |select s, i, round(avg(d) over (partition by s order by i) / 10.0 , 2) avg + |from over1k order by s, i, avg; + |select s, i, round((avg(d) over w1 + 10.0) - (avg(d) over w1 - 10.0),2) avg + |from over1k + |order by s, i, avg window w1 as (partition by s order by i); + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_rank.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_rank.q (deterministic) 1", + s""" + |select s, rank() over (partition by f order by t) r from over1k order by s, r; + |select s, dense_rank() over (partition by ts order by i,s desc) as r from over1k + |order by s desc, r desc; + |select s, cume_dist() over (partition by bo order by b,s) cd from over1k + |order by s, cd; + |select s, percent_rank() over (partition by dec order by f) r from over1k + |order by s desc, r desc; + """.stripMargin, reset = false) + + createQueryTest("windowing_rank.q (deterministic) 2", + s""" + |select ts, dec, rnk + |from + | (select ts, dec, + | rank() over (partition by ts order by dec) as rnk + | from + | (select other.ts, other.dec + | from over1k other + | join over1k on (other.b = over1k.b) + | ) joined + | ) ranked + |where rnk = 1 + |order by ts, dec, rnk; + """.stripMargin, reset = false) + + createQueryTest("windowing_rank.q (deterministic) 3", + s""" + |select ts, dec, rnk + |from + | (select ts, dec, + | rank() over (partition by ts order by dec) as rnk + | from + | (select other.ts, other.dec + | from over1k other + | join over1k on (other.b = over1k.b) + | ) joined + | ) ranked + |where dec = 89.5 + |order by ts, dec, rnk; + """.stripMargin, reset = false) + + createQueryTest("windowing_rank.q (deterministic) 4", + s""" + |select ts, dec, rnk + |from + | (select ts, dec, + | rank() over (partition by ts order by dec) as rnk + | from + | (select other.ts, other.dec + | from over1k other + | join over1k on (other.b = over1k.b) + | where other.t < 10 + | ) joined + | ) ranked + |where rnk = 1 + |order by ts, dec, rnk; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests from windowing.q + // We port tests in windowing.q to here because this query file contains too + // many tests and the syntax of test "-- 7. testJoinWithWindowingAndPTF" + // is not supported right now. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing.q -- 1. testWindowing", + s""" + |select p_mfgr, p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |sum(p_retailprice) over + |(distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 2. testGroupByWithPartitioning", + s""" + |select p_mfgr, p_name, p_size, + |min(p_retailprice), + |rank() over(distribute by p_mfgr sort by p_name)as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + |group by p_mfgr, p_name, p_size + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 3. testGroupByHavingWithSWQ", + s""" + |select p_mfgr, p_name, p_size, min(p_retailprice), + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + |group by p_mfgr, p_name, p_size + |having p_size > 0 + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 4. testCount", + s""" + |select p_mfgr, p_name, + |count(p_size) over(distribute by p_mfgr sort by p_name) as cd + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 5. testCountWithWindowingUDAF", + s""" + |select p_mfgr, p_name, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |count(p_size) over(distribute by p_mfgr sort by p_name) as cd, + |p_retailprice, sum(p_retailprice) over (distribute by p_mfgr sort by p_name + | rows between unbounded preceding and current row) as s1, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 6. testCountInSubQ", + s""" + |select sub1.r, sub1.dr, sub1.cd, sub1.s1, sub1.deltaSz + |from (select p_mfgr, p_name, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |count(p_size) over(distribute by p_mfgr sort by p_name) as cd, + |p_retailprice, sum(p_retailprice) over (distribute by p_mfgr sort by p_name + | rows between unbounded preceding and current row) as s1, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + |) sub1 + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 8. testMixedCaseAlias", + s""" + |select p_mfgr, p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name, p_size desc) as R + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 9. testHavingWithWindowingNoGBY", + s""" + |select p_mfgr, p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_name + | rows between unbounded preceding and current row) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 10. testHavingWithWindowingCondRankNoGBY", + s""" + |select p_mfgr, p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_name + | rows between unbounded preceding and current row) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 11. testFirstLast", + s""" + |select p_mfgr,p_name, p_size, + |sum(p_size) over (distribute by p_mfgr sort by p_name + |rows between current row and current row) as s2, + |first_value(p_size) over w1 as f, + |last_value(p_size, false) over w1 as l + |from part + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 12. testFirstLastWithWhere", + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |sum(p_size) over (distribute by p_mfgr sort by p_name + |rows between current row and current row) as s2, + |first_value(p_size) over w1 as f, + |last_value(p_size, false) over w1 as l + |from part + |where p_mfgr = 'Manufacturer#3' + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 13. testSumWindow", + s""" + |select p_mfgr,p_name, p_size, + |sum(p_size) over w1 as s1, + |sum(p_size) over (distribute by p_mfgr sort by p_name + |rows between current row and current row) as s2 + |from part + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 14. testNoSortClause", + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr + |from part + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 15. testExpressions", + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |cume_dist() over(distribute by p_mfgr sort by p_name) as cud, + |percent_rank() over(distribute by p_mfgr sort by p_name) as pr, + |ntile(3) over(distribute by p_mfgr sort by p_name) as nt, + |count(p_size) over(distribute by p_mfgr sort by p_name) as ca, + |avg(p_size) over(distribute by p_mfgr sort by p_name) as avg, + |stddev(p_size) over(distribute by p_mfgr sort by p_name) as st, + |first_value(p_size % 5) over(distribute by p_mfgr sort by p_name) as fv, + |last_value(p_size) over(distribute by p_mfgr sort by p_name) as lv, + |first_value(p_size) over w1 as fvW1 + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 16. testMultipleWindows", + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |cume_dist() over(distribute by p_mfgr sort by p_name) as cud, + |sum(p_size) over (distribute by p_mfgr sort by p_name + |range between unbounded preceding and current row) as s1, + |sum(p_size) over (distribute by p_mfgr sort by p_size + |range between 5 preceding and current row) as s2, + |first_value(p_size) over w1 as fv1 + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + + createQueryTest("windowing.q -- 17. testCountStar", + s""" + |select p_mfgr,p_name, p_size, + |count(*) over(distribute by p_mfgr sort by p_name ) as c, + |count(p_size) over(distribute by p_mfgr sort by p_name) as ca, + |first_value(p_size) over w1 as fvW1 + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 18. testUDAFs", + s""" + |select p_mfgr,p_name, p_size, + |sum(p_retailprice) over w1 as s, + |min(p_retailprice) over w1 as mi, + |max(p_retailprice) over w1 as ma, + |avg(p_retailprice) over w1 as ag + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 19. testUDAFsWithGBY", + """ + |select p_mfgr,p_name, p_size, p_retailprice, + |sum(p_retailprice) over w1 as s, + |min(p_retailprice) as mi , + |max(p_retailprice) as ma , + |avg(p_retailprice) over w1 as ag + |from part + |group by p_mfgr,p_name, p_size, p_retailprice + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following); + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 20. testSTATs", + """ + |select p_mfgr,p_name, p_size, + |stddev(p_retailprice) over w1 as sdev, + |stddev_pop(p_retailprice) over w1 as sdev_pop, + |collect_set(p_size) over w1 as uniq_size, + |variance(p_retailprice) over w1 as var, + |corr(p_size, p_retailprice) over w1 as cor, + |covar_pop(p_size, p_retailprice) over w1 as covarp + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 21. testDISTs", + """ + |select p_mfgr,p_name, p_size, + |histogram_numeric(p_retailprice, 5) over w1 as hist, + |percentile(p_partkey, 0.5) over w1 as per, + |row_number() over(distribute by p_mfgr sort by p_name) as rn + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 24. testLateralViews", + """ + |select p_mfgr, p_name, + |lv_col, p_size, sum(p_size) over w1 as s + |from (select p_mfgr, p_name, p_size, array(1,2,3) arr from part) p + |lateral view explode(arr) part_lv as lv_col + |window w1 as (distribute by p_mfgr sort by p_size, lv_col + | rows between 2 preceding and current row) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 26. testGroupByHavingWithSWQAndAlias", + """ + |select p_mfgr, p_name, p_size, min(p_retailprice) as mi, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + |group by p_mfgr, p_name, p_size + |having p_size > 0 + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 27. testMultipleRangeWindows", + """ + |select p_mfgr,p_name, p_size, + |sum(p_size) over (distribute by p_mfgr sort by p_size + |range between 10 preceding and current row) as s2, + |sum(p_size) over (distribute by p_mfgr sort by p_size + |range between current row and 10 following ) as s1 + |from part + |window w1 as (rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 28. testPartOrderInUDAFInvoke", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over (partition by p_mfgr order by p_name + |rows between 2 preceding and 2 following) as s + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 29. testPartOrderInWdwDef", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s + |from part + |window w1 as (partition by p_mfgr order by p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 30. testDefaultPartitioningSpecRules", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s, + |sum(p_size) over w2 as s2 + |from part + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following), + | w2 as (partition by p_mfgr order by p_name) + """.stripMargin, reset = false) + + /* p_name is not a numeric column. What is Hive's semantic? + createQueryTest("windowing.q -- 31. testWindowCrossReference", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s1, + |sum(p_size) over w2 as s2 + |from part + |window w1 as (partition by p_mfgr order by p_name + | range between 2 preceding and 2 following), + | w2 as w1 + """.stripMargin, reset = false) + */ + /* + createQueryTest("windowing.q -- 32. testWindowInheritance", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s1, + |sum(p_size) over w2 as s2 + |from part + |window w1 as (partition by p_mfgr order by p_name + | range between 2 preceding and 2 following), + | w2 as (w1 rows between unbounded preceding and current row) + """.stripMargin, reset = false) + */ + + /* p_name is not a numeric column. What is Hive's semantic? + createQueryTest("windowing.q -- 33. testWindowForwardReference", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s1, + |sum(p_size) over w2 as s2, + |sum(p_size) over w3 as s3 + |from part + |window w1 as (distribute by p_mfgr sort by p_name + | range between 2 preceding and 2 following), + | w2 as w3, + | w3 as (distribute by p_mfgr sort by p_name + | range between unbounded preceding and current row) + """.stripMargin, reset = false) + */ + /* + createQueryTest("windowing.q -- 34. testWindowDefinitionPropagation", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s1, + |sum(p_size) over w2 as s2, + |sum(p_size) over (w3 rows between 2 preceding and 2 following) as s3 + |from part + |window w1 as (distribute by p_mfgr sort by p_name + | range between 2 preceding and 2 following), + | w2 as w3, + | w3 as (distribute by p_mfgr sort by p_name + | range between unbounded preceding and current row) + """.stripMargin, reset = false) + */ + + /* Seems Hive evaluate SELECT DISTINCT before window functions? + createQueryTest("windowing.q -- 35. testDistinctWithWindowing", + """ + |select DISTINCT p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s + |from part + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + */ + + createQueryTest("windowing.q -- 36. testRankWithPartitioning", + """ + |select p_mfgr, p_name, p_size, + |rank() over (partition by p_mfgr order by p_name ) as r + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 37. testPartitioningVariousForms", + """ + |select p_mfgr, + |round(sum(p_retailprice) over (partition by p_mfgr order by p_mfgr),2) as s1, + |min(p_retailprice) over (partition by p_mfgr) as s2, + |max(p_retailprice) over (distribute by p_mfgr sort by p_mfgr) as s3, + |round(avg(p_retailprice) over (distribute by p_mfgr),2) as s4, + |count(p_retailprice) over (cluster by p_mfgr ) as s5 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 38. testPartitioningVariousForms2", + """ + |select p_mfgr, p_name, p_size, + |sum(p_retailprice) over (partition by p_mfgr, p_name order by p_mfgr, p_name + |rows between unbounded preceding and current row) as s1, + |min(p_retailprice) over (distribute by p_mfgr, p_name sort by p_mfgr, p_name + |rows between unbounded preceding and current row) as s2, + |max(p_retailprice) over (partition by p_mfgr, p_name order by p_name) as s3 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 39. testUDFOnOrderCols", + """ + |select p_mfgr, p_type, substr(p_type, 2) as short_ptype, + |rank() over (partition by p_mfgr order by substr(p_type, 2)) as r + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 40. testNoBetweenForRows", + """ + |select p_mfgr, p_name, p_size, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows unbounded preceding) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 41. testNoBetweenForRange", + """ + |select p_mfgr, p_name, p_size, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_size range unbounded preceding) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 42. testUnboundedFollowingForRows", + """ + |select p_mfgr, p_name, p_size, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_name + |rows between current row and unbounded following) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 43. testUnboundedFollowingForRange", + """ + |select p_mfgr, p_name, p_size, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_size + |range between current row and unbounded following) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 44. testOverNoPartitionSingleAggregate", + """ + |select p_name, p_retailprice, + |round(avg(p_retailprice) over(),2) + |from part + |order by p_name + """.stripMargin, reset = false) +} + +class HiveWindowFunctionQueryWithoutCodeGenSuite extends HiveWindowFunctionQueryBaseSuite { + var originalCodegenEnabled: Boolean = _ + override def beforeAll(): Unit = { + super.beforeAll() + originalCodegenEnabled = conf.codegenEnabled + sql("set spark.sql.codegen=false") + } + + override def afterAll(): Unit = { + sql(s"set spark.sql.codegen=$originalCodegenEnabled") + super.afterAll() + } +} + +abstract class HiveWindowFunctionQueryFileBaseSuite + extends HiveCompatibilitySuite with BeforeAndAfter { + private val originalTimeZone = TimeZone.getDefault + private val originalLocale = Locale.getDefault + private val testTempDir = Utils.createTempDir() + + override def beforeAll() { + TestHive.cacheTables = true + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // Add Locale setting + Locale.setDefault(Locale.US) + + // The following settings are used for generating golden files with Hive. + // We have to use kryo to correctly let Hive serialize plans with window functions. + // This is used to generate golden files. + sql("set hive.plan.serialization.format=kryo") + // Explicitly set fs to local fs. + sql(s"set fs.default.name=file://$testTempDir/") + // Ask Hive to run jobs in-process as a single map and reduce task. + sql("set mapred.job.tracker=local") + } + + override def afterAll() { + TestHive.cacheTables = false + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + TestHive.reset() + } + + override def blackList: Seq[String] = Seq( + // Partitioned table functions are not supported. + "ptf*", + // tests of windowing.q are in HiveWindowFunctionQueryBaseSuite + "windowing.q", + + // This one failed on the expression of + // sum(lag(p_retailprice,1,0.0)) over w1 + // lag(p_retailprice,1,0.0) is a GenericUDF and the argument inspector of + // p_retailprice created by HiveInspectors is + // PrimitiveObjectInspectorFactory.javaDoubleObjectInspector. + // However, seems Hive assumes it is + // PrimitiveObjectInspectorFactory.writableDoubleObjectInspector, which introduces an error. + "windowing_expressions", + + // Hive's results are not deterministic + "windowing_multipartitioning", + "windowing_navfn", + "windowing_ntile", + "windowing_udaf", + "windowing_windowspec", + "windowing_rank" + ) + + override def whiteList: Seq[String] = Seq( + "windowing_udaf2", + "windowing_columnPruning", + "windowing_adjust_rowcontainer_sz" + ) + + override def testCases: Seq[(String, File)] = super.testCases.filter { + case (name, _) => realWhiteList.contains(name) + } +} + +class HiveWindowFunctionQueryFileWithoutCodeGenSuite extends HiveWindowFunctionQueryFileBaseSuite { + var originalCodegenEnabled: Boolean = _ + override def beforeAll(): Unit = { + super.beforeAll() + originalCodegenEnabled = conf.codegenEnabled + sql("set spark.sql.codegen=false") + } + + override def afterAll(): Unit = { + sql(s"set spark.sql.codegen=$originalCodegenEnabled") + super.afterAll() + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 1d8d0b5c322ad..538c6c7f0a200 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -17,8 +17,9 @@ package org.apache.spark.sql.hive -import java.io.{BufferedReader, InputStreamReader, PrintStream} +import java.io.{BufferedReader, File, InputStreamReader, PrintStream} import java.sql.Timestamp +import java.util.{ArrayList => JArrayList} import org.apache.hadoop.hive.ql.parse.VariableSubstitution import org.apache.spark.sql.catalyst.Dialect @@ -35,15 +36,19 @@ import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.io.{DateWritable, TimestampWritable} -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.annotation.Experimental +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateSubQueries, OverrideCatalog, OverrideFunctionRegistry} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.{ExecutedCommand, ExtractPythonUdfs, QueryExecutionException, SetCommand} +import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, HiveNativeCommand} import org.apache.spark.sql.sources.{DDLParser, DataSourceStrategy} import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils + /** * This is the HiveQL Dialect, this dialect is strongly bind with HiveContext @@ -61,6 +66,8 @@ private[hive] class HiveQLDialect extends Dialect { class HiveContext(sc: SparkContext) extends SQLContext(sc) { self => + import HiveContext._ + /** * When true, enables an experimental feature where metastore tables that use the parquet SerDe * are automatically converted to use the Spark SQL parquet table scan, instead of the Hive @@ -93,9 +100,118 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected[sql] def convertCTAS: Boolean = getConf("spark.sql.hive.convertCTAS", "false").toBoolean + /** + * The version of the hive client that will be used to communicate with the metastore. Note that + * this does not necessarily need to be the same version of Hive that is used internally by + * Spark SQL for execution. + */ + protected[hive] def hiveMetastoreVersion: String = + getConf(HIVE_METASTORE_VERSION, hiveExecutionVersion) + + /** + * The location of the jars that should be used to instantiate the HiveMetastoreClient. This + * property can be one of three options: + * - a classpath in the standard format for both hive and hadoop. + * - builtin - attempt to discover the jars that were used to load Spark SQL and use those. This + * option is only valid when using the execution version of Hive. + * - maven - download the correct version of hive on demand from maven. + */ + protected[hive] def hiveMetastoreJars: String = + getConf(HIVE_METASTORE_JARS, "builtin") + @transient protected[sql] lazy val substitutor = new VariableSubstitution() + /** + * The copy of the hive client that is used for execution. Currently this must always be + * Hive 13 as this is the version of Hive that is packaged with Spark SQL. This copy of the + * client is used for execution related tasks like registering temporary functions or ensuring + * that the ThreadLocal SessionState is correctly populated. This copy of Hive is *not* used + * for storing peristent metadata, and only point to a dummy metastore in a temporary directory. + */ + @transient + protected[hive] lazy val executionHive: ClientWrapper = { + logInfo(s"Initilizing execution hive, version $hiveExecutionVersion") + new ClientWrapper( + version = IsolatedClientLoader.hiveVersion(hiveExecutionVersion), + config = newTemporaryConfiguration()) + } + SessionState.setCurrentSessionState(executionHive.state) + + /** + * The copy of the Hive client that is used to retrieve metadata from the Hive MetaStore. + * The version of the Hive client that is used here must match the metastore that is configured + * in the hive-site.xml file. + */ + @transient + protected[hive] lazy val metadataHive: ClientInterface = { + val metaVersion = IsolatedClientLoader.hiveVersion(hiveMetastoreVersion) + + // We instantiate a HiveConf here to read in the hive-site.xml file and then pass the options + // into the isolated client loader + val metadataConf = new HiveConf() + // `configure` goes second to override other settings. + val allConfig = metadataConf.iterator.map(e => e.getKey -> e.getValue).toMap ++ configure + + val isolatedLoader = if (hiveMetastoreJars == "builtin") { + if (hiveExecutionVersion != hiveMetastoreVersion) { + throw new IllegalArgumentException( + "Builtin jars can only be used when hive execution version == hive metastore version. " + + s"Execution: ${hiveExecutionVersion} != Metastore: ${hiveMetastoreVersion}. " + + "Specify a vaild path to the correct hive jars using $HIVE_METASTORE_JARS " + + s"or change $HIVE_METASTORE_VERSION to $hiveExecutionVersion.") + } + val jars = getClass.getClassLoader match { + case urlClassLoader: java.net.URLClassLoader => urlClassLoader.getURLs + case other => + throw new IllegalArgumentException( + "Unable to locate hive jars to connect to metastore " + + s"using classloader ${other.getClass.getName}. " + + "Please set spark.sql.hive.metastore.jars") + } + + logInfo( + s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using Spark classes.") + new IsolatedClientLoader( + version = metaVersion, + execJars = jars.toSeq, + config = allConfig, + isolationOn = true) + } else if (hiveMetastoreJars == "maven") { + // TODO: Support for loading the jars from an already downloaded location. + logInfo( + s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using maven.") + IsolatedClientLoader.forVersion(hiveMetastoreVersion, allConfig ) + } else { + // Convert to files and expand any directories. + val jars = + hiveMetastoreJars + .split(File.pathSeparator) + .flatMap { + case path if new File(path).getName() == "*" => + val files = new File(path).getParentFile().listFiles() + if (files == null) { + logWarning(s"Hive jar path '$path' does not exist.") + Nil + } else { + files.filter(_.getName().toLowerCase().endsWith(".jar")) + } + case path => + new File(path) :: Nil + } + .map(_.toURI.toURL) + + logInfo( + s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using $jars") + new IsolatedClientLoader( + version = metaVersion, + execJars = jars.toSeq, + config = allConfig, + isolationOn = true) + } + isolatedLoader.client + } + protected[sql] override def parseSql(sql: String): LogicalPlan = { super.parseSql(substitutor.substitute(hiveconf, sql)) } @@ -178,15 +294,10 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { // recorded in the Hive metastore. // This logic is based on org.apache.hadoop.hive.ql.exec.StatsTask.aggregateStats(). if (newTotalSize > 0 && newTotalSize != oldTotalSize) { - tableParameters.put(HiveShim.getStatsSetupConstTotalSize, newTotalSize.toString) - val hiveTTable = relation.hiveQlTable.getTTable - hiveTTable.setParameters(tableParameters) - val tableFullName = - relation.hiveQlTable.getDbName + "." + relation.hiveQlTable.getTableName - - catalog.synchronized { - catalog.client.alterTable(tableFullName, new Table(hiveTTable)) - } + catalog.client.alterTable( + relation.table.copy( + properties = relation.table.properties + + (HiveShim.getStatsSetupConstTotalSize -> newTotalSize.toString))) } case otherRelation => throw new UnsupportedOperationException( @@ -194,47 +305,19 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } - // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. - @transient - protected lazy val outputBuffer = new java.io.OutputStream { - var pos: Int = 0 - var buffer = new Array[Int](10240) - def write(i: Int): Unit = { - buffer(pos) = i - pos = (pos + 1) % buffer.size - } - - override def toString: String = { - val (end, start) = buffer.splitAt(pos) - val input = new java.io.InputStream { - val iterator = (start ++ end).iterator - - def read(): Int = if (iterator.hasNext) iterator.next() else -1 - } - val reader = new BufferedReader(new InputStreamReader(input)) - val stringBuilder = new StringBuilder - var line = reader.readLine() - while(line != null) { - stringBuilder.append(line) - stringBuilder.append("\n") - line = reader.readLine() - } - stringBuilder.toString() - } - } - - protected[hive] def sessionState = tlSession.get().asInstanceOf[this.SQLSession].sessionState - protected[hive] def hiveconf = tlSession.get().asInstanceOf[this.SQLSession].hiveconf override def setConf(key: String, value: String): Unit = { super.setConf(key, value) - runSqlHive(s"SET $key=$value") + hiveconf.set(key, value) + executionHive.runSqlHive(s"SET $key=$value") + metadataHive.runSqlHive(s"SET $key=$value") } /* A catalyst metadata catalog that points to the Hive Metastore. */ @transient - override protected[sql] lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog + override protected[sql] lazy val catalog = + new HiveMetastoreCatalog(metadataHive, this) with OverrideCatalog // Note that HiveUDFs will be overridden by functions registered in this context. @transient @@ -252,6 +335,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { catalog.CreateTables :: catalog.PreInsertionCasts :: ExtractPythonUdfs :: + ResolveHiveWindowFunction :: sources.PreInsertCastAndRename :: Nil } @@ -260,16 +344,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { new this.SQLSession() } + /** Overridden by child classes that need to set configuration before the client init. */ + protected def configure(): Map[String, String] = Map.empty + protected[hive] class SQLSession extends super.SQLSession { protected[sql] override lazy val conf: SQLConf = new SQLConf { override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") } - protected[hive] lazy val hiveconf: HiveConf = { - setConf(sessionState.getConf.getAllProperties) - sessionState.getConf - } - /** * SQLConf and HiveConf contracts: * @@ -284,78 +366,12 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { state = new SessionState(new HiveConf(classOf[SessionState])) SessionState.start(state) } - if (state.out == null) { - state.out = new PrintStream(outputBuffer, true, "UTF-8") - } - if (state.err == null) { - state.err = new PrintStream(outputBuffer, true, "UTF-8") - } state } - } - - /** - * Runs the specified SQL query using Hive. - */ - protected[sql] def runSqlHive(sql: String): Seq[String] = { - val maxResults = 100000 - val results = runHive(sql, maxResults) - // It is very confusing when you only get back some of the results... - if (results.size == maxResults) sys.error("RESULTS POSSIBLY TRUNCATED") - results - } - - /** - * Execute the command using Hive and return the results as a sequence. Each element - * in the sequence is one row. - */ - protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = synchronized { - try { - val cmd_trimmed: String = cmd.trim() - val tokens: Array[String] = cmd_trimmed.split("\\s+") - val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() - val proc: CommandProcessor = HiveShim.getCommandProcessor(Array(tokens(0)), hiveconf) - - // Makes sure the session represented by the `sessionState` field is activated. This implies - // Spark SQL Hive support uses a single `SessionState` for all Hive operations and breaks - // session isolation under multi-user scenarios (i.e. HiveThriftServer2). - // TODO Fix session isolation - if (SessionState.get() != sessionState) { - SessionState.start(sessionState) - } - proc match { - case driver: Driver => - val results = HiveShim.createDriverResultsArray - val response: CommandProcessorResponse = driver.run(cmd) - // Throw an exception if there is an error in query processing. - if (response.getResponseCode != 0) { - driver.close() - throw new QueryExecutionException(response.getErrorMessage) - } - driver.setMaxRows(maxRows) - driver.getResults(results) - driver.close() - HiveShim.processResults(results) - case _ => - if (sessionState.out != null) { - sessionState.out.println(tokens(0) + " " + cmd_1) - } - Seq(proc.run(cmd_1).getResponseCode.toString) - } - } catch { - case e: Exception => - logError( - s""" - |====================== - |HIVE FAILURE OUTPUT - |====================== - |${outputBuffer.toString} - |====================== - |END HIVE FAILURE OUTPUT - |====================== - """.stripMargin) - throw e + protected[hive] lazy val hiveconf: HiveConf = { + setConf(sessionState.getConf.getAllProperties) + sessionState.getConf } } @@ -390,17 +406,23 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { ) } + protected[hive] def runSqlHive(sql: String): Seq[String] = { + if (sql.toLowerCase.contains("create temporary function")) { + executionHive.runSqlHive(sql) + } else if (sql.trim.toLowerCase.startsWith("set")) { + metadataHive.runSqlHive(sql) + executionHive.runSqlHive(sql) + } else { + metadataHive.runSqlHive(sql) + } + } + @transient override protected[sql] val planner = hivePlanner /** Extends QueryExecution with hive specific features. */ protected[sql] class QueryExecution(logicalPlan: LogicalPlan) extends super.QueryExecution(logicalPlan) { - // Like what we do in runHive, makes sure the session represented by the - // `sessionState` field is activated. - if (SessionState.get() != sessionState) { - SessionState.start(sessionState) - } /** * Returns the result as a hive compatible sequence of strings. For native commands, the @@ -438,7 +460,21 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } -private object HiveContext { +private[hive] object HiveContext { + /** The version of hive used internally by Spark SQL. */ + val hiveExecutionVersion: String = "0.13.1" + + val HIVE_METASTORE_VERSION: String = "spark.sql.hive.metastore.version" + val HIVE_METASTORE_JARS: String = "spark.sql.hive.metastore.jars" + + /** Constructs a configuration for hive, where the metastore is located in a temp directory. */ + def newTemporaryConfiguration(): Map[String, String] = { + val tempDir = Utils.createTempDir() + val localMetastore = new File(tempDir, "metastore").getAbsolutePath + Map( + "javax.jdo.option.ConnectionURL" -> s"jdbc:derby:;databaseName=$localMetastore;create=true") + } + protected val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, ShortType, DateType, TimestampType, BinaryType) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 4d222cf88e5e8..8fcdf3d0ab119 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -22,6 +22,8 @@ import java.util.{List => JList} import com.google.common.base.Objects import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} + +import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition => TPartition, Table => TTable} import org.apache.hadoop.hive.metastore.{TableType, Warehouse} import org.apache.hadoop.hive.ql.metadata._ @@ -32,6 +34,7 @@ import org.apache.hadoop.hive.serde2.{Deserializer, SerDeException} import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.Logging +import org.apache.spark.sql.hive.client.IsolatedClientLoader import org.apache.spark.sql.{SaveMode, AnalysisException, SQLContext} import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NoSuchTableException, Catalog, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions._ @@ -39,6 +42,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.parquet.{ParquetRelation2, Partition => ParquetPartition, PartitionSpec} import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, DDLParser, LogicalRelation, ResolvedDataSource} import org.apache.spark.sql.types._ @@ -47,11 +51,10 @@ import org.apache.spark.util.Utils /* Implicit conversions */ import scala.collection.JavaConversions._ -private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { - import org.apache.spark.sql.hive.HiveMetastoreTypes._ +private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: HiveContext) + extends Catalog with Logging { - /** Connection to hive metastore. Usages should lock on `this`. */ - protected[hive] val client = Hive.get(hive.hiveconf) + import org.apache.spark.sql.hive.HiveMetastoreTypes._ /** Usages should lock on `this`. */ protected[hive] lazy val hiveWarehouse = new Warehouse(hive.hiveconf) @@ -67,14 +70,12 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val cacheLoader = new CacheLoader[QualifiedTableName, LogicalPlan]() { override def load(in: QualifiedTableName): LogicalPlan = { logDebug(s"Creating new cached data source for $in") - val table = HiveMetastoreCatalog.this.synchronized { - client.getTable(in.database, in.name) - } + val table = client.getTable(in.database, in.name) def schemaStringFromParts: Option[String] = { - Option(table.getProperty("spark.sql.sources.schema.numParts")).map { numParts => + table.properties.get("spark.sql.sources.schema.numParts").map { numParts => val parts = (0 until numParts.toInt).map { index => - val part = table.getProperty(s"spark.sql.sources.schema.part.${index}") + val part = table.properties.get(s"spark.sql.sources.schema.part.${index}").orNull if (part == null) { throw new AnalysisException( s"Could not read schema from the metastore because it is corrupted " + @@ -92,20 +93,20 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // After SPARK-6024, we removed this flag. // Although we are not using spark.sql.sources.schema any more, we need to still support. val schemaString = - Option(table.getProperty("spark.sql.sources.schema")).orElse(schemaStringFromParts) + table.properties.get("spark.sql.sources.schema").orElse(schemaStringFromParts) val userSpecifiedSchema = schemaString.map(s => DataType.fromJson(s).asInstanceOf[StructType]) // It does not appear that the ql client for the metastore has a way to enumerate all the // SerDe properties directly... - val options = table.getTTable.getSd.getSerdeInfo.getParameters.toMap + val options = table.serdeProperties val resolvedRelation = ResolvedDataSource( hive, userSpecifiedSchema, - table.getProperty("spark.sql.sources.provider"), + table.properties("spark.sql.sources.provider"), options) LogicalRelation(resolvedRelation.relation) @@ -144,49 +145,53 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with options: Map[String, String], isExternal: Boolean): Unit = { val (dbName, tblName) = processDatabaseAndTableName("default", tableName) - val tbl = new Table(dbName, tblName) - - tbl.setProperty("spark.sql.sources.provider", provider) + val tableProperties = new scala.collection.mutable.HashMap[String, String] + tableProperties.put("spark.sql.sources.provider", provider) if (userSpecifiedSchema.isDefined) { val threshold = hive.conf.schemaStringLengthThreshold val schemaJsonString = userSpecifiedSchema.get.json // Split the JSON string. val parts = schemaJsonString.grouped(threshold).toSeq - tbl.setProperty("spark.sql.sources.schema.numParts", parts.size.toString) + tableProperties.put("spark.sql.sources.schema.numParts", parts.size.toString) parts.zipWithIndex.foreach { case (part, index) => - tbl.setProperty(s"spark.sql.sources.schema.part.${index}", part) + tableProperties.put(s"spark.sql.sources.schema.part.${index}", part) } } - options.foreach { case (key, value) => tbl.setSerdeParam(key, value) } - if (isExternal) { - tbl.setProperty("EXTERNAL", "TRUE") - tbl.setTableType(TableType.EXTERNAL_TABLE) + val tableType = if (isExternal) { + tableProperties.put("EXTERNAL", "TRUE") + ExternalTable } else { - tbl.setProperty("EXTERNAL", "FALSE") - tbl.setTableType(TableType.MANAGED_TABLE) - } - - // create the table - synchronized { - client.createTable(tbl, false) - } + tableProperties.put("EXTERNAL", "FALSE") + ManagedTable + } + + client.createTable( + HiveTable( + specifiedDatabase = Option(dbName), + name = tblName, + schema = Seq.empty, + partitionColumns = Seq.empty, + tableType = tableType, + properties = tableProperties.toMap, + serdeProperties = options)) } - def hiveDefaultTableFilePath(tableName: String): String = synchronized { - val currentDatabase = client.getDatabase(hive.sessionState.getCurrentDatabase) - - hiveWarehouse.getTablePath(currentDatabase, tableName).toString + def hiveDefaultTableFilePath(tableName: String): String = { + // Code based on: hiveWarehouse.getTablePath(currentDatabase, tableName) + new Path( + new Path(client.getDatabase(client.currentDatabase).location), + tableName.toLowerCase).toString } - def tableExists(tableIdentifier: Seq[String]): Boolean = synchronized { + def tableExists(tableIdentifier: Seq[String]): Boolean = { val tableIdent = processTableIdentifier(tableIdentifier) val databaseName = tableIdent .lift(tableIdent.size - 2) - .getOrElse(hive.sessionState.getCurrentDatabase) + .getOrElse(client.currentDatabase) val tblName = tableIdent.last - client.getTable(databaseName, tblName, false) != null + client.getTableOption(databaseName, tblName).isDefined } def lookupRelation( @@ -194,18 +199,11 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with alias: Option[String]): LogicalPlan = { val tableIdent = processTableIdentifier(tableIdentifier) val databaseName = tableIdent.lift(tableIdent.size - 2).getOrElse( - hive.sessionState.getCurrentDatabase) + client.currentDatabase) val tblName = tableIdent.last - val table = try { - synchronized { - client.getTable(databaseName, tblName) - } - } catch { - case te: org.apache.hadoop.hive.ql.metadata.InvalidTableException => - throw new NoSuchTableException - } + val table = client.getTable(databaseName, tblName) - if (table.getProperty("spark.sql.sources.provider") != null) { + if (table.properties.get("spark.sql.sources.provider").isDefined) { val dataSourceTable = cachedDataSourceTables(QualifiedTableName(databaseName, tblName).toLowerCase) // Then, if alias is specified, wrap the table with a Subquery using the alias. @@ -215,22 +213,16 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Subquery(tableIdent.last, dataSourceTable)) withAlias - } else if (table.isView) { - // if the unresolved relation is from hive view - // parse the text into logic node. - HiveQl.createPlanForView(table, alias) + } else if (table.tableType == VirtualView) { + val viewText = table.viewText.getOrElse(sys.error("Invalid view without text.")) + alias match { + // because hive use things like `_c0` to build the expanded text + // currently we cannot support view from "create view v1(c1) as ..." + case None => Subquery(table.name, HiveQl.createPlan(viewText)) + case Some(aliasText) => Subquery(aliasText, HiveQl.createPlan(viewText)) + } } else { - val partitions: Seq[Partition] = - if (table.isPartitioned) { - synchronized { - HiveShim.getAllPartitionsOf(client, table).toSeq - } - } else { - Nil - } - - MetastoreRelation(databaseName, tblName, alias)( - table.getTTable, partitions.map(part => part.getTPartition))(hive) + MetastoreRelation(databaseName, tblName, alias)(table)(hive) } } @@ -318,178 +310,10 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with result.newInstance() } - override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = synchronized { - val dbName = if (!caseSensitive) { - if (databaseName.isDefined) Some(databaseName.get.toLowerCase) else None - } else { - databaseName - } - val db = dbName.getOrElse(hive.sessionState.getCurrentDatabase) - - client.getAllTables(db).map(tableName => (tableName, false)) - } - - /** - * Create table with specified database, table name, table description and schema - * @param databaseName Database Name - * @param tableName Table Name - * @param schema Schema of the new table, if not specified, will use the schema - * specified in crtTbl - * @param allowExisting if true, ignore AlreadyExistsException - * @param desc CreateTableDesc object which contains the SerDe info. Currently - * we support most of the features except the bucket. - */ - def createTable( - databaseName: String, - tableName: String, - schema: Seq[Attribute], - allowExisting: Boolean = false, - desc: Option[CreateTableDesc] = None) { - val hconf = hive.hiveconf - - val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) - val tbl = new Table(dbName, tblName) - - val crtTbl: CreateTableDesc = desc.getOrElse(null) - - // We should respect the passed in schema, unless it's not set - val hiveSchema: JList[FieldSchema] = if (schema == null || schema.isEmpty) { - crtTbl.getCols - } else { - schema.map(attr => new FieldSchema(attr.name, toMetastoreType(attr.dataType), null)) - } - tbl.setFields(hiveSchema) - - // Most of code are similar with the DDLTask.createTable() of Hive, - if (crtTbl != null && crtTbl.getTblProps() != null) { - tbl.getTTable().getParameters().putAll(crtTbl.getTblProps()) - } - - if (crtTbl != null && crtTbl.getPartCols() != null) { - tbl.setPartCols(crtTbl.getPartCols()) - } - - if (crtTbl != null && crtTbl.getStorageHandler() != null) { - tbl.setProperty( - org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE, - crtTbl.getStorageHandler()) - } - - /* - * We use LazySimpleSerDe by default. - * - * If the user didn't specify a SerDe, and any of the columns are not simple - * types, we will have to use DynamicSerDe instead. - */ - if (crtTbl == null || crtTbl.getSerName() == null) { - val storageHandler = tbl.getStorageHandler() - if (storageHandler == null) { - logInfo(s"Default to LazySimpleSerDe for table $dbName.$tblName") - tbl.setSerializationLib(classOf[LazySimpleSerDe].getName()) - - import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - import org.apache.hadoop.io.Text - import org.apache.hadoop.mapred.TextInputFormat - - tbl.setInputFormatClass(classOf[TextInputFormat]) - tbl.setOutputFormatClass(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]]) - tbl.setSerializationLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") - } else { - val serDeClassName = storageHandler.getSerDeClass().getName() - logInfo(s"Use StorageHandler-supplied $serDeClassName for table $dbName.$tblName") - tbl.setSerializationLib(serDeClassName) - } - } else { - // let's validate that the serde exists - val serdeName = crtTbl.getSerName() - try { - val d = ReflectionUtils.newInstance(hconf.getClassByName(serdeName), hconf) - if (d != null) { - logDebug("Found class for $serdeName") - } - } catch { - case e: SerDeException => throw new HiveException("Cannot validate serde: " + serdeName, e) - } - tbl.setSerializationLib(serdeName) - } - - if (crtTbl != null && crtTbl.getFieldDelim() != null) { - tbl.setSerdeParam(serdeConstants.FIELD_DELIM, crtTbl.getFieldDelim()) - tbl.setSerdeParam(serdeConstants.SERIALIZATION_FORMAT, crtTbl.getFieldDelim()) - } - if (crtTbl != null && crtTbl.getFieldEscape() != null) { - tbl.setSerdeParam(serdeConstants.ESCAPE_CHAR, crtTbl.getFieldEscape()) - } - - if (crtTbl != null && crtTbl.getCollItemDelim() != null) { - tbl.setSerdeParam(serdeConstants.COLLECTION_DELIM, crtTbl.getCollItemDelim()) - } - if (crtTbl != null && crtTbl.getMapKeyDelim() != null) { - tbl.setSerdeParam(serdeConstants.MAPKEY_DELIM, crtTbl.getMapKeyDelim()) - } - if (crtTbl != null && crtTbl.getLineDelim() != null) { - tbl.setSerdeParam(serdeConstants.LINE_DELIM, crtTbl.getLineDelim()) - } - HiveShim.setTblNullFormat(crtTbl, tbl) - - if (crtTbl != null && crtTbl.getSerdeProps() != null) { - val iter = crtTbl.getSerdeProps().entrySet().iterator() - while (iter.hasNext()) { - val m = iter.next() - tbl.setSerdeParam(m.getKey(), m.getValue()) - } - } - - if (crtTbl != null && crtTbl.getComment() != null) { - tbl.setProperty("comment", crtTbl.getComment()) - } - - if (crtTbl != null && crtTbl.getLocation() != null) { - HiveShim.setLocation(tbl, crtTbl) - } - - if (crtTbl != null && crtTbl.getSkewedColNames() != null) { - tbl.setSkewedColNames(crtTbl.getSkewedColNames()) - } - if (crtTbl != null && crtTbl.getSkewedColValues() != null) { - tbl.setSkewedColValues(crtTbl.getSkewedColValues()) - } - - if (crtTbl != null) { - tbl.setStoredAsSubDirectories(crtTbl.isStoredAsSubDirectories()) - tbl.setInputFormatClass(crtTbl.getInputFormat()) - tbl.setOutputFormatClass(crtTbl.getOutputFormat()) - } - - tbl.getTTable().getSd().setInputFormat(tbl.getInputFormatClass().getName()) - tbl.getTTable().getSd().setOutputFormat(tbl.getOutputFormatClass().getName()) - - if (crtTbl != null && crtTbl.isExternal()) { - tbl.setProperty("EXTERNAL", "TRUE") - tbl.setTableType(TableType.EXTERNAL_TABLE) - } - - // set owner - try { - tbl.setOwner(hive.hiveconf.getUser) - } catch { - case e: IOException => throw new HiveException("Unable to get current user", e) - } - - // set create time - tbl.setCreateTime((System.currentTimeMillis() / 1000).asInstanceOf[Int]) - - // TODO add bucket support - // TODO set more info if Hive upgrade + override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { + val db = databaseName.getOrElse(client.currentDatabase) - // create the table - synchronized { - try client.createTable(tbl, allowExisting) catch { - case e: org.apache.hadoop.hive.metastore.api.AlreadyExistsException - if allowExisting => // Do nothing - case e: Throwable => throw e - } - } + client.listTables(db).map(tableName => (tableName, false)) } protected def processDatabaseAndTableName( @@ -598,42 +422,11 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - // TODO extra is in type of ASTNode which means the logical plan is not resolved - // Need to think about how to implement the CreateTableAsSelect.resolved - case CreateTableAsSelect(db, tableName, child, allowExisting, Some(extra: ASTNode)) => - val (dbName, tblName) = processDatabaseAndTableName(db, tableName) - val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) - - // Get the CreateTableDesc from Hive SemanticAnalyzer - val desc: Option[CreateTableDesc] = if (tableExists(Seq(databaseName, tblName))) { - None - } else { - val sa = new SemanticAnalyzer(hive.hiveconf) { - override def analyzeInternal(ast: ASTNode) { - // A hack to intercept the SemanticAnalyzer.analyzeInternal, - // to ignore the SELECT clause of the CTAS - val method = classOf[SemanticAnalyzer].getDeclaredMethod( - "analyzeCreateTable", classOf[ASTNode], classOf[QB]) - method.setAccessible(true) - method.invoke(this, ast, this.getQB) - } - } - - sa.analyze(extra, new Context(hive.hiveconf)) - Some(sa.getQB().getTableDesc) - } - - // Check if the query specifies file format or storage handler. - val hasStorageSpec = desc match { - case Some(crtTbl) => - crtTbl != null && (crtTbl.getSerName != null || crtTbl.getStorageHandler != null) - case None => false - } - - if (hive.convertCTAS && !hasStorageSpec) { + case CreateTableAsSelect(desc, child, allowExisting) => + if (hive.convertCTAS && !desc.serde.isDefined) { // Do the conversion when spark.sql.hive.convertCTAS is true and the query // does not specify any storage format (file format and storage handler). - if (dbName.isDefined) { + if (desc.specifiedDatabase.isDefined) { throw new AnalysisException( "Cannot specify database name in a CTAS statement " + "when spark.sql.hive.convertCTAS is set to true.") @@ -641,7 +434,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val mode = if (allowExisting) SaveMode.Ignore else SaveMode.ErrorIfExists CreateTableUsingAsSelect( - tblName, + desc.name, hive.conf.defaultDataSourceName, temporary = false, mode, @@ -650,19 +443,19 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with ) } else { execution.CreateTableAsSelect( - databaseName, - tableName, + desc.copy( + specifiedDatabase = Option(desc.specifiedDatabase.getOrElse(client.currentDatabase))), child, - allowExisting, - desc) + allowExisting) } case p: LogicalPlan if p.resolved => p - case p @ CreateTableAsSelect(db, tableName, child, allowExisting, None) => - val (dbName, tblName) = processDatabaseAndTableName(db, tableName) + case p @ CreateTableAsSelect(desc, child, allowExisting) => + val (dbName, tblName) = processDatabaseAndTableName(desc.database, desc.name) + if (hive.convertCTAS) { - if (dbName.isDefined) { + if (desc.specifiedDatabase.isDefined) { throw new AnalysisException( "Cannot specify database name in a CTAS statement " + "when spark.sql.hive.convertCTAS is set to true.") @@ -678,13 +471,10 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with child ) } else { - val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) execution.CreateTableAsSelect( - databaseName, - tableName, + desc, child, - allowExisting, - None) + allowExisting) } } } @@ -767,7 +557,7 @@ private[hive] case class InsertIntoHiveTable( private[hive] case class MetastoreRelation (databaseName: String, tableName: String, alias: Option[String]) - (val table: TTable, val partitions: Seq[TPartition]) + (val table: HiveTable) (@transient sqlContext: SQLContext) extends LeafNode with MultiInstanceRelation { @@ -786,16 +576,63 @@ private[hive] case class MetastoreRelation Objects.hashCode(databaseName, tableName, alias, output) } - // TODO: Can we use org.apache.hadoop.hive.ql.metadata.Table as the type of table and - // use org.apache.hadoop.hive.ql.metadata.Partition as the type of elements of partitions. - // Right now, using org.apache.hadoop.hive.ql.metadata.Table and - // org.apache.hadoop.hive.ql.metadata.Partition will cause a NotSerializableException - // which indicates the SerDe we used is not Serializable. + @transient val hiveQlTable: Table = { + // We start by constructing an API table as Hive performs several important transformations + // internally when converting an API table to a QL table. + val tTable = new org.apache.hadoop.hive.metastore.api.Table() + tTable.setTableName(table.name) + tTable.setDbName(table.database) + + val tableParameters = new java.util.HashMap[String, String]() + tTable.setParameters(tableParameters) + table.properties.foreach { case (k, v) => tableParameters.put(k, v) } + + tTable.setTableType(table.tableType.name) + + val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() + tTable.setSd(sd) + sd.setCols(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) + tTable.setPartitionKeys( + table.partitionColumns.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) + + table.location.foreach(sd.setLocation) + table.inputFormat.foreach(sd.setInputFormat) + table.outputFormat.foreach(sd.setOutputFormat) + + val serdeInfo = new org.apache.hadoop.hive.metastore.api.SerDeInfo + sd.setSerdeInfo(serdeInfo) + table.serde.foreach(serdeInfo.setSerializationLib) + val serdeParameters = new java.util.HashMap[String, String]() + serdeInfo.setParameters(serdeParameters) + table.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } + + new Table(tTable) + } + + @transient val hiveQlPartitions: Seq[Partition] = table.getAllPartitions.map { p => + val tPartition = new org.apache.hadoop.hive.metastore.api.Partition + tPartition.setDbName(databaseName) + tPartition.setTableName(tableName) + tPartition.setValues(p.values) + + val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() + tPartition.setSd(sd) + sd.setCols(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) + + sd.setLocation(p.storage.location) + sd.setInputFormat(p.storage.inputFormat) + sd.setOutputFormat(p.storage.outputFormat) + + val serdeInfo = new org.apache.hadoop.hive.metastore.api.SerDeInfo + sd.setSerdeInfo(serdeInfo) + serdeInfo.setSerializationLib(p.storage.serde) - @transient val hiveQlTable: Table = new Table(table) + val serdeParameters = new java.util.HashMap[String, String]() + serdeInfo.setParameters(serdeParameters) + table.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } + p.storage.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } - @transient val hiveQlPartitions: Seq[Partition] = partitions.map { p => - new Partition(hiveQlTable, p) + new Partition(hiveQlTable, tPartition) } @transient override lazy val statistics: Statistics = Statistics( @@ -865,7 +702,7 @@ private[hive] case class MetastoreRelation val columnOrdinals = AttributeMap(attributes.zipWithIndex) override def newInstance(): MetastoreRelation = { - MetastoreRelation(databaseName, tableName, alias)(table, partitions)(sqlContext) + MetastoreRelation(databaseName, tableName, alias)(table)(sqlContext) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 63a8c05f775b8..04d40bbb2bced 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.hive.ql.lib.Node import org.apache.hadoop.hive.ql.metadata.Table import org.apache.hadoop.hive.ql.parse._ import org.apache.hadoop.hive.ql.plan.PlanUtils -import org.apache.spark.sql.{AnalysisException, SparkSQLParser} +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ @@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.execution.ExplainCommand import org.apache.spark.sql.sources.DescribeCommand +import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable, HiveScriptIOSchema} import org.apache.spark.sql.types._ import org.apache.spark.util.random.RandomSampler @@ -50,7 +51,19 @@ import scala.collection.JavaConversions._ * back for Hive to execute natively. Will be replaced with a native command that contains the * cmd string. */ -private[hive] case object NativePlaceholder extends Command +private[hive] case object NativePlaceholder extends LogicalPlan { + override def children: Seq[LogicalPlan] = Seq.empty + override def output: Seq[Attribute] = Seq.empty +} + +case class CreateTableAsSelect( + tableDesc: HiveTable, + child: LogicalPlan, + allowExisting: Boolean) extends UnaryNode with Command { + + override def output: Seq[Attribute] = Seq.empty[Attribute] + override lazy val resolved: Boolean = tableDesc.specifiedDatabase.isDefined && childrenResolved +} /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ private[hive] object HiveQl { @@ -78,16 +91,16 @@ private[hive] object HiveQl { "TOK_ALTERVIEW_DROPPARTS", "TOK_ALTERVIEW_PROPERTIES", "TOK_ALTERVIEW_RENAME", - + "TOK_CREATEDATABASE", "TOK_CREATEFUNCTION", "TOK_CREATEINDEX", "TOK_CREATEROLE", "TOK_CREATEVIEW", - + "TOK_DESCDATABASE", "TOK_DESCFUNCTION", - + "TOK_DROPDATABASE", "TOK_DROPFUNCTION", "TOK_DROPINDEX", @@ -95,22 +108,22 @@ private[hive] object HiveQl { "TOK_DROPTABLE_PROPERTIES", "TOK_DROPVIEW", "TOK_DROPVIEW_PROPERTIES", - + "TOK_EXPORT", - + "TOK_GRANT", "TOK_GRANT_ROLE", - + "TOK_IMPORT", - + "TOK_LOAD", - + "TOK_LOCKTABLE", - + "TOK_MSCK", - + "TOK_REVOKE", - + "TOK_SHOW_COMPACTIONS", "TOK_SHOW_CREATETABLE", "TOK_SHOW_GRANT", @@ -127,9 +140,9 @@ private[hive] object HiveQl { "TOK_SHOWINDEXES", "TOK_SHOWLOCKS", "TOK_SHOWPARTITIONS", - + "TOK_SWITCHDATABASE", - + "TOK_UNLOCKTABLE" ) @@ -140,10 +153,7 @@ private[hive] object HiveQl { "TOK_TRUNCATETABLE" // truncate table" is a NativeCommand, does not need to explain. ) ++ nativeCommands - protected val hqlParser = { - val fallback = new ExtendedHiveQlParser - new SparkSQLParser(fallback.parse(_)) - } + protected val hqlParser = new ExtendedHiveQlParser /** * A set of implicit transformations that allow Hive ASTNodes to be rewritten by transformations @@ -262,6 +272,7 @@ private[hive] object HiveQl { case otherMessage => throw new AnalysisException(otherMessage) } + case e: MatchError => throw e case e: Exception => throw new AnalysisException(e.getMessage) case e: NotImplementedError => @@ -275,14 +286,6 @@ private[hive] object HiveQl { } } - /** Creates LogicalPlan for a given VIEW */ - def createPlanForView(view: Table, alias: Option[String]): Subquery = alias match { - // because hive use things like `_c0` to build the expanded text - // currently we cannot support view from "create view v1(c1) as ..." - case None => Subquery(view.getTableName, createPlan(view.getViewExpandedText)) - case Some(aliasText) => Subquery(aliasText, createPlan(view.getViewExpandedText)) - } - def parseDdl(ddl: String): Seq[Attribute] = { val tree = try { @@ -421,16 +424,16 @@ private[hive] object HiveQl { } /** - * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2)) - * is equivalent to + * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2)) + * is equivalent to * SELECT MAX(value) FROM src GROUP BY k1, k2 UNION SELECT MAX(value) FROM src GROUP BY k2 * Check the following link for details. - * + * https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C+Grouping+and+Rollup * * The bitmask denotes the grouping expressions validity for a grouping set, * the bitmask also be called as grouping id (`GROUPING__ID`, the virtual column in Hive) - * e.g. In superset (k1, k2, k3), (bit 0: k1, bit 1: k2, and bit 2: k3), the grouping id of + * e.g. In superset (k1, k2, k3), (bit 0: k1, bit 1: k2, and bit 2: k3), the grouping id of * GROUPING SETS (k1, k2) and (k2) should be 3 and 2 respectively. */ protected def extractGroupingSet(children: Seq[ASTNode]): (Seq[Expression], Seq[Int]) = { @@ -444,7 +447,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val bitmasks: Seq[Int] = setASTs.map(set => set match { case Token("TOK_GROUPING_SETS_EXPRESSION", null) => 0 - case Token("TOK_GROUPING_SETS_EXPRESSION", children) => + case Token("TOK_GROUPING_SETS_EXPRESSION", children) => children.foldLeft(0)((bitmap, col) => { val colString = col.asInstanceOf[ASTNode].toStringTree() require(keyMap.contains(colString), s"$colString doens't show up in the GROUP BY list") @@ -456,6 +459,14 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C (keys, bitmasks) } + protected def getProperties(node: Node): Seq[(String, String)] = node match { + case Token("TOK_TABLEPROPLIST", list) => + list.map { + case Token("TOK_TABLEPROPERTY", Token(key, Nil) :: Token(value, Nil) :: Nil) => + (unquoteString(key) -> unquoteString(value)) + } + } + protected def nodeToPlan(node: Node): LogicalPlan = node match { // Special drop table that also uncaches. case Token("TOK_DROPTABLE", @@ -565,7 +576,62 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C children) val (db, tableName) = extractDbNameTableName(tableNameParts) - CreateTableAsSelect(db, tableName, nodeToPlan(query), allowExisting != None, Some(node)) + var tableDesc = + HiveTable( + specifiedDatabase = db, + name = tableName, + schema = Seq.empty, + partitionColumns = Seq.empty, + properties = Map.empty, + serdeProperties = Map.empty, + tableType = ManagedTable, + location = None, + inputFormat = None, + outputFormat = None, + serde = None) + + // TODO: Handle all the cases here... + children.foreach { + case Token("TOK_TBLRCFILE", Nil) => + import org.apache.hadoop.hive.ql.io.{RCFileInputFormat, RCFileOutputFormat} + tableDesc = tableDesc.copy( + outputFormat = Option(classOf[RCFileOutputFormat].getName), + inputFormat = Option(classOf[RCFileInputFormat[_, _]].getName)) + + if (tableDesc.serde.isEmpty) { + tableDesc = tableDesc.copy( + serde = Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe")) + } + case Token("TOK_TBLORCFILE", Nil) => + tableDesc = tableDesc.copy( + inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"), + serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) + + case Token("TOK_TBLPARQUETFILE", Nil) => + tableDesc = tableDesc.copy( + inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"), + serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) + + case Token("TOK_TABLESERIALIZER", + Token("TOK_SERDENAME", Token(serdeName, Nil) :: otherProps) :: Nil) => + tableDesc = tableDesc.copy(serde = Option(unquoteString(serdeName))) + + otherProps match { + case Token("TOK_TABLEPROPERTIES", list :: Nil) :: Nil => + tableDesc = tableDesc.copy( + serdeProperties = tableDesc.serdeProperties ++ getProperties(list)) + case Nil => + } + + case Token("TOK_TABLEPROPERTIES", list :: Nil) => + tableDesc = tableDesc.copy(properties = tableDesc.properties ++ getProperties(list)) + + case _ => + } + + CreateTableAsSelect(tableDesc, nodeToPlan(query), allowExisting != None) // If its not a "CREATE TABLE AS" like above then just pass it back to hive as a native command. case Token("TOK_CREATETABLE", _) => NativePlaceholder @@ -613,7 +679,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C clusterByClause :: distributeByClause :: limitClause :: - lateralViewClause :: Nil) = { + lateralViewClause :: + windowClause :: Nil) = { getClauses( Seq( "TOK_INSERT_INTO", @@ -631,15 +698,16 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C "TOK_CLUSTERBY", "TOK_DISTRIBUTEBY", "TOK_LIMIT", - "TOK_LATERAL_VIEW"), + "TOK_LATERAL_VIEW", + "WINDOW"), singleInsert) } - + val relations = fromClause match { case Some(f) => nodeToRelation(f) case None => OneRowRelation } - + val withWhere = whereClause.map { whereNode => val Seq(whereExpr) = whereNode.getChildren.toSeq Filter(nodeToExpr(whereExpr), relations) @@ -691,7 +759,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val serdeProps = propsClause.map { case Token("TOK_TABLEPROPERTY", Token(name, Nil) :: Token(value, Nil) :: Nil) => (name, value) - } + } (Nil, serdeClass, serdeProps) case Nil => (Nil, "", Nil) @@ -736,7 +804,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C // The projection of the query can either be a normal projection, an aggregation // (if there is a group by) or a script transformation. val withProject: LogicalPlan = transformation.getOrElse { - val selectExpressions = + val selectExpressions = nameExpressions(select.getChildren.flatMap(selExprNodeToExpr).toSeq) Seq( groupByClause.map(e => e match { @@ -760,35 +828,38 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token("TOK_CUBE_GROUPBY", children) => Cube(children.map(nodeToExpr), withLateralView, selectExpressions) case _ => sys.error("Expect WITH CUBE") - }), + }), Some(Project(selectExpressions, withLateralView))).flatten.head } - val withDistinct = - if (selectDistinctClause.isDefined) Distinct(withProject) else withProject - + // Handle HAVING clause. val withHaving = havingClause.map { h => val havingExpr = h.getChildren.toSeq match { case Seq(hexpr) => nodeToExpr(hexpr) } // Note that we added a cast to boolean. If the expression itself is already boolean, // the optimizer will get rid of the unnecessary cast. - Filter(Cast(havingExpr, BooleanType), withDistinct) - }.getOrElse(withDistinct) + Filter(Cast(havingExpr, BooleanType), withProject) + }.getOrElse(withProject) + + // Handle SELECT DISTINCT + val withDistinct = + if (selectDistinctClause.isDefined) Distinct(withHaving) else withHaving + // Handle ORDER BY, SORT BY, DISTRIBETU BY, and CLUSTER BY clause. val withSort = (orderByClause, sortByClause, distributeByClause, clusterByClause) match { case (Some(totalOrdering), None, None, None) => - Sort(totalOrdering.getChildren.map(nodeToSortOrder), true, withHaving) + Sort(totalOrdering.getChildren.map(nodeToSortOrder), true, withDistinct) case (None, Some(perPartitionOrdering), None, None) => - Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, withHaving) + Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, withDistinct) case (None, None, Some(partitionExprs), None) => - RepartitionByExpression(partitionExprs.getChildren.map(nodeToExpr), withHaving) + RepartitionByExpression(partitionExprs.getChildren.map(nodeToExpr), withDistinct) case (None, Some(perPartitionOrdering), Some(partitionExprs), None) => Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, - RepartitionByExpression(partitionExprs.getChildren.map(nodeToExpr), withHaving)) + RepartitionByExpression(partitionExprs.getChildren.map(nodeToExpr), withDistinct)) case (None, None, None, Some(clusterExprs)) => Sort(clusterExprs.getChildren.map(nodeToExpr).map(SortOrder(_, Ascending)), false, - RepartitionByExpression(clusterExprs.getChildren.map(nodeToExpr), withHaving)) - case (None, None, None, None) => withHaving + RepartitionByExpression(clusterExprs.getChildren.map(nodeToExpr), withDistinct)) + case (None, None, None, None) => withDistinct case _ => sys.error("Unsupported set of ordering / distribution clauses.") } @@ -797,6 +868,27 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C .map(Limit(_, withSort)) .getOrElse(withSort) + // Collect all window specifications defined in the WINDOW clause. + val windowDefinitions = windowClause.map(_.getChildren.toSeq.collect { + case Token("TOK_WINDOWDEF", + Token(windowName, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) => + windowName -> nodesToWindowSpecification(spec) + }.toMap) + // Handle cases like + // window w1 as (partition by p_mfgr order by p_name + // range between 2 preceding and 2 following), + // w2 as w1 + val resolvedCrossReference = windowDefinitions.map { + windowDefMap => windowDefMap.map { + case (windowName, WindowSpecReference(other)) => + (windowName, windowDefMap(other).asInstanceOf[WindowSpecDefinition]) + case o => o.asInstanceOf[(String, WindowSpecDefinition)] + } + } + + val withWindowDefinitions = + resolvedCrossReference.map(WithWindowDefinition(_, withLimit)).getOrElse(withLimit) + // TOK_INSERT_INTO means to add files to the table. // TOK_DESTINATION means to overwrite the table. val resultDestination = @@ -804,7 +896,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val overwrite = intoClause.isEmpty nodeToDest( resultDestination, - withLimit, + withWindowDefinitions, overwrite) } @@ -1053,8 +1145,16 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") } - protected val escapedIdentifier = "`([^`]+)`".r + protected val doubleQuotedString = "\"([^\"]+)\"".r + protected val singleQuotedString = "'([^']+)'".r + + protected def unquoteString(str: String) = str match { + case singleQuotedString(s) => s + case doubleQuotedString(s) => s + case other => other + } + /** Strips backticks from ident if present */ protected def cleanIdentifier(ident: String): String = ident match { case escapedIdentifier(i) => i @@ -1104,7 +1204,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C nodeToExpr(qualifier) match { case UnresolvedAttribute(qualifierName) => UnresolvedAttribute(qualifierName :+ cleanIdentifier(attr)) - case other => UnresolvedGetField(other, attr) + case other => UnresolvedExtractValue(other, Literal(attr)) } /* Stars (*) */ @@ -1224,20 +1324,12 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token("TOK_FUNCTION", Token(WHEN(), Nil) :: branches) => CaseWhen(branches.map(nodeToExpr)) case Token("TOK_FUNCTION", Token(CASE(), Nil) :: branches) => - val transformed = branches.drop(1).sliding(2, 2).map { - case Seq(condVal, value) => - // FIXME (SPARK-2155): the key will get evaluated for multiple times in CaseWhen's eval(). - // Hence effectful / non-deterministic key expressions are *not* supported at the moment. - // We should consider adding new Expressions to get around this. - Seq(EqualTo(nodeToExpr(branches(0)), nodeToExpr(condVal)), - nodeToExpr(value)) - case Seq(elseVal) => Seq(nodeToExpr(elseVal)) - }.toSeq.reduce(_ ++ _) - CaseWhen(transformed) + val keyExpr = nodeToExpr(branches.head) + CaseKeyWhen(keyExpr, branches.drop(1).map(nodeToExpr)) /* Complex datatype manipulation */ case Token("[", child :: ordinal :: Nil) => - GetItem(nodeToExpr(child), nodeToExpr(ordinal)) + UnresolvedExtractValue(nodeToExpr(child), nodeToExpr(ordinal)) /* Other functions */ case Token("TOK_FUNCTION", Token(ARRAY(), Nil) :: children) => @@ -1250,6 +1342,25 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C Substring(nodeToExpr(string), nodeToExpr(pos), nodeToExpr(length)) case Token("TOK_FUNCTION", Token(COALESCE(), Nil) :: list) => Coalesce(list.map(nodeToExpr)) + /* Window Functions */ + case Token("TOK_FUNCTION", Token(name, Nil) +: args :+ Token("TOK_WINDOWSPEC", spec)) => + val function = UnresolvedWindowFunction(name, args.map(nodeToExpr)) + nodesToWindowSpecification(spec) match { + case reference: WindowSpecReference => + UnresolvedWindowExpression(function, reference) + case definition: WindowSpecDefinition => + WindowExpression(function, definition) + } + case Token("TOK_FUNCTIONSTAR", Token(name, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) => + // Safe to use Literal(1)? + val function = UnresolvedWindowFunction(name, Literal(1) :: Nil) + nodesToWindowSpecification(spec) match { + case reference: WindowSpecReference => + UnresolvedWindowExpression(function, reference) + case definition: WindowSpecDefinition => + WindowExpression(function, definition) + } + /* UDFs - Must be last otherwise will preempt built in functions */ case Token("TOK_FUNCTION", Token(name, Nil) :: args) => UnresolvedFunction(name, args.map(nodeToExpr)) @@ -1312,6 +1423,89 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C """.stripMargin) } + def nodesToWindowSpecification(nodes: Seq[ASTNode]): WindowSpec = nodes match { + case Token(windowName, Nil) :: Nil => + // Refer to a window spec defined in the window clause. + WindowSpecReference(windowName) + case Nil => + // OVER() + WindowSpecDefinition( + partitionSpec = Nil, + orderSpec = Nil, + frameSpecification = UnspecifiedFrame) + case spec => + val (partitionClause :: rowFrame :: rangeFrame :: Nil) = + getClauses( + Seq( + "TOK_PARTITIONINGSPEC", + "TOK_WINDOWRANGE", + "TOK_WINDOWVALUES"), + spec) + + // Handle Partition By and Order By. + val (partitionSpec, orderSpec) = partitionClause.map { partitionAndOrdering => + val (partitionByClause :: orderByClause :: sortByClause :: clusterByClause :: Nil) = + getClauses( + Seq("TOK_DISTRIBUTEBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_CLUSTERBY"), + partitionAndOrdering.getChildren.toSeq.asInstanceOf[Seq[ASTNode]]) + + (partitionByClause, orderByClause.orElse(sortByClause), clusterByClause) match { + case (Some(partitionByExpr), Some(orderByExpr), None) => + (partitionByExpr.getChildren.map(nodeToExpr), + orderByExpr.getChildren.map(nodeToSortOrder)) + case (Some(partitionByExpr), None, None) => + (partitionByExpr.getChildren.map(nodeToExpr), Nil) + case (None, Some(orderByExpr), None) => + (Nil, orderByExpr.getChildren.map(nodeToSortOrder)) + case (None, None, Some(clusterByExpr)) => + val expressions = clusterByExpr.getChildren.map(nodeToExpr) + (expressions, expressions.map(SortOrder(_, Ascending))) + case _ => + throw new NotImplementedError( + s"""No parse rules for Node ${partitionAndOrdering.getName} + """.stripMargin) + } + }.getOrElse { + (Nil, Nil) + } + + // Handle Window Frame + val windowFrame = + if (rowFrame.isEmpty && rangeFrame.isEmpty) { + UnspecifiedFrame + } else { + val frameType = rowFrame.map(_ => RowFrame).getOrElse(RangeFrame) + def nodeToBoundary(node: Node): FrameBoundary = node match { + case Token("preceding", Token(count, Nil) :: Nil) => + if (count == "unbounded") UnboundedPreceding else ValuePreceding(count.toInt) + case Token("following", Token(count, Nil) :: Nil) => + if (count == "unbounded") UnboundedFollowing else ValueFollowing(count.toInt) + case Token("current", Nil) => CurrentRow + case _ => + throw new NotImplementedError( + s"""No parse rules for the Window Frame Boundary based on Node ${node.getName} + """.stripMargin) + } + + rowFrame.orElse(rangeFrame).map { frame => + frame.getChildren.toList match { + case precedingNode :: followingNode :: Nil => + SpecifiedWindowFrame( + frameType, + nodeToBoundary(precedingNode), + nodeToBoundary(followingNode)) + case precedingNode :: Nil => + SpecifiedWindowFrame(frameType, nodeToBoundary(precedingNode), CurrentRow) + case _ => + throw new NotImplementedError( + s"""No parse rules for the Window Frame based on Node ${frame.getName} + """.stripMargin) + } + }.getOrElse(sys.error(s"If you see this, please file a bug report with your query.")) + } + + WindowSpecDefinition(partitionSpec, orderSpec, windowFrame) + } val explode = "(?i)explode".r def nodesToGenerator(nodes: Seq[Node]): (Generator, Seq[String]) = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index e556c74ffb015..b69312f0f8717 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -32,6 +32,7 @@ import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} import org.apache.spark.SerializableWritable import org.apache.spark.broadcast.Broadcast +import org.apache.spark.Logging import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.DateUtils @@ -57,7 +58,7 @@ class HadoopTableReader( @transient relation: MetastoreRelation, @transient sc: HiveContext, @transient hiveExtraConf: HiveConf) - extends TableReader { + extends TableReader with Logging { // Hadoop honors "mapred.map.tasks" as hint, but will ignore when mapred.job.tracker is "local". // https://hadoop.apache.org/docs/r1.0.4/mapred-default.html @@ -78,7 +79,7 @@ class HadoopTableReader( makeRDDForTable( hiveTable, Class.forName( - relation.tableDesc.getSerdeClassName, true, sc.sessionState.getConf.getClassLoader) + relation.tableDesc.getSerdeClassName, true, Utils.getSparkClassLoader) .asInstanceOf[Class[Deserializer]], filterOpt = None) @@ -145,7 +146,7 @@ class HadoopTableReader( partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], filterOpt: Option[PathFilter]): RDD[Row] = { - + // SPARK-5068:get FileStatus and do the filtering locally when the path is not exists def verifyPartitionPath( partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]]): @@ -288,7 +289,7 @@ class HadoopTableReader( } } -private[hive] object HadoopTableReader extends HiveInspectors { +private[hive] object HadoopTableReader extends HiveInspectors with Logging { /** * Curried. After given an argument for 'path', the resulting JobConf => Unit closure is used to * instantiate a HadoopRDD. @@ -329,6 +330,8 @@ private[hive] object HadoopTableReader extends HiveInspectors { tableDeser.getObjectInspector).asInstanceOf[StructObjectInspector] } + logDebug(soi.toString) + val (fieldRefs, fieldOrdinals) = nonPartitionKeyAttrs.map { case (attr, ordinal) => soi.getStructFieldRef(attr.name) -> ordinal }.unzip diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala index a863aa77cb7e0..0a1d761a52f88 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala @@ -17,30 +17,35 @@ package org.apache.spark.sql.hive.client +import java.io.PrintStream +import java.util.{Map => JMap} + import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchTableException} -case class HiveDatabase( +private[hive] case class HiveDatabase( name: String, location: String) -abstract class TableType { val name: String } -case object ExternalTable extends TableType { override val name = "EXTERNAL_TABLE" } -case object IndexTable extends TableType { override val name = "INDEX_TABLE" } -case object ManagedTable extends TableType { override val name = "MANAGED_TABLE" } -case object VirtualView extends TableType { override val name = "VIRTUAL_VIEW" } +private[hive] abstract class TableType { val name: String } +private[hive] case object ExternalTable extends TableType { override val name = "EXTERNAL_TABLE" } +private[hive] case object IndexTable extends TableType { override val name = "INDEX_TABLE" } +private[hive] case object ManagedTable extends TableType { override val name = "MANAGED_TABLE" } +private[hive] case object VirtualView extends TableType { override val name = "VIRTUAL_VIEW" } -case class HiveStorageDescriptor( +// TODO: Use this for Tables and Partitions +private[hive] case class HiveStorageDescriptor( location: String, inputFormat: String, outputFormat: String, - serde: String) + serde: String, + serdeProperties: Map[String, String]) -case class HivePartition( +private[hive] case class HivePartition( values: Seq[String], storage: HiveStorageDescriptor) -case class HiveColumn(name: String, hiveType: String, comment: String) -case class HiveTable( +private[hive] case class HiveColumn(name: String, hiveType: String, comment: String) +private[hive] case class HiveTable( specifiedDatabase: Option[String], name: String, schema: Seq[HiveColumn], @@ -51,7 +56,8 @@ case class HiveTable( location: Option[String] = None, inputFormat: Option[String] = None, outputFormat: Option[String] = None, - serde: Option[String] = None) { + serde: Option[String] = None, + viewText: Option[String] = None) { @transient private[client] var client: ClientInterface = _ @@ -76,13 +82,17 @@ case class HiveTable( * internal and external classloaders for a given version of Hive and thus must expose only * shared classes. */ -trait ClientInterface { +private[hive] trait ClientInterface { /** * Runs a HiveQL command using Hive, returning the results as a list of strings. Each row will * result in one string. */ def runSqlHive(sql: String): Seq[String] + def setOut(stream: PrintStream): Unit + def setInfo(stream: PrintStream): Unit + def setError(stream: PrintStream): Unit + /** Returns the names of all tables in the given database. */ def listTables(dbName: String): Seq[String] @@ -114,6 +124,11 @@ trait ClientInterface { /** Creates a new database with the given name. */ def createDatabase(database: HiveDatabase): Unit + /** Returns the specified paritition or None if it does not exist. */ + def getPartitionOption( + hTable: HiveTable, + partitionSpec: JMap[String, String]): Option[HivePartition] + /** Returns all partitions for the given table. */ def getAllPartitions(hTable: HiveTable): Seq[HivePartition] diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala index ea52fea037f1f..6bca9d0179fe3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.client import java.io.{BufferedReader, InputStreamReader, File, PrintStream} import java.net.URI -import java.util.{ArrayList => JArrayList} +import java.util.{ArrayList => JArrayList, Map => JMap, List => JList, Set => JSet} import scala.collection.JavaConversions._ import scala.language.reflectiveCalls @@ -27,6 +27,7 @@ import scala.language.reflectiveCalls import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.metastore.api.Database import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.TableType import org.apache.hadoop.hive.metastore.api import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.ql.metadata @@ -54,19 +55,13 @@ import org.apache.spark.sql.execution.QueryExecutionException * @param config a collection of configuration options that will be added to the hive conf before * opening the hive client. */ -class ClientWrapper( +private[hive] class ClientWrapper( version: HiveVersion, config: Map[String, String]) extends ClientInterface with Logging with ReflectionMagic { - private val conf = new HiveConf(classOf[SessionState]) - config.foreach { case (k, v) => - logDebug(s"Hive Config: $k=$v") - conf.set(k, v) - } - // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. private val outputBuffer = new java.io.OutputStream { var pos: Int = 0 @@ -99,17 +94,31 @@ class ClientWrapper( val original = Thread.currentThread().getContextClassLoader Thread.currentThread().setContextClassLoader(getClass.getClassLoader) val ret = try { - val newState = new SessionState(conf) - SessionState.start(newState) - newState.out = new PrintStream(outputBuffer, true, "UTF-8") - newState.err = new PrintStream(outputBuffer, true, "UTF-8") - newState + val oldState = SessionState.get() + if (oldState == null) { + val initialConf = new HiveConf(classOf[SessionState]) + config.foreach { case (k, v) => + logDebug(s"Hive Config: $k=$v") + initialConf.set(k, v) + } + val newState = new SessionState(initialConf) + SessionState.start(newState) + newState.out = new PrintStream(outputBuffer, true, "UTF-8") + newState.err = new PrintStream(outputBuffer, true, "UTF-8") + newState + } else { + oldState + } } finally { Thread.currentThread().setContextClassLoader(original) } ret } + /** Returns the configuration for the current session. */ + def conf: HiveConf = SessionState.get().getConf + + // TODO: should be a def?s private val client = Hive.get(conf) /** @@ -133,6 +142,18 @@ class ClientWrapper( ret } + def setOut(stream: PrintStream): Unit = withHiveState { + state.out = stream + } + + def setInfo(stream: PrintStream): Unit = withHiveState { + state.info = stream + } + + def setError(stream: PrintStream): Unit = withHiveState { + state.err = stream + } + override def currentDatabase: String = withHiveState { state.getCurrentDatabase } @@ -171,14 +192,20 @@ class ClientWrapper( partitionColumns = h.getPartCols.map(f => HiveColumn(f.getName, f.getType, f.getComment)), properties = h.getParameters.toMap, serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.toMap, - tableType = ManagedTable, // TODO + tableType = h.getTableType match { + case TableType.MANAGED_TABLE => ManagedTable + case TableType.EXTERNAL_TABLE => ExternalTable + case TableType.VIRTUAL_VIEW => VirtualView + case TableType.INDEX_TABLE => IndexTable + }, location = version match { case hive.v12 => Option(h.call[URI]("getDataLocation")).map(_.toString) case hive.v13 => Option(h.call[Path]("getDataLocation")).map(_.toString) }, inputFormat = Option(h.getInputFormatClass).map(_.getName), outputFormat = Option(h.getOutputFormatClass).map(_.getName), - serde = Option(h.getSerializationLib)).withClient(this) + serde = Option(h.getSerializationLib), + viewText = Option(h.getViewExpandedText)).withClient(this) } converted } @@ -223,27 +250,40 @@ class ClientWrapper( client.alterTable(table.qualifiedName, qlTable) } + private def toHivePartition(partition: metadata.Partition): HivePartition = { + val apiPartition = partition.getTPartition + HivePartition( + values = Option(apiPartition.getValues).map(_.toSeq).getOrElse(Seq.empty), + storage = HiveStorageDescriptor( + location = apiPartition.getSd.getLocation, + inputFormat = apiPartition.getSd.getInputFormat, + outputFormat = apiPartition.getSd.getOutputFormat, + serde = apiPartition.getSd.getSerdeInfo.getSerializationLib, + serdeProperties = apiPartition.getSd.getSerdeInfo.getParameters.toMap)) + } + + override def getPartitionOption( + table: HiveTable, + partitionSpec: JMap[String, String]): Option[HivePartition] = withHiveState { + + val qlTable = toQlTable(table) + val qlPartition = client.getPartition(qlTable, partitionSpec, false) + Option(qlPartition).map(toHivePartition) + } + override def getAllPartitions(hTable: HiveTable): Seq[HivePartition] = withHiveState { val qlTable = toQlTable(hTable) val qlPartitions = version match { case hive.v12 => - client.call[metadata.Table, Set[metadata.Partition]]("getAllPartitionsForPruner", qlTable) + client.call[metadata.Table, JSet[metadata.Partition]]("getAllPartitionsForPruner", qlTable) case hive.v13 => - client.call[metadata.Table, Set[metadata.Partition]]("getAllPartitionsOf", qlTable) + client.call[metadata.Table, JSet[metadata.Partition]]("getAllPartitionsOf", qlTable) } - qlPartitions.map(_.getTPartition).map { p => - HivePartition( - values = Option(p.getValues).map(_.toSeq).getOrElse(Seq.empty), - storage = HiveStorageDescriptor( - location = p.getSd.getLocation, - inputFormat = p.getSd.getInputFormat, - outputFormat = p.getSd.getOutputFormat, - serde = p.getSd.getSerdeInfo.getSerializationLib)) - }.toSeq + qlPartitions.toSeq.map(toHivePartition) } override def listTables(dbName: String): Seq[String] = withHiveState { - client.getAllTables + client.getAllTables(dbName) } /** @@ -267,11 +307,12 @@ class ClientWrapper( try { val cmd_trimmed: String = cmd.trim() val tokens: Array[String] = cmd_trimmed.split("\\s+") + // The remainder of the command. val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() val proc: CommandProcessor = version match { case hive.v12 => classOf[CommandProcessorFactory] - .callStatic[String, HiveConf, CommandProcessor]("get", cmd_1, conf) + .callStatic[String, HiveConf, CommandProcessor]("get", tokens(0), conf) case hive.v13 => classOf[CommandProcessorFactory] .callStatic[Array[String], HiveConf, CommandProcessor]("get", Array(tokens(0)), conf) @@ -294,7 +335,7 @@ class ClientWrapper( res.toSeq case hive.v13 => val res = new JArrayList[Object] - driver.call[JArrayList[Object], Boolean]("getResults", res) + driver.call[JList[Object], Boolean]("getResults", res) res.map { r => r match { case s: String => s diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 710dbca6e3c66..7f94c93ba49c1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive.client import java.io.File -import java.net.URLClassLoader +import java.net.{URL, URLClassLoader} import java.util import scala.language.reflectiveCalls @@ -30,9 +30,10 @@ import org.apache.spark.Logging import org.apache.spark.deploy.SparkSubmitUtils import org.apache.spark.sql.catalyst.util.quietly +import org.apache.spark.sql.hive.HiveContext /** Factory for `IsolatedClientLoader` with specific versions of hive. */ -object IsolatedClientLoader { +private[hive] object IsolatedClientLoader { /** * Creates isolated Hive client loaders by downloading the requested version from maven. */ @@ -49,7 +50,7 @@ object IsolatedClientLoader { case "13" | "0.13" | "0.13.0" | "0.13.1" => hive.v13 } - private def downloadVersion(version: HiveVersion): Seq[File] = { + private def downloadVersion(version: HiveVersion): Seq[URL] = { val hiveArtifacts = (Seq("hive-metastore", "hive-exec", "hive-common", "hive-serde") ++ (if (version.hasBuiltinsJar) "hive-builtins" :: Nil else Nil)) @@ -72,10 +73,10 @@ object IsolatedClientLoader { tempDir.mkdir() allFiles.foreach(f => FileUtils.copyFileToDirectory(f, tempDir)) - tempDir.listFiles() + tempDir.listFiles().map(_.toURL) } - private def resolvedVersions = new scala.collection.mutable.HashMap[HiveVersion, Seq[File]] + private def resolvedVersions = new scala.collection.mutable.HashMap[HiveVersion, Seq[URL]] } /** @@ -99,9 +100,9 @@ object IsolatedClientLoader { * @param baseClassLoader The spark classloader that is used to load shared classes. * */ -class IsolatedClientLoader( +private[hive] class IsolatedClientLoader( val version: HiveVersion, - val execJars: Seq[File] = Seq.empty, + val execJars: Seq[URL] = Seq.empty, val config: Map[String, String] = Map.empty, val isolationOn: Boolean = true, val rootClassLoader: ClassLoader = ClassLoader.getSystemClassLoader.getParent.getParent, @@ -112,7 +113,7 @@ class IsolatedClientLoader( assert(Try(baseClassLoader.loadClass("org.apache.hive.HiveConf")).isFailure) /** All jars used by the hive specific classloader. */ - protected def allJars = execJars.map(_.toURI.toURL).toArray + protected def allJars = execJars.toArray protected def isSharedClass(name: String): Boolean = name.contains("slf4j") || @@ -166,6 +167,12 @@ class IsolatedClientLoader( .getConstructors.head .newInstance(version, config) .asInstanceOf[ClientInterface] + } catch { + case ReflectionException(cnf: NoClassDefFoundError) => + throw new ClassNotFoundException( + s"$cnf when creating Hive client using classpath: ${execJars.mkString(", ")}\n" + + "Please make sure that jars for your version of hive and hadoop are included in the " + + s"paths passed to ${HiveContext.HIVE_METASTORE_JARS}.") } finally { Thread.currentThread.setContextClassLoader(baseClassLoader) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala index 90d03049356b5..c600b158c5460 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala @@ -19,6 +19,14 @@ package org.apache.spark.sql.hive.client import scala.reflect._ +/** Unwraps reflection exceptions. */ +private[client] object ReflectionException { + def unapply(a: Throwable): Option[Throwable] = a match { + case ite: java.lang.reflect.InvocationTargetException => Option(ite.getCause) + case _ => None + } +} + /** * Provides implicit functions on any object for calling methods reflectively. */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index 76a1965f3cb25..91e6ac4032204 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -24,8 +24,8 @@ import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} import org.apache.spark.sql.execution.RunnableCommand -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.hive.MetastoreRelation +import org.apache.spark.sql.hive.client.{HiveTable, HiveColumn} +import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation, HiveMetastoreTypes} /** * Create table and insert the query result into it. @@ -39,17 +39,34 @@ import org.apache.spark.sql.hive.MetastoreRelation */ private[hive] case class CreateTableAsSelect( - database: String, - tableName: String, + tableDesc: HiveTable, query: LogicalPlan, - allowExisting: Boolean, - desc: Option[CreateTableDesc]) extends RunnableCommand { + allowExisting: Boolean) + extends RunnableCommand { + + def database: String = tableDesc.database + def tableName: String = tableDesc.name override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] lazy val metastoreRelation: MetastoreRelation = { - // Create Hive Table - hiveContext.catalog.createTable(database, tableName, query.output, allowExisting, desc) + import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe + import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + import org.apache.hadoop.io.Text + import org.apache.hadoop.mapred.TextInputFormat + + val withSchema = + tableDesc.copy( + schema = + query.output.map(c => + HiveColumn(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType), null)), + inputFormat = + tableDesc.inputFormat.orElse(Some(classOf[TextInputFormat].getName)), + outputFormat = + tableDesc.outputFormat + .orElse(Some(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]].getName)), + serde = tableDesc.serde.orElse(Some(classOf[LazySimpleSerDe].getName()))) + hiveContext.catalog.client.createTable(withSchema) // Get the Metastore Relation hiveContext.catalog.lookupRelation(Seq(database, tableName), None) match { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 89995a91b1a92..de8954d5dec99 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -200,9 +200,7 @@ case class InsertIntoHiveTable( orderedPartitionSpec.put(entry.getName,partitionSpec.get(entry.getName).getOrElse("")) } val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols, partitionSpec) - catalog.synchronized { - catalog.client.validatePartitionNameCharacters(partVals) - } + // inheritTableSpecs is set to true. It should be set to false for a IMPORT query // which is currently considered as a Hive native command. val inheritTableSpecs = true @@ -211,7 +209,7 @@ case class InsertIntoHiveTable( if (numDynamicPartitions > 0) { catalog.synchronized { catalog.client.loadDynamicPartitions( - outputPath, + outputPath.toString, qualifiedTableName, orderedPartitionSpec, overwrite, @@ -224,31 +222,28 @@ case class InsertIntoHiveTable( // ifNotExists is only valid with static partition, refer to // https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DML#LanguageManualDML-InsertingdataintoHiveTablesfromqueries // scalastyle:on - val oldPart = catalog.synchronized { - catalog.client.getPartition( - catalog.client.getTable(qualifiedTableName), partitionSpec, false) - } - if (oldPart == null || !ifNotExists) { - catalog.synchronized { + val oldPart = + catalog.client.getPartitionOption( + catalog.client.getTable(table.databaseName, table.tableName), + partitionSpec) + + if (oldPart.isEmpty || !ifNotExists) { catalog.client.loadPartition( - outputPath, + outputPath.toString, qualifiedTableName, orderedPartitionSpec, overwrite, holdDDLTime, inheritTableSpecs, isSkewedStoreAsSubdir) - } } } } else { - catalog.synchronized { - catalog.client.loadTable( - outputPath, - qualifiedTableName, - overwrite, - holdDDLTime) - } + catalog.client.loadTable( + outputPath.toString, // TODO: URI + qualifiedTableName, + overwrite, + holdDDLTime) } // Invalidate the cache. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index a40a1e53117cd..abab1a223a43a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils /** * Analyzes the given table in the current database to generate statistics, which will be @@ -84,8 +85,20 @@ case class AddJar(path: String) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] + val currentClassLoader = Utils.getContextOrSparkClassLoader + + // Add jar to current context + val jarURL = new java.io.File(path).toURL + val newClassLoader = new java.net.URLClassLoader(Array(jarURL), currentClassLoader) + Thread.currentThread.setContextClassLoader(newClassLoader) + org.apache.hadoop.hive.ql.metadata.Hive.get().getConf().setClassLoader(newClassLoader) + + // Add jar to isolated hive classloader hiveContext.runSqlHive(s"ADD JAR $path") + + // Add jar to executors hiveContext.sparkContext.addJar(path) + Seq(Row(0)) } } 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 4b6f0ad75f54f..fd0b6f058595d 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,27 +17,27 @@ package org.apache.spark.sql.hive +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper +import org.apache.spark.sql.AnalysisException import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ConstantObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory -import org.apache.hadoop.hive.ql.exec.{UDF, UDAF} -import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} +import org.apache.hadoop.hive.ql.exec._ import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic._ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ import org.apache.spark.Logging import org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{Generate, Project, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.types._ -import org.apache.spark.sql.catalyst.analysis.MultiAlias -import org.apache.spark.sql.catalyst.errors.TreeNodeException /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -189,6 +189,219 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr } } +/** + * Resolves [[UnresolvedWindowFunction]] to [[HiveWindowFunction]]. + */ +private[spark] object ResolveHiveWindowFunction extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { + case p: LogicalPlan if !p.childrenResolved => p + + // We are resolving WindowExpressions at here. When we get here, we have already + // replaced those WindowSpecReferences. + case p: LogicalPlan => + p transformExpressions { + case WindowExpression( + UnresolvedWindowFunction(name, children), + windowSpec: WindowSpecDefinition) => + // First, let's find the window function info. + val windowFunctionInfo: WindowFunctionInfo = + Option(FunctionRegistry.getWindowFunctionInfo(name.toLowerCase)).getOrElse( + throw new AnalysisException(s"Couldn't find window function $name")) + + // Get the class of this function. + // In Hive 0.12, there is no windowFunctionInfo.getFunctionClass. So, we use + // windowFunctionInfo.getfInfo().getFunctionClass for both Hive 0.13 and Hive 0.13.1. + val functionClass = windowFunctionInfo.getfInfo().getFunctionClass + val newChildren = + // Rank(), DENSE_RANK(), CUME_DIST(), and PERCENT_RANK() do not take explicit + // input parameters and requires implicit parameters, which + // are expressions in Order By clause. + if (classOf[GenericUDAFRank].isAssignableFrom(functionClass)) { + if (children.nonEmpty) { + throw new AnalysisException(s"$name does not take input parameters.") + } + windowSpec.orderSpec.map(_.child) + } else { + children + } + + // If the class is UDAF, we need to use UDAFBridge. + val isUDAFBridgeRequired = + if (classOf[UDAF].isAssignableFrom(functionClass)) { + true + } else { + false + } + + // Create the HiveWindowFunction. For the meaning of isPivotResult, see the doc of + // HiveWindowFunction. + val windowFunction = + HiveWindowFunction( + new HiveFunctionWrapper(functionClass.getName), + windowFunctionInfo.isPivotResult, + isUDAFBridgeRequired, + newChildren) + + // Second, check if the specified window function can accept window definition. + windowSpec.frameSpecification match { + case frame: SpecifiedWindowFrame if !windowFunctionInfo.isSupportsWindow => + // This Hive window function does not support user-speficied window frame. + throw new AnalysisException( + s"Window function $name does not take a frame specification.") + case frame: SpecifiedWindowFrame if windowFunctionInfo.isSupportsWindow && + windowFunctionInfo.isPivotResult => + // These two should not be true at the same time when a window frame is defined. + // If so, throw an exception. + throw new AnalysisException(s"Could not handle Hive window function $name because " + + s"it supports both a user specified window frame and pivot result.") + case _ => // OK + } + // Resolve those UnspecifiedWindowFrame because the physical Window operator still needs + // a window frame specification to work. + val newWindowSpec = windowSpec.frameSpecification match { + case UnspecifiedFrame => + val newWindowFrame = + SpecifiedWindowFrame.defaultWindowFrame( + windowSpec.orderSpec.nonEmpty, + windowFunctionInfo.isSupportsWindow) + WindowSpecDefinition(windowSpec.partitionSpec, windowSpec.orderSpec, newWindowFrame) + case _ => windowSpec + } + + // Finally, we create a WindowExpression with the resolved window function and + // specified window spec. + WindowExpression(windowFunction, newWindowSpec) + } + } +} + +/** + * A [[WindowFunction]] implementation wrapping Hive's window function. + * @param funcWrapper The wrapper for the Hive Window Function. + * @param pivotResult If it is true, the Hive function will return a list of values representing + * the values of the added columns. Otherwise, a single value is returned for + * current row. + * @param isUDAFBridgeRequired If it is true, the function returned by functionWrapper's + * createFunction is UDAF, we need to use GenericUDAFBridge to wrap + * it as a GenericUDAFResolver2. + * @param children Input parameters. + */ +private[hive] case class HiveWindowFunction( + funcWrapper: HiveFunctionWrapper, + pivotResult: Boolean, + isUDAFBridgeRequired: Boolean, + children: Seq[Expression]) extends WindowFunction + with HiveInspectors { + + // Hive window functions are based on GenericUDAFResolver2. + type UDFType = GenericUDAFResolver2 + + @transient + protected lazy val resolver: GenericUDAFResolver2 = + if (isUDAFBridgeRequired) { + new GenericUDAFBridge(funcWrapper.createFunction[UDAF]()) + } else { + funcWrapper.createFunction[GenericUDAFResolver2]() + } + + @transient + protected lazy val inputInspectors = children.map(toInspector).toArray + + // The GenericUDAFEvaluator used to evaluate the window function. + @transient + protected lazy val evaluator: GenericUDAFEvaluator = { + val parameterInfo = new SimpleGenericUDAFParameterInfo(inputInspectors, false, false) + resolver.getEvaluator(parameterInfo) + } + + // The object inspector of values returned from the Hive window function. + @transient + protected lazy val returnInspector = { + evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inputInspectors) + } + + def dataType: DataType = + if (!pivotResult) { + inspectorToDataType(returnInspector) + } else { + // If pivotResult is true, we should take the element type out as the data type of this + // function. + inspectorToDataType(returnInspector) match { + case ArrayType(dt, _) => dt + case _ => + sys.error( + s"error resolve the data type of window function ${funcWrapper.functionClassName}") + } + } + + def nullable: Boolean = true + + override type EvaluatedType = Any + + override def eval(input: Row): Any = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + + @transient + lazy val inputProjection = new InterpretedProjection(children) + + @transient + private var hiveEvaluatorBuffer: AggregationBuffer = _ + // Output buffer. + private var outputBuffer: Any = _ + + override def init(): Unit = { + evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inputInspectors) + } + + // Reset the hiveEvaluatorBuffer and outputPosition + override def reset(): Unit = { + // We create a new aggregation buffer to workaround the bug in GenericUDAFRowNumber. + // Basically, GenericUDAFRowNumberEvaluator.reset calls RowNumberBuffer.init. + // However, RowNumberBuffer.init does not really reset this buffer. + hiveEvaluatorBuffer = evaluator.getNewAggregationBuffer + evaluator.reset(hiveEvaluatorBuffer) + } + + override def prepareInputParameters(input: Row): AnyRef = { + wrap(inputProjection(input), inputInspectors, new Array[AnyRef](children.length)) + } + // Add input parameters for a single row. + override def update(input: AnyRef): Unit = { + evaluator.iterate(hiveEvaluatorBuffer, input.asInstanceOf[Array[AnyRef]]) + } + + override def batchUpdate(inputs: Array[AnyRef]): Unit = { + var i = 0 + while (i < inputs.length) { + evaluator.iterate(hiveEvaluatorBuffer, inputs(i).asInstanceOf[Array[AnyRef]]) + i += 1 + } + } + + override def evaluate(): Unit = { + outputBuffer = unwrap(evaluator.evaluate(hiveEvaluatorBuffer), returnInspector) + } + + override def get(index: Int): Any = { + if (!pivotResult) { + // if pivotResult is false, we will get a single value for all rows in the frame. + outputBuffer + } else { + // if pivotResult is true, we will get a Seq having the same size with the size + // of the window frame. At here, we will return the result at the position of + // index in the output buffer. + outputBuffer.asInstanceOf[Seq[Any]].get(index) + } + } + + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } + + override def newInstance: WindowFunction = + new HiveWindowFunction(funcWrapper, pivotResult, isUDAFBridgeRequired, children) +} + private[hive] case class HiveGenericUdaf( funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) extends AggregateExpression diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index edeab5158df62..1f40a5340c2ce 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.test import java.io.File import java.util.{Set => JavaSet} +import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.FunctionRegistry import org.apache.hadoop.hive.ql.io.avro.{AvroContainerInputFormat, AvroContainerOutputFormat} import org.apache.hadoop.hive.ql.metadata.Table @@ -62,6 +63,8 @@ object TestHive class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { self => + import HiveContext._ + // By clearing the port we force Spark to pick a new one. This allows us to rerun tests // without restarting the JVM. System.clearProperty("spark.hostPort") @@ -70,31 +73,24 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { hiveconf.set("hive.plan.serialization.format", "javaXML") lazy val warehousePath = Utils.createTempDir() - lazy val metastorePath = Utils.createTempDir() /** Sets up the system initially or after a RESET command */ - protected def configure(): Unit = { - warehousePath.delete() - metastorePath.delete() - setConf("javax.jdo.option.ConnectionURL", - s"jdbc:derby:;databaseName=$metastorePath;create=true") - setConf("hive.metastore.warehouse.dir", warehousePath.toString) - } + protected override def configure(): Map[String, String] = + newTemporaryConfiguration() ++ Map("hive.metastore.warehouse.dir" -> warehousePath.toString) val testTempDir = Utils.createTempDir() // For some hive test case which contain ${system:test.tmp.dir} System.setProperty("test.tmp.dir", testTempDir.getCanonicalPath) - configure() // Must be called before initializing the catalog below. - /** The location of the compiled hive distribution */ lazy val hiveHome = envVarToFile("HIVE_HOME") /** The location of the hive source code. */ lazy val hiveDevHome = envVarToFile("HIVE_DEV_HOME") // Override so we can intercept relative paths and rewrite them to point at hive. - override def runSqlHive(sql: String): Seq[String] = super.runSqlHive(rewritePaths(sql)) + override def runSqlHive(sql: String): Seq[String] = + super.runSqlHive(rewritePaths(substitutor.substitute(this.hiveconf, sql))) override def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution(plan) @@ -157,22 +153,12 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { val describedTable = "DESCRIBE (\\w+)".r - val vs = new VariableSubstitution() - - // we should substitute variables in hql to pass the text to parseSql() as a parameter. - // Hive parser need substituted text. HiveContext.sql() does this but return a DataFrame, - // while we need a logicalPlan so we cannot reuse that. - protected[hive] class HiveQLQueryExecution(hql: String) - extends this.QueryExecution(HiveQl.parseSql(vs.substitute(hiveconf, hql))) { - def hiveExec(): Seq[String] = runSqlHive(hql) - override def toString: String = hql + "\n" + super.toString - } - /** * Override QueryExecution with special debug workflow. */ class QueryExecution(logicalPlan: LogicalPlan) extends super.QueryExecution(logicalPlan) { + def this(sql: String) = this(parseSql(sql)) override lazy val analyzed = { val describedTables = logical match { case HiveNativeCommand(describedTable(tbl)) => tbl :: Nil @@ -196,7 +182,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { protected[hive] implicit class SqlCmd(sql: String) { def cmd: () => Unit = { - () => new HiveQLQueryExecution(sql).stringResult(): Unit + () => new QueryExecution(sql).stringResult(): Unit } } @@ -204,6 +190,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { * A list of test tables and the DDL required to initialize them. A test table is loaded on * demand when a query are run against it. */ + @transient lazy val testTables = new mutable.HashMap[String, TestTable]() def registerTestTable(testTable: TestTable): Unit = { @@ -213,6 +200,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { // The test tables that are defined in the Hive QTestUtil. // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java // https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql + @transient val hiveQTestUtilTables = Seq( TestTable("src", "CREATE TABLE src (key INT, value STRING)".cmd, @@ -245,16 +233,18 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { import org.apache.hadoop.mapred.{SequenceFileInputFormat, SequenceFileOutputFormat} import org.apache.thrift.protocol.TBinaryProtocol - val srcThrift = new Table("default", "src_thrift") - srcThrift.setFields(Nil) - srcThrift.setInputFormatClass(classOf[SequenceFileInputFormat[_,_]].getName) - // In Hive, SequenceFileOutputFormat will be substituted by HiveSequenceFileOutputFormat. - srcThrift.setOutputFormatClass(classOf[SequenceFileOutputFormat[_,_]].getName) - srcThrift.setSerializationLib(classOf[ThriftDeserializer].getName) - srcThrift.setSerdeParam("serialization.class", classOf[Complex].getName) - srcThrift.setSerdeParam("serialization.format", classOf[TBinaryProtocol].getName) - catalog.client.createTable(srcThrift) - + runSqlHive( + s""" + |CREATE TABLE src_thrift(fake INT) + |ROW FORMAT SERDE '${classOf[ThriftDeserializer].getName}' + |WITH SERDEPROPERTIES( + | 'serialization.class'='${classOf[Complex].getName}', + | 'serialization.format'='${classOf[TBinaryProtocol].getName}' + |) + |STORED AS + |INPUTFORMAT '${classOf[SequenceFileInputFormat[_,_]].getName}' + |OUTPUTFORMAT '${classOf[SequenceFileOutputFormat[_,_]].getName}' + """.stripMargin) runSqlHive( s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/complex.seq")}' INTO TABLE src_thrift") @@ -376,7 +366,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { if (!(loadedTables contains name)) { // Marks the table as loaded first to prevent infinite mutually recursive table loading. loadedTables += name - logInfo(s"Loading test table $name") + logDebug(s"Loading test table $name") val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) createCmds.foreach(_()) @@ -393,9 +383,6 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { */ protected val originalUdfs: JavaSet[String] = FunctionRegistry.getFunctionNames - // Database default may not exist in 0.13.1, create it if not exist - HiveShim.createDefaultDBIfNeeded(this) - /** * Resets the test instance by deleting any tables that have been created. * TODO: also clear out UDFs, views, etc. @@ -410,24 +397,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { cacheManager.clearCache() loadedTables.clear() catalog.cachedDataSourceTables.invalidateAll() - catalog.client.getAllTables("default").foreach { t => - logDebug(s"Deleting table $t") - val table = catalog.client.getTable("default", t) - - catalog.client.getIndexes("default", t, 255).foreach { index => - catalog.client.dropIndex("default", t, index.getIndexName, true) - } - - if (!table.isIndexTable) { - catalog.client.dropTable("default", t) - } - } - - catalog.client.getAllDatabases.filterNot(_ == "default").foreach { db => - logDebug(s"Dropping Database: $db") - catalog.client.dropDatabase(db, true, false, true) - } - + catalog.client.reset() catalog.unregisterAllTables() FunctionRegistry.getFunctionNames.filterNot(originalUdfs.contains(_)).foreach { udfName => @@ -438,7 +408,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { hiveconf.set("fs.default.name", new File(".").toURI.toString) // It is important that we RESET first as broken hooks that might have been set could break // other sql exec here. - runSqlHive("RESET") + executionHive.runSqlHive("RESET") + metadataHive.runSqlHive("RESET") // For some reason, RESET does not reset the following variables... // https://issues.apache.org/jira/browse/HIVE-9004 runSqlHive("set hive.table.parameters.default=") @@ -446,7 +417,11 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { runSqlHive("set datanucleus.cache.collections.lazy=true") // Lots of tests fail if we do not change the partition whitelist from the default. runSqlHive("set hive.metastore.partition.name.whitelist.pattern=.*") - configure() + + configure().foreach { + case (k, v) => + metadataHive.runSqlHive(s"SET $k=$v") + } runSqlHive("USE default") diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 1. testWindowing-0-327a8cd39fe30255ff492ee86f660522 b/sql/hive/src/test/resources/golden/windowing.q -- 1. testWindowing-0-327a8cd39fe30255ff492ee86f660522 new file mode 100644 index 0000000000000..850c41c8115d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 1. testWindowing-0-327a8cd39fe30255ff492ee86f660522 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 10. testHavingWithWindowingCondRankNoGBY-0-fef4bf638d52a9a601845347010602fd b/sql/hive/src/test/resources/golden/windowing.q -- 10. testHavingWithWindowingCondRankNoGBY-0-fef4bf638d52a9a601845347010602fd new file mode 100644 index 0000000000000..850c41c8115d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 10. testHavingWithWindowingCondRankNoGBY-0-fef4bf638d52a9a601845347010602fd @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 11. testFirstLast-0-86bb9c97d92fdcd941bcb5143513e2e6 b/sql/hive/src/test/resources/golden/windowing.q -- 11. testFirstLast-0-86bb9c97d92fdcd941bcb5143513e2e6 new file mode 100644 index 0000000000000..921679cdcf569 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 11. testFirstLast-0-86bb9c97d92fdcd941bcb5143513e2e6 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2 2 34 +Manufacturer#1 almond antique burnished rose metallic 2 2 2 6 +Manufacturer#1 almond antique chartreuse lavender yellow 34 34 2 28 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 6 2 42 +Manufacturer#1 almond aquamarine burnished black steel 28 28 34 42 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 42 6 42 +Manufacturer#2 almond antique violet chocolate turquoise 14 14 14 2 +Manufacturer#2 almond antique violet turquoise frosted 40 40 14 25 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 14 18 +Manufacturer#2 almond aquamarine rose maroon antique 25 25 40 18 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 18 2 18 +Manufacturer#3 almond antique chartreuse khaki white 17 17 17 19 +Manufacturer#3 almond antique forest lavender goldenrod 14 14 17 1 +Manufacturer#3 almond antique metallic orange dim 19 19 17 45 +Manufacturer#3 almond antique misty red olive 1 1 14 45 +Manufacturer#3 almond antique olive coral navajo 45 45 19 45 +Manufacturer#4 almond antique gainsboro frosted violet 10 10 10 27 +Manufacturer#4 almond antique violet mint lemon 39 39 10 7 +Manufacturer#4 almond aquamarine floral ivory bisque 27 27 10 12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7 39 12 +Manufacturer#4 almond azure aquamarine papaya violet 12 12 27 12 +Manufacturer#5 almond antique blue firebrick mint 31 31 31 2 +Manufacturer#5 almond antique medium spring khaki 6 6 31 46 +Manufacturer#5 almond antique sky peru orange 2 2 31 23 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 46 6 23 +Manufacturer#5 almond azure blanched chiffon midnight 23 23 2 23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 12. testFirstLastWithWhere-0-84345a9f685ba63b87caa4bb16b122b5 b/sql/hive/src/test/resources/golden/windowing.q -- 12. testFirstLastWithWhere-0-84345a9f685ba63b87caa4bb16b122b5 new file mode 100644 index 0000000000000..09e30c7c57349 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 12. testFirstLastWithWhere-0-84345a9f685ba63b87caa4bb16b122b5 @@ -0,0 +1,5 @@ +Manufacturer#3 almond antique chartreuse khaki white 17 1 17 17 19 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 14 17 1 +Manufacturer#3 almond antique metallic orange dim 19 3 19 17 45 +Manufacturer#3 almond antique misty red olive 1 4 1 14 45 +Manufacturer#3 almond antique olive coral navajo 45 5 45 19 45 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 13. testSumWindow-0-6cfc8840d3a4469b0fe11d63182cb59f b/sql/hive/src/test/resources/golden/windowing.q -- 13. testSumWindow-0-6cfc8840d3a4469b0fe11d63182cb59f new file mode 100644 index 0000000000000..01ee88ff23302 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 13. testSumWindow-0-6cfc8840d3a4469b0fe11d63182cb59f @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 38 2 +Manufacturer#1 almond antique burnished rose metallic 2 44 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 34 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 6 +Manufacturer#1 almond aquamarine burnished black steel 28 110 28 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 42 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 14 +Manufacturer#2 almond antique violet turquoise frosted 40 81 40 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 2 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 25 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 18 +Manufacturer#3 almond antique chartreuse khaki white 17 50 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 14 +Manufacturer#3 almond antique metallic orange dim 19 96 19 +Manufacturer#3 almond antique misty red olive 1 79 1 +Manufacturer#3 almond antique olive coral navajo 45 65 45 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 10 +Manufacturer#4 almond antique violet mint lemon 39 83 39 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 27 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 7 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 12 +Manufacturer#5 almond antique blue firebrick mint 31 39 31 +Manufacturer#5 almond antique medium spring khaki 6 85 6 +Manufacturer#5 almond antique sky peru orange 2 108 2 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 46 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 14. testNoSortClause-0-88d96a526d3cae6ed8168c5b228974d1 b/sql/hive/src/test/resources/golden/windowing.q -- 14. testNoSortClause-0-88d96a526d3cae6ed8168c5b228974d1 new file mode 100644 index 0000000000000..c78eb640c9c27 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 14. testNoSortClause-0-88d96a526d3cae6ed8168c5b228974d1 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 +Manufacturer#3 almond antique metallic orange dim 19 3 3 +Manufacturer#3 almond antique misty red olive 1 4 4 +Manufacturer#3 almond antique olive coral navajo 45 5 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 +Manufacturer#4 almond antique violet mint lemon 39 2 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 +Manufacturer#5 almond antique medium spring khaki 6 2 2 +Manufacturer#5 almond antique sky peru orange 2 3 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 15. testExpressions-0-11f6c13cf2710ce7054654cca136e73e b/sql/hive/src/test/resources/golden/windowing.q -- 15. testExpressions-0-11f6c13cf2710ce7054654cca136e73e new file mode 100644 index 0000000000000..050138ccf04ce --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 15. testExpressions-0-11f6c13cf2710ce7054654cca136e73e @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 0.3333333333333333 0.0 1 2 2.0 0.0 2 2 2 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 0.3333333333333333 0.0 1 2 2.0 0.0 2 2 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 0.5 0.4 2 3 12.666666666666666 15.084944665313014 2 34 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 0.6666666666666666 0.6 2 4 11.0 13.379088160259652 2 6 2 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 0.8333333333333334 0.8 3 5 14.4 13.763720427268202 2 28 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 1.0 1.0 3 6 19.0 16.237815945091466 2 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 0.2 0.0 1 1 14.0 0.0 4 14 14 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 0.4 0.25 1 2 27.0 13.0 4 40 14 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 0.6 0.5 2 3 18.666666666666668 15.86050300449376 4 2 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 0.8 0.75 2 4 20.25 14.00669482783144 4 25 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 1.0 1.0 3 5 19.8 12.560254774486067 4 18 2 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 0.2 0.0 1 1 17.0 0.0 2 17 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 0.4 0.25 1 2 15.5 1.5 2 14 17 +Manufacturer#3 almond antique metallic orange dim 19 3 3 0.6 0.5 2 3 16.666666666666668 2.0548046676563256 2 19 17 +Manufacturer#3 almond antique misty red olive 1 4 4 0.8 0.75 2 4 12.75 7.013380069552769 2 1 14 +Manufacturer#3 almond antique olive coral navajo 45 5 5 1.0 1.0 3 5 19.2 14.344336861632886 2 45 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 0.2 0.0 1 1 10.0 0.0 0 10 10 +Manufacturer#4 almond antique violet mint lemon 39 2 2 0.4 0.25 1 2 24.5 14.5 0 39 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 0.6 0.5 2 3 25.333333333333332 11.897712198383164 0 27 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 0.8 0.75 2 4 20.75 13.007209539328564 0 7 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 1.0 1.0 3 5 19.0 12.149074038789951 0 12 27 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 0.2 0.0 1 1 31.0 0.0 1 31 31 +Manufacturer#5 almond antique medium spring khaki 6 2 2 0.4 0.25 1 2 18.5 12.5 1 6 31 +Manufacturer#5 almond antique sky peru orange 2 3 3 0.6 0.5 2 3 13.0 12.832251036613439 1 2 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 0.8 0.75 2 4 21.25 18.102140757380052 1 46 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 1.0 1.0 3 5 21.6 16.206171663906314 1 23 2 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 16. testMultipleWindows-0-efd1476255eeb1b1961149144f574b7a b/sql/hive/src/test/resources/golden/windowing.q -- 16. testMultipleWindows-0-efd1476255eeb1b1961149144f574b7a new file mode 100644 index 0000000000000..c10888852b504 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 16. testMultipleWindows-0-efd1476255eeb1b1961149144f574b7a @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 0.3333333333333333 4 4 2 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 0.3333333333333333 4 4 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 0.5 38 34 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 0.6666666666666666 44 10 2 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 0.8333333333333334 72 28 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 1.0 114 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 0.2 14 14 14 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 0.4 54 40 14 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 0.6 56 2 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 0.8 81 25 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 1.0 99 32 2 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 0.2 17 31 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 0.4 31 14 17 +Manufacturer#3 almond antique metallic orange dim 19 3 3 0.6 50 50 17 +Manufacturer#3 almond antique misty red olive 1 4 4 0.8 51 1 14 +Manufacturer#3 almond antique olive coral navajo 45 5 5 1.0 96 45 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 0.2 10 17 10 +Manufacturer#4 almond antique violet mint lemon 39 2 2 0.4 49 39 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 0.6 76 27 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 0.8 83 7 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 1.0 95 29 27 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 0.2 31 31 31 +Manufacturer#5 almond antique medium spring khaki 6 2 2 0.4 37 8 31 +Manufacturer#5 almond antique sky peru orange 2 3 3 0.6 39 2 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 0.8 85 46 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 1.0 108 23 2 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 17. testCountStar-0-1b1fc185c8fddf68e58e92f29052ab2d b/sql/hive/src/test/resources/golden/windowing.q -- 17. testCountStar-0-1b1fc185c8fddf68e58e92f29052ab2d new file mode 100644 index 0000000000000..b1309a497d68e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 17. testCountStar-0-1b1fc185c8fddf68e58e92f29052ab2d @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2 2 2 +Manufacturer#1 almond antique burnished rose metallic 2 2 2 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 3 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 4 2 +Manufacturer#1 almond aquamarine burnished black steel 28 5 5 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 6 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 14 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 14 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 2 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 17 +Manufacturer#3 almond antique metallic orange dim 19 3 3 17 +Manufacturer#3 almond antique misty red olive 1 4 4 14 +Manufacturer#3 almond antique olive coral navajo 45 5 5 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 10 +Manufacturer#4 almond antique violet mint lemon 39 2 2 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 27 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 31 +Manufacturer#5 almond antique medium spring khaki 6 2 2 31 +Manufacturer#5 almond antique sky peru orange 2 3 3 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 2 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 18. testUDAFs-0-6974e5959e41a661e09db18547fef58a b/sql/hive/src/test/resources/golden/windowing.q -- 18. testUDAFs-0-6974e5959e41a661e09db18547fef58a new file mode 100644 index 0000000000000..52d2ee8d0cd3f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 18. testUDAFs-0-6974e5959e41a661e09db18547fef58a @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 4100.06 1173.15 1753.76 1366.6866666666667 +Manufacturer#1 almond antique burnished rose metallic 2 5702.650000000001 1173.15 1753.76 1425.6625000000001 +Manufacturer#1 almond antique chartreuse lavender yellow 34 7117.070000000001 1173.15 1753.76 1423.4140000000002 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 7576.58 1173.15 1753.76 1515.316 +Manufacturer#1 almond aquamarine burnished black steel 28 6403.43 1414.42 1753.76 1600.8575 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 4649.67 1414.42 1632.66 1549.89 +Manufacturer#2 almond antique violet chocolate turquoise 14 5523.360000000001 1690.68 2031.98 1841.1200000000001 +Manufacturer#2 almond antique violet turquoise frosted 40 7222.02 1690.68 2031.98 1805.505 +Manufacturer#2 almond aquamarine midnight light salmon 2 8923.62 1690.68 2031.98 1784.7240000000002 +Manufacturer#2 almond aquamarine rose maroon antique 25 7232.9400000000005 1698.66 2031.98 1808.2350000000001 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5432.24 1698.66 2031.98 1810.7466666666667 +Manufacturer#3 almond antique chartreuse khaki white 17 4272.34 1190.27 1671.68 1424.1133333333335 +Manufacturer#3 almond antique forest lavender goldenrod 14 6195.32 1190.27 1922.98 1548.83 +Manufacturer#3 almond antique metallic orange dim 19 7532.61 1190.27 1922.98 1506.522 +Manufacturer#3 almond antique misty red olive 1 5860.929999999999 1190.27 1922.98 1465.2324999999998 +Manufacturer#3 almond antique olive coral navajo 45 4670.66 1337.29 1922.98 1556.8866666666665 +Manufacturer#4 almond antique gainsboro frosted violet 10 4202.35 1206.26 1620.67 1400.7833333333335 +Manufacturer#4 almond antique violet mint lemon 39 6047.27 1206.26 1844.92 1511.8175 +Manufacturer#4 almond aquamarine floral ivory bisque 27 7337.620000000001 1206.26 1844.92 1467.5240000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 5716.950000000001 1206.26 1844.92 1429.2375000000002 +Manufacturer#4 almond azure aquamarine papaya violet 12 4341.530000000001 1206.26 1844.92 1447.176666666667 +Manufacturer#5 almond antique blue firebrick mint 31 5190.08 1611.66 1789.69 1730.0266666666666 +Manufacturer#5 almond antique medium spring khaki 6 6208.18 1018.1 1789.69 1552.045 +Manufacturer#5 almond antique sky peru orange 2 7672.66 1018.1 1789.69 1534.532 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 5882.970000000001 1018.1 1788.73 1470.7425000000003 +Manufacturer#5 almond azure blanched chiffon midnight 23 4271.3099999999995 1018.1 1788.73 1423.7699999999998 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 19. testUDAFsWithGBY-0-67d15ee5915ac64a738fd4b60d75eb35 b/sql/hive/src/test/resources/golden/windowing.q -- 19. testUDAFsWithGBY-0-67d15ee5915ac64a738fd4b60d75eb35 new file mode 100644 index 0000000000000..6461642d34a21 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 19. testUDAFsWithGBY-0-67d15ee5915ac64a738fd4b60d75eb35 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 4529.5 1173.15 1173.15 1509.8333333333333 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 5943.92 1753.76 1753.76 1485.98 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 7576.58 1602.59 1602.59 1515.316 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 6403.43 1414.42 1414.42 1600.8575 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 4649.67 1632.66 1632.66 1549.89 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 5523.360000000001 1690.68 1690.68 1841.1200000000001 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 7222.02 1800.7 1800.7 1805.505 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 8923.62 2031.98 2031.98 1784.7240000000002 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 7232.9400000000005 1698.66 1698.66 1808.2350000000001 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5432.24 1701.6 1701.6 1810.7466666666667 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 4272.34 1671.68 1671.68 1424.1133333333335 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 6195.32 1190.27 1190.27 1548.83 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 7532.61 1410.39 1410.39 1506.522 +Manufacturer#3 almond antique misty red olive 1 1922.98 5860.929999999999 1922.98 1922.98 1465.2324999999998 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 4670.66 1337.29 1337.29 1556.8866666666665 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 4202.35 1620.67 1620.67 1400.7833333333335 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 6047.27 1375.42 1375.42 1511.8175 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 7337.620000000001 1206.26 1206.26 1467.5240000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 5716.950000000001 1844.92 1844.92 1429.2375000000002 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 4341.530000000001 1290.35 1290.35 1447.176666666667 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 5190.08 1789.69 1789.69 1730.0266666666666 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 6208.18 1611.66 1611.66 1552.045 +Manufacturer#5 almond antique sky peru orange 2 1788.73 7672.66 1788.73 1788.73 1534.532 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 5882.970000000001 1018.1 1018.1 1470.7425000000003 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 4271.3099999999995 1464.48 1464.48 1423.7699999999998 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 2. testGroupByWithPartitioning-0-cb5618b1e626f3a9d4a030b508b5d251 b/sql/hive/src/test/resources/golden/windowing.q -- 2. testGroupByWithPartitioning-0-cb5618b1e626f3a9d4a030b508b5d251 new file mode 100644 index 0000000000000..2c30e652aa26d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 2. testGroupByWithPartitioning-0-cb5618b1e626f3a9d4a030b508b5d251 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 2 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 3 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 4 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 5 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 1922.98 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 1788.73 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 5 5 23 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838 b/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838 new file mode 100644 index 0000000000000..1f7e8a5d67036 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 273.70217881648074 273.70217881648074 [34,2] 74912.8826888888 1.0 4128.782222222221 +Manufacturer#1 almond antique burnished rose metallic 2 258.10677784349235 258.10677784349235 [34,2,6] 66619.10876874991 0.811328754177887 2801.7074999999995 +Manufacturer#1 almond antique chartreuse lavender yellow 34 230.90151585470358 230.90151585470358 [34,2,6,28] 53315.51002399992 0.695639377397664 2210.7864 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 202.73109328368946 202.73109328368946 [34,2,6,42,28] 41099.896184 0.630785977101214 2009.9536000000007 +Manufacturer#1 almond aquamarine burnished black steel 28 121.6064517973862 121.6064517973862 [34,6,42,28] 14788.129118750014 0.2036684720435979 331.1337500000004 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 96.5751586416853 96.5751586416853 [6,42,28] 9326.761266666683 -1.4442181184933883E-4 -0.20666666666708502 +Manufacturer#2 almond antique violet chocolate turquoise 14 142.2363169751898 142.2363169751898 [2,40,14] 20231.169866666663 -0.49369526554523185 -1113.7466666666658 +Manufacturer#2 almond antique violet turquoise frosted 40 137.76306498840682 137.76306498840682 [2,25,40,14] 18978.662075 -0.5205630897335946 -1004.4812499999995 +Manufacturer#2 almond aquamarine midnight light salmon 2 130.03972279269132 130.03972279269132 [2,18,25,40,14] 16910.329504000005 -0.46908967495720255 -766.1791999999995 +Manufacturer#2 almond aquamarine rose maroon antique 25 135.55100986344584 135.55100986344584 [2,18,25,40] 18374.07627499999 -0.6091405874714462 -1128.1787499999987 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 156.44019460768044 156.44019460768044 [2,18,25] 24473.534488888927 -0.9571686373491608 -1441.4466666666676 +Manufacturer#3 almond antique chartreuse khaki white 17 196.7742266885805 196.7742266885805 [17,19,14] 38720.09628888887 0.5557168646224995 224.6944444444446 +Manufacturer#3 almond antique forest lavender goldenrod 14 275.14144189852607 275.14144189852607 [17,1,19,14] 75702.81305 -0.6720833036576083 -1296.9000000000003 +Manufacturer#3 almond antique metallic orange dim 19 260.23473614412046 260.23473614412046 [17,1,19,14,45] 67722.117896 -0.5703526513979519 -2129.0664 +Manufacturer#3 almond antique misty red olive 1 275.9139962356932 275.9139962356932 [1,19,14,45] 76128.53331875012 -0.577476899644802 -2547.7868749999993 +Manufacturer#3 almond antique olive coral navajo 45 260.5815918713796 260.5815918713796 [1,19,45] 67902.76602222225 -0.8710736366736884 -4099.731111111111 +Manufacturer#4 almond antique gainsboro frosted violet 10 170.13011889596618 170.13011889596618 [39,27,10] 28944.25735555559 -0.6656975320098423 -1347.4777777777779 +Manufacturer#4 almond antique violet mint lemon 39 242.26834609323197 242.26834609323197 [39,7,27,10] 58693.95151875002 -0.8051852719193339 -2537.328125 +Manufacturer#4 almond aquamarine floral ivory bisque 27 234.10001662537326 234.10001662537326 [39,7,27,10,12] 54802.817784000035 -0.6046935574240581 -1719.8079999999995 +Manufacturer#4 almond aquamarine yellow dodger mint 7 247.3342714197732 247.3342714197732 [39,7,27,12] 61174.24181875003 -0.5508665654707869 -1719.0368749999975 +Manufacturer#4 almond azure aquamarine papaya violet 12 283.3344330566893 283.3344330566893 [7,27,12] 80278.40095555557 -0.7755740084632333 -1867.4888888888881 +Manufacturer#5 almond antique blue firebrick mint 31 83.69879024746363 83.69879024746363 [2,6,31] 7005.487488888913 0.39004303087285047 418.9233333333353 +Manufacturer#5 almond antique medium spring khaki 6 316.68049612345885 316.68049612345885 [2,6,46,31] 100286.53662500004 -0.713612911776183 -4090.853749999999 +Manufacturer#5 almond antique sky peru orange 2 285.40506298242155 285.40506298242155 [2,23,6,46,31] 81456.04997600002 -0.712858514567818 -3297.2011999999986 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 285.43749038756283 285.43749038756283 [2,23,6,46] 81474.56091875004 -0.984128787153391 -4871.028125000002 +Manufacturer#5 almond azure blanched chiffon midnight 23 315.9225931564038 315.9225931564038 [2,23,46] 99807.08486666664 -0.9978877469246936 -5664.856666666666 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-672d4cb385b7ced2e446f132474293ad b/sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-672d4cb385b7ced2e446f132474293ad new file mode 100644 index 0000000000000..e7c39f454fb37 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-672d4cb385b7ced2e446f132474293ad @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 [{"x":1173.15,"y":2.0},{"x":1753.76,"y":1.0}] 121152.0 1 +Manufacturer#1 almond antique burnished rose metallic 2 [{"x":1173.15,"y":2.0},{"x":1602.59,"y":1.0},{"x":1753.76,"y":1.0}] 115872.0 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 [{"x":1173.15,"y":2.0},{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1753.76,"y":1.0}] 110592.0 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 [{"x":1173.15,"y":1.0},{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1632.66,"y":1.0},{"x":1753.76,"y":1.0}] 86428.0 4 +Manufacturer#1 almond aquamarine burnished black steel 28 [{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1632.66,"y":1.0},{"x":1753.76,"y":1.0}] 86098.0 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 [{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1632.66,"y":1.0}] 86428.0 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 [{"x":1690.68,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 146985.0 1 +Manufacturer#2 almond antique violet turquoise frosted 40 [{"x":1690.68,"y":1.0},{"x":1698.66,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 139825.5 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 [{"x":1690.68,"y":1.0},{"x":1698.66,"y":1.0},{"x":1701.6,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 146985.0 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 [{"x":1698.66,"y":1.0},{"x":1701.6,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 169347.0 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 [{"x":1698.66,"y":1.0},{"x":1701.6,"y":1.0},{"x":2031.98,"y":1.0}] 146985.0 5 +Manufacturer#3 almond antique chartreuse khaki white 17 [{"x":1190.27,"y":1.0},{"x":1410.39,"y":1.0},{"x":1671.68,"y":1.0}] 90681.0 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 [{"x":1190.27,"y":1.0},{"x":1410.39,"y":1.0},{"x":1671.68,"y":1.0},{"x":1922.98,"y":1.0}] 65831.5 2 +Manufacturer#3 almond antique metallic orange dim 19 [{"x":1190.27,"y":1.0},{"x":1337.29,"y":1.0},{"x":1410.39,"y":1.0},{"x":1671.68,"y":1.0},{"x":1922.98,"y":1.0}] 90681.0 3 +Manufacturer#3 almond antique misty red olive 1 [{"x":1190.27,"y":1.0},{"x":1337.29,"y":1.0},{"x":1410.39,"y":1.0},{"x":1922.98,"y":1.0}] 76690.0 4 +Manufacturer#3 almond antique olive coral navajo 45 [{"x":1337.29,"y":1.0},{"x":1410.39,"y":1.0},{"x":1922.98,"y":1.0}] 112398.0 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 [{"x":1206.26,"y":1.0},{"x":1375.42,"y":1.0},{"x":1620.67,"y":1.0}] 48427.0 1 +Manufacturer#4 almond antique violet mint lemon 39 [{"x":1206.26,"y":1.0},{"x":1375.42,"y":1.0},{"x":1620.67,"y":1.0},{"x":1844.92,"y":1.0}] 46844.0 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 [{"x":1206.26,"y":1.0},{"x":1290.35,"y":1.0},{"x":1375.42,"y":1.0},{"x":1620.67,"y":1.0},{"x":1844.92,"y":1.0}] 45261.0 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 [{"x":1206.26,"y":1.0},{"x":1290.35,"y":1.0},{"x":1375.42,"y":1.0},{"x":1844.92,"y":1.0}] 39309.0 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 [{"x":1206.26,"y":1.0},{"x":1290.35,"y":1.0},{"x":1844.92,"y":1.0}] 33357.0 5 +Manufacturer#5 almond antique blue firebrick mint 31 [{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0},{"x":1789.69,"y":1.0}] 155733.0 1 +Manufacturer#5 almond antique medium spring khaki 6 [{"x":1018.1,"y":1.0},{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0},{"x":1789.69,"y":1.0}] 99201.0 2 +Manufacturer#5 almond antique sky peru orange 2 [{"x":1018.1,"y":1.0},{"x":1464.48,"y":1.0},{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0},{"x":1789.69,"y":1.0}] 78486.0 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 [{"x":1018.1,"y":1.0},{"x":1464.48,"y":1.0},{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0}] 60577.5 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 [{"x":1018.1,"y":1.0},{"x":1464.48,"y":1.0},{"x":1788.73,"y":1.0}] 78486.0 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 24. testLateralViews-0-dea06072f0a64fe4537fae854944ed5a b/sql/hive/src/test/resources/golden/windowing.q -- 24. testLateralViews-0-dea06072f0a64fe4537fae854944ed5a new file mode 100644 index 0000000000000..dc83c9fffe932 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 24. testLateralViews-0-dea06072f0a64fe4537fae854944ed5a @@ -0,0 +1,78 @@ +Manufacturer#1 almond antique burnished rose metallic 1 2 2 +Manufacturer#1 almond antique burnished rose metallic 1 2 4 +Manufacturer#1 almond antique burnished rose metallic 2 2 6 +Manufacturer#1 almond antique burnished rose metallic 2 2 6 +Manufacturer#1 almond antique burnished rose metallic 3 2 6 +Manufacturer#1 almond antique burnished rose metallic 3 2 6 +Manufacturer#1 almond antique salmon chartreuse burlywood 1 6 10 +Manufacturer#1 almond antique salmon chartreuse burlywood 2 6 14 +Manufacturer#1 almond antique salmon chartreuse burlywood 3 6 18 +Manufacturer#1 almond aquamarine burnished black steel 1 28 40 +Manufacturer#1 almond aquamarine burnished black steel 2 28 62 +Manufacturer#1 almond aquamarine burnished black steel 3 28 84 +Manufacturer#1 almond antique chartreuse lavender yellow 1 34 90 +Manufacturer#1 almond antique chartreuse lavender yellow 2 34 96 +Manufacturer#1 almond antique chartreuse lavender yellow 3 34 102 +Manufacturer#1 almond aquamarine pink moccasin thistle 1 42 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 2 42 118 +Manufacturer#1 almond aquamarine pink moccasin thistle 3 42 126 +Manufacturer#2 almond aquamarine midnight light salmon 1 2 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 4 +Manufacturer#2 almond aquamarine midnight light salmon 3 2 6 +Manufacturer#2 almond antique violet chocolate turquoise 1 14 18 +Manufacturer#2 almond antique violet chocolate turquoise 2 14 30 +Manufacturer#2 almond antique violet chocolate turquoise 3 14 42 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 1 18 46 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 2 18 50 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 3 18 54 +Manufacturer#2 almond aquamarine rose maroon antique 1 25 61 +Manufacturer#2 almond aquamarine rose maroon antique 2 25 68 +Manufacturer#2 almond aquamarine rose maroon antique 3 25 75 +Manufacturer#2 almond antique violet turquoise frosted 1 40 90 +Manufacturer#2 almond antique violet turquoise frosted 2 40 105 +Manufacturer#2 almond antique violet turquoise frosted 3 40 120 +Manufacturer#3 almond antique misty red olive 1 1 1 +Manufacturer#3 almond antique misty red olive 2 1 2 +Manufacturer#3 almond antique misty red olive 3 1 3 +Manufacturer#3 almond antique forest lavender goldenrod 1 14 16 +Manufacturer#3 almond antique forest lavender goldenrod 2 14 29 +Manufacturer#3 almond antique forest lavender goldenrod 3 14 42 +Manufacturer#3 almond antique chartreuse khaki white 1 17 45 +Manufacturer#3 almond antique chartreuse khaki white 2 17 48 +Manufacturer#3 almond antique chartreuse khaki white 3 17 51 +Manufacturer#3 almond antique metallic orange dim 1 19 53 +Manufacturer#3 almond antique metallic orange dim 2 19 55 +Manufacturer#3 almond antique metallic orange dim 3 19 57 +Manufacturer#3 almond antique olive coral navajo 1 45 83 +Manufacturer#3 almond antique olive coral navajo 2 45 109 +Manufacturer#3 almond antique olive coral navajo 3 45 135 +Manufacturer#4 almond aquamarine yellow dodger mint 1 7 7 +Manufacturer#4 almond aquamarine yellow dodger mint 2 7 14 +Manufacturer#4 almond aquamarine yellow dodger mint 3 7 21 +Manufacturer#4 almond antique gainsboro frosted violet 1 10 24 +Manufacturer#4 almond antique gainsboro frosted violet 2 10 27 +Manufacturer#4 almond antique gainsboro frosted violet 3 10 30 +Manufacturer#4 almond azure aquamarine papaya violet 1 12 32 +Manufacturer#4 almond azure aquamarine papaya violet 2 12 34 +Manufacturer#4 almond azure aquamarine papaya violet 3 12 36 +Manufacturer#4 almond aquamarine floral ivory bisque 1 27 51 +Manufacturer#4 almond aquamarine floral ivory bisque 2 27 66 +Manufacturer#4 almond aquamarine floral ivory bisque 3 27 81 +Manufacturer#4 almond antique violet mint lemon 1 39 93 +Manufacturer#4 almond antique violet mint lemon 2 39 105 +Manufacturer#4 almond antique violet mint lemon 3 39 117 +Manufacturer#5 almond antique sky peru orange 1 2 2 +Manufacturer#5 almond antique sky peru orange 2 2 4 +Manufacturer#5 almond antique sky peru orange 3 2 6 +Manufacturer#5 almond antique medium spring khaki 1 6 10 +Manufacturer#5 almond antique medium spring khaki 2 6 14 +Manufacturer#5 almond antique medium spring khaki 3 6 18 +Manufacturer#5 almond azure blanched chiffon midnight 1 23 35 +Manufacturer#5 almond azure blanched chiffon midnight 2 23 52 +Manufacturer#5 almond azure blanched chiffon midnight 3 23 69 +Manufacturer#5 almond antique blue firebrick mint 1 31 77 +Manufacturer#5 almond antique blue firebrick mint 2 31 85 +Manufacturer#5 almond antique blue firebrick mint 3 31 93 +Manufacturer#5 almond aquamarine dodger light gainsboro 1 46 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 2 46 123 +Manufacturer#5 almond aquamarine dodger light gainsboro 3 46 138 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 26. testGroupByHavingWithSWQAndAlias-0-b996a664b06e5741c08079d5c38241bc b/sql/hive/src/test/resources/golden/windowing.q -- 26. testGroupByHavingWithSWQAndAlias-0-b996a664b06e5741c08079d5c38241bc new file mode 100644 index 0000000000000..2c30e652aa26d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 26. testGroupByHavingWithSWQAndAlias-0-b996a664b06e5741c08079d5c38241bc @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 2 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 3 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 4 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 5 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 1922.98 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 1788.73 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 5 5 23 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 27. testMultipleRangeWindows-0-227e080e337d734dd88ff814b3b412e4 b/sql/hive/src/test/resources/golden/windowing.q -- 27. testMultipleRangeWindows-0-227e080e337d734dd88ff814b3b412e4 new file mode 100644 index 0000000000000..b2a91ba727a72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 27. testMultipleRangeWindows-0-227e080e337d734dd88ff814b3b412e4 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 4 10 +Manufacturer#1 almond antique burnished rose metallic 2 4 10 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 10 6 +Manufacturer#1 almond aquamarine burnished black steel 28 28 62 +Manufacturer#1 almond antique chartreuse lavender yellow 34 62 76 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 42 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 2 +Manufacturer#2 almond antique violet chocolate turquoise 14 14 32 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 32 43 +Manufacturer#2 almond aquamarine rose maroon antique 25 43 25 +Manufacturer#2 almond antique violet turquoise frosted 40 40 40 +Manufacturer#3 almond antique misty red olive 1 1 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 14 50 +Manufacturer#3 almond antique chartreuse khaki white 17 31 36 +Manufacturer#3 almond antique metallic orange dim 19 50 19 +Manufacturer#3 almond antique olive coral navajo 45 45 45 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7 29 +Manufacturer#4 almond antique gainsboro frosted violet 10 17 22 +Manufacturer#4 almond azure aquamarine papaya violet 12 29 12 +Manufacturer#4 almond aquamarine floral ivory bisque 27 27 27 +Manufacturer#4 almond antique violet mint lemon 39 39 39 +Manufacturer#5 almond antique sky peru orange 2 2 8 +Manufacturer#5 almond antique medium spring khaki 6 8 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 23 54 +Manufacturer#5 almond antique blue firebrick mint 31 54 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 46 46 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 28. testPartOrderInUDAFInvoke-0-25912ae7d18c91cc09e17e57968fb5db b/sql/hive/src/test/resources/golden/windowing.q -- 28. testPartOrderInUDAFInvoke-0-25912ae7d18c91cc09e17e57968fb5db new file mode 100644 index 0000000000000..5bcb0fa941d65 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 28. testPartOrderInUDAFInvoke-0-25912ae7d18c91cc09e17e57968fb5db @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 38 +Manufacturer#1 almond antique burnished rose metallic 2 44 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 +Manufacturer#1 almond aquamarine burnished black steel 28 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 +Manufacturer#2 almond antique violet turquoise frosted 40 81 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 +Manufacturer#3 almond antique chartreuse khaki white 17 50 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 +Manufacturer#3 almond antique metallic orange dim 19 96 +Manufacturer#3 almond antique misty red olive 1 79 +Manufacturer#3 almond antique olive coral navajo 45 65 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 +Manufacturer#4 almond antique violet mint lemon 39 83 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 +Manufacturer#5 almond antique blue firebrick mint 31 39 +Manufacturer#5 almond antique medium spring khaki 6 85 +Manufacturer#5 almond antique sky peru orange 2 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 29. testPartOrderInWdwDef-0-88945892370ccbc1125a927a3d55342a b/sql/hive/src/test/resources/golden/windowing.q -- 29. testPartOrderInWdwDef-0-88945892370ccbc1125a927a3d55342a new file mode 100644 index 0000000000000..5bcb0fa941d65 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 29. testPartOrderInWdwDef-0-88945892370ccbc1125a927a3d55342a @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 38 +Manufacturer#1 almond antique burnished rose metallic 2 44 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 +Manufacturer#1 almond aquamarine burnished black steel 28 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 +Manufacturer#2 almond antique violet turquoise frosted 40 81 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 +Manufacturer#3 almond antique chartreuse khaki white 17 50 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 +Manufacturer#3 almond antique metallic orange dim 19 96 +Manufacturer#3 almond antique misty red olive 1 79 +Manufacturer#3 almond antique olive coral navajo 45 65 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 +Manufacturer#4 almond antique violet mint lemon 39 83 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 +Manufacturer#5 almond antique blue firebrick mint 31 39 +Manufacturer#5 almond antique medium spring khaki 6 85 +Manufacturer#5 almond antique sky peru orange 2 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 3. testGroupByHavingWithSWQ-0-a5a5339330a6a6660d32ccb0cc5d7100 b/sql/hive/src/test/resources/golden/windowing.q -- 3. testGroupByHavingWithSWQ-0-a5a5339330a6a6660d32ccb0cc5d7100 new file mode 100644 index 0000000000000..2c30e652aa26d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 3. testGroupByHavingWithSWQ-0-a5a5339330a6a6660d32ccb0cc5d7100 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 2 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 3 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 4 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 5 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 1922.98 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 1788.73 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 5 5 23 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 30. testDefaultPartitioningSpecRules-0-fa80b09c99e3c1487de48ea71a88dada b/sql/hive/src/test/resources/golden/windowing.q -- 30. testDefaultPartitioningSpecRules-0-fa80b09c99e3c1487de48ea71a88dada new file mode 100644 index 0000000000000..698a44349d2a6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 30. testDefaultPartitioningSpecRules-0-fa80b09c99e3c1487de48ea71a88dada @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 38 4 +Manufacturer#1 almond antique burnished rose metallic 2 44 4 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 38 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 44 +Manufacturer#1 almond aquamarine burnished black steel 28 110 72 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 114 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 14 +Manufacturer#2 almond antique violet turquoise frosted 40 81 54 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 56 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 81 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 99 +Manufacturer#3 almond antique chartreuse khaki white 17 50 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 31 +Manufacturer#3 almond antique metallic orange dim 19 96 50 +Manufacturer#3 almond antique misty red olive 1 79 51 +Manufacturer#3 almond antique olive coral navajo 45 65 96 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 10 +Manufacturer#4 almond antique violet mint lemon 39 83 49 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 76 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 83 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 95 +Manufacturer#5 almond antique blue firebrick mint 31 39 31 +Manufacturer#5 almond antique medium spring khaki 6 85 37 +Manufacturer#5 almond antique sky peru orange 2 108 39 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 85 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 108 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 36. testRankWithPartitioning-0-45ccbaf0ee083858f7661c66b11d4768 b/sql/hive/src/test/resources/golden/windowing.q -- 36. testRankWithPartitioning-0-45ccbaf0ee083858f7661c66b11d4768 new file mode 100644 index 0000000000000..e35257d98382e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 36. testRankWithPartitioning-0-45ccbaf0ee083858f7661c66b11d4768 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 +Manufacturer#1 almond aquamarine burnished black steel 28 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 +Manufacturer#2 almond antique violet turquoise frosted 40 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 +Manufacturer#3 almond antique chartreuse khaki white 17 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 +Manufacturer#3 almond antique metallic orange dim 19 3 +Manufacturer#3 almond antique misty red olive 1 4 +Manufacturer#3 almond antique olive coral navajo 45 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 +Manufacturer#4 almond antique violet mint lemon 39 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 +Manufacturer#5 almond antique medium spring khaki 6 2 +Manufacturer#5 almond antique sky peru orange 2 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 37. testPartitioningVariousForms-0-3436e50214f9afdec84334e10faa931a b/sql/hive/src/test/resources/golden/windowing.q -- 37. testPartitioningVariousForms-0-3436e50214f9afdec84334e10faa931a new file mode 100644 index 0000000000000..9c0ca6c7a00ba --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 37. testPartitioningVariousForms-0-3436e50214f9afdec84334e10faa931a @@ -0,0 +1,26 @@ +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 38. testPartitioningVariousForms2-0-cba9d84a6b1bb5e36595338d4602377e b/sql/hive/src/test/resources/golden/windowing.q -- 38. testPartitioningVariousForms2-0-cba9d84a6b1bb5e36595338d4602377e new file mode 100644 index 0000000000000..fc27df2f2b648 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 38. testPartitioningVariousForms2-0-cba9d84a6b1bb5e36595338d4602377e @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1173.15 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 1173.15 1173.15 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 1753.76 1753.76 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 1602.59 1602.59 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 1414.42 1414.42 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 1632.66 1632.66 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1690.68 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 1800.7 1800.7 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 2031.98 2031.98 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 1698.66 1698.66 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 1701.6 1701.6 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1671.68 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 1190.27 1190.27 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 1410.39 1410.39 +Manufacturer#3 almond antique misty red olive 1 1922.98 1922.98 1922.98 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 1337.29 1337.29 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1620.67 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 1375.42 1375.42 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 1206.26 1206.26 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 1844.92 1844.92 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 1290.35 1290.35 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1789.69 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 1611.66 1611.66 +Manufacturer#5 almond antique sky peru orange 2 1788.73 1788.73 1788.73 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 1018.1 1018.1 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 1464.48 1464.48 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 39. testUDFOnOrderCols-0-7647562850dd367ef1e6c63117805423 b/sql/hive/src/test/resources/golden/windowing.q -- 39. testUDFOnOrderCols-0-7647562850dd367ef1e6c63117805423 new file mode 100644 index 0000000000000..e5a541f56f6f3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 39. testUDFOnOrderCols-0-7647562850dd367ef1e6c63117805423 @@ -0,0 +1,26 @@ +Manufacturer#1 LARGE BRUSHED STEEL ARGE BRUSHED STEEL 1 +Manufacturer#1 LARGE BURNISHED STEEL ARGE BURNISHED STEEL 2 +Manufacturer#1 PROMO BURNISHED NICKEL ROMO BURNISHED NICKEL 3 +Manufacturer#1 PROMO PLATED TIN ROMO PLATED TIN 4 +Manufacturer#1 PROMO PLATED TIN ROMO PLATED TIN 4 +Manufacturer#1 STANDARD ANODIZED STEEL TANDARD ANODIZED STEEL 6 +Manufacturer#2 ECONOMY POLISHED STEEL CONOMY POLISHED STEEL 1 +Manufacturer#2 MEDIUM ANODIZED COPPER EDIUM ANODIZED COPPER 2 +Manufacturer#2 MEDIUM BURNISHED COPPER EDIUM BURNISHED COPPER 3 +Manufacturer#2 SMALL POLISHED NICKEL MALL POLISHED NICKEL 4 +Manufacturer#2 STANDARD PLATED TIN TANDARD PLATED TIN 5 +Manufacturer#3 ECONOMY PLATED COPPER CONOMY PLATED COPPER 1 +Manufacturer#3 MEDIUM BURNISHED BRASS EDIUM BURNISHED BRASS 2 +Manufacturer#3 MEDIUM BURNISHED TIN EDIUM BURNISHED TIN 3 +Manufacturer#3 PROMO ANODIZED TIN ROMO ANODIZED TIN 4 +Manufacturer#3 STANDARD POLISHED STEEL TANDARD POLISHED STEEL 5 +Manufacturer#4 ECONOMY BRUSHED COPPER CONOMY BRUSHED COPPER 1 +Manufacturer#4 SMALL BRUSHED BRASS MALL BRUSHED BRASS 2 +Manufacturer#4 SMALL PLATED STEEL MALL PLATED STEEL 3 +Manufacturer#4 PROMO POLISHED STEEL ROMO POLISHED STEEL 4 +Manufacturer#4 STANDARD ANODIZED TIN TANDARD ANODIZED TIN 5 +Manufacturer#5 LARGE BRUSHED BRASS ARGE BRUSHED BRASS 1 +Manufacturer#5 ECONOMY BURNISHED STEEL CONOMY BURNISHED STEEL 2 +Manufacturer#5 MEDIUM BURNISHED TIN EDIUM BURNISHED TIN 3 +Manufacturer#5 SMALL PLATED BRASS MALL PLATED BRASS 4 +Manufacturer#5 STANDARD BURNISHED TIN TANDARD BURNISHED TIN 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 4. testCount-0-e6e97e884327df86f16b870527ec026c b/sql/hive/src/test/resources/golden/windowing.q -- 4. testCount-0-e6e97e884327df86f16b870527ec026c new file mode 100644 index 0000000000000..bf8e620a304af --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 4. testCount-0-e6e97e884327df86f16b870527ec026c @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#1 almond antique chartreuse lavender yellow 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 4 +Manufacturer#1 almond aquamarine burnished black steel 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 6 +Manufacturer#2 almond antique violet chocolate turquoise 1 +Manufacturer#2 almond antique violet turquoise frosted 2 +Manufacturer#2 almond aquamarine midnight light salmon 3 +Manufacturer#2 almond aquamarine rose maroon antique 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 +Manufacturer#3 almond antique chartreuse khaki white 1 +Manufacturer#3 almond antique forest lavender goldenrod 2 +Manufacturer#3 almond antique metallic orange dim 3 +Manufacturer#3 almond antique misty red olive 4 +Manufacturer#3 almond antique olive coral navajo 5 +Manufacturer#4 almond antique gainsboro frosted violet 1 +Manufacturer#4 almond antique violet mint lemon 2 +Manufacturer#4 almond aquamarine floral ivory bisque 3 +Manufacturer#4 almond aquamarine yellow dodger mint 4 +Manufacturer#4 almond azure aquamarine papaya violet 5 +Manufacturer#5 almond antique blue firebrick mint 1 +Manufacturer#5 almond antique medium spring khaki 2 +Manufacturer#5 almond antique sky peru orange 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 +Manufacturer#5 almond azure blanched chiffon midnight 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 40. testNoBetweenForRows-0-99007f45b6406869e048b0e4eb9213f1 b/sql/hive/src/test/resources/golden/windowing.q -- 40. testNoBetweenForRows-0-99007f45b6406869e048b0e4eb9213f1 new file mode 100644 index 0000000000000..1e29df62901d0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 40. testNoBetweenForRows-0-99007f45b6406869e048b0e4eb9213f1 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 4272.34 +Manufacturer#3 almond antique misty red olive 1 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 41. testNoBetweenForRange-0-d81a591e90950de291d2f133793e9283 b/sql/hive/src/test/resources/golden/windowing.q -- 41. testNoBetweenForRange-0-d81a591e90950de291d2f133793e9283 new file mode 100644 index 0000000000000..a620479fe4064 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 41. testNoBetweenForRange-0-d81a591e90950de291d2f133793e9283 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 3948.8900000000003 +Manufacturer#1 almond aquamarine burnished black steel 28 5363.31 +Manufacturer#1 almond antique chartreuse lavender yellow 34 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 8749.730000000001 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 +Manufacturer#2 almond antique violet chocolate turquoise 14 3722.66 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5424.26 +Manufacturer#2 almond aquamarine rose maroon antique 25 7122.92 +Manufacturer#2 almond antique violet turquoise frosted 40 8923.62 +Manufacturer#3 almond antique misty red olive 1 1922.98 +Manufacturer#3 almond antique forest lavender goldenrod 14 3113.25 +Manufacturer#3 almond antique chartreuse khaki white 17 4784.93 +Manufacturer#3 almond antique metallic orange dim 19 6195.320000000001 +Manufacturer#3 almond antique olive coral navajo 45 7532.610000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 +Manufacturer#4 almond antique gainsboro frosted violet 10 3465.59 +Manufacturer#4 almond azure aquamarine papaya violet 12 4755.9400000000005 +Manufacturer#4 almond aquamarine floral ivory bisque 27 5962.200000000001 +Manufacturer#4 almond antique violet mint lemon 39 7337.620000000001 +Manufacturer#5 almond antique sky peru orange 2 1788.73 +Manufacturer#5 almond antique medium spring khaki 6 3400.3900000000003 +Manufacturer#5 almond azure blanched chiffon midnight 23 4864.870000000001 +Manufacturer#5 almond antique blue firebrick mint 31 6654.560000000001 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 7672.660000000002 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 42. testUnboundedFollowingForRows-0-fb8648e82e4dd56d6bdcfd739dd1edf0 b/sql/hive/src/test/resources/golden/windowing.q -- 42. testUnboundedFollowingForRows-0-fb8648e82e4dd56d6bdcfd739dd1edf0 new file mode 100644 index 0000000000000..74147d2571a15 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 42. testUnboundedFollowingForRows-0-fb8648e82e4dd56d6bdcfd739dd1edf0 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 8749.730000000001 +Manufacturer#1 almond antique burnished rose metallic 2 7576.58 +Manufacturer#1 almond antique chartreuse lavender yellow 34 6403.43 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4649.67 +Manufacturer#1 almond aquamarine burnished black steel 28 3047.08 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 +Manufacturer#2 almond antique violet chocolate turquoise 14 8923.62 +Manufacturer#2 almond antique violet turquoise frosted 40 7232.9400000000005 +Manufacturer#2 almond aquamarine midnight light salmon 2 5432.24 +Manufacturer#2 almond aquamarine rose maroon antique 25 3400.26 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 +Manufacturer#3 almond antique chartreuse khaki white 17 7532.61 +Manufacturer#3 almond antique forest lavender goldenrod 14 5860.929999999999 +Manufacturer#3 almond antique metallic orange dim 19 4670.66 +Manufacturer#3 almond antique misty red olive 1 3260.27 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 +Manufacturer#4 almond antique gainsboro frosted violet 10 7337.620000000001 +Manufacturer#4 almond antique violet mint lemon 39 5716.950000000001 +Manufacturer#4 almond aquamarine floral ivory bisque 27 4341.530000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 3135.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 +Manufacturer#5 almond antique blue firebrick mint 31 7672.66 +Manufacturer#5 almond antique medium spring khaki 6 5882.970000000001 +Manufacturer#5 almond antique sky peru orange 2 4271.3099999999995 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 2482.58 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 43. testUnboundedFollowingForRange-0-3cd04e5f2398853c4850f4f86142bb39 b/sql/hive/src/test/resources/golden/windowing.q -- 43. testUnboundedFollowingForRange-0-3cd04e5f2398853c4850f4f86142bb39 new file mode 100644 index 0000000000000..49d003b5de139 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 43. testUnboundedFollowingForRange-0-3cd04e5f2398853c4850f4f86142bb39 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 8749.730000000001 +Manufacturer#1 almond antique burnished rose metallic 2 8749.730000000001 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 6403.43 +Manufacturer#1 almond aquamarine burnished black steel 28 4800.84 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3386.42 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 +Manufacturer#2 almond aquamarine midnight light salmon 2 8923.62 +Manufacturer#2 almond antique violet chocolate turquoise 14 6891.639999999999 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5200.96 +Manufacturer#2 almond aquamarine rose maroon antique 25 3499.36 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 +Manufacturer#3 almond antique misty red olive 1 7532.610000000001 +Manufacturer#3 almond antique forest lavender goldenrod 14 5609.63 +Manufacturer#3 almond antique chartreuse khaki white 17 4419.360000000001 +Manufacturer#3 almond antique metallic orange dim 19 2747.6800000000003 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7337.620000000001 +Manufacturer#4 almond antique gainsboro frosted violet 10 5492.7 +Manufacturer#4 almond azure aquamarine papaya violet 12 3872.0299999999997 +Manufacturer#4 almond aquamarine floral ivory bisque 27 2581.6800000000003 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 +Manufacturer#5 almond antique sky peru orange 2 7672.660000000002 +Manufacturer#5 almond antique medium spring khaki 6 5883.93 +Manufacturer#5 almond azure blanched chiffon midnight 23 4272.27 +Manufacturer#5 almond antique blue firebrick mint 31 2807.79 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 44. testOverNoPartitionSingleAggregate-0-cb3d2f8c1296044dc2658876bb6103ae b/sql/hive/src/test/resources/golden/windowing.q -- 44. testOverNoPartitionSingleAggregate-0-cb3d2f8c1296044dc2658876bb6103ae new file mode 100644 index 0000000000000..5982c9ee2a4d4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 44. testOverNoPartitionSingleAggregate-0-cb3d2f8c1296044dc2658876bb6103ae @@ -0,0 +1,26 @@ +almond antique blue firebrick mint 1789.69 1546.78 +almond antique burnished rose metallic 1173.15 1546.78 +almond antique burnished rose metallic 1173.15 1546.78 +almond antique chartreuse khaki white 1671.68 1546.78 +almond antique chartreuse lavender yellow 1753.76 1546.78 +almond antique forest lavender goldenrod 1190.27 1546.78 +almond antique gainsboro frosted violet 1620.67 1546.78 +almond antique medium spring khaki 1611.66 1546.78 +almond antique metallic orange dim 1410.39 1546.78 +almond antique misty red olive 1922.98 1546.78 +almond antique olive coral navajo 1337.29 1546.78 +almond antique salmon chartreuse burlywood 1602.59 1546.78 +almond antique sky peru orange 1788.73 1546.78 +almond antique violet chocolate turquoise 1690.68 1546.78 +almond antique violet mint lemon 1375.42 1546.78 +almond antique violet turquoise frosted 1800.7 1546.78 +almond aquamarine burnished black steel 1414.42 1546.78 +almond aquamarine dodger light gainsboro 1018.1 1546.78 +almond aquamarine floral ivory bisque 1206.26 1546.78 +almond aquamarine midnight light salmon 2031.98 1546.78 +almond aquamarine pink moccasin thistle 1632.66 1546.78 +almond aquamarine rose maroon antique 1698.66 1546.78 +almond aquamarine sandy cyan gainsboro 1701.6 1546.78 +almond aquamarine yellow dodger mint 1844.92 1546.78 +almond azure aquamarine papaya violet 1290.35 1546.78 +almond azure blanched chiffon midnight 1464.48 1546.78 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 5. testCountWithWindowingUDAF-0-3bde93728761b780a745c2ce0398aa0f b/sql/hive/src/test/resources/golden/windowing.q -- 5. testCountWithWindowingUDAF-0-3bde93728761b780a745c2ce0398aa0f new file mode 100644 index 0000000000000..00d41fc0bcd9c --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 5. testCountWithWindowingUDAF-0-3bde93728761b780a745c2ce0398aa0f @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 1 1 2 1173.15 1173.15 2 0 +Manufacturer#1 almond antique burnished rose metallic 1 1 2 1173.15 2346.3 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 3 2 3 1753.76 4100.06 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 4 3 4 1602.59 5702.650000000001 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 5 4 5 1414.42 7117.070000000001 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 6 5 6 1632.66 8749.730000000001 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 1 1690.68 1690.68 14 0 +Manufacturer#2 almond antique violet turquoise frosted 2 2 2 1800.7 3491.38 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 3 3 3 2031.98 5523.360000000001 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 4 4 4 1698.66 7222.02 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 5 5 1701.6 8923.62 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 1 1 1 1671.68 1671.68 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 2 2 2 1190.27 2861.95 14 -3 +Manufacturer#3 almond antique metallic orange dim 3 3 3 1410.39 4272.34 19 5 +Manufacturer#3 almond antique misty red olive 4 4 4 1922.98 6195.32 1 -18 +Manufacturer#3 almond antique olive coral navajo 5 5 5 1337.29 7532.61 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 1 1620.67 1620.67 10 0 +Manufacturer#4 almond antique violet mint lemon 2 2 2 1375.42 2996.09 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 3 3 3 1206.26 4202.35 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 4 4 4 1844.92 6047.27 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 5 5 5 1290.35 7337.620000000001 12 5 +Manufacturer#5 almond antique blue firebrick mint 1 1 1 1789.69 1789.69 31 0 +Manufacturer#5 almond antique medium spring khaki 2 2 2 1611.66 3401.3500000000004 6 -25 +Manufacturer#5 almond antique sky peru orange 3 3 3 1788.73 5190.08 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 4 4 1018.1 6208.18 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 5 5 5 1464.48 7672.66 23 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 6. testCountInSubQ-0-73d5274a21d4f4fd51d2a0f1d98516ce b/sql/hive/src/test/resources/golden/windowing.q -- 6. testCountInSubQ-0-73d5274a21d4f4fd51d2a0f1d98516ce new file mode 100644 index 0000000000000..98c09e4fe15c3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 6. testCountInSubQ-0-73d5274a21d4f4fd51d2a0f1d98516ce @@ -0,0 +1,26 @@ +1 1 2 1173.15 0 +1 1 2 2346.3 0 +3 2 3 4100.06 32 +4 3 4 5702.650000000001 -28 +5 4 5 7117.070000000001 22 +6 5 6 8749.730000000001 14 +1 1 1 1690.68 0 +2 2 2 3491.38 26 +3 3 3 5523.360000000001 -38 +4 4 4 7222.02 23 +5 5 5 8923.62 -7 +1 1 1 1671.68 0 +2 2 2 2861.95 -3 +3 3 3 4272.34 5 +4 4 4 6195.32 -18 +5 5 5 7532.61 44 +1 1 1 1620.67 0 +2 2 2 2996.09 29 +3 3 3 4202.35 -12 +4 4 4 6047.27 -20 +5 5 5 7337.620000000001 5 +1 1 1 1789.69 0 +2 2 2 3401.3500000000004 -25 +3 3 3 5190.08 -4 +4 4 4 6208.18 44 +5 5 5 7672.66 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 8. testMixedCaseAlias-0-4b1ad2515fb079012467e987f484a722 b/sql/hive/src/test/resources/golden/windowing.q -- 8. testMixedCaseAlias-0-4b1ad2515fb079012467e987f484a722 new file mode 100644 index 0000000000000..e35257d98382e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 8. testMixedCaseAlias-0-4b1ad2515fb079012467e987f484a722 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 +Manufacturer#1 almond aquamarine burnished black steel 28 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 +Manufacturer#2 almond antique violet turquoise frosted 40 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 +Manufacturer#3 almond antique chartreuse khaki white 17 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 +Manufacturer#3 almond antique metallic orange dim 19 3 +Manufacturer#3 almond antique misty red olive 1 4 +Manufacturer#3 almond antique olive coral navajo 45 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 +Manufacturer#4 almond antique violet mint lemon 39 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 +Manufacturer#5 almond antique medium spring khaki 6 2 +Manufacturer#5 almond antique sky peru orange 2 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 9. testHavingWithWindowingNoGBY-0-70cdc0555a61ef08534a9ebebb95ebbf b/sql/hive/src/test/resources/golden/windowing.q -- 9. testHavingWithWindowingNoGBY-0-70cdc0555a61ef08534a9ebebb95ebbf new file mode 100644 index 0000000000000..850c41c8115d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 9. testHavingWithWindowingNoGBY-0-70cdc0555a61ef08534a9ebebb95ebbf @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-0-d3f50875bd5dff172cf813fdb7d738eb b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-0-d3f50875bd5dff172cf813fdb7d738eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-1-dda16565b98926fc3587de937b9401c7 b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-1-dda16565b98926fc3587de937b9401c7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-2-374e39786feb745cd70f25be58bfa24 b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-2-374e39786feb745cd70f25be58bfa24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-3-d2b5e23edec42a62e61750b110ecbaac b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-3-d2b5e23edec42a62e61750b110ecbaac new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-3-d2b5e23edec42a62e61750b110ecbaac @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-4-50d0c630159068b5b8ccdeb76493f1f7 b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-4-50d0c630159068b5b8ccdeb76493f1f7 new file mode 100644 index 0000000000000..850c41c8115d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-4-50d0c630159068b5b8ccdeb76493f1f7 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-5-3f95cd6f4add7a2d0101fe3dd97e5082 b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-5-3f95cd6f4add7a2d0101fe3dd97e5082 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-5-3f95cd6f4add7a2d0101fe3dd97e5082 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-0-d3f50875bd5dff172cf813fdb7d738eb b/sql/hive/src/test/resources/golden/windowing_columnPruning-0-d3f50875bd5dff172cf813fdb7d738eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-1-dda16565b98926fc3587de937b9401c7 b/sql/hive/src/test/resources/golden/windowing_columnPruning-1-dda16565b98926fc3587de937b9401c7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-2-374e39786feb745cd70f25be58bfa24 b/sql/hive/src/test/resources/golden/windowing_columnPruning-2-374e39786feb745cd70f25be58bfa24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-3-9294b4a22bc396ff2accabd53c5da98b b/sql/hive/src/test/resources/golden/windowing_columnPruning-3-9294b4a22bc396ff2accabd53c5da98b new file mode 100644 index 0000000000000..1b5ae55383a4a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_columnPruning-3-9294b4a22bc396ff2accabd53c5da98b @@ -0,0 +1,26 @@ +2 1173.15 +2 2346.3 +34 4100.06 +6 5702.650000000001 +28 7117.070000000001 +42 8749.730000000001 +14 1690.68 +40 3491.38 +2 5523.360000000001 +25 7222.02 +18 8923.62 +17 1671.68 +14 2861.95 +19 4272.34 +1 6195.32 +45 7532.61 +10 1620.67 +39 2996.09 +27 4202.35 +7 6047.27 +12 7337.620000000001 +31 1789.69 +6 3401.3500000000004 +2 5190.08 +46 6208.18 +23 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-4-445cab062581c449ceffcb368cdf133 b/sql/hive/src/test/resources/golden/windowing_columnPruning-4-445cab062581c449ceffcb368cdf133 new file mode 100644 index 0000000000000..1b5ae55383a4a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_columnPruning-4-445cab062581c449ceffcb368cdf133 @@ -0,0 +1,26 @@ +2 1173.15 +2 2346.3 +34 4100.06 +6 5702.650000000001 +28 7117.070000000001 +42 8749.730000000001 +14 1690.68 +40 3491.38 +2 5523.360000000001 +25 7222.02 +18 8923.62 +17 1671.68 +14 2861.95 +19 4272.34 +1 6195.32 +45 7532.61 +10 1620.67 +39 2996.09 +27 4202.35 +7 6047.27 +12 7337.620000000001 +31 1789.69 +6 3401.3500000000004 +2 5190.08 +46 6208.18 +23 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-5-89110070c761eafb992eb9315128b53f b/sql/hive/src/test/resources/golden/windowing_columnPruning-5-89110070c761eafb992eb9315128b53f new file mode 100644 index 0000000000000..e426c725b0e36 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_columnPruning-5-89110070c761eafb992eb9315128b53f @@ -0,0 +1,26 @@ +Manufacturer#1 1173.15 +Manufacturer#1 2346.3 +Manufacturer#1 4100.06 +Manufacturer#1 5702.650000000001 +Manufacturer#1 7117.070000000001 +Manufacturer#1 8749.730000000001 +Manufacturer#2 1690.68 +Manufacturer#2 3491.38 +Manufacturer#2 5523.360000000001 +Manufacturer#2 7222.02 +Manufacturer#2 8923.62 +Manufacturer#3 1671.68 +Manufacturer#3 2861.95 +Manufacturer#3 4272.34 +Manufacturer#3 6195.32 +Manufacturer#3 7532.61 +Manufacturer#4 1620.67 +Manufacturer#4 2996.09 +Manufacturer#4 4202.35 +Manufacturer#4 6047.27 +Manufacturer#4 7337.620000000001 +Manufacturer#5 1789.69 +Manufacturer#5 3401.3500000000004 +Manufacturer#5 5190.08 +Manufacturer#5 6208.18 +Manufacturer#5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 1-0-12a92d8800e0da8b515ba3eaf6a7fd0f b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 1-0-12a92d8800e0da8b515ba3eaf6a7fd0f new file mode 100644 index 0000000000000..acc4f3bc2a2d5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 1-0-12a92d8800e0da8b515ba3eaf6a7fd0f @@ -0,0 +1,1049 @@ + 1 4294967354 + 2 8589934811 + 3 12884902227 +alice allen 1 4294967487 +alice allen 2 8589935012 +alice allen 3 12884902543 +alice brown 1 4294967355 +alice carson 1 4294967370 +alice davidson 1 4294967517 +alice falkner 1 4294967316 +alice garcia 1 4294967369 +alice hernandez 1 4294967314 +alice hernandez 2 8589934613 +alice johnson 1 4294967424 +alice king 1 4294967387 +alice king 2 8589934903 +alice king 3 12884902449 +alice laertes 1 4294967519 +alice laertes 2 8589935038 +alice miller 1 4294967324 +alice nixon 1 4294967484 +alice nixon 2 8589934894 +alice nixon 3 12884902307 +alice ovid 1 4294967412 +alice polk 1 4294967366 +alice quirinius 1 4294967505 +alice quirinius 2 8589935054 +alice robinson 1 4294967502 +alice robinson 2 8589934947 +alice steinbeck 1 4294967549 +alice steinbeck 2 8589934913 +alice steinbeck 3 12884902387 +alice underhill 1 4294967441 +alice van buren 1 4294967428 +alice xylophone 1 4294967519 +alice xylophone 2 8589934877 +alice xylophone 3 12884902240 +alice zipper 1 4294967380 +alice zipper 2 8589934919 +alice zipper 3 12884902439 +bob brown 1 4294967431 +bob brown 2 8589934853 +bob brown 3 12884902280 +bob carson 1 4294967408 +bob davidson 1 4294967435 +bob davidson 2 8589934939 +bob davidson 3 12884902293 +bob ellison 1 4294967530 +bob ellison 2 8589934966 +bob ellison 3 12884902328 +bob ellison 4 17179869672 +bob falkner 1 4294967464 +bob garcia 1 4294967435 +bob garcia 2 8589934804 +bob garcia 3 12884902148 +bob garcia 4 17179869587 +bob garcia 5 21474836905 +bob hernandez 1 4294967500 +bob ichabod 1 4294967424 +bob king 1 4294967443 +bob king 2 8589934740 +bob king 3 12884902279 +bob laertes 1 4294967472 +bob laertes 2 8589934852 +bob miller 1 4294967349 +bob ovid 1 4294967401 +bob ovid 2 8589934801 +bob ovid 3 12884902313 +bob ovid 4 17179869708 +bob polk 1 4294967337 +bob quirinius 1 4294967346 +bob steinbeck 1 4294967342 +bob van buren 1 4294967422 +bob white 1 4294967493 +bob white 2 8589934855 +bob xylophone 1 4294967407 +bob xylophone 2 8589934872 +bob young 1 4294967413 +bob zipper 1 4294967416 +bob zipper 2 8589934769 +bob zipper 3 12884902068 +calvin allen 1 4294967373 +calvin brown 1 4294967530 +calvin brown 2 8589934967 +calvin brown 3 12884902378 +calvin carson 1 4294967534 +calvin davidson 1 4294967437 +calvin davidson 2 8589934905 +calvin ellison 1 4294967480 +calvin falkner 1 4294967305 +calvin falkner 2 8589934723 +calvin falkner 3 12884902155 +calvin falkner 4 17179869455 +calvin falkner 5 21474836800 +calvin falkner 6 25769804250 +calvin garcia 1 4294967492 +calvin hernandez 1 4294967341 +calvin johnson 1 4294967546 +calvin laertes 1 4294967499 +calvin laertes 2 8589934930 +calvin nixon 1 4294967488 +calvin nixon 2 8589934788 +calvin nixon 3 12884902200 +calvin ovid 1 4294967343 +calvin ovid 2 8589934881 +calvin ovid 3 12884902210 +calvin ovid 4 17179869559 +calvin polk 1 4294967416 +calvin quirinius 1 4294967532 +calvin quirinius 2 8589935053 +calvin robinson 1 4294967326 +calvin steinbeck 1 4294967417 +calvin steinbeck 2 8589934891 +calvin steinbeck 3 12884902396 +calvin thompson 1 4294967346 +calvin thompson 2 8589934859 +calvin underhill 1 4294967478 +calvin van buren 1 4294967300 +calvin van buren 2 8589934808 +calvin white 1 4294967304 +calvin white 2 8589934848 +calvin xylophone 1 4294967299 +calvin xylophone 2 8589934675 +calvin xylophone 3 12884902133 +calvin young 1 4294967410 +calvin young 2 8589934752 +calvin zipper 1 4294967441 +calvin zipper 2 8589934960 +david allen 1 4294967381 +david allen 2 8589934752 +david brown 1 4294967544 +david brown 2 8589934870 +david davidson 1 4294967487 +david davidson 2 8589934952 +david davidson 3 12884902474 +david davidson 4 17179869819 +david ellison 1 4294967477 +david ellison 2 8589934963 +david ellison 3 12884902426 +david hernandez 1 4294967324 +david ichabod 1 4294967385 +david ichabod 2 8589934872 +david laertes 1 4294967385 +david nixon 1 4294967381 +david ovid 1 4294967396 +david ovid 2 8589934839 +david quirinius 1 4294967375 +david quirinius 2 8589934905 +david quirinius 3 12884902362 +david robinson 1 4294967465 +david robinson 2 8589934933 +david thompson 1 4294967361 +david underhill 1 4294967331 +david underhill 2 8589934715 +david underhill 3 12884902185 +david van buren 1 4294967438 +david van buren 2 8589934747 +david white 1 4294967428 +david xylophone 1 8589934898 +david xylophone 1 8589934898 +david xylophone 3 12884902378 +david young 1 4294967296 +david young 2 8589934601 +ethan allen 1 4294967351 +ethan brown 1 4294967477 +ethan brown 2 8589934897 +ethan brown 3 12884902217 +ethan brown 4 17179869548 +ethan brown 5 21474836951 +ethan brown 6 25769804375 +ethan carson 1 4294967352 +ethan ellison 1 4294967514 +ethan ellison 2 8589934923 +ethan falkner 1 4294967318 +ethan falkner 2 8589934779 +ethan garcia 1 4294967310 +ethan hernandez 1 4294967349 +ethan johnson 1 4294967357 +ethan king 1 4294967413 +ethan laertes 1 4294967402 +ethan laertes 2 8589934859 +ethan laertes 3 12884902390 +ethan laertes 4 17179869880 +ethan laertes 5 21474837302 +ethan laertes 6 25769804603 +ethan laertes 7 30064771974 +ethan miller 1 4294967352 +ethan nixon 1 4294967499 +ethan ovid 1 4294967452 +ethan polk 1 4294967329 +ethan polk 2 8589934711 +ethan polk 3 12884902253 +ethan polk 4 17179869732 +ethan quirinius 1 4294967501 +ethan quirinius 2 8589934852 +ethan quirinius 3 12884902200 +ethan robinson 1 4294967353 +ethan robinson 2 8589934855 +ethan underhill 1 4294967466 +ethan van buren 1 4294967511 +ethan white 1 4294967445 +ethan white 2 8589934872 +ethan xylophone 1 4294967543 +ethan zipper 1 4294967462 +ethan zipper 2 8589934815 +fred davidson 1 4294967512 +fred davidson 2 8589934936 +fred davidson 3 12884902424 +fred ellison 1 4294967470 +fred ellison 2 8589934901 +fred ellison 3 12884902294 +fred falkner 1 4294967340 +fred falkner 2 8589934887 +fred falkner 3 12884902187 +fred hernandez 1 4294967365 +fred ichabod 1 4294967342 +fred ichabod 2 8589934831 +fred johnson 1 4294967373 +fred king 1 4294967346 +fred king 2 8589934766 +fred laertes 1 4294967351 +fred miller 1 4294967490 +fred nixon 1 4294967514 +fred nixon 2 8589934811 +fred nixon 3 12884902293 +fred nixon 4 17179869668 +fred polk 1 4294967332 +fred polk 2 8589934775 +fred polk 3 12884902233 +fred polk 4 17179869740 +fred quirinius 1 4294967426 +fred quirinius 2 8589934951 +fred robinson 1 4294967461 +fred steinbeck 1 4294967411 +fred steinbeck 2 8589934740 +fred steinbeck 3 12884902212 +fred underhill 1 4294967387 +fred van buren 1 4294967431 +fred van buren 2 8589934812 +fred van buren 3 12884902338 +fred van buren 4 17179869801 +fred white 1 4294967434 +fred young 1 4294967495 +fred young 2 8589934980 +fred zipper 1 4294967447 +gabriella allen 1 4294967405 +gabriella allen 2 8589934939 +gabriella brown 1 4294967543 +gabriella brown 2 8589934946 +gabriella carson 1 4294967540 +gabriella davidson 1 4294967507 +gabriella ellison 1 4294967393 +gabriella ellison 2 8589934733 +gabriella falkner 1 4294967378 +gabriella falkner 2 8589934901 +gabriella falkner 3 12884902335 +gabriella garcia 1 4294967419 +gabriella hernandez 1 4294967481 +gabriella hernandez 2 8589934943 +gabriella ichabod 1 4294967337 +gabriella ichabod 2 8589934725 +gabriella ichabod 3 12884902062 +gabriella ichabod 4 17179869382 +gabriella ichabod 5 21474836880 +gabriella king 1 4294967434 +gabriella king 2 8589934827 +gabriella laertes 1 4294967410 +gabriella miller 1 4294967363 +gabriella ovid 1 4294967482 +gabriella ovid 2 8589935004 +gabriella polk 1 4294967410 +gabriella polk 2 8589934712 +gabriella steinbeck 1 4294967500 +gabriella steinbeck 2 8589934935 +gabriella thompson 1 4294967299 +gabriella thompson 2 8589934711 +gabriella thompson 3 12884902196 +gabriella van buren 1 4294967457 +gabriella van buren 2 8589934927 +gabriella white 1 4294967335 +gabriella young 1 4294967493 +gabriella young 2 8589934924 +gabriella zipper 1 4294967357 +gabriella zipper 2 8589934867 +holly allen 1 4294967327 +holly brown 1 4294967321 +holly brown 2 8589934659 +holly falkner 1 4294967324 +holly hernandez 1 4294967378 +holly hernandez 2 8589934921 +holly hernandez 3 12884902465 +holly hernandez 4 17179869773 +holly ichabod 1 4294967342 +holly ichabod 2 8589934800 +holly ichabod 3 12884902129 +holly johnson 1 4294967517 +holly johnson 2 8589934897 +holly johnson 3 12884902432 +holly king 1 4294967392 +holly king 2 8589934753 +holly laertes 1 4294967406 +holly miller 1 4294967388 +holly nixon 1 4294967383 +holly nixon 2 8589934707 +holly polk 1 4294967398 +holly polk 2 8589934832 +holly robinson 1 4294967532 +holly thompson 1 4294967529 +holly thompson 2 8589934868 +holly thompson 3 12884902242 +holly underhill 1 4294967383 +holly underhill 2 8589934894 +holly underhill 3 12884902330 +holly underhill 4 17179869856 +holly van buren 1 4294967539 +holly white 1 4294967320 +holly white 2 8589934735 +holly xylophone 1 4294967435 +holly young 1 4294967487 +holly young 2 8589934987 +holly zipper 1 4294967337 +holly zipper 2 8589934846 +irene allen 1 4294967518 +irene brown 1 4294967434 +irene brown 2 8589934862 +irene brown 3 12884902272 +irene carson 1 4294967473 +irene ellison 1 4294967379 +irene ellison 2 8589934797 +irene falkner 1 4294967404 +irene falkner 2 8589934952 +irene garcia 1 4294967498 +irene garcia 2 8589934869 +irene garcia 3 12884902192 +irene ichabod 1 4294967529 +irene ichabod 2 8589935038 +irene johnson 1 4294967468 +irene laertes 1 4294967481 +irene laertes 2 8589934780 +irene laertes 3 12884902116 +irene miller 1 4294967387 +irene nixon 1 4294967323 +irene nixon 2 8589934824 +irene nixon 3 12884902362 +irene ovid 1 4294967499 +irene ovid 2 8589934870 +irene ovid 3 12884902230 +irene polk 1 4294967521 +irene polk 2 8589934930 +irene polk 3 12884902395 +irene polk 4 17179869941 +irene polk 5 21474837237 +irene quirinius 1 4294967365 +irene quirinius 2 8589934751 +irene quirinius 3 12884902141 +irene robinson 1 4294967347 +irene steinbeck 1 4294967549 +irene thompson 1 4294967479 +irene underhill 1 4294967371 +irene underhill 2 8589934753 +irene van buren 1 4294967439 +irene van buren 2 8589934906 +irene xylophone 1 4294967436 +jessica brown 1 4294967496 +jessica carson 1 4294967389 +jessica carson 2 8589934897 +jessica carson 3 12884902345 +jessica davidson 1 4294967325 +jessica davidson 2 8589934709 +jessica davidson 3 12884902098 +jessica davidson 4 17179869569 +jessica ellison 1 4294967316 +jessica ellison 2 8589934721 +jessica falkner 1 4294967549 +jessica garcia 1 4294967540 +jessica garcia 2 8589935041 +jessica ichabod 1 4294967413 +jessica johnson 1 4294967497 +jessica johnson 2 8589934870 +jessica miller 1 4294967495 +jessica nixon 1 4294967311 +jessica nixon 2 8589934754 +jessica ovid 1 4294967330 +jessica ovid 2 8589934855 +jessica polk 1 4294967403 +jessica quirinius 1 4294967523 +jessica quirinius 2 8589934942 +jessica quirinius 3 12884902388 +jessica quirinius 4 17179869696 +jessica robinson 1 4294967542 +jessica thompson 1 4294967449 +jessica thompson 2 8589934763 +jessica underhill 1 4294967541 +jessica underhill 2 8589934844 +jessica underhill 3 12884902153 +jessica van buren 1 4294967344 +jessica white 1 4294967482 +jessica white 2 8589934929 +jessica white 3 12884902378 +jessica white 4 17179869687 +jessica white 5 21474837086 +jessica xylophone 1 4294967421 +jessica young 1 4294967382 +jessica young 2 8589934903 +jessica zipper 1 4294967334 +jessica zipper 2 8589934785 +jessica zipper 3 12884902157 +katie allen 1 4294967443 +katie brown 1 4294967420 +katie davidson 1 4294967459 +katie ellison 1 4294967486 +katie ellison 2 8589934861 +katie falkner 1 4294967362 +katie garcia 1 4294967306 +katie garcia 2 8589934680 +katie hernandez 1 4294967451 +katie ichabod 1 4294967330 +katie ichabod 2 8589934742 +katie ichabod 3 12884902209 +katie king 1 4294967339 +katie king 2 8589934760 +katie king 3 12884902199 +katie miller 1 4294967425 +katie miller 2 8589934859 +katie nixon 1 4294967500 +katie ovid 1 4294967519 +katie polk 1 4294967384 +katie polk 2 8589934926 +katie robinson 1 4294967310 +katie van buren 1 4294967335 +katie van buren 2 8589934647 +katie white 1 4294967337 +katie white 2 8589934643 +katie xylophone 1 4294967486 +katie young 1 4294967349 +katie young 2 8589934681 +katie young 3 12884902225 +katie zipper 1 4294967354 +katie zipper 2 8589934766 +luke allen 1 4294967533 +luke allen 2 8589934836 +luke allen 3 12884902346 +luke allen 4 17179869863 +luke allen 5 21474837208 +luke brown 1 4294967473 +luke davidson 1 4294967550 +luke davidson 2 8589934904 +luke ellison 1 4294967322 +luke ellison 2 8589934675 +luke ellison 3 12884902103 +luke falkner 1 4294967359 +luke falkner 2 8589934782 +luke garcia 1 4294967304 +luke garcia 2 8589934683 +luke ichabod 1 4294967324 +luke ichabod 2 8589934766 +luke johnson 1 4294967527 +luke johnson 2 8589934987 +luke johnson 3 12884902342 +luke laertes 1 4294967505 +luke laertes 2 8589935011 +luke laertes 3 12884902497 +luke laertes 4 17179869806 +luke laertes 5 21474837193 +luke miller 1 4294967497 +luke ovid 1 4294967492 +luke ovid 2 8589934901 +luke polk 1 4294967545 +luke polk 2 8589934873 +luke quirinius 1 4294967320 +luke robinson 1 4294967299 +luke robinson 2 8589934606 +luke thompson 1 4294967521 +luke underhill 1 4294967393 +luke underhill 2 8589934803 +luke underhill 3 12884902122 +luke van buren 1 4294967424 +luke white 1 4294967505 +luke xylophone 1 4294967382 +luke zipper 1 4294967353 +mike allen 1 4294967466 +mike brown 1 4294967369 +mike carson 1 4294967477 +mike carson 2 8589934934 +mike carson 3 12884902482 +mike davidson 1 4294967501 +mike davidson 2 8589934965 +mike ellison 1 4294967353 +mike ellison 2 8589934747 +mike ellison 3 12884902282 +mike ellison 4 17179869806 +mike ellison 5 21474837309 +mike falkner 1 4294967301 +mike garcia 1 4294967428 +mike garcia 2 8589934826 +mike garcia 3 12884902289 +mike hernandez 1 4294967316 +mike hernandez 2 8589934800 +mike ichabod 1 4294967494 +mike king 1 4294967323 +mike king 2 8589934848 +mike king 3 12884902248 +mike king 4 17179869595 +mike king 5 21474837046 +mike king 6 25769804478 +mike miller 1 4294967449 +mike nixon 1 4294967527 +mike nixon 2 8589935004 +mike polk 1 4294967389 +mike polk 2 8589934848 +mike polk 3 12884902351 +mike quirinius 1 4294967422 +mike steinbeck 1 4294967519 +mike steinbeck 2 8589934827 +mike steinbeck 3 12884902316 +mike steinbeck 4 17179869850 +mike van buren 1 4294967544 +mike van buren 2 8589935061 +mike white 1 4294967336 +mike white 2 8589934882 +mike white 3 12884902374 +mike white 4 17179869843 +mike young 1 4294967453 +mike young 2 8589934804 +mike young 3 12884902198 +mike zipper 1 4294967402 +mike zipper 2 8589934727 +mike zipper 3 12884902228 +nick allen 1 4294967507 +nick allen 2 8589934807 +nick brown 1 4294967334 +nick davidson 1 4294967357 +nick ellison 1 4294967397 +nick ellison 2 8589934699 +nick falkner 1 4294967480 +nick falkner 2 8589934923 +nick garcia 1 4294967384 +nick garcia 2 8589934797 +nick garcia 3 12884902319 +nick ichabod 1 4294967388 +nick ichabod 2 8589934758 +nick ichabod 3 12884902225 +nick johnson 1 4294967398 +nick johnson 2 8589934809 +nick laertes 1 4294967389 +nick miller 1 4294967550 +nick nixon 1 4294967482 +nick ovid 1 4294967488 +nick polk 1 4294967551 +nick quirinius 1 4294967316 +nick quirinius 2 8589934612 +nick robinson 1 4294967409 +nick robinson 2 8589934731 +nick steinbeck 1 4294967355 +nick thompson 1 4294967401 +nick underhill 1 4294967527 +nick van buren 1 4294967303 +nick xylophone 1 4294967460 +nick young 1 4294967405 +nick young 2 8589934917 +nick zipper 1 4294967430 +nick zipper 2 8589934796 +oscar allen 1 4294967500 +oscar brown 1 4294967331 +oscar carson 1 4294967460 +oscar carson 2 8589934904 +oscar carson 3 12884902286 +oscar carson 4 17179869599 +oscar carson 5 21474836960 +oscar davidson 1 4294967482 +oscar ellison 1 8589934740 +oscar ellison 1 8589934740 +oscar falkner 1 4294967526 +oscar garcia 1 4294967301 +oscar hernandez 1 4294967343 +oscar hernandez 2 8589934843 +oscar ichabod 1 4294967513 +oscar ichabod 2 8589934837 +oscar ichabod 3 12884902165 +oscar ichabod 4 17179869569 +oscar johnson 1 4294967418 +oscar johnson 2 8589934763 +oscar king 1 4294967465 +oscar king 2 8589934936 +oscar king 3 12884902469 +oscar laertes 1 4294967425 +oscar laertes 2 8589934876 +oscar laertes 3 12884902426 +oscar laertes 4 17179869786 +oscar nixon 1 4294967532 +oscar ovid 1 4294967508 +oscar ovid 2 8589934910 +oscar ovid 3 12884902418 +oscar polk 1 4294967325 +oscar polk 2 8589934713 +oscar quirinius 1 4294967416 +oscar quirinius 2 8589934932 +oscar quirinius 3 12884902390 +oscar quirinius 4 17179869763 +oscar robinson 1 4294967355 +oscar robinson 2 8589934681 +oscar robinson 3 12884902031 +oscar robinson 4 17179869383 +oscar steinbeck 1 4294967548 +oscar thompson 1 4294967453 +oscar thompson 2 8589934824 +oscar thompson 3 12884902197 +oscar thompson 4 17179869496 +oscar underhill 1 4294967374 +oscar van buren 1 4294967520 +oscar van buren 2 8589934990 +oscar van buren 3 12884902490 +oscar white 1 4294967454 +oscar white 2 8589934761 +oscar white 3 12884902163 +oscar white 4 17179869512 +oscar xylophone 1 4294967400 +oscar xylophone 2 8589934806 +oscar xylophone 3 12884902124 +oscar zipper 1 4294967449 +oscar zipper 2 8589934969 +oscar zipper 3 12884902458 +priscilla brown 1 4294967369 +priscilla brown 2 8589934897 +priscilla brown 3 12884902360 +priscilla carson 1 4294967489 +priscilla carson 2 8589934838 +priscilla carson 3 12884902270 +priscilla ichabod 1 4294967379 +priscilla ichabod 2 8589934926 +priscilla johnson 1 4294967535 +priscilla johnson 2 8589935003 +priscilla johnson 3 12884902308 +priscilla johnson 4 17179869707 +priscilla johnson 5 21474837167 +priscilla king 1 4294967385 +priscilla nixon 1 4294967388 +priscilla nixon 2 8589934849 +priscilla ovid 1 4294967528 +priscilla ovid 2 8589935035 +priscilla polk 1 4294967434 +priscilla quirinius 1 4294967347 +priscilla thompson 1 4294967497 +priscilla underhill 1 4294967520 +priscilla underhill 2 8589934853 +priscilla van buren 1 4294967318 +priscilla van buren 2 8589934809 +priscilla van buren 3 12884902351 +priscilla white 1 4294967419 +priscilla xylophone 1 4294967503 +priscilla xylophone 2 8589934956 +priscilla xylophone 3 12884902406 +priscilla young 1 4294967401 +priscilla young 2 8589934931 +priscilla zipper 1 4294967516 +priscilla zipper 2 8589934950 +quinn allen 1 4294967339 +quinn allen 2 8589934881 +quinn brown 1 4294967335 +quinn brown 2 8589934651 +quinn brown 3 12884902099 +quinn davidson 1 4294967478 +quinn davidson 2 8589934849 +quinn davidson 3 12884902238 +quinn davidson 4 17179869565 +quinn ellison 1 4294967392 +quinn ellison 2 8589934907 +quinn garcia 1 4294967344 +quinn garcia 2 8589934882 +quinn garcia 3 12884902395 +quinn garcia 4 17179869868 +quinn ichabod 1 4294967405 +quinn king 1 4294967538 +quinn king 2 8589934996 +quinn laertes 1 4294967533 +quinn laertes 2 8589934977 +quinn laertes 3 12884902524 +quinn nixon 1 4294967432 +quinn ovid 1 4294967340 +quinn quirinius 1 4294967347 +quinn robinson 1 4294967365 +quinn steinbeck 1 4294967358 +quinn steinbeck 2 8589934810 +quinn thompson 1 4294967488 +quinn thompson 2 8589934888 +quinn underhill 1 4294967307 +quinn underhill 2 8589934744 +quinn underhill 3 12884902278 +quinn van buren 1 4294967362 +quinn young 1 4294967392 +quinn zipper 1 4294967521 +quinn zipper 2 8589934944 +rachel allen 1 4294967334 +rachel allen 2 8589934713 +rachel brown 1 4294967451 +rachel brown 2 8589934886 +rachel brown 3 12884902325 +rachel brown 4 17179869632 +rachel brown 5 21474836938 +rachel carson 1 4294967461 +rachel carson 2 8589934777 +rachel davidson 1 4294967387 +rachel ellison 1 4294967423 +rachel falkner 1 4294967544 +rachel falkner 2 8589934892 +rachel falkner 3 12884902350 +rachel falkner 4 17179869809 +rachel johnson 1 4294967541 +rachel king 1 4294967442 +rachel king 2 8589934771 +rachel laertes 1 4294967446 +rachel laertes 2 8589934804 +rachel ovid 1 4294967481 +rachel ovid 2 8589934832 +rachel polk 1 4294967335 +rachel quirinius 1 4294967297 +rachel robinson 1 4294967344 +rachel robinson 2 8589934807 +rachel robinson 3 12884902135 +rachel thompson 1 4294967518 +rachel thompson 2 8589934881 +rachel thompson 3 12884902306 +rachel underhill 1 4294967382 +rachel white 1 4294967457 +rachel white 2 8589934793 +rachel young 1 4294967391 +rachel zipper 1 4294967434 +rachel zipper 2 8589934813 +sarah carson 1 4294967503 +sarah carson 2 8589934822 +sarah carson 3 12884902167 +sarah ellison 1 4294967542 +sarah falkner 1 4294967525 +sarah falkner 2 8589934974 +sarah garcia 1 4294967391 +sarah garcia 2 8589934849 +sarah garcia 3 12884902247 +sarah ichabod 1 4294967370 +sarah ichabod 2 8589934909 +sarah johnson 1 4294967433 +sarah johnson 2 8589934926 +sarah johnson 3 12884902235 +sarah johnson 4 17179869559 +sarah king 1 4294967496 +sarah king 2 8589935039 +sarah miller 1 4294967458 +sarah ovid 1 4294967350 +sarah robinson 1 4294967419 +sarah robinson 2 8589934917 +sarah steinbeck 1 4294967456 +sarah white 1 4294967514 +sarah white 2 8589934882 +sarah xylophone 1 4294967355 +sarah young 1 4294967442 +sarah zipper 1 4294967432 +tom brown 1 4294967432 +tom brown 2 8589934950 +tom carson 1 4294967388 +tom carson 2 8589934693 +tom carson 3 12884902227 +tom davidson 1 4294967507 +tom ellison 1 4294967487 +tom ellison 2 8589934844 +tom ellison 3 12884902188 +tom falkner 1 4294967382 +tom falkner 2 8589934837 +tom hernandez 1 8589934733 +tom hernandez 1 8589934733 +tom ichabod 1 4294967445 +tom johnson 1 4294967492 +tom johnson 2 8589934923 +tom king 1 4294967331 +tom laertes 1 4294967431 +tom laertes 2 8589934744 +tom miller 1 4294967366 +tom miller 2 8589934723 +tom miller 3 12884902078 +tom nixon 1 4294967506 +tom ovid 1 4294967512 +tom polk 1 4294967329 +tom polk 2 8589934869 +tom quirinius 1 4294967507 +tom quirinius 2 8589934823 +tom robinson 1 4294967457 +tom robinson 2 8589935008 +tom robinson 3 12884902462 +tom robinson 4 17179869770 +tom steinbeck 1 4294967447 +tom van buren 1 4294967374 +tom van buren 2 8589934703 +tom van buren 3 12884902195 +tom white 1 4294967413 +tom young 1 4294967539 +tom young 2 8589935074 +tom zipper 1 4294967526 +ulysses brown 1 4294967537 +ulysses carson 1 4294967323 +ulysses carson 2 8589934815 +ulysses carson 3 12884902127 +ulysses carson 4 17179869485 +ulysses davidson 1 4294967467 +ulysses ellison 1 4294967442 +ulysses garcia 1 4294967470 +ulysses hernandez 1 4294967449 +ulysses hernandez 2 8589934995 +ulysses hernandez 3 12884902393 +ulysses ichabod 1 4294967353 +ulysses ichabod 2 8589934728 +ulysses johnson 1 4294967432 +ulysses king 1 4294967537 +ulysses laertes 1 4294967391 +ulysses laertes 2 8589934938 +ulysses laertes 3 12884902431 +ulysses miller 1 4294967373 +ulysses miller 2 8589934808 +ulysses nixon 1 4294967296 +ulysses ovid 1 4294967394 +ulysses polk 1 4294967509 +ulysses polk 2 8589934960 +ulysses polk 3 12884902440 +ulysses polk 4 17179869745 +ulysses quirinius 1 4294967449 +ulysses robinson 1 4294967531 +ulysses steinbeck 1 4294967303 +ulysses steinbeck 2 8589934788 +ulysses thompson 1 4294967389 +ulysses underhill 1 4294967544 +ulysses underhill 2 8589934949 +ulysses underhill 3 12884902275 +ulysses underhill 4 17179869726 +ulysses underhill 5 21474837190 +ulysses underhill 6 25769804570 +ulysses underhill 7 30064771927 +ulysses van buren 1 4294967439 +ulysses white 1 4294967429 +ulysses white 2 8589934878 +ulysses xylophone 1 4294967524 +ulysses xylophone 2 8589935025 +ulysses xylophone 3 12884902473 +ulysses young 1 4294967427 +ulysses young 2 8589934763 +ulysses young 3 12884902154 +victor allen 1 4294967450 +victor allen 2 8589934776 +victor brown 1 4294967521 +victor brown 2 8589934864 +victor brown 3 12884902170 +victor brown 4 17179869625 +victor davidson 1 4294967419 +victor davidson 2 8589934720 +victor davidson 3 12884902156 +victor ellison 1 4294967362 +victor ellison 2 8589934831 +victor hernandez 1 4294967428 +victor hernandez 2 8589934733 +victor hernandez 3 12884902062 +victor hernandez 4 17179869402 +victor hernandez 5 21474836874 +victor johnson 1 4294967496 +victor johnson 2 8589934824 +victor johnson 3 12884902246 +victor king 1 4294967401 +victor king 2 8589934884 +victor laertes 1 4294967407 +victor laertes 2 8589934862 +victor miller 1 4294967410 +victor nixon 1 4294967424 +victor nixon 2 8589934803 +victor ovid 1 4294967355 +victor polk 1 4294967333 +victor quirinius 1 4294967520 +victor quirinius 2 8589934846 +victor robinson 1 4294967440 +victor robinson 2 8589934930 +victor steinbeck 1 4294967390 +victor steinbeck 2 8589934707 +victor steinbeck 3 12884902037 +victor thompson 1 4294967319 +victor van buren 1 4294967365 +victor van buren 2 8589934906 +victor white 1 4294967403 +victor white 2 8589934862 +victor xylophone 1 4294967331 +victor xylophone 2 8589934864 +victor xylophone 3 12884902262 +victor xylophone 4 17179869633 +victor xylophone 5 21474837062 +victor young 1 4294967337 +victor zipper 1 4294967428 +wendy allen 1 4294967473 +wendy allen 2 8589934989 +wendy allen 3 12884902367 +wendy brown 1 4294967337 +wendy brown 2 8589934817 +wendy ellison 1 4294967475 +wendy ellison 2 8589934989 +wendy falkner 1 4294967313 +wendy falkner 2 8589934810 +wendy falkner 3 12884902236 +wendy garcia 1 4294967394 +wendy garcia 2 8589934775 +wendy garcia 3 12884902088 +wendy garcia 4 17179869400 +wendy hernandez 1 4294967299 +wendy ichabod 1 4294967516 +wendy king 1 4294967420 +wendy king 2 8589934811 +wendy king 3 12884902252 +wendy laertes 1 4294967519 +wendy laertes 2 8589934939 +wendy laertes 3 12884902315 +wendy miller 1 4294967478 +wendy miller 2 8589934957 +wendy nixon 1 4294967407 +wendy nixon 2 8589934901 +wendy ovid 1 4294967464 +wendy ovid 2 8589934894 +wendy polk 1 4294967434 +wendy polk 2 8589934824 +wendy quirinius 1 4294967334 +wendy quirinius 2 8589934782 +wendy robinson 1 4294967302 +wendy robinson 2 8589934613 +wendy robinson 3 12884901977 +wendy steinbeck 1 4294967444 +wendy thompson 1 4294967301 +wendy thompson 2 8589934621 +wendy underhill 1 4294967540 +wendy underhill 2 8589934993 +wendy underhill 3 12884902410 +wendy van buren 1 4294967488 +wendy van buren 2 8589934835 +wendy white 1 4294967490 +wendy xylophone 1 4294967488 +wendy xylophone 2 8589934939 +wendy young 1 4294967395 +wendy young 2 8589934708 +xavier allen 1 4294967304 +xavier allen 2 8589934743 +xavier allen 3 12884902129 +xavier brown 1 4294967546 +xavier brown 2 8589935074 +xavier brown 3 12884902532 +xavier carson 1 4294967547 +xavier carson 2 8589934862 +xavier davidson 1 4294967361 +xavier davidson 2 8589934760 +xavier davidson 3 12884902204 +xavier ellison 1 4294967441 +xavier ellison 2 8589934914 +xavier garcia 1 4294967465 +xavier hernandez 1 4294967383 +xavier hernandez 2 8589934743 +xavier hernandez 3 12884902274 +xavier ichabod 1 4294967511 +xavier ichabod 2 8589934950 +xavier johnson 1 4294967507 +xavier johnson 2 8589934898 +xavier king 1 4294967456 +xavier king 2 8589934758 +xavier laertes 1 4294967450 +xavier ovid 1 4294967403 +xavier polk 1 4294967506 +xavier polk 2 8589934925 +xavier polk 3 12884902406 +xavier polk 4 17179869906 +xavier quirinius 1 4294967383 +xavier quirinius 2 8589934748 +xavier quirinius 3 12884902060 +xavier quirinius 4 17179869562 +xavier thompson 1 4294967444 +xavier underhill 1 4294967332 +xavier white 1 4294967473 +xavier white 2 8589934952 +xavier xylophone 1 4294967499 +xavier zipper 1 4294967547 +yuri allen 1 4294967528 +yuri allen 2 8589935079 +yuri brown 1 4294967433 +yuri brown 2 8589934960 +yuri carson 1 4294967317 +yuri carson 2 8589934851 +yuri ellison 1 4294967299 +yuri ellison 2 8589934697 +yuri falkner 1 4294967368 +yuri falkner 2 8589934891 +yuri garcia 1 4294967362 +yuri hernandez 1 4294967367 +yuri johnson 1 4294967421 +yuri johnson 2 8589934877 +yuri johnson 3 12884902361 +yuri king 1 4294967376 +yuri laertes 1 4294967402 +yuri laertes 2 8589934924 +yuri nixon 1 4294967400 +yuri nixon 2 8589934706 +yuri polk 1 4294967391 +yuri polk 2 8589934861 +yuri polk 3 12884902167 +yuri quirinius 1 4294967398 +yuri quirinius 2 8589934768 +yuri quirinius 3 12884902081 +yuri steinbeck 1 4294967535 +yuri steinbeck 2 8589934873 +yuri thompson 1 4294967447 +yuri underhill 1 4294967499 +yuri underhill 2 8589934900 +yuri white 1 4294967341 +yuri xylophone 1 4294967420 +zach allen 1 4294967507 +zach brown 1 4294967316 +zach brown 2 8589934728 +zach brown 3 12884902099 +zach brown 4 17179869452 +zach brown 5 21474836769 +zach carson 1 4294967463 +zach ellison 1 4294967471 +zach falkner 1 4294967362 +zach falkner 2 8589934717 +zach garcia 1 4294967481 +zach garcia 2 8589934854 +zach garcia 3 12884902240 +zach garcia 4 17179869723 +zach ichabod 1 4294967539 +zach ichabod 2 8589934912 +zach king 1 4294967424 +zach king 2 8589934956 +zach king 3 12884902458 +zach miller 1 4294967442 +zach miller 2 8589934772 +zach miller 3 12884902163 +zach ovid 1 4294967412 +zach ovid 2 8589934775 +zach ovid 3 12884902244 +zach ovid 4 17179869574 +zach quirinius 1 4294967299 +zach robinson 1 4294967325 +zach steinbeck 1 4294967469 +zach steinbeck 2 8589934834 +zach thompson 1 4294967405 +zach thompson 2 8589934730 +zach underhill 1 4294967496 +zach white 1 4294967501 +zach xylophone 1 4294967452 +zach xylophone 2 8589934755 +zach young 1 4294967297 +zach zipper 1 4294967497 +zach zipper 2 8589934855 +zach zipper 3 12884902222 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 3-0-455e41d9949a2d22bab634fd8e42f2b1 b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 3-0-455e41d9949a2d22bab634fd8e42f2b1 new file mode 100644 index 0000000000000..f47923618a1aa --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 3-0-455e41d9949a2d22bab634fd8e42f2b1 @@ -0,0 +1 @@ +bob steinbeck 65637 9.699999809265137 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 4-0-cfad06ae8eba6b047d32a6a61dd59392 b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 4-0-cfad06ae8eba6b047d32a6a61dd59392 new file mode 100644 index 0000000000000..f41eaa259cec0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 4-0-cfad06ae8eba6b047d32a6a61dd59392 @@ -0,0 +1 @@ +bob steinbeck 1 1 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 5-0-d7ca7a61377cef3a9f721a28afdae012 b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 5-0-d7ca7a61377cef3a9f721a28afdae012 new file mode 100644 index 0000000000000..5308b2eb457e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 5-0-d7ca7a61377cef3a9f721a28afdae012 @@ -0,0 +1 @@ +bob steinbeck 9.699999809265137 1 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 6-0-287bcc7679822bc7b684532b267bf11f b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 6-0-287bcc7679822bc7b684532b267bf11f new file mode 100644 index 0000000000000..f41eaa259cec0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 6-0-287bcc7679822bc7b684532b267bf11f @@ -0,0 +1 @@ +bob steinbeck 1 1 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-0-36217f6074daaacddb9fcb50a3f4fb5b b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-0-36217f6074daaacddb9fcb50a3f4fb5b new file mode 100644 index 0000000000000..8150409e62d33 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-0-36217f6074daaacddb9fcb50a3f4fb5b @@ -0,0 +1,1049 @@ + 1 + 1 + 1 +alice allen 2 +alice allen 1 +alice allen 1 +alice brown 1 +alice carson 1 +alice davidson 1 +alice falkner 1 +alice garcia 1 +alice hernandez 2 +alice hernandez 1 +alice johnson 2 +alice king 1 +alice king 1 +alice king 1 +alice laertes 1 +alice laertes 1 +alice miller 1 +alice nixon 1 +alice nixon 1 +alice nixon 1 +alice ovid 1 +alice polk 3 +alice quirinius 1 +alice quirinius 1 +alice robinson 1 +alice robinson 1 +alice steinbeck 1 +alice steinbeck 1 +alice steinbeck 1 +alice underhill 2 +alice van buren 1 +alice xylophone 1 +alice xylophone 1 +alice xylophone 1 +alice zipper 1 +alice zipper 1 +alice zipper 1 +bob brown 1 +bob brown 1 +bob brown 1 +bob carson 1 +bob davidson 1 +bob davidson 1 +bob davidson 1 +bob ellison 2 +bob ellison 1 +bob ellison 1 +bob ellison 1 +bob falkner 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob hernandez 1 +bob ichabod 1 +bob king 2 +bob king 1 +bob king 1 +bob laertes 2 +bob laertes 1 +bob miller 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob polk 1 +bob quirinius 1 +bob steinbeck 1 +bob van buren 1 +bob white 1 +bob white 1 +bob xylophone 1 +bob xylophone 1 +bob young 1 +bob zipper 2 +bob zipper 1 +bob zipper 1 +calvin allen 1 +calvin brown 2 +calvin brown 1 +calvin brown 1 +calvin carson 2 +calvin davidson 2 +calvin davidson 1 +calvin ellison 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin garcia 1 +calvin hernandez 2 +calvin johnson 1 +calvin laertes 1 +calvin laertes 1 +calvin nixon 3 +calvin nixon 1 +calvin nixon 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 1 +calvin polk 1 +calvin quirinius 1 +calvin quirinius 1 +calvin robinson 2 +calvin steinbeck 1 +calvin steinbeck 1 +calvin steinbeck 1 +calvin thompson 1 +calvin thompson 1 +calvin underhill 1 +calvin van buren 1 +calvin van buren 1 +calvin white 1 +calvin white 1 +calvin xylophone 2 +calvin xylophone 1 +calvin xylophone 1 +calvin young 1 +calvin young 1 +calvin zipper 1 +calvin zipper 1 +david allen 1 +david allen 1 +david brown 1 +david brown 1 +david davidson 1 +david davidson 1 +david davidson 1 +david davidson 1 +david ellison 1 +david ellison 1 +david ellison 1 +david hernandez 1 +david ichabod 1 +david ichabod 1 +david laertes 1 +david nixon 1 +david ovid 1 +david ovid 1 +david quirinius 1 +david quirinius 1 +david quirinius 1 +david robinson 1 +david robinson 1 +david thompson 3 +david underhill 1 +david underhill 1 +david underhill 1 +david van buren 1 +david van buren 1 +david white 1 +david xylophone 1 +david xylophone 1 +david xylophone 1 +david young 1 +david young 1 +ethan allen 1 +ethan brown 2 +ethan brown 2 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan carson 1 +ethan ellison 1 +ethan ellison 1 +ethan falkner 3 +ethan falkner 1 +ethan garcia 1 +ethan hernandez 1 +ethan johnson 1 +ethan king 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan miller 2 +ethan nixon 1 +ethan ovid 1 +ethan polk 2 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan quirinius 1 +ethan quirinius 1 +ethan quirinius 1 +ethan robinson 1 +ethan robinson 1 +ethan underhill 1 +ethan van buren 2 +ethan white 1 +ethan white 1 +ethan xylophone 1 +ethan zipper 1 +ethan zipper 1 +fred davidson 1 +fred davidson 1 +fred davidson 1 +fred ellison 1 +fred ellison 1 +fred ellison 1 +fred falkner 2 +fred falkner 1 +fred falkner 1 +fred hernandez 1 +fred ichabod 1 +fred ichabod 1 +fred johnson 1 +fred king 2 +fred king 1 +fred laertes 1 +fred miller 1 +fred nixon 2 +fred nixon 1 +fred nixon 1 +fred nixon 1 +fred polk 1 +fred polk 1 +fred polk 1 +fred polk 1 +fred quirinius 1 +fred quirinius 1 +fred robinson 1 +fred steinbeck 1 +fred steinbeck 1 +fred steinbeck 1 +fred underhill 1 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred white 1 +fred young 3 +fred young 1 +fred zipper 1 +gabriella allen 1 +gabriella allen 1 +gabriella brown 2 +gabriella brown 1 +gabriella carson 1 +gabriella davidson 1 +gabriella ellison 1 +gabriella ellison 1 +gabriella falkner 1 +gabriella falkner 1 +gabriella falkner 1 +gabriella garcia 1 +gabriella hernandez 1 +gabriella hernandez 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella king 1 +gabriella king 1 +gabriella laertes 1 +gabriella miller 1 +gabriella ovid 1 +gabriella ovid 1 +gabriella polk 1 +gabriella polk 1 +gabriella steinbeck 1 +gabriella steinbeck 1 +gabriella thompson 1 +gabriella thompson 1 +gabriella thompson 1 +gabriella van buren 2 +gabriella van buren 1 +gabriella white 1 +gabriella young 1 +gabriella young 1 +gabriella zipper 1 +gabriella zipper 1 +holly allen 1 +holly brown 1 +holly brown 1 +holly falkner 1 +holly hernandez 1 +holly hernandez 1 +holly hernandez 1 +holly hernandez 1 +holly ichabod 1 +holly ichabod 1 +holly ichabod 1 +holly johnson 2 +holly johnson 1 +holly johnson 1 +holly king 1 +holly king 1 +holly laertes 1 +holly miller 1 +holly nixon 1 +holly nixon 1 +holly polk 2 +holly polk 1 +holly robinson 2 +holly thompson 1 +holly thompson 1 +holly thompson 1 +holly underhill 1 +holly underhill 1 +holly underhill 1 +holly underhill 1 +holly van buren 1 +holly white 1 +holly white 1 +holly xylophone 1 +holly young 1 +holly young 1 +holly zipper 1 +holly zipper 1 +irene allen 1 +irene brown 1 +irene brown 1 +irene brown 1 +irene carson 1 +irene ellison 1 +irene ellison 1 +irene falkner 1 +irene falkner 1 +irene garcia 1 +irene garcia 1 +irene garcia 1 +irene ichabod 1 +irene ichabod 1 +irene johnson 2 +irene laertes 1 +irene laertes 1 +irene laertes 1 +irene miller 2 +irene nixon 1 +irene nixon 1 +irene nixon 1 +irene ovid 1 +irene ovid 1 +irene ovid 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene quirinius 1 +irene quirinius 1 +irene quirinius 1 +irene robinson 1 +irene steinbeck 1 +irene thompson 2 +irene underhill 2 +irene underhill 1 +irene van buren 1 +irene van buren 1 +irene xylophone 1 +jessica brown 2 +jessica carson 1 +jessica carson 1 +jessica carson 1 +jessica davidson 2 +jessica davidson 1 +jessica davidson 1 +jessica davidson 1 +jessica ellison 2 +jessica ellison 1 +jessica falkner 1 +jessica garcia 1 +jessica garcia 1 +jessica ichabod 1 +jessica johnson 1 +jessica johnson 1 +jessica miller 1 +jessica nixon 1 +jessica nixon 1 +jessica ovid 2 +jessica ovid 1 +jessica polk 1 +jessica quirinius 1 +jessica quirinius 1 +jessica quirinius 1 +jessica quirinius 1 +jessica robinson 1 +jessica thompson 1 +jessica thompson 1 +jessica underhill 1 +jessica underhill 1 +jessica underhill 1 +jessica van buren 1 +jessica white 2 +jessica white 1 +jessica white 1 +jessica white 1 +jessica white 1 +jessica xylophone 1 +jessica young 1 +jessica young 1 +jessica zipper 1 +jessica zipper 1 +jessica zipper 1 +katie allen 1 +katie brown 1 +katie davidson 1 +katie ellison 1 +katie ellison 1 +katie falkner 1 +katie garcia 1 +katie garcia 1 +katie hernandez 1 +katie ichabod 2 +katie ichabod 1 +katie ichabod 1 +katie king 1 +katie king 1 +katie king 1 +katie miller 1 +katie miller 1 +katie nixon 1 +katie ovid 1 +katie polk 1 +katie polk 1 +katie robinson 1 +katie van buren 2 +katie van buren 1 +katie white 1 +katie white 1 +katie xylophone 1 +katie young 1 +katie young 1 +katie young 1 +katie zipper 1 +katie zipper 1 +luke allen 2 +luke allen 1 +luke allen 1 +luke allen 1 +luke allen 1 +luke brown 2 +luke davidson 1 +luke davidson 1 +luke ellison 1 +luke ellison 1 +luke ellison 1 +luke falkner 2 +luke falkner 1 +luke garcia 1 +luke garcia 1 +luke ichabod 1 +luke ichabod 1 +luke johnson 1 +luke johnson 1 +luke johnson 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke miller 2 +luke ovid 2 +luke ovid 1 +luke polk 1 +luke polk 1 +luke quirinius 1 +luke robinson 1 +luke robinson 1 +luke thompson 1 +luke underhill 1 +luke underhill 1 +luke underhill 1 +luke van buren 2 +luke white 1 +luke xylophone 1 +luke zipper 1 +mike allen 2 +mike brown 1 +mike carson 1 +mike carson 1 +mike carson 1 +mike davidson 1 +mike davidson 1 +mike ellison 2 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike falkner 1 +mike garcia 1 +mike garcia 1 +mike garcia 1 +mike hernandez 1 +mike hernandez 1 +mike ichabod 1 +mike king 2 +mike king 1 +mike king 1 +mike king 1 +mike king 1 +mike king 1 +mike miller 1 +mike nixon 2 +mike nixon 1 +mike polk 2 +mike polk 1 +mike polk 1 +mike quirinius 1 +mike steinbeck 1 +mike steinbeck 1 +mike steinbeck 1 +mike steinbeck 1 +mike van buren 2 +mike van buren 1 +mike white 1 +mike white 1 +mike white 1 +mike white 1 +mike young 1 +mike young 1 +mike young 1 +mike zipper 1 +mike zipper 1 +mike zipper 1 +nick allen 1 +nick allen 1 +nick brown 1 +nick davidson 1 +nick ellison 2 +nick ellison 1 +nick falkner 1 +nick falkner 1 +nick garcia 1 +nick garcia 1 +nick garcia 1 +nick ichabod 1 +nick ichabod 1 +nick ichabod 1 +nick johnson 1 +nick johnson 1 +nick laertes 1 +nick miller 1 +nick nixon 1 +nick ovid 1 +nick polk 2 +nick quirinius 2 +nick quirinius 1 +nick robinson 1 +nick robinson 1 +nick steinbeck 1 +nick thompson 2 +nick underhill 1 +nick van buren 1 +nick xylophone 1 +nick young 1 +nick young 1 +nick zipper 2 +nick zipper 1 +oscar allen 1 +oscar brown 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar davidson 1 +oscar ellison 2 +oscar ellison 2 +oscar falkner 1 +oscar garcia 1 +oscar hernandez 1 +oscar hernandez 1 +oscar ichabod 2 +oscar ichabod 1 +oscar ichabod 1 +oscar ichabod 1 +oscar johnson 2 +oscar johnson 1 +oscar king 1 +oscar king 1 +oscar king 1 +oscar laertes 1 +oscar laertes 1 +oscar laertes 1 +oscar laertes 1 +oscar nixon 1 +oscar ovid 1 +oscar ovid 1 +oscar ovid 1 +oscar polk 1 +oscar polk 1 +oscar quirinius 2 +oscar quirinius 2 +oscar quirinius 1 +oscar quirinius 1 +oscar robinson 2 +oscar robinson 1 +oscar robinson 1 +oscar robinson 1 +oscar steinbeck 1 +oscar thompson 1 +oscar thompson 1 +oscar thompson 1 +oscar thompson 1 +oscar underhill 1 +oscar van buren 1 +oscar van buren 1 +oscar van buren 1 +oscar white 1 +oscar white 1 +oscar white 1 +oscar white 1 +oscar xylophone 2 +oscar xylophone 1 +oscar xylophone 1 +oscar zipper 2 +oscar zipper 1 +oscar zipper 1 +priscilla brown 2 +priscilla brown 1 +priscilla brown 1 +priscilla carson 2 +priscilla carson 1 +priscilla carson 1 +priscilla ichabod 2 +priscilla ichabod 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla king 1 +priscilla nixon 2 +priscilla nixon 2 +priscilla ovid 2 +priscilla ovid 1 +priscilla polk 1 +priscilla quirinius 1 +priscilla thompson 1 +priscilla underhill 1 +priscilla underhill 1 +priscilla van buren 2 +priscilla van buren 1 +priscilla van buren 1 +priscilla white 1 +priscilla xylophone 1 +priscilla xylophone 1 +priscilla xylophone 1 +priscilla young 1 +priscilla young 1 +priscilla zipper 1 +priscilla zipper 1 +quinn allen 1 +quinn allen 1 +quinn brown 1 +quinn brown 1 +quinn brown 1 +quinn davidson 2 +quinn davidson 1 +quinn davidson 1 +quinn davidson 1 +quinn ellison 1 +quinn ellison 1 +quinn garcia 1 +quinn garcia 1 +quinn garcia 1 +quinn garcia 1 +quinn ichabod 1 +quinn king 1 +quinn king 1 +quinn laertes 1 +quinn laertes 1 +quinn laertes 1 +quinn nixon 2 +quinn ovid 1 +quinn quirinius 1 +quinn robinson 2 +quinn steinbeck 1 +quinn steinbeck 1 +quinn thompson 1 +quinn thompson 1 +quinn underhill 2 +quinn underhill 1 +quinn underhill 1 +quinn van buren 1 +quinn young 2 +quinn zipper 1 +quinn zipper 1 +rachel allen 1 +rachel allen 1 +rachel brown 3 +rachel brown 1 +rachel brown 1 +rachel brown 1 +rachel brown 1 +rachel carson 2 +rachel carson 1 +rachel davidson 1 +rachel ellison 1 +rachel falkner 1 +rachel falkner 1 +rachel falkner 1 +rachel falkner 1 +rachel johnson 1 +rachel king 2 +rachel king 1 +rachel laertes 1 +rachel laertes 1 +rachel ovid 1 +rachel ovid 1 +rachel polk 1 +rachel quirinius 1 +rachel robinson 1 +rachel robinson 1 +rachel robinson 1 +rachel thompson 1 +rachel thompson 1 +rachel thompson 1 +rachel underhill 1 +rachel white 1 +rachel white 1 +rachel young 1 +rachel zipper 1 +rachel zipper 1 +sarah carson 1 +sarah carson 1 +sarah carson 1 +sarah ellison 1 +sarah falkner 1 +sarah falkner 1 +sarah garcia 1 +sarah garcia 1 +sarah garcia 1 +sarah ichabod 1 +sarah ichabod 1 +sarah johnson 1 +sarah johnson 1 +sarah johnson 1 +sarah johnson 1 +sarah king 1 +sarah king 1 +sarah miller 1 +sarah ovid 1 +sarah robinson 1 +sarah robinson 1 +sarah steinbeck 1 +sarah white 1 +sarah white 1 +sarah xylophone 1 +sarah young 1 +sarah zipper 1 +tom brown 1 +tom brown 1 +tom carson 1 +tom carson 1 +tom carson 1 +tom davidson 1 +tom ellison 1 +tom ellison 1 +tom ellison 1 +tom falkner 1 +tom falkner 1 +tom hernandez 1 +tom hernandez 1 +tom ichabod 1 +tom johnson 1 +tom johnson 1 +tom king 1 +tom laertes 2 +tom laertes 1 +tom miller 2 +tom miller 1 +tom miller 1 +tom nixon 2 +tom ovid 1 +tom polk 1 +tom polk 1 +tom quirinius 1 +tom quirinius 1 +tom robinson 1 +tom robinson 1 +tom robinson 1 +tom robinson 1 +tom steinbeck 3 +tom van buren 1 +tom van buren 1 +tom van buren 1 +tom white 2 +tom young 1 +tom young 1 +tom zipper 3 +ulysses brown 1 +ulysses carson 4 +ulysses carson 1 +ulysses carson 1 +ulysses carson 1 +ulysses davidson 2 +ulysses ellison 1 +ulysses garcia 1 +ulysses hernandez 1 +ulysses hernandez 1 +ulysses hernandez 1 +ulysses ichabod 1 +ulysses ichabod 1 +ulysses johnson 2 +ulysses king 1 +ulysses laertes 2 +ulysses laertes 1 +ulysses laertes 1 +ulysses miller 1 +ulysses miller 1 +ulysses nixon 1 +ulysses ovid 1 +ulysses polk 1 +ulysses polk 1 +ulysses polk 1 +ulysses polk 1 +ulysses quirinius 1 +ulysses robinson 1 +ulysses steinbeck 1 +ulysses steinbeck 1 +ulysses thompson 1 +ulysses underhill 2 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses van buren 1 +ulysses white 1 +ulysses white 1 +ulysses xylophone 2 +ulysses xylophone 1 +ulysses xylophone 1 +ulysses young 2 +ulysses young 1 +ulysses young 1 +victor allen 1 +victor allen 1 +victor brown 1 +victor brown 1 +victor brown 1 +victor brown 1 +victor davidson 1 +victor davidson 1 +victor davidson 1 +victor ellison 2 +victor ellison 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor johnson 2 +victor johnson 1 +victor johnson 1 +victor king 1 +victor king 1 +victor laertes 1 +victor laertes 1 +victor miller 1 +victor nixon 1 +victor nixon 1 +victor ovid 2 +victor polk 2 +victor quirinius 1 +victor quirinius 1 +victor robinson 2 +victor robinson 1 +victor steinbeck 2 +victor steinbeck 1 +victor steinbeck 1 +victor thompson 1 +victor van buren 1 +victor van buren 1 +victor white 2 +victor white 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor young 1 +victor zipper 1 +wendy allen 1 +wendy allen 1 +wendy allen 1 +wendy brown 1 +wendy brown 1 +wendy ellison 1 +wendy ellison 1 +wendy falkner 2 +wendy falkner 1 +wendy falkner 1 +wendy garcia 2 +wendy garcia 1 +wendy garcia 1 +wendy garcia 1 +wendy hernandez 1 +wendy ichabod 1 +wendy king 1 +wendy king 1 +wendy king 1 +wendy laertes 1 +wendy laertes 1 +wendy laertes 1 +wendy miller 1 +wendy miller 1 +wendy nixon 1 +wendy nixon 1 +wendy ovid 1 +wendy ovid 1 +wendy polk 2 +wendy polk 1 +wendy quirinius 1 +wendy quirinius 1 +wendy robinson 2 +wendy robinson 1 +wendy robinson 1 +wendy steinbeck 1 +wendy thompson 2 +wendy thompson 1 +wendy underhill 2 +wendy underhill 1 +wendy underhill 1 +wendy van buren 1 +wendy van buren 1 +wendy white 1 +wendy xylophone 1 +wendy xylophone 1 +wendy young 1 +wendy young 1 +xavier allen 1 +xavier allen 1 +xavier allen 1 +xavier brown 1 +xavier brown 1 +xavier brown 1 +xavier carson 1 +xavier carson 1 +xavier davidson 1 +xavier davidson 1 +xavier davidson 1 +xavier ellison 1 +xavier ellison 1 +xavier garcia 1 +xavier hernandez 1 +xavier hernandez 1 +xavier hernandez 1 +xavier ichabod 1 +xavier ichabod 1 +xavier johnson 1 +xavier johnson 1 +xavier king 1 +xavier king 1 +xavier laertes 1 +xavier ovid 2 +xavier polk 1 +xavier polk 1 +xavier polk 1 +xavier polk 1 +xavier quirinius 2 +xavier quirinius 2 +xavier quirinius 1 +xavier quirinius 1 +xavier thompson 1 +xavier underhill 1 +xavier white 2 +xavier white 1 +xavier xylophone 1 +xavier zipper 1 +yuri allen 1 +yuri allen 1 +yuri brown 1 +yuri brown 1 +yuri carson 1 +yuri carson 1 +yuri ellison 1 +yuri ellison 1 +yuri falkner 1 +yuri falkner 1 +yuri garcia 1 +yuri hernandez 1 +yuri johnson 1 +yuri johnson 1 +yuri johnson 1 +yuri king 2 +yuri laertes 1 +yuri laertes 1 +yuri nixon 1 +yuri nixon 1 +yuri polk 1 +yuri polk 1 +yuri polk 1 +yuri quirinius 1 +yuri quirinius 1 +yuri quirinius 1 +yuri steinbeck 1 +yuri steinbeck 1 +yuri thompson 1 +yuri underhill 1 +yuri underhill 1 +yuri white 1 +yuri xylophone 1 +zach allen 1 +zach brown 2 +zach brown 1 +zach brown 1 +zach brown 1 +zach brown 1 +zach carson 2 +zach ellison 1 +zach falkner 1 +zach falkner 1 +zach garcia 2 +zach garcia 1 +zach garcia 1 +zach garcia 1 +zach ichabod 1 +zach ichabod 1 +zach king 2 +zach king 1 +zach king 1 +zach miller 1 +zach miller 1 +zach miller 1 +zach ovid 1 +zach ovid 1 +zach ovid 1 +zach ovid 1 +zach quirinius 1 +zach robinson 1 +zach steinbeck 1 +zach steinbeck 1 +zach thompson 2 +zach thompson 1 +zach underhill 1 +zach white 1 +zach xylophone 2 +zach xylophone 1 +zach young 1 +zach zipper 1 +zach zipper 1 +zach zipper 1 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-1-9ee79e711248dd6e0a6ce27e439e55f4 b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-1-9ee79e711248dd6e0a6ce27e439e55f4 new file mode 100644 index 0000000000000..275772e1f643a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-1-9ee79e711248dd6e0a6ce27e439e55f4 @@ -0,0 +1,1049 @@ +65791 calvin nixon +65791 katie garcia +65790 fred nixon +65790 victor polk +65790 yuri ellison +65789 NULL +65789 alice laertes +65789 gabriella king +65789 katie zipper +65789 oscar zipper +65789 quinn davidson +65789 wendy brown +65788 irene brown +65788 oscar zipper +65788 rachel king +65788 xavier thompson +65787 alice laertes +65787 david laertes +65787 katie ichabod +65787 ulysses king +65786 bob carson +65786 quinn king +65786 victor xylophone +65786 xavier allen +65786 xavier davidson +65785 sarah carson +65785 sarah johnson +65784 holly young +65784 jessica ellison +65784 jessica ovid +65784 jessica zipper +65784 quinn van buren +65783 david allen +65783 fred ellison +65783 irene nixon +65783 mike ichabod +65783 wendy miller +65783 zach garcia +65782 tom miller +65782 ulysses underhill +65782 victor nixon +65782 yuri white +65782 zach ovid +65781 ethan carson +65781 luke laertes +65781 quinn brown +65780 holly hernandez +65780 holly zipper +65780 wendy falkner +65779 gabriella ichabod +65779 irene brown +65779 irene underhill +65779 ulysses xylophone +65779 wendy hernandez +65779 yuri nixon +65779 zach ellison +65778 jessica davidson +65778 katie allen +65778 oscar van buren +65778 oscar white +65778 rachel johnson +65778 victor quirinius +65778 yuri polk +65778 yuri quirinius +65777 ethan garcia +65777 irene garcia +65777 katie polk +65777 rachel thompson +65776 NULL +65776 luke garcia +65776 luke quirinius +65776 priscilla ovid +65776 sarah king +65775 bob ellison +65775 calvin steinbeck +65775 ethan laertes +65775 luke robinson +65775 oscar polk +65774 calvin young +65774 irene quirinius +65774 katie brown +65774 oscar hernandez +65774 sarah ichabod +65774 zach king +65773 NULL +65773 calvin garcia +65773 irene polk +65773 jessica davidson +65773 nick ichabod +65773 nick zipper +65773 quinn ovid +65773 zach falkner +65772 oscar garcia +65771 ethan underhill +65771 yuri hernandez +65771 zach brown +65770 alice nixon +65770 gabriella ellison +65769 mike king +65769 nick steinbeck +65769 oscar nixon +65769 rachel zipper +65769 xavier quirinius +65769 zach ichabod +65768 jessica davidson +65767 tom robinson +65767 victor davidson +65767 xavier king +65766 fred davidson +65766 luke underhill +65766 ulysses carson +65766 wendy nixon +65766 xavier brown +65765 bob polk +65765 gabriella brown +65765 wendy allen +65764 alice hernandez +65764 alice robinson +65764 luke ellison +65763 calvin van buren +65763 ethan brown +65763 priscilla johnson +65763 tom ichabod +65763 tom miller +65762 david xylophone +65762 priscilla xylophone +65762 quinn allen +65762 sarah johnson +65762 tom carson +65761 david ovid +65760 NULL +65760 ethan king +65760 priscilla johnson +65760 wendy van buren +65759 alice xylophone +65759 ethan quirinius +65759 fred steinbeck +65759 sarah robinson +65759 xavier johnson +65758 alice van buren +65758 ethan white +65758 irene johnson +65758 jessica underhill +65758 rachel laertes +65757 bob quirinius +65757 nick quirinius +65757 tom van buren +65757 xavier carson +65757 zach brown +65756 gabriella falkner +65756 mike polk +65756 nick allen +65756 yuri xylophone +65755 alice johnson +65755 david ellison +65755 fred polk +65755 gabriella hernandez +65755 irene van buren +65755 jessica nixon +65755 mike ellison +65755 nick van buren +65755 zach miller +65754 luke johnson +65754 xavier quirinius +65753 rachel robinson +65753 yuri garcia +65752 bob miller +65752 oscar zipper +65751 calvin quirinius +65751 katie king +65751 mike allen +65751 mike quirinius +65751 mike white +65751 zach zipper +65750 mike white +65750 nick thompson +65750 oscar quirinius +65750 priscilla polk +65750 rachel brown +65749 david allen +65749 gabriella polk +65749 mike ellison +65749 sarah carson +65749 wendy thompson +65748 david davidson +65748 oscar laertes +65747 calvin falkner +65747 fred steinbeck +65747 priscilla zipper +65747 quinn underhill +65747 rachel falkner +65747 wendy falkner +65747 yuri falkner +65746 +65746 katie robinson +65746 luke garcia +65746 tom brown +65746 zach steinbeck +65745 oscar steinbeck +65745 oscar xylophone +65745 priscilla nixon +65745 victor laertes +65744 alice allen +65744 priscilla ichabod +65743 jessica carson +65743 oscar ichabod +65743 sarah falkner +65743 tom brown +65742 jessica brown +65742 jessica ellison +65742 wendy polk +65741 calvin thompson +65740 irene van buren +65740 mike zipper +65740 rachel quirinius +65739 gabriella van buren +65739 victor robinson +65739 wendy nixon +65738 sarah ichabod +65738 wendy robinson +65738 wendy young +65738 xavier king +65737 holly brown +65737 irene falkner +65737 jessica white +65737 quinn garcia +65737 wendy laertes +65736 fred young +65736 quinn laertes +65736 rachel allen +65736 victor ellison +65735 oscar laertes +65735 tom falkner +65735 ulysses thompson +65734 calvin falkner +65734 mike garcia +65733 gabriella van buren +65733 holly nixon +65733 luke underhill +65733 oscar allen +65733 xavier polk +65733 yuri ellison +65732 mike steinbeck +65732 tom carson +65732 tom ellison +65731 alice robinson +65731 priscilla carson +65731 tom johnson +65731 ulysses ovid +65730 bob king +65730 calvin polk +65730 gabriella ichabod +65730 rachel brown +65729 fred quirinius +65728 priscilla king +65728 victor brown +65727 ethan brown +65727 fred polk +65727 jessica white +65727 priscilla underhill +65727 sarah white +65726 mike garcia +65726 ulysses hernandez +65726 wendy quirinius +65725 zach thompson +65724 NULL +65724 alice king +65724 alice steinbeck +65724 mike ellison +65724 nick ovid +65724 priscilla zipper +65723 nick allen +65723 rachel white +65722 fred falkner +65722 jessica underhill +65722 luke ichabod +65721 ethan falkner +65721 jessica zipper +65721 luke laertes +65721 luke underhill +65721 mike carson +65721 oscar laertes +65721 ulysses ichabod +65720 calvin nixon +65720 calvin thompson +65720 gabriella young +65720 holly polk +65720 mike king +65719 bob brown +65719 holly ichabod +65719 ulysses ellison +65719 ulysses young +65718 jessica ichabod +65718 oscar johnson +65718 victor hernandez +65718 zach ovid +65717 holly hernandez +65717 mike nixon +65717 quinn garcia +65717 ulysses davidson +65717 ulysses polk +65716 ethan zipper +65716 holly xylophone +65716 jessica garcia +65716 nick falkner +65716 sarah king +65716 victor xylophone +65715 alice hernandez +65715 david young +65715 ethan polk +65715 oscar falkner +65715 priscilla brown +65714 NULL +65714 ethan laertes +65714 luke allen +65713 bob ellison +65713 nick nixon +65713 ulysses carson +65713 ulysses ichabod +65713 zach garcia +65712 NULL +65712 katie king +65712 luke davidson +65712 quinn garcia +65712 zach carson +65711 bob zipper +65711 fred miller +65711 holly nixon +65711 katie ellison +65711 wendy ovid +65711 zach zipper +65710 luke polk +65710 mike young +65709 bob laertes +65709 calvin laertes +65709 oscar laertes +65708 gabriella laertes +65708 priscilla van buren +65707 bob garcia +65707 mike steinbeck +65706 bob ellison +65706 bob xylophone +65706 luke allen +65706 ulysses xylophone +65706 wendy ichabod +65705 david ellison +65705 mike white +65705 priscilla johnson +65705 tom ellison +65704 ethan brown +65704 holly king +65704 jessica young +65704 tom steinbeck +65704 victor steinbeck +65704 zach falkner +65703 holly underhill +65703 mike ellison +65703 victor davidson +65703 xavier underhill +65702 NULL +65702 bob brown +65702 bob zipper +65702 ulysses quirinius +65701 alice allen +65701 mike steinbeck +65701 oscar thompson +65700 luke allen +65700 quinn laertes +65700 tom quirinius +65699 ethan brown +65699 ethan van buren +65699 irene laertes +65699 mike young +65699 nick garcia +65699 sarah white +65698 gabriella thompson +65698 nick polk +65697 NULL +65697 NULL +65697 holly miller +65697 oscar quirinius +65697 oscar thompson +65697 xavier davidson +65696 alice xylophone +65696 jessica davidson +65696 luke allen +65696 oscar xylophone +65695 oscar king +65695 rachel young +65695 wendy ellison +65695 yuri quirinius +65694 david brown +65694 holly underhill +65694 victor quirinius +65694 zach brown +65693 bob hernandez +65693 bob young +65693 david brown +65693 holly hernandez +65693 tom polk +65693 ulysses polk +65693 victor brown +65692 holly johnson +65692 tom robinson +65691 calvin ovid +65691 ethan nixon +65691 ethan robinson +65691 fred underhill +65691 holly white +65691 irene polk +65691 oscar white +65691 rachel brown +65690 fred van buren +65690 jessica quirinius +65689 oscar ovid +65689 wendy thompson +65688 bob steinbeck +65688 victor steinbeck +65687 gabriella ichabod +65687 jessica underhill +65687 mike zipper +65687 quinn thompson +65686 bob king +65686 bob zipper +65686 david quirinius +65686 luke ichabod +65685 ethan robinson +65685 gabriella hernandez +65685 katie garcia +65685 sarah ellison +65685 victor hernandez +65685 victor nixon +65684 priscilla brown +65684 victor laertes +65684 wendy van buren +65683 NULL +65683 mike king +65683 tom laertes +65682 calvin quirinius +65682 ethan brown +65682 katie ellison +65681 gabriella allen +65681 luke laertes +65681 oscar quirinius +65681 ulysses laertes +65681 wendy ellison +65681 xavier polk +65680 NULL +65680 alice nixon +65680 gabriella ovid +65680 jessica carson +65680 ulysses nixon +65680 zach zipper +65679 bob garcia +65679 wendy underhill +65678 bob falkner +65678 victor xylophone +65678 wendy king +65677 alice allen +65677 fred van buren +65677 mike brown +65677 nick xylophone +65677 ulysses underhill +65677 zach robinson +65676 bob davidson +65676 bob laertes +65676 tom ovid +65676 xavier johnson +65675 david hernandez +65675 david nixon +65675 holly falkner +65675 quinn steinbeck +65675 rachel robinson +65675 sarah zipper +65675 tom polk +65675 victor allen +65674 gabriella falkner +65673 nick johnson +65673 quinn brown +65673 quinn underhill +65673 rachel ovid +65673 wendy brown +65672 nick laertes +65672 nick underhill +65672 rachel zipper +65672 tom white +65672 victor king +65671 fred ellison +65671 fred falkner +65671 zach white +65670 david robinson +65670 jessica zipper +65670 luke van buren +65670 oscar ovid +65670 quinn steinbeck +65669 NULL +65669 alice king +65669 calvin hernandez +65669 katie polk +65669 nick miller +65669 oscar van buren +65668 luke ellison +65667 bob brown +65667 irene nixon +65667 oscar brown +65667 tom falkner +65666 +65666 david underhill +65666 fred van buren +65665 rachel brown +65664 NULL +65664 bob davidson +65664 david ichabod +65664 ethan laertes +65664 irene robinson +65664 mike carson +65664 priscilla young +65664 victor king +65663 calvin underhill +65663 jessica johnson +65663 priscilla carson +65663 zach ichabod +65662 ethan allen +65662 katie ovid +65662 oscar johnson +65662 ulysses carson +65662 ulysses polk +65662 victor ovid +65661 david van buren +65661 luke xylophone +65661 mike falkner +65661 priscilla van buren +65661 victor johnson +65660 holly ichabod +65660 priscilla johnson +65660 victor thompson +65659 david robinson +65659 gabriella king +65659 luke davidson +65659 mike king +65659 mike zipper +65659 nick brown +65659 nick zipper +65659 yuri underhill +65658 NULL +65658 alice zipper +65658 calvin allen +65658 calvin johnson +65658 jessica garcia +65658 quinn davidson +65658 sarah ovid +65658 ulysses brown +65658 ulysses miller +65658 yuri king +65657 ethan falkner +65657 holly zipper +65657 irene ovid +65657 luke ovid +65657 priscilla white +65656 david davidson +65656 irene ovid +65656 jessica xylophone +65656 luke laertes +65656 oscar ichabod +65656 xavier ellison +65655 calvin falkner +65655 yuri laertes +65654 alice carson +65654 alice quirinius +65654 gabriella falkner +65654 nick young +65654 oscar robinson +65654 quinn robinson +65654 rachel falkner +65654 tom laertes +65654 yuri johnson +65653 calvin ellison +65653 holly underhill +65653 ulysses polk +65653 wendy xylophone +65652 NULL +65652 tom ellison +65652 victor johnson +65651 NULL +65651 ethan laertes +65651 ethan laertes +65651 irene garcia +65651 mike young +65650 irene ellison +65650 oscar white +65650 sarah steinbeck +65650 ulysses underhill +65650 ulysses xylophone +65650 victor xylophone +65649 irene underhill +65649 priscilla quirinius +65649 quinn ellison +65649 tom quirinius +65648 alice nixon +65648 calvin brown +65648 sarah carson +65648 xavier ellison +65647 irene allen +65647 mike ellison +65646 bob ovid +65646 xavier brown +65646 xavier ovid +65645 jessica quirinius +65645 katie miller +65645 ulysses hernandez +65644 alice king +65644 calvin ovid +65644 jessica white +65644 katie van buren +65644 sarah young +65644 ulysses hernandez +65644 yuri carson +65643 david davidson +65643 ethan polk +65643 ethan zipper +65643 gabriella ichabod +65643 mike davidson +65643 mike hernandez +65643 oscar robinson +65643 priscilla underhill +65643 zach king +65642 gabriella thompson +65641 fred laertes +65641 sarah garcia +65641 tom miller +65641 xavier hernandez +65640 david underhill +65639 wendy garcia +65638 fred nixon +65638 luke polk +65638 rachel carson +65637 alice underhill +65637 david davidson +65637 fred davidson +65637 gabriella davidson +65637 oscar carson +65637 rachel laertes +65637 sarah garcia +65637 wendy garcia +65636 +65636 irene polk +65636 wendy allen +65635 alice steinbeck +65635 alice zipper +65635 ulysses white +65634 NULL +65634 calvin white +65634 holly underhill +65634 sarah falkner +65633 NULL +65633 holly polk +65633 jessica nixon +65633 oscar quirinius +65632 alice falkner +65632 zach quirinius +65631 fred ellison +65630 rachel brown +65630 xavier hernandez +65629 jessica quirinius +65629 priscilla carson +65629 victor young +65629 xavier carson +65628 bob ovid +65628 ethan ovid +65628 irene ichabod +65628 oscar hernandez +65628 oscar robinson +65628 xavier quirinius +65627 alice miller +65627 holly johnson +65627 luke falkner +65627 yuri polk +65626 ethan polk +65626 holly robinson +65626 tom young +65626 yuri johnson +65625 david xylophone +65625 fred ichabod +65625 katie white +65625 ulysses garcia +65624 calvin steinbeck +65624 calvin xylophone +65624 rachel carson +65624 tom van buren +65624 yuri brown +65623 alice quirinius +65623 jessica miller +65623 oscar ichabod +65623 quinn zipper +65623 tom van buren +65623 victor brown +65623 wendy young +65622 nick davidson +65622 rachel king +65622 wendy robinson +65622 xavier ichabod +65622 zach xylophone +65622 zach young +65621 quinn underhill +65621 ulysses young +65620 nick garcia +65620 oscar thompson +65620 quinn quirinius +65620 victor white +65620 victor xylophone +65620 wendy quirinius +65619 calvin brown +65619 gabriella polk +65619 oscar king +65619 ulysses miller +65619 ulysses robinson +65619 ulysses steinbeck +65618 gabriella ovid +65618 irene laertes +65618 katie king +65618 oscar ovid +65618 quinn thompson +65617 fred van buren +65617 gabriella carson +65617 sarah johnson +65617 ulysses underhill +65616 calvin steinbeck +65616 xavier ichabod +65615 alice ovid +65615 david quirinius +65615 irene quirinius +65615 katie nixon +65614 wendy king +65614 xavier quirinius +65614 xavier white +65613 xavier zipper +65612 irene miller +65612 victor hernandez +65612 wendy white +65612 yuri polk +65611 ethan johnson +65611 fred zipper +65611 irene carson +65611 nick quirinius +65610 tom king +65610 victor steinbeck +65610 wendy garcia +65610 yuri carson +65610 zach ovid +65609 sarah robinson +65608 katie van buren +65608 mike van buren +65608 quinn ichabod +65608 zach underhill +65607 katie miller +65607 luke falkner +65607 mike polk +65607 priscilla xylophone +65607 yuri allen +65607 yuri allen +65606 bob white +65606 gabriella white +65606 oscar carson +65606 victor white +65606 xavier allen +65606 zach allen +65605 holly king +65604 katie zipper +65604 oscar davidson +65604 wendy laertes +65604 zach brown +65603 alice davidson +65603 ethan miller +65603 katie davidson +65603 katie young +65603 mike garcia +65602 NULL +65602 calvin laertes +65602 ethan laertes +65602 fred steinbeck +65602 jessica young +65602 xavier brown +65601 priscilla ovid +65601 sarah xylophone +65601 tom robinson +65600 gabriella thompson +65600 jessica polk +65600 nick robinson +65600 rachel allen +65599 fred quirinius +65599 luke johnson +65599 nick garcia +65599 oscar xylophone +65599 ulysses underhill +65598 ulysses van buren +65598 victor zipper +65597 ethan ellison +65597 nick ellison +65597 quinn davidson +65596 NULL +65596 calvin zipper +65596 david ellison +65596 irene ichabod +65596 wendy laertes +65595 bob white +65595 holly hernandez +65595 luke brown +65595 oscar ellison +65595 oscar ichabod +65595 quinn ellison +65594 gabriella ellison +65594 oscar robinson +65594 ulysses underhill +65594 victor robinson +65593 oscar white +65593 zach xylophone +65592 calvin xylophone +65591 alice zipper +65591 nick ichabod +65591 priscilla ichabod +65591 rachel underhill +65590 NULL +65590 katie falkner +65590 oscar van buren +65590 xavier garcia +65590 yuri underhill +65589 ethan white +65589 gabriella zipper +65589 irene ovid +65589 oscar king +65589 wendy xylophone +65588 bob van buren +65588 david ichabod +65588 mike miller +65588 tom hernandez +65588 victor van buren +65587 bob garcia +65587 luke johnson +65587 mike king +65587 victor allen +65587 xavier white +65586 david young +65586 irene brown +65586 priscilla brown +65586 wendy allen +65586 xavier laertes +65585 alice garcia +65585 bob garcia +65585 ethan ellison +65585 nick ellison +65585 priscilla thompson +65584 jessica carson +65584 jessica van buren +65584 jessica white +65583 bob xylophone +65583 nick ichabod +65583 yuri brown +65583 yuri steinbeck +65582 holly johnson +65582 mike carson +65582 victor van buren +65582 zach miller +65581 gabriella steinbeck +65581 irene quirinius +65581 luke allen +65581 nick robinson +65581 nick young +65581 wendy robinson +65580 alice steinbeck +65580 alice xylophone +65580 irene xylophone +65579 irene polk +65579 luke ovid +65579 quinn nixon +65579 sarah garcia +65579 wendy ovid +65578 calvin robinson +65578 fred king +65578 holly thompson +65578 katie ichabod +65578 quinn king +65578 rachel davidson +65578 victor hernandez +65577 holly white +65576 calvin falkner +65576 calvin ovid +65576 fred polk +65576 luke robinson +65575 calvin falkner +65575 irene steinbeck +65575 luke zipper +65575 zach king +65574 gabriella steinbeck +65574 priscilla nixon +65574 rachel thompson +65573 victor ellison +65573 victor hernandez +65573 yuri nixon +65572 calvin davidson +65572 calvin young +65572 katie young +65572 oscar ellison +65572 quinn garcia +65571 bob king +65571 irene polk +65571 katie ichabod +65571 mike steinbeck +65570 NULL +65570 bob ovid +65570 fred polk +65570 luke ellison +65570 mike hernandez +65570 yuri quirinius +65569 nick falkner +65568 bob ichabod +65568 holly thompson +65568 jessica thompson +65567 katie xylophone +65566 gabriella garcia +65566 rachel white +65565 katie young +65565 quinn young +65564 alice polk +65564 calvin carson +65564 calvin white +65564 ethan hernandez +65564 ethan quirinius +65564 jessica thompson +65564 katie hernandez +65563 calvin zipper +65563 priscilla young +65563 xavier davidson +65563 yuri steinbeck +65562 calvin falkner +65562 ethan xylophone +65562 luke white +65562 quinn allen +65562 rachel polk +65562 wendy polk +65561 bob davidson +65561 ethan polk +65561 jessica robinson +65560 fred white +65560 jessica johnson +65560 oscar thompson +65560 ulysses steinbeck +65560 zach brown +65559 NULL +65559 ethan laertes +65559 gabriella ichabod +65559 gabriella zipper +65559 irene garcia +65558 fred robinson +65557 fred hernandez +65557 nick johnson +65556 oscar underhill +65556 xavier hernandez +65556 yuri falkner +65556 zach garcia +65556 zach steinbeck +65555 fred nixon +65554 gabriella miller +65554 rachel falkner +65553 calvin van buren +65553 david van buren +65553 irene nixon +65553 luke laertes +65553 oscar carson +65552 NULL +65552 irene ellison +65552 oscar polk +65552 wendy falkner +65552 zach miller +65551 fred young +65551 ulysses underhill +65551 wendy underhill +65550 ethan quirinius +65550 fred davidson +65550 holly young +65550 jessica ovid +65550 quinn brown +65550 quinn laertes +65550 tom johnson +65549 bob garcia +65549 bob ovid +65549 fred ichabod +65549 fred king +65549 jessica white +65549 ulysses laertes +65549 victor davidson +65549 victor miller +65548 calvin ovid +65548 gabriella allen +65548 holly ichabod +65548 priscilla johnson +65548 quinn zipper +65548 tom hernandez +65548 wendy king +65547 bob ellison +65547 jessica quirinius +65547 mike davidson +65547 xavier allen +65546 katie white +65545 mike king +65545 tom carson +65545 victor brown +65544 calvin davidson +65544 calvin nixon +65544 david ovid +65544 irene thompson +65544 ulysses young +65544 xavier polk +65544 xavier xylophone +65544 zach ovid +65543 fred johnson +65543 sarah johnson +65542 fred falkner +65542 holly thompson +65542 luke miller +65542 mike white +65542 tom davidson +65541 calvin brown +65541 ethan brown +65541 holly brown +65541 jessica falkner +65541 rachel thompson +65541 tom zipper +65541 wendy underhill +65541 xavier polk +65541 yuri johnson +65540 rachel falkner +65539 gabriella young +65539 holly laertes +65539 oscar carson +65538 irene laertes +65538 mike polk +65538 tom robinson +65537 NULL +65537 david quirinius +65537 rachel ovid +65537 ulysses laertes +65537 zach garcia +65536 calvin xylophone +65536 david thompson +65536 irene falkner +65536 ulysses johnson +65536 victor johnson +65536 wendy miller +65536 yuri thompson diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 new file mode 100644 index 0000000000000..62d71abc6fc7d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 @@ -0,0 +1,1049 @@ +65536 NULL +65536 16.85 +65536 32.68 +65536 33.45 +65536 58.86 +65536 75.7 +65536 83.48 +65537 NULL +65537 4.49 +65537 11.87 +65537 51.91 +65537 99.34 +65538 NULL +65538 9.53 +65538 30.27 +65539 NULL +65539 58.85 +65539 96.64 +65540 NULL +65541 NULL +65541 9.04 +65541 14.94 +65541 15.85 +65541 27.89 +65541 35.38 +65541 72.33 +65541 89.14 +65541 98.87 +65542 NULL +65542 42.85 +65542 73.72 +65542 73.93 +65542 84.63 +65543 NULL +65543 21.59 +65544 NULL +65544 26.94 +65544 32.03 +65544 48.84 +65544 65.77 +65544 65.85 +65544 73.08 +65544 79.21 +65545 NULL +65545 34.65 +65545 65.81 +65546 NULL +65547 NULL +65547 17.71 +65547 62.31 +65547 83.21 +65548 NULL +65548 52.94 +65548 53.52 +65548 57.61 +65548 58.51 +65548 75.39 +65548 77.24 +65549 NULL +65549 13.3 +65549 28.93 +65549 50.6 +65549 55.04 +65549 64.91 +65549 76.06 +65549 80.09 +65550 NULL +65550 1.75 +65550 8.46 +65550 33.01 +65550 57.63 +65550 91.38 +65550 96.9 +65551 NULL +65551 39.43 +65551 73.93 +65552 NULL +65552 3.43 +65552 18.11 +65552 48.96 +65552 82.21 +65553 NULL +65553 25.31 +65553 29.62 +65553 71.07 +65553 72.16 +65554 NULL +65554 0.24 +65555 NULL +65556 NULL +65556 9.96 +65556 45.81 +65556 50.42 +65556 95.12 +65557 NULL +65557 21.14 +65558 NULL +65559 NULL +65559 29.55 +65559 56.06 +65559 73.94 +65559 83.5 +65560 NULL +65560 16.86 +65560 21.81 +65560 64.38 +65560 71.59 +65561 NULL +65561 32.86 +65561 47.71 +65562 NULL +65562 26.86 +65562 45.94 +65562 54.94 +65562 69.41 +65562 72.04 +65563 NULL +65563 14.36 +65563 33.29 +65563 39.96 +65564 NULL +65564 9.47 +65564 15.82 +65564 26.97 +65564 30.78 +65564 90.34 +65564 98.36 +65565 NULL +65565 81.72 +65566 NULL +65566 7.8 +65567 NULL +65568 NULL +65568 21.79 +65568 58.66 +65569 NULL +65570 NULL +65570 17.09 +65570 18.2 +65570 25.57 +65570 45.23 +65570 76.8 +65571 NULL +65571 26.64 +65571 40.68 +65571 82.5 +65572 NULL +65572 22.64 +65572 43.49 +65572 70.91 +65572 93.48 +65573 NULL +65573 53.56 +65573 96.32 +65574 NULL +65574 31.28 +65574 38.54 +65575 NULL +65575 17 +65575 32.85 +65575 83.4 +65576 NULL +65576 2.04 +65576 4.88 +65576 66.55 +65577 NULL +65578 NULL +65578 16.01 +65578 41.1 +65578 51.36 +65578 54.35 +65578 58.78 +65578 75.62 +65579 NULL +65579 21.36 +65579 33.37 +65579 73.48 +65579 91.42 +65580 NULL +65580 56.33 +65580 81.42 +65581 NULL +65581 29.74 +65581 45.48 +65581 56.59 +65581 60.88 +65581 88.09 +65582 NULL +65582 1.23 +65582 9.35 +65582 96.6 +65583 NULL +65583 28.07 +65583 50.57 +65583 57.67 +65584 NULL +65584 6.88 +65584 65.78 +65585 NULL +65585 31.23 +65585 37.34 +65585 39.32 +65585 50.38 +65586 NULL +65586 1.15 +65586 18.65 +65586 87.57 +65586 94.25 +65587 NULL +65587 5.83 +65587 11.86 +65587 53.84 +65587 94.47 +65588 NULL +65588 64.63 +65588 69.48 +65588 76.44 +65588 98.33 +65589 NULL +65589 49.49 +65589 72.3 +65589 74.83 +65589 94.73 +65590 NULL +65590 13.51 +65590 22.13 +65590 42.53 +65590 92.71 +65591 NULL +65591 9.85 +65591 11.43 +65591 60.78 +65592 NULL +65593 NULL +65593 35.15 +65594 NULL +65594 35.28 +65594 60.57 +65594 65.61 +65595 NULL +65595 8.76 +65595 67.56 +65595 72.7 +65595 89.6 +65595 90.24 +65596 NULL +65596 12.72 +65596 54.75 +65596 85.74 +65596 94.35 +65597 NULL +65597 37.41 +65597 69.05 +65598 NULL +65598 63.3 +65599 NULL +65599 0.56 +65599 4.93 +65599 41.61 +65599 76.29 +65600 NULL +65600 36.57 +65600 43.03 +65600 92.92 +65601 NULL +65601 26.54 +65601 37.93 +65602 NULL +65602 8.13 +65602 47.16 +65602 83.67 +65602 86.23 +65602 95.58 +65603 NULL +65603 41.44 +65603 45.63 +65603 69.26 +65603 80.24 +65604 NULL +65604 63.36 +65604 63.65 +65604 85.91 +65605 NULL +65606 NULL +65606 7.51 +65606 24.8 +65606 57.69 +65606 67.94 +65606 87.16 +65607 NULL +65607 9.67 +65607 36.58 +65607 71.75 +65607 75.86 +65607 91.52 +65608 NULL +65608 48.9 +65608 69.42 +65608 87.9 +65609 NULL +65610 NULL +65610 7.59 +65610 11.99 +65610 36.77 +65610 39.74 +65611 NULL +65611 21.21 +65611 25.92 +65611 64.89 +65612 NULL +65612 16.05 +65612 25.1 +65612 52.64 +65613 NULL +65614 NULL +65614 1.42 +65614 94.47 +65615 NULL +65615 10.79 +65615 39.4 +65615 99.88 +65616 NULL +65616 75.2 +65617 NULL +65617 18.51 +65617 47.45 +65617 64.9 +65618 NULL +65618 10.06 +65618 16.6 +65618 81.99 +65618 88.38 +65619 NULL +65619 27.32 +65619 32.64 +65619 34.72 +65619 36.48 +65619 36.59 +65620 NULL +65620 6.85 +65620 8.16 +65620 29.14 +65620 64.65 +65620 81.28 +65621 NULL +65621 95.14 +65622 NULL +65622 28.37 +65622 50.08 +65622 74.31 +65622 88.6 +65622 93.7 +65623 NULL +65623 30.83 +65623 31.22 +65623 39.74 +65623 48.51 +65623 95.58 +65623 97.2 +65624 NULL +65624 58.02 +65624 65.31 +65624 70.08 +65624 93.3 +65625 NULL +65625 20.61 +65625 42.86 +65625 55.06 +65626 NULL +65626 63.54 +65626 64.61 +65626 75.15 +65627 NULL +65627 19.65 +65627 61.89 +65627 93.29 +65628 NULL +65628 14.83 +65628 30.43 +65628 37.8 +65628 74.31 +65628 83.26 +65629 NULL +65629 19.33 +65629 58.81 +65629 72.9 +65630 NULL +65630 72.13 +65631 NULL +65632 NULL +65632 88.51 +65633 NULL +65633 59.56 +65633 72.54 +65633 81.02 +65634 NULL +65634 57.09 +65634 64.36 +65634 99.34 +65635 NULL +65635 64.99 +65635 82.29 +65636 NULL +65636 21.15 +65636 86.29 +65637 NULL +65637 16.89 +65637 26.78 +65637 29.34 +65637 35.51 +65637 44.32 +65637 48.88 +65637 93.41 +65638 NULL +65638 11.2 +65638 19.13 +65639 NULL +65640 NULL +65641 NULL +65641 26.02 +65641 84.27 +65641 91.46 +65642 NULL +65643 NULL +65643 22.05 +65643 50.79 +65643 52.56 +65643 61.29 +65643 71.29 +65643 80.96 +65643 92.24 +65643 93.11 +65644 NULL +65644 1.97 +65644 30.25 +65644 58.05 +65644 87.31 +65644 89.95 +65644 96.45 +65645 NULL +65645 3.95 +65645 63.22 +65646 NULL +65646 17.92 +65646 27.34 +65647 NULL +65647 58.03 +65648 NULL +65648 0.08 +65648 17.66 +65648 64.06 +65649 NULL +65649 8.69 +65649 43.92 +65649 91.03 +65650 NULL +65650 23.55 +65650 59.55 +65650 85.89 +65650 89.12 +65650 90.77 +65651 NULL +65651 24.25 +65651 58.25 +65651 74.13 +65651 84.42 +65652 NULL +65652 55.04 +65652 73.61 +65653 NULL +65653 3.81 +65653 52.23 +65653 85.09 +65654 NULL +65654 8.91 +65654 11.64 +65654 26.73 +65654 29.85 +65654 37.74 +65654 37.8 +65654 53.55 +65654 88.23 +65655 NULL +65655 77.41 +65656 NULL +65656 14 +65656 14.96 +65656 53.27 +65656 64.44 +65656 82.67 +65657 NULL +65657 11.93 +65657 26.4 +65657 64.39 +65657 65.01 +65658 NULL +65658 2.63 +65658 20.69 +65658 42.93 +65658 46.61 +65658 60.94 +65658 66.53 +65658 68.85 +65658 77.66 +65658 92.67 +65659 NULL +65659 8.95 +65659 46.57 +65659 53.8 +65659 94.3 +65659 94.69 +65659 95.71 +65659 99.87 +65660 NULL +65660 28.05 +65660 62.82 +65661 NULL +65661 5.24 +65661 8.06 +65661 26.8 +65661 68.98 +65662 NULL +65662 59.92 +65662 76.11 +65662 76.51 +65662 88.64 +65662 99.18 +65663 NULL +65663 5.42 +65663 78.56 +65663 94.16 +65664 NULL +65664 11.46 +65664 27.6 +65664 34.71 +65664 38.42 +65664 45.4 +65664 55.82 +65664 97.64 +65665 NULL +65666 NULL +65666 32.73 +65666 83.95 +65667 NULL +65667 13.96 +65667 63.9 +65667 97.87 +65668 NULL +65669 NULL +65669 1.76 +65669 16.95 +65669 38.6 +65669 54.25 +65669 93.79 +65670 NULL +65670 5.37 +65670 61.06 +65670 61.54 +65670 92.97 +65671 NULL +65671 8.65 +65671 52.05 +65672 NULL +65672 52.6 +65672 58.1 +65672 64.09 +65672 75.27 +65673 NULL +65673 0.9 +65673 33.27 +65673 43.81 +65673 87.78 +65674 NULL +65675 NULL +65675 4.19 +65675 24.19 +65675 35.33 +65675 35.78 +65675 79.9 +65675 83.09 +65675 87.36 +65676 NULL +65676 8.77 +65676 58.12 +65676 80.13 +65677 NULL +65677 5.06 +65677 25.37 +65677 44.47 +65677 48.79 +65677 87.67 +65678 NULL +65678 8.72 +65678 33.9 +65679 NULL +65679 64.15 +65680 NULL +65680 1.01 +65680 34.08 +65680 54.11 +65680 55.3 +65680 65.88 +65681 NULL +65681 35.45 +65681 41.57 +65681 61.3 +65681 71.17 +65681 75.85 +65682 NULL +65682 67.17 +65682 92.95 +65683 NULL +65683 17.62 +65683 99.56 +65684 NULL +65684 3.51 +65684 67.34 +65685 NULL +65685 38.71 +65685 43.48 +65685 63.27 +65685 87.84 +65685 90.69 +65686 NULL +65686 31.75 +65686 58.87 +65686 98.68 +65687 NULL +65687 3.37 +65687 21.79 +65687 48.73 +65688 NULL +65688 76.21 +65689 NULL +65689 9.12 +65690 NULL +65690 3.43 +65691 NULL +65691 5.01 +65691 6.93 +65691 28.47 +65691 56.02 +65691 58.01 +65691 69.8 +65691 76.98 +65692 NULL +65692 54.76 +65693 NULL +65693 8.38 +65693 32.33 +65693 45.69 +65693 69.32 +65693 71.72 +65693 84.88 +65694 NULL +65694 58.23 +65694 82.24 +65694 88.5 +65695 NULL +65695 57.33 +65695 59.96 +65695 77.09 +65696 NULL +65696 17.35 +65696 40.3 +65696 54.02 +65697 NULL +65697 3.18 +65697 50.01 +65697 67.9 +65697 86.79 +65697 90.16 +65698 NULL +65698 42.98 +65699 NULL +65699 13.29 +65699 38.71 +65699 68.94 +65699 84.79 +65699 88.09 +65700 NULL +65700 2.83 +65700 37.61 +65701 NULL +65701 1.81 +65701 6.35 +65702 NULL +65702 37.6 +65702 55.68 +65702 79.5 +65703 NULL +65703 37.18 +65703 40.81 +65703 90.89 +65704 NULL +65704 16.22 +65704 37.12 +65704 48.48 +65704 54.76 +65704 93.21 +65705 NULL +65705 20.57 +65705 25.89 +65705 65.13 +65706 NULL +65706 3.91 +65706 9.74 +65706 55.94 +65706 72.87 +65707 NULL +65707 76.2 +65708 NULL +65708 1.29 +65709 NULL +65709 5.64 +65709 49.79 +65710 NULL +65710 86.7 +65711 NULL +65711 8.66 +65711 50.26 +65711 71.89 +65711 78.69 +65711 96.1 +65712 NULL +65712 30.27 +65712 34.7 +65712 49.69 +65712 53.65 +65713 NULL +65713 10.94 +65713 39.47 +65713 72.37 +65713 90.91 +65714 NULL +65714 14.85 +65714 47.42 +65715 NULL +65715 39.62 +65715 54.79 +65715 81.28 +65715 89.4 +65716 NULL +65716 9 +65716 10.07 +65716 33.4 +65716 71.53 +65716 85.93 +65717 NULL +65717 1.23 +65717 5.81 +65717 57.61 +65717 80.05 +65718 NULL +65718 63.06 +65718 84.35 +65718 89.67 +65719 NULL +65719 51.13 +65719 66.85 +65719 82.1 +65720 NULL +65720 2.72 +65720 18.8 +65720 22.34 +65720 62.04 +65721 NULL +65721 23.78 +65721 39.19 +65721 55.75 +65721 72.82 +65721 95.12 +65721 95.38 +65722 NULL +65722 1.76 +65722 38.82 +65723 NULL +65723 39.9 +65724 NULL +65724 10.52 +65724 36.05 +65724 50.96 +65724 71.66 +65724 85.52 +65725 NULL +65726 NULL +65726 6 +65726 60.46 +65727 NULL +65727 19.81 +65727 49.19 +65727 87.37 +65727 88.11 +65728 NULL +65728 55.37 +65729 NULL +65730 NULL +65730 1.35 +65730 30.6 +65730 81.44 +65731 NULL +65731 24.48 +65731 61.52 +65731 97.18 +65732 NULL +65732 30.06 +65732 91.15 +65733 NULL +65733 11.44 +65733 20.72 +65733 88.46 +65733 93.45 +65733 99.8 +65734 NULL +65734 31.71 +65735 NULL +65735 12.67 +65735 61.16 +65736 NULL +65736 28.9 +65736 48.54 +65736 86.51 +65737 NULL +65737 3.98 +65737 20.85 +65737 29.92 +65737 80.97 +65738 NULL +65738 30.94 +65738 82.32 +65738 95.1 +65739 NULL +65739 74.77 +65739 92.4 +65740 NULL +65740 7.49 +65740 58.65 +65741 NULL +65742 NULL +65742 6.61 +65742 43.84 +65743 NULL +65743 26.6 +65743 52.65 +65743 62 +65744 NULL +65744 46.98 +65745 NULL +65745 25.19 +65745 66.36 +65745 80.12 +65746 NULL +65746 36.74 +65746 93.21 +65746 97.52 +65746 98.1 +65747 NULL +65747 11.16 +65747 15.07 +65747 21.8 +65747 39.77 +65747 52.77 +65747 71.87 +65748 NULL +65748 29.49 +65749 NULL +65749 15.14 +65749 45 +65749 65.49 +65749 73.24 +65750 NULL +65750 20.91 +65750 83.44 +65750 85.44 +65750 96.85 +65751 NULL +65751 2.96 +65751 9.02 +65751 30.68 +65751 47.81 +65751 78.75 +65752 NULL +65752 47.82 +65753 NULL +65753 86.97 +65754 NULL +65754 54.35 +65755 NULL +65755 11.23 +65755 22.44 +65755 64 +65755 67.54 +65755 76.75 +65755 81.44 +65755 90.08 +65755 96.8 +65756 NULL +65756 1.45 +65756 11.81 +65756 63.51 +65757 NULL +65757 1.86 +65757 9.24 +65757 34.84 +65757 90.09 +65758 NULL +65758 25.62 +65758 56.56 +65758 60.88 +65758 94.9 +65759 NULL +65759 10.63 +65759 14.1 +65759 47.54 +65759 92.81 +65760 NULL +65760 21.14 +65760 27.52 +65760 95.45 +65761 NULL +65762 NULL +65762 5.49 +65762 45.7 +65762 77.96 +65762 87.5 +65763 NULL +65763 0.72 +65763 43.8 +65763 86.43 +65763 87.99 +65764 NULL +65764 31.41 +65764 57.1 +65765 NULL +65765 88.52 +65765 88.56 +65766 NULL +65766 37.06 +65766 66.34 +65766 86.53 +65766 98.9 +65767 NULL +65767 90.88 +65767 95.57 +65768 NULL +65769 NULL +65769 11.45 +65769 38.98 +65769 58.05 +65769 70.52 +65769 91.49 +65770 NULL +65770 51.9 +65771 NULL +65771 6.15 +65771 7.5 +65772 NULL +65773 NULL +65773 3.81 +65773 18.2 +65773 30.49 +65773 47.09 +65773 53.09 +65773 63.26 +65773 76.46 +65774 NULL +65774 45.74 +65774 45.97 +65774 48.8 +65774 56.84 +65774 94.77 +65775 NULL +65775 7.88 +65775 66.56 +65775 66.68 +65775 98.43 +65776 NULL +65776 18.7 +65776 28.47 +65776 49.73 +65776 98.87 +65777 NULL +65777 54.39 +65777 73.79 +65777 82.62 +65778 NULL +65778 7.37 +65778 51.64 +65778 59.03 +65778 62.17 +65778 64.69 +65778 89.51 +65778 95.69 +65779 NULL +65779 11.87 +65779 28.2 +65779 39.48 +65779 45.61 +65779 64.41 +65779 65.24 +65780 NULL +65780 10.95 +65780 38.58 +65781 NULL +65781 70.59 +65781 95.52 +65782 NULL +65782 30.24 +65782 34.31 +65782 76.14 +65782 81.9 +65783 NULL +65783 46.34 +65783 51.08 +65783 52.43 +65783 62.58 +65783 77.4 +65784 NULL +65784 15.7 +65784 31.35 +65784 68.18 +65784 93.95 +65785 NULL +65785 29.61 +65786 NULL +65786 8.99 +65786 29.32 +65786 66.89 +65786 80.94 +65787 NULL +65787 18.78 +65787 31.19 +65787 64.88 +65788 NULL +65788 16.1 +65788 21.81 +65788 25.77 +65789 NULL +65789 20.44 +65789 43.53 +65789 52.49 +65789 83.18 +65789 92.74 +65789 96.9 +65790 NULL +65790 46.91 +65790 84.87 +65791 NULL +65791 4.24 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-3-34d9ee4120f21d0d0ae914fba0acc60c b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-3-34d9ee4120f21d0d0ae914fba0acc60c new file mode 100644 index 0000000000000..569c1d4e5f7b6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-3-34d9ee4120f21d0d0ae914fba0acc60c @@ -0,0 +1,1049 @@ + 7 + 11 + 38 +alice allen 48 +alice allen 78 +alice allen 79 +alice brown 91 +alice carson 28 +alice davidson 88 +alice falkner 117 +alice garcia 106 +alice hernandez 37 +alice hernandez 85 +alice johnson 65 +alice king 109 +alice king 113 +alice king 118 +alice laertes 6 +alice laertes 65 +alice miller 97 +alice nixon 15 +alice nixon 31 +alice nixon 56 +alice ovid 15 +alice polk 90 +alice quirinius 27 +alice quirinius 89 +alice robinson 5 +alice robinson 68 +alice steinbeck 45 +alice steinbeck 50 +alice steinbeck 77 +alice underhill 34 +alice van buren 31 +alice xylophone 0 +alice xylophone 5 +alice xylophone 113 +alice zipper 19 +alice zipper 51 +alice zipper 74 +bob brown 35 +bob brown 61 +bob brown 71 +bob carson 40 +bob davidson 44 +bob davidson 103 +bob davidson 124 +bob ellison 7 +bob ellison 46 +bob ellison 85 +bob ellison 103 +bob falkner 2 +bob garcia 16 +bob garcia 42 +bob garcia 71 +bob garcia 77 +bob garcia 83 +bob hernandez 93 +bob ichabod 72 +bob king 9 +bob king 22 +bob king 81 +bob laertes -1 +bob laertes 105 +bob miller 31 +bob ovid 26 +bob ovid 27 +bob ovid 81 +bob ovid 86 +bob polk 55 +bob quirinius 26 +bob steinbeck 95 +bob van buren 88 +bob white 1 +bob white 16 +bob xylophone -2 +bob xylophone 49 +bob young -1 +bob zipper 36 +bob zipper 78 +bob zipper 92 +calvin allen 98 +calvin brown 81 +calvin brown 87 +calvin brown 121 +calvin carson 105 +calvin davidson 34 +calvin davidson 89 +calvin ellison 34 +calvin falkner -1 +calvin falkner 12 +calvin falkner 46 +calvin falkner 50 +calvin falkner 53 +calvin falkner 101 +calvin garcia 79 +calvin hernandez 22 +calvin johnson 34 +calvin laertes 37 +calvin laertes 100 +calvin nixon 50 +calvin nixon 71 +calvin nixon 72 +calvin ovid -1 +calvin ovid 50 +calvin ovid 65 +calvin ovid 71 +calvin polk 23 +calvin quirinius 5 +calvin quirinius 96 +calvin robinson 0 +calvin steinbeck 13 +calvin steinbeck 14 +calvin steinbeck 33 +calvin thompson 12 +calvin thompson 40 +calvin underhill 19 +calvin van buren 57 +calvin van buren 78 +calvin white 26 +calvin white 80 +calvin xylophone 17 +calvin xylophone 48 +calvin xylophone 78 +calvin young 8 +calvin young 99 +calvin zipper 31 +calvin zipper 46 +david allen 27 +david allen 80 +david brown 75 +david brown 117 +david davidson 11 +david davidson 38 +david davidson 54 +david davidson 74 +david ellison 50 +david ellison 54 +david ellison 120 +david hernandez 72 +david ichabod 6 +david ichabod 55 +david laertes 88 +david nixon 118 +david ovid 61 +david ovid 95 +david quirinius 43 +david quirinius 85 +david quirinius 121 +david robinson 47 +david robinson 59 +david thompson 89 +david underhill 69 +david underhill 87 +david underhill 98 +david van buren 1 +david van buren 38 +david white 93 +david xylophone 0 +david xylophone 22 +david xylophone 82 +david young 2 +david young 79 +ethan allen 24 +ethan brown 3 +ethan brown 29 +ethan brown 55 +ethan brown 64 +ethan brown 84 +ethan brown 108 +ethan carson 83 +ethan ellison 28 +ethan ellison 65 +ethan falkner 45 +ethan falkner 49 +ethan garcia 16 +ethan hernandez 71 +ethan johnson 108 +ethan king 44 +ethan laertes -1 +ethan laertes 27 +ethan laertes 46 +ethan laertes 68 +ethan laertes 81 +ethan laertes 103 +ethan laertes 114 +ethan miller 40 +ethan nixon 18 +ethan ovid 66 +ethan polk 46 +ethan polk 71 +ethan polk 114 +ethan polk 117 +ethan quirinius 16 +ethan quirinius 41 +ethan quirinius 85 +ethan robinson 32 +ethan robinson 34 +ethan underhill 89 +ethan van buren 43 +ethan white 38 +ethan white 51 +ethan xylophone 107 +ethan zipper 37 +ethan zipper 89 +fred davidson 18 +fred davidson 18 +fred davidson 77 +fred ellison -3 +fred ellison 44 +fred ellison 116 +fred falkner 18 +fred falkner 33 +fred falkner 100 +fred hernandez 36 +fred ichabod 1 +fred ichabod 77 +fred johnson 62 +fred king 33 +fred king 92 +fred laertes 17 +fred miller NULL +fred nixon 33 +fred nixon 36 +fred nixon 102 +fred nixon 111 +fred polk -2 +fred polk 39 +fred polk 60 +fred polk 85 +fred quirinius 25 +fred quirinius 124 +fred robinson 89 +fred steinbeck 79 +fred steinbeck 118 +fred steinbeck 119 +fred underhill 122 +fred van buren 4 +fred van buren 24 +fred van buren 63 +fred van buren 106 +fred white 97 +fred young 33 +fred young 103 +fred zipper 66 +gabriella allen 114 +gabriella allen 119 +gabriella brown 25 +gabriella brown 92 +gabriella carson 112 +gabriella davidson 45 +gabriella ellison 21 +gabriella ellison 101 +gabriella falkner 14 +gabriella falkner 66 +gabriella falkner 77 +gabriella garcia 110 +gabriella hernandez 20 +gabriella hernandez 36 +gabriella ichabod 17 +gabriella ichabod 66 +gabriella ichabod 71 +gabriella ichabod 90 +gabriella ichabod 91 +gabriella king 100 +gabriella king 115 +gabriella laertes 50 +gabriella miller 35 +gabriella ovid 38 +gabriella ovid 89 +gabriella polk 42 +gabriella polk 105 +gabriella steinbeck 18 +gabriella steinbeck 115 +gabriella thompson 45 +gabriella thompson 70 +gabriella thompson 88 +gabriella van buren 5 +gabriella van buren 117 +gabriella white 37 +gabriella young 48 +gabriella young 107 +gabriella zipper 57 +gabriella zipper 71 +holly allen 63 +holly brown 50 +holly brown 117 +holly falkner NULL +holly hernandez 31 +holly hernandez 43 +holly hernandez 48 +holly hernandez 100 +holly ichabod 28 +holly ichabod 53 +holly ichabod 83 +holly johnson 60 +holly johnson 112 +holly johnson 121 +holly king 90 +holly king 114 +holly laertes 13 +holly miller 28 +holly nixon -2 +holly nixon 120 +holly polk 54 +holly polk 124 +holly robinson 1 +holly thompson 28 +holly thompson 33 +holly thompson 83 +holly underhill 5 +holly underhill 31 +holly underhill 101 +holly underhill 113 +holly van buren 19 +holly white 18 +holly white 90 +holly xylophone 18 +holly young 32 +holly young 72 +holly zipper 78 +holly zipper 88 +irene allen 56 +irene brown 78 +irene brown 93 +irene brown 108 +irene carson 0 +irene ellison -3 +irene ellison 40 +irene falkner 13 +irene falkner 104 +irene garcia 34 +irene garcia 62 +irene garcia 124 +irene ichabod 83 +irene ichabod 112 +irene johnson 88 +irene laertes 9 +irene laertes 54 +irene laertes 60 +irene miller 108 +irene nixon -1 +irene nixon 12 +irene nixon 101 +irene ovid 26 +irene ovid 32 +irene ovid 53 +irene polk 18 +irene polk 92 +irene polk 99 +irene polk 109 +irene polk 116 +irene quirinius 7 +irene quirinius 76 +irene quirinius 97 +irene robinson 51 +irene steinbeck 46 +irene thompson 10 +irene underhill 27 +irene underhill 63 +irene van buren 17 +irene van buren 104 +irene xylophone 18 +jessica brown 117 +jessica carson 3 +jessica carson 13 +jessica carson 88 +jessica davidson 11 +jessica davidson 28 +jessica davidson 89 +jessica davidson 124 +jessica ellison 38 +jessica ellison 50 +jessica falkner 71 +jessica garcia 25 +jessica garcia 43 +jessica ichabod 104 +jessica johnson 31 +jessica johnson 69 +jessica miller 74 +jessica nixon 22 +jessica nixon 120 +jessica ovid 47 +jessica ovid 73 +jessica polk 118 +jessica quirinius 0 +jessica quirinius 87 +jessica quirinius 105 +jessica quirinius 114 +jessica robinson 15 +jessica thompson 1 +jessica thompson 77 +jessica underhill 32 +jessica underhill 46 +jessica underhill 83 +jessica van buren 54 +jessica white 5 +jessica white 30 +jessica white 45 +jessica white 65 +jessica white 98 +jessica xylophone 67 +jessica young 61 +jessica young 123 +jessica zipper 27 +jessica zipper 33 +jessica zipper 54 +katie allen 114 +katie brown 39 +katie davidson 35 +katie ellison 5 +katie ellison 58 +katie falkner 15 +katie garcia 49 +katie garcia 65 +katie hernandez 83 +katie ichabod 9 +katie ichabod 75 +katie ichabod 104 +katie king 44 +katie king 59 +katie king 93 +katie miller 23 +katie miller 117 +katie nixon 43 +katie ovid 81 +katie polk 17 +katie polk 85 +katie robinson 92 +katie van buren 25 +katie van buren 88 +katie white 34 +katie white 86 +katie xylophone 84 +katie young 2 +katie young 24 +katie young 70 +katie zipper 25 +katie zipper 87 +luke allen 7 +luke allen 44 +luke allen 62 +luke allen 100 +luke allen 114 +luke brown 112 +luke davidson 51 +luke davidson 84 +luke ellison 35 +luke ellison 40 +luke ellison 86 +luke falkner 59 +luke falkner 97 +luke garcia 51 +luke garcia 100 +luke ichabod 42 +luke ichabod 123 +luke johnson 9 +luke johnson 17 +luke johnson 53 +luke laertes 66 +luke laertes 73 +luke laertes 76 +luke laertes 101 +luke laertes 118 +luke miller 93 +luke ovid 43 +luke ovid 70 +luke polk 53 +luke polk 88 +luke quirinius 82 +luke robinson 0 +luke robinson 114 +luke thompson 51 +luke underhill 2 +luke underhill 109 +luke underhill 119 +luke van buren 43 +luke white 110 +luke xylophone 15 +luke zipper 10 +mike allen 0 +mike brown 88 +mike carson 12 +mike carson 17 +mike carson 122 +mike davidson 9 +mike davidson 110 +mike ellison 5 +mike ellison 50 +mike ellison 70 +mike ellison 94 +mike ellison 95 +mike falkner 61 +mike garcia 2 +mike garcia 68 +mike garcia 110 +mike hernandez 91 +mike hernandez 106 +mike ichabod 18 +mike king 4 +mike king 58 +mike king 83 +mike king 96 +mike king 103 +mike king 118 +mike miller 51 +mike nixon 97 +mike nixon 106 +mike polk 6 +mike polk 65 +mike polk 119 +mike quirinius 22 +mike steinbeck 75 +mike steinbeck 85 +mike steinbeck 101 +mike steinbeck 116 +mike van buren 16 +mike van buren 111 +mike white -1 +mike white 22 +mike white 45 +mike white 61 +mike young 37 +mike young 53 +mike young 72 +mike zipper 27 +mike zipper 76 +mike zipper 106 +nick allen 8 +nick allen 57 +nick brown 114 +nick davidson 84 +nick ellison 10 +nick ellison 107 +nick falkner 83 +nick falkner 86 +nick garcia 53 +nick garcia 69 +nick garcia 108 +nick ichabod 59 +nick ichabod 71 +nick ichabod 84 +nick johnson 47 +nick johnson 88 +nick laertes 17 +nick miller 101 +nick nixon 43 +nick ovid 42 +nick polk 1 +nick quirinius 22 +nick quirinius 36 +nick robinson 48 +nick robinson 54 +nick steinbeck 33 +nick thompson 73 +nick underhill 122 +nick van buren 53 +nick xylophone 80 +nick young 6 +nick young 60 +nick zipper 3 +nick zipper 21 +oscar allen 58 +oscar brown 80 +oscar carson 10 +oscar carson 27 +oscar carson 36 +oscar carson 72 +oscar carson 88 +oscar davidson 14 +oscar ellison 50 +oscar ellison 74 +oscar falkner 96 +oscar garcia 44 +oscar hernandez 1 +oscar hernandez 93 +oscar ichabod 20 +oscar ichabod 28 +oscar ichabod 69 +oscar ichabod 120 +oscar johnson 44 +oscar johnson 53 +oscar king 67 +oscar king 71 +oscar king 81 +oscar laertes 4 +oscar laertes 28 +oscar laertes 53 +oscar laertes 63 +oscar nixon 58 +oscar ovid 3 +oscar ovid 27 +oscar ovid 47 +oscar polk 8 +oscar polk 112 +oscar quirinius NULL +oscar quirinius 0 +oscar quirinius 17 +oscar quirinius 114 +oscar robinson 16 +oscar robinson 42 +oscar robinson 59 +oscar robinson 93 +oscar steinbeck 51 +oscar thompson 44 +oscar thompson 44 +oscar thompson 60 +oscar thompson 66 +oscar underhill 86 +oscar van buren 40 +oscar van buren 51 +oscar van buren 114 +oscar white 2 +oscar white 20 +oscar white 49 +oscar white 58 +oscar xylophone 18 +oscar xylophone 73 +oscar xylophone 74 +oscar zipper 0 +oscar zipper 23 +oscar zipper 95 +priscilla brown 51 +priscilla brown 75 +priscilla brown 97 +priscilla carson 16 +priscilla carson 52 +priscilla carson 124 +priscilla ichabod 117 +priscilla ichabod 122 +priscilla johnson 5 +priscilla johnson 17 +priscilla johnson 62 +priscilla johnson 77 +priscilla johnson 117 +priscilla king 43 +priscilla nixon 61 +priscilla nixon 66 +priscilla ovid 46 +priscilla ovid 118 +priscilla polk 45 +priscilla quirinius 83 +priscilla thompson 82 +priscilla underhill 117 +priscilla underhill 122 +priscilla van buren 0 +priscilla van buren 22 +priscilla van buren 102 +priscilla white 88 +priscilla xylophone 8 +priscilla xylophone 90 +priscilla xylophone 109 +priscilla young 17 +priscilla young 113 +priscilla zipper 27 +priscilla zipper 35 +quinn allen 27 +quinn allen 114 +quinn brown 70 +quinn brown 88 +quinn brown 117 +quinn davidson 93 +quinn davidson 93 +quinn davidson 109 +quinn davidson 121 +quinn ellison 83 +quinn ellison 116 +quinn garcia 78 +quinn garcia 104 +quinn garcia 110 +quinn garcia 120 +quinn ichabod 60 +quinn king 14 +quinn king 46 +quinn laertes -2 +quinn laertes 65 +quinn laertes 95 +quinn nixon 11 +quinn ovid 123 +quinn quirinius 94 +quinn robinson 60 +quinn steinbeck 82 +quinn steinbeck 122 +quinn thompson 41 +quinn thompson 60 +quinn underhill 19 +quinn underhill 28 +quinn underhill 34 +quinn van buren 18 +quinn young 15 +quinn zipper 44 +quinn zipper 103 +rachel allen 76 +rachel allen 122 +rachel brown 23 +rachel brown 56 +rachel brown 71 +rachel brown 101 +rachel brown 108 +rachel carson 27 +rachel carson 74 +rachel davidson 84 +rachel ellison 51 +rachel falkner -2 +rachel falkner 43 +rachel falkner 72 +rachel falkner 104 +rachel johnson 32 +rachel king 84 +rachel king 95 +rachel laertes 37 +rachel laertes 106 +rachel ovid 5 +rachel ovid 31 +rachel polk 79 +rachel quirinius 108 +rachel robinson 24 +rachel robinson 41 +rachel robinson 91 +rachel thompson -3 +rachel thompson -2 +rachel thompson 74 +rachel underhill 11 +rachel white 108 +rachel white 119 +rachel young 77 +rachel zipper 16 +rachel zipper 116 +sarah carson 41 +sarah carson 58 +sarah carson 119 +sarah ellison 14 +sarah falkner 112 +sarah falkner 123 +sarah garcia 72 +sarah garcia 91 +sarah garcia 98 +sarah ichabod 38 +sarah ichabod 80 +sarah johnson 5 +sarah johnson 51 +sarah johnson 69 +sarah johnson 116 +sarah king 13 +sarah king 120 +sarah miller 31 +sarah ovid 122 +sarah robinson 26 +sarah robinson 35 +sarah steinbeck 30 +sarah white 11 +sarah white 32 +sarah xylophone 28 +sarah young 120 +sarah zipper 107 +tom brown 27 +tom brown 89 +tom carson 11 +tom carson 70 +tom carson 123 +tom davidson 72 +tom ellison 28 +tom ellison 118 +tom ellison 120 +tom falkner 11 +tom falkner 35 +tom hernandez -3 +tom hernandez 118 +tom ichabod 19 +tom johnson 42 +tom johnson 82 +tom king 59 +tom laertes 33 +tom laertes 54 +tom miller 9 +tom miller 48 +tom miller 94 +tom nixon 45 +tom ovid 68 +tom polk 70 +tom polk 107 +tom quirinius 10 +tom quirinius 38 +tom robinson 52 +tom robinson 104 +tom robinson 109 +tom robinson 115 +tom steinbeck 113 +tom van buren 5 +tom van buren 48 +tom van buren 63 +tom white 81 +tom young 13 +tom young 99 +tom zipper 31 +ulysses brown 46 +ulysses carson 5 +ulysses carson 26 +ulysses carson 55 +ulysses carson 109 +ulysses davidson 18 +ulysses ellison 61 +ulysses garcia 12 +ulysses hernandez 9 +ulysses hernandez 22 +ulysses hernandez 53 +ulysses ichabod 32 +ulysses ichabod 99 +ulysses johnson 41 +ulysses king 2 +ulysses laertes 40 +ulysses laertes 51 +ulysses laertes 95 +ulysses miller 23 +ulysses miller 85 +ulysses nixon 92 +ulysses ovid 31 +ulysses polk 28 +ulysses polk 74 +ulysses polk 86 +ulysses polk 89 +ulysses quirinius 7 +ulysses robinson 79 +ulysses steinbeck 6 +ulysses steinbeck 45 +ulysses thompson 24 +ulysses underhill 6 +ulysses underhill 27 +ulysses underhill 42 +ulysses underhill 51 +ulysses underhill 93 +ulysses underhill 98 +ulysses underhill 111 +ulysses van buren 58 +ulysses white 67 +ulysses white 109 +ulysses xylophone 47 +ulysses xylophone 105 +ulysses xylophone 123 +ulysses young 61 +ulysses young 86 +ulysses young 89 +victor allen 2 +victor allen 17 +victor brown 0 +victor brown 23 +victor brown 60 +victor brown 64 +victor davidson 42 +victor davidson 89 +victor davidson 123 +victor ellison 35 +victor ellison 84 +victor hernandez 1 +victor hernandez 17 +victor hernandez 91 +victor hernandez 94 +victor hernandez 116 +victor johnson 34 +victor johnson 53 +victor johnson 57 +victor king 59 +victor king 112 +victor laertes 18 +victor laertes 118 +victor miller 79 +victor nixon 50 +victor nixon 104 +victor ovid 120 +victor polk 106 +victor quirinius 77 +victor quirinius 85 +victor robinson 29 +victor robinson 105 +victor steinbeck 20 +victor steinbeck 92 +victor steinbeck 100 +victor thompson 124 +victor van buren 41 +victor van buren 71 +victor white 15 +victor white 49 +victor xylophone -3 +victor xylophone 41 +victor xylophone 43 +victor xylophone 54 +victor xylophone 91 +victor young 24 +victor zipper 3 +wendy allen 25 +wendy allen 38 +wendy allen 95 +wendy brown 92 +wendy brown 119 +wendy ellison 53 +wendy ellison 103 +wendy falkner 23 +wendy falkner 28 +wendy falkner 58 +wendy garcia 3 +wendy garcia 48 +wendy garcia 60 +wendy garcia 99 +wendy hernandez 53 +wendy ichabod 87 +wendy king -2 +wendy king 45 +wendy king 124 +wendy laertes 26 +wendy laertes 51 +wendy laertes 72 +wendy miller 51 +wendy miller 105 +wendy nixon 5 +wendy nixon 25 +wendy ovid 17 +wendy ovid 85 +wendy polk 44 +wendy polk 99 +wendy quirinius 77 +wendy quirinius 88 +wendy robinson -3 +wendy robinson 71 +wendy robinson 97 +wendy steinbeck 37 +wendy thompson 28 +wendy thompson 31 +wendy underhill 58 +wendy underhill 82 +wendy underhill 120 +wendy van buren 27 +wendy van buren 82 +wendy white 63 +wendy xylophone 53 +wendy xylophone 119 +wendy young 66 +wendy young 112 +xavier allen 18 +xavier allen 41 +xavier allen 106 +xavier brown 10 +xavier brown 63 +xavier brown 108 +xavier carson 20 +xavier carson 57 +xavier davidson 21 +xavier davidson 24 +xavier davidson 106 +xavier ellison 0 +xavier ellison 53 +xavier garcia 42 +xavier hernandez 9 +xavier hernandez 80 +xavier hernandez 114 +xavier ichabod 20 +xavier ichabod 58 +xavier johnson 44 +xavier johnson 85 +xavier king 26 +xavier king 107 +xavier laertes 60 +xavier ovid 3 +xavier polk 29 +xavier polk 83 +xavier polk 91 +xavier polk 122 +xavier quirinius 27 +xavier quirinius 35 +xavier quirinius 39 +xavier quirinius 111 +xavier thompson 2 +xavier underhill 102 +xavier white 8 +xavier white 56 +xavier xylophone 24 +xavier zipper 48 +yuri allen 31 +yuri allen 121 +yuri brown 101 +yuri brown 106 +yuri carson 1 +yuri carson 36 +yuri ellison -1 +yuri ellison 43 +yuri falkner 31 +yuri falkner 96 +yuri garcia 49 +yuri hernandez 92 +yuri johnson 1 +yuri johnson 2 +yuri johnson 111 +yuri king 44 +yuri laertes 84 +yuri laertes 115 +yuri nixon 5 +yuri nixon 111 +yuri polk 13 +yuri polk 49 +yuri polk 115 +yuri quirinius 24 +yuri quirinius 28 +yuri quirinius 90 +yuri steinbeck 8 +yuri steinbeck 65 +yuri thompson 42 +yuri underhill 10 +yuri underhill 66 +yuri white 73 +yuri xylophone 63 +zach allen 35 +zach brown 7 +zach brown 15 +zach brown 37 +zach brown 61 +zach brown 94 +zach carson 114 +zach ellison 16 +zach falkner 70 +zach falkner 115 +zach garcia -2 +zach garcia 59 +zach garcia 68 +zach garcia 97 +zach ichabod 14 +zach ichabod 73 +zach king 66 +zach king 70 +zach king 81 +zach miller 4 +zach miller 9 +zach miller 73 +zach ovid 61 +zach ovid 68 +zach ovid 77 +zach ovid 114 +zach quirinius 79 +zach robinson 69 +zach steinbeck 6 +zach steinbeck 122 +zach thompson 75 +zach thompson 95 +zach underhill 123 +zach white 58 +zach xylophone 19 +zach xylophone 85 +zach young 11 +zach zipper 68 +zach zipper 100 +zach zipper 101 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-4-dfd39236756a3951bc1ec354799d69e4 b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-4-dfd39236756a3951bc1ec354799d69e4 new file mode 100644 index 0000000000000..86ca4e49d21bd --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-4-dfd39236756a3951bc1ec354799d69e4 @@ -0,0 +1,1049 @@ + + + +alice allen +alice allen +alice allen +alice brown +alice carson +alice davidson +alice falkner +alice garcia +alice hernandez +alice hernandez +alice johnson +alice king +alice king +alice king +alice laertes +alice laertes +alice miller +alice nixon +alice nixon +alice nixon +alice ovid +alice polk +alice quirinius +alice quirinius +alice robinson +alice robinson +alice steinbeck +alice steinbeck +alice steinbeck +alice underhill +alice van buren +alice xylophone +alice xylophone +alice xylophone +alice zipper +alice zipper +alice zipper +bob brown +bob brown +bob brown +bob carson +bob davidson +bob davidson +bob davidson +bob ellison +bob ellison +bob ellison +bob ellison +bob falkner +bob garcia +bob garcia +bob garcia +bob garcia +bob garcia +bob hernandez +bob ichabod +bob king +bob king +bob king +bob laertes +bob laertes +bob miller +bob ovid +bob ovid +bob ovid +bob ovid +bob polk +bob quirinius +bob steinbeck +bob van buren +bob white +bob white +bob xylophone +bob xylophone +bob young +bob zipper +bob zipper +bob zipper +calvin allen +calvin brown +calvin brown +calvin brown +calvin carson +calvin davidson +calvin davidson +calvin ellison +calvin falkner +calvin falkner +calvin falkner +calvin falkner +calvin falkner +calvin falkner +calvin garcia +calvin hernandez +calvin johnson +calvin laertes +calvin laertes +calvin nixon +calvin nixon +calvin nixon +calvin ovid +calvin ovid +calvin ovid +calvin ovid +calvin polk +calvin quirinius +calvin quirinius +calvin robinson +calvin steinbeck +calvin steinbeck +calvin steinbeck +calvin thompson +calvin thompson +calvin underhill +calvin van buren +calvin van buren +calvin white +calvin white +calvin xylophone +calvin xylophone +calvin xylophone +calvin young +calvin young +calvin zipper +calvin zipper +david allen +david allen +david brown +david brown +david davidson +david davidson +david davidson +david davidson +david ellison +david ellison +david ellison +david hernandez +david ichabod +david ichabod +david laertes +david nixon +david ovid +david ovid +david quirinius +david quirinius +david quirinius +david robinson +david robinson +david thompson +david underhill +david underhill +david underhill +david van buren +david van buren +david white +david xylophone +david xylophone +david xylophone +david young +david young +ethan allen +ethan brown +ethan brown +ethan brown +ethan brown +ethan brown +ethan brown +ethan carson +ethan ellison +ethan ellison +ethan falkner +ethan falkner +ethan garcia +ethan hernandez +ethan johnson +ethan king +ethan laertes +ethan laertes +ethan laertes +ethan laertes +ethan laertes +ethan laertes +ethan laertes +ethan miller +ethan nixon +ethan ovid +ethan polk +ethan polk +ethan polk +ethan polk +ethan quirinius +ethan quirinius +ethan quirinius +ethan robinson +ethan robinson +ethan underhill +ethan van buren +ethan white +ethan white +ethan xylophone +ethan zipper +ethan zipper +fred davidson +fred davidson +fred davidson +fred ellison +fred ellison +fred ellison +fred falkner +fred falkner +fred falkner +fred hernandez +fred ichabod +fred ichabod +fred johnson +fred king +fred king +fred laertes +fred miller +fred nixon +fred nixon +fred nixon +fred nixon +fred polk +fred polk +fred polk +fred polk +fred quirinius +fred quirinius +fred robinson +fred steinbeck +fred steinbeck +fred steinbeck +fred underhill +fred van buren +fred van buren +fred van buren +fred van buren +fred white +fred young +fred young +fred zipper +gabriella allen +gabriella allen +gabriella brown +gabriella brown +gabriella carson +gabriella davidson +gabriella ellison +gabriella ellison +gabriella falkner +gabriella falkner +gabriella falkner +gabriella garcia +gabriella hernandez +gabriella hernandez +gabriella ichabod +gabriella ichabod +gabriella ichabod +gabriella ichabod +gabriella ichabod +gabriella king +gabriella king +gabriella laertes +gabriella miller +gabriella ovid +gabriella ovid +gabriella polk +gabriella polk +gabriella steinbeck +gabriella steinbeck +gabriella thompson +gabriella thompson +gabriella thompson +gabriella van buren +gabriella van buren +gabriella white +gabriella young +gabriella young +gabriella zipper +gabriella zipper +holly allen +holly brown +holly brown +holly falkner +holly hernandez +holly hernandez +holly hernandez +holly hernandez +holly ichabod +holly ichabod +holly ichabod +holly johnson +holly johnson +holly johnson +holly king +holly king +holly laertes +holly miller +holly nixon +holly nixon +holly polk +holly polk +holly robinson +holly thompson +holly thompson +holly thompson +holly underhill +holly underhill +holly underhill +holly underhill +holly van buren +holly white +holly white +holly xylophone +holly young +holly young +holly zipper +holly zipper +irene allen +irene brown +irene brown +irene brown +irene carson +irene ellison +irene ellison +irene falkner +irene falkner +irene garcia +irene garcia +irene garcia +irene ichabod +irene ichabod +irene johnson +irene laertes +irene laertes +irene laertes +irene miller +irene nixon +irene nixon +irene nixon +irene ovid +irene ovid +irene ovid +irene polk +irene polk +irene polk +irene polk +irene polk +irene quirinius +irene quirinius +irene quirinius +irene robinson +irene steinbeck +irene thompson +irene underhill +irene underhill +irene van buren +irene van buren +irene xylophone +jessica brown +jessica carson +jessica carson +jessica carson +jessica davidson +jessica davidson +jessica davidson +jessica davidson +jessica ellison +jessica ellison +jessica falkner +jessica garcia +jessica garcia +jessica ichabod +jessica johnson +jessica johnson +jessica miller +jessica nixon +jessica nixon +jessica ovid +jessica ovid +jessica polk +jessica quirinius +jessica quirinius +jessica quirinius +jessica quirinius +jessica robinson +jessica thompson +jessica thompson +jessica underhill +jessica underhill +jessica underhill +jessica van buren +jessica white +jessica white +jessica white +jessica white +jessica white +jessica xylophone +jessica young +jessica young +jessica zipper +jessica zipper +jessica zipper +katie allen +katie brown +katie davidson +katie ellison +katie ellison +katie falkner +katie garcia +katie garcia +katie hernandez +katie ichabod +katie ichabod +katie ichabod +katie king +katie king +katie king +katie miller +katie miller +katie nixon +katie ovid +katie polk +katie polk +katie robinson +katie van buren +katie van buren +katie white +katie white +katie xylophone +katie young +katie young +katie young +katie zipper +katie zipper +luke allen +luke allen +luke allen +luke allen +luke allen +luke brown +luke davidson +luke davidson +luke ellison +luke ellison +luke ellison +luke falkner +luke falkner +luke garcia +luke garcia +luke ichabod +luke ichabod +luke johnson +luke johnson +luke johnson +luke laertes +luke laertes +luke laertes +luke laertes +luke laertes +luke miller +luke ovid +luke ovid +luke polk +luke polk +luke quirinius +luke robinson +luke robinson +luke thompson +luke underhill +luke underhill +luke underhill +luke van buren +luke white +luke xylophone +luke zipper +mike allen +mike brown +mike carson +mike carson +mike carson +mike davidson +mike davidson +mike ellison +mike ellison +mike ellison +mike ellison +mike ellison +mike falkner +mike garcia +mike garcia +mike garcia +mike hernandez +mike hernandez +mike ichabod +mike king +mike king +mike king +mike king +mike king +mike king +mike miller +mike nixon +mike nixon +mike polk +mike polk +mike polk +mike quirinius +mike steinbeck +mike steinbeck +mike steinbeck +mike steinbeck +mike van buren +mike van buren +mike white +mike white +mike white +mike white +mike young +mike young +mike young +mike zipper +mike zipper +mike zipper +nick allen +nick allen +nick brown +nick davidson +nick ellison +nick ellison +nick falkner +nick falkner +nick garcia +nick garcia +nick garcia +nick ichabod +nick ichabod +nick ichabod +nick johnson +nick johnson +nick laertes +nick miller +nick nixon +nick ovid +nick polk +nick quirinius +nick quirinius +nick robinson +nick robinson +nick steinbeck +nick thompson +nick underhill +nick van buren +nick xylophone +nick young +nick young +nick zipper +nick zipper +oscar allen +oscar brown +oscar carson +oscar carson +oscar carson +oscar carson +oscar carson +oscar davidson +oscar ellison +oscar ellison +oscar falkner +oscar garcia +oscar hernandez +oscar hernandez +oscar ichabod +oscar ichabod +oscar ichabod +oscar ichabod +oscar johnson +oscar johnson +oscar king +oscar king +oscar king +oscar laertes +oscar laertes +oscar laertes +oscar laertes +oscar nixon +oscar ovid +oscar ovid +oscar ovid +oscar polk +oscar polk +oscar quirinius +oscar quirinius +oscar quirinius +oscar quirinius +oscar robinson +oscar robinson +oscar robinson +oscar robinson +oscar steinbeck +oscar thompson +oscar thompson +oscar thompson +oscar thompson +oscar underhill +oscar van buren +oscar van buren +oscar van buren +oscar white +oscar white +oscar white +oscar white +oscar xylophone +oscar xylophone +oscar xylophone +oscar zipper +oscar zipper +oscar zipper +priscilla brown +priscilla brown +priscilla brown +priscilla carson +priscilla carson +priscilla carson +priscilla ichabod +priscilla ichabod +priscilla johnson +priscilla johnson +priscilla johnson +priscilla johnson +priscilla johnson +priscilla king +priscilla nixon +priscilla nixon +priscilla ovid +priscilla ovid +priscilla polk +priscilla quirinius +priscilla thompson +priscilla underhill +priscilla underhill +priscilla van buren +priscilla van buren +priscilla van buren +priscilla white +priscilla xylophone +priscilla xylophone +priscilla xylophone +priscilla young +priscilla young +priscilla zipper +priscilla zipper +quinn allen +quinn allen +quinn brown +quinn brown +quinn brown +quinn davidson +quinn davidson +quinn davidson +quinn davidson +quinn ellison +quinn ellison +quinn garcia +quinn garcia +quinn garcia +quinn garcia +quinn ichabod +quinn king +quinn king +quinn laertes +quinn laertes +quinn laertes +quinn nixon +quinn ovid +quinn quirinius +quinn robinson +quinn steinbeck +quinn steinbeck +quinn thompson +quinn thompson +quinn underhill +quinn underhill +quinn underhill +quinn van buren +quinn young +quinn zipper +quinn zipper +rachel allen +rachel allen +rachel brown +rachel brown +rachel brown +rachel brown +rachel brown +rachel carson +rachel carson +rachel davidson +rachel ellison +rachel falkner +rachel falkner +rachel falkner +rachel falkner +rachel johnson +rachel king +rachel king +rachel laertes +rachel laertes +rachel ovid +rachel ovid +rachel polk +rachel quirinius +rachel robinson +rachel robinson +rachel robinson +rachel thompson +rachel thompson +rachel thompson +rachel underhill +rachel white +rachel white +rachel young +rachel zipper +rachel zipper +sarah carson +sarah carson +sarah carson +sarah ellison +sarah falkner +sarah falkner +sarah garcia +sarah garcia +sarah garcia +sarah ichabod +sarah ichabod +sarah johnson +sarah johnson +sarah johnson +sarah johnson +sarah king +sarah king +sarah miller +sarah ovid +sarah robinson +sarah robinson +sarah steinbeck +sarah white +sarah white +sarah xylophone +sarah young +sarah zipper +tom brown +tom brown +tom carson +tom carson +tom carson +tom davidson +tom ellison +tom ellison +tom ellison +tom falkner +tom falkner +tom hernandez +tom hernandez +tom ichabod +tom johnson +tom johnson +tom king +tom laertes +tom laertes +tom miller +tom miller +tom miller +tom nixon +tom ovid +tom polk +tom polk +tom quirinius +tom quirinius +tom robinson +tom robinson +tom robinson +tom robinson +tom steinbeck +tom van buren +tom van buren +tom van buren +tom white +tom young +tom young +tom zipper +ulysses brown +ulysses carson +ulysses carson +ulysses carson +ulysses carson +ulysses davidson +ulysses ellison +ulysses garcia +ulysses hernandez +ulysses hernandez +ulysses hernandez +ulysses ichabod +ulysses ichabod +ulysses johnson +ulysses king +ulysses laertes +ulysses laertes +ulysses laertes +ulysses miller +ulysses miller +ulysses nixon +ulysses ovid +ulysses polk +ulysses polk +ulysses polk +ulysses polk +ulysses quirinius +ulysses robinson +ulysses steinbeck +ulysses steinbeck +ulysses thompson +ulysses underhill +ulysses underhill +ulysses underhill +ulysses underhill +ulysses underhill +ulysses underhill +ulysses underhill +ulysses van buren +ulysses white +ulysses white +ulysses xylophone +ulysses xylophone +ulysses xylophone +ulysses young +ulysses young +ulysses young +victor allen +victor allen +victor brown +victor brown +victor brown +victor brown +victor davidson +victor davidson +victor davidson +victor ellison +victor ellison +victor hernandez +victor hernandez +victor hernandez +victor hernandez +victor hernandez +victor johnson +victor johnson +victor johnson +victor king +victor king +victor laertes +victor laertes +victor miller +victor nixon +victor nixon +victor ovid +victor polk +victor quirinius +victor quirinius +victor robinson +victor robinson +victor steinbeck +victor steinbeck +victor steinbeck +victor thompson +victor van buren +victor van buren +victor white +victor white +victor xylophone +victor xylophone +victor xylophone +victor xylophone +victor xylophone +victor young +victor zipper +wendy allen +wendy allen +wendy allen +wendy brown +wendy brown +wendy ellison +wendy ellison +wendy falkner +wendy falkner +wendy falkner +wendy garcia +wendy garcia +wendy garcia +wendy garcia +wendy hernandez +wendy ichabod +wendy king +wendy king +wendy king +wendy laertes +wendy laertes +wendy laertes +wendy miller +wendy miller +wendy nixon +wendy nixon +wendy ovid +wendy ovid +wendy polk +wendy polk +wendy quirinius +wendy quirinius +wendy robinson +wendy robinson +wendy robinson +wendy steinbeck +wendy thompson +wendy thompson +wendy underhill +wendy underhill +wendy underhill +wendy van buren +wendy van buren +wendy white +wendy xylophone +wendy xylophone +wendy young +wendy young +xavier allen +xavier allen +xavier allen +xavier brown +xavier brown +xavier brown +xavier carson +xavier carson +xavier davidson +xavier davidson +xavier davidson +xavier ellison +xavier ellison +xavier garcia +xavier hernandez +xavier hernandez +xavier hernandez +xavier ichabod +xavier ichabod +xavier johnson +xavier johnson +xavier king +xavier king +xavier laertes +xavier ovid +xavier polk +xavier polk +xavier polk +xavier polk +xavier quirinius +xavier quirinius +xavier quirinius +xavier quirinius +xavier thompson +xavier underhill +xavier white +xavier white +xavier xylophone +xavier zipper +yuri allen +yuri allen +yuri brown +yuri brown +yuri carson +yuri carson +yuri ellison +yuri ellison +yuri falkner +yuri falkner +yuri garcia +yuri hernandez +yuri johnson +yuri johnson +yuri johnson +yuri king +yuri laertes +yuri laertes +yuri nixon +yuri nixon +yuri polk +yuri polk +yuri polk +yuri quirinius +yuri quirinius +yuri quirinius +yuri steinbeck +yuri steinbeck +yuri thompson +yuri underhill +yuri underhill +yuri white +yuri xylophone +zach allen +zach brown +zach brown +zach brown +zach brown +zach brown +zach carson +zach ellison +zach falkner +zach falkner +zach garcia +zach garcia +zach garcia +zach garcia +zach ichabod +zach ichabod +zach king +zach king +zach king +zach miller +zach miller +zach miller +zach ovid +zach ovid +zach ovid +zach ovid +zach quirinius +zach robinson +zach steinbeck +zach steinbeck +zach thompson +zach thompson +zach underhill +zach white +zach xylophone +zach xylophone +zach young +zach zipper +zach zipper +zach zipper diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-5-8d0ee3e1605f38214bfad28a5ce897cc b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-5-8d0ee3e1605f38214bfad28a5ce897cc new file mode 100644 index 0000000000000..ddb15e338263f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-5-8d0ee3e1605f38214bfad28a5ce897cc @@ -0,0 +1 @@ +10 oscar carson 65549 65549 diff --git a/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-0-b7cb25303831392a51cd996e758ac79a b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-0-b7cb25303831392a51cd996e758ac79a new file mode 100644 index 0000000000000..42e5151fe211b --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-0-b7cb25303831392a51cd996e758ac79a @@ -0,0 +1,1049 @@ +65536 1 +65536 1 +65536 1 +65536 1 +65536 1 +65536 1 +65536 1 +65537 1 +65537 1 +65537 1 +65537 1 +65537 1 +65538 1 +65538 1 +65538 1 +65539 1 +65539 1 +65539 1 +65540 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65542 1 +65542 1 +65542 1 +65542 1 +65542 1 +65543 1 +65543 1 +65544 1 +65544 1 +65544 1 +65544 1 +65544 1 +65544 1 +65544 1 +65544 2 +65545 1 +65545 1 +65545 1 +65546 2 +65547 1 +65547 1 +65547 1 +65547 1 +65548 1 +65548 1 +65548 1 +65548 1 +65548 1 +65548 1 +65548 2 +65549 1 +65549 1 +65549 1 +65549 1 +65549 1 +65549 1 +65549 1 +65549 3 +65550 1 +65550 1 +65550 1 +65550 1 +65550 1 +65550 1 +65550 2 +65551 1 +65551 1 +65551 1 +65552 1 +65552 1 +65552 1 +65552 1 +65552 2 +65553 1 +65553 1 +65553 1 +65553 1 +65553 1 +65554 1 +65554 1 +65555 2 +65556 1 +65556 1 +65556 1 +65556 1 +65556 1 +65557 1 +65557 1 +65558 1 +65559 1 +65559 1 +65559 1 +65559 1 +65559 1 +65560 1 +65560 1 +65560 1 +65560 2 +65560 2 +65561 1 +65561 2 +65561 2 +65562 1 +65562 1 +65562 1 +65562 1 +65562 1 +65562 2 +65563 1 +65563 1 +65563 1 +65563 1 +65564 1 +65564 1 +65564 1 +65564 1 +65564 2 +65564 2 +65564 2 +65565 1 +65565 1 +65566 1 +65566 2 +65567 1 +65568 1 +65568 1 +65568 1 +65569 1 +65570 1 +65570 1 +65570 1 +65570 1 +65570 1 +65570 2 +65571 1 +65571 1 +65571 1 +65571 2 +65572 1 +65572 1 +65572 1 +65572 1 +65572 1 +65573 1 +65573 1 +65573 3 +65574 1 +65574 1 +65574 1 +65575 1 +65575 1 +65575 1 +65575 2 +65576 1 +65576 1 +65576 1 +65576 3 +65577 1 +65578 1 +65578 1 +65578 1 +65578 1 +65578 1 +65578 1 +65578 2 +65579 1 +65579 1 +65579 1 +65579 1 +65579 3 +65580 1 +65580 2 +65580 2 +65581 1 +65581 1 +65581 1 +65581 1 +65581 2 +65581 2 +65582 1 +65582 1 +65582 1 +65582 1 +65583 1 +65583 1 +65583 2 +65583 2 +65584 1 +65584 1 +65584 1 +65585 1 +65585 1 +65585 1 +65585 1 +65585 2 +65586 1 +65586 1 +65586 1 +65586 1 +65586 2 +65587 1 +65587 1 +65587 1 +65587 2 +65587 3 +65588 1 +65588 1 +65588 1 +65588 1 +65588 2 +65589 1 +65589 1 +65589 1 +65589 1 +65589 2 +65590 1 +65590 1 +65590 1 +65590 1 +65590 1 +65591 1 +65591 1 +65591 2 +65591 3 +65592 1 +65593 1 +65593 1 +65594 1 +65594 1 +65594 1 +65594 3 +65595 1 +65595 1 +65595 1 +65595 2 +65595 2 +65595 4 +65596 1 +65596 1 +65596 1 +65596 2 +65596 2 +65597 1 +65597 1 +65597 2 +65598 1 +65598 2 +65599 1 +65599 1 +65599 1 +65599 1 +65599 3 +65600 1 +65600 1 +65600 2 +65600 2 +65601 1 +65601 2 +65601 2 +65602 1 +65602 1 +65602 1 +65602 1 +65602 1 +65602 2 +65603 1 +65603 1 +65603 1 +65603 2 +65603 2 +65604 1 +65604 2 +65604 2 +65604 3 +65605 2 +65606 1 +65606 1 +65606 1 +65606 1 +65606 2 +65606 2 +65607 1 +65607 1 +65607 1 +65607 1 +65607 2 +65607 3 +65608 1 +65608 1 +65608 1 +65608 2 +65609 1 +65610 1 +65610 1 +65610 3 +65610 4 +65610 5 +65611 1 +65611 1 +65611 1 +65611 1 +65612 1 +65612 2 +65612 2 +65612 3 +65613 1 +65614 1 +65614 1 +65614 2 +65615 1 +65615 1 +65615 2 +65615 2 +65616 1 +65616 2 +65617 1 +65617 2 +65617 2 +65617 2 +65618 1 +65618 1 +65618 2 +65618 2 +65618 2 +65619 1 +65619 1 +65619 1 +65619 1 +65619 2 +65619 2 +65620 1 +65620 1 +65620 1 +65620 1 +65620 1 +65620 3 +65621 1 +65621 1 +65622 1 +65622 1 +65622 1 +65622 3 +65622 3 +65622 4 +65623 1 +65623 1 +65623 1 +65623 1 +65623 1 +65623 2 +65623 4 +65624 1 +65624 2 +65624 2 +65624 2 +65624 4 +65625 1 +65625 1 +65625 1 +65625 1 +65626 1 +65626 1 +65626 1 +65626 2 +65627 1 +65627 1 +65627 2 +65627 3 +65628 1 +65628 1 +65628 1 +65628 2 +65628 2 +65628 3 +65629 1 +65629 1 +65629 1 +65629 3 +65630 1 +65630 2 +65631 1 +65632 2 +65632 2 +65633 1 +65633 2 +65633 2 +65633 2 +65634 1 +65634 1 +65634 1 +65634 2 +65635 1 +65635 2 +65635 3 +65636 1 +65636 1 +65636 2 +65637 1 +65637 1 +65637 1 +65637 1 +65637 1 +65637 1 +65637 2 +65637 3 +65638 1 +65638 1 +65638 2 +65639 1 +65640 2 +65641 1 +65641 1 +65641 1 +65641 5 +65642 4 +65643 1 +65643 1 +65643 2 +65643 2 +65643 2 +65643 2 +65643 2 +65643 2 +65643 4 +65644 1 +65644 1 +65644 1 +65644 1 +65644 2 +65644 2 +65644 3 +65645 1 +65645 1 +65645 1 +65646 1 +65646 1 +65646 1 +65647 1 +65647 2 +65648 1 +65648 1 +65648 2 +65648 2 +65649 1 +65649 1 +65649 2 +65649 2 +65650 1 +65650 1 +65650 1 +65650 1 +65650 2 +65650 2 +65651 1 +65651 1 +65651 2 +65651 2 +65651 3 +65652 1 +65652 1 +65652 2 +65653 1 +65653 2 +65653 2 +65653 3 +65654 1 +65654 1 +65654 1 +65654 1 +65654 1 +65654 1 +65654 2 +65654 2 +65654 2 +65655 1 +65655 1 +65656 1 +65656 1 +65656 1 +65656 1 +65656 2 +65656 3 +65657 1 +65657 2 +65657 2 +65657 2 +65657 2 +65658 1 +65658 1 +65658 1 +65658 1 +65658 1 +65658 2 +65658 2 +65658 2 +65658 2 +65658 2 +65659 1 +65659 1 +65659 1 +65659 1 +65659 1 +65659 2 +65659 2 +65659 3 +65660 1 +65660 1 +65660 2 +65661 1 +65661 2 +65661 2 +65661 2 +65661 3 +65662 1 +65662 1 +65662 2 +65662 2 +65662 2 +65662 2 +65663 2 +65663 2 +65663 2 +65663 3 +65664 1 +65664 1 +65664 1 +65664 1 +65664 1 +65664 2 +65664 2 +65664 2 +65665 2 +65666 1 +65666 1 +65666 2 +65667 1 +65667 1 +65667 1 +65667 1 +65668 3 +65669 1 +65669 1 +65669 1 +65669 1 +65669 1 +65669 3 +65670 1 +65670 1 +65670 2 +65670 2 +65670 3 +65671 2 +65671 2 +65671 3 +65672 1 +65672 1 +65672 1 +65672 2 +65672 2 +65673 2 +65673 2 +65673 3 +65673 4 +65673 4 +65674 1 +65675 1 +65675 1 +65675 2 +65675 2 +65675 2 +65675 2 +65675 3 +65675 3 +65676 1 +65676 1 +65676 2 +65676 3 +65677 1 +65677 1 +65677 1 +65677 2 +65677 3 +65677 4 +65678 1 +65678 1 +65678 1 +65679 1 +65679 2 +65680 1 +65680 1 +65680 2 +65680 2 +65680 2 +65680 5 +65681 1 +65681 2 +65681 2 +65681 2 +65681 4 +65681 4 +65682 1 +65682 2 +65682 2 +65683 1 +65683 1 +65683 2 +65684 1 +65684 1 +65684 2 +65685 2 +65685 2 +65685 2 +65685 3 +65685 3 +65685 4 +65686 1 +65686 2 +65686 3 +65686 3 +65687 1 +65687 1 +65687 2 +65687 3 +65688 1 +65688 2 +65689 1 +65689 3 +65690 2 +65690 2 +65691 1 +65691 1 +65691 1 +65691 1 +65691 1 +65691 2 +65691 3 +65691 3 +65692 1 +65692 2 +65693 1 +65693 2 +65693 2 +65693 2 +65693 2 +65693 3 +65693 5 +65694 1 +65694 2 +65694 2 +65694 3 +65695 1 +65695 2 +65695 2 +65695 3 +65696 1 +65696 2 +65696 2 +65696 4 +65697 1 +65697 1 +65697 1 +65697 1 +65697 2 +65697 5 +65698 1 +65698 2 +65699 1 +65699 1 +65699 1 +65699 2 +65699 2 +65699 2 +65700 1 +65700 2 +65700 2 +65701 1 +65701 2 +65701 3 +65702 2 +65702 2 +65702 2 +65702 3 +65703 1 +65703 2 +65703 3 +65703 3 +65704 1 +65704 1 +65704 2 +65704 2 +65704 3 +65704 3 +65705 1 +65705 1 +65705 3 +65705 4 +65706 1 +65706 1 +65706 1 +65706 3 +65706 4 +65707 2 +65707 2 +65708 2 +65708 3 +65709 1 +65709 1 +65709 2 +65710 1 +65710 2 +65711 1 +65711 2 +65711 2 +65711 2 +65711 2 +65711 2 +65712 1 +65712 2 +65712 3 +65712 3 +65712 4 +65713 1 +65713 2 +65713 3 +65713 3 +65713 6 +65714 1 +65714 1 +65714 2 +65715 1 +65715 1 +65715 2 +65715 2 +65715 4 +65716 1 +65716 2 +65716 2 +65716 2 +65716 4 +65716 4 +65717 1 +65717 2 +65717 2 +65717 2 +65717 5 +65718 1 +65718 2 +65718 3 +65718 3 +65719 1 +65719 1 +65719 2 +65719 3 +65720 1 +65720 2 +65720 2 +65720 3 +65720 4 +65721 1 +65721 1 +65721 1 +65721 1 +65721 3 +65721 3 +65721 3 +65722 2 +65722 3 +65722 5 +65723 2 +65723 3 +65724 1 +65724 1 +65724 2 +65724 2 +65724 3 +65724 3 +65725 1 +65726 2 +65726 2 +65726 4 +65727 1 +65727 1 +65727 3 +65727 3 +65727 4 +65728 1 +65728 2 +65729 2 +65730 1 +65730 1 +65730 2 +65730 4 +65731 1 +65731 1 +65731 1 +65731 3 +65732 1 +65732 2 +65732 2 +65733 1 +65733 1 +65733 2 +65733 3 +65733 3 +65733 6 +65734 2 +65734 2 +65735 1 +65735 4 +65735 4 +65736 1 +65736 2 +65736 2 +65736 3 +65737 1 +65737 1 +65737 2 +65737 3 +65737 4 +65738 3 +65738 3 +65738 4 +65738 4 +65739 1 +65739 2 +65739 3 +65740 2 +65740 2 +65740 3 +65741 1 +65742 1 +65742 2 +65742 3 +65743 1 +65743 1 +65743 1 +65743 2 +65744 1 +65744 2 +65745 2 +65745 2 +65745 3 +65745 6 +65746 1 +65746 2 +65746 2 +65746 2 +65746 2 +65747 1 +65747 1 +65747 1 +65747 2 +65747 2 +65747 3 +65747 3 +65748 1 +65748 3 +65749 2 +65749 3 +65749 3 +65749 3 +65749 4 +65750 1 +65750 1 +65750 2 +65750 3 +65750 3 +65751 1 +65751 2 +65751 2 +65751 3 +65751 3 +65751 4 +65752 1 +65752 3 +65753 2 +65753 3 +65754 2 +65754 4 +65755 1 +65755 2 +65755 2 +65755 3 +65755 3 +65755 3 +65755 3 +65755 5 +65755 5 +65756 2 +65756 3 +65756 3 +65756 5 +65757 1 +65757 1 +65757 1 +65757 2 +65757 3 +65758 1 +65758 2 +65758 2 +65758 3 +65758 4 +65759 2 +65759 2 +65759 2 +65759 4 +65759 4 +65760 2 +65760 4 +65760 5 +65760 7 +65761 1 +65762 1 +65762 1 +65762 2 +65762 4 +65762 5 +65763 1 +65763 2 +65763 2 +65763 2 +65763 4 +65764 3 +65764 3 +65764 4 +65765 2 +65765 2 +65765 3 +65766 1 +65766 1 +65766 3 +65766 3 +65766 3 +65767 2 +65767 3 +65767 3 +65768 3 +65769 1 +65769 2 +65769 2 +65769 2 +65769 3 +65769 5 +65770 2 +65770 3 +65771 2 +65771 3 +65771 3 +65772 2 +65773 1 +65773 2 +65773 2 +65773 2 +65773 2 +65773 3 +65773 3 +65773 4 +65774 2 +65774 2 +65774 2 +65774 2 +65774 3 +65774 3 +65775 1 +65775 2 +65775 3 +65775 3 +65775 5 +65776 1 +65776 3 +65776 3 +65776 5 +65776 6 +65777 2 +65777 3 +65777 4 +65777 4 +65778 1 +65778 2 +65778 2 +65778 2 +65778 3 +65778 3 +65778 4 +65778 5 +65779 3 +65779 3 +65779 3 +65779 3 +65779 4 +65779 4 +65779 5 +65780 1 +65780 2 +65780 3 +65781 1 +65781 3 +65781 3 +65782 2 +65782 2 +65782 3 +65782 4 +65782 5 +65783 2 +65783 3 +65783 3 +65783 3 +65783 3 +65783 3 +65784 2 +65784 2 +65784 2 +65784 4 +65784 6 +65785 2 +65785 7 +65786 1 +65786 2 +65786 3 +65786 3 +65786 4 +65787 1 +65787 2 +65787 3 +65787 3 +65788 1 +65788 1 +65788 2 +65788 4 +65789 1 +65789 1 +65789 2 +65789 2 +65789 2 +65789 2 +65789 4 +65790 2 +65790 2 +65790 4 +65791 2 +65791 2 diff --git a/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-1-a3d352560ac835993001665db6954965 b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-1-a3d352560ac835993001665db6954965 new file mode 100644 index 0000000000000..dc72606a83db6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-1-a3d352560ac835993001665db6954965 @@ -0,0 +1,1049 @@ + 1 + 1 + 1 +alice allen 1 +alice allen 1 +alice allen 1 +alice brown 1 +alice carson 1 +alice davidson 1 +alice falkner 1 +alice garcia 1 +alice hernandez 1 +alice hernandez 1 +alice johnson 1 +alice king 1 +alice king 1 +alice king 1 +alice laertes 1 +alice laertes 2 +alice miller 1 +alice nixon 1 +alice nixon 1 +alice nixon 1 +alice ovid 2 +alice polk 1 +alice quirinius 1 +alice quirinius 1 +alice robinson 1 +alice robinson 1 +alice steinbeck 1 +alice steinbeck 1 +alice steinbeck 1 +alice underhill 1 +alice van buren 1 +alice xylophone 1 +alice xylophone 1 +alice xylophone 2 +alice zipper 1 +alice zipper 1 +alice zipper 2 +bob brown 1 +bob brown 1 +bob brown 1 +bob carson 1 +bob davidson 1 +bob davidson 1 +bob davidson 1 +bob ellison 1 +bob ellison 1 +bob ellison 1 +bob ellison 1 +bob falkner 2 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 2 +bob garcia 2 +bob hernandez 1 +bob ichabod 1 +bob king 1 +bob king 1 +bob king 2 +bob laertes 1 +bob laertes 1 +bob miller 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob polk 1 +bob quirinius 1 +bob steinbeck 1 +bob van buren 1 +bob white 1 +bob white 1 +bob xylophone 1 +bob xylophone 1 +bob young 1 +bob zipper 1 +bob zipper 2 +bob zipper 2 +calvin allen 3 +calvin brown 1 +calvin brown 1 +calvin brown 1 +calvin carson 2 +calvin davidson 1 +calvin davidson 2 +calvin ellison 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 2 +calvin falkner 2 +calvin falkner 2 +calvin garcia 2 +calvin hernandez 3 +calvin johnson 1 +calvin laertes 1 +calvin laertes 1 +calvin nixon 1 +calvin nixon 1 +calvin nixon 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 2 +calvin ovid 2 +calvin polk 2 +calvin quirinius 1 +calvin quirinius 1 +calvin robinson 1 +calvin steinbeck 1 +calvin steinbeck 1 +calvin steinbeck 2 +calvin thompson 1 +calvin thompson 2 +calvin underhill 1 +calvin van buren 1 +calvin van buren 1 +calvin white 1 +calvin white 2 +calvin xylophone 1 +calvin xylophone 2 +calvin xylophone 2 +calvin young 1 +calvin young 2 +calvin zipper 3 +calvin zipper 4 +david allen 1 +david allen 1 +david brown 2 +david brown 3 +david davidson 1 +david davidson 2 +david davidson 3 +david davidson 3 +david ellison 1 +david ellison 2 +david ellison 3 +david hernandez 1 +david ichabod 1 +david ichabod 3 +david laertes 3 +david nixon 1 +david ovid 1 +david ovid 1 +david quirinius 1 +david quirinius 1 +david quirinius 3 +david robinson 1 +david robinson 4 +david thompson 1 +david underhill 1 +david underhill 2 +david underhill 3 +david van buren 1 +david van buren 2 +david white 1 +david xylophone 1 +david xylophone 1 +david xylophone 2 +david young 1 +david young 1 +ethan allen 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 2 +ethan brown 2 +ethan carson 1 +ethan ellison 1 +ethan ellison 2 +ethan falkner 1 +ethan falkner 1 +ethan garcia 1 +ethan hernandez 1 +ethan johnson 2 +ethan king 2 +ethan laertes 1 +ethan laertes 1 +ethan laertes 2 +ethan laertes 2 +ethan laertes 2 +ethan laertes 2 +ethan laertes 3 +ethan miller 1 +ethan nixon 2 +ethan ovid 2 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan polk 2 +ethan quirinius 1 +ethan quirinius 1 +ethan quirinius 2 +ethan robinson 1 +ethan robinson 2 +ethan underhill 2 +ethan van buren 1 +ethan white 1 +ethan white 2 +ethan xylophone 2 +ethan zipper 1 +ethan zipper 3 +fred davidson 2 +fred davidson 2 +fred davidson 3 +fred ellison 1 +fred ellison 2 +fred ellison 2 +fred falkner 1 +fred falkner 3 +fred falkner 4 +fred hernandez 2 +fred ichabod 2 +fred ichabod 3 +fred johnson 2 +fred king 2 +fred king 2 +fred laertes 2 +fred miller 3 +fred nixon 1 +fred nixon 1 +fred nixon 1 +fred nixon 3 +fred polk 1 +fred polk 1 +fred polk 1 +fred polk 2 +fred quirinius 2 +fred quirinius 3 +fred robinson 2 +fred steinbeck 1 +fred steinbeck 1 +fred steinbeck 1 +fred underhill 1 +fred van buren 2 +fred van buren 3 +fred van buren 3 +fred van buren 4 +fred white 2 +fred young 1 +fred young 2 +fred zipper 3 +gabriella allen 1 +gabriella allen 3 +gabriella brown 1 +gabriella brown 1 +gabriella carson 2 +gabriella davidson 1 +gabriella ellison 1 +gabriella ellison 3 +gabriella falkner 1 +gabriella falkner 1 +gabriella falkner 3 +gabriella garcia 1 +gabriella hernandez 1 +gabriella hernandez 1 +gabriella ichabod 1 +gabriella ichabod 2 +gabriella ichabod 2 +gabriella ichabod 2 +gabriella ichabod 4 +gabriella king 1 +gabriella king 2 +gabriella laertes 1 +gabriella miller 2 +gabriella ovid 1 +gabriella ovid 2 +gabriella polk 1 +gabriella polk 2 +gabriella steinbeck 1 +gabriella steinbeck 1 +gabriella thompson 1 +gabriella thompson 2 +gabriella thompson 3 +gabriella van buren 1 +gabriella van buren 2 +gabriella white 1 +gabriella young 1 +gabriella young 2 +gabriella zipper 1 +gabriella zipper 2 +holly allen 3 +holly brown 2 +holly brown 2 +holly falkner 2 +holly hernandez 2 +holly hernandez 2 +holly hernandez 2 +holly hernandez 3 +holly ichabod 1 +holly ichabod 2 +holly ichabod 2 +holly johnson 1 +holly johnson 3 +holly johnson 4 +holly king 2 +holly king 2 +holly laertes 3 +holly miller 2 +holly nixon 1 +holly nixon 2 +holly polk 1 +holly polk 2 +holly robinson 3 +holly thompson 1 +holly thompson 3 +holly thompson 4 +holly underhill 2 +holly underhill 2 +holly underhill 3 +holly underhill 3 +holly van buren 1 +holly white 4 +holly white 4 +holly xylophone 2 +holly young 1 +holly young 2 +holly zipper 1 +holly zipper 4 +irene allen 3 +irene brown 1 +irene brown 2 +irene brown 3 +irene carson 2 +irene ellison 2 +irene ellison 2 +irene falkner 1 +irene falkner 1 +irene garcia 1 +irene garcia 2 +irene garcia 3 +irene ichabod 1 +irene ichabod 2 +irene johnson 2 +irene laertes 1 +irene laertes 3 +irene laertes 4 +irene miller 1 +irene nixon 1 +irene nixon 3 +irene nixon 3 +irene ovid 2 +irene ovid 2 +irene ovid 2 +irene polk 1 +irene polk 1 +irene polk 2 +irene polk 2 +irene polk 4 +irene quirinius 2 +irene quirinius 3 +irene quirinius 4 +irene robinson 2 +irene steinbeck 1 +irene thompson 1 +irene underhill 2 +irene underhill 3 +irene van buren 2 +irene van buren 3 +irene xylophone 2 +jessica brown 2 +jessica carson 1 +jessica carson 2 +jessica carson 4 +jessica davidson 1 +jessica davidson 2 +jessica davidson 3 +jessica davidson 3 +jessica ellison 1 +jessica ellison 3 +jessica falkner 2 +jessica garcia 1 +jessica garcia 5 +jessica ichabod 2 +jessica johnson 2 +jessica johnson 3 +jessica miller 2 +jessica nixon 2 +jessica nixon 3 +jessica ovid 2 +jessica ovid 3 +jessica polk 5 +jessica quirinius 2 +jessica quirinius 2 +jessica quirinius 3 +jessica quirinius 3 +jessica robinson 1 +jessica thompson 2 +jessica thompson 3 +jessica underhill 2 +jessica underhill 2 +jessica underhill 4 +jessica van buren 2 +jessica white 3 +jessica white 3 +jessica white 3 +jessica white 3 +jessica white 4 +jessica xylophone 4 +jessica young 4 +jessica young 4 +jessica zipper 1 +jessica zipper 2 +jessica zipper 4 +katie allen 2 +katie brown 4 +katie davidson 3 +katie ellison 3 +katie ellison 3 +katie falkner 2 +katie garcia 2 +katie garcia 3 +katie hernandez 2 +katie ichabod 2 +katie ichabod 2 +katie ichabod 2 +katie king 1 +katie king 1 +katie king 2 +katie miller 2 +katie miller 3 +katie nixon 5 +katie ovid 1 +katie polk 2 +katie polk 3 +katie robinson 4 +katie van buren 2 +katie van buren 4 +katie white 1 +katie white 2 +katie xylophone 3 +katie young 2 +katie young 2 +katie young 3 +katie zipper 1 +katie zipper 3 +luke allen 2 +luke allen 2 +luke allen 2 +luke allen 3 +luke allen 3 +luke brown 2 +luke davidson 1 +luke davidson 3 +luke ellison 3 +luke ellison 5 +luke ellison 5 +luke falkner 2 +luke falkner 4 +luke garcia 1 +luke garcia 5 +luke ichabod 3 +luke ichabod 3 +luke johnson 1 +luke johnson 2 +luke johnson 3 +luke laertes 2 +luke laertes 3 +luke laertes 3 +luke laertes 3 +luke laertes 3 +luke miller 2 +luke ovid 1 +luke ovid 2 +luke polk 2 +luke polk 3 +luke quirinius 2 +luke robinson 1 +luke robinson 4 +luke thompson 1 +luke underhill 2 +luke underhill 3 +luke underhill 5 +luke van buren 2 +luke white 3 +luke xylophone 2 +luke zipper 1 +mike allen 4 +mike brown 4 +mike carson 1 +mike carson 2 +mike carson 4 +mike davidson 3 +mike davidson 4 +mike ellison 2 +mike ellison 3 +mike ellison 3 +mike ellison 4 +mike ellison 4 +mike falkner 1 +mike garcia 1 +mike garcia 2 +mike garcia 3 +mike hernandez 2 +mike hernandez 3 +mike ichabod 1 +mike king 1 +mike king 1 +mike king 3 +mike king 3 +mike king 4 +mike king 4 +mike miller 4 +mike nixon 3 +mike nixon 4 +mike polk 4 +mike polk 5 +mike polk 5 +mike quirinius 3 +mike steinbeck 2 +mike steinbeck 3 +mike steinbeck 3 +mike steinbeck 4 +mike van buren 2 +mike van buren 3 +mike white 3 +mike white 4 +mike white 5 +mike white 6 +mike young 2 +mike young 2 +mike young 4 +mike zipper 1 +mike zipper 4 +mike zipper 6 +nick allen 2 +nick allen 2 +nick brown 5 +nick davidson 1 +nick ellison 3 +nick ellison 4 +nick falkner 2 +nick falkner 3 +nick garcia 2 +nick garcia 4 +nick garcia 4 +nick ichabod 3 +nick ichabod 3 +nick ichabod 3 +nick johnson 4 +nick johnson 4 +nick laertes 2 +nick miller 3 +nick nixon 2 +nick ovid 3 +nick polk 4 +nick quirinius 1 +nick quirinius 3 +nick robinson 4 +nick robinson 4 +nick steinbeck 1 +nick thompson 2 +nick underhill 6 +nick van buren 2 +nick xylophone 3 +nick young 3 +nick young 5 +nick zipper 3 +nick zipper 4 +oscar allen 4 +oscar brown 3 +oscar carson 3 +oscar carson 4 +oscar carson 4 +oscar carson 5 +oscar carson 5 +oscar davidson 4 +oscar ellison 1 +oscar ellison 2 +oscar falkner 1 +oscar garcia 5 +oscar hernandez 1 +oscar hernandez 2 +oscar ichabod 2 +oscar ichabod 5 +oscar ichabod 5 +oscar ichabod 6 +oscar johnson 3 +oscar johnson 7 +oscar king 3 +oscar king 4 +oscar king 4 +oscar laertes 1 +oscar laertes 2 +oscar laertes 3 +oscar laertes 3 +oscar nixon 4 +oscar ovid 3 +oscar ovid 3 +oscar ovid 6 +oscar polk 5 +oscar polk 5 +oscar quirinius 2 +oscar quirinius 4 +oscar quirinius 5 +oscar quirinius 6 +oscar robinson 2 +oscar robinson 3 +oscar robinson 5 +oscar robinson 6 +oscar steinbeck 1 +oscar thompson 2 +oscar thompson 3 +oscar thompson 3 +oscar thompson 4 +oscar underhill 2 +oscar van buren 1 +oscar van buren 2 +oscar van buren 5 +oscar white 1 +oscar white 2 +oscar white 5 +oscar white 5 +oscar xylophone 3 +oscar xylophone 3 +oscar xylophone 4 +oscar zipper 2 +oscar zipper 2 +oscar zipper 2 +priscilla brown 2 +priscilla brown 2 +priscilla brown 4 +priscilla carson 3 +priscilla carson 5 +priscilla carson 7 +priscilla ichabod 1 +priscilla ichabod 4 +priscilla johnson 1 +priscilla johnson 2 +priscilla johnson 4 +priscilla johnson 4 +priscilla johnson 6 +priscilla king 3 +priscilla nixon 3 +priscilla nixon 6 +priscilla ovid 3 +priscilla ovid 7 +priscilla polk 4 +priscilla quirinius 3 +priscilla thompson 6 +priscilla underhill 1 +priscilla underhill 4 +priscilla van buren 3 +priscilla van buren 5 +priscilla van buren 5 +priscilla white 1 +priscilla xylophone 2 +priscilla xylophone 3 +priscilla xylophone 3 +priscilla young 5 +priscilla young 8 +priscilla zipper 3 +priscilla zipper 3 +quinn allen 1 +quinn allen 4 +quinn brown 3 +quinn brown 4 +quinn brown 4 +quinn davidson 2 +quinn davidson 4 +quinn davidson 6 +quinn davidson 7 +quinn ellison 3 +quinn ellison 8 +quinn garcia 2 +quinn garcia 3 +quinn garcia 3 +quinn garcia 5 +quinn ichabod 7 +quinn king 1 +quinn king 1 +quinn laertes 2 +quinn laertes 4 +quinn laertes 5 +quinn nixon 4 +quinn ovid 4 +quinn quirinius 5 +quinn robinson 3 +quinn steinbeck 4 +quinn steinbeck 5 +quinn thompson 4 +quinn thompson 6 +quinn underhill 2 +quinn underhill 3 +quinn underhill 7 +quinn van buren 1 +quinn young 2 +quinn zipper 3 +quinn zipper 4 +rachel allen 2 +rachel allen 3 +rachel brown 2 +rachel brown 3 +rachel brown 4 +rachel brown 4 +rachel brown 5 +rachel carson 2 +rachel carson 4 +rachel davidson 7 +rachel ellison 1 +rachel falkner 1 +rachel falkner 3 +rachel falkner 5 +rachel falkner 6 +rachel johnson 9 +rachel king 3 +rachel king 7 +rachel laertes 4 +rachel laertes 6 +rachel ovid 3 +rachel ovid 4 +rachel polk 3 +rachel quirinius 4 +rachel robinson 4 +rachel robinson 4 +rachel robinson 6 +rachel thompson 4 +rachel thompson 5 +rachel thompson 5 +rachel underhill 2 +rachel white 4 +rachel white 5 +rachel young 4 +rachel zipper 1 +rachel zipper 5 +sarah carson 1 +sarah carson 4 +sarah carson 7 +sarah ellison 1 +sarah falkner 4 +sarah falkner 5 +sarah garcia 2 +sarah garcia 2 +sarah garcia 4 +sarah ichabod 3 +sarah ichabod 3 +sarah johnson 3 +sarah johnson 5 +sarah johnson 5 +sarah johnson 6 +sarah king 3 +sarah king 5 +sarah miller 2 +sarah ovid 5 +sarah robinson 5 +sarah robinson 5 +sarah steinbeck 5 +sarah white 4 +sarah white 6 +sarah xylophone 3 +sarah young 5 +sarah zipper 6 +tom brown 2 +tom brown 5 +tom carson 1 +tom carson 3 +tom carson 5 +tom davidson 2 +tom ellison 3 +tom ellison 4 +tom ellison 6 +tom falkner 3 +tom falkner 4 +tom hernandez 1 +tom hernandez 3 +tom ichabod 4 +tom johnson 6 +tom johnson 7 +tom king 3 +tom laertes 3 +tom laertes 3 +tom miller 3 +tom miller 4 +tom miller 5 +tom nixon 4 +tom ovid 3 +tom polk 2 +tom polk 2 +tom quirinius 3 +tom quirinius 5 +tom robinson 2 +tom robinson 3 +tom robinson 3 +tom robinson 5 +tom steinbeck 2 +tom van buren 2 +tom van buren 3 +tom van buren 6 +tom white 5 +tom young 1 +tom young 5 +tom zipper 7 +ulysses brown 2 +ulysses carson 2 +ulysses carson 5 +ulysses carson 6 +ulysses carson 8 +ulysses davidson 3 +ulysses ellison 4 +ulysses garcia 3 +ulysses hernandez 3 +ulysses hernandez 3 +ulysses hernandez 4 +ulysses ichabod 1 +ulysses ichabod 3 +ulysses johnson 5 +ulysses king 2 +ulysses laertes 2 +ulysses laertes 5 +ulysses laertes 6 +ulysses miller 5 +ulysses miller 7 +ulysses nixon 4 +ulysses ovid 4 +ulysses polk 2 +ulysses polk 2 +ulysses polk 3 +ulysses polk 4 +ulysses quirinius 4 +ulysses robinson 1 +ulysses steinbeck 2 +ulysses steinbeck 5 +ulysses thompson 3 +ulysses underhill 2 +ulysses underhill 2 +ulysses underhill 3 +ulysses underhill 4 +ulysses underhill 4 +ulysses underhill 4 +ulysses underhill 5 +ulysses van buren 2 +ulysses white 6 +ulysses white 7 +ulysses xylophone 2 +ulysses xylophone 3 +ulysses xylophone 6 +ulysses young 1 +ulysses young 4 +ulysses young 7 +victor allen 2 +victor allen 3 +victor brown 1 +victor brown 4 +victor brown 5 +victor brown 7 +victor davidson 4 +victor davidson 4 +victor davidson 6 +victor ellison 4 +victor ellison 4 +victor hernandez 2 +victor hernandez 3 +victor hernandez 4 +victor hernandez 4 +victor hernandez 6 +victor johnson 4 +victor johnson 5 +victor johnson 6 +victor king 2 +victor king 6 +victor laertes 3 +victor laertes 5 +victor miller 5 +victor nixon 2 +victor nixon 3 +victor ovid 3 +victor polk 4 +victor quirinius 5 +victor quirinius 5 +victor robinson 5 +victor robinson 5 +victor steinbeck 3 +victor steinbeck 4 +victor steinbeck 5 +victor thompson 6 +victor van buren 5 +victor van buren 6 +victor white 2 +victor white 7 +victor xylophone 4 +victor xylophone 6 +victor xylophone 6 +victor xylophone 8 +victor xylophone 8 +victor young 5 +victor zipper 3 +wendy allen 5 +wendy allen 6 +wendy allen 6 +wendy brown 3 +wendy brown 5 +wendy ellison 3 +wendy ellison 5 +wendy falkner 2 +wendy falkner 4 +wendy falkner 6 +wendy garcia 4 +wendy garcia 4 +wendy garcia 7 +wendy garcia 7 +wendy hernandez 4 +wendy ichabod 4 +wendy king 4 +wendy king 5 +wendy king 7 +wendy laertes 2 +wendy laertes 3 +wendy laertes 5 +wendy miller 4 +wendy miller 4 +wendy nixon 3 +wendy nixon 5 +wendy ovid 5 +wendy ovid 9 +wendy polk 2 +wendy polk 5 +wendy quirinius 3 +wendy quirinius 4 +wendy robinson 5 +wendy robinson 6 +wendy robinson 6 +wendy steinbeck 3 +wendy thompson 4 +wendy thompson 5 +wendy underhill 4 +wendy underhill 5 +wendy underhill 6 +wendy van buren 6 +wendy van buren 6 +wendy white 4 +wendy xylophone 4 +wendy xylophone 6 +wendy young 1 +wendy young 6 +xavier allen 3 +xavier allen 4 +xavier allen 5 +xavier brown 2 +xavier brown 4 +xavier brown 6 +xavier carson 4 +xavier carson 5 +xavier davidson 7 +xavier davidson 8 +xavier davidson 9 +xavier ellison 8 +xavier ellison 8 +xavier garcia 5 +xavier hernandez 5 +xavier hernandez 6 +xavier hernandez 9 +xavier ichabod 3 +xavier ichabod 4 +xavier johnson 2 +xavier johnson 9 +xavier king 3 +xavier king 5 +xavier laertes 4 +xavier ovid 4 +xavier polk 3 +xavier polk 4 +xavier polk 4 +xavier polk 8 +xavier quirinius 3 +xavier quirinius 5 +xavier quirinius 6 +xavier quirinius 6 +xavier thompson 4 +xavier underhill 2 +xavier white 3 +xavier white 3 +xavier xylophone 4 +xavier zipper 3 +yuri allen 2 +yuri allen 3 +yuri brown 2 +yuri brown 3 +yuri carson 5 +yuri carson 6 +yuri ellison 6 +yuri ellison 6 +yuri falkner 6 +yuri falkner 10 +yuri garcia 1 +yuri hernandez 5 +yuri johnson 5 +yuri johnson 5 +yuri johnson 6 +yuri king 7 +yuri laertes 7 +yuri laertes 8 +yuri nixon 3 +yuri nixon 3 +yuri polk 3 +yuri polk 5 +yuri polk 6 +yuri quirinius 3 +yuri quirinius 4 +yuri quirinius 7 +yuri steinbeck 1 +yuri steinbeck 2 +yuri thompson 3 +yuri underhill 4 +yuri underhill 4 +yuri white 8 +yuri xylophone 3 +zach allen 4 +zach brown 5 +zach brown 5 +zach brown 5 +zach brown 5 +zach brown 7 +zach carson 5 +zach ellison 2 +zach falkner 4 +zach falkner 6 +zach garcia 4 +zach garcia 5 +zach garcia 7 +zach garcia 8 +zach ichabod 4 +zach ichabod 4 +zach king 4 +zach king 5 +zach king 8 +zach miller 1 +zach miller 3 +zach miller 4 +zach ovid 4 +zach ovid 5 +zach ovid 5 +zach ovid 7 +zach quirinius 8 +zach robinson 5 +zach steinbeck 4 +zach steinbeck 6 +zach thompson 3 +zach thompson 4 +zach underhill 3 +zach white 6 +zach xylophone 3 +zach xylophone 5 +zach young 4 +zach zipper 4 +zach zipper 4 +zach zipper 5 diff --git a/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-2-fafa16c0f7697ca28aeb6f2698799562 b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-2-fafa16c0f7697ca28aeb6f2698799562 new file mode 100644 index 0000000000000..76cbeb254c0e9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-2-fafa16c0f7697ca28aeb6f2698799562 @@ -0,0 +1,1049 @@ +0.08 1 +0.1 1 +0.13 1 +0.15 1 +0.27 1 +0.28 1 +0.43 1 +0.52 1 +0.56 1 +0.6 1 +0.61 1 +0.79 1 +0.84 1 +0.98 1 +1.02 1 +1.08 1 +1.08 1 +1.12 1 +1.21 1 +1.25 1 +1.27 1 +1.29 1 +1.31 1 +1.58 1 +1.87 1 +1.91 1 +1.92 1 +2.07 1 +2.18 1 +2.2 1 +2.35 1 +2.6 1 +2.79 1 +2.92 1 +2.96 1 +2.96 1 +2.97 1 +3.0 1 +3.21 1 +3.28 1 +3.33 1 +3.61 1 +3.62 1 +3.82 1 +3.86 1 +3.96 1 +3.97 1 +4.17 1 +4.32 1 +4.35 1 +4.41 1 +4.46 1 +4.47 1 +4.57 1 +4.59 1 +4.71 1 +4.72 1 +4.79 1 +4.8 1 +4.92 1 +5.08 1 +5.24 1 +5.28 1 +5.4 1 +5.44 1 +5.45 1 +5.51 1 +5.54 1 +5.62 1 +5.67 1 +5.85 1 +5.88 1 +6.29 1 +6.55 1 +6.57 1 +6.63 1 +6.67 1 +6.72 1 +6.74 1 +6.84 1 +6.87 1 +7.05 1 +7.06 1 +7.11 1 +7.54 1 +7.56 1 +7.79 1 +7.82 1 +7.96 1 +7.96 1 +7.98 1 +8.07 1 +8.07 1 +8.32 1 +8.37 1 +8.42 1 +8.45 1 +8.45 1 +8.45 1 +8.45 1 +8.57 1 +8.61 1 +8.67 2 +8.71 1 +8.79 1 +8.91 1 +9.04 1 +9.13 1 +9.19 1 +9.22 1 +9.25 1 +9.26 1 +9.35 1 +9.48 1 +9.56 1 +9.57 1 +9.57 1 +9.68 1 +9.7 1 +9.71 1 +9.74 1 +9.8 1 +9.81 1 +9.93 1 +10.09 1 +10.09 1 +10.13 1 +10.16 1 +10.17 1 +10.19 1 +10.2 1 +10.22 1 +10.25 1 +10.26 1 +10.29 1 +10.6 1 +10.66 1 +10.67 1 +10.73 1 +11.15 1 +11.18 1 +11.19 1 +11.22 1 +11.34 1 +11.55 1 +11.57 1 +11.68 1 +11.82 1 +11.89 1 +11.91 1 +12.02 1 +12.16 1 +12.19 1 +12.32 1 +12.42 1 +12.44 1 +12.45 1 +12.46 1 +12.5 1 +12.54 1 +12.85 1 +12.9 1 +13.01 1 +13.1 1 +13.15 1 +13.35 1 +13.87 1 +13.89 1 +13.94 1 +13.99 1 +14.13 1 +14.21 2 +14.3 1 +14.44 1 +14.84 2 +14.92 1 +14.92 1 +14.93 1 +15.1 1 +15.15 1 +15.18 1 +15.22 1 +15.26 2 +15.3 1 +15.37 1 +15.45 1 +15.63 2 +15.75 1 +15.81 1 +15.86 1 +15.9 1 +15.92 1 +16.08 1 +16.09 1 +16.24 1 +16.25 1 +16.48 1 +16.69 1 +16.99 1 +16.99 1 +17.16 1 +17.37 1 +17.74 1 +17.79 1 +17.87 1 +18.2 1 +18.5 1 +18.56 1 +18.63 1 +18.63 1 +18.86 1 +18.89 1 +18.93 1 +19.0 1 +19.03 1 +19.06 1 +19.06 1 +19.13 1 +19.14 1 +19.28 1 +19.69 1 +20.07 1 +20.38 1 +20.64 1 +20.67 1 +20.79 1 +20.81 1 +20.82 1 +20.82 1 +21.18 1 +21.19 1 +21.23 1 +21.28 1 +21.32 1 +21.45 1 +21.49 1 +21.61 1 +21.7 1 +21.8 1 +21.94 1 +22.01 1 +22.08 1 +22.12 1 +22.12 1 +22.25 1 +22.27 1 +22.36 1 +22.68 1 +22.78 1 +22.85 1 +22.85 1 +22.94 1 +23.07 1 +23.13 1 +23.17 1 +23.19 1 +23.44 1 +23.45 1 +23.6 1 +23.77 1 +23.96 1 +24.02 1 +24.28 1 +24.49 1 +24.52 1 +24.73 1 +24.79 1 +24.8 1 +24.83 1 +24.86 1 +25.11 1 +25.28 1 +25.37 1 +25.42 1 +25.55 1 +25.67 1 +25.88 1 +26.08 1 +26.39 1 +26.43 1 +26.47 1 +26.49 1 +26.49 1 +26.64 1 +26.71 2 +26.73 1 +26.76 1 +27.07 1 +27.12 1 +27.3 1 +27.31 1 +27.63 1 +27.66 1 +27.72 2 +27.87 1 +28.11 1 +28.31 1 +28.45 1 +28.5 1 +28.56 1 +28.69 1 +28.71 1 +28.79 1 +28.89 1 +28.95 1 +29.02 1 +29.24 1 +29.36 1 +29.4 1 +29.41 2 +29.54 1 +29.59 1 +29.78 1 +30.25 2 +30.36 1 +30.37 1 +30.61 1 +30.62 1 +30.63 1 +30.65 1 +30.71 1 +30.81 1 +31.01 1 +31.15 1 +31.4 1 +31.61 1 +31.67 1 +31.77 1 +31.86 1 +31.91 1 +32.01 1 +32.18 1 +32.2 1 +32.23 1 +32.25 1 +32.37 1 +32.41 1 +32.47 1 +32.52 1 +32.75 1 +32.89 2 +32.92 1 +33.36 1 +33.52 1 +33.55 1 +33.58 1 +33.67 1 +33.76 1 +33.83 1 +33.85 2 +33.87 1 +34.03 1 +34.21 1 +34.35 1 +34.41 2 +34.58 1 +34.73 1 +34.97 1 +35.0 2 +35.08 1 +35.13 1 +35.17 1 +35.17 1 +35.56 1 +35.62 1 +35.65 1 +35.68 1 +35.72 1 +35.8 1 +35.89 1 +36.22 1 +36.26 1 +36.58 1 +36.7 1 +36.79 1 +36.89 1 +36.95 1 +37.07 2 +37.1 1 +37.14 1 +37.14 1 +37.24 1 +37.59 1 +37.6 1 +37.72 2 +37.78 1 +37.8 1 +37.85 1 +37.9 1 +38.05 1 +38.05 1 +38.3 2 +38.33 1 +38.57 1 +38.62 1 +38.79 1 +38.85 1 +38.88 1 +38.94 1 +39.01 1 +39.03 1 +39.18 1 +39.21 1 +39.34 1 +39.69 1 +39.81 1 +39.82 1 +39.83 1 +39.87 1 +39.9 1 +39.98 1 +40.0 1 +40.04 1 +40.17 1 +40.24 1 +40.42 1 +40.44 1 +40.78 1 +40.8 1 +40.98 1 +41.2 2 +41.29 1 +41.29 1 +41.31 2 +41.34 1 +41.34 1 +41.36 1 +41.44 1 +41.45 2 +41.62 1 +41.68 1 +41.71 1 +41.81 1 +41.85 1 +41.87 1 +41.89 2 +42.24 1 +42.31 1 +42.42 2 +42.48 1 +42.51 1 +42.55 1 +42.56 1 +42.67 2 +42.76 1 +42.85 1 +43.01 1 +43.02 1 +43.13 1 +43.16 1 +43.17 1 +43.19 1 +43.31 1 +43.37 1 +43.57 1 +43.71 1 +43.73 1 +43.92 1 +44.1 1 +44.22 1 +44.27 1 +44.43 1 +44.57 1 +45.06 2 +45.1 1 +45.19 1 +45.19 1 +45.24 1 +45.34 1 +45.35 1 +45.42 1 +45.45 1 +45.56 1 +45.59 1 +45.68 1 +45.92 1 +45.99 2 +46.02 1 +46.09 1 +46.1 1 +46.15 1 +46.18 1 +46.21 1 +46.27 1 +46.43 1 +46.45 1 +46.62 1 +46.8 1 +46.86 1 +46.87 1 +46.88 1 +46.97 2 +47.08 2 +47.27 1 +47.32 2 +47.57 1 +47.59 1 +47.69 1 +47.88 1 +48.01 1 +48.08 1 +48.11 1 +48.15 1 +48.22 1 +48.23 1 +48.25 1 +48.28 1 +48.37 1 +48.45 1 +48.45 1 +48.52 1 +48.59 1 +49.12 1 +49.28 2 +49.44 1 +49.68 1 +49.77 2 +49.78 1 +50.02 3 +50.08 1 +50.09 1 +50.26 1 +50.28 1 +50.31 1 +50.32 1 +50.4 1 +50.41 1 +50.66 1 +50.7 1 +50.83 2 +50.92 2 +50.96 1 +51.25 1 +51.29 1 +51.29 2 +51.72 1 +51.79 2 +51.84 1 +51.85 1 +52.17 1 +52.23 1 +52.44 1 +52.5 1 +52.53 2 +52.72 1 +52.73 1 +52.85 2 +52.87 1 +53.02 1 +53.06 1 +53.18 2 +53.27 1 +53.59 2 +53.78 1 +53.93 1 +53.94 1 +54.1 2 +54.31 1 +54.34 1 +54.43 1 +54.44 1 +54.47 1 +54.73 1 +54.75 1 +54.83 1 +54.99 1 +55.1 1 +55.18 1 +55.2 1 +55.39 1 +55.51 1 +55.63 1 +55.99 1 +56.04 1 +56.07 1 +56.1 1 +56.15 1 +56.33 1 +56.62 1 +56.68 2 +56.81 1 +57.08 1 +57.11 1 +57.12 2 +57.23 1 +57.25 1 +57.29 1 +57.35 1 +57.37 1 +57.46 1 +57.64 1 +57.67 1 +57.89 2 +57.93 1 +58.0 1 +58.08 2 +58.09 2 +58.13 1 +58.43 1 +58.52 1 +58.66 1 +58.67 1 +58.75 1 +58.86 2 +59.07 1 +59.16 1 +59.21 1 +59.34 1 +59.43 1 +59.45 1 +59.45 2 +59.5 1 +59.55 1 +59.61 1 +59.62 1 +59.68 1 +59.68 1 +59.7 1 +59.71 1 +59.83 1 +59.87 1 +59.99 1 +60.02 1 +60.06 1 +60.12 1 +60.13 1 +60.22 2 +60.26 1 +60.26 1 +60.53 1 +60.6 2 +60.71 1 +60.85 1 +61.21 1 +61.7 1 +61.86 1 +61.88 1 +61.92 2 +61.94 1 +62.14 1 +62.2 1 +62.23 1 +62.3 1 +62.39 1 +62.52 1 +62.72 1 +62.74 1 +62.85 2 +62.9 3 +62.92 1 +63.12 1 +63.33 1 +63.35 2 +63.42 1 +63.51 1 +63.9 1 +64.0 1 +64.22 1 +64.25 1 +64.3 1 +64.36 2 +64.46 1 +64.65 2 +64.67 1 +64.77 1 +64.87 1 +64.95 1 +65.02 1 +65.02 1 +65.38 1 +65.43 1 +65.43 1 +65.44 2 +65.55 1 +65.62 1 +65.7 1 +65.72 1 +66.17 1 +66.17 2 +66.36 1 +66.51 1 +66.61 1 +66.61 1 +66.67 1 +66.89 1 +67.12 1 +67.18 1 +67.26 1 +67.38 1 +67.45 1 +67.48 1 +67.59 1 +67.94 1 +67.98 1 +68.01 2 +68.04 1 +68.22 1 +68.25 1 +68.25 1 +68.32 1 +68.41 1 +68.5 1 +68.81 1 +68.85 2 +68.89 1 +68.95 1 +68.96 1 +69.32 2 +69.53 1 +69.74 3 +69.8 2 +69.88 1 +69.96 1 +69.97 1 +70.0 2 +70.04 1 +70.06 1 +70.24 1 +70.35 1 +70.38 1 +70.39 1 +70.52 1 +70.53 1 +70.56 1 +70.85 1 +70.89 1 +70.93 1 +71.01 1 +71.07 2 +71.13 1 +71.19 1 +71.26 1 +71.31 1 +71.32 1 +71.35 1 +71.5 1 +71.54 1 +71.55 3 +71.68 1 +71.68 2 +71.78 2 +71.8 1 +71.89 2 +72.04 1 +72.18 1 +72.51 1 +72.53 2 +72.56 1 +72.62 1 +72.79 1 +72.98 1 +73.18 1 +73.32 1 +73.48 2 +73.63 2 +73.65 1 +73.68 1 +73.88 1 +73.93 1 +74.0 1 +74.02 1 +74.15 1 +74.19 1 +74.19 1 +74.3 1 +74.42 1 +74.45 1 +74.52 1 +74.53 1 +74.59 1 +74.62 1 +74.72 1 +74.78 1 +75.03 2 +75.1 3 +75.19 1 +75.29 1 +75.35 2 +75.42 2 +75.66 2 +75.73 2 +75.83 1 +75.88 1 +76.05 3 +76.1 1 +76.28 1 +76.28 1 +76.33 1 +76.52 1 +76.69 1 +76.7 1 +76.71 1 +76.72 2 +76.72 2 +76.74 1 +76.92 1 +76.93 1 +77.02 3 +77.1 1 +77.36 2 +77.42 1 +77.57 2 +77.66 1 +77.81 1 +77.84 2 +77.89 1 +77.97 1 +78.21 2 +78.26 1 +78.28 1 +78.3 2 +78.31 1 +78.62 1 +78.64 1 +78.73 1 +78.89 1 +78.98 2 +79.12 1 +79.19 1 +79.21 1 +79.38 1 +79.42 1 +79.48 1 +79.48 2 +79.49 2 +79.54 1 +79.55 2 +79.75 1 +79.83 1 +79.96 1 +79.97 2 +79.99 1 +80.23 2 +80.3 1 +80.3 2 +80.46 1 +80.52 1 +80.58 1 +80.6 1 +80.71 1 +80.74 1 +80.84 2 +80.92 1 +80.96 1 +80.97 1 +80.99 1 +81.17 1 +81.32 1 +81.32 1 +81.47 1 +81.58 1 +81.64 1 +81.66 1 +82.24 1 +82.3 1 +82.34 2 +82.41 1 +82.52 1 +82.55 1 +82.56 1 +82.72 1 +82.97 1 +83.08 1 +83.27 1 +83.33 1 +83.4 1 +83.54 2 +83.57 1 +83.58 1 +83.87 1 +83.92 1 +83.93 1 +84.03 1 +84.23 2 +84.31 1 +84.38 1 +84.4 1 +84.69 1 +84.72 1 +84.83 1 +85.0 1 +85.03 2 +85.1 1 +85.14 1 +85.23 1 +85.49 1 +85.49 2 +85.51 2 +85.74 1 +85.76 1 +85.87 1 +85.9 1 +86.0 1 +86.22 1 +86.23 1 +86.63 1 +86.69 1 +86.92 2 +86.93 1 +86.93 1 +87.14 2 +87.22 1 +87.4 1 +87.48 1 +87.57 1 +87.61 1 +87.67 1 +87.83 2 +87.94 1 +87.99 1 +88.02 1 +88.05 1 +88.07 2 +88.17 1 +88.22 1 +88.36 1 +88.47 1 +88.48 1 +88.55 1 +88.77 1 +88.78 1 +88.8 1 +88.91 2 +89.01 2 +89.03 1 +89.1 3 +89.15 2 +89.28 1 +89.38 1 +89.53 1 +89.55 1 +89.55 1 +89.55 1 +89.8 1 +89.81 1 +89.93 1 +90.05 1 +90.05 1 +90.07 1 +90.12 1 +90.2 1 +90.25 1 +90.28 2 +90.35 1 +90.38 1 +90.51 1 +90.56 2 +90.69 1 +90.69 1 +90.73 1 +90.77 1 +91.05 1 +91.16 1 +91.42 1 +91.48 1 +91.53 1 +91.61 1 +91.63 1 +91.78 1 +91.88 1 +91.97 1 +92.05 2 +92.11 2 +92.33 2 +92.37 1 +92.4 2 +92.55 1 +92.61 1 +92.82 1 +92.96 2 +92.98 1 +93.03 1 +93.09 1 +93.11 1 +93.61 1 +93.64 1 +93.73 1 +94.08 1 +94.15 1 +94.25 1 +94.27 1 +94.31 1 +94.33 1 +94.34 1 +94.38 1 +94.43 1 +94.54 1 +94.66 1 +94.68 1 +94.68 1 +94.72 1 +95.07 1 +95.11 1 +95.28 1 +95.33 1 +95.34 1 +95.38 2 +95.48 1 +95.53 1 +95.53 1 +95.81 1 +95.81 2 +95.84 1 +96.09 1 +96.23 1 +96.25 1 +96.29 1 +96.38 1 +96.62 1 +96.68 1 +96.73 1 +96.78 1 +96.91 2 +96.94 1 +97.09 1 +97.24 1 +97.26 1 +97.39 1 +97.46 1 +97.51 2 +97.56 1 +97.57 1 +97.65 2 +97.68 1 +97.71 4 +97.81 1 +97.83 1 +97.87 1 +98.18 2 +98.22 1 +98.23 1 +98.31 1 +98.48 1 +98.51 1 +98.57 1 +98.72 1 +98.96 1 +99.13 1 +99.15 1 +99.21 1 +99.24 1 +99.29 1 +99.36 1 +99.62 1 +99.65 1 +99.67 1 +99.68 1 +99.91 1 +99.92 1 diff --git a/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-3-bda0e7c77d6f4712a03389cb5032bc6d b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-3-bda0e7c77d6f4712a03389cb5032bc6d new file mode 100644 index 0000000000000..a9ec53c0cb21c --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-3-bda0e7c77d6f4712a03389cb5032bc6d @@ -0,0 +1,1049 @@ +0.02 1 +0.21 1 +0.27 1 +0.37 1 +0.37 1 +0.47 1 +0.48 1 +0.52 1 +0.6 1 +0.63 1 +0.63 1 +0.66 1 +0.73 1 +0.74 1 +0.74 1 +0.78 1 +0.8 1 +0.86 1 +0.86 1 +0.98 1 +1.17 1 +1.29 1 +1.3 1 +1.31 1 +1.37 1 +1.42 1 +1.45 1 +1.5 1 +1.53 1 +1.61 1 +1.62 1 +1.69 1 +1.71 1 +1.79 1 +1.98 1 +2.07 1 +2.09 1 +2.13 1 +2.16 1 +2.26 1 +2.29 1 +2.34 1 +2.34 1 +2.36 1 +2.43 1 +2.43 1 +2.52 1 +2.53 1 +2.53 1 +2.61 1 +2.63 1 +2.74 1 +2.75 1 +2.75 1 +2.79 1 +2.8 1 +2.82 1 +2.89 1 +2.89 1 +2.89 1 +2.9 1 +2.92 1 +3.03 1 +3.07 1 +3.07 1 +3.12 1 +3.15 1 +3.17 1 +3.27 1 +3.27 1 +3.29 1 +3.4 1 +3.4 1 +3.4 1 +3.4 1 +3.42 1 +3.66 1 +3.67 1 +3.69 1 +3.71 1 +3.78 1 +3.83 1 +3.86 1 +3.92 1 +3.98 1 +3.99 1 +4.04 1 +4.16 1 +4.25 1 +4.27 1 +4.44 1 +4.48 1 +4.53 1 +4.57 1 +4.58 1 +4.62 1 +4.81 1 +4.82 1 +4.83 1 +4.92 1 +4.95 1 +4.96 1 +4.97 1 +4.98 1 +5.09 1 +5.11 1 +5.19 1 +5.23 1 +5.3 1 +5.31 1 +5.31 1 +5.35 1 +5.42 1 +5.51 1 +5.55 1 +5.58 1 +5.74 1 +5.82 1 +5.84 1 +5.93 1 +5.93 1 +5.96 1 +6.06 1 +6.06 1 +6.21 1 +6.28 1 +6.46 1 +6.52 2 +6.54 1 +6.56 1 +6.57 1 +6.58 1 +6.61 1 +6.62 1 +6.76 1 +6.81 1 +6.81 1 +6.96 1 +6.98 1 +7.02 1 +7.03 1 +7.14 1 +7.18 1 +7.24 1 +7.24 1 +7.31 1 +7.36 1 +7.37 1 +7.45 1 +7.53 1 +7.62 1 +7.66 1 +7.71 1 +7.71 1 +7.8 1 +7.92 1 +8.05 1 +8.09 1 +8.21 1 +8.33 1 +8.33 1 +8.49 1 +8.49 1 +8.52 1 +8.56 1 +8.61 1 +8.62 2 +8.72 1 +8.76 1 +8.79 1 +8.82 1 +8.84 1 +8.95 1 +8.98 1 +9.14 1 +9.19 1 +9.21 1 +9.22 1 +9.26 1 +9.27 1 +9.39 2 +9.4 1 +9.42 1 +9.5 1 +9.51 1 +9.56 1 +9.6 1 +9.61 1 +9.62 1 +9.64 1 +9.81 1 +9.87 1 +9.88 1 +9.93 1 +9.94 1 +9.96 1 +9.99 1 +10.15 1 +10.21 1 +10.22 1 +10.23 1 +10.24 1 +10.36 1 +10.38 1 +10.38 1 +10.41 1 +10.47 1 +10.49 1 +10.49 1 +10.51 1 +10.52 1 +10.7 1 +10.71 1 +10.85 1 +10.99 1 +11.02 1 +11.12 1 +11.12 1 +11.16 1 +11.2 1 +11.26 1 +11.27 1 +11.35 1 +11.35 1 +11.4 1 +11.43 1 +11.44 1 +11.44 1 +11.46 1 +11.48 1 +11.5 1 +11.54 1 +11.63 1 +11.66 1 +11.69 1 +11.83 1 +11.9 1 +11.91 1 +11.96 1 +12.02 1 +12.13 1 +12.14 1 +12.15 1 +12.3 1 +12.3 1 +12.3 2 +12.34 1 +12.35 1 +12.43 1 +12.43 1 +12.64 1 +12.66 1 +12.7 1 +12.72 1 +12.73 1 +12.74 2 +12.82 1 +12.85 1 +13.02 1 +13.04 1 +13.08 1 +13.14 1 +13.2 1 +13.2 1 +13.22 1 +13.23 1 +13.3 1 +13.3 1 +13.44 1 +13.44 1 +13.44 1 +13.49 1 +13.6 1 +13.66 1 +13.71 1 +13.72 1 +13.8 1 +13.83 1 +13.84 1 +13.88 1 +13.95 1 +14.07 1 +14.16 1 +14.17 1 +14.22 1 +14.24 1 +14.26 1 +14.29 1 +14.33 1 +14.39 1 +14.44 1 +14.51 1 +14.51 1 +14.52 1 +14.62 1 +14.69 1 +14.72 1 +14.75 1 +14.83 1 +14.83 1 +14.84 1 +14.9 1 +14.91 1 +14.92 1 +14.99 1 +15.0 1 +15.01 1 +15.09 1 +15.09 1 +15.09 1 +15.1 1 +15.12 1 +15.13 1 +15.16 1 +15.18 1 +15.22 1 +15.27 1 +15.28 1 +15.32 1 +15.38 1 +15.46 1 +15.46 1 +15.51 1 +15.54 1 +15.87 1 +15.94 1 +15.97 1 +15.98 1 +16.04 2 +16.1 1 +16.12 1 +16.13 1 +16.15 1 +16.29 1 +16.35 1 +16.36 1 +16.38 1 +16.4 1 +16.42 1 +16.47 1 +16.49 1 +16.54 1 +16.61 1 +16.66 1 +16.79 1 +16.79 1 +16.82 1 +16.87 1 +16.87 1 +16.9 1 +16.9 1 +16.91 1 +16.92 1 +17.03 1 +17.03 2 +17.08 1 +17.15 1 +17.19 1 +17.29 1 +17.33 1 +17.44 1 +17.46 1 +17.47 1 +17.51 1 +17.52 2 +17.55 1 +17.59 1 +17.63 1 +17.69 1 +17.76 1 +17.86 1 +17.89 1 +17.99 1 +18.09 1 +18.19 1 +18.2 1 +18.28 1 +18.29 1 +18.31 1 +18.34 1 +18.35 1 +18.36 1 +18.38 1 +18.38 1 +18.41 1 +18.47 1 +18.48 1 +18.79 1 +18.82 1 +18.83 1 +18.86 1 +18.86 1 +19.03 1 +19.12 1 +19.15 1 +19.2 1 +19.31 1 +19.32 1 +19.41 1 +19.47 1 +19.47 2 +19.56 1 +19.59 1 +19.63 1 +19.65 1 +19.72 1 +19.72 1 +19.79 1 +19.79 1 +19.85 1 +19.87 1 +19.9 1 +19.92 1 +19.93 1 +19.98 1 +20.02 1 +20.02 1 +20.17 1 +20.19 1 +20.22 1 +20.3 1 +20.3 1 +20.34 1 +20.39 1 +20.42 1 +20.42 1 +20.44 1 +20.55 1 +20.55 1 +20.56 1 +20.56 1 +20.58 1 +20.58 1 +20.64 1 +20.65 1 +20.75 1 +20.76 1 +20.76 1 +20.8 1 +20.82 1 +20.91 1 +20.93 1 +20.94 1 +20.94 1 +20.97 1 +21.0 1 +21.01 1 +21.01 1 +21.02 1 +21.02 1 +21.11 1 +21.11 1 +21.14 1 +21.16 1 +21.26 1 +21.27 1 +21.3 1 +21.3 1 +21.33 1 +21.33 1 +21.38 1 +21.42 1 +21.52 1 +21.53 1 +21.57 1 +21.66 1 +21.67 1 +21.69 1 +21.77 1 +21.81 1 +21.86 1 +21.91 1 +21.95 1 +22.15 1 +22.19 1 +22.19 1 +22.22 1 +22.27 1 +22.27 1 +22.33 1 +22.48 1 +22.6 1 +22.61 1 +22.64 1 +22.68 1 +22.73 1 +22.75 1 +22.94 1 +22.95 1 +23.03 1 +23.07 1 +23.15 1 +23.15 1 +23.18 1 +23.18 1 +23.25 1 +23.27 1 +23.3 1 +23.31 1 +23.45 1 +23.48 1 +23.53 1 +23.55 1 +23.59 1 +23.61 1 +23.63 1 +23.73 1 +23.77 1 +23.78 2 +23.88 1 +23.91 1 +24.03 1 +24.03 1 +24.13 1 +24.17 1 +24.18 1 +24.25 1 +24.35 1 +24.35 2 +24.42 1 +24.46 1 +24.53 2 +24.59 1 +24.61 1 +24.61 1 +24.84 1 +24.85 1 +24.86 1 +24.95 1 +25.01 1 +25.02 1 +25.03 1 +25.08 1 +25.11 1 +25.17 1 +25.32 2 +25.36 1 +25.36 1 +25.43 1 +25.49 1 +25.51 1 +25.51 1 +25.58 1 +25.59 1 +25.63 1 +25.71 1 +25.75 1 +25.8 1 +25.92 1 +25.92 1 +25.95 1 +25.97 2 +26.0 1 +26.17 2 +26.21 1 +26.22 1 +26.24 1 +26.28 1 +26.44 1 +26.55 1 +26.55 1 +26.65 1 +26.67 1 +26.71 1 +26.71 1 +26.73 1 +26.74 1 +26.79 1 +26.84 1 +26.87 1 +27.02 1 +27.14 1 +27.2 1 +27.27 1 +27.29 1 +27.36 1 +27.39 1 +27.4 1 +27.42 1 +27.46 1 +27.54 1 +27.54 1 +27.61 1 +27.62 1 +27.89 1 +28.02 1 +28.1 1 +28.13 1 +28.14 1 +28.15 1 +28.17 1 +28.19 1 +28.29 1 +28.36 1 +28.4 1 +28.42 2 +28.44 1 +28.52 1 +28.52 1 +28.61 1 +28.64 1 +28.68 1 +28.69 1 +28.69 1 +28.71 1 +28.71 2 +28.77 1 +28.77 1 +28.85 1 +28.86 1 +28.91 1 +28.96 1 +28.96 1 +28.98 1 +29.0 1 +29.11 1 +29.19 1 +29.22 1 +29.24 1 +29.25 1 +29.36 1 +29.41 1 +29.46 1 +29.49 1 +29.52 2 +29.62 1 +29.63 1 +29.66 1 +29.73 1 +29.76 1 +29.78 1 +29.88 1 +29.96 1 +30.04 1 +30.04 1 +30.09 1 +30.12 1 +30.13 1 +30.16 1 +30.17 1 +30.21 1 +30.22 1 +30.28 1 +30.35 1 +30.37 2 +30.41 1 +30.41 1 +30.49 1 +30.55 1 +30.55 1 +30.58 1 +30.61 1 +30.62 1 +30.66 1 +30.67 1 +30.78 1 +30.78 1 +30.87 1 +30.89 1 +30.9 1 +30.92 1 +30.98 2 +30.99 1 +31.01 1 +31.16 1 +31.23 1 +31.3 1 +31.33 1 +31.36 1 +31.45 1 +31.46 1 +31.5 1 +31.61 1 +31.63 1 +31.64 1 +31.66 1 +31.68 1 +31.74 1 +31.75 1 +31.76 1 +31.84 1 +32.04 1 +32.06 1 +32.13 1 +32.17 1 +32.2 1 +32.25 1 +32.25 1 +32.31 1 +32.33 1 +32.39 1 +32.56 1 +32.56 1 +32.61 1 +32.74 1 +32.85 1 +32.89 1 +32.98 1 +33.0 1 +33.01 1 +33.02 1 +33.02 1 +33.02 1 +33.11 1 +33.12 1 +33.18 1 +33.18 1 +33.19 1 +33.24 1 +33.3 1 +33.36 1 +33.36 1 +33.38 1 +33.49 1 +33.52 2 +33.6 2 +33.64 1 +33.64 1 +33.66 1 +33.67 1 +33.72 1 +33.76 1 +33.9 1 +34.05 1 +34.06 1 +34.11 1 +34.14 1 +34.15 1 +34.17 1 +34.2 1 +34.41 2 +34.48 2 +34.49 1 +34.52 1 +34.53 1 +34.54 1 +34.62 1 +34.68 1 +34.68 1 +34.72 1 +34.81 1 +34.83 1 +34.84 1 +34.9 1 +34.95 1 +34.95 1 +34.97 1 +34.97 1 +34.98 1 +35.01 1 +35.02 1 +35.1 1 +35.15 1 +35.16 1 +35.23 1 +35.24 1 +35.36 1 +35.36 1 +35.49 1 +35.62 1 +35.68 1 +35.72 1 +35.84 1 +35.85 1 +35.9 1 +36.05 2 +36.09 1 +36.11 1 +36.12 1 +36.13 1 +36.22 1 +36.56 1 +36.57 1 +36.57 1 +36.62 1 +36.7 1 +36.72 1 +36.73 1 +36.84 1 +36.86 1 +36.93 1 +36.93 1 +37.02 1 +37.08 1 +37.12 1 +37.23 2 +37.32 1 +37.34 1 +37.37 1 +37.76 1 +37.77 1 +37.8 1 +37.93 1 +37.94 1 +37.96 1 +38.0 1 +38.04 1 +38.04 1 +38.04 1 +38.05 1 +38.07 1 +38.14 1 +38.22 1 +38.28 1 +38.37 2 +38.39 1 +38.43 1 +38.53 1 +38.6 1 +38.62 1 +38.66 1 +38.67 1 +38.73 1 +38.74 1 +38.92 1 +38.94 1 +39.01 1 +39.04 1 +39.05 1 +39.11 1 +39.27 1 +39.29 1 +39.41 1 +39.46 1 +39.49 1 +39.55 1 +39.57 1 +39.6 1 +39.63 1 +39.73 1 +39.74 1 +39.78 2 +39.8 1 +39.84 1 +39.84 1 +39.85 1 +39.92 1 +40.01 1 +40.04 1 +40.15 1 +40.21 2 +40.22 1 +40.24 1 +40.26 1 +40.27 1 +40.39 2 +40.43 1 +40.46 1 +40.5 1 +40.59 1 +40.6 1 +40.63 1 +40.76 1 +40.79 1 +40.84 1 +40.89 1 +40.91 1 +40.94 1 +40.96 1 +41.02 1 +41.08 1 +41.2 1 +41.24 1 +41.33 1 +41.34 1 +41.34 2 +41.36 1 +41.36 1 +41.37 1 +41.54 1 +41.56 1 +41.69 1 +41.73 1 +41.75 1 +41.75 1 +41.83 1 +41.86 1 +41.9 1 +42.0 2 +42.02 1 +42.03 1 +42.04 1 +42.22 1 +42.3 1 +42.37 1 +42.41 2 +42.47 1 +42.55 1 +42.56 1 +42.57 1 +42.76 1 +42.89 1 +42.96 1 +42.96 1 +43.0 1 +43.04 1 +43.04 3 +43.18 1 +43.23 1 +43.3 1 +43.34 1 +43.34 2 +43.4 1 +43.42 1 +43.58 1 +43.64 1 +43.67 1 +43.76 2 +43.84 1 +43.85 1 +43.92 1 +43.95 1 +43.95 1 +43.96 1 +43.96 1 +44.04 1 +44.11 1 +44.12 1 +44.12 1 +44.22 3 +44.24 1 +44.27 1 +44.32 1 +44.36 1 +44.4 1 +44.57 1 +44.6 1 +44.63 1 +44.66 1 +44.73 1 +44.75 1 +44.8 1 +44.83 1 +44.9 1 +44.92 1 +44.93 1 +45.0 1 +45.02 1 +45.06 1 +45.06 1 +45.09 1 +45.1 1 +45.1 1 +45.11 1 +45.12 2 +45.14 1 +45.14 2 +45.28 2 +45.29 1 +45.46 1 +45.49 1 +45.53 1 +45.53 2 +45.54 1 +45.69 1 +45.71 1 +45.78 1 +45.81 2 +45.86 1 +45.9 1 +45.94 1 +46.03 1 +46.03 1 +46.09 1 +46.18 1 +46.19 1 +46.28 1 +46.3 1 +46.31 1 +46.33 1 +46.36 1 +46.39 1 +46.52 1 +46.53 1 +46.54 1 +46.57 1 +46.59 1 +46.67 1 +46.69 1 +46.73 1 +46.73 1 +46.73 2 +46.74 1 +46.81 1 +46.87 1 +46.88 1 +46.9 1 +46.93 1 +46.98 1 +47.0 1 +47.03 1 +47.03 1 +47.06 1 +47.15 1 +47.22 1 +47.3 1 +47.31 2 +47.37 2 +47.4 1 +47.46 1 +47.49 1 +47.55 1 +47.6 1 +47.66 1 +47.68 1 +47.71 1 +47.72 1 +47.82 1 +47.86 2 +47.91 1 +47.91 2 +47.95 1 +47.98 1 +48.0 1 +48.08 1 +48.5 1 +48.52 1 +48.71 1 +48.78 1 +48.8 1 +48.85 1 +48.89 1 +48.96 1 +48.98 1 +49.04 1 +49.05 1 +49.16 1 +49.21 1 +49.32 1 +49.34 1 +49.34 1 +49.38 1 +49.44 2 +49.45 1 +49.45 1 +49.46 1 +49.46 1 +49.52 1 +49.56 1 +49.59 1 +49.63 1 +49.67 1 +49.69 1 +49.71 2 +49.72 1 +49.73 1 +49.79 1 +49.84 1 +49.85 2 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-0-2e0cbc2d7c5f16657edacd9e7209e6e7 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-0-2e0cbc2d7c5f16657edacd9e7209e6e7 new file mode 100644 index 0000000000000..612bdf44c0cdf --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-0-2e0cbc2d7c5f16657edacd9e7209e6e7 @@ -0,0 +1,1049 @@ + 1 + 1 + 1 +alice allen 1 +alice allen 1 +alice allen 1 +alice brown 1 +alice carson 1 +alice davidson 1 +alice falkner 1 +alice garcia 2 +alice hernandez 1 +alice hernandez 1 +alice johnson 1 +alice king 1 +alice king 1 +alice king 1 +alice laertes 1 +alice laertes 1 +alice miller 1 +alice nixon 1 +alice nixon 1 +alice nixon 1 +alice ovid 1 +alice polk 1 +alice quirinius 1 +alice quirinius 1 +alice robinson 1 +alice robinson 1 +alice steinbeck 1 +alice steinbeck 1 +alice steinbeck 1 +alice underhill 1 +alice van buren 1 +alice xylophone 1 +alice xylophone 1 +alice xylophone 1 +alice zipper 1 +alice zipper 1 +alice zipper 1 +bob brown 1 +bob brown 1 +bob brown 2 +bob carson 1 +bob davidson 1 +bob davidson 1 +bob davidson 1 +bob ellison 1 +bob ellison 1 +bob ellison 1 +bob ellison 2 +bob falkner 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 2 +bob hernandez 1 +bob ichabod 1 +bob king 1 +bob king 1 +bob king 1 +bob laertes 1 +bob laertes 1 +bob miller 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob polk 1 +bob quirinius 1 +bob steinbeck 1 +bob van buren 1 +bob white 1 +bob white 1 +bob xylophone 1 +bob xylophone 1 +bob young 1 +bob zipper 1 +bob zipper 1 +bob zipper 1 +calvin allen 1 +calvin brown 1 +calvin brown 1 +calvin brown 1 +calvin carson 1 +calvin davidson 1 +calvin davidson 1 +calvin ellison 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin garcia 1 +calvin hernandez 1 +calvin johnson 2 +calvin laertes 1 +calvin laertes 1 +calvin nixon 1 +calvin nixon 1 +calvin nixon 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 1 +calvin polk 1 +calvin quirinius 1 +calvin quirinius 1 +calvin robinson 1 +calvin steinbeck 1 +calvin steinbeck 1 +calvin steinbeck 1 +calvin thompson 1 +calvin thompson 1 +calvin underhill 1 +calvin van buren 1 +calvin van buren 1 +calvin white 1 +calvin white 1 +calvin xylophone 1 +calvin xylophone 1 +calvin xylophone 1 +calvin young 1 +calvin young 1 +calvin zipper 1 +calvin zipper 1 +david allen 1 +david allen 1 +david brown 1 +david brown 1 +david davidson 1 +david davidson 1 +david davidson 1 +david davidson 2 +david ellison 1 +david ellison 1 +david ellison 1 +david hernandez 1 +david ichabod 1 +david ichabod 1 +david laertes 1 +david nixon 1 +david ovid 1 +david ovid 1 +david quirinius 1 +david quirinius 1 +david quirinius 1 +david robinson 1 +david robinson 1 +david thompson 1 +david underhill 1 +david underhill 1 +david underhill 1 +david van buren 1 +david van buren 1 +david white 2 +david xylophone 1 +david xylophone 1 +david xylophone 1 +david young 1 +david young 1 +ethan allen 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan carson 1 +ethan ellison 1 +ethan ellison 1 +ethan falkner 1 +ethan falkner 1 +ethan garcia 1 +ethan hernandez 1 +ethan johnson 1 +ethan king 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan miller 1 +ethan nixon 1 +ethan ovid 1 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan quirinius 1 +ethan quirinius 1 +ethan quirinius 1 +ethan robinson 1 +ethan robinson 1 +ethan underhill 1 +ethan van buren 1 +ethan white 1 +ethan white 1 +ethan xylophone 1 +ethan zipper 1 +ethan zipper 1 +fred davidson 1 +fred davidson 1 +fred davidson 1 +fred ellison 1 +fred ellison 1 +fred ellison 1 +fred falkner 1 +fred falkner 1 +fred falkner 1 +fred hernandez 1 +fred ichabod 1 +fred ichabod 2 +fred johnson 1 +fred king 1 +fred king 1 +fred laertes 1 +fred miller 1 +fred nixon 1 +fred nixon 1 +fred nixon 1 +fred nixon 2 +fred polk 1 +fred polk 1 +fred polk 1 +fred polk 1 +fred quirinius 1 +fred quirinius 1 +fred robinson 1 +fred steinbeck 1 +fred steinbeck 1 +fred steinbeck 1 +fred underhill 1 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred white 1 +fred young 1 +fred young 1 +fred zipper 1 +gabriella allen 1 +gabriella allen 1 +gabriella brown 1 +gabriella brown 1 +gabriella carson 1 +gabriella davidson 1 +gabriella ellison 1 +gabriella ellison 1 +gabriella falkner 1 +gabriella falkner 1 +gabriella falkner 1 +gabriella garcia 1 +gabriella hernandez 1 +gabriella hernandez 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella king 1 +gabriella king 1 +gabriella laertes 1 +gabriella miller 1 +gabriella ovid 1 +gabriella ovid 1 +gabriella polk 1 +gabriella polk 1 +gabriella steinbeck 1 +gabriella steinbeck 1 +gabriella thompson 1 +gabriella thompson 1 +gabriella thompson 1 +gabriella van buren 1 +gabriella van buren 1 +gabriella white 1 +gabriella young 1 +gabriella young 1 +gabriella zipper 1 +gabriella zipper 1 +holly allen 1 +holly brown 1 +holly brown 1 +holly falkner 1 +holly hernandez 1 +holly hernandez 1 +holly hernandez 1 +holly hernandez 2 +holly ichabod 1 +holly ichabod 1 +holly ichabod 1 +holly johnson 1 +holly johnson 1 +holly johnson 1 +holly king 1 +holly king 1 +holly laertes 1 +holly miller 1 +holly nixon 1 +holly nixon 1 +holly polk 1 +holly polk 1 +holly robinson 1 +holly thompson 1 +holly thompson 1 +holly thompson 1 +holly underhill 1 +holly underhill 1 +holly underhill 1 +holly underhill 1 +holly van buren 1 +holly white 1 +holly white 2 +holly xylophone 1 +holly young 1 +holly young 1 +holly zipper 1 +holly zipper 1 +irene allen 1 +irene brown 1 +irene brown 1 +irene brown 1 +irene carson 1 +irene ellison 1 +irene ellison 1 +irene falkner 1 +irene falkner 1 +irene garcia 1 +irene garcia 1 +irene garcia 1 +irene ichabod 1 +irene ichabod 1 +irene johnson 1 +irene laertes 1 +irene laertes 1 +irene laertes 1 +irene miller 1 +irene nixon 1 +irene nixon 1 +irene nixon 1 +irene ovid 1 +irene ovid 1 +irene ovid 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene quirinius 1 +irene quirinius 1 +irene quirinius 1 +irene robinson 1 +irene steinbeck 1 +irene thompson 1 +irene underhill 1 +irene underhill 1 +irene van buren 1 +irene van buren 1 +irene xylophone 2 +jessica brown 2 +jessica carson 1 +jessica carson 1 +jessica carson 1 +jessica davidson 1 +jessica davidson 1 +jessica davidson 1 +jessica davidson 1 +jessica ellison 1 +jessica ellison 1 +jessica falkner 1 +jessica garcia 1 +jessica garcia 1 +jessica ichabod 1 +jessica johnson 1 +jessica johnson 1 +jessica miller 1 +jessica nixon 1 +jessica nixon 1 +jessica ovid 1 +jessica ovid 2 +jessica polk 1 +jessica quirinius 1 +jessica quirinius 1 +jessica quirinius 1 +jessica quirinius 1 +jessica robinson 1 +jessica thompson 1 +jessica thompson 3 +jessica underhill 1 +jessica underhill 1 +jessica underhill 1 +jessica van buren 1 +jessica white 1 +jessica white 1 +jessica white 1 +jessica white 1 +jessica white 1 +jessica xylophone 1 +jessica young 1 +jessica young 1 +jessica zipper 1 +jessica zipper 1 +jessica zipper 1 +katie allen 1 +katie brown 1 +katie davidson 1 +katie ellison 1 +katie ellison 1 +katie falkner 1 +katie garcia 1 +katie garcia 1 +katie hernandez 1 +katie ichabod 1 +katie ichabod 1 +katie ichabod 1 +katie king 1 +katie king 1 +katie king 1 +katie miller 1 +katie miller 1 +katie nixon 1 +katie ovid 1 +katie polk 1 +katie polk 1 +katie robinson 1 +katie van buren 1 +katie van buren 1 +katie white 1 +katie white 1 +katie xylophone 1 +katie young 1 +katie young 1 +katie young 1 +katie zipper 1 +katie zipper 1 +luke allen 1 +luke allen 1 +luke allen 1 +luke allen 1 +luke allen 2 +luke brown 1 +luke davidson 1 +luke davidson 1 +luke ellison 1 +luke ellison 1 +luke ellison 1 +luke falkner 1 +luke falkner 1 +luke garcia 1 +luke garcia 1 +luke ichabod 1 +luke ichabod 1 +luke johnson 1 +luke johnson 1 +luke johnson 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke miller 1 +luke ovid 1 +luke ovid 1 +luke polk 1 +luke polk 1 +luke quirinius 1 +luke robinson 1 +luke robinson 1 +luke thompson 1 +luke underhill 1 +luke underhill 1 +luke underhill 2 +luke van buren 1 +luke white 1 +luke xylophone 1 +luke zipper 1 +mike allen 1 +mike brown 1 +mike carson 1 +mike carson 1 +mike carson 1 +mike davidson 1 +mike davidson 1 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike falkner 1 +mike garcia 1 +mike garcia 1 +mike garcia 1 +mike hernandez 1 +mike hernandez 2 +mike ichabod 1 +mike king 1 +mike king 1 +mike king 1 +mike king 1 +mike king 1 +mike king 2 +mike miller 1 +mike nixon 1 +mike nixon 1 +mike polk 1 +mike polk 1 +mike polk 1 +mike quirinius 1 +mike steinbeck 1 +mike steinbeck 1 +mike steinbeck 1 +mike steinbeck 1 +mike van buren 1 +mike van buren 1 +mike white 1 +mike white 1 +mike white 1 +mike white 2 +mike young 1 +mike young 1 +mike young 1 +mike zipper 1 +mike zipper 1 +mike zipper 1 +nick allen 1 +nick allen 1 +nick brown 1 +nick davidson 1 +nick ellison 1 +nick ellison 1 +nick falkner 1 +nick falkner 1 +nick garcia 1 +nick garcia 1 +nick garcia 1 +nick ichabod 1 +nick ichabod 1 +nick ichabod 1 +nick johnson 1 +nick johnson 1 +nick laertes 1 +nick miller 1 +nick nixon 1 +nick ovid 1 +nick polk 1 +nick quirinius 1 +nick quirinius 1 +nick robinson 1 +nick robinson 1 +nick steinbeck 1 +nick thompson 1 +nick underhill 1 +nick van buren 1 +nick xylophone 1 +nick young 1 +nick young 1 +nick zipper 1 +nick zipper 1 +oscar allen 2 +oscar brown 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar davidson 1 +oscar ellison 1 +oscar ellison 1 +oscar falkner 1 +oscar garcia 1 +oscar hernandez 1 +oscar hernandez 1 +oscar ichabod 1 +oscar ichabod 1 +oscar ichabod 1 +oscar ichabod 1 +oscar johnson 1 +oscar johnson 1 +oscar king 1 +oscar king 1 +oscar king 2 +oscar laertes 1 +oscar laertes 1 +oscar laertes 1 +oscar laertes 1 +oscar nixon 1 +oscar ovid 1 +oscar ovid 1 +oscar ovid 2 +oscar polk 1 +oscar polk 1 +oscar quirinius 1 +oscar quirinius 1 +oscar quirinius 1 +oscar quirinius 1 +oscar robinson 1 +oscar robinson 1 +oscar robinson 1 +oscar robinson 1 +oscar steinbeck 1 +oscar thompson 1 +oscar thompson 1 +oscar thompson 1 +oscar thompson 2 +oscar underhill 1 +oscar van buren 1 +oscar van buren 1 +oscar van buren 1 +oscar white 1 +oscar white 1 +oscar white 1 +oscar white 1 +oscar xylophone 1 +oscar xylophone 1 +oscar xylophone 1 +oscar zipper 1 +oscar zipper 1 +oscar zipper 1 +priscilla brown 1 +priscilla brown 1 +priscilla brown 1 +priscilla carson 1 +priscilla carson 1 +priscilla carson 1 +priscilla ichabod 1 +priscilla ichabod 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla king 1 +priscilla nixon 1 +priscilla nixon 2 +priscilla ovid 1 +priscilla ovid 1 +priscilla polk 1 +priscilla quirinius 1 +priscilla thompson 1 +priscilla underhill 1 +priscilla underhill 1 +priscilla van buren 1 +priscilla van buren 1 +priscilla van buren 1 +priscilla white 1 +priscilla xylophone 1 +priscilla xylophone 1 +priscilla xylophone 1 +priscilla young 1 +priscilla young 1 +priscilla zipper 1 +priscilla zipper 1 +quinn allen 1 +quinn allen 1 +quinn brown 1 +quinn brown 1 +quinn brown 1 +quinn davidson 1 +quinn davidson 1 +quinn davidson 1 +quinn davidson 1 +quinn ellison 1 +quinn ellison 1 +quinn garcia 1 +quinn garcia 1 +quinn garcia 1 +quinn garcia 1 +quinn ichabod 1 +quinn king 1 +quinn king 1 +quinn laertes 1 +quinn laertes 1 +quinn laertes 1 +quinn nixon 1 +quinn ovid 1 +quinn quirinius 1 +quinn robinson 1 +quinn steinbeck 1 +quinn steinbeck 4 +quinn thompson 1 +quinn thompson 2 +quinn underhill 1 +quinn underhill 1 +quinn underhill 2 +quinn van buren 1 +quinn young 1 +quinn zipper 1 +quinn zipper 1 +rachel allen 1 +rachel allen 1 +rachel brown 1 +rachel brown 1 +rachel brown 1 +rachel brown 1 +rachel brown 2 +rachel carson 1 +rachel carson 1 +rachel davidson 1 +rachel ellison 1 +rachel falkner 1 +rachel falkner 1 +rachel falkner 1 +rachel falkner 1 +rachel johnson 1 +rachel king 1 +rachel king 1 +rachel laertes 1 +rachel laertes 1 +rachel ovid 1 +rachel ovid 1 +rachel polk 1 +rachel quirinius 1 +rachel robinson 1 +rachel robinson 1 +rachel robinson 1 +rachel thompson 1 +rachel thompson 1 +rachel thompson 1 +rachel underhill 1 +rachel white 1 +rachel white 1 +rachel young 1 +rachel zipper 1 +rachel zipper 1 +sarah carson 1 +sarah carson 1 +sarah carson 1 +sarah ellison 2 +sarah falkner 1 +sarah falkner 1 +sarah garcia 1 +sarah garcia 1 +sarah garcia 2 +sarah ichabod 1 +sarah ichabod 1 +sarah johnson 1 +sarah johnson 1 +sarah johnson 1 +sarah johnson 1 +sarah king 1 +sarah king 1 +sarah miller 1 +sarah ovid 1 +sarah robinson 1 +sarah robinson 1 +sarah steinbeck 1 +sarah white 1 +sarah white 1 +sarah xylophone 1 +sarah young 1 +sarah zipper 1 +tom brown 1 +tom brown 1 +tom carson 1 +tom carson 1 +tom carson 1 +tom davidson 1 +tom ellison 1 +tom ellison 1 +tom ellison 1 +tom falkner 1 +tom falkner 1 +tom hernandez 1 +tom hernandez 1 +tom ichabod 1 +tom johnson 1 +tom johnson 2 +tom king 1 +tom laertes 1 +tom laertes 1 +tom miller 1 +tom miller 1 +tom miller 2 +tom nixon 1 +tom ovid 1 +tom polk 1 +tom polk 1 +tom quirinius 1 +tom quirinius 1 +tom robinson 1 +tom robinson 1 +tom robinson 2 +tom robinson 2 +tom steinbeck 2 +tom van buren 1 +tom van buren 1 +tom van buren 1 +tom white 1 +tom young 1 +tom young 2 +tom zipper 1 +ulysses brown 1 +ulysses carson 1 +ulysses carson 1 +ulysses carson 1 +ulysses carson 1 +ulysses davidson 1 +ulysses ellison 1 +ulysses garcia 1 +ulysses hernandez 1 +ulysses hernandez 1 +ulysses hernandez 2 +ulysses ichabod 1 +ulysses ichabod 1 +ulysses johnson 1 +ulysses king 1 +ulysses laertes 1 +ulysses laertes 1 +ulysses laertes 1 +ulysses miller 1 +ulysses miller 1 +ulysses nixon 1 +ulysses ovid 1 +ulysses polk 1 +ulysses polk 1 +ulysses polk 1 +ulysses polk 2 +ulysses quirinius 1 +ulysses robinson 1 +ulysses steinbeck 1 +ulysses steinbeck 1 +ulysses thompson 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses van buren 1 +ulysses white 1 +ulysses white 1 +ulysses xylophone 1 +ulysses xylophone 1 +ulysses xylophone 1 +ulysses young 1 +ulysses young 1 +ulysses young 1 +victor allen 1 +victor allen 1 +victor brown 1 +victor brown 1 +victor brown 1 +victor brown 1 +victor davidson 1 +victor davidson 1 +victor davidson 2 +victor ellison 1 +victor ellison 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor johnson 1 +victor johnson 1 +victor johnson 1 +victor king 1 +victor king 1 +victor laertes 1 +victor laertes 1 +victor miller 2 +victor nixon 1 +victor nixon 1 +victor ovid 1 +victor polk 1 +victor quirinius 1 +victor quirinius 1 +victor robinson 1 +victor robinson 1 +victor steinbeck 1 +victor steinbeck 1 +victor steinbeck 1 +victor thompson 1 +victor van buren 1 +victor van buren 1 +victor white 1 +victor white 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 2 +victor young 1 +victor zipper 1 +wendy allen 1 +wendy allen 1 +wendy allen 1 +wendy brown 1 +wendy brown 1 +wendy ellison 1 +wendy ellison 1 +wendy falkner 1 +wendy falkner 1 +wendy falkner 1 +wendy garcia 1 +wendy garcia 1 +wendy garcia 1 +wendy garcia 1 +wendy hernandez 1 +wendy ichabod 1 +wendy king 1 +wendy king 1 +wendy king 1 +wendy laertes 1 +wendy laertes 1 +wendy laertes 1 +wendy miller 1 +wendy miller 1 +wendy nixon 1 +wendy nixon 1 +wendy ovid 1 +wendy ovid 1 +wendy polk 1 +wendy polk 1 +wendy quirinius 1 +wendy quirinius 1 +wendy robinson 1 +wendy robinson 1 +wendy robinson 1 +wendy steinbeck 1 +wendy thompson 1 +wendy thompson 1 +wendy underhill 1 +wendy underhill 1 +wendy underhill 1 +wendy van buren 1 +wendy van buren 1 +wendy white 1 +wendy xylophone 1 +wendy xylophone 1 +wendy young 1 +wendy young 3 +xavier allen 1 +xavier allen 1 +xavier allen 1 +xavier brown 1 +xavier brown 1 +xavier brown 1 +xavier carson 1 +xavier carson 1 +xavier davidson 1 +xavier davidson 1 +xavier davidson 1 +xavier ellison 1 +xavier ellison 1 +xavier garcia 1 +xavier hernandez 1 +xavier hernandez 1 +xavier hernandez 1 +xavier ichabod 1 +xavier ichabod 1 +xavier johnson 1 +xavier johnson 1 +xavier king 1 +xavier king 1 +xavier laertes 1 +xavier ovid 1 +xavier polk 1 +xavier polk 1 +xavier polk 1 +xavier polk 1 +xavier quirinius 1 +xavier quirinius 1 +xavier quirinius 1 +xavier quirinius 1 +xavier thompson 1 +xavier underhill 1 +xavier white 1 +xavier white 1 +xavier xylophone 1 +xavier zipper 2 +yuri allen 1 +yuri allen 1 +yuri brown 1 +yuri brown 1 +yuri carson 1 +yuri carson 1 +yuri ellison 1 +yuri ellison 1 +yuri falkner 1 +yuri falkner 1 +yuri garcia 1 +yuri hernandez 1 +yuri johnson 1 +yuri johnson 1 +yuri johnson 1 +yuri king 1 +yuri laertes 1 +yuri laertes 1 +yuri nixon 1 +yuri nixon 1 +yuri polk 1 +yuri polk 1 +yuri polk 1 +yuri quirinius 1 +yuri quirinius 1 +yuri quirinius 1 +yuri steinbeck 1 +yuri steinbeck 1 +yuri thompson 1 +yuri underhill 1 +yuri underhill 1 +yuri white 1 +yuri xylophone 1 +zach allen 2 +zach brown 1 +zach brown 1 +zach brown 1 +zach brown 1 +zach brown 1 +zach carson 1 +zach ellison 1 +zach falkner 1 +zach falkner 1 +zach garcia 1 +zach garcia 1 +zach garcia 1 +zach garcia 1 +zach ichabod 1 +zach ichabod 1 +zach king 1 +zach king 1 +zach king 2 +zach miller 1 +zach miller 1 +zach miller 1 +zach ovid 1 +zach ovid 1 +zach ovid 1 +zach ovid 1 +zach quirinius 1 +zach robinson 2 +zach steinbeck 2 +zach steinbeck 2 +zach thompson 1 +zach thompson 1 +zach underhill 1 +zach white 1 +zach xylophone 1 +zach xylophone 1 +zach young 1 +zach zipper 1 +zach zipper 1 +zach zipper 1 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-1-5c5f373e325115d710a7a23fcb1626f1 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-1-5c5f373e325115d710a7a23fcb1626f1 new file mode 100644 index 0000000000000..22a6f27253dcf --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-1-5c5f373e325115d710a7a23fcb1626f1 @@ -0,0 +1,1049 @@ +zach zipper 4 +zach zipper 3 +zach zipper 1 +zach young 4 +zach xylophone 4 +zach xylophone 1 +zach white 1 +zach underhill 1 +zach thompson 2 +zach thompson 2 +zach steinbeck 5 +zach steinbeck 1 +zach robinson 1 +zach quirinius 3 +zach ovid 5 +zach ovid 4 +zach ovid 3 +zach ovid 1 +zach miller 5 +zach miller 3 +zach miller 1 +zach king 6 +zach king 4 +zach king 1 +zach ichabod 3 +zach ichabod 2 +zach garcia 6 +zach garcia 3 +zach garcia 1 +zach garcia 1 +zach falkner 2 +zach falkner 1 +zach ellison 4 +zach carson 3 +zach brown 5 +zach brown 4 +zach brown 3 +zach brown 2 +zach brown 1 +zach allen 4 +yuri xylophone 3 +yuri white 2 +yuri underhill 6 +yuri underhill 4 +yuri thompson 4 +yuri steinbeck 6 +yuri steinbeck 2 +yuri quirinius 4 +yuri quirinius 3 +yuri quirinius 1 +yuri polk 4 +yuri polk 3 +yuri polk 2 +yuri nixon 3 +yuri nixon 2 +yuri laertes 3 +yuri laertes 1 +yuri king 5 +yuri johnson 4 +yuri johnson 3 +yuri johnson 1 +yuri hernandez 4 +yuri garcia 3 +yuri falkner 7 +yuri falkner 3 +yuri ellison 1 +yuri ellison 1 +yuri carson 7 +yuri carson 4 +yuri brown 3 +yuri brown 1 +yuri allen 3 +yuri allen 2 +xavier zipper 1 +xavier xylophone 1 +xavier white 3 +xavier white 3 +xavier underhill 2 +xavier thompson 3 +xavier quirinius 6 +xavier quirinius 5 +xavier quirinius 2 +xavier quirinius 1 +xavier polk 5 +xavier polk 3 +xavier polk 3 +xavier polk 3 +xavier ovid 5 +xavier laertes 4 +xavier king 3 +xavier king 1 +xavier johnson 3 +xavier johnson 1 +xavier ichabod 2 +xavier ichabod 2 +xavier hernandez 3 +xavier hernandez 1 +xavier hernandez 1 +xavier garcia 4 +xavier ellison 1 +xavier ellison 1 +xavier davidson 5 +xavier davidson 4 +xavier davidson 1 +xavier carson 5 +xavier carson 3 +xavier brown 4 +xavier brown 2 +xavier brown 2 +xavier allen 6 +xavier allen 3 +xavier allen 1 +wendy young 8 +wendy young 2 +wendy xylophone 6 +wendy xylophone 4 +wendy white 5 +wendy van buren 2 +wendy van buren 2 +wendy underhill 6 +wendy underhill 5 +wendy underhill 4 +wendy thompson 5 +wendy thompson 2 +wendy steinbeck 1 +wendy robinson 5 +wendy robinson 3 +wendy robinson 2 +wendy quirinius 6 +wendy quirinius 4 +wendy polk 2 +wendy polk 2 +wendy ovid 4 +wendy ovid 1 +wendy nixon 3 +wendy nixon 1 +wendy miller 2 +wendy miller 1 +wendy laertes 3 +wendy laertes 3 +wendy laertes 1 +wendy king 5 +wendy king 4 +wendy king 1 +wendy ichabod 3 +wendy hernandez 1 +wendy garcia 7 +wendy garcia 5 +wendy garcia 4 +wendy garcia 1 +wendy falkner 3 +wendy falkner 1 +wendy falkner 1 +wendy ellison 2 +wendy ellison 1 +wendy brown 5 +wendy brown 2 +wendy allen 6 +wendy allen 2 +wendy allen 2 +victor zipper 3 +victor young 1 +victor xylophone 6 +victor xylophone 6 +victor xylophone 2 +victor xylophone 1 +victor xylophone 1 +victor white 2 +victor white 1 +victor van buren 4 +victor van buren 4 +victor thompson 2 +victor steinbeck 5 +victor steinbeck 2 +victor steinbeck 1 +victor robinson 4 +victor robinson 2 +victor quirinius 3 +victor quirinius 1 +victor polk 3 +victor ovid 2 +victor nixon 6 +victor nixon 4 +victor miller 1 +victor laertes 4 +victor laertes 3 +victor king 6 +victor king 1 +victor johnson 2 +victor johnson 2 +victor johnson 1 +victor hernandez 6 +victor hernandez 4 +victor hernandez 3 +victor hernandez 1 +victor hernandez 1 +victor ellison 7 +victor ellison 4 +victor davidson 6 +victor davidson 2 +victor davidson 2 +victor brown 4 +victor brown 3 +victor brown 2 +victor brown 1 +victor allen 4 +victor allen 2 +ulysses young 7 +ulysses young 6 +ulysses young 3 +ulysses xylophone 6 +ulysses xylophone 3 +ulysses xylophone 2 +ulysses white 6 +ulysses white 2 +ulysses van buren 3 +ulysses underhill 8 +ulysses underhill 4 +ulysses underhill 3 +ulysses underhill 2 +ulysses underhill 2 +ulysses underhill 1 +ulysses underhill 1 +ulysses thompson 5 +ulysses steinbeck 3 +ulysses steinbeck 1 +ulysses robinson 5 +ulysses quirinius 8 +ulysses polk 6 +ulysses polk 4 +ulysses polk 1 +ulysses polk 1 +ulysses ovid 3 +ulysses nixon 1 +ulysses miller 3 +ulysses miller 2 +ulysses laertes 5 +ulysses laertes 4 +ulysses laertes 2 +ulysses king 2 +ulysses johnson 5 +ulysses ichabod 1 +ulysses ichabod 1 +ulysses hernandez 6 +ulysses hernandez 3 +ulysses hernandez 2 +ulysses garcia 2 +ulysses ellison 2 +ulysses davidson 8 +ulysses carson 4 +ulysses carson 3 +ulysses carson 2 +ulysses carson 1 +ulysses brown 3 +tom zipper 5 +tom young 2 +tom young 1 +tom white 1 +tom van buren 5 +tom van buren 2 +tom van buren 1 +tom steinbeck 4 +tom robinson 8 +tom robinson 4 +tom robinson 3 +tom robinson 2 +tom quirinius 5 +tom quirinius 1 +tom polk 3 +tom polk 2 +tom ovid 2 +tom nixon 5 +tom miller 1 +tom miller 1 +tom miller 1 +tom laertes 4 +tom laertes 2 +tom king 1 +tom johnson 8 +tom johnson 1 +tom ichabod 1 +tom hernandez 3 +tom hernandez 2 +tom falkner 3 +tom falkner 2 +tom ellison 5 +tom ellison 3 +tom ellison 1 +tom davidson 7 +tom carson 3 +tom carson 3 +tom carson 1 +tom brown 4 +tom brown 2 +sarah zipper 1 +sarah young 1 +sarah xylophone 2 +sarah white 4 +sarah white 3 +sarah steinbeck 6 +sarah robinson 3 +sarah robinson 2 +sarah ovid 1 +sarah miller 1 +sarah king 3 +sarah king 2 +sarah johnson 7 +sarah johnson 6 +sarah johnson 4 +sarah johnson 2 +sarah ichabod 4 +sarah ichabod 3 +sarah garcia 2 +sarah garcia 2 +sarah garcia 2 +sarah falkner 7 +sarah falkner 1 +sarah ellison 1 +sarah carson 6 +sarah carson 4 +sarah carson 4 +rachel zipper 8 +rachel zipper 5 +rachel young 3 +rachel white 2 +rachel white 2 +rachel underhill 2 +rachel thompson 5 +rachel thompson 4 +rachel thompson 3 +rachel robinson 10 +rachel robinson 3 +rachel robinson 1 +rachel quirinius 5 +rachel polk 4 +rachel ovid 5 +rachel ovid 4 +rachel laertes 1 +rachel laertes 1 +rachel king 3 +rachel king 1 +rachel johnson 1 +rachel falkner 8 +rachel falkner 5 +rachel falkner 5 +rachel falkner 2 +rachel ellison 6 +rachel davidson 6 +rachel carson 7 +rachel carson 2 +rachel brown 5 +rachel brown 4 +rachel brown 3 +rachel brown 3 +rachel brown 1 +rachel allen 5 +rachel allen 1 +quinn zipper 2 +quinn zipper 2 +quinn young 2 +quinn van buren 2 +quinn underhill 7 +quinn underhill 6 +quinn underhill 2 +quinn thompson 5 +quinn thompson 2 +quinn steinbeck 3 +quinn steinbeck 2 +quinn robinson 2 +quinn quirinius 5 +quinn ovid 6 +quinn nixon 3 +quinn laertes 2 +quinn laertes 2 +quinn laertes 1 +quinn king 2 +quinn king 1 +quinn ichabod 1 +quinn garcia 6 +quinn garcia 3 +quinn garcia 2 +quinn garcia 1 +quinn ellison 7 +quinn ellison 5 +quinn davidson 7 +quinn davidson 4 +quinn davidson 3 +quinn davidson 2 +quinn brown 5 +quinn brown 3 +quinn brown 2 +quinn allen 5 +quinn allen 2 +priscilla zipper 5 +priscilla zipper 2 +priscilla young 4 +priscilla young 1 +priscilla xylophone 7 +priscilla xylophone 2 +priscilla xylophone 1 +priscilla white 4 +priscilla van buren 3 +priscilla van buren 3 +priscilla van buren 2 +priscilla underhill 5 +priscilla underhill 4 +priscilla thompson 2 +priscilla quirinius 4 +priscilla polk 5 +priscilla ovid 4 +priscilla ovid 1 +priscilla nixon 2 +priscilla nixon 1 +priscilla king 4 +priscilla johnson 4 +priscilla johnson 2 +priscilla johnson 2 +priscilla johnson 2 +priscilla johnson 1 +priscilla ichabod 3 +priscilla ichabod 2 +priscilla carson 6 +priscilla carson 5 +priscilla carson 4 +priscilla brown 5 +priscilla brown 5 +priscilla brown 3 +oscar zipper 4 +oscar zipper 4 +oscar zipper 2 +oscar xylophone 7 +oscar xylophone 5 +oscar xylophone 3 +oscar white 5 +oscar white 5 +oscar white 3 +oscar white 2 +oscar van buren 5 +oscar van buren 3 +oscar van buren 2 +oscar underhill 1 +oscar thompson 6 +oscar thompson 3 +oscar thompson 3 +oscar thompson 2 +oscar steinbeck 7 +oscar robinson 7 +oscar robinson 3 +oscar robinson 3 +oscar robinson 1 +oscar quirinius 3 +oscar quirinius 3 +oscar quirinius 2 +oscar quirinius 1 +oscar polk 2 +oscar polk 2 +oscar ovid 4 +oscar ovid 2 +oscar ovid 1 +oscar nixon 1 +oscar laertes 6 +oscar laertes 4 +oscar laertes 3 +oscar laertes 2 +oscar king 4 +oscar king 2 +oscar king 1 +oscar johnson 6 +oscar johnson 3 +oscar ichabod 3 +oscar ichabod 3 +oscar ichabod 1 +oscar ichabod 1 +oscar hernandez 6 +oscar hernandez 6 +oscar garcia 4 +oscar falkner 2 +oscar ellison 2 +oscar ellison 1 +oscar davidson 1 +oscar carson 4 +oscar carson 2 +oscar carson 2 +oscar carson 1 +oscar carson 1 +oscar brown 4 +oscar allen 2 +nick zipper 7 +nick zipper 5 +nick young 4 +nick young 2 +nick xylophone 2 +nick van buren 2 +nick underhill 2 +nick thompson 2 +nick steinbeck 4 +nick robinson 3 +nick robinson 1 +nick quirinius 5 +nick quirinius 1 +nick polk 5 +nick ovid 6 +nick nixon 4 +nick miller 2 +nick laertes 3 +nick johnson 4 +nick johnson 4 +nick ichabod 3 +nick ichabod 3 +nick ichabod 1 +nick garcia 5 +nick garcia 4 +nick garcia 4 +nick falkner 3 +nick falkner 1 +nick ellison 3 +nick ellison 2 +nick davidson 4 +nick brown 3 +nick allen 5 +nick allen 4 +mike zipper 4 +mike zipper 4 +mike zipper 1 +mike young 3 +mike young 1 +mike young 1 +mike white 9 +mike white 7 +mike white 5 +mike white 2 +mike van buren 2 +mike van buren 1 +mike steinbeck 4 +mike steinbeck 2 +mike steinbeck 2 +mike steinbeck 1 +mike quirinius 7 +mike polk 4 +mike polk 2 +mike polk 2 +mike nixon 3 +mike nixon 2 +mike miller 1 +mike king 6 +mike king 5 +mike king 4 +mike king 3 +mike king 1 +mike king 1 +mike ichabod 3 +mike hernandez 2 +mike hernandez 1 +mike garcia 3 +mike garcia 2 +mike garcia 1 +mike falkner 2 +mike ellison 6 +mike ellison 5 +mike ellison 3 +mike ellison 1 +mike ellison 1 +mike davidson 5 +mike davidson 5 +mike carson 9 +mike carson 4 +mike carson 3 +mike brown 2 +mike allen 3 +luke zipper 2 +luke xylophone 1 +luke white 1 +luke van buren 2 +luke underhill 2 +luke underhill 2 +luke underhill 1 +luke thompson 3 +luke robinson 6 +luke robinson 1 +luke quirinius 3 +luke polk 3 +luke polk 1 +luke ovid 3 +luke ovid 1 +luke miller 4 +luke laertes 4 +luke laertes 2 +luke laertes 2 +luke laertes 2 +luke laertes 1 +luke johnson 4 +luke johnson 2 +luke johnson 1 +luke ichabod 4 +luke ichabod 1 +luke garcia 5 +luke garcia 2 +luke falkner 4 +luke falkner 2 +luke ellison 3 +luke ellison 2 +luke ellison 1 +luke davidson 2 +luke davidson 2 +luke brown 5 +luke allen 5 +luke allen 2 +luke allen 1 +luke allen 1 +luke allen 1 +katie zipper 1 +katie zipper 1 +katie young 11 +katie young 6 +katie young 1 +katie xylophone 1 +katie white 5 +katie white 3 +katie van buren 6 +katie van buren 4 +katie robinson 2 +katie polk 5 +katie polk 2 +katie ovid 3 +katie nixon 1 +katie miller 1 +katie miller 1 +katie king 7 +katie king 5 +katie king 4 +katie ichabod 6 +katie ichabod 2 +katie ichabod 1 +katie hernandez 1 +katie garcia 4 +katie garcia 3 +katie falkner 4 +katie ellison 5 +katie ellison 4 +katie davidson 1 +katie brown 6 +katie allen 1 +jessica zipper 7 +jessica zipper 6 +jessica zipper 1 +jessica young 4 +jessica young 3 +jessica xylophone 3 +jessica white 8 +jessica white 6 +jessica white 3 +jessica white 1 +jessica white 1 +jessica van buren 1 +jessica underhill 5 +jessica underhill 3 +jessica underhill 2 +jessica thompson 3 +jessica thompson 2 +jessica robinson 2 +jessica quirinius 4 +jessica quirinius 4 +jessica quirinius 3 +jessica quirinius 1 +jessica polk 4 +jessica ovid 2 +jessica ovid 1 +jessica nixon 3 +jessica nixon 2 +jessica miller 5 +jessica johnson 4 +jessica johnson 3 +jessica ichabod 5 +jessica garcia 4 +jessica garcia 3 +jessica falkner 2 +jessica ellison 5 +jessica ellison 2 +jessica davidson 5 +jessica davidson 2 +jessica davidson 2 +jessica davidson 1 +jessica carson 4 +jessica carson 2 +jessica carson 1 +jessica brown 3 +irene xylophone 3 +irene van buren 2 +irene van buren 1 +irene underhill 5 +irene underhill 1 +irene thompson 6 +irene steinbeck 1 +irene robinson 1 +irene quirinius 6 +irene quirinius 5 +irene quirinius 5 +irene polk 3 +irene polk 2 +irene polk 2 +irene polk 1 +irene polk 1 +irene ovid 6 +irene ovid 6 +irene ovid 5 +irene nixon 4 +irene nixon 4 +irene nixon 1 +irene miller 6 +irene laertes 5 +irene laertes 3 +irene laertes 3 +irene johnson 2 +irene ichabod 7 +irene ichabod 1 +irene garcia 4 +irene garcia 2 +irene garcia 2 +irene falkner 5 +irene falkner 2 +irene ellison 4 +irene ellison 3 +irene carson 1 +irene brown 4 +irene brown 4 +irene brown 1 +irene allen 2 +holly zipper 3 +holly zipper 3 +holly young 2 +holly young 2 +holly xylophone 1 +holly white 3 +holly white 1 +holly van buren 4 +holly underhill 6 +holly underhill 3 +holly underhill 3 +holly underhill 2 +holly thompson 2 +holly thompson 1 +holly thompson 1 +holly robinson 2 +holly polk 7 +holly polk 4 +holly nixon 5 +holly nixon 1 +holly miller 4 +holly laertes 5 +holly king 4 +holly king 1 +holly johnson 2 +holly johnson 2 +holly johnson 2 +holly ichabod 4 +holly ichabod 4 +holly ichabod 2 +holly hernandez 9 +holly hernandez 3 +holly hernandez 3 +holly hernandez 2 +holly falkner 6 +holly brown 3 +holly brown 2 +holly allen 1 +gabriella zipper 5 +gabriella zipper 1 +gabriella young 3 +gabriella young 1 +gabriella white 3 +gabriella van buren 3 +gabriella van buren 1 +gabriella thompson 5 +gabriella thompson 5 +gabriella thompson 5 +gabriella steinbeck 4 +gabriella steinbeck 1 +gabriella polk 4 +gabriella polk 4 +gabriella ovid 2 +gabriella ovid 1 +gabriella miller 1 +gabriella laertes 4 +gabriella king 3 +gabriella king 3 +gabriella ichabod 3 +gabriella ichabod 3 +gabriella ichabod 3 +gabriella ichabod 2 +gabriella ichabod 1 +gabriella hernandez 9 +gabriella hernandez 5 +gabriella garcia 2 +gabriella falkner 4 +gabriella falkner 3 +gabriella falkner 2 +gabriella ellison 3 +gabriella ellison 1 +gabriella davidson 2 +gabriella carson 1 +gabriella brown 6 +gabriella brown 3 +gabriella allen 5 +gabriella allen 3 +fred zipper 1 +fred young 2 +fred young 1 +fred white 3 +fred van buren 7 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred underhill 4 +fred steinbeck 4 +fred steinbeck 2 +fred steinbeck 1 +fred robinson 3 +fred quirinius 7 +fred quirinius 4 +fred polk 7 +fred polk 6 +fred polk 4 +fred polk 2 +fred nixon 7 +fred nixon 5 +fred nixon 1 +fred nixon 1 +fred miller 1 +fred laertes 4 +fred king 6 +fred king 3 +fred johnson 4 +fred ichabod 3 +fred ichabod 2 +fred hernandez 1 +fred falkner 4 +fred falkner 3 +fred falkner 3 +fred ellison 5 +fred ellison 2 +fred ellison 1 +fred davidson 2 +fred davidson 2 +fred davidson 1 +ethan zipper 2 +ethan zipper 1 +ethan xylophone 3 +ethan white 5 +ethan white 2 +ethan van buren 1 +ethan underhill 1 +ethan robinson 3 +ethan robinson 1 +ethan quirinius 6 +ethan quirinius 2 +ethan quirinius 1 +ethan polk 3 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan ovid 2 +ethan nixon 7 +ethan miller 5 +ethan laertes 4 +ethan laertes 4 +ethan laertes 3 +ethan laertes 2 +ethan laertes 2 +ethan laertes 2 +ethan laertes 1 +ethan king 1 +ethan johnson 1 +ethan hernandez 3 +ethan garcia 8 +ethan falkner 2 +ethan falkner 1 +ethan ellison 6 +ethan ellison 4 +ethan carson 6 +ethan brown 4 +ethan brown 3 +ethan brown 3 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan allen 4 +david young 4 +david young 1 +david xylophone 6 +david xylophone 4 +david xylophone 1 +david white 2 +david van buren 3 +david van buren 2 +david underhill 7 +david underhill 4 +david underhill 1 +david thompson 1 +david robinson 3 +david robinson 2 +david quirinius 4 +david quirinius 4 +david quirinius 2 +david ovid 4 +david ovid 3 +david nixon 1 +david laertes 4 +david ichabod 6 +david ichabod 3 +david hernandez 7 +david ellison 5 +david ellison 3 +david ellison 3 +david davidson 4 +david davidson 3 +david davidson 1 +david davidson 1 +david brown 6 +david brown 2 +david allen 5 +david allen 2 +calvin zipper 9 +calvin zipper 3 +calvin young 3 +calvin young 1 +calvin xylophone 6 +calvin xylophone 3 +calvin xylophone 1 +calvin white 1 +calvin white 1 +calvin van buren 9 +calvin van buren 1 +calvin underhill 4 +calvin thompson 3 +calvin thompson 2 +calvin steinbeck 6 +calvin steinbeck 3 +calvin steinbeck 3 +calvin robinson 2 +calvin quirinius 4 +calvin quirinius 3 +calvin polk 2 +calvin ovid 5 +calvin ovid 4 +calvin ovid 3 +calvin ovid 1 +calvin nixon 7 +calvin nixon 3 +calvin nixon 2 +calvin laertes 3 +calvin laertes 1 +calvin johnson 2 +calvin hernandez 1 +calvin garcia 3 +calvin falkner 8 +calvin falkner 4 +calvin falkner 4 +calvin falkner 3 +calvin falkner 2 +calvin falkner 1 +calvin ellison 3 +calvin davidson 1 +calvin davidson 1 +calvin carson 1 +calvin brown 5 +calvin brown 3 +calvin brown 1 +calvin allen 1 +bob zipper 4 +bob zipper 1 +bob zipper 1 +bob young 1 +bob xylophone 3 +bob xylophone 2 +bob white 3 +bob white 1 +bob van buren 3 +bob steinbeck 2 +bob quirinius 4 +bob polk 2 +bob ovid 7 +bob ovid 2 +bob ovid 2 +bob ovid 1 +bob miller 1 +bob laertes 5 +bob laertes 1 +bob king 3 +bob king 3 +bob king 2 +bob ichabod 1 +bob hernandez 1 +bob garcia 4 +bob garcia 3 +bob garcia 2 +bob garcia 1 +bob garcia 1 +bob falkner 6 +bob ellison 3 +bob ellison 2 +bob ellison 1 +bob ellison 1 +bob davidson 5 +bob davidson 2 +bob davidson 2 +bob carson 3 +bob brown 8 +bob brown 6 +bob brown 2 +alice zipper 2 +alice zipper 1 +alice zipper 1 +alice xylophone 2 +alice xylophone 2 +alice xylophone 1 +alice van buren 2 +alice underhill 2 +alice steinbeck 7 +alice steinbeck 3 +alice steinbeck 1 +alice robinson 4 +alice robinson 1 +alice quirinius 6 +alice quirinius 4 +alice polk 1 +alice ovid 2 +alice nixon 2 +alice nixon 2 +alice nixon 1 +alice miller 2 +alice laertes 3 +alice laertes 2 +alice king 8 +alice king 4 +alice king 2 +alice johnson 5 +alice hernandez 8 +alice hernandez 8 +alice garcia 1 +alice falkner 5 +alice davidson 2 +alice carson 1 +alice brown 5 +alice allen 5 +alice allen 5 +alice allen 4 + 5 + 4 + 3 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-2-ac487cc1b94130bf9ce00e07c7075f65 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-2-ac487cc1b94130bf9ce00e07c7075f65 new file mode 100644 index 0000000000000..c38e7bbabc21e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-2-ac487cc1b94130bf9ce00e07c7075f65 @@ -0,0 +1,1049 @@ + 0.25047801147227533 + 0.47992351816443596 + 0.6197718631178707 +alice allen 0.7300380228136882 +alice allen 0.8954372623574145 +alice allen 0.9216061185468452 +alice brown 0.22053231939163498 +alice carson 0.2889733840304182 +alice davidson 0.8593155893536122 +alice falkner 0.08604206500956023 +alice garcia 0.2870722433460076 +alice hernandez 0.011472275334608031 +alice hernandez 0.07604562737642585 +alice johnson 0.5181644359464627 +alice king 0.3652007648183556 +alice king 0.8536121673003803 +alice king 0.9771863117870723 +alice laertes 0.870722433460076 +alice laertes 0.870722433460076 +alice miller 0.12045889101338432 +alice nixon 0.4372623574144487 +alice nixon 0.47036328871892924 +alice nixon 0.768642447418738 +alice ovid 0.4665391969407266 +alice polk 0.279467680608365 +alice quirinius 0.8432122370936902 +alice quirinius 0.9923518164435946 +alice robinson 0.5722433460076045 +alice robinson 0.7984790874524715 +alice steinbeck 0.27151051625239003 +alice steinbeck 0.739961759082218 +alice steinbeck 0.9923954372623575 +alice underhill 0.5513307984790875 +alice van buren 0.4923954372623574 +alice xylophone 0.2376425855513308 +alice xylophone 0.26806083650190116 +alice xylophone 0.8776290630975143 +alice zipper 0.33460076045627374 +alice zipper 0.8814531548757171 +alice zipper 0.9445506692160612 +bob brown 0.5038022813688213 +bob brown 0.5066921606118547 +bob brown 0.5372848948374761 +bob carson 0.43346007604562736 +bob davidson 0.21673003802281368 +bob davidson 0.5285171102661597 +bob davidson 0.8413001912045889 +bob ellison 0.2045889101338432 +bob ellison 0.26577437858508607 +bob ellison 0.5793499043977055 +bob ellison 0.9144486692015209 +bob falkner 0.6940726577437859 +bob garcia 0.08555133079847908 +bob garcia 0.17680608365019013 +bob garcia 0.2887189292543021 +bob garcia 0.5418250950570342 +bob garcia 0.5736137667304015 +bob hernandez 0.7813688212927756 +bob ichabod 0.5200764818355641 +bob king 0.0076481835564053535 +bob king 0.5627376425855514 +bob king 0.9524714828897338 +bob laertes 0.32887189292543023 +bob laertes 0.6825095057034221 +bob miller 0.19771863117870722 +bob ovid 0.40304182509505704 +bob ovid 0.40344168260038243 +bob ovid 0.42065009560229444 +bob ovid 0.8403041825095057 +bob polk 0.15019011406844107 +bob quirinius 0.1844106463878327 +bob steinbeck 0.16920152091254753 +bob van buren 0.5086042065009561 +bob white 0.26045627376425856 +bob white 0.7623574144486692 +bob xylophone 0.4474187380497132 +bob xylophone 0.6539923954372624 +bob young 0.4722753346080306 +bob zipper 0.009505703422053232 +bob zipper 0.24091778202676864 +bob zipper 0.4600760456273764 +calvin allen 0.30975143403441685 +calvin brown 0.4448669201520912 +calvin brown 0.5361216730038023 +calvin brown 0.9196940726577438 +calvin carson 0.9315589353612167 +calvin davidson 0.5869980879541109 +calvin davidson 0.6653992395437263 +calvin ellison 0.6977186311787072 +calvin falkner 0.02091254752851711 +calvin falkner 0.03824091778202677 +calvin falkner 0.21223709369024857 +calvin falkner 0.46577946768060835 +calvin falkner 0.5114068441064639 +calvin falkner 0.5950570342205324 +calvin garcia 0.7896749521988528 +calvin hernandez 0.16730038022813687 +calvin johnson 0.9790874524714829 +calvin laertes 0.5487571701720841 +calvin laertes 0.8145315487571702 +calvin nixon 0.019120458891013385 +calvin nixon 0.4467680608365019 +calvin nixon 0.7395437262357415 +calvin ovid 0.14531548757170173 +calvin ovid 0.17490494296577946 +calvin ovid 0.19961977186311788 +calvin ovid 0.9407265774378585 +calvin polk 0.4619771863117871 +calvin quirinius 0.8802281368821293 +calvin quirinius 0.9254302103250478 +calvin robinson 0.13193116634799235 +calvin steinbeck 0.4818355640535373 +calvin steinbeck 0.7418738049713193 +calvin steinbeck 0.8060836501901141 +calvin thompson 0.2179732313575526 +calvin thompson 0.8422053231939164 +calvin underhill 0.7495219885277247 +calvin van buren 0.022813688212927757 +calvin van buren 0.8508604206500956 +calvin white 0.04182509505703422 +calvin white 0.9674952198852772 +calvin xylophone 0.011406844106463879 +calvin xylophone 0.3193116634799235 +calvin xylophone 0.6634799235181644 +calvin young 0.1988527724665392 +calvin young 0.4391634980988593 +calvin zipper 0.5532319391634981 +calvin zipper 0.8726235741444867 +david allen 0.30019120458891013 +david allen 0.3326959847036329 +david brown 0.1338432122370937 +david brown 0.9694072657743786 +david davidson 0.21414913957934992 +david davidson 0.655893536121673 +david davidson 0.7319391634980988 +david davidson 0.8878326996197718 +david ellison 0.6863117870722434 +david ellison 0.6883365200764818 +david ellison 0.7243346007604563 +david hernandez 0.12237093690248566 +david ichabod 0.35564053537284895 +david ichabod 0.7338403041825095 +david laertes 0.3575525812619503 +david nixon 0.33460803059273425 +david ovid 0.3916349809885932 +david ovid 0.6022944550669216 +david quirinius 0.3155893536121673 +david quirinius 0.6577437858508605 +david quirinius 0.9163498098859315 +david robinson 0.6673003802281369 +david robinson 0.6998087954110899 +david thompson 0.25285171102661597 +david underhill 0.1586998087954111 +david underhill 0.35181644359464626 +david underhill 0.7189292543021033 +david van buren 0.05927342256214149 +david van buren 0.5889101338432122 +david white 0.49429657794676807 +david xylophone 0.4875717017208413 +david xylophone 0.6901140684410646 +david xylophone 0.7571701720841301 +david young 0.0019011406844106464 +david young 0.040152963671128104 +ethan allen 0.20532319391634982 +ethan brown 0.10707456978967496 +ethan brown 0.13307984790874525 +ethan brown 0.4340344168260038 +ethan brown 0.4752851711026616 +ethan brown 0.5219885277246654 +ethan brown 0.745697896749522 +ethan carson 0.20912547528517111 +ethan ellison 0.45124282982791586 +ethan ellison 0.8680688336520076 +ethan falkner 0.0994263862332696 +ethan falkner 0.6845124282982792 +ethan garcia 0.06653992395437262 +ethan hernandez 0.2237093690248566 +ethan johnson 0.2300380228136882 +ethan king 0.47418738049713194 +ethan laertes 0.022944550669216062 +ethan laertes 0.2908745247148289 +ethan laertes 0.42638623326959846 +ethan laertes 0.48098859315589354 +ethan laertes 0.6596558317399618 +ethan laertes 0.7839388145315488 +ethan laertes 0.9201520912547528 +ethan miller 0.23709369024856597 +ethan nixon 0.8164435946462715 +ethan ovid 0.6121673003802282 +ethan polk 0.12167300380228137 +ethan polk 0.3384321223709369 +ethan polk 0.6920152091254753 +ethan polk 0.9619771863117871 +ethan quirinius 0.19391634980988592 +ethan quirinius 0.23135755258126195 +ethan quirinius 0.7908745247148289 +ethan robinson 0.24282982791587 +ethan robinson 0.8003802281368821 +ethan underhill 0.6615969581749049 +ethan van buren 0.8365019011406845 +ethan white 0.48859315589353614 +ethan white 0.5741444866920152 +ethan xylophone 0.9695817490494296 +ethan zipper 0.21102661596958175 +ethan zipper 0.6425855513307985 +fred davidson 0.5239005736137667 +fred davidson 0.7414448669201521 +fred davidson 0.8604206500956023 +fred ellison 0.3977055449330784 +fred ellison 0.5506692160611855 +fred ellison 0.7208413001912046 +fred falkner 0.024714828897338403 +fred falkner 0.19120458891013384 +fred falkner 0.9809885931558935 +fred hernandez 0.2734225621414914 +fred ichabod 0.17110266159695817 +fred ichabod 0.780114722753346 +fred johnson 0.30038022813688214 +fred king 0.2198852772466539 +fred king 0.47718631178707227 +fred laertes 0.2332695984703633 +fred miller 0.7858508604206501 +fred nixon 0.005703422053231939 +fred nixon 0.31749049429657794 +fred nixon 0.7648183556405354 +fred nixon 0.8460076045627376 +fred polk 0.16252390057361377 +fred polk 0.564638783269962 +fred polk 0.6273764258555133 +fred polk 0.8155893536121673 +fred quirinius 0.4866920152091255 +fred quirinius 0.8973384030418251 +fred robinson 0.6387832699619772 +fred steinbeck 0.14722753346080306 +fred steinbeck 0.4627151051625239 +fred steinbeck 0.7265774378585086 +fred underhill 0.35361216730038025 +fred van buren 0.3365200764818356 +fred van buren 0.5057034220532319 +fred van buren 0.6463878326996197 +fred van buren 0.904397705544933 +fred white 0.5171102661596958 +fred young 0.7705544933078394 +fred young 0.7992351816443595 +fred zipper 0.615678776290631 +gabriella allen 0.4435946462715105 +gabriella allen 0.9334600760456274 +gabriella brown 0.4359464627151052 +gabriella brown 0.9636711281070746 +gabriella carson 0.9562737642585551 +gabriella davidson 0.8174904942965779 +gabriella ellison 0.1931166347992352 +gabriella ellison 0.38022813688212925 +gabriella falkner 0.3231939163498099 +gabriella falkner 0.5659655831739961 +gabriella falkner 0.8948374760994264 +gabriella garcia 0.4695817490494297 +gabriella hernandez 0.6444866920152091 +gabriella hernandez 0.7015209125475285 +gabriella ichabod 0.09125475285171103 +gabriella ichabod 0.1520912547528517 +gabriella ichabod 0.1835564053537285 +gabriella ichabod 0.372848948374761 +gabriella ichabod 0.8107074569789675 +gabriella king 0.39961759082217974 +gabriella king 0.5190114068441065 +gabriella laertes 0.4569789674952199 +gabriella miller 0.26996197718631176 +gabriella ovid 0.7091254752851711 +gabriella ovid 0.8897338403041825 +gabriella polk 0.030418250950570342 +gabriella polk 0.44106463878326996 +gabriella steinbeck 0.5755258126195029 +gabriella steinbeck 0.8221797323135756 +gabriella thompson 0.013307984790874524 +gabriella thompson 0.44866920152091255 +gabriella thompson 0.7224334600760456 +gabriella van buren 0.6216730038022814 +gabriella van buren 0.6730038022813688 +gabriella white 0.17208413001912046 +gabriella young 0.5076045627376425 +gabriella young 0.7934990439770554 +gabriella zipper 0.23193916349809887 +gabriella zipper 0.8565965583173997 +holly allen 0.11596958174904944 +holly brown 0.11281070745697896 +holly brown 0.155893536121673 +holly falkner 0.124282982791587 +holly hernandez 0.055449330783938815 +holly hernandez 0.32509505703422054 +holly hernandez 0.97131931166348 +holly hernandez 0.9714828897338403 +holly ichabod 0.12357414448669202 +holly ichabod 0.17300380228136883 +holly ichabod 0.629277566539924 +holly johnson 0.33078393881453155 +holly johnson 0.8612167300380228 +holly johnson 0.9391634980988594 +holly king 0.25475285171102663 +holly king 0.3745247148288973 +holly laertes 0.42775665399239543 +holly miller 0.37476099426386233 +holly nixon 0.10076045627376426 +holly nixon 0.34608030592734224 +holly polk 0.40535372848948376 +holly polk 0.5209125475285171 +holly robinson 0.9273422562141491 +holly thompson 0.1596958174904943 +holly thompson 0.311787072243346 +holly thompson 0.9125475285171103 +holly underhill 0.3479923518164436 +holly underhill 0.5812619502868069 +holly underhill 0.8384030418250951 +holly underhill 0.903041825095057 +holly van buren 0.9464627151051626 +holly white 0.1089866156787763 +holly white 0.4780114722753346 +holly xylophone 0.5304182509505704 +holly young 0.7357414448669202 +holly young 0.8240917782026769 +holly zipper 0.15399239543726237 +holly zipper 0.8546845124282982 +irene allen 0.8738049713193117 +irene brown 0.4588910133843212 +irene brown 0.49619771863117873 +irene brown 0.5678776290630975 +irene carson 0.6844106463878327 +irene ellison 0.32504780114722753 +irene ellison 0.48565965583174 +irene falkner 0.41825095057034223 +irene falkner 0.9866920152091255 +irene garcia 0.11663479923518165 +irene garcia 0.29277566539923955 +irene garcia 0.8126195028680688 +irene ichabod 0.8307984790874525 +irene ichabod 0.9177820267686424 +irene johnson 0.7112810707456979 +irene laertes 0.01338432122370937 +irene laertes 0.1482889733840304 +irene laertes 0.7034220532319392 +irene miller 0.367112810707457 +irene nixon 0.11854684512428298 +irene nixon 0.7927756653992395 +irene nixon 0.9426386233269598 +irene ovid 0.24714828897338403 +irene ovid 0.30210325047801145 +irene ovid 0.779467680608365 +irene polk 0.0038022813688212928 +irene polk 0.45315487571701724 +irene polk 0.6577946768060836 +irene polk 0.8891013384321224 +irene polk 0.9789674952198852 +irene quirinius 0.27533460803059273 +irene quirinius 0.35946462715105165 +irene quirinius 0.384321223709369 +irene robinson 0.18631178707224336 +irene steinbeck 0.9942965779467681 +irene thompson 0.6939163498098859 +irene underhill 0.30401529636711283 +irene underhill 0.3403041825095057 +irene van buren 0.5908221797323135 +irene van buren 0.6634980988593155 +irene xylophone 0.5342205323193916 +jessica brown 0.7680608365019012 +jessica carson 0.3574144486692015 +jessica carson 0.6195028680688337 +jessica carson 0.8269961977186312 +jessica davidson 0.10646387832699619 +jessica davidson 0.34790874524714827 +jessica davidson 0.3593155893536122 +jessica davidson 0.6768060836501901 +jessica ellison 0.0779467680608365 +jessica ellison 0.42015209125475284 +jessica falkner 0.994263862332696 +jessica garcia 0.8279158699808795 +jessica garcia 0.9581749049429658 +jessica ichabod 0.45627376425855515 +jessica johnson 0.30228136882129275 +jessica johnson 0.8049713193116634 +jessica miller 0.8011472275334608 +jessica nixon 0.06500956022944551 +jessica nixon 0.6042065009560229 +jessica ovid 0.15105162523900573 +jessica ovid 0.8992395437262357 +jessica polk 0.4378585086042065 +jessica quirinius 0.058935361216730035 +jessica quirinius 0.4714828897338403 +jessica quirinius 0.5760456273764258 +jessica quirinius 0.8935361216730038 +jessica robinson 0.9638783269961977 +jessica thompson 0.08221797323135756 +jessica thompson 0.5893536121673004 +jessica underhill 0.034220532319391636 +jessica underhill 0.06118546845124283 +jessica underhill 0.9541108986615678 +jessica van buren 0.20650095602294455 +jessica white 0.06273764258555133 +jessica white 0.4149139579349904 +jessica white 0.5798479087452472 +jessica white 0.591254752851711 +jessica white 0.7667304015296367 +jessica xylophone 0.5009560229445507 +jessica young 0.3403441682600382 +jessica young 0.8821292775665399 +jessica zipper 0.14068441064638784 +jessica zipper 0.2984790874524715 +jessica zipper 0.6007604562737643 +katie allen 0.5665399239543726 +katie brown 0.49521988527724664 +katie davidson 0.6730401529636711 +katie ellison 0.3173996175908222 +katie ellison 0.7262357414448669 +katie falkner 0.2676864244741874 +katie garcia 0.049429657794676805 +katie garcia 0.3135755258126195 +katie hernandez 0.6026615969581749 +katie ichabod 0.15296367112810708 +katie ichabod 0.4684512428298279 +katie ichabod 0.7055449330783938 +katie king 0.16159695817490494 +katie king 0.502868068833652 +katie king 0.5927342256214149 +katie miller 0.5228136882129277 +katie miller 0.5296367112810707 +katie nixon 0.7832699619771863 +katie ovid 0.8795411089866156 +katie polk 0.35372848948374763 +katie polk 0.9657794676806084 +katie robinson 0.06844106463878327 +katie van buren 0.06883365200764818 +katie van buren 0.1739961759082218 +katie white 0.045889101338432124 +katie white 0.18546845124282982 +katie xylophone 0.7281368821292775 +katie young 0.16443594646271512 +katie young 0.20152091254752852 +katie young 0.9732313575525813 +katie zipper 0.21863117870722434 +katie zipper 0.4505703422053232 +luke allen 0.03612167300380228 +luke allen 0.21606118546845124 +luke allen 0.8346007604562737 +luke allen 0.8631178707224335 +luke allen 0.9311663479923518 +luke brown 0.7304015296367112 +luke davidson 0.25239005736137665 +luke davidson 0.9961977186311787 +luke ellison 0.1147227533460803 +luke ellison 0.2447418738049713 +luke ellison 0.49809885931558934 +luke falkner 0.24524714828897337 +luke falkner 0.5124282982791587 +luke garcia 0.03441682600382409 +luke garcia 0.32695984703632885 +luke ichabod 0.10266159695817491 +luke ichabod 0.5551330798479087 +luke johnson 0.25430210325047803 +luke johnson 0.6787762906309751 +luke johnson 0.9082217973231358 +luke laertes 0.06309751434034416 +luke laertes 0.3690248565965583 +luke laertes 0.7743785850860421 +luke laertes 0.8079847908745247 +luke laertes 0.811787072243346 +luke miller 0.8068833652007649 +luke ovid 0.435361216730038 +luke ovid 0.7547528517110266 +luke polk 0.13957934990439771 +luke polk 0.9770554493307839 +luke quirinius 0.09315589353612168 +luke robinson 0.015209125475285171 +luke robinson 0.053231939163498096 +luke thompson 0.8840304182509505 +luke underhill 0.08745247148288973 +luke underhill 0.40152963671128106 +luke underhill 0.4608030592734226 +luke van buren 0.4847908745247148 +luke white 0.8098859315589354 +luke xylophone 0.34220532319391633 +luke zipper 0.21292775665399238 +mike allen 0.7036328871892925 +mike brown 0.29063097514340347 +mike carson 0.623574144486692 +mike carson 0.7476099426386233 +mike carson 0.9885931558935361 +mike davidson 0.6520912547528517 +mike davidson 0.8298279158699808 +mike ellison 0.24665391969407266 +mike ellison 0.3821292775665399 +mike ellison 0.8355640535372849 +mike ellison 0.8986615678776291 +mike ellison 0.94106463878327 +mike falkner 0.0248565965583174 +mike garcia 0.39543726235741444 +mike garcia 0.5391969407265774 +mike garcia 0.6482889733840305 +mike hernandez 0.07984790874524715 +mike hernandez 0.7186311787072244 +mike ichabod 0.7642585551330798 +mike king 0.09695817490494296 +mike king 0.188212927756654 +mike king 0.4049429657794677 +mike king 0.5544933078393881 +mike king 0.6045627376425855 +mike king 0.9011406844106464 +mike miller 0.621414913957935 +mike nixon 0.688212927756654 +mike nixon 0.9068441064638784 +mike polk 0.3612167300380228 +mike polk 0.6749521988527725 +mike polk 0.8374760994263862 +mike quirinius 0.5105162523900574 +mike steinbeck 0.05736137667304015 +mike steinbeck 0.747148288973384 +mike steinbeck 0.8745247148288974 +mike steinbeck 0.9330783938814532 +mike van buren 0.8650190114068441 +mike van buren 0.973384030418251 +mike white 0.17782026768642448 +mike white 0.7151051625239006 +mike white 0.7566539923954373 +mike white 0.9808795411089866 +mike young 0.20722433460076045 +mike young 0.3840304182509506 +mike young 0.6405353728489483 +mike zipper 0.12810707456978968 +mike zipper 0.42829827915869984 +mike zipper 0.7946768060836502 +nick allen 0.021032504780114723 +nick allen 0.847036328871893 +nick brown 0.14258555133079848 +nick davidson 0.26003824091778205 +nick ellison 0.028680688336520075 +nick ellison 0.3935361216730038 +nick falkner 0.5684410646387833 +nick falkner 0.7590822179732314 +nick garcia 0.34980988593155893 +nick garcia 0.45817490494296575 +nick garcia 0.892925430210325 +nick ichabod 0.2944550669216061 +nick ichabod 0.37667304015296366 +nick ichabod 0.7074569789674953 +nick johnson 0.3973384030418251 +nick johnson 0.4646271510516252 +nick laertes 0.36311787072243346 +nick miller 0.9961759082217974 +nick nixon 0.7110266159695817 +nick ovid 0.7762906309751434 +nick polk 1.0 +nick quirinius 0.0019120458891013384 +nick quirinius 0.08795411089866156 +nick robinson 0.09505703422053231 +nick robinson 0.45506692160611856 +nick steinbeck 0.2224334600760456 +nick thompson 0.4225621414913958 +nick underhill 0.9101338432122371 +nick van buren 0.03802281368821293 +nick xylophone 0.6806883365200764 +nick young 0.4220532319391635 +nick young 0.8623326959847036 +nick zipper 0.2829827915869981 +nick zipper 0.5468451242829828 +oscar allen 0.785171102661597 +oscar brown 0.13498098859315588 +oscar carson 0.07224334600760456 +oscar carson 0.25665399239543724 +oscar carson 0.3422562141491396 +oscar carson 0.6061185468451242 +oscar carson 0.6826003824091779 +oscar davidson 0.7129277566539924 +oscar ellison 0.036328871892925434 +oscar ellison 0.5831739961759083 +oscar falkner 0.9049429657794676 +oscar garcia 0.02676864244741874 +oscar hernandez 0.20076481835564053 +oscar hernandez 0.7870722433460076 +oscar ichabod 0.12619502868068833 +oscar ichabod 0.14149139579349904 +oscar ichabod 0.4416826003824092 +oscar ichabod 0.8661567877629063 +oscar johnson 0.1806083650190114 +oscar johnson 0.467680608365019 +oscar king 0.6596958174904943 +oscar king 0.6787072243346007 +oscar king 0.9258555133079848 +oscar laertes 0.24904942965779467 +oscar laertes 0.5315487571701721 +oscar laertes 0.6328871892925431 +oscar laertes 0.9980988593155894 +oscar nixon 0.9292543021032504 +oscar ovid 0.43021032504780116 +oscar ovid 0.8288973384030418 +oscar ovid 0.8527724665391969 +oscar polk 0.10836501901140684 +oscar polk 0.37858508604206503 +oscar quirinius 0.3041825095057034 +oscar quirinius 0.46387832699619774 +oscar quirinius 0.6311787072243346 +oscar quirinius 0.8555133079847909 +oscar robinson 0.11216730038022814 +oscar robinson 0.22433460076045628 +oscar robinson 0.2294455066921606 +oscar robinson 0.2390057361376673 +oscar steinbeck 0.9904942965779467 +oscar thompson 0.015296367112810707 +oscar thompson 0.2946768060836502 +oscar thompson 0.3060836501901141 +oscar thompson 0.6140684410646388 +oscar underhill 0.31368821292775667 +oscar van buren 0.722753346080306 +oscar van buren 0.7889733840304183 +oscar van buren 0.8833652007648184 +oscar white 0.055133079847908745 +oscar white 0.22562141491395793 +oscar white 0.4321223709369025 +oscar white 0.6443594646271511 +oscar xylophone 0.10133843212237094 +oscar xylophone 0.4187380497131931 +oscar xylophone 0.4296577946768061 +oscar zipper 0.6233269598470363 +oscar zipper 0.7490494296577946 +oscar zipper 0.8783269961977186 +priscilla brown 0.2925430210325048 +priscilla brown 0.6501901140684411 +priscilla brown 0.9120458891013384 +priscilla carson 0.22753346080305928 +priscilla carson 0.5564053537284895 +priscilla carson 0.7820267686424475 +priscilla ichabod 0.3269961977186312 +priscilla ichabod 0.9828897338403042 +priscilla johnson 0.04206500956022945 +priscilla johnson 0.4011406844106464 +priscilla johnson 0.6368821292775665 +priscilla johnson 0.7131931166347992 +priscilla johnson 0.9429657794676806 +priscilla king 0.3517110266159696 +priscilla nixon 0.38049713193116635 +priscilla nixon 0.6864244741873805 +priscilla ovid 0.8193916349809885 +priscilla ovid 0.9139579349904398 +priscilla polk 0.5697896749521989 +priscilla quirinius 0.22179732313575526 +priscilla thompson 0.7737642585551331 +priscilla underhill 0.1682600382409178 +priscilla underhill 0.8852772466539197 +priscilla van buren 0.10325047801147227 +priscilla van buren 0.7877629063097514 +priscilla van buren 0.9598470363288719 +priscilla white 0.4894837476099426 +priscilla xylophone 0.596958174904943 +priscilla xylophone 0.6159695817490495 +priscilla xylophone 0.8393881453154876 +priscilla young 0.41064638783269963 +priscilla young 0.9182509505703422 +priscilla zipper 0.5247148288973384 +priscilla zipper 0.8574144486692015 +quinn allen 0.1634980988593156 +quinn allen 0.9617590822179732 +quinn brown 0.08986615678776291 +quinn brown 0.17590822179732313 +quinn brown 0.5836501901140685 +quinn davidson 0.11787072243346007 +quinn davidson 0.30592734225621415 +quinn davidson 0.3650190114068441 +quinn davidson 0.751434034416826 +quinn ellison 0.376425855513308 +quinn ellison 0.8517110266159695 +quinn garcia 0.17870722433460076 +quinn garcia 0.7323135755258127 +quinn garcia 0.844106463878327 +quinn garcia 0.9486692015209125 +quinn ichabod 0.42395437262357416 +quinn king 0.6653919694072657 +quinn king 0.9505703422053232 +quinn laertes 0.6080305927342257 +quinn laertes 0.9277566539923955 +quinn laertes 0.9847908745247148 +quinn nixon 0.5133079847908745 +quinn ovid 0.16539923954372623 +quinn quirinius 0.19011406844106463 +quinn robinson 0.27756653992395436 +quinn steinbeck 0.23954372623574144 +quinn steinbeck 0.6367112810707457 +quinn thompson 0.4068441064638783 +quinn thompson 0.7782026768642447 +quinn underhill 0.05353728489483748 +quinn underhill 0.5380228136882129 +quinn underhill 0.9349904397705545 +quinn van buren 0.2623574144486692 +quinn young 0.37832699619771865 +quinn zipper 0.51434034416826 +quinn zipper 0.8859315589353612 +rachel allen 0.1701720841300191 +rachel allen 0.3288973384030418 +rachel brown 0.04780114722753346 +rachel brown 0.057034220532319393 +rachel brown 0.532319391634981 +rachel brown 0.5946462715105163 +rachel brown 0.6064638783269962 +rachel carson 0.09177820267686425 +rachel carson 0.6406844106463878 +rachel davidson 0.37093690248565964 +rachel ellison 0.5162523900573613 +rachel falkner 0.1958174904942966 +rachel falkner 0.6330798479087453 +rachel falkner 0.6768642447418738 +rachel falkner 0.9751434034416826 +rachel johnson 0.9560229445506692 +rachel king 0.12547528517110265 +rachel king 0.6003824091778203 +rachel laertes 0.2638623326959847 +rachel laertes 0.5779467680608364 +rachel ovid 0.23518164435946462 +rachel ovid 0.7053231939163498 +rachel polk 0.14638783269961977 +rachel quirinius 0.0076045627376425855 +rachel robinson 0.14340344168260039 +rachel robinson 0.2084130019120459 +rachel robinson 0.6902485659655831 +rachel thompson 0.2718631178707224 +rachel thompson 0.5334608030592735 +rachel thompson 0.875717017208413 +rachel underhill 0.344106463878327 +rachel white 0.17973231357552583 +rachel white 0.6615678776290631 +rachel young 0.3862332695984704 +rachel zipper 0.33079847908745247 +rachel zipper 0.5717017208413002 +sarah carson 0.08935361216730038 +sarah carson 0.18250950570342206 +sarah carson 0.8041825095057035 +sarah ellison 0.967680608365019 +sarah falkner 0.6252390057361377 +sarah falkner 0.9024856596558317 +sarah garcia 0.3881453154875717 +sarah garcia 0.4072657743785851 +sarah garcia 0.6673040152963671 +sarah ichabod 0.29636711281070743 +sarah ichabod 0.9483747609942639 +sarah johnson 0.06463878326996197 +sarah johnson 0.10456273764258556 +sarah johnson 0.5640535372848948 +sarah johnson 0.7954110898661568 +sarah king 0.8030592734225621 +sarah king 0.9655831739961759 +sarah miller 0.6692160611854685 +sarah ovid 0.20342205323193915 +sarah robinson 0.47338403041825095 +sarah robinson 0.7775665399239544 +sarah steinbeck 0.6520076481835564 +sarah white 0.28517110266159695 +sarah white 0.8479087452471483 +sarah xylophone 0.25621414913957935 +sarah young 0.5570342205323194 +sarah zipper 0.5583173996175909 +tom brown 0.5602294455066922 +tom brown 0.8669201520912547 +tom carson 0.045627376425855515 +tom carson 0.35551330798479086 +tom carson 0.935361216730038 +tom davidson 0.8212927756653993 +tom ellison 0.21032504780114722 +tom ellison 0.26195028680688337 +tom ellison 0.7376425855513308 +tom falkner 0.3441682600382409 +tom falkner 0.6481835564053537 +tom hernandez 0.0038240917782026767 +tom hernandez 0.5399239543726235 +tom ichabod 0.6137667304015296 +tom johnson 0.5525812619502868 +tom johnson 0.7915869980879541 +tom king 0.16061185468451242 +tom laertes 0.0745697896749522 +tom laertes 0.5095057034220533 +tom miller 0.2262357414448669 +tom miller 0.2338403041825095 +tom miller 0.2813688212927757 +tom nixon 0.8451242829827916 +tom ovid 0.864244741873805 +tom polk 0.1491395793499044 +tom polk 0.9521988527724665 +tom quirinius 0.09369024856596558 +tom quirinius 0.8489483747609943 +tom robinson 0.060836501901140684 +tom robinson 0.6254752851711026 +tom robinson 0.6462715105162524 +tom robinson 0.9980879541108987 +tom steinbeck 0.5817490494296578 +tom van buren 0.12737642585551331 +tom van buren 0.3154875717017208 +tom van buren 0.7585551330798479 +tom white 0.47609942638623326 +tom young 0.9369024856596558 +tom young 0.9543726235741445 +tom zipper 0.9063097514340345 +ulysses brown 0.9448669201520913 +ulysses carson 0.07034220532319392 +ulysses carson 0.09885931558935361 +ulysses carson 0.2414448669201521 +ulysses carson 0.7604562737642585 +ulysses davidson 0.7093690248565966 +ulysses ellison 0.55893536121673 +ulysses garcia 0.7246653919694073 +ulysses hernandez 0.4091778202676864 +ulysses hernandez 0.627151051625239 +ulysses hernandez 0.982791586998088 +ulysses ichabod 0.21482889733840305 +ulysses ichabod 0.3193916349809886 +ulysses johnson 0.5621414913957935 +ulysses king 0.9467680608365019 +ulysses laertes 0.390057361376673 +ulysses laertes 0.7973231357552581 +ulysses laertes 0.9866156787762906 +ulysses miller 0.31166347992351817 +ulysses miller 0.5774378585086042 +ulysses nixon 0.0057361376673040155 +ulysses ovid 0.38593155893536124 +ulysses polk 0.04752851711026616 +ulysses polk 0.6083650190114068 +ulysses polk 0.7609942638623327 +ulysses polk 0.8326996197718631 +ulysses quirinius 0.6290630975143403 +ulysses robinson 0.9235181644359465 +ulysses steinbeck 0.039923954372623575 +ulysses steinbeck 0.7724665391969407 +ulysses thompson 0.3824091778202677 +ulysses underhill 0.11406844106463879 +ulysses underhill 0.23574144486692014 +ulysses underhill 0.3365019011406844 +ulysses underhill 0.42585551330798477 +ulysses underhill 0.6102661596958175 +ulysses underhill 0.6959847036328872 +ulysses underhill 0.9752851711026616 +ulysses van buren 0.5437262357414449 +ulysses white 0.5 +ulysses white 0.5931558935361216 +ulysses xylophone 0.5855513307984791 +ulysses xylophone 0.8317399617590823 +ulysses xylophone 0.9005736137667304 +ulysses young 0.18164435946462715 +ulysses young 0.3919694072657744 +ulysses young 0.49049429657794674 +victor allen 0.13575525812619502 +victor allen 0.6309751434034416 +victor brown 0.0497131931166348 +victor brown 0.20267686424474188 +victor brown 0.6178707224334601 +victor brown 0.8910133843212237 +victor davidson 0.026615969581749048 +victor davidson 0.491395793499044 +victor davidson 0.5850860420650096 +victor ellison 0.26425855513307983 +victor ellison 0.6692015209125475 +victor hernandez 0.04397705544933078 +victor hernandez 0.12927756653992395 +victor hernandez 0.1950286806883365 +victor hernandez 0.5411089866156787 +victor hernandez 0.7284894837476099 +victor johnson 0.11977186311787072 +victor johnson 0.4828897338403042 +victor johnson 0.7699619771863118 +victor king 0.41254752851711024 +victor king 0.714828897338403 +victor laertes 0.43155893536121676 +victor laertes 0.6500956022944551 +victor miller 0.4429657794676806 +victor nixon 0.33269961977186313 +victor nixon 0.5258126195028681 +victor ovid 0.22813688212927757 +victor polk 0.13878326996197718 +victor quirinius 0.13766730401529637 +victor quirinius 0.887189292543021 +victor robinson 0.5494296577946768 +victor robinson 0.7509505703422054 +victor steinbeck 0.08365019011406843 +victor steinbeck 0.15487571701720843 +victor steinbeck 0.3669201520912547 +victor thompson 0.10516252390057361 +victor van buren 0.27724665391969405 +victor van buren 0.9579349904397706 +victor white 0.41634980988593157 +victor white 0.6349809885931559 +victor xylophone 0.13688212927756654 +victor xylophone 0.3078393881453155 +victor xylophone 0.4110898661567878 +victor xylophone 0.5449330783938815 +victor xylophone 0.9296577946768061 +victor young 0.18738049713193117 +victor zipper 0.5430210325047801 +wendy allen 0.3231357552581262 +wendy allen 0.734225621414914 +wendy allen 0.869980879541109 +wendy brown 0.18929254302103252 +wendy brown 0.6996197718631179 +wendy ellison 0.7437858508604207 +wendy ellison 0.8498098859315589 +wendy falkner 0.07648183556405354 +wendy falkner 0.5353728489483748 +wendy falkner 0.7756653992395437 +wendy garcia 0.07074569789674952 +wendy garcia 0.0741444866920152 +wendy garcia 0.33840304182509506 +wendy garcia 0.38783269961977185 +wendy hernandez 0.017110266159695818 +wendy ichabod 0.8718929254302104 +wendy king 0.37072243346007605 +wendy king 0.497131931166348 +wendy king 0.5965583173996176 +wendy laertes 0.32122370936902483 +wendy laertes 0.49904397705544934 +wendy laertes 0.876425855513308 +wendy miller 0.7533460803059273 +wendy miller 0.7552581261950286 +wendy nixon 0.44933078393881454 +wendy nixon 0.7661596958174905 +wendy ovid 0.5019011406844106 +wendy ovid 0.6978967495219885 +wendy polk 0.3688212927756654 +wendy polk 0.526615969581749 +wendy quirinius 0.1444866920152091 +wendy quirinius 0.5874524714828897 +wendy robinson 0.030592734225621414 +wendy robinson 0.06692160611854685 +wendy robinson 0.27566539923954375 +wendy steinbeck 0.5703422053231939 +wendy thompson 0.028517110266159697 +wendy thompson 0.11089866156787763 +wendy underhill 0.4837476099426386 +wendy underhill 0.6424474187380497 +wendy underhill 0.9600760456273765 +wendy van buren 0.1920152091254753 +wendy van buren 0.7433460076045627 +wendy white 0.752851711026616 +wendy xylophone 0.6347992351816444 +wendy xylophone 0.7452471482889734 +wendy young 0.07839388145315487 +wendy young 0.3897338403041825 +xavier allen 0.043726235741444866 +xavier allen 0.361376673040153 +xavier allen 0.5456273764258555 +xavier brown 0.6711281070745698 +xavier brown 0.9158699808795411 +xavier brown 0.9847036328871893 +xavier carson 0.0841300191204589 +xavier carson 0.988527724665392 +xavier davidson 0.2585551330798479 +xavier davidson 0.4168260038240918 +xavier davidson 0.609942638623327 +xavier ellison 0.5984703632887189 +xavier ellison 0.7361376673040153 +xavier garcia 0.7017208413001912 +xavier hernandez 0.2509505703422053 +xavier hernandez 0.34990439770554493 +xavier hernandez 0.9220532319391636 +xavier ichabod 0.5475285171102662 +xavier ichabod 0.858508604206501 +xavier johnson 0.3938814531548757 +xavier johnson 0.8231939163498099 +xavier king 0.03231939163498099 +xavier king 0.6539196940726577 +xavier laertes 0.5988593155893536 +xavier ovid 0.4397705544933078 +xavier polk 0.4933078393881453 +xavier polk 0.762906309751434 +xavier polk 0.8136882129277566 +xavier polk 0.8260038240917782 +xavier quirinius 0.07265774378585087 +xavier quirinius 0.27915869980879543 +xavier quirinius 0.34600760456273766 +xavier quirinius 0.8022813688212928 +xavier thompson 0.6118546845124283 +xavier underhill 0.16634799235181644 +xavier white 0.6958174904942965 +xavier white 0.7380497131931166 +xavier xylophone 0.8183556405353728 +xavier zipper 0.9904397705544933 +yuri allen 0.9106463878326996 +yuri allen 1.0 +yuri brown 0.5152091254752852 +yuri brown 0.908745247148289 +yuri carson 0.09560229445506692 +yuri carson 0.9372623574144486 +yuri ellison 0.017208413001912046 +yuri ellison 0.39923954372623577 +yuri falkner 0.28680688336520077 +yuri falkner 0.8967495219885278 +yuri garcia 0.2661596958174905 +yuri hernandez 0.28489483747609945 +yuri johnson 0.5047801147227533 +yuri johnson 0.655831739961759 +yuri johnson 0.720532319391635 +yuri king 0.32129277566539927 +yuri laertes 0.4144486692015209 +yuri laertes 0.8916349809885932 +yuri nixon 0.05162523900573614 +yuri nixon 0.40874524714828897 +yuri polk 0.051330798479087454 +yuri polk 0.39579349904397704 +yuri polk 0.6749049429657795 +yuri quirinius 0.08030592734225621 +yuri quirinius 0.2982791586998088 +yuri quirinius 0.4130019120458891 +yuri steinbeck 0.15779467680608364 +yuri steinbeck 0.9388145315487572 +yuri thompson 0.6175908221797323 +yuri underhill 0.42447418738049714 +yuri underhill 0.8202676864244742 +yuri white 0.19694072657743786 +yuri xylophone 0.4790874524714829 +zach allen 0.8250950570342205 +zach brown 0.0817490494296578 +zach brown 0.09751434034416825 +zach brown 0.248565965583174 +zach brown 0.2965779467680608 +zach brown 0.4524714828897338 +zach carson 0.6921606118546845 +zach ellison 0.6806083650190115 +zach falkner 0.25812619502868067 +zach falkner 0.2695984703632887 +zach garcia 0.30798479087452474 +zach garcia 0.3632887189292543 +zach garcia 0.7072243346007605 +zach garcia 0.7167300380228137 +zach ichabod 0.30988593155893535 +zach ichabod 0.9502868068833652 +zach king 0.5277246653919694 +zach king 0.8336520076481836 +zach king 0.9239543726235742 +zach miller 0.15678776290630975 +zach miller 0.3726235741444867 +zach miller 0.5608365019011406 +zach ovid 0.1311787072243346 +zach ovid 0.2737642585551331 +zach ovid 0.4543726235741445 +zach ovid 0.6711026615969582 +zach quirinius 0.019011406844106463 +zach robinson 0.11026615969581749 +zach steinbeck 0.28107074569789675 +zach steinbeck 0.7170172084130019 +zach thompson 0.13001912045889102 +zach thompson 0.44550669216061184 +zach underhill 0.7718631178707225 +zach white 0.7965779467680608 +zach xylophone 0.032504780114722756 +zach xylophone 0.638623326959847 +zach young 0.009560229445506692 +zach zipper 0.24334600760456274 +zach zipper 0.2832699619771863 +zach zipper 0.8087954110898662 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-3-b82dfa24123047be4b4e3c27c3997d34 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-3-b82dfa24123047be4b4e3c27c3997d34 new file mode 100644 index 0000000000000..1e0cf03db63a0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-3-b82dfa24123047be4b4e3c27c3997d34 @@ -0,0 +1,1049 @@ +zach zipper 0.0 +zach zipper 0.0 +zach zipper 0.0 +zach young 0.0 +zach xylophone 0.0 +zach xylophone 0.0 +zach white 0.0 +zach underhill 0.0 +zach thompson 0.0 +zach thompson 0.0 +zach steinbeck 0.0 +zach steinbeck 0.0 +zach robinson 0.0 +zach quirinius 0.0 +zach ovid 0.0 +zach ovid 0.0 +zach ovid 0.0 +zach ovid 0.0 +zach miller 0.0 +zach miller 0.0 +zach miller 0.0 +zach king 0.0 +zach king 0.0 +zach king 0.0 +zach ichabod 0.0 +zach ichabod 0.0 +zach garcia 0.0 +zach garcia 0.0 +zach garcia 0.0 +zach garcia 0.0 +zach falkner 0.0 +zach falkner 0.0 +zach ellison 0.0 +zach carson 0.0 +zach brown 0.0 +zach brown 0.0 +zach brown 0.0 +zach brown 0.0 +zach brown 0.0 +zach allen 0.0 +yuri xylophone 0.0 +yuri white 0.0 +yuri underhill 0.0 +yuri underhill 0.0 +yuri thompson 0.0 +yuri steinbeck 0.0 +yuri steinbeck 0.0 +yuri quirinius 0.0 +yuri quirinius 0.0 +yuri quirinius 0.0 +yuri polk 0.0 +yuri polk 0.0 +yuri polk 0.0 +yuri nixon 0.0 +yuri nixon 0.0 +yuri laertes 0.0 +yuri laertes 0.0 +yuri king 0.0 +yuri johnson 0.0 +yuri johnson 0.0 +yuri johnson 0.0 +yuri hernandez 0.0 +yuri garcia 0.0 +yuri falkner 0.0 +yuri falkner 0.0 +yuri ellison 0.0 +yuri ellison 0.0 +yuri carson 0.0 +yuri carson 0.0 +yuri brown 0.0 +yuri brown 0.0 +yuri allen 0.0 +yuri allen 0.0 +xavier zipper 1.0 +xavier xylophone 0.0 +xavier white 0.0 +xavier white 0.0 +xavier underhill 0.0 +xavier thompson 0.0 +xavier quirinius 0.0 +xavier quirinius 0.0 +xavier quirinius 0.0 +xavier quirinius 0.0 +xavier polk 0.0 +xavier polk 0.0 +xavier polk 0.0 +xavier polk 0.0 +xavier ovid 0.0 +xavier laertes 0.0 +xavier king 0.0 +xavier king 0.0 +xavier johnson 0.0 +xavier johnson 0.0 +xavier ichabod 0.0 +xavier ichabod 0.0 +xavier hernandez 0.0 +xavier hernandez 0.0 +xavier hernandez 0.0 +xavier garcia 0.0 +xavier ellison 0.0 +xavier ellison 0.0 +xavier davidson 0.0 +xavier davidson 0.0 +xavier davidson 0.0 +xavier carson 1.0 +xavier carson 0.0 +xavier brown 0.0 +xavier brown 0.0 +xavier brown 0.0 +xavier allen 0.0 +xavier allen 0.0 +xavier allen 0.0 +wendy young 1.0 +wendy young 0.0 +wendy xylophone 0.0 +wendy xylophone 0.0 +wendy white 0.0 +wendy van buren 0.0 +wendy van buren 0.0 +wendy underhill 0.0 +wendy underhill 0.0 +wendy underhill 0.0 +wendy thompson 0.0 +wendy thompson 0.0 +wendy steinbeck 0.0 +wendy robinson 0.0 +wendy robinson 0.0 +wendy robinson 0.0 +wendy quirinius 0.0 +wendy quirinius 0.0 +wendy polk 0.0 +wendy polk 0.0 +wendy ovid 0.0 +wendy ovid 0.0 +wendy nixon 0.0 +wendy nixon 0.0 +wendy miller 0.0 +wendy miller 0.0 +wendy laertes 0.0 +wendy laertes 0.0 +wendy laertes 0.0 +wendy king 0.0 +wendy king 0.0 +wendy king 0.0 +wendy ichabod 0.0 +wendy hernandez 0.0 +wendy garcia 0.0 +wendy garcia 0.0 +wendy garcia 0.0 +wendy garcia 0.0 +wendy falkner 0.0 +wendy falkner 0.0 +wendy falkner 0.0 +wendy ellison 0.0 +wendy ellison 0.0 +wendy brown 0.0 +wendy brown 0.0 +wendy allen 0.0 +wendy allen 0.0 +wendy allen 0.0 +victor zipper 0.0 +victor young 0.0 +victor xylophone 0.0 +victor xylophone 0.0 +victor xylophone 0.0 +victor xylophone 0.0 +victor xylophone 0.0 +victor white 1.0 +victor white 0.0 +victor van buren 0.0 +victor van buren 0.0 +victor thompson 0.0 +victor steinbeck 0.0 +victor steinbeck 0.0 +victor steinbeck 0.0 +victor robinson 0.0 +victor robinson 0.0 +victor quirinius 0.0 +victor quirinius 0.0 +victor polk 0.0 +victor ovid 0.0 +victor nixon 0.0 +victor nixon 0.0 +victor miller 0.0 +victor laertes 0.0 +victor laertes 0.0 +victor king 0.0 +victor king 0.0 +victor johnson 0.0 +victor johnson 0.0 +victor johnson 0.0 +victor hernandez 0.0 +victor hernandez 0.0 +victor hernandez 0.0 +victor hernandez 0.0 +victor hernandez 0.0 +victor ellison 0.0 +victor ellison 0.0 +victor davidson 0.0 +victor davidson 0.0 +victor davidson 0.0 +victor brown 0.0 +victor brown 0.0 +victor brown 0.0 +victor brown 0.0 +victor allen 0.0 +victor allen 0.0 +ulysses young 0.0 +ulysses young 0.0 +ulysses young 0.0 +ulysses xylophone 0.0 +ulysses xylophone 0.0 +ulysses xylophone 0.0 +ulysses white 0.0 +ulysses white 0.0 +ulysses van buren 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses thompson 1.0 +ulysses steinbeck 0.0 +ulysses steinbeck 0.0 +ulysses robinson 0.0 +ulysses quirinius 0.0 +ulysses polk 1.0 +ulysses polk 0.0 +ulysses polk 0.0 +ulysses polk 0.0 +ulysses ovid 0.0 +ulysses nixon 0.0 +ulysses miller 0.0 +ulysses miller 0.0 +ulysses laertes 0.0 +ulysses laertes 0.0 +ulysses laertes 0.0 +ulysses king 0.0 +ulysses johnson 0.0 +ulysses ichabod 0.0 +ulysses ichabod 0.0 +ulysses hernandez 1.0 +ulysses hernandez 0.0 +ulysses hernandez 0.0 +ulysses garcia 0.0 +ulysses ellison 1.0 +ulysses davidson 0.0 +ulysses carson 0.0 +ulysses carson 0.0 +ulysses carson 0.0 +ulysses carson 0.0 +ulysses brown 0.0 +tom zipper 0.0 +tom young 0.0 +tom young 0.0 +tom white 0.0 +tom van buren 0.0 +tom van buren 0.0 +tom van buren 0.0 +tom steinbeck 0.0 +tom robinson 0.0 +tom robinson 0.0 +tom robinson 0.0 +tom robinson 0.0 +tom quirinius 0.0 +tom quirinius 0.0 +tom polk 0.0 +tom polk 0.0 +tom ovid 0.0 +tom nixon 0.0 +tom miller 0.0 +tom miller 0.0 +tom miller 0.0 +tom laertes 0.0 +tom laertes 0.0 +tom king 0.0 +tom johnson 0.0 +tom johnson 0.0 +tom ichabod 0.0 +tom hernandez 0.0 +tom hernandez 0.0 +tom falkner 0.0 +tom falkner 0.0 +tom ellison 0.0 +tom ellison 0.0 +tom ellison 0.0 +tom davidson 0.0 +tom carson 0.0 +tom carson 0.0 +tom carson 0.0 +tom brown 0.0 +tom brown 0.0 +sarah zipper 0.0 +sarah young 0.0 +sarah xylophone 0.0 +sarah white 0.0 +sarah white 0.0 +sarah steinbeck 0.0 +sarah robinson 0.0 +sarah robinson 0.0 +sarah ovid 0.0 +sarah miller 0.0 +sarah king 0.0 +sarah king 0.0 +sarah johnson 0.0 +sarah johnson 0.0 +sarah johnson 0.0 +sarah johnson 0.0 +sarah ichabod 0.0 +sarah ichabod 0.0 +sarah garcia 0.0 +sarah garcia 0.0 +sarah garcia 0.0 +sarah falkner 0.0 +sarah falkner 0.0 +sarah ellison 0.0 +sarah carson 0.0 +sarah carson 0.0 +sarah carson 0.0 +rachel zipper 0.0 +rachel zipper 0.0 +rachel young 0.0 +rachel white 0.0 +rachel white 0.0 +rachel underhill 0.0 +rachel thompson 0.0 +rachel thompson 0.0 +rachel thompson 0.0 +rachel robinson 1.0 +rachel robinson 0.0 +rachel robinson 0.0 +rachel quirinius 0.0 +rachel polk 0.0 +rachel ovid 0.0 +rachel ovid 0.0 +rachel laertes 0.0 +rachel laertes 0.0 +rachel king 0.0 +rachel king 0.0 +rachel johnson 0.0 +rachel falkner 0.0 +rachel falkner 0.0 +rachel falkner 0.0 +rachel falkner 0.0 +rachel ellison 0.0 +rachel davidson 0.0 +rachel carson 0.0 +rachel carson 0.0 +rachel brown 1.0 +rachel brown 0.0 +rachel brown 0.0 +rachel brown 0.0 +rachel brown 0.0 +rachel allen 0.0 +rachel allen 0.0 +quinn zipper 0.0 +quinn zipper 0.0 +quinn young 0.0 +quinn van buren 0.0 +quinn underhill 0.0 +quinn underhill 0.0 +quinn underhill 0.0 +quinn thompson 0.0 +quinn thompson 0.0 +quinn steinbeck 0.0 +quinn steinbeck 0.0 +quinn robinson 0.0 +quinn quirinius 0.0 +quinn ovid 0.0 +quinn nixon 0.0 +quinn laertes 1.0 +quinn laertes 0.0 +quinn laertes 0.0 +quinn king 1.0 +quinn king 0.0 +quinn ichabod 0.0 +quinn garcia 1.0 +quinn garcia 0.0 +quinn garcia 0.0 +quinn garcia 0.0 +quinn ellison 0.0 +quinn ellison 0.0 +quinn davidson 1.0 +quinn davidson 0.0 +quinn davidson 0.0 +quinn davidson 0.0 +quinn brown 0.0 +quinn brown 0.0 +quinn brown 0.0 +quinn allen 1.0 +quinn allen 0.0 +priscilla zipper 0.0 +priscilla zipper 0.0 +priscilla young 0.0 +priscilla young 0.0 +priscilla xylophone 0.0 +priscilla xylophone 0.0 +priscilla xylophone 0.0 +priscilla white 1.0 +priscilla van buren 0.0 +priscilla van buren 0.0 +priscilla van buren 0.0 +priscilla underhill 0.0 +priscilla underhill 0.0 +priscilla thompson 0.0 +priscilla quirinius 0.0 +priscilla polk 0.0 +priscilla ovid 0.0 +priscilla ovid 0.0 +priscilla nixon 0.0 +priscilla nixon 0.0 +priscilla king 0.0 +priscilla johnson 0.0 +priscilla johnson 0.0 +priscilla johnson 0.0 +priscilla johnson 0.0 +priscilla johnson 0.0 +priscilla ichabod 0.0 +priscilla ichabod 0.0 +priscilla carson 0.0 +priscilla carson 0.0 +priscilla carson 0.0 +priscilla brown 0.0 +priscilla brown 0.0 +priscilla brown 0.0 +oscar zipper 0.0 +oscar zipper 0.0 +oscar zipper 0.0 +oscar xylophone 0.0 +oscar xylophone 0.0 +oscar xylophone 0.0 +oscar white 0.0 +oscar white 0.0 +oscar white 0.0 +oscar white 0.0 +oscar van buren 1.0 +oscar van buren 0.0 +oscar van buren 0.0 +oscar underhill 0.0 +oscar thompson 0.0 +oscar thompson 0.0 +oscar thompson 0.0 +oscar thompson 0.0 +oscar steinbeck 0.0 +oscar robinson 0.0 +oscar robinson 0.0 +oscar robinson 0.0 +oscar robinson 0.0 +oscar quirinius 0.0 +oscar quirinius 0.0 +oscar quirinius 0.0 +oscar quirinius 0.0 +oscar polk 1.0 +oscar polk 0.0 +oscar ovid 0.0 +oscar ovid 0.0 +oscar ovid 0.0 +oscar nixon 0.0 +oscar laertes 0.0 +oscar laertes 0.0 +oscar laertes 0.0 +oscar laertes 0.0 +oscar king 0.0 +oscar king 0.0 +oscar king 0.0 +oscar johnson 0.0 +oscar johnson 0.0 +oscar ichabod 0.0 +oscar ichabod 0.0 +oscar ichabod 0.0 +oscar ichabod 0.0 +oscar hernandez 0.0 +oscar hernandez 0.0 +oscar garcia 0.0 +oscar falkner 1.0 +oscar ellison 0.0 +oscar ellison 0.0 +oscar davidson 0.0 +oscar carson 0.0 +oscar carson 0.0 +oscar carson 0.0 +oscar carson 0.0 +oscar carson 0.0 +oscar brown 0.0 +oscar allen 0.0 +nick zipper 0.0 +nick zipper 0.0 +nick young 1.0 +nick young 0.0 +nick xylophone 0.0 +nick van buren 0.0 +nick underhill 0.0 +nick thompson 0.0 +nick steinbeck 0.0 +nick robinson 0.0 +nick robinson 0.0 +nick quirinius 0.0 +nick quirinius 0.0 +nick polk 0.0 +nick ovid 0.0 +nick nixon 0.0 +nick miller 0.0 +nick laertes 0.0 +nick johnson 0.0 +nick johnson 0.0 +nick ichabod 0.0 +nick ichabod 0.0 +nick ichabod 0.0 +nick garcia 0.0 +nick garcia 0.0 +nick garcia 0.0 +nick falkner 0.0 +nick falkner 0.0 +nick ellison 0.0 +nick ellison 0.0 +nick davidson 0.0 +nick brown 0.0 +nick allen 0.0 +nick allen 0.0 +mike zipper 0.0 +mike zipper 0.0 +mike zipper 0.0 +mike young 0.0 +mike young 0.0 +mike young 0.0 +mike white 0.0 +mike white 0.0 +mike white 0.0 +mike white 0.0 +mike van buren 0.0 +mike van buren 0.0 +mike steinbeck 0.0 +mike steinbeck 0.0 +mike steinbeck 0.0 +mike steinbeck 0.0 +mike quirinius 0.0 +mike polk 0.0 +mike polk 0.0 +mike polk 0.0 +mike nixon 0.0 +mike nixon 0.0 +mike miller 0.0 +mike king 0.0 +mike king 0.0 +mike king 0.0 +mike king 0.0 +mike king 0.0 +mike king 0.0 +mike ichabod 0.0 +mike hernandez 0.0 +mike hernandez 0.0 +mike garcia 0.0 +mike garcia 0.0 +mike garcia 0.0 +mike falkner 0.0 +mike ellison 0.0 +mike ellison 0.0 +mike ellison 0.0 +mike ellison 0.0 +mike ellison 0.0 +mike davidson 0.0 +mike davidson 0.0 +mike carson 0.0 +mike carson 0.0 +mike carson 0.0 +mike brown 0.0 +mike allen 0.0 +luke zipper 0.0 +luke xylophone 0.0 +luke white 0.0 +luke van buren 0.0 +luke underhill 1.0 +luke underhill 0.0 +luke underhill 0.0 +luke thompson 0.0 +luke robinson 0.0 +luke robinson 0.0 +luke quirinius 0.0 +luke polk 0.0 +luke polk 0.0 +luke ovid 0.0 +luke ovid 0.0 +luke miller 0.0 +luke laertes 0.0 +luke laertes 0.0 +luke laertes 0.0 +luke laertes 0.0 +luke laertes 0.0 +luke johnson 0.0 +luke johnson 0.0 +luke johnson 0.0 +luke ichabod 0.0 +luke ichabod 0.0 +luke garcia 0.0 +luke garcia 0.0 +luke falkner 0.0 +luke falkner 0.0 +luke ellison 0.0 +luke ellison 0.0 +luke ellison 0.0 +luke davidson 0.0 +luke davidson 0.0 +luke brown 0.0 +luke allen 1.0 +luke allen 0.0 +luke allen 0.0 +luke allen 0.0 +luke allen 0.0 +katie zipper 1.0 +katie zipper 0.0 +katie young 1.0 +katie young 0.0 +katie young 0.0 +katie xylophone 0.0 +katie white 0.0 +katie white 0.0 +katie van buren 0.0 +katie van buren 0.0 +katie robinson 0.0 +katie polk 0.0 +katie polk 0.0 +katie ovid 0.0 +katie nixon 1.0 +katie miller 0.0 +katie miller 0.0 +katie king 0.0 +katie king 0.0 +katie king 0.0 +katie ichabod 0.0 +katie ichabod 0.0 +katie ichabod 0.0 +katie hernandez 0.0 +katie garcia 0.0 +katie garcia 0.0 +katie falkner 0.0 +katie ellison 0.0 +katie ellison 0.0 +katie davidson 0.0 +katie brown 0.0 +katie allen 0.0 +jessica zipper 0.0 +jessica zipper 0.0 +jessica zipper 0.0 +jessica young 0.0 +jessica young 0.0 +jessica xylophone 0.0 +jessica white 0.0 +jessica white 0.0 +jessica white 0.0 +jessica white 0.0 +jessica white 0.0 +jessica van buren 0.0 +jessica underhill 1.0 +jessica underhill 0.0 +jessica underhill 0.0 +jessica thompson 0.0 +jessica thompson 0.0 +jessica robinson 0.0 +jessica quirinius 1.0 +jessica quirinius 0.0 +jessica quirinius 0.0 +jessica quirinius 0.0 +jessica polk 0.0 +jessica ovid 0.0 +jessica ovid 0.0 +jessica nixon 0.0 +jessica nixon 0.0 +jessica miller 0.0 +jessica johnson 0.0 +jessica johnson 0.0 +jessica ichabod 1.0 +jessica garcia 0.0 +jessica garcia 0.0 +jessica falkner 0.0 +jessica ellison 0.0 +jessica ellison 0.0 +jessica davidson 0.0 +jessica davidson 0.0 +jessica davidson 0.0 +jessica davidson 0.0 +jessica carson 0.0 +jessica carson 0.0 +jessica carson 0.0 +jessica brown 0.0 +irene xylophone 0.0 +irene van buren 0.0 +irene van buren 0.0 +irene underhill 0.0 +irene underhill 0.0 +irene thompson 0.0 +irene steinbeck 0.0 +irene robinson 0.0 +irene quirinius 0.5 +irene quirinius 0.0 +irene quirinius 0.0 +irene polk 0.0 +irene polk 0.0 +irene polk 0.0 +irene polk 0.0 +irene polk 0.0 +irene ovid 0.0 +irene ovid 0.0 +irene ovid 0.0 +irene nixon 0.0 +irene nixon 0.0 +irene nixon 0.0 +irene miller 0.0 +irene laertes 0.0 +irene laertes 0.0 +irene laertes 0.0 +irene johnson 0.0 +irene ichabod 0.0 +irene ichabod 0.0 +irene garcia 0.0 +irene garcia 0.0 +irene garcia 0.0 +irene falkner 0.0 +irene falkner 0.0 +irene ellison 0.0 +irene ellison 0.0 +irene carson 0.0 +irene brown 0.0 +irene brown 0.0 +irene brown 0.0 +irene allen 0.0 +holly zipper 1.0 +holly zipper 0.0 +holly young 0.0 +holly young 0.0 +holly xylophone 0.0 +holly white 0.0 +holly white 0.0 +holly van buren 0.0 +holly underhill 1.0 +holly underhill 0.0 +holly underhill 0.0 +holly underhill 0.0 +holly thompson 1.0 +holly thompson 0.0 +holly thompson 0.0 +holly robinson 0.0 +holly polk 0.0 +holly polk 0.0 +holly nixon 0.0 +holly nixon 0.0 +holly miller 1.0 +holly laertes 0.0 +holly king 0.0 +holly king 0.0 +holly johnson 0.0 +holly johnson 0.0 +holly johnson 0.0 +holly ichabod 0.0 +holly ichabod 0.0 +holly ichabod 0.0 +holly hernandez 0.0 +holly hernandez 0.0 +holly hernandez 0.0 +holly hernandez 0.0 +holly falkner 0.0 +holly brown 0.0 +holly brown 0.0 +holly allen 0.0 +gabriella zipper 0.0 +gabriella zipper 0.0 +gabriella young 0.0 +gabriella young 0.0 +gabriella white 0.0 +gabriella van buren 0.0 +gabriella van buren 0.0 +gabriella thompson 0.0 +gabriella thompson 0.0 +gabriella thompson 0.0 +gabriella steinbeck 0.0 +gabriella steinbeck 0.0 +gabriella polk 0.0 +gabriella polk 0.0 +gabriella ovid 0.0 +gabriella ovid 0.0 +gabriella miller 0.0 +gabriella laertes 0.0 +gabriella king 0.0 +gabriella king 0.0 +gabriella ichabod 1.0 +gabriella ichabod 0.0 +gabriella ichabod 0.0 +gabriella ichabod 0.0 +gabriella ichabod 0.0 +gabriella hernandez 1.0 +gabriella hernandez 0.0 +gabriella garcia 0.0 +gabriella falkner 0.0 +gabriella falkner 0.0 +gabriella falkner 0.0 +gabriella ellison 0.0 +gabriella ellison 0.0 +gabriella davidson 0.0 +gabriella carson 0.0 +gabriella brown 0.0 +gabriella brown 0.0 +gabriella allen 0.0 +gabriella allen 0.0 +fred zipper 0.0 +fred young 0.0 +fred young 0.0 +fred white 0.0 +fred van buren 0.0 +fred van buren 0.0 +fred van buren 0.0 +fred van buren 0.0 +fred underhill 0.0 +fred steinbeck 0.0 +fred steinbeck 0.0 +fred steinbeck 0.0 +fred robinson 1.0 +fred quirinius 0.0 +fred quirinius 0.0 +fred polk 0.0 +fred polk 0.0 +fred polk 0.0 +fred polk 0.0 +fred nixon 0.0 +fred nixon 0.0 +fred nixon 0.0 +fred nixon 0.0 +fred miller 0.0 +fred laertes 0.0 +fred king 0.0 +fred king 0.0 +fred johnson 1.0 +fred ichabod 0.0 +fred ichabod 0.0 +fred hernandez 0.0 +fred falkner 1.0 +fred falkner 0.0 +fred falkner 0.0 +fred ellison 0.0 +fred ellison 0.0 +fred ellison 0.0 +fred davidson 0.0 +fred davidson 0.0 +fred davidson 0.0 +ethan zipper 0.0 +ethan zipper 0.0 +ethan xylophone 0.0 +ethan white 0.0 +ethan white 0.0 +ethan van buren 0.0 +ethan underhill 0.5 +ethan robinson 0.0 +ethan robinson 0.0 +ethan quirinius 0.0 +ethan quirinius 0.0 +ethan quirinius 0.0 +ethan polk 1.0 +ethan polk 0.0 +ethan polk 0.0 +ethan polk 0.0 +ethan ovid 0.0 +ethan nixon 0.0 +ethan miller 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan king 0.0 +ethan johnson 0.0 +ethan hernandez 0.0 +ethan garcia 0.0 +ethan falkner 0.0 +ethan falkner 0.0 +ethan ellison 0.0 +ethan ellison 0.0 +ethan carson 0.0 +ethan brown 1.0 +ethan brown 0.0 +ethan brown 0.0 +ethan brown 0.0 +ethan brown 0.0 +ethan brown 0.0 +ethan allen 0.0 +david young 0.0 +david young 0.0 +david xylophone 1.0 +david xylophone 0.0 +david xylophone 0.0 +david white 0.0 +david van buren 0.0 +david van buren 0.0 +david underhill 0.0 +david underhill 0.0 +david underhill 0.0 +david thompson 1.0 +david robinson 0.0 +david robinson 0.0 +david quirinius 0.0 +david quirinius 0.0 +david quirinius 0.0 +david ovid 0.0 +david ovid 0.0 +david nixon 0.0 +david laertes 0.0 +david ichabod 1.0 +david ichabod 0.0 +david hernandez 1.0 +david ellison 0.0 +david ellison 0.0 +david ellison 0.0 +david davidson 0.0 +david davidson 0.0 +david davidson 0.0 +david davidson 0.0 +david brown 0.0 +david brown 0.0 +david allen 0.0 +david allen 0.0 +calvin zipper 0.0 +calvin zipper 0.0 +calvin young 0.0 +calvin young 0.0 +calvin xylophone 0.0 +calvin xylophone 0.0 +calvin xylophone 0.0 +calvin white 0.0 +calvin white 0.0 +calvin van buren 1.0 +calvin van buren 0.0 +calvin underhill 0.0 +calvin thompson 0.0 +calvin thompson 0.0 +calvin steinbeck 0.0 +calvin steinbeck 0.0 +calvin steinbeck 0.0 +calvin robinson 0.0 +calvin quirinius 0.0 +calvin quirinius 0.0 +calvin polk 0.0 +calvin ovid 0.0 +calvin ovid 0.0 +calvin ovid 0.0 +calvin ovid 0.0 +calvin nixon 0.0 +calvin nixon 0.0 +calvin nixon 0.0 +calvin laertes 0.0 +calvin laertes 0.0 +calvin johnson 0.0 +calvin hernandez 0.0 +calvin garcia 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin ellison 0.0 +calvin davidson 0.0 +calvin davidson 0.0 +calvin carson 0.0 +calvin brown 0.0 +calvin brown 0.0 +calvin brown 0.0 +calvin allen 0.0 +bob zipper 0.0 +bob zipper 0.0 +bob zipper 0.0 +bob young 0.0 +bob xylophone 0.0 +bob xylophone 0.0 +bob white 0.0 +bob white 0.0 +bob van buren 0.0 +bob steinbeck 0.0 +bob quirinius 0.0 +bob polk 0.0 +bob ovid 0.0 +bob ovid 0.0 +bob ovid 0.0 +bob ovid 0.0 +bob miller 0.0 +bob laertes 0.0 +bob laertes 0.0 +bob king 1.0 +bob king 0.0 +bob king 0.0 +bob ichabod 0.0 +bob hernandez 1.0 +bob garcia 0.0 +bob garcia 0.0 +bob garcia 0.0 +bob garcia 0.0 +bob garcia 0.0 +bob falkner 0.0 +bob ellison 1.0 +bob ellison 0.0 +bob ellison 0.0 +bob ellison 0.0 +bob davidson 0.0 +bob davidson 0.0 +bob davidson 0.0 +bob carson 0.0 +bob brown 0.0 +bob brown 0.0 +bob brown 0.0 +alice zipper 0.0 +alice zipper 0.0 +alice zipper 0.0 +alice xylophone 0.0 +alice xylophone 0.0 +alice xylophone 0.0 +alice van buren 0.0 +alice underhill 0.0 +alice steinbeck 0.0 +alice steinbeck 0.0 +alice steinbeck 0.0 +alice robinson 0.0 +alice robinson 0.0 +alice quirinius 0.0 +alice quirinius 0.0 +alice polk 1.0 +alice ovid 0.0 +alice nixon 0.0 +alice nixon 0.0 +alice nixon 0.0 +alice miller 0.0 +alice laertes 0.0 +alice laertes 0.0 +alice king 0.0 +alice king 0.0 +alice king 0.0 +alice johnson 0.0 +alice hernandez 0.0 +alice hernandez 0.0 +alice garcia 0.0 +alice falkner 0.0 +alice davidson 0.0 +alice carson 0.0 +alice brown 0.0 +alice allen 0.0 +alice allen 0.0 +alice allen 0.0 + 0.0 + 0.0 + 0.0 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 new file mode 100644 index 0000000000000..9091a9156134c --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 @@ -0,0 +1,1294 @@ +2013-03-01 09:11:58.70307 52.64 1 +2013-03-01 09:11:58.70307 52.64 1 +2013-03-01 09:11:58.70307 52.64 1 +2013-03-01 09:11:58.70307 52.64 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703073 10.07 1 +2013-03-01 09:11:58.703073 10.07 1 +2013-03-01 09:11:58.703073 10.07 1 +2013-03-01 09:11:58.703073 10.07 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703075 5.64 1 +2013-03-01 09:11:58.703075 5.64 1 +2013-03-01 09:11:58.703075 5.64 1 +2013-03-01 09:11:58.703075 5.64 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703079 27.32 1 +2013-03-01 09:11:58.703079 27.32 1 +2013-03-01 09:11:58.703079 27.32 1 +2013-03-01 09:11:58.703079 27.32 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703087 10.63 1 +2013-03-01 09:11:58.703087 10.63 1 +2013-03-01 09:11:58.703087 10.63 1 +2013-03-01 09:11:58.703087 10.63 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703089 8.76 1 +2013-03-01 09:11:58.703089 8.76 1 +2013-03-01 09:11:58.703089 8.76 1 +2013-03-01 09:11:58.703089 8.76 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.703091 15.85 1 +2013-03-01 09:11:58.703091 15.85 1 +2013-03-01 09:11:58.703091 15.85 1 +2013-03-01 09:11:58.703091 15.85 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703093 14.85 1 +2013-03-01 09:11:58.703093 14.85 1 +2013-03-01 09:11:58.703093 14.85 1 +2013-03-01 09:11:58.703094 57.11 1 +2013-03-01 09:11:58.703094 57.11 1 +2013-03-01 09:11:58.703094 57.11 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703099 11.69 1 +2013-03-01 09:11:58.703099 11.69 1 +2013-03-01 09:11:58.703099 11.69 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703105 28.47 1 +2013-03-01 09:11:58.703105 28.47 1 +2013-03-01 09:11:58.703106 11.81 1 +2013-03-01 09:11:58.703106 11.81 1 +2013-03-01 09:11:58.703106 11.81 1 +2013-03-01 09:11:58.703106 11.81 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703108 28.47 1 +2013-03-01 09:11:58.703108 28.47 1 +2013-03-01 09:11:58.703108 28.47 1 +2013-03-01 09:11:58.703108 28.47 1 +2013-03-01 09:11:58.703109 38.98 1 +2013-03-01 09:11:58.703109 38.98 1 +2013-03-01 09:11:58.703109 38.98 1 +2013-03-01 09:11:58.703109 38.98 1 +2013-03-01 09:11:58.70311 8.16 1 +2013-03-01 09:11:58.70311 8.16 1 +2013-03-01 09:11:58.70311 8.16 1 +2013-03-01 09:11:58.70311 8.16 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703116 33.45 1 +2013-03-01 09:11:58.703116 33.45 1 +2013-03-01 09:11:58.703116 33.45 1 +2013-03-01 09:11:58.703116 33.45 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703118 8.69 1 +2013-03-01 09:11:58.703118 8.69 1 +2013-03-01 09:11:58.703119 58.02 1 +2013-03-01 09:11:58.703119 58.02 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703123 94.35 1 +2013-03-01 09:11:58.703123 94.35 1 +2013-03-01 09:11:58.703123 94.35 1 +2013-03-01 09:11:58.703123 94.35 1 +2013-03-01 09:11:58.703124 8.93 1 +2013-03-01 09:11:58.703124 8.93 1 +2013-03-01 09:11:58.703124 8.93 1 +2013-03-01 09:11:58.703125 14.94 1 +2013-03-01 09:11:58.703125 14.94 1 +2013-03-01 09:11:58.703125 14.94 1 +2013-03-01 09:11:58.703126 5.49 1 +2013-03-01 09:11:58.703126 5.49 1 +2013-03-01 09:11:58.703126 5.49 1 +2013-03-01 09:11:58.703126 5.49 1 +2013-03-01 09:11:58.703127 3.98 1 +2013-03-01 09:11:58.703127 3.98 1 +2013-03-01 09:11:58.703127 3.98 1 +2013-03-01 09:11:58.703127 3.98 1 +2013-03-01 09:11:58.703128 11.45 1 +2013-03-01 09:11:58.703128 11.45 1 +2013-03-01 09:11:58.703128 11.45 1 +2013-03-01 09:11:58.703128 11.45 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703132 1.86 1 +2013-03-01 09:11:58.703132 1.86 1 +2013-03-01 09:11:58.703133 27.34 1 +2013-03-01 09:11:58.703133 27.34 1 +2013-03-01 09:11:58.703133 27.34 1 +2013-03-01 09:11:58.703133 27.34 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703135 29.14 1 +2013-03-01 09:11:58.703135 29.14 1 +2013-03-01 09:11:58.703135 29.14 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703138 55.68 1 +2013-03-01 09:11:58.703138 55.68 1 +2013-03-01 09:11:58.703138 55.68 1 +2013-03-01 09:11:58.703138 55.68 1 +2013-03-01 09:11:58.703139 12.67 1 +2013-03-01 09:11:58.703139 12.67 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703142 24.25 1 +2013-03-01 09:11:58.703142 24.25 1 +2013-03-01 09:11:58.703142 24.25 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703144 3.43 1 +2013-03-01 09:11:58.703144 3.43 1 +2013-03-01 09:11:58.703144 3.43 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.70315 33.01 1 +2013-03-01 09:11:58.70315 33.01 1 +2013-03-01 09:11:58.70315 33.01 1 +2013-03-01 09:11:58.70315 33.01 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703153 4.11 1 +2013-03-01 09:11:58.703153 4.11 1 +2013-03-01 09:11:58.703153 4.11 1 +2013-03-01 09:11:58.703153 4.11 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703158 71.89 1 +2013-03-01 09:11:58.703158 71.89 1 +2013-03-01 09:11:58.703158 71.89 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.70316 11.93 1 +2013-03-01 09:11:58.70316 11.93 1 +2013-03-01 09:11:58.70316 11.93 1 +2013-03-01 09:11:58.70316 11.93 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703165 8.38 1 +2013-03-01 09:11:58.703165 8.38 1 +2013-03-01 09:11:58.703165 8.38 1 +2013-03-01 09:11:58.703166 16.6 1 +2013-03-01 09:11:58.703166 16.6 1 +2013-03-01 09:11:58.703166 16.6 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703173 8.77 1 +2013-03-01 09:11:58.703173 8.77 1 +2013-03-01 09:11:58.703173 8.77 1 +2013-03-01 09:11:58.703173 8.77 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703179 10.82 1 +2013-03-01 09:11:58.703179 10.82 1 +2013-03-01 09:11:58.703179 10.82 1 +2013-03-01 09:11:58.703179 10.82 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703183 36.74 1 +2013-03-01 09:11:58.703183 36.74 1 +2013-03-01 09:11:58.703183 36.74 1 +2013-03-01 09:11:58.703183 36.74 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703186 13.81 1 +2013-03-01 09:11:58.703186 13.81 1 +2013-03-01 09:11:58.703186 13.81 1 +2013-03-01 09:11:58.703186 13.81 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.70319 90.09 1 +2013-03-01 09:11:58.70319 90.09 1 +2013-03-01 09:11:58.70319 90.09 1 +2013-03-01 09:11:58.703192 2.63 1 +2013-03-01 09:11:58.703192 2.63 1 +2013-03-01 09:11:58.703192 2.63 1 +2013-03-01 09:11:58.703192 2.63 1 +2013-03-01 09:11:58.703193 28.42 1 +2013-03-01 09:11:58.703193 28.42 1 +2013-03-01 09:11:58.703193 28.42 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703196 0.08 1 +2013-03-01 09:11:58.703196 0.08 1 +2013-03-01 09:11:58.703196 0.08 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703199 45.69 1 +2013-03-01 09:11:58.703199 45.69 1 +2013-03-01 09:11:58.703199 45.69 1 +2013-03-01 09:11:58.703199 45.69 1 +2013-03-01 09:11:58.7032 12.72 1 +2013-03-01 09:11:58.7032 12.72 1 +2013-03-01 09:11:58.703201 35.15 1 +2013-03-01 09:11:58.703201 35.15 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703206 6.61 1 +2013-03-01 09:11:58.703206 6.61 1 +2013-03-01 09:11:58.703206 6.61 1 +2013-03-01 09:11:58.703206 6.61 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703209 25.92 1 +2013-03-01 09:11:58.703209 25.92 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703213 38.71 1 +2013-03-01 09:11:58.703213 38.71 1 +2013-03-01 09:11:58.703213 38.71 1 +2013-03-01 09:11:58.703213 38.71 1 +2013-03-01 09:11:58.703214 31.35 1 +2013-03-01 09:11:58.703214 31.35 1 +2013-03-01 09:11:58.703214 31.35 1 +2013-03-01 09:11:58.703215 18.78 1 +2013-03-01 09:11:58.703215 18.78 1 +2013-03-01 09:11:58.703215 18.78 1 +2013-03-01 09:11:58.703215 18.78 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703217 23.57 1 +2013-03-01 09:11:58.703217 23.57 1 +2013-03-01 09:11:58.703217 23.57 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703225 35.51 1 +2013-03-01 09:11:58.703225 35.51 1 +2013-03-01 09:11:58.703225 35.51 1 +2013-03-01 09:11:58.703225 35.51 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703228 4.19 1 +2013-03-01 09:11:58.703228 4.19 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703232 61.96 1 +2013-03-01 09:11:58.703232 61.96 1 +2013-03-01 09:11:58.703232 61.96 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703234 44.1 1 +2013-03-01 09:11:58.703234 44.1 1 +2013-03-01 09:11:58.703234 44.1 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.70324 5.1 1 +2013-03-01 09:11:58.70324 5.1 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703243 6.15 1 +2013-03-01 09:11:58.703243 6.15 1 +2013-03-01 09:11:58.703243 6.15 1 +2013-03-01 09:11:58.703243 6.15 1 +2013-03-01 09:11:58.703244 4.47 1 +2013-03-01 09:11:58.703244 4.47 1 +2013-03-01 09:11:58.703244 4.47 1 +2013-03-01 09:11:58.703245 0.72 1 +2013-03-01 09:11:58.703245 0.72 1 +2013-03-01 09:11:58.703245 0.72 1 +2013-03-01 09:11:58.703245 0.72 1 +2013-03-01 09:11:58.703246 45.94 1 +2013-03-01 09:11:58.703246 45.94 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703249 19.42 1 +2013-03-01 09:11:58.703249 19.42 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.703251 68.98 1 +2013-03-01 09:11:58.703251 68.98 1 +2013-03-01 09:11:58.703251 68.98 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703256 23.78 1 +2013-03-01 09:11:58.703256 23.78 1 +2013-03-01 09:11:58.703256 23.78 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.70326 3.81 1 +2013-03-01 09:11:58.70326 3.81 1 +2013-03-01 09:11:58.70326 3.81 1 +2013-03-01 09:11:58.70326 3.81 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703266 47.71 1 +2013-03-01 09:11:58.703266 47.71 1 +2013-03-01 09:11:58.703266 47.71 1 +2013-03-01 09:11:58.703266 47.71 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703269 43.84 1 +2013-03-01 09:11:58.703269 43.84 1 +2013-03-01 09:11:58.703269 43.84 1 +2013-03-01 09:11:58.70327 5.01 1 +2013-03-01 09:11:58.70327 5.01 1 +2013-03-01 09:11:58.70327 5.01 1 +2013-03-01 09:11:58.70327 5.01 1 +2013-03-01 09:11:58.703271 61.16 1 +2013-03-01 09:11:58.703271 61.16 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703273 10.94 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703275 20.57 1 +2013-03-01 09:11:58.703275 20.57 1 +2013-03-01 09:11:58.703275 20.57 1 +2013-03-01 09:11:58.703276 63.54 1 +2013-03-01 09:11:58.703276 63.54 1 +2013-03-01 09:11:58.703276 63.54 1 +2013-03-01 09:11:58.703276 63.54 1 +2013-03-01 09:11:58.703277 3.37 1 +2013-03-01 09:11:58.703277 3.37 1 +2013-03-01 09:11:58.703277 3.37 1 +2013-03-01 09:11:58.703277 3.37 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.703281 19.95 1 +2013-03-01 09:11:58.703281 19.95 1 +2013-03-01 09:11:58.703281 19.95 1 +2013-03-01 09:11:58.703281 19.95 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703286 9.53 1 +2013-03-01 09:11:58.703286 9.53 1 +2013-03-01 09:11:58.703286 9.53 1 +2013-03-01 09:11:58.703286 9.53 1 +2013-03-01 09:11:58.703287 0.89 1 +2013-03-01 09:11:58.703287 0.89 1 +2013-03-01 09:11:58.703287 0.89 1 +2013-03-01 09:11:58.703288 60.57 1 +2013-03-01 09:11:58.703288 60.57 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.703291 1.15 1 +2013-03-01 09:11:58.703291 1.15 1 +2013-03-01 09:11:58.703291 1.15 1 +2013-03-01 09:11:58.703291 1.15 1 +2013-03-01 09:11:58.703292 4.24 1 +2013-03-01 09:11:58.703292 4.24 1 +2013-03-01 09:11:58.703292 4.24 1 +2013-03-01 09:11:58.703292 4.24 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703299 9 1 +2013-03-01 09:11:58.703299 9 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703304 9.04 1 +2013-03-01 09:11:58.703304 9.04 1 +2013-03-01 09:11:58.703304 9.04 1 +2013-03-01 09:11:58.703304 9.04 1 +2013-03-01 09:11:58.703305 18.68 1 +2013-03-01 09:11:58.703305 18.68 1 +2013-03-01 09:11:58.703305 18.68 1 +2013-03-01 09:11:58.703305 18.68 1 +2013-03-01 09:11:58.703306 3.95 1 +2013-03-01 09:11:58.703306 3.95 1 +2013-03-01 09:11:58.703306 3.95 1 +2013-03-01 09:11:58.703306 3.95 1 +2013-03-01 09:11:58.703307 31.28 1 +2013-03-01 09:11:58.703307 31.28 1 +2013-03-01 09:11:58.703307 31.28 1 +2013-03-01 09:11:58.703308 16.95 1 +2013-03-01 09:11:58.703308 16.95 1 +2013-03-01 09:11:58.703308 16.95 1 +2013-03-01 09:11:58.703308 16.95 1 +2013-03-01 09:11:58.703309 11.16 1 +2013-03-01 09:11:58.703309 11.16 1 +2013-03-01 09:11:58.703309 11.16 1 +2013-03-01 09:11:58.703309 11.16 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703314 39.12 1 +2013-03-01 09:11:58.703314 39.12 1 +2013-03-01 09:11:58.703314 39.12 1 +2013-03-01 09:11:58.703314 39.12 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703317 1.45 1 +2013-03-01 09:11:58.703317 1.45 1 +2013-03-01 09:11:58.703317 1.45 1 +2013-03-01 09:11:58.703317 1.45 1 +2013-03-01 09:11:58.703318 21.81 1 +2013-03-01 09:11:58.703318 21.81 1 +2013-03-01 09:11:58.703318 21.81 1 +2013-03-01 09:11:58.703318 21.81 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.703321 3.91 1 +2013-03-01 09:11:58.703321 3.91 1 +2013-03-01 09:11:58.703321 3.91 1 +2013-03-01 09:11:58.703321 3.91 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703323 36.22 1 +2013-03-01 09:11:58.703323 36.22 1 +2013-03-01 09:11:58.703323 36.22 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 3-0-58a982694ba2b1e34de82b1de54936a0 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 3-0-58a982694ba2b1e34de82b1de54936a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 new file mode 100644 index 0000000000000..d02ca48857b5f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 @@ -0,0 +1,474 @@ +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703077 66.68 1 +2013-03-01 09:11:58.703077 66.68 1 +2013-03-01 09:11:58.703077 66.68 1 +2013-03-01 09:11:58.703077 66.68 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703092 54.02 1 +2013-03-01 09:11:58.703092 54.02 1 +2013-03-01 09:11:58.703092 54.02 1 +2013-03-01 09:11:58.703096 87.84 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.70311 65.88 1 +2013-03-01 09:11:58.70311 65.88 1 +2013-03-01 09:11:58.70311 65.88 1 +2013-03-01 09:11:58.70311 65.88 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703118 8.69 1 +2013-03-01 09:11:58.703118 8.69 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703132 1.86 1 +2013-03-01 09:11:58.703132 1.86 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703139 43.53 1 +2013-03-01 09:11:58.703139 43.53 1 +2013-03-01 09:11:58.703139 43.53 1 +2013-03-01 09:11:58.703139 43.53 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703156 62.42 1 +2013-03-01 09:11:58.703156 62.42 1 +2013-03-01 09:11:58.703156 62.42 1 +2013-03-01 09:11:58.703156 62.42 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703178 93.29 1 +2013-03-01 09:11:58.703178 93.29 1 +2013-03-01 09:11:58.703178 93.29 1 +2013-03-01 09:11:58.703178 93.29 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703198 97.18 1 +2013-03-01 09:11:58.703198 97.18 1 +2013-03-01 09:11:58.703198 97.18 1 +2013-03-01 09:11:58.703198 97.18 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703209 25.92 1 +2013-03-01 09:11:58.703209 25.92 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703224 42.93 1 +2013-03-01 09:11:58.703224 42.93 1 +2013-03-01 09:11:58.703224 42.93 1 +2013-03-01 09:11:58.703224 42.93 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703246 72.87 1 +2013-03-01 09:11:58.703246 72.87 1 +2013-03-01 09:11:58.703248 81.28 1 +2013-03-01 09:11:58.703248 81.28 1 +2013-03-01 09:11:58.703248 81.28 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703258 21.21 1 +2013-03-01 09:11:58.703258 21.21 1 +2013-03-01 09:11:58.703258 21.21 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703262 78.56 1 +2013-03-01 09:11:58.703262 78.56 1 +2013-03-01 09:11:58.703262 78.56 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703265 11.46 1 +2013-03-01 09:11:58.703265 11.46 1 +2013-03-01 09:11:58.703265 11.46 1 +2013-03-01 09:11:58.703265 11.46 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703273 30.49 1 +2013-03-01 09:11:58.703273 30.49 1 +2013-03-01 09:11:58.703273 30.49 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703276 88.46 1 +2013-03-01 09:11:58.703276 88.46 1 +2013-03-01 09:11:58.703276 88.46 1 +2013-03-01 09:11:58.703276 88.46 1 +2013-03-01 09:11:58.703278 69.42 1 +2013-03-01 09:11:58.703278 69.42 1 +2013-03-01 09:11:58.703278 69.42 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.70331 69.26 1 +2013-03-01 09:11:58.70331 69.26 1 +2013-03-01 09:11:58.70331 69.26 1 +2013-03-01 09:11:58.703313 20.69 1 +2013-03-01 09:11:58.703313 20.69 1 +2013-03-01 09:11:58.703313 20.69 1 +2013-03-01 09:11:58.703313 20.69 1 +2013-03-01 09:11:58.703315 53.04 1 +2013-03-01 09:11:58.703315 53.04 1 +2013-03-01 09:11:58.703315 53.04 1 +2013-03-01 09:11:58.703315 53.04 1 +2013-03-01 09:11:58.703318 85.62 1 +2013-03-01 09:11:58.703318 85.62 1 +2013-03-01 09:11:58.703318 85.62 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703324 98.36 1 +2013-03-01 09:11:58.703324 98.36 1 +2013-03-01 09:11:58.703324 98.36 1 +2013-03-01 09:11:58.703324 98.36 1 +2013-03-01 09:11:58.703325 65.81 1 +2013-03-01 09:11:58.703325 65.81 1 +2013-03-01 09:11:58.703325 65.81 1 +2013-03-01 09:11:58.703325 65.81 1 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-0-6642a21d87e0401ba1a668ea8b244f0c b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-0-6642a21d87e0401ba1a668ea8b244f0c new file mode 100644 index 0000000000000..119dd71df1426 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-0-6642a21d87e0401ba1a668ea8b244f0c @@ -0,0 +1,1049 @@ + 65560 + 65560 + 65560 +alice allen 65662 +alice allen 65662 +alice allen 65662 +alice brown 65696 +alice carson 65559 +alice davidson 65547 +alice falkner 65669 +alice garcia 65613 +alice hernandez 65737 +alice hernandez 65737 +alice johnson 65739 +alice king 65660 +alice king 65660 +alice king 65660 +alice laertes 65669 +alice laertes 65669 +alice miller 65590 +alice nixon 65586 +alice nixon 65586 +alice nixon 65586 +alice ovid 65737 +alice polk 65548 +alice quirinius 65636 +alice quirinius 65636 +alice robinson 65606 +alice robinson 65606 +alice steinbeck 65578 +alice steinbeck 65578 +alice steinbeck 65578 +alice underhill 65750 +alice van buren 65562 +alice xylophone 65578 +alice xylophone 65578 +alice xylophone 65578 +alice zipper 65553 +alice zipper 65553 +alice zipper 65553 +bob brown 65584 +bob brown 65584 +bob brown 65584 +bob carson 65713 +bob davidson 65664 +bob davidson 65664 +bob davidson 65664 +bob ellison 65591 +bob ellison 65591 +bob ellison 65591 +bob ellison 65591 +bob falkner 65789 +bob garcia 65585 +bob garcia 65585 +bob garcia 65585 +bob garcia 65585 +bob garcia 65585 +bob hernandez 65557 +bob ichabod 65549 +bob king 65715 +bob king 65715 +bob king 65715 +bob laertes 65602 +bob laertes 65602 +bob miller 65608 +bob ovid 65564 +bob ovid 65564 +bob ovid 65564 +bob ovid 65564 +bob polk 65594 +bob quirinius 65700 +bob steinbeck 65637 +bob van buren 65778 +bob white 65543 +bob white 65543 +bob xylophone 65574 +bob xylophone 65574 +bob young 65556 +bob zipper 65559 +bob zipper 65559 +bob zipper 65559 +calvin allen 65669 +calvin brown 65537 +calvin brown 65537 +calvin brown 65537 +calvin carson 65637 +calvin davidson 65541 +calvin davidson 65541 +calvin ellison 65667 +calvin falkner 65573 +calvin falkner 65573 +calvin falkner 65573 +calvin falkner 65573 +calvin falkner 65573 +calvin falkner 65573 +calvin garcia 65664 +calvin hernandez 65578 +calvin johnson 65731 +calvin laertes 65570 +calvin laertes 65570 +calvin nixon 65654 +calvin nixon 65654 +calvin nixon 65654 +calvin ovid 65554 +calvin ovid 65554 +calvin ovid 65554 +calvin ovid 65554 +calvin polk 65731 +calvin quirinius 65741 +calvin quirinius 65741 +calvin robinson 65581 +calvin steinbeck 65680 +calvin steinbeck 65680 +calvin steinbeck 65680 +calvin thompson 65560 +calvin thompson 65560 +calvin underhill 65732 +calvin van buren 65552 +calvin van buren 65552 +calvin white 65553 +calvin white 65553 +calvin xylophone 65575 +calvin xylophone 65575 +calvin xylophone 65575 +calvin young 65574 +calvin young 65574 +calvin zipper 65669 +calvin zipper 65669 +david allen 65588 +david allen 65588 +david brown 65637 +david brown 65637 +david davidson 65559 +david davidson 65559 +david davidson 65559 +david davidson 65559 +david ellison 65634 +david ellison 65634 +david ellison 65634 +david hernandez 65763 +david ichabod 65699 +david ichabod 65699 +david laertes 65762 +david nixon 65536 +david ovid 65623 +david ovid 65623 +david quirinius 65697 +david quirinius 65697 +david quirinius 65697 +david robinson 65762 +david robinson 65762 +david thompson 65550 +david underhill 65602 +david underhill 65602 +david underhill 65602 +david van buren 65625 +david van buren 65625 +david white 65678 +david xylophone 65537 +david xylophone 65537 +david xylophone 65537 +david young 65551 +david young 65551 +ethan allen 65747 +ethan brown 65539 +ethan brown 65539 +ethan brown 65539 +ethan brown 65539 +ethan brown 65539 +ethan brown 65539 +ethan carson 65742 +ethan ellison 65714 +ethan ellison 65714 +ethan falkner 65577 +ethan falkner 65577 +ethan garcia 65736 +ethan hernandez 65618 +ethan johnson 65536 +ethan king 65614 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan miller 65712 +ethan nixon 65766 +ethan ovid 65697 +ethan polk 65589 +ethan polk 65589 +ethan polk 65589 +ethan polk 65589 +ethan quirinius 65542 +ethan quirinius 65542 +ethan quirinius 65542 +ethan robinson 65547 +ethan robinson 65547 +ethan underhill 65570 +ethan van buren 65572 +ethan white 65677 +ethan white 65677 +ethan xylophone 65595 +ethan zipper 65593 +ethan zipper 65593 +fred davidson 65552 +fred davidson 65552 +fred davidson 65552 +fred ellison 65548 +fred ellison 65548 +fred ellison 65548 +fred falkner 65637 +fred falkner 65637 +fred falkner 65637 +fred hernandez 65541 +fred ichabod 65572 +fred ichabod 65572 +fred johnson 65758 +fred king 65694 +fred king 65694 +fred laertes 65769 +fred miller 65536 +fred nixon 65560 +fred nixon 65560 +fred nixon 65560 +fred nixon 65560 +fred polk 65603 +fred polk 65603 +fred polk 65603 +fred polk 65603 +fred quirinius 65697 +fred quirinius 65697 +fred robinson 65623 +fred steinbeck 65544 +fred steinbeck 65544 +fred steinbeck 65544 +fred underhill 65629 +fred van buren 65537 +fred van buren 65537 +fred van buren 65537 +fred van buren 65537 +fred white 65589 +fred young 65594 +fred young 65594 +fred zipper 65553 +gabriella allen 65646 +gabriella allen 65646 +gabriella brown 65704 +gabriella brown 65704 +gabriella carson 65586 +gabriella davidson 65565 +gabriella ellison 65706 +gabriella ellison 65706 +gabriella falkner 65623 +gabriella falkner 65623 +gabriella falkner 65623 +gabriella garcia 65571 +gabriella hernandez 65587 +gabriella hernandez 65587 +gabriella ichabod 65559 +gabriella ichabod 65559 +gabriella ichabod 65559 +gabriella ichabod 65559 +gabriella ichabod 65559 +gabriella king 65651 +gabriella king 65651 +gabriella laertes 65781 +gabriella miller 65646 +gabriella ovid 65556 +gabriella ovid 65556 +gabriella polk 65701 +gabriella polk 65701 +gabriella steinbeck 65582 +gabriella steinbeck 65582 +gabriella thompson 65682 +gabriella thompson 65682 +gabriella thompson 65682 +gabriella van buren 65581 +gabriella van buren 65581 +gabriella white 65638 +gabriella young 65699 +gabriella young 65699 +gabriella zipper 65540 +gabriella zipper 65540 +holly allen 65596 +holly brown 65599 +holly brown 65599 +holly falkner 65720 +holly hernandez 65602 +holly hernandez 65602 +holly hernandez 65602 +holly hernandez 65602 +holly ichabod 65711 +holly ichabod 65711 +holly ichabod 65711 +holly johnson 65655 +holly johnson 65655 +holly johnson 65655 +holly king 65549 +holly king 65549 +holly laertes 65664 +holly miller 65653 +holly nixon 65539 +holly nixon 65539 +holly polk 65743 +holly polk 65743 +holly robinson 65564 +holly thompson 65538 +holly thompson 65538 +holly thompson 65538 +holly underhill 65634 +holly underhill 65634 +holly underhill 65634 +holly underhill 65634 +holly van buren 65727 +holly white 65536 +holly white 65536 +holly xylophone 65544 +holly young 65606 +holly young 65606 +holly zipper 65607 +holly zipper 65607 +irene allen 65556 +irene brown 65633 +irene brown 65633 +irene brown 65633 +irene carson 65590 +irene ellison 65659 +irene ellison 65659 +irene falkner 65620 +irene falkner 65620 +irene garcia 65660 +irene garcia 65660 +irene garcia 65660 +irene ichabod 65645 +irene ichabod 65645 +irene johnson 65583 +irene laertes 65664 +irene laertes 65664 +irene laertes 65664 +irene miller 65730 +irene nixon 65631 +irene nixon 65631 +irene nixon 65631 +irene ovid 65691 +irene ovid 65691 +irene ovid 65691 +irene polk 65551 +irene polk 65551 +irene polk 65551 +irene polk 65551 +irene polk 65551 +irene quirinius 65724 +irene quirinius 65724 +irene quirinius 65724 +irene robinson 65554 +irene steinbeck 65683 +irene thompson 65688 +irene underhill 65591 +irene underhill 65591 +irene van buren 65579 +irene van buren 65579 +irene xylophone 65775 +jessica brown 65588 +jessica carson 65553 +jessica carson 65553 +jessica carson 65553 +jessica davidson 65549 +jessica davidson 65549 +jessica davidson 65549 +jessica davidson 65549 +jessica ellison 65567 +jessica ellison 65567 +jessica falkner 65584 +jessica garcia 65676 +jessica garcia 65676 +jessica ichabod 65704 +jessica johnson 65607 +jessica johnson 65607 +jessica miller 65733 +jessica nixon 65590 +jessica nixon 65590 +jessica ovid 65582 +jessica ovid 65582 +jessica polk 65637 +jessica quirinius 65562 +jessica quirinius 65562 +jessica quirinius 65562 +jessica quirinius 65562 +jessica robinson 65576 +jessica thompson 65581 +jessica thompson 65581 +jessica underhill 65656 +jessica underhill 65656 +jessica underhill 65656 +jessica van buren 65615 +jessica white 65544 +jessica white 65544 +jessica white 65544 +jessica white 65544 +jessica white 65544 +jessica xylophone 65562 +jessica young 65623 +jessica young 65623 +jessica zipper 65600 +jessica zipper 65600 +jessica zipper 65600 +katie allen 65542 +katie brown 65590 +katie davidson 65619 +katie ellison 65675 +katie ellison 65675 +katie falkner 65728 +katie garcia 65625 +katie garcia 65625 +katie hernandez 65550 +katie ichabod 65658 +katie ichabod 65658 +katie ichabod 65658 +katie king 65629 +katie king 65629 +katie king 65629 +katie miller 65541 +katie miller 65541 +katie nixon 65669 +katie ovid 65681 +katie polk 65746 +katie polk 65746 +katie robinson 65697 +katie van buren 65643 +katie van buren 65643 +katie white 65620 +katie white 65620 +katie xylophone 65585 +katie young 65644 +katie young 65644 +katie young 65644 +katie zipper 65568 +katie zipper 65568 +luke allen 65547 +luke allen 65547 +luke allen 65547 +luke allen 65547 +luke allen 65547 +luke brown 65719 +luke davidson 65656 +luke davidson 65656 +luke ellison 65582 +luke ellison 65582 +luke ellison 65582 +luke falkner 65589 +luke falkner 65589 +luke garcia 65687 +luke garcia 65687 +luke ichabod 65629 +luke ichabod 65629 +luke johnson 65545 +luke johnson 65545 +luke johnson 65545 +luke laertes 65608 +luke laertes 65608 +luke laertes 65608 +luke laertes 65608 +luke laertes 65608 +luke miller 65752 +luke ovid 65569 +luke ovid 65569 +luke polk 65645 +luke polk 65645 +luke quirinius 65655 +luke robinson 65634 +luke robinson 65634 +luke thompson 65626 +luke underhill 65553 +luke underhill 65553 +luke underhill 65553 +luke van buren 65678 +luke white 65693 +luke xylophone 65597 +luke zipper 65641 +mike allen 65706 +mike brown 65654 +mike carson 65698 +mike carson 65698 +mike carson 65698 +mike davidson 65658 +mike davidson 65658 +mike ellison 65598 +mike ellison 65598 +mike ellison 65598 +mike ellison 65598 +mike ellison 65598 +mike falkner 65609 +mike garcia 65571 +mike garcia 65571 +mike garcia 65571 +mike hernandez 65548 +mike hernandez 65548 +mike ichabod 65621 +mike king 65563 +mike king 65563 +mike king 65563 +mike king 65563 +mike king 65563 +mike king 65563 +mike miller 65549 +mike nixon 65619 +mike nixon 65619 +mike polk 65619 +mike polk 65619 +mike polk 65619 +mike quirinius 65717 +mike steinbeck 65550 +mike steinbeck 65550 +mike steinbeck 65550 +mike steinbeck 65550 +mike van buren 65620 +mike van buren 65620 +mike white 65648 +mike white 65648 +mike white 65648 +mike white 65648 +mike young 65545 +mike young 65545 +mike young 65545 +mike zipper 65552 +mike zipper 65552 +mike zipper 65552 +nick allen 65641 +nick allen 65641 +nick brown 65724 +nick davidson 65601 +nick ellison 65691 +nick ellison 65691 +nick falkner 65583 +nick falkner 65583 +nick garcia 65695 +nick garcia 65695 +nick garcia 65695 +nick ichabod 65572 +nick ichabod 65572 +nick ichabod 65572 +nick johnson 65585 +nick johnson 65585 +nick laertes 65624 +nick miller 65757 +nick nixon 65650 +nick ovid 65719 +nick polk 65716 +nick quirinius 65588 +nick quirinius 65588 +nick robinson 65547 +nick robinson 65547 +nick steinbeck 65689 +nick thompson 65610 +nick underhill 65619 +nick van buren 65603 +nick xylophone 65644 +nick young 65654 +nick young 65654 +nick zipper 65757 +nick zipper 65757 +oscar allen 65644 +oscar brown 65614 +oscar carson 65537 +oscar carson 65537 +oscar carson 65537 +oscar carson 65537 +oscar carson 65537 +oscar davidson 65556 +oscar ellison 65630 +oscar ellison 65630 +oscar falkner 65692 +oscar garcia 65751 +oscar hernandez 65683 +oscar hernandez 65683 +oscar ichabod 65536 +oscar ichabod 65536 +oscar ichabod 65536 +oscar ichabod 65536 +oscar johnson 65645 +oscar johnson 65645 +oscar king 65541 +oscar king 65541 +oscar king 65541 +oscar laertes 65625 +oscar laertes 65625 +oscar laertes 65625 +oscar laertes 65625 +oscar nixon 65596 +oscar ovid 65536 +oscar ovid 65536 +oscar ovid 65536 +oscar polk 65541 +oscar polk 65541 +oscar quirinius 65541 +oscar quirinius 65541 +oscar quirinius 65541 +oscar quirinius 65541 +oscar robinson 65537 +oscar robinson 65537 +oscar robinson 65537 +oscar robinson 65537 +oscar steinbeck 65709 +oscar thompson 65542 +oscar thompson 65542 +oscar thompson 65542 +oscar thompson 65542 +oscar underhill 65626 +oscar van buren 65581 +oscar van buren 65581 +oscar van buren 65581 +oscar white 65552 +oscar white 65552 +oscar white 65552 +oscar white 65552 +oscar xylophone 65773 +oscar xylophone 65773 +oscar xylophone 65773 +oscar zipper 65568 +oscar zipper 65568 +oscar zipper 65568 +priscilla brown 65670 +priscilla brown 65670 +priscilla brown 65670 +priscilla carson 65658 +priscilla carson 65658 +priscilla carson 65658 +priscilla ichabod 65627 +priscilla ichabod 65627 +priscilla johnson 65543 +priscilla johnson 65543 +priscilla johnson 65543 +priscilla johnson 65543 +priscilla johnson 65543 +priscilla king 65646 +priscilla nixon 65564 +priscilla nixon 65564 +priscilla ovid 65541 +priscilla ovid 65541 +priscilla polk 65747 +priscilla quirinius 65672 +priscilla thompson 65654 +priscilla underhill 65715 +priscilla underhill 65715 +priscilla van buren 65607 +priscilla van buren 65607 +priscilla van buren 65607 +priscilla white 65652 +priscilla xylophone 65538 +priscilla xylophone 65538 +priscilla xylophone 65538 +priscilla young 65585 +priscilla young 65585 +priscilla zipper 65622 +priscilla zipper 65622 +quinn allen 65657 +quinn allen 65657 +quinn brown 65691 +quinn brown 65691 +quinn brown 65691 +quinn davidson 65549 +quinn davidson 65549 +quinn davidson 65549 +quinn davidson 65549 +quinn ellison 65705 +quinn ellison 65705 +quinn garcia 65568 +quinn garcia 65568 +quinn garcia 65568 +quinn garcia 65568 +quinn ichabod 65564 +quinn king 65558 +quinn king 65558 +quinn laertes 65542 +quinn laertes 65542 +quinn laertes 65542 +quinn nixon 65659 +quinn ovid 65699 +quinn quirinius 65747 +quinn robinson 65627 +quinn steinbeck 65578 +quinn steinbeck 65578 +quinn thompson 65643 +quinn thompson 65643 +quinn underhill 65549 +quinn underhill 65549 +quinn underhill 65549 +quinn van buren 65725 +quinn young 65647 +quinn zipper 65579 +quinn zipper 65579 +rachel allen 65661 +rachel allen 65661 +rachel brown 65586 +rachel brown 65586 +rachel brown 65586 +rachel brown 65586 +rachel brown 65586 +rachel carson 65677 +rachel carson 65677 +rachel davidson 65755 +rachel ellison 65761 +rachel falkner 65616 +rachel falkner 65616 +rachel falkner 65616 +rachel falkner 65616 +rachel johnson 65658 +rachel king 65604 +rachel king 65604 +rachel laertes 65562 +rachel laertes 65562 +rachel ovid 65721 +rachel ovid 65721 +rachel polk 65686 +rachel quirinius 65787 +rachel robinson 65544 +rachel robinson 65544 +rachel robinson 65544 +rachel thompson 65648 +rachel thompson 65648 +rachel thompson 65648 +rachel underhill 65667 +rachel white 65615 +rachel white 65615 +rachel young 65727 +rachel zipper 65757 +rachel zipper 65757 +sarah carson 65679 +sarah carson 65679 +sarah carson 65679 +sarah ellison 65611 +sarah falkner 65606 +sarah falkner 65606 +sarah garcia 65563 +sarah garcia 65563 +sarah garcia 65563 +sarah ichabod 65667 +sarah ichabod 65667 +sarah johnson 65659 +sarah johnson 65659 +sarah johnson 65659 +sarah johnson 65659 +sarah king 65650 +sarah king 65650 +sarah miller 65557 +sarah ovid 65550 +sarah robinson 65677 +sarah robinson 65677 +sarah steinbeck 65721 +sarah white 65622 +sarah white 65622 +sarah xylophone 65678 +sarah young 65595 +sarah zipper 65550 +tom brown 65593 +tom brown 65593 +tom carson 65539 +tom carson 65539 +tom carson 65539 +tom davidson 65780 +tom ellison 65578 +tom ellison 65578 +tom ellison 65578 +tom falkner 65574 +tom falkner 65574 +tom hernandez 65575 +tom hernandez 65575 +tom ichabod 65588 +tom johnson 65536 +tom johnson 65536 +tom king 65576 +tom laertes 65617 +tom laertes 65617 +tom miller 65594 +tom miller 65594 +tom miller 65594 +tom nixon 65672 +tom ovid 65628 +tom polk 65652 +tom polk 65652 +tom quirinius 65563 +tom quirinius 65563 +tom robinson 65626 +tom robinson 65626 +tom robinson 65626 +tom robinson 65626 +tom steinbeck 65666 +tom van buren 65621 +tom van buren 65621 +tom van buren 65621 +tom white 65548 +tom young 65544 +tom young 65544 +tom zipper 65789 +ulysses brown 65735 +ulysses carson 65602 +ulysses carson 65602 +ulysses carson 65602 +ulysses carson 65602 +ulysses davidson 65750 +ulysses ellison 65575 +ulysses garcia 65666 +ulysses hernandez 65651 +ulysses hernandez 65651 +ulysses hernandez 65651 +ulysses ichabod 65551 +ulysses ichabod 65551 +ulysses johnson 65776 +ulysses king 65649 +ulysses laertes 65691 +ulysses laertes 65691 +ulysses laertes 65691 +ulysses miller 65610 +ulysses miller 65610 +ulysses nixon 65603 +ulysses ovid 65656 +ulysses polk 65563 +ulysses polk 65563 +ulysses polk 65563 +ulysses polk 65563 +ulysses quirinius 65786 +ulysses robinson 65744 +ulysses steinbeck 65611 +ulysses steinbeck 65611 +ulysses thompson 65788 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses van buren 65684 +ulysses white 65654 +ulysses white 65654 +ulysses xylophone 65623 +ulysses xylophone 65623 +ulysses xylophone 65623 +ulysses young 65675 +ulysses young 65675 +ulysses young 65675 +victor allen 65684 +victor allen 65684 +victor brown 65550 +victor brown 65550 +victor brown 65550 +victor brown 65550 +victor davidson 65579 +victor davidson 65579 +victor davidson 65579 +victor ellison 65641 +victor ellison 65641 +victor hernandez 65571 +victor hernandez 65571 +victor hernandez 65571 +victor hernandez 65571 +victor hernandez 65571 +victor johnson 65606 +victor johnson 65606 +victor johnson 65606 +victor king 65721 +victor king 65721 +victor laertes 65638 +victor laertes 65638 +victor miller 65570 +victor nixon 65709 +victor nixon 65709 +victor ovid 65649 +victor polk 65625 +victor quirinius 65620 +victor quirinius 65620 +victor robinson 65596 +victor robinson 65596 +victor steinbeck 65618 +victor steinbeck 65618 +victor steinbeck 65618 +victor thompson 65548 +victor van buren 65664 +victor van buren 65664 +victor white 65548 +victor white 65548 +victor xylophone 65549 +victor xylophone 65549 +victor xylophone 65549 +victor xylophone 65549 +victor xylophone 65549 +victor young 65628 +victor zipper 65743 +wendy allen 65628 +wendy allen 65628 +wendy allen 65628 +wendy brown 65580 +wendy brown 65580 +wendy ellison 65545 +wendy ellison 65545 +wendy falkner 65595 +wendy falkner 65595 +wendy falkner 65595 +wendy garcia 65659 +wendy garcia 65659 +wendy garcia 65659 +wendy garcia 65659 +wendy hernandez 65650 +wendy ichabod 65730 +wendy king 65586 +wendy king 65586 +wendy king 65586 +wendy laertes 65566 +wendy laertes 65566 +wendy laertes 65566 +wendy miller 65582 +wendy miller 65582 +wendy nixon 65611 +wendy nixon 65611 +wendy ovid 65589 +wendy ovid 65589 +wendy polk 65656 +wendy polk 65656 +wendy quirinius 65766 +wendy quirinius 65766 +wendy robinson 65622 +wendy robinson 65622 +wendy robinson 65622 +wendy steinbeck 65612 +wendy thompson 65650 +wendy thompson 65650 +wendy underhill 65662 +wendy underhill 65662 +wendy underhill 65662 +wendy van buren 65680 +wendy van buren 65680 +wendy white 65705 +wendy xylophone 65687 +wendy xylophone 65687 +wendy young 65674 +wendy young 65674 +xavier allen 65611 +xavier allen 65611 +xavier allen 65611 +xavier brown 65600 +xavier brown 65600 +xavier brown 65600 +xavier carson 65731 +xavier carson 65731 +xavier davidson 65644 +xavier davidson 65644 +xavier davidson 65644 +xavier ellison 65541 +xavier ellison 65541 +xavier garcia 65672 +xavier hernandez 65541 +xavier hernandez 65541 +xavier hernandez 65541 +xavier ichabod 65597 +xavier ichabod 65597 +xavier johnson 65654 +xavier johnson 65654 +xavier king 65590 +xavier king 65590 +xavier laertes 65743 +xavier ovid 65788 +xavier polk 65587 +xavier polk 65587 +xavier polk 65587 +xavier polk 65587 +xavier quirinius 65599 +xavier quirinius 65599 +xavier quirinius 65599 +xavier quirinius 65599 +xavier thompson 65608 +xavier underhill 65710 +xavier white 65703 +xavier white 65703 +xavier xylophone 65572 +xavier zipper 65561 +yuri allen 65565 +yuri allen 65565 +yuri brown 65538 +yuri brown 65538 +yuri carson 65670 +yuri carson 65670 +yuri ellison 65570 +yuri ellison 65570 +yuri falkner 65658 +yuri falkner 65658 +yuri garcia 65639 +yuri hernandez 65706 +yuri johnson 65587 +yuri johnson 65587 +yuri johnson 65587 +yuri king 65721 +yuri laertes 65637 +yuri laertes 65637 +yuri nixon 65635 +yuri nixon 65635 +yuri polk 65607 +yuri polk 65607 +yuri polk 65607 +yuri quirinius 65544 +yuri quirinius 65544 +yuri quirinius 65544 +yuri steinbeck 65592 +yuri steinbeck 65592 +yuri thompson 65676 +yuri underhill 65718 +yuri underhill 65718 +yuri white 65659 +yuri xylophone 65714 +zach allen 65667 +zach brown 65559 +zach brown 65559 +zach brown 65559 +zach brown 65559 +zach brown 65559 +zach carson 65572 +zach ellison 65748 +zach falkner 65620 +zach falkner 65620 +zach garcia 65544 +zach garcia 65544 +zach garcia 65544 +zach garcia 65544 +zach ichabod 65599 +zach ichabod 65599 +zach king 65556 +zach king 65556 +zach king 65556 +zach miller 65584 +zach miller 65584 +zach miller 65584 +zach ovid 65578 +zach ovid 65578 +zach ovid 65578 +zach ovid 65578 +zach quirinius 65691 +zach robinson 65599 +zach steinbeck 65602 +zach steinbeck 65602 +zach thompson 65636 +zach thompson 65636 +zach underhill 65573 +zach white 65733 +zach xylophone 65542 +zach xylophone 65542 +zach young 65576 +zach zipper 65579 +zach zipper 65579 +zach zipper 65579 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-1-2bf20f39e6ffef258858f7943a974e7e b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-1-2bf20f39e6ffef258858f7943a974e7e new file mode 100644 index 0000000000000..657e81a94f4c5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-1-2bf20f39e6ffef258858f7943a974e7e @@ -0,0 +1,1049 @@ + 9.220000267028809 + 43.72999954223633 + 89.52999877929688 +alice allen 2.7899999618530273 +alice allen 21.450000762939453 +alice allen 73.62999725341797 +alice brown 71.30999755859375 +alice carson 39.029998779296875 +alice davidson 70.3499984741211 +alice falkner 90.25 +alice garcia 48.45000076293945 +alice hernandez 88.16999816894531 +alice hernandez 90.55999755859375 +alice johnson 47.359999895095825 +alice king 19.139999389648438 +alice king 23.170000076293945 +alice king 52.22999954223633 +alice laertes 68.94999694824219 +alice laertes 69.52999877929688 +alice miller 68.95999908447266 +alice nixon 40.0 +alice nixon 48.150001525878906 +alice nixon 79.83000183105469 +alice ovid 9.039999961853027 +alice polk 62.900001525878906 +alice quirinius 37.13999938964844 +alice quirinius 62.29999923706055 +alice robinson 3.934999942779541 +alice robinson 56.099998474121094 +alice steinbeck 38.619998931884766 +alice steinbeck 63.40999794006348 +alice steinbeck 92.37000274658203 +alice underhill 93.17499923706055 +alice van buren 38.939998626708984 +alice xylophone 13.816667238871256 +alice xylophone 43.15999984741211 +alice xylophone 78.20999908447266 +alice zipper 26.43000030517578 +alice zipper 42.47999954223633 +alice zipper 89.93000030517578 +bob brown 8.069999694824219 +bob brown 70.93000030517578 +bob brown 93.08999633789062 +bob carson 50.09000015258789 +bob davidson 1.2899999618530273 +bob davidson 71.93000030517578 +bob davidson 74.72000122070312 +bob ellison 41.34000015258789 +bob ellison 56.584999084472656 +bob ellison 75.02999877929688 +bob ellison 80.30000305175781 +bob falkner 16.989999771118164 +bob garcia 4.460000038146973 +bob garcia 5.400000095367432 +bob garcia 43.46500015258789 +bob garcia 80.30000305175781 +bob garcia 87.56999969482422 +bob hernandez 55.51333363850912 +bob ichabod 82.55999755859375 +bob king 8.789999961853027 +bob king 12.539999961853027 +bob king 39.0099983215332 +bob laertes 0.7900000214576721 +bob laertes 10.670000076293945 +bob miller 61.91999816894531 +bob ovid 46.86000061035156 +bob ovid 62.849998474121094 +bob ovid 88.77999877929688 +bob ovid 97.08999633789062 +bob polk 7.980000019073486 +bob quirinius 48.09499931335449 +bob steinbeck 9.699999809265137 +bob van buren 33.66999816894531 +bob white 45.34000015258789 +bob white 45.349998474121094 +bob xylophone 27.6299991607666 +bob xylophone 29.359999656677246 +bob young 35.16999816894531 +bob zipper 3.819999933242798 +bob zipper 32.07500076293945 +bob zipper 52.63999938964844 +calvin allen 63.119998931884766 +calvin brown 28.110000610351562 +calvin brown 85.9000015258789 +calvin brown 90.19999694824219 +calvin carson 59.42500114440918 +calvin davidson 20.40500020980835 +calvin davidson 86.54000091552734 +calvin ellison 26.489999771118164 +calvin falkner 2.9700000286102295 +calvin falkner 56.040000915527344 +calvin falkner 56.33000183105469 +calvin falkner 80.5999984741211 +calvin falkner 93.61000061035156 +calvin falkner 94.30999755859375 +calvin garcia 41.849998474121094 +calvin hernandez 33.869998931884766 +calvin johnson 66.61000061035156 +calvin laertes 23.1299991607666 +calvin laertes 62.670000076293945 +calvin nixon 9.8100004196167 +calvin nixon 41.20000076293945 +calvin nixon 69.73999786376953 +calvin ovid 69.95999908447266 +calvin ovid 71.26000213623047 +calvin ovid 79.12000274658203 +calvin ovid 84.72000122070312 +calvin polk 65.72000122070312 +calvin quirinius 29.540000915527344 +calvin quirinius 60.98499870300293 +calvin robinson 40.439998626708984 +calvin steinbeck 15.220000267028809 +calvin steinbeck 22.850000381469727 +calvin steinbeck 52.06666819254557 +calvin thompson 8.90999984741211 +calvin thompson 89.22500228881836 +calvin underhill 59.70000076293945 +calvin van buren 34.209999084472656 +calvin van buren 64.0 +calvin white 35.864999771118164 +calvin white 90.69000244140625 +calvin xylophone 21.700000762939453 +calvin xylophone 25.420000076293945 +calvin xylophone 56.810001373291016 +calvin young 39.810001373291016 +calvin young 70.27999941507976 +calvin zipper 9.1899995803833 +calvin zipper 95.37999725341797 +david allen 51.005001068115234 +david allen 51.25 +david brown 32.56499910354614 +david brown 93.63999938964844 +david davidson 1.0800000429153442 +david davidson 54.17499876022339 +david davidson 62.720001220703125 +david davidson 74.1500015258789 +david ellison 47.689998626708984 +david ellison 62.587501525878906 +david ellison 85.2300033569336 +david hernandez 75.0833346048991 +david ichabod 35.763334115346275 +david ichabod 82.55000305175781 +david laertes 76.70999908447266 +david nixon 34.72999954223633 +david ovid 43.915000915527344 +david ovid 58.89999961853027 +david quirinius 23.5649995803833 +david quirinius 29.239999771118164 +david quirinius 79.97000122070312 +david robinson 37.703334172566734 +david robinson 70.55999755859375 +david thompson 43.619998931884766 +david underhill 49.719998359680176 +david underhill 60.36499881744385 +david underhill 60.46666717529297 +david van buren 25.500000476837158 +david van buren 50.27999973297119 +david white 45.189998626708984 +david xylophone 30.465000867843628 +david xylophone 30.465000867843628 +david xylophone 54.34000015258789 +david young 10.25 +david young 19.310000777244568 +ethan allen 23.790000518163044 +ethan brown 15.630000114440918 +ethan brown 22.93666648864746 +ethan brown 27.78000044822693 +ethan brown 52.19333346684774 +ethan brown 73.18000030517578 +ethan brown 82.30000305175781 +ethan carson 57.635000228881836 +ethan ellison 0.2800000011920929 +ethan ellison 81.47000122070312 +ethan falkner 50.02000045776367 +ethan falkner 59.43000030517578 +ethan garcia 26.44499921798706 +ethan hernandez 32.30333264668783 +ethan johnson 90.05000305175781 +ethan king 4.349999904632568 +ethan laertes 54.75 +ethan laertes 54.87999868392944 +ethan laertes 59.209999084472656 +ethan laertes 75.31500053405762 +ethan laertes 76.94499969482422 +ethan laertes 80.4749984741211 +ethan laertes 95.06999969482422 +ethan miller 25.3700008392334 +ethan nixon 50.88999938964844 +ethan ovid 58.4950008392334 +ethan polk 2.3499999046325684 +ethan polk 21.31999969482422 +ethan polk 23.440000534057617 +ethan polk 59.869998931884766 +ethan quirinius 13.650000214576721 +ethan quirinius 70.94500160217285 +ethan quirinius 88.18000030517578 +ethan robinson 67.94000244140625 +ethan robinson 80.58500289916992 +ethan underhill 55.630001068115234 +ethan van buren 37.85499954223633 +ethan white 58.70666631062826 +ethan white 63.41999816894531 +ethan xylophone 57.11000061035156 +ethan zipper 5.914999961853027 +ethan zipper 97.51000213623047 +fred davidson 26.435000896453857 +fred davidson 28.144000816345216 +fred davidson 78.30999755859375 +fred ellison 46.65999984741211 +fred ellison 65.40666516621907 +fred ellison 71.98499870300293 +fred falkner 25.144999980926514 +fred falkner 37.62000068028768 +fred falkner 75.125 +fred hernandez 55.9900016784668 +fred ichabod 37.06999969482422 +fred ichabod 75.83499908447266 +fred johnson 96.08999633789062 +fred king 10.220000267028809 +fred king 68.40500068664551 +fred laertes 42.68750047683716 +fred miller 70.02999877929688 +fred nixon 30.589999198913574 +fred nixon 32.71666653951009 +fred nixon 70.5199966430664 +fred nixon 93.02999877929688 +fred polk 23.959999084472656 +fred polk 47.31999969482422 +fred polk 63.97999954223633 +fred polk 90.12000274658203 +fred quirinius 15.300000190734863 +fred quirinius 27.40999984741211 +fred robinson 64.42000007629395 +fred steinbeck 21.239999771118164 +fred steinbeck 21.30000066757202 +fred steinbeck 65.44333394368489 +fred underhill 85.36999893188477 +fred van buren 22.37499976158142 +fred van buren 44.49000096321106 +fred van buren 45.94999980926514 +fred van buren 56.88999938964844 +fred white 42.329999923706055 +fred young 46.79999923706055 +fred young 72.69999885559082 +fred zipper 50.14000129699707 +gabriella allen 28.040000438690186 +gabriella allen 79.64500045776367 +gabriella brown 26.164999961853027 +gabriella brown 84.83000183105469 +gabriella carson 42.7599983215332 +gabriella davidson 18.946666717529297 +gabriella ellison 48.08000183105469 +gabriella ellison 71.54000091552734 +gabriella falkner 31.609999974568684 +gabriella falkner 51.720001220703125 +gabriella falkner 87.61000061035156 +gabriella garcia 43.0099983215332 +gabriella hernandez 62.22666517893473 +gabriella hernandez 92.9800033569336 +gabriella ichabod 10.729999542236328 +gabriella ichabod 26.639999389648438 +gabriella ichabod 66.36000061035156 +gabriella ichabod 71.12999725341797 +gabriella ichabod 83.76666514078777 +gabriella king 20.670000076293945 +gabriella king 62.89999961853027 +gabriella laertes 62.62499809265137 +gabriella miller 26.043334086736042 +gabriella ovid 44.78000068664551 +gabriella ovid 92.4000015258789 +gabriella polk 35.68000030517578 +gabriella polk 90.22500228881836 +gabriella steinbeck 46.45000076293945 +gabriella steinbeck 66.86999893188477 +gabriella thompson 72.34500122070312 +gabriella thompson 75.73666636149089 +gabriella thompson 94.25 +gabriella van buren 38.935001373291016 +gabriella van buren 48.349998474121094 +gabriella white 55.18000030517578 +gabriella young 32.16333262125651 +gabriella young 59.709999084472656 +gabriella zipper 36.2599983215332 +gabriella zipper 91.62999725341797 +holly allen 63.435001373291016 +holly brown 68.51666641235352 +holly brown 86.08666737874348 +holly falkner 83.82666524251302 +holly hernandez 21.190000534057617 +holly hernandez 24.790000915527344 +holly hernandez 28.649999300638836 +holly hernandez 50.22999954223633 +holly ichabod 48.86333433787028 +holly ichabod 73.77000045776367 +holly ichabod 82.91499710083008 +holly johnson 23.447500228881836 +holly johnson 64.36000061035156 +holly johnson 65.62000274658203 +holly king 35.34499979019165 +holly king 42.310001373291016 +holly laertes 70.59666697184245 +holly miller 35.86000061035156 +holly nixon 43.82499885559082 +holly nixon 78.80749893188477 +holly polk 30.365000247955322 +holly polk 75.96499919891357 +holly robinson 82.70499801635742 +holly thompson 0.07999999821186066 +holly thompson 65.52499842643738 +holly thompson 86.69000244140625 +holly underhill 42.54999923706055 +holly underhill 53.02000045776367 +holly underhill 56.13333384195963 +holly underhill 65.84000015258789 +holly van buren 48.893333435058594 +holly white 26.5633331934611 +holly white 40.5 +holly xylophone 49.55666716893514 +holly young 41.698571750095915 +holly young 75.20999908447266 +holly zipper 79.72999827067058 +holly zipper 81.08666610717773 +irene allen 29.75999927520752 +irene brown 28.596666653951008 +irene brown 47.189998626708984 +irene brown 49.46666622161865 +irene carson 86.64999898274739 +irene ellison 38.255001068115234 +irene ellison 45.71333376566569 +irene falkner 22.079999923706055 +irene falkner 83.44666544596355 +irene garcia 38.93499994277954 +irene garcia 42.84666601816813 +irene garcia 58.43000030517578 +irene ichabod 60.7079984664917 +irene ichabod 64.58000183105469 +irene johnson 26.165000438690186 +irene laertes 21.02999997138977 +irene laertes 40.04499816894531 +irene laertes 47.04333241780599 +irene miller 65.44000244140625 +irene nixon 46.03999996185303 +irene nixon 46.96666653951009 +irene nixon 67.09499931335449 +irene ovid 35.130001068115234 +irene ovid 42.535000801086426 +irene ovid 79.75 +irene polk 0.9800000190734863 +irene polk 35.17500114440918 +irene polk 48.94666576385498 +irene polk 49.6 +irene polk 51.885000228881836 +irene quirinius 33.78000005086263 +irene quirinius 42.610000133514404 +irene quirinius 53.4800001780192 +irene robinson 92.19499969482422 +irene steinbeck 73.28000068664551 +irene thompson 46.27375066280365 +irene underhill 24.75 +irene underhill 57.349998474121094 +irene van buren 50.8799991607666 +irene van buren 74.5625 +irene xylophone 83.53499984741211 +jessica brown 26.185000479221344 +jessica carson 56.22999954223633 +jessica carson 62.20000076293945 +jessica carson 62.2400016784668 +jessica davidson 50.01666768391927 +jessica davidson 63.59499931335449 +jessica davidson 69.26666514078777 +jessica davidson 94.53333282470703 +jessica ellison 11.180000305175781 +jessica ellison 64.2060001373291 +jessica falkner 61.57333437601725 +jessica garcia 38.55250036716461 +jessica garcia 57.00999975204468 +jessica ichabod 32.63250035047531 +jessica johnson 9.5600004196167 +jessica johnson 51.959999084472656 +jessica miller 77.83999633789062 +jessica nixon 40.72249960899353 +jessica nixon 90.06999969482422 +jessica ovid 37.45250064134598 +jessica ovid 59.68000030517578 +jessica polk 49.68000030517578 +jessica quirinius 25.65750002861023 +jessica quirinius 37.64200019836426 +jessica quirinius 54.25500011444092 +jessica quirinius 58.019999186197914 +jessica robinson 42.66333484649658 +jessica thompson 30.40666675567627 +jessica thompson 43.87500023841858 +jessica underhill 43.33333269755045 +jessica underhill 45.639999866485596 +jessica underhill 57.584999084472656 +jessica van buren 67.00000047683716 +jessica white 6.170000106096268 +jessica white 63.32500076293945 +jessica white 65.1450023651123 +jessica white 73.93000030517578 +jessica white 96.62000274658203 +jessica xylophone 69.87500190734863 +jessica young 11.1899995803833 +jessica young 43.369998931884766 +jessica zipper 42.43833335240682 +jessica zipper 46.7450008392334 +jessica zipper 56.97999954223633 +katie allen 55.47666549682617 +katie brown 31.699999809265137 +katie davidson 93.22000122070312 +katie ellison 48.31999933719635 +katie ellison 64.08499892552693 +katie falkner 51.665000915527344 +katie garcia 57.71000099182129 +katie garcia 61.21000051498413 +katie hernandez 41.150000381469724 +katie ichabod 44.243333180745445 +katie ichabod 51.800000508626304 +katie ichabod 69.18799896240235 +katie king 39.83000183105469 +katie king 46.80333296457926 +katie king 51.85000038146973 +katie miller 31.399999618530273 +katie miller 74.77999877929688 +katie nixon 23.190000534057617 +katie ovid 67.94500160217285 +katie polk 26.62750005722046 +katie polk 33.9350004196167 +katie robinson 13.890000343322754 +katie van buren 44.434998512268066 +katie van buren 65.41999816894531 +katie white 37.96500015258789 +katie white 59.223333517710365 +katie xylophone 39.30000019073486 +katie young 36.660000801086426 +katie young 67.78333282470703 +katie young 72.76666577657063 +katie zipper 23.766667087872822 +katie zipper 58.75 +luke allen 50.959999084472656 +luke allen 53.36666742960612 +luke allen 54.63249969482422 +luke allen 57.670000076293945 +luke allen 70.39500045776367 +luke brown 49.595000982284546 +luke davidson 7.050000190734863 +luke davidson 18.87000036239624 +luke ellison 16.25 +luke ellison 32.9519996881485 +luke ellison 71.93500137329102 +luke falkner 21.71999979019165 +luke falkner 31.81250023841858 +luke garcia 18.65499973297119 +luke garcia 41.2300001780192 +luke ichabod 41.25750005245209 +luke ichabod 73.55000114440918 +luke johnson 31.670000076293945 +luke johnson 32.84499979019165 +luke johnson 39.54500102996826 +luke laertes 11.819999694824219 +luke laertes 21.184999227523804 +luke laertes 21.993332862854004 +luke laertes 26.696666717529297 +luke laertes 45.9900016784668 +luke miller 52.350000858306885 +luke ovid 23.804999828338623 +luke ovid 64.30000305175781 +luke polk 41.02499961853027 +luke polk 58.4566650390625 +luke quirinius 40.41999816894531 +luke robinson 48.559998750686646 +luke robinson 56.76499938964844 +luke thompson 78.04333368937175 +luke underhill 34.0166662534078 +luke underhill 47.28999996185303 +luke underhill 59.32000160217285 +luke van buren 59.91999944051107 +luke white 74.19599990844726 +luke xylophone 64.77999925613403 +luke zipper 30.434999465942383 +mike allen 30.539999961853027 +mike brown 69.86833318074544 +mike carson 30.25333309173584 +mike carson 61.33799934387207 +mike carson 89.375 +mike davidson 32.55333391825358 +mike davidson 66.74333318074544 +mike ellison 35.905999755859376 +mike ellison 39.82499885559082 +mike ellison 58.56399993896484 +mike ellison 64.52999877929688 +mike ellison 66.93749856948853 +mike falkner 48.53750002384186 +mike garcia 51.02999973297119 +mike garcia 67.93000030517578 +mike garcia 70.8499984741211 +mike hernandez 37.900001525878906 +mike hernandez 59.45000076293945 +mike ichabod 64.7699966430664 +mike king 36.17800045013428 +mike king 41.69500136375427 +mike king 49.57000017166138 +mike king 59.654998779296875 +mike king 71.57000122070312 +mike king 78.50999927520752 +mike miller 29.570000171661377 +mike nixon 45.029999828338624 +mike nixon 48.429999669392906 +mike polk 30.864000129699708 +mike polk 46.95499873161316 +mike polk 79.55500030517578 +mike quirinius 85.0699971516927 +mike steinbeck 24.267500042915344 +mike steinbeck 43.52500021457672 +mike steinbeck 61.426666577657066 +mike steinbeck 68.46000022888184 +mike van buren 27.639999389648438 +mike van buren 56.16333134969076 +mike white 34.8924994468689 +mike white 43.5566660563151 +mike white 53.689998626708984 +mike white 77.54499864578247 +mike young 34.3319993019104 +mike young 52.8100004196167 +mike young 55.64333359400431 +mike zipper 56.86666742960612 +mike zipper 63.3149995803833 +mike zipper 83.91999816894531 +nick allen 57.086001586914065 +nick allen 60.15400094985962 +nick brown 42.939998626708984 +nick davidson 63.07499885559082 +nick ellison 45.34000015258789 +nick ellison 65.88500022888184 +nick falkner 41.87999868392944 +nick falkner 64.05666732788086 +nick garcia 34.34499979019165 +nick garcia 51.08666737874349 +nick garcia 62.88600044250488 +nick ichabod 20.253333409627277 +nick ichabod 53.635000228881836 +nick ichabod 77.36000061035156 +nick johnson 20.114999175071716 +nick johnson 81.91666666666667 +nick laertes 91.56666819254558 +nick miller 71.5500005086263 +nick nixon 77.04249954223633 +nick ovid 74.62666702270508 +nick polk 39.27500009536743 +nick quirinius 60.79499816894531 +nick quirinius 67.44999694824219 +nick robinson 31.672499418258667 +nick robinson 57.66999816894531 +nick steinbeck 59.15999984741211 +nick thompson 18.88666645685832 +nick underhill 43.009998893737794 +nick van buren 34.720000902811684 +nick xylophone 75.3499984741211 +nick young 0.27000001072883606 +nick young 47.813334147135414 +nick zipper 46.22333272298177 +nick zipper 52.54333209991455 +oscar allen 37.396666844685875 +oscar brown 13.100000381469727 +oscar carson 31.91333230336507 +oscar carson 41.77333331108093 +oscar carson 57.3149995803833 +oscar carson 73.59500122070312 +oscar carson 95.44000244140625 +oscar davidson 75.18000030517578 +oscar ellison 34.04499959945679 +oscar ellison 34.04499959945679 +oscar falkner 61.72000050544739 +oscar garcia 67.4800033569336 +oscar hernandez 41.63333400090536 +oscar hernandez 47.93999986648559 +oscar ichabod 45.839999516805015 +oscar ichabod 68.62000274658203 +oscar ichabod 72.18249797821045 +oscar ichabod 76.69000244140625 +oscar johnson 23.880000114440918 +oscar johnson 65.04000091552734 +oscar king 36.69500017166138 +oscar king 49.7049994468689 +oscar king 67.98399925231934 +oscar laertes 43.616665522257485 +oscar laertes 44.755000591278076 +oscar laertes 45.26666831970215 +oscar laertes 53.710001945495605 +oscar nixon 36.56999937693278 +oscar ovid 45.89999961853027 +oscar ovid 46.93999934196472 +oscar ovid 55.277999591827395 +oscar polk 42.31999969482422 +oscar polk 63.900001525878906 +oscar quirinius 63.81500053405762 +oscar quirinius 66.28428527287075 +oscar quirinius 70.24000295003255 +oscar quirinius 81.26249980926514 +oscar robinson 11.34000015258789 +oscar robinson 47.845001220703125 +oscar robinson 59.74333349863688 +oscar robinson 63.346666971842446 +oscar steinbeck 42.49999976158142 +oscar thompson 38.23500061035156 +oscar thompson 51.469999154408775 +oscar thompson 60.029999542236325 +oscar thompson 63.079999923706055 +oscar underhill 66.97666676839192 +oscar van buren 24.085000872612 +oscar van buren 61.880001068115234 +oscar van buren 72.9533322652181 +oscar white 44.72333272298177 +oscar white 46.60999870300293 +oscar white 54.7599983215332 +oscar white 60.85500144958496 +oscar xylophone 34.946666399637856 +oscar xylophone 39.8299994468689 +oscar xylophone 57.119998931884766 +oscar zipper 28.499999046325684 +oscar zipper 47.46750068664551 +oscar zipper 59.1933339436849 +priscilla brown 47.40400066375732 +priscilla brown 77.1479995727539 +priscilla brown 80.5199966430664 +priscilla carson 7.960000038146973 +priscilla carson 28.480000153183937 +priscilla carson 45.92750144004822 +priscilla ichabod 38.95666758219401 +priscilla ichabod 62.32999928792318 +priscilla johnson 44.04499912261963 +priscilla johnson 50.53750038146973 +priscilla johnson 55.98333485921224 +priscilla johnson 59.64499855041504 +priscilla johnson 89.1500015258789 +priscilla king 50.44666735331217 +priscilla nixon 44.32222270965576 +priscilla nixon 45.267999792099 +priscilla ovid 44.78333361943563 +priscilla ovid 52.72999954223633 +priscilla polk 34.89399948120117 +priscilla quirinius 35.609999895095825 +priscilla thompson 35.16249918937683 +priscilla underhill 68.22000122070312 +priscilla underhill 73.97200012207031 +priscilla van buren 50.47000026702881 +priscilla van buren 51.39500045776367 +priscilla van buren 53.541999435424806 +priscilla white 50.47599992752075 +priscilla xylophone 0.15000000596046448 +priscilla xylophone 41.106666485468544 +priscilla xylophone 63.9574990272522 +priscilla young 0.2900000065565109 +priscilla young 19.866666316986084 +priscilla zipper 32.084999322891235 +priscilla zipper 43.90333366394043 +quinn allen 47.90333382288615 +quinn allen 83.33000183105469 +quinn brown 24.280000686645508 +quinn brown 53.98666508992513 +quinn brown 66.82500171661377 +quinn davidson 40.666666666666664 +quinn davidson 54.095001220703125 +quinn davidson 79.78333282470703 +quinn davidson 92.13000106811523 +quinn ellison 52.714999198913574 +quinn ellison 63.352500438690186 +quinn garcia 20.19000056385994 +quinn garcia 54.60000038146973 +quinn garcia 59.010000824928284 +quinn garcia 68.98999881744385 +quinn ichabod 48.60499930381775 +quinn king 61.27333450317383 +quinn king 81.46000289916992 +quinn laertes 32.08000040054321 +quinn laertes 44.45666694641113 +quinn laertes 49.85499858856201 +quinn nixon 72.2471422467913 +quinn ovid 34.423333168029785 +quinn quirinius 53.165000915527344 +quinn robinson 32.624999046325684 +quinn steinbeck 24.802499771118164 +quinn steinbeck 55.477500915527344 +quinn thompson 50.500000381469725 +quinn thompson 55.68600006103516 +quinn underhill 39.66600060462952 +quinn underhill 41.47666676839193 +quinn underhill 56.580000162124634 +quinn van buren 49.40333207448324 +quinn young 55.59000142415365 +quinn zipper 11.359999974568685 +quinn zipper 48.45000123977661 +rachel allen 45.940001249313354 +rachel allen 85.97999954223633 +rachel brown 33.01999984184901 +rachel brown 34.08250021934509 +rachel brown 37.999999046325684 +rachel brown 41.75000019868215 +rachel brown 53.679999669392906 +rachel carson 43.32400016784668 +rachel carson 66.2233320871989 +rachel davidson 14.220000267028809 +rachel ellison 17.549999833106995 +rachel falkner 56.883334159851074 +rachel falkner 57.5199998219808 +rachel falkner 58.80666637420654 +rachel falkner 70.69428443908691 +rachel johnson 36.22499990463257 +rachel king 50.970001220703125 +rachel king 83.53750133514404 +rachel laertes 42.29857151848929 +rachel laertes 71.65999984741211 +rachel ovid 42.25333329041799 +rachel ovid 47.01749947667122 +rachel polk 64.90333239237468 +rachel quirinius 53.2624990940094 +rachel robinson 40.712857246398926 +rachel robinson 53.092498898506165 +rachel robinson 64.94999694824219 +rachel thompson 24.555000439286232 +rachel thompson 31.460000038146973 +rachel thompson 46.804000282287596 +rachel underhill 47.22333272298177 +rachel white 39.87999979654948 +rachel white 41.83428575311388 +rachel young 75.7966677347819 +rachel zipper 45.794999519983925 +rachel zipper 56.909999179840085 +sarah carson 24.576666196187336 +sarah carson 36.33750060200691 +sarah carson 43.65749907493591 +sarah ellison 37.054999351501465 +sarah falkner 48.58285754067557 +sarah falkner 62.36500072479248 +sarah garcia 33.38000011444092 +sarah garcia 35.513333002726235 +sarah garcia 64.31333287556966 +sarah ichabod 36.10599975585937 +sarah ichabod 45.830000162124634 +sarah johnson 26.464999675750732 +sarah johnson 40.9300012588501 +sarah johnson 43.44000196456909 +sarah johnson 64.24333318074544 +sarah king 49.06999909877777 +sarah king 63.01333363850912 +sarah miller 41.709999084472656 +sarah ovid 63.682499408721924 +sarah robinson 39.196666399637856 +sarah robinson 66.88999938964844 +sarah steinbeck 66.89000034332275 +sarah white 41.42599945068359 +sarah white 52.95249938964844 +sarah xylophone 68.31999969482422 +sarah young 35.92750024795532 +sarah zipper 53.697500705718994 +tom brown 38.37000020345052 +tom brown 44.68000049591065 +tom carson 27.994999766349792 +tom carson 54.25250005722046 +tom carson 62.790000915527344 +tom davidson 38.679999113082886 +tom ellison 33.68600053787232 +tom ellison 46.00666618347168 +tom ellison 67.79666646321614 +tom falkner 55.61800079345703 +tom falkner 58.82500012715658 +tom hernandez 50.52250051498413 +tom hernandez 50.52250051498413 +tom ichabod 24.98399963378906 +tom johnson 34.83750009536743 +tom johnson 73.72399978637695 +tom king 69.98000106811523 +tom laertes 41.97285750934056 +tom laertes 70.40333429972331 +tom miller 43.885000586509705 +tom miller 57.10500144958496 +tom miller 76.20499992370605 +tom nixon 62.43000030517578 +tom ovid 38.096666971842446 +tom polk 51.26750087738037 +tom polk 68.22666676839192 +tom quirinius 37.720001220703125 +tom quirinius 53.20399913787842 +tom robinson 43.44333299001058 +tom robinson 54.637142998831614 +tom robinson 59.34250068664551 +tom robinson 99.1500015258789 +tom steinbeck 51.883334477742515 +tom van buren 28.380000829696655 +tom van buren 35.64999930063883 +tom van buren 54.59000015258789 +tom white 51.970001220703125 +tom young 44.7319995880127 +tom young 53.894999980926514 +tom zipper 55.44000116984049 +ulysses brown 48.72666708628336 +ulysses carson 38.742000579833984 +ulysses carson 45.513333002726235 +ulysses carson 48.75249934196472 +ulysses carson 74.64600067138672 +ulysses davidson 63.20857129778181 +ulysses ellison 68.52666759490967 +ulysses garcia 58.77250051498413 +ulysses hernandez 32.371999168395995 +ulysses hernandez 50.57000102996826 +ulysses hernandez 61.39999961853027 +ulysses ichabod 19.1299991607666 +ulysses ichabod 83.06666692097981 +ulysses johnson 51.485000451405845 +ulysses king 46.98333422342936 +ulysses laertes 29.046666741371155 +ulysses laertes 32.88599967956543 +ulysses laertes 60.12399845123291 +ulysses miller 44.552857535226 +ulysses miller 71.39249873161316 +ulysses nixon 51.300000286102296 +ulysses ovid 29.360000610351562 +ulysses polk 40.74399948120117 +ulysses polk 48.9800017674764 +ulysses polk 57.86249828338623 +ulysses polk 81.21333312988281 +ulysses quirinius 68.41500091552734 +ulysses robinson 69.53999853134155 +ulysses steinbeck 44.61833381652832 +ulysses steinbeck 48.362499713897705 +ulysses thompson 45.063334465026855 +ulysses underhill 30.829999764760334 +ulysses underhill 41.43857192993164 +ulysses underhill 44.08333269755045 +ulysses underhill 55.470001220703125 +ulysses underhill 58.9471435546875 +ulysses underhill 68.1900007724762 +ulysses underhill 78.83333333333333 +ulysses van buren 72.38428633553642 +ulysses white 36.17250043153763 +ulysses white 39.084000778198245 +ulysses xylophone 27.519999504089355 +ulysses xylophone 47.65999937057495 +ulysses xylophone 50.29999923706055 +ulysses young 23.308333079020183 +ulysses young 34.6339994430542 +ulysses young 88.06999969482422 +victor allen 49.43800010681152 +victor allen 56.7299998147147 +victor brown 40.80600037574768 +victor brown 63.5024995803833 +victor brown 71.03500080108643 +victor brown 81.71999931335449 +victor davidson 44.70333290100098 +victor davidson 59.070000076293944 +victor davidson 67.27199935913086 +victor ellison 31.28999964396159 +victor ellison 42.54999923706055 +victor hernandez 44.41333452860514 +victor hernandez 47.20249938964844 +victor hernandez 47.73333215713501 +victor hernandez 51.04999923706055 +victor hernandez 59.2399995803833 +victor johnson 54.868000626564026 +victor johnson 55.22999954223633 +victor johnson 57.41000175476074 +victor king 38.27999997138977 +victor king 49.993333180745445 +victor laertes 40.63500006993612 +victor laertes 80.5999984741211 +victor miller 71.00000190734863 +victor nixon 38.393332640329994 +victor nixon 52.920000076293945 +victor ovid 53.260000228881836 +victor polk 3.0 +victor quirinius 57.81666644414266 +victor quirinius 59.39999923706055 +victor robinson 24.614999771118164 +victor robinson 74.5049991607666 +victor steinbeck 28.862000381946565 +victor steinbeck 36.61000006539481 +victor steinbeck 43.09000015258789 +victor thompson 42.67599925994873 +victor van buren 44.669999877611794 +victor van buren 45.121999740600586 +victor white 53.67999887466431 +victor white 54.45000012715658 +victor xylophone 12.160000324249268 +victor xylophone 26.0 +victor xylophone 31.769999821980793 +victor xylophone 52.31499926249186 +victor xylophone 69.2899996439616 +victor young 64.25833320617676 +victor zipper 65.24999904632568 +wendy allen 34.04999955495199 +wendy allen 36.88199939727783 +wendy allen 44.96000012755394 +wendy brown 45.97833283742269 +wendy brown 52.73857225690569 +wendy ellison 42.91333246231079 +wendy ellison 53.56000073750814 +wendy falkner 47.602500438690186 +wendy falkner 64.9099988937378 +wendy falkner 77.5999984741211 +wendy garcia 37.38571425846645 +wendy garcia 48.76666768391927 +wendy garcia 53.225000858306885 +wendy garcia 63.93999926249186 +wendy hernandez 36.195000648498535 +wendy ichabod 10.56499981880188 +wendy king 37.57500076293945 +wendy king 53.44333457946777 +wendy king 56.319997787475586 +wendy laertes 38.39249920845032 +wendy laertes 60.19999885559082 +wendy laertes 65.30624961853027 +wendy miller 44.273332595825195 +wendy miller 53.5675014257431 +wendy nixon 54.995998764038085 +wendy nixon 64.28250026702881 +wendy ovid 43.80499863624573 +wendy ovid 61.64600105285645 +wendy polk 26.784999758005142 +wendy polk 35.21599998474121 +wendy quirinius 28.75666618347168 +wendy quirinius 60.70000092188517 +wendy robinson 42.5799994468689 +wendy robinson 42.90799944400787 +wendy robinson 43.426000237464905 +wendy steinbeck 43.42333388328552 +wendy thompson 40.352857317243306 +wendy thompson 75.93666712443034 +wendy underhill 24.459999561309814 +wendy underhill 33.440000693003334 +wendy underhill 45.51625019311905 +wendy van buren 43.30333296457926 +wendy van buren 65.58666737874348 +wendy white 39.015000104904175 +wendy xylophone 42.42500034968058 +wendy xylophone 53.981666247049965 +wendy young 27.929999828338623 +wendy young 59.609999656677246 +xavier allen 49.24500061571598 +xavier allen 67.39000034332275 +xavier allen 70.29800033569336 +xavier brown 19.772500306367874 +xavier brown 58.87000111171177 +xavier brown 74.6200008392334 +xavier carson 47.3199987411499 +xavier carson 61.52250027656555 +xavier davidson 48.41999936103821 +xavier davidson 52.70666758219401 +xavier davidson 52.78166747093201 +xavier ellison 41.84999942779541 +xavier ellison 62.80200090408325 +xavier garcia 35.0600004568696 +xavier hernandez 47.45200023651123 +xavier hernandez 49.676666259765625 +xavier hernandez 53.446667432785034 +xavier ichabod 56.70625042915344 +xavier ichabod 60.54799928665161 +xavier johnson 41.16333262125651 +xavier johnson 53.85333331425985 +xavier king 42.72800064086914 +xavier king 66.05333455403645 +xavier laertes 38.47999954223633 +xavier ovid 48.89250057935715 +xavier polk 37.05500018596649 +xavier polk 46.82666703065237 +xavier polk 55.385000705718994 +xavier polk 55.65000057220459 +xavier quirinius 59.62499964237213 +xavier quirinius 60.055998992919925 +xavier quirinius 62.52000045776367 +xavier quirinius 65.2933349609375 +xavier thompson 40.244998931884766 +xavier underhill 26.27800006866455 +xavier white 47.8671429497855 +xavier white 63.38428551810129 +xavier xylophone 49.072500228881836 +xavier zipper 8.204999923706055 +yuri allen 53.61250066757202 +yuri allen 64.86833254496257 +yuri brown 46.57500044504801 +yuri brown 66.75250005722046 +yuri carson 39.40750050544739 +yuri carson 49.01600036621094 +yuri ellison 27.49000017642975 +yuri ellison 70.5933329264323 +yuri falkner 47.23285675048828 +yuri falkner 62.807999801635745 +yuri garcia 43.967499017715454 +yuri hernandez 31.94000039100647 +yuri johnson 21.40666739145915 +yuri johnson 34.02333414554596 +yuri johnson 65.7750015258789 +yuri king 49.47333272298177 +yuri laertes 42.070000648498535 +yuri laertes 60.7549991607666 +yuri nixon 49.87142838750567 +yuri nixon 59.945000330607094 +yuri polk 37.56249952316284 +yuri polk 47.583333333333336 +yuri polk 72.60888735453288 +yuri quirinius 18.62000060081482 +yuri quirinius 51.217501401901245 +yuri quirinius 67.24000072479248 +yuri steinbeck 55.757999420166016 +yuri steinbeck 75.87999725341797 +yuri thompson 36.93499946594238 +yuri underhill 51.533334732055664 +yuri underhill 62.31888887617323 +yuri white 44.34999983651297 +yuri xylophone 25.117499828338623 +zach allen 25.92333350578944 +zach brown 38.3799991607666 +zach brown 47.404998779296875 +zach brown 54.30600090026856 +zach brown 58.970001220703125 +zach brown 65.22499942779541 +zach carson 60.783999633789065 +zach ellison 36.211428437914165 +zach falkner 41.225714683532715 +zach falkner 65.99499940872192 +zach garcia 42.8885714326586 +zach garcia 46.8870005607605 +zach garcia 47.5049991607666 +zach garcia 66.09399967193603 +zach ichabod 40.10166613260905 +zach ichabod 53.16749954223633 +zach king 39.137500405311584 +zach king 48.2825003862381 +zach king 61.18999965985616 +zach miller 44.82800054550171 +zach miller 48.52428477151053 +zach miller 53.593332608540855 +zach ovid 35.19399921447039 +zach ovid 38.35833342870077 +zach ovid 43.87200012207031 +zach ovid 83.01999918619792 +zach quirinius 42.638333320617676 +zach robinson 82.04999923706055 +zach steinbeck 55.86599960327148 +zach steinbeck 67.81428473336356 +zach thompson 29.303333282470703 +zach thompson 46.48999913533529 +zach underhill 48.681429045540945 +zach white 66.60250091552734 +zach xylophone 41.875 +zach xylophone 57.2416664759318 +zach young 73.5999984741211 +zach zipper 58.1480016708374 +zach zipper 60.1825008392334 +zach zipper 62.794999837875366 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-2-16239d2b069789ba99fbac50c4f0724f b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-2-16239d2b069789ba99fbac50c4f0724f new file mode 100644 index 0000000000000..6cfa5ad413fa4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-2-16239d2b069789ba99fbac50c4f0724f @@ -0,0 +1,1049 @@ + 65560.0 + 65718.0 + 65740.0 +alice allen 65662.0 +alice allen 65720.0 +alice allen 65758.0 +alice brown 65696.0 +alice carson 65559.0 +alice davidson 65547.0 +alice falkner 65669.0 +alice garcia 65613.0 +alice hernandez 65737.0 +alice hernandez 65784.0 +alice johnson 65739.0 +alice king 65660.0 +alice king 65734.0 +alice king 65738.0 +alice laertes 65669.0 +alice laertes 65671.0 +alice miller 65590.0 +alice nixon 65586.0 +alice nixon 65595.0 +alice nixon 65604.0 +alice ovid 65737.0 +alice polk 65548.0 +alice quirinius 65636.0 +alice quirinius 65728.0 +alice robinson 65606.0 +alice robinson 65789.0 +alice steinbeck 65578.0 +alice steinbeck 65673.0 +alice steinbeck 65786.0 +alice underhill 65750.0 +alice van buren 65562.0 +alice xylophone 65578.0 +alice xylophone 65585.0 +alice xylophone 65599.0 +alice zipper 65553.0 +alice zipper 65662.0 +alice zipper 65766.0 +bob brown 65584.0 +bob brown 65777.0 +bob brown 65783.0 +bob carson 65713.0 +bob davidson 65664.0 +bob davidson 65693.0 +bob davidson 65768.0 +bob ellison 65591.0 +bob ellison 65624.0 +bob ellison 65721.0 +bob ellison 65760.0 +bob falkner 65789.0 +bob garcia 65585.0 +bob garcia 65598.0 +bob garcia 65673.0 +bob garcia 65754.0 +bob garcia 65782.0 +bob hernandez 65557.0 +bob ichabod 65549.0 +bob king 65715.0 +bob king 65757.0 +bob king 65783.0 +bob laertes 65602.0 +bob laertes 65663.0 +bob miller 65608.0 +bob ovid 65564.0 +bob ovid 65619.0 +bob ovid 65686.0 +bob ovid 65726.0 +bob polk 65594.0 +bob quirinius 65700.0 +bob steinbeck 65637.0 +bob van buren 65778.0 +bob white 65543.0 +bob white 65605.0 +bob xylophone 65574.0 +bob xylophone 65666.0 +bob young 65556.0 +bob zipper 65559.0 +bob zipper 65633.0 +bob zipper 65739.0 +calvin allen 65669.0 +calvin brown 65537.0 +calvin brown 65580.0 +calvin brown 65677.0 +calvin carson 65637.0 +calvin davidson 65541.0 +calvin davidson 65564.0 +calvin ellison 65667.0 +calvin falkner 65573.0 +calvin falkner 65596.0 +calvin falkner 65738.0 +calvin falkner 65762.0 +calvin falkner 65778.0 +calvin falkner 65784.0 +calvin garcia 65664.0 +calvin hernandez 65578.0 +calvin johnson 65731.0 +calvin laertes 65570.0 +calvin laertes 65684.0 +calvin nixon 65654.0 +calvin nixon 65724.0 +calvin nixon 65749.0 +calvin ovid 65554.0 +calvin ovid 65643.0 +calvin ovid 65663.0 +calvin ovid 65715.0 +calvin polk 65731.0 +calvin quirinius 65741.0 +calvin quirinius 65769.0 +calvin robinson 65581.0 +calvin steinbeck 65680.0 +calvin steinbeck 65762.0 +calvin steinbeck 65779.0 +calvin thompson 65560.0 +calvin thompson 65640.0 +calvin underhill 65732.0 +calvin van buren 65552.0 +calvin van buren 65771.0 +calvin white 65553.0 +calvin white 65561.0 +calvin xylophone 65575.0 +calvin xylophone 65614.5 +calvin xylophone 65713.0 +calvin young 65574.0 +calvin young 65746.0 +calvin zipper 65669.0 +calvin zipper 65739.0 +david allen 65588.0 +david allen 65617.0 +david brown 65637.0 +david brown 65760.0 +david davidson 65559.0 +david davidson 65756.0 +david davidson 65778.0 +david davidson 65779.0 +david ellison 65634.0 +david ellison 65724.0 +david ellison 65724.0 +david hernandez 65763.0 +david ichabod 65699.0 +david ichabod 65715.0 +david laertes 65762.0 +david nixon 65536.0 +david ovid 65623.0 +david ovid 65628.0 +david quirinius 65697.0 +david quirinius 65759.0 +david quirinius 65779.0 +david robinson 65762.0 +david robinson 65775.0 +david thompson 65550.0 +david underhill 65602.0 +david underhill 65662.0 +david underhill 65751.0 +david van buren 65625.0 +david van buren 65634.0 +david white 65678.0 +david xylophone 65537.0 +david xylophone 65670.0 +david xylophone 65764.0 +david young 65551.0 +david young 65694.0 +ethan allen 65747.0 +ethan brown 65539.0 +ethan brown 65617.0 +ethan brown 65685.0 +ethan brown 65685.0 +ethan brown 65722.0 +ethan brown 65733.0 +ethan carson 65742.0 +ethan ellison 65714.0 +ethan ellison 65732.0 +ethan falkner 65577.0 +ethan falkner 65614.0 +ethan garcia 65736.0 +ethan hernandez 65630.5 +ethan johnson 65536.0 +ethan king 65614.0 +ethan laertes 65562.0 +ethan laertes 65597.0 +ethan laertes 65628.0 +ethan laertes 65643.0 +ethan laertes 65680.0 +ethan laertes 65745.0 +ethan laertes 65760.0 +ethan miller 65712.0 +ethan nixon 65766.0 +ethan ovid 65697.0 +ethan polk 65589.0 +ethan polk 65615.0 +ethan polk 65622.0 +ethan polk 65622.0 +ethan quirinius 65542.0 +ethan quirinius 65591.0 +ethan quirinius 65706.0 +ethan robinson 65547.0 +ethan robinson 65659.0 +ethan underhill 65570.0 +ethan van buren 65572.0 +ethan white 65677.0 +ethan white 65788.0 +ethan xylophone 65595.0 +ethan zipper 65593.0 +ethan zipper 65680.0 +fred davidson 65552.0 +fred davidson 65595.0 +fred davidson 65721.0 +fred ellison 65548.0 +fred ellison 65691.0 +fred ellison 65771.0 +fred falkner 65637.0 +fred falkner 65648.0 +fred falkner 65783.0 +fred hernandez 65541.0 +fred ichabod 65572.0 +fred ichabod 65789.0 +fred johnson 65758.0 +fred king 65694.0 +fred king 65745.0 +fred laertes 65769.0 +fred miller 65536.0 +fred nixon 65560.0 +fred nixon 65612.0 +fred nixon 65703.0 +fred nixon 65705.0 +fred polk 65603.0 +fred polk 65656.0 +fred polk 65701.0 +fred polk 65706.0 +fred quirinius 65697.0 +fred quirinius 65782.0 +fred robinson 65623.0 +fred steinbeck 65544.0 +fred steinbeck 65651.0 +fred steinbeck 65755.0 +fred underhill 65629.0 +fred van buren 65537.0 +fred van buren 65561.0 +fred van buren 65745.0 +fred van buren 65789.0 +fred white 65589.0 +fred young 65594.0 +fred young 65773.0 +fred zipper 65553.0 +gabriella allen 65646.0 +gabriella allen 65677.0 +gabriella brown 65704.0 +gabriella brown 65753.0 +gabriella carson 65586.0 +gabriella davidson 65565.0 +gabriella ellison 65706.0 +gabriella ellison 65716.0 +gabriella falkner 65623.0 +gabriella falkner 65711.0 +gabriella falkner 65767.0 +gabriella garcia 65571.0 +gabriella hernandez 65587.0 +gabriella hernandez 65717.0 +gabriella ichabod 65559.0 +gabriella ichabod 65633.0 +gabriella ichabod 65702.0 +gabriella ichabod 65712.0 +gabriella ichabod 65717.0 +gabriella king 65651.0 +gabriella king 65657.0 +gabriella laertes 65781.0 +gabriella miller 65646.0 +gabriella ovid 65556.0 +gabriella ovid 65583.0 +gabriella polk 65701.0 +gabriella polk 65790.0 +gabriella steinbeck 65582.0 +gabriella steinbeck 65653.0 +gabriella thompson 65682.0 +gabriella thompson 65755.0 +gabriella thompson 65766.0 +gabriella van buren 65581.0 +gabriella van buren 65644.0 +gabriella white 65638.0 +gabriella young 65699.0 +gabriella young 65774.0 +gabriella zipper 65540.0 +gabriella zipper 65754.0 +holly allen 65596.0 +holly brown 65599.0 +holly brown 65619.0 +holly falkner 65720.0 +holly hernandez 65602.0 +holly hernandez 65686.0 +holly hernandez 65750.0 +holly hernandez 65788.0 +holly ichabod 65711.0 +holly ichabod 65749.0 +holly ichabod 65752.0 +holly johnson 65655.0 +holly johnson 65662.0 +holly johnson 65755.0 +holly king 65549.0 +holly king 65648.0 +holly laertes 65664.0 +holly miller 65653.0 +holly nixon 65539.0 +holly nixon 65658.0 +holly polk 65743.0 +holly polk 65751.0 +holly robinson 65564.0 +holly thompson 65538.0 +holly thompson 65578.0 +holly thompson 65713.0 +holly underhill 65634.0 +holly underhill 65654.0 +holly underhill 65721.0 +holly underhill 65759.0 +holly van buren 65727.0 +holly white 65536.0 +holly white 65602.0 +holly xylophone 65544.0 +holly young 65606.0 +holly young 65765.0 +holly zipper 65607.0 +holly zipper 65755.0 +irene allen 65556.0 +irene brown 65633.0 +irene brown 65664.0 +irene brown 65765.0 +irene carson 65590.0 +irene ellison 65659.0 +irene ellison 65696.0 +irene falkner 65620.0 +irene falkner 65661.0 +irene garcia 65660.0 +irene garcia 65711.0 +irene garcia 65787.0 +irene ichabod 65645.0 +irene ichabod 65722.0 +irene johnson 65583.0 +irene laertes 65664.0 +irene laertes 65710.0 +irene laertes 65722.0 +irene miller 65730.0 +irene nixon 65631.0 +irene nixon 65643.0 +irene nixon 65653.0 +irene ovid 65691.0 +irene ovid 65734.0 +irene ovid 65753.0 +irene polk 65551.0 +irene polk 65575.0 +irene polk 65579.0 +irene polk 65595.0 +irene polk 65610.0 +irene quirinius 65724.0 +irene quirinius 65769.0 +irene quirinius 65773.0 +irene robinson 65554.0 +irene steinbeck 65683.0 +irene thompson 65688.0 +irene underhill 65591.0 +irene underhill 65707.5 +irene van buren 65579.0 +irene van buren 65589.0 +irene xylophone 65775.0 +jessica brown 65588.0 +jessica carson 65553.0 +jessica carson 65672.0 +jessica carson 65747.0 +jessica davidson 65549.0 +jessica davidson 65606.0 +jessica davidson 65675.0 +jessica davidson 65727.0 +jessica ellison 65567.0 +jessica ellison 65663.0 +jessica falkner 65584.0 +jessica garcia 65676.0 +jessica garcia 65789.0 +jessica ichabod 65704.0 +jessica johnson 65607.0 +jessica johnson 65720.0 +jessica miller 65733.0 +jessica nixon 65590.0 +jessica nixon 65774.0 +jessica ovid 65582.0 +jessica ovid 65751.0 +jessica polk 65637.0 +jessica quirinius 65562.0 +jessica quirinius 65608.0 +jessica quirinius 65712.0 +jessica quirinius 65716.0 +jessica robinson 65576.0 +jessica thompson 65581.0 +jessica thompson 65675.0 +jessica underhill 65656.0 +jessica underhill 65702.0 +jessica underhill 65783.0 +jessica van buren 65615.0 +jessica white 65544.0 +jessica white 65570.0 +jessica white 65594.0 +jessica white 65673.0 +jessica white 65779.0 +jessica xylophone 65562.0 +jessica young 65623.0 +jessica young 65711.0 +jessica zipper 65600.0 +jessica zipper 65657.0 +jessica zipper 65778.0 +katie allen 65542.0 +katie brown 65590.0 +katie davidson 65619.0 +katie ellison 65675.0 +katie ellison 65699.0 +katie falkner 65728.0 +katie garcia 65625.0 +katie garcia 65747.0 +katie hernandez 65550.0 +katie ichabod 65658.0 +katie ichabod 65726.0 +katie ichabod 65757.0 +katie king 65629.0 +katie king 65647.0 +katie king 65776.0 +katie miller 65541.0 +katie miller 65661.0 +katie nixon 65669.0 +katie ovid 65681.0 +katie polk 65746.0 +katie polk 65784.0 +katie robinson 65697.0 +katie van buren 65643.0 +katie van buren 65730.0 +katie white 65620.0 +katie white 65719.0 +katie xylophone 65585.0 +katie young 65644.0 +katie young 65746.0 +katie young 65764.0 +katie zipper 65568.0 +katie zipper 65733.0 +luke allen 65547.0 +luke allen 65552.0 +luke allen 65576.0 +luke allen 65681.0 +luke allen 65776.0 +luke brown 65719.0 +luke davidson 65656.0 +luke davidson 65791.0 +luke ellison 65582.0 +luke ellison 65664.0 +luke ellison 65779.0 +luke falkner 65589.0 +luke falkner 65618.0 +luke garcia 65687.0 +luke garcia 65778.0 +luke ichabod 65629.0 +luke ichabod 65654.0 +luke johnson 65545.0 +luke johnson 65716.0 +luke johnson 65718.0 +luke laertes 65608.0 +luke laertes 65657.0 +luke laertes 65685.0 +luke laertes 65730.0 +luke laertes 65756.0 +luke miller 65752.0 +luke ovid 65569.0 +luke ovid 65693.0 +luke polk 65645.0 +luke polk 65658.0 +luke quirinius 65655.0 +luke robinson 65634.0 +luke robinson 65772.0 +luke thompson 65626.0 +luke underhill 65553.0 +luke underhill 65571.0 +luke underhill 65651.0 +luke van buren 65678.0 +luke white 65693.0 +luke xylophone 65597.0 +luke zipper 65641.0 +mike allen 65706.0 +mike brown 65654.0 +mike carson 65698.0 +mike carson 65700.0 +mike carson 65751.0 +mike davidson 65658.0 +mike davidson 65759.0 +mike ellison 65598.0 +mike ellison 65606.0 +mike ellison 65718.0 +mike ellison 65738.0 +mike ellison 65760.0 +mike falkner 65609.0 +mike garcia 65571.0 +mike garcia 65600.0 +mike garcia 65770.0 +mike hernandez 65548.0 +mike hernandez 65672.0 +mike ichabod 65621.0 +mike king 65563.0 +mike king 65586.0 +mike king 65591.0 +mike king 65642.0 +mike king 65769.0 +mike king 65776.0 +mike miller 65549.0 +mike nixon 65619.0 +mike nixon 65704.0 +mike polk 65619.0 +mike polk 65658.0 +mike polk 65704.0 +mike quirinius 65717.0 +mike steinbeck 65550.0 +mike steinbeck 65564.0 +mike steinbeck 65573.0 +mike steinbeck 65749.0 +mike van buren 65620.0 +mike van buren 65770.0 +mike white 65648.0 +mike white 65685.0 +mike white 65769.0 +mike white 65778.0 +mike young 65545.0 +mike young 65581.0 +mike young 65736.0 +mike zipper 65552.0 +mike zipper 65695.0 +mike zipper 65779.0 +nick allen 65641.0 +nick allen 65786.0 +nick brown 65724.0 +nick davidson 65601.0 +nick ellison 65691.0 +nick ellison 65745.0 +nick falkner 65583.0 +nick falkner 65676.0 +nick garcia 65712.0 +nick garcia 65720.0 +nick garcia 65723.0 +nick ichabod 65572.0 +nick ichabod 65681.0 +nick ichabod 65737.0 +nick johnson 65585.0 +nick johnson 65784.0 +nick laertes 65624.0 +nick miller 65757.0 +nick nixon 65650.0 +nick ovid 65719.0 +nick polk 65716.0 +nick quirinius 65588.0 +nick quirinius 65723.0 +nick robinson 65547.0 +nick robinson 65675.0 +nick steinbeck 65689.0 +nick thompson 65610.0 +nick underhill 65619.0 +nick van buren 65603.0 +nick xylophone 65644.0 +nick young 65654.0 +nick young 65660.0 +nick zipper 65757.0 +nick zipper 65765.0 +oscar allen 65644.0 +oscar brown 65614.0 +oscar carson 65537.0 +oscar carson 65548.0 +oscar carson 65549.0 +oscar carson 65624.0 +oscar carson 65697.0 +oscar davidson 65556.0 +oscar ellison 65630.0 +oscar ellison 65630.0 +oscar falkner 65692.0 +oscar garcia 65751.0 +oscar hernandez 65683.0 +oscar hernandez 65707.0 +oscar ichabod 65536.0 +oscar ichabod 65562.0 +oscar ichabod 65637.0 +oscar ichabod 65763.0 +oscar johnson 65645.0 +oscar johnson 65778.0 +oscar king 65541.0 +oscar king 65550.0 +oscar king 65787.0 +oscar laertes 65625.0 +oscar laertes 65690.0 +oscar laertes 65756.0 +oscar laertes 65790.0 +oscar nixon 65596.0 +oscar ovid 65536.0 +oscar ovid 65615.0 +oscar ovid 65665.5 +oscar polk 65541.0 +oscar polk 65643.0 +oscar quirinius 65541.0 +oscar quirinius 65560.0 +oscar quirinius 65689.0 +oscar quirinius 65720.0 +oscar robinson 65537.0 +oscar robinson 65658.0 +oscar robinson 65687.0 +oscar robinson 65782.0 +oscar steinbeck 65709.0 +oscar thompson 65542.0 +oscar thompson 65681.0 +oscar thompson 65727.0 +oscar thompson 65738.0 +oscar underhill 65626.0 +oscar van buren 65581.0 +oscar van buren 65635.0 +oscar van buren 65705.0 +oscar white 65552.0 +oscar white 65564.0 +oscar white 65671.0 +oscar white 65735.0 +oscar xylophone 65773.0 +oscar xylophone 65773.0 +oscar xylophone 65775.0 +oscar zipper 65568.0 +oscar zipper 65740.0 +oscar zipper 65777.0 +priscilla brown 65670.0 +priscilla brown 65690.0 +priscilla brown 65749.0 +priscilla carson 65658.0 +priscilla carson 65687.0 +priscilla carson 65755.0 +priscilla ichabod 65627.0 +priscilla ichabod 65759.0 +priscilla johnson 65543.0 +priscilla johnson 65668.0 +priscilla johnson 65674.5 +priscilla johnson 65681.0 +priscilla johnson 65755.0 +priscilla king 65646.0 +priscilla nixon 65564.0 +priscilla nixon 65600.0 +priscilla ovid 65541.0 +priscilla ovid 65790.0 +priscilla polk 65747.0 +priscilla quirinius 65672.0 +priscilla thompson 65654.0 +priscilla underhill 65715.0 +priscilla underhill 65729.0 +priscilla van buren 65607.0 +priscilla van buren 65685.0 +priscilla van buren 65749.0 +priscilla white 65652.0 +priscilla xylophone 65538.0 +priscilla xylophone 65763.0 +priscilla xylophone 65774.0 +priscilla young 65585.0 +priscilla young 65658.0 +priscilla zipper 65622.0 +priscilla zipper 65726.0 +quinn allen 65657.0 +quinn allen 65708.0 +quinn brown 65691.0 +quinn brown 65700.0 +quinn brown 65733.0 +quinn davidson 65549.0 +quinn davidson 65714.0 +quinn davidson 65776.0 +quinn davidson 65779.0 +quinn ellison 65705.0 +quinn ellison 65778.0 +quinn garcia 65568.0 +quinn garcia 65604.0 +quinn garcia 65610.0 +quinn garcia 65773.0 +quinn ichabod 65609.0 +quinn king 65558.0 +quinn king 65649.0 +quinn laertes 65542.0 +quinn laertes 65560.0 +quinn laertes 65627.0 +quinn nixon 65659.0 +quinn ovid 65699.0 +quinn quirinius 65747.0 +quinn robinson 65627.0 +quinn steinbeck 65578.0 +quinn steinbeck 65763.0 +quinn thompson 65643.0 +quinn thompson 65774.0 +quinn underhill 65549.0 +quinn underhill 65694.0 +quinn underhill 65767.0 +quinn van buren 65725.0 +quinn young 65647.0 +quinn zipper 65579.0 +quinn zipper 65693.0 +rachel allen 65661.0 +rachel allen 65709.0 +rachel brown 65586.0 +rachel brown 65587.0 +rachel brown 65587.0 +rachel brown 65610.0 +rachel brown 65693.0 +rachel carson 65677.0 +rachel carson 65682.0 +rachel davidson 65755.0 +rachel ellison 65761.0 +rachel falkner 65616.0 +rachel falkner 65681.0 +rachel falkner 65693.0 +rachel falkner 65764.0 +rachel johnson 65658.0 +rachel king 65604.0 +rachel king 65643.0 +rachel laertes 65562.0 +rachel laertes 65624.0 +rachel ovid 65721.0 +rachel ovid 65736.0 +rachel polk 65686.0 +rachel quirinius 65787.0 +rachel robinson 65544.0 +rachel robinson 65717.0 +rachel robinson 65724.0 +rachel thompson 65648.0 +rachel thompson 65662.0 +rachel thompson 65733.0 +rachel underhill 65667.0 +rachel white 65615.0 +rachel white 65717.0 +rachel young 65727.0 +rachel zipper 65757.0 +rachel zipper 65785.0 +sarah carson 65616.0 +sarah carson 65693.0 +sarah carson 65694.0 +sarah ellison 65611.0 +sarah falkner 65606.0 +sarah falkner 65680.0 +sarah garcia 65563.0 +sarah garcia 65638.0 +sarah garcia 65661.0 +sarah ichabod 65667.0 +sarah ichabod 65671.0 +sarah johnson 65659.0 +sarah johnson 65716.0 +sarah johnson 65731.0 +sarah johnson 65751.0 +sarah king 65650.0 +sarah king 65699.0 +sarah miller 65557.0 +sarah ovid 65550.0 +sarah robinson 65677.0 +sarah robinson 65763.0 +sarah steinbeck 65721.0 +sarah white 65622.0 +sarah white 65747.0 +sarah xylophone 65678.0 +sarah young 65595.0 +sarah zipper 65550.0 +tom brown 65593.0 +tom brown 65675.0 +tom carson 65539.0 +tom carson 65624.0 +tom carson 65780.0 +tom davidson 65780.0 +tom ellison 65578.0 +tom ellison 65670.0 +tom ellison 65756.0 +tom falkner 65574.0 +tom falkner 65625.0 +tom hernandez 65575.0 +tom hernandez 65632.0 +tom ichabod 65588.0 +tom johnson 65536.0 +tom johnson 65789.0 +tom king 65576.0 +tom laertes 65617.0 +tom laertes 65701.0 +tom miller 65594.0 +tom miller 65603.0 +tom miller 65704.0 +tom nixon 65672.0 +tom ovid 65628.0 +tom polk 65652.0 +tom polk 65742.0 +tom quirinius 65563.0 +tom quirinius 65783.0 +tom robinson 65626.0 +tom robinson 65632.0 +tom robinson 65691.0 +tom robinson 65758.0 +tom steinbeck 65666.0 +tom van buren 65621.0 +tom van buren 65652.0 +tom van buren 65669.0 +tom white 65548.0 +tom young 65544.0 +tom young 65546.0 +tom zipper 65789.0 +ulysses brown 65735.0 +ulysses carson 65602.0 +ulysses carson 65643.0 +ulysses carson 65703.0 +ulysses carson 65716.0 +ulysses davidson 65750.0 +ulysses ellison 65575.0 +ulysses garcia 65666.0 +ulysses hernandez 65651.0 +ulysses hernandez 65702.0 +ulysses hernandez 65786.0 +ulysses ichabod 65551.0 +ulysses ichabod 65566.0 +ulysses johnson 65776.0 +ulysses king 65649.0 +ulysses laertes 65691.0 +ulysses laertes 65711.0 +ulysses laertes 65781.0 +ulysses miller 65610.0 +ulysses miller 65637.0 +ulysses nixon 65603.0 +ulysses ovid 65656.0 +ulysses polk 65563.0 +ulysses polk 65580.0 +ulysses polk 65612.0 +ulysses polk 65777.0 +ulysses quirinius 65786.0 +ulysses robinson 65744.0 +ulysses steinbeck 65611.0 +ulysses steinbeck 65680.0 +ulysses thompson 65788.0 +ulysses underhill 65570.0 +ulysses underhill 65616.0 +ulysses underhill 65620.0 +ulysses underhill 65623.0 +ulysses underhill 65641.0 +ulysses underhill 65713.0 +ulysses underhill 65785.0 +ulysses van buren 65684.0 +ulysses white 65654.0 +ulysses white 65675.0 +ulysses xylophone 65623.0 +ulysses xylophone 65636.0 +ulysses xylophone 65781.0 +ulysses young 65675.0 +ulysses young 65736.0 +ulysses young 65748.0 +victor allen 65684.0 +victor allen 65707.0 +victor brown 65550.0 +victor brown 65555.0 +victor brown 65622.0 +victor brown 65673.0 +victor davidson 65579.0 +victor davidson 65628.0 +victor davidson 65783.0 +victor ellison 65641.0 +victor ellison 65782.0 +victor hernandez 65571.0 +victor hernandez 65659.0 +victor hernandez 65708.0 +victor hernandez 65735.0 +victor hernandez 65775.0 +victor johnson 65606.0 +victor johnson 65607.0 +victor johnson 65607.0 +victor king 65721.0 +victor king 65743.0 +victor laertes 65638.0 +victor laertes 65644.0 +victor miller 65570.0 +victor nixon 65709.0 +victor nixon 65791.0 +victor ovid 65649.0 +victor polk 65625.0 +victor quirinius 65620.0 +victor quirinius 65651.0 +victor robinson 65596.0 +victor robinson 65673.0 +victor steinbeck 65618.0 +victor steinbeck 65661.0 +victor steinbeck 65686.0 +victor thompson 65548.0 +victor van buren 65664.0 +victor van buren 65774.0 +victor white 65548.0 +victor white 65601.0 +victor xylophone 65549.0 +victor xylophone 65618.0 +victor xylophone 65644.0 +victor xylophone 65677.0 +victor xylophone 65755.0 +victor young 65628.0 +victor zipper 65743.0 +wendy allen 65628.0 +wendy allen 65711.0 +wendy allen 65782.0 +wendy brown 65580.0 +wendy brown 65657.0 +wendy ellison 65545.0 +wendy ellison 65603.0 +wendy falkner 65595.0 +wendy falkner 65604.0 +wendy falkner 65635.0 +wendy garcia 65659.0 +wendy garcia 65746.0 +wendy garcia 65747.0 +wendy garcia 65777.0 +wendy hernandez 65650.0 +wendy ichabod 65730.0 +wendy king 65586.0 +wendy king 65664.0 +wendy king 65670.0 +wendy laertes 65566.0 +wendy laertes 65683.0 +wendy laertes 65727.0 +wendy miller 65582.0 +wendy miller 65626.0 +wendy nixon 65611.0 +wendy nixon 65746.0 +wendy ovid 65589.0 +wendy ovid 65643.0 +wendy polk 65656.0 +wendy polk 65692.0 +wendy quirinius 65766.0 +wendy quirinius 65767.0 +wendy robinson 65622.0 +wendy robinson 65715.0 +wendy robinson 65774.0 +wendy steinbeck 65612.0 +wendy thompson 65650.0 +wendy thompson 65737.0 +wendy underhill 65662.0 +wendy underhill 65758.0 +wendy underhill 65775.0 +wendy van buren 65680.0 +wendy van buren 65699.0 +wendy white 65705.0 +wendy xylophone 65687.0 +wendy xylophone 65773.0 +wendy young 65674.0 +wendy young 65685.0 +xavier allen 65611.0 +xavier allen 65618.0 +xavier allen 65771.0 +xavier brown 65600.0 +xavier brown 65704.0 +xavier brown 65723.0 +xavier carson 65731.0 +xavier carson 65758.0 +xavier davidson 65644.0 +xavier davidson 65664.0 +xavier davidson 65755.0 +xavier ellison 65541.0 +xavier ellison 65622.0 +xavier garcia 65672.0 +xavier hernandez 65541.0 +xavier hernandez 65544.0 +xavier hernandez 65766.0 +xavier ichabod 65597.0 +xavier ichabod 65663.0 +xavier johnson 65655.0 +xavier johnson 65744.0 +xavier king 65590.0 +xavier king 65601.0 +xavier laertes 65743.0 +xavier ovid 65788.0 +xavier polk 65587.0 +xavier polk 65653.0 +xavier polk 65675.0 +xavier polk 65696.0 +xavier quirinius 65599.0 +xavier quirinius 65650.0 +xavier quirinius 65656.0 +xavier quirinius 65737.0 +xavier thompson 65608.0 +xavier underhill 65710.0 +xavier white 65703.0 +xavier white 65732.0 +xavier xylophone 65572.0 +xavier zipper 65561.0 +yuri allen 65565.0 +yuri allen 65682.0 +yuri brown 65538.0 +yuri brown 65688.0 +yuri carson 65670.0 +yuri carson 65769.0 +yuri ellison 65570.0 +yuri ellison 65581.0 +yuri falkner 65658.0 +yuri falkner 65681.0 +yuri garcia 65639.0 +yuri hernandez 65706.0 +yuri johnson 65587.0 +yuri johnson 65697.0 +yuri johnson 65712.0 +yuri king 65721.0 +yuri laertes 65637.0 +yuri laertes 65773.0 +yuri nixon 65635.0 +yuri nixon 65740.0 +yuri polk 65607.0 +yuri polk 65713.0 +yuri polk 65742.0 +yuri quirinius 65544.0 +yuri quirinius 65617.0 +yuri quirinius 65695.0 +yuri steinbeck 65592.0 +yuri steinbeck 65679.0 +yuri thompson 65676.0 +yuri underhill 65718.0 +yuri underhill 65750.0 +yuri white 65659.0 +yuri xylophone 65714.0 +zach allen 65667.0 +zach brown 65559.0 +zach brown 65588.0 +zach brown 65691.0 +zach brown 65759.0 +zach brown 65762.0 +zach carson 65572.0 +zach ellison 65748.0 +zach falkner 65620.0 +zach falkner 65627.0 +zach garcia 65544.0 +zach garcia 65623.0 +zach garcia 65629.0 +zach garcia 65764.5 +zach ichabod 65599.0 +zach ichabod 65612.0 +zach king 65556.0 +zach king 65702.0 +zach king 65773.0 +zach miller 65583.0 +zach miller 65665.0 +zach miller 65719.0 +zach ovid 65578.0 +zach ovid 65669.0 +zach ovid 65703.0 +zach ovid 65784.0 +zach quirinius 65691.0 +zach robinson 65599.0 +zach steinbeck 65602.0 +zach steinbeck 65695.0 +zach thompson 65636.0 +zach thompson 65696.0 +zach underhill 65573.0 +zach white 65733.0 +zach xylophone 65542.0 +zach xylophone 65780.0 +zach young 65576.0 +zach zipper 65579.0 +zach zipper 65649.0 +zach zipper 65676.0 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-3-d90b27fca067b0b3c48d873b3ef32af7 b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-3-d90b27fca067b0b3c48d873b3ef32af7 new file mode 100644 index 0000000000000..072a8a891a836 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-3-d90b27fca067b0b3c48d873b3ef32af7 @@ -0,0 +1,1049 @@ +65536 +65536 +65536 +65536 +65536 +65536 +65537 +65537 +65537 +65537 +65538 +65538 +65538 +65539 +65539 +65539 +65540 +65541 +65541 +65541 +65541 +65541 +65541 +65541 +65541 +65542 +65542 +65542 +65542 +65543 +65543 +65544 +65544 +65544 +65544 +65544 +65544 +65544 +65544 +65545 +65545 +65545 +65547 +65547 +65547 +65547 +65548 +65548 +65548 +65548 +65548 +65548 +65548 +65549 +65549 +65549 +65549 +65549 +65549 +65549 +65549 +65550 +65550 +65550 +65550 +65550 +65550 +65550 +65551 +65551 +65551 +65552 +65552 +65552 +65552 +65552 +65553 +65553 +65553 +65553 +65554 +65554 +65555 +65556 +65556 +65556 +65557 +65557 +65558 +65559 +65559 +65559 +65559 +65560 +65560 +65560 +65560 +65561 +65562 +65562 +65562 +65562 +65562 +65562 +65563 +65563 +65563 +65563 +65563 +65564 +65564 +65564 +65564 +65564 +65564 +65564 +65564 +65565 +65565 +65566 +65566 +65567 +65568 +65568 +65568 +65569 +65570 +65570 +65570 +65570 +65570 +65571 +65571 +65571 +65572 +65572 +65572 +65572 +65572 +65573 +65573 +65573 +65574 +65574 +65574 +65575 +65575 +65575 +65575 +65576 +65576 +65576 +65576 +65577 +65578 +65578 +65578 +65578 +65578 +65578 +65579 +65579 +65579 +65579 +65579 +65580 +65580 +65580 +65581 +65581 +65581 +65581 +65581 +65582 +65582 +65582 +65582 +65582 +65583 +65583 +65583 +65583 +65584 +65584 +65584 +65584 +65585 +65585 +65585 +65585 +65585 +65586 +65586 +65586 +65586 +65586 +65587 +65587 +65587 +65587 +65588 +65588 +65588 +65588 +65588 +65589 +65589 +65589 +65589 +65589 +65590 +65590 +65590 +65590 +65590 +65591 +65591 +65591 +65591 +65592 +65593 +65593 +65594 +65594 +65594 +65594 +65595 +65595 +65595 +65595 +65595 +65595 +65596 +65596 +65596 +65596 +65596 +65597 +65597 +65597 +65598 +65599 +65599 +65599 +65600 +65600 +65600 +65600 +65601 +65601 +65601 +65602 +65602 +65602 +65602 +65603 +65603 +65603 +65603 +65603 +65604 +65604 +65604 +65605 +65606 +65606 +65606 +65606 +65606 +65607 +65607 +65607 +65607 +65607 +65607 +65608 +65608 +65608 +65608 +65609 +65610 +65610 +65610 +65610 +65610 +65610 +65611 +65611 +65611 +65612 +65612 +65612 +65614 +65614 +65614 +65615 +65615 +65615 +65616 +65616 +65617 +65617 +65617 +65617 +65618 +65618 +65618 +65618 +65618 +65619 +65619 +65619 +65619 +65619 +65619 +65620 +65620 +65620 +65620 +65620 +65620 +65621 +65621 +65622 +65622 +65622 +65622 +65622 +65622 +65623 +65623 +65623 +65623 +65623 +65623 +65623 +65624 +65624 +65624 +65624 +65625 +65625 +65625 +65625 +65626 +65626 +65626 +65626 +65627 +65627 +65627 +65627 +65628 +65628 +65628 +65628 +65628 +65628 +65629 +65629 +65629 +65630 +65630 +65631 +65632 +65632 +65632 +65633 +65633 +65633 +65633 +65634 +65634 +65634 +65634 +65635 +65635 +65635 +65636 +65636 +65636 +65636 +65637 +65637 +65637 +65637 +65637 +65637 +65637 +65638 +65638 +65638 +65639 +65640 +65641 +65641 +65641 +65641 +65642 +65643 +65643 +65643 +65643 +65643 +65643 +65643 +65643 +65643 +65643 +65644 +65644 +65644 +65644 +65644 +65645 +65645 +65645 +65646 +65646 +65646 +65647 +65647 +65648 +65648 +65648 +65648 +65649 +65649 +65649 +65650 +65650 +65650 +65650 +65650 +65650 +65650 +65651 +65651 +65651 +65651 +65651 +65651 +65652 +65652 +65652 +65653 +65653 +65653 +65653 +65654 +65654 +65654 +65654 +65654 +65654 +65654 +65654 +65654 +65655 +65655 +65656 +65656 +65656 +65656 +65656 +65656 +65657 +65657 +65657 +65657 +65657 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65659 +65659 +65659 +65659 +65659 +65659 +65659 +65659 +65660 +65660 +65660 +65661 +65661 +65661 +65661 +65661 +65662 +65662 +65662 +65662 +65662 +65662 +65663 +65663 +65663 +65663 +65664 +65664 +65664 +65664 +65664 +65664 +65664 +65665 +65666 +65666 +65667 +65667 +65667 +65667 +65667 +65667 +65668 +65669 +65669 +65669 +65669 +65669 +65669 +65670 +65670 +65670 +65670 +65670 +65671 +65671 +65671 +65672 +65672 +65672 +65672 +65672 +65672 +65673 +65673 +65673 +65673 +65673 +65674 +65674 +65674 +65674 +65675 +65675 +65675 +65675 +65675 +65675 +65675 +65675 +65676 +65676 +65676 +65677 +65677 +65677 +65677 +65677 +65677 +65678 +65678 +65678 +65678 +65679 +65679 +65680 +65680 +65680 +65680 +65680 +65680 +65680 +65681 +65681 +65681 +65681 +65681 +65682 +65682 +65682 +65683 +65683 +65683 +65683 +65684 +65684 +65684 +65684 +65685 +65685 +65685 +65685 +65685 +65685 +65686 +65686 +65686 +65687 +65687 +65687 +65687 +65688 +65688 +65689 +65689 +65690 +65690 +65691 +65691 +65691 +65691 +65691 +65691 +65691 +65691 +65692 +65692 +65693 +65693 +65693 +65693 +65693 +65693 +65694 +65694 +65694 +65695 +65695 +65695 +65695 +65695 +65696 +65696 +65696 +65696 +65697 +65697 +65697 +65697 +65697 +65697 +65698 +65698 +65698 +65699 +65699 +65699 +65699 +65699 +65699 +65700 +65700 +65700 +65701 +65701 +65701 +65702 +65702 +65702 +65702 +65702 +65703 +65703 +65703 +65703 +65703 +65704 +65704 +65704 +65704 +65704 +65704 +65705 +65705 +65705 +65705 +65706 +65706 +65706 +65706 +65706 +65706 +65707 +65707 +65708 +65708 +65709 +65709 +65709 +65710 +65711 +65711 +65711 +65711 +65711 +65711 +65712 +65712 +65712 +65712 +65712 +65713 +65713 +65713 +65713 +65713 +65714 +65714 +65714 +65715 +65715 +65715 +65715 +65715 +65716 +65716 +65716 +65716 +65716 +65716 +65717 +65717 +65717 +65717 +65717 +65718 +65718 +65718 +65718 +65719 +65719 +65719 +65719 +65720 +65720 +65720 +65720 +65720 +65720 +65721 +65721 +65721 +65721 +65721 +65721 +65721 +65722 +65722 +65722 +65722 +65723 +65723 +65724 +65724 +65724 +65724 +65724 +65724 +65725 +65726 +65726 +65726 +65726 +65727 +65727 +65727 +65727 +65727 +65728 +65728 +65729 +65730 +65730 +65730 +65730 +65731 +65731 +65731 +65731 +65732 +65732 +65732 +65733 +65733 +65733 +65733 +65733 +65733 +65734 +65734 +65735 +65735 +65735 +65736 +65736 +65736 +65736 +65737 +65737 +65737 +65737 +65737 +65738 +65738 +65738 +65738 +65739 +65739 +65739 +65740 +65740 +65740 +65741 +65742 +65742 +65742 +65743 +65743 +65743 +65743 +65744 +65744 +65745 +65745 +65745 +65745 +65746 +65746 +65746 +65746 +65747 +65747 +65747 +65747 +65747 +65747 +65747 +65748 +65748 +65749 +65749 +65749 +65749 +65749 +65750 +65750 +65750 +65750 +65750 +65751 +65751 +65751 +65751 +65751 +65752 +65752 +65753 +65753 +65754 +65754 +65755 +65755 +65755 +65755 +65755 +65755 +65755 +65755 +65755 +65756 +65756 +65756 +65756 +65756 +65757 +65757 +65757 +65757 +65757 +65758 +65758 +65758 +65758 +65758 +65758 +65759 +65759 +65759 +65759 +65759 +65760 +65760 +65760 +65760 +65760 +65761 +65762 +65762 +65762 +65762 +65762 +65763 +65763 +65763 +65763 +65763 +65764 +65764 +65764 +65765 +65765 +65765 +65766 +65766 +65766 +65766 +65766 +65767 +65767 +65767 +65768 +65769 +65769 +65769 +65769 +65769 +65769 +65769 +65770 +65770 +65771 +65771 +65771 +65772 +65773 +65773 +65773 +65773 +65773 +65773 +65773 +65773 +65774 +65774 +65774 +65774 +65774 +65774 +65775 +65775 +65775 +65775 +65775 +65775 +65776 +65776 +65776 +65776 +65776 +65776 +65776 +65777 +65777 +65777 +65777 +65777 +65777 +65778 +65778 +65778 +65778 +65778 +65778 +65778 +65778 +65778 +65779 +65779 +65779 +65779 +65779 +65779 +65779 +65780 +65780 +65780 +65781 +65781 +65781 +65782 +65782 +65782 +65782 +65782 +65783 +65783 +65783 +65783 +65783 +65783 +65783 +65784 +65784 +65784 +65784 +65784 +65785 +65785 +65786 +65786 +65786 +65786 +65786 +65787 +65787 +65787 +65787 +65787 +65788 +65788 +65788 +65788 +65789 +65789 +65789 +65789 +65789 +65789 +65789 +65789 +65789 +65789 +65790 +65790 +65790 +65791 +65791 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-4-f2e4d659b65a833e9281b6786d3d55c1 b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-4-f2e4d659b65a833e9281b6786d3d55c1 new file mode 100644 index 0000000000000..9cc7e7ea6c2b2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-4-f2e4d659b65a833e9281b6786d3d55c1 @@ -0,0 +1,1049 @@ + 24.37875 + 27.900000000000002 + 43.64 +alice allen 16.919999999999998 +alice allen 20.39 +alice allen 23.59 +alice brown 6.91 +alice carson 41.74 +alice davidson 26.346000000000004 +alice falkner 32.166666666666664 +alice garcia 15.412 +alice hernandez 19.958181818181817 +alice hernandez 30.482857142857142 +alice johnson 25.51 +alice king 13.085 +alice king 25.616666666666664 +alice king 42.335 +alice laertes 20.549999999999997 +alice laertes 30.436 +alice miller 29.563333333333333 +alice nixon 19.28666666666667 +alice nixon 24.5625 +alice nixon 32.99 +alice ovid 31.35 +alice polk 17.863999999999997 +alice quirinius 19.032857142857143 +alice quirinius 23.9425 +alice robinson 23.338750000000005 +alice robinson 39.85 +alice steinbeck 22.862000000000002 +alice steinbeck 26.328000000000003 +alice steinbeck 27.08777777777778 +alice underhill 24.032222222222224 +alice van buren 19.642000000000003 +alice xylophone 24.438000000000002 +alice xylophone 28.739999999999995 +alice xylophone 30.0825 +alice zipper 26.3 +alice zipper 28.735000000000003 +alice zipper 31.05545454545455 +bob brown 12.902222222222223 +bob brown 13.945 +bob brown 33.843333333333334 +bob carson 28.627999999999997 +bob davidson 19.8525 +bob davidson 23.482 +bob davidson 24.67 +bob ellison 16.315714285714286 +bob ellison 18.4 +bob ellison 26.913999999999998 +bob ellison 27.59 +bob falkner 9.27 +bob garcia 11.63 +bob garcia 22.221249999999998 +bob garcia 23.59636363636364 +bob garcia 26.88857142857143 +bob garcia 28.715000000000003 +bob hernandez 37.23 +bob ichabod 28.33875 +bob king 8.615 +bob king 19.77 +bob king 26.7325 +bob laertes 21.33 +bob laertes 37.88 +bob miller 25.495 +bob ovid 25.675 +bob ovid 25.83 +bob ovid 28.37875 +bob ovid 32.5025 +bob polk 9.74 +bob quirinius 34.57 +bob steinbeck 9.725 +bob van buren 29.552857142857142 +bob white 17.685 +bob white 29.46285714285715 +bob xylophone 17.03 +bob xylophone 33.24 +bob young 19.824 +bob zipper 24.095 +bob zipper 33.36 +bob zipper 34.99 +calvin allen 21.3 +calvin brown 20.808 +calvin brown 24.16 +calvin brown 24.636666666666667 +calvin carson 22.815 +calvin davidson 22.116666666666664 +calvin davidson 22.364 +calvin ellison 24.92 +calvin falkner 18.343999999999998 +calvin falkner 19.56 +calvin falkner 22.946000000000005 +calvin falkner 23.327777777777776 +calvin falkner 23.974999999999998 +calvin falkner 33.382 +calvin garcia 17.285 +calvin hernandez 12.663333333333334 +calvin johnson 24.898571428571433 +calvin laertes 28.105 +calvin laertes 28.362000000000002 +calvin nixon 26.784285714285716 +calvin nixon 27.36 +calvin nixon 32.282 +calvin ovid 22.063333333333336 +calvin ovid 22.81500000000001 +calvin ovid 25.495714285714286 +calvin ovid 30.926666666666666 +calvin polk 27.820000000000004 +calvin quirinius 16.28 +calvin quirinius 25.552500000000002 +calvin robinson 31.814999999999998 +calvin steinbeck 12.85 +calvin steinbeck 14.939999999999998 +calvin steinbeck 17.535 +calvin thompson 28.592857142857145 +calvin thompson 40.79 +calvin underhill 24.062 +calvin van buren 26.525 +calvin van buren 28.865 +calvin white 28.256249999999998 +calvin white 43.275 +calvin xylophone 24.13111111111111 +calvin xylophone 25.27 +calvin xylophone 36.455 +calvin young 19.06 +calvin young 21.455999999999996 +calvin zipper 10.674999999999999 +calvin zipper 26.012857142857143 +david allen 25.134285714285713 +david allen 41.72333333333333 +david brown 8.52 +david brown 28.968181818181815 +david davidson 17.63 +david davidson 26.563333333333336 +david davidson 30.7325 +david davidson 33.33 +david ellison 23.79909090909091 +david ellison 24.74888888888889 +david ellison 26.198571428571427 +david hernandez 27.766 +david ichabod 16.66 +david ichabod 19.538 +david laertes 24.587500000000002 +david nixon 26.01375 +david ovid 24.131428571428575 +david ovid 32.72 +david quirinius 16.5 +david quirinius 25.08 +david quirinius 29.415 +david robinson 22.2175 +david robinson 30.99 +david thompson 25.38 +david underhill 1.17 +david underhill 21.546666666666667 +david underhill 28.26 +david van buren 26.45833333333334 +david van buren 35.7825 +david white 15.833333333333334 +david xylophone 10.71 +david xylophone 26.341428571428565 +david xylophone 33.224000000000004 +david young 9.64 +david young 21.22 +ethan allen 22.68 +ethan brown 19.37 +ethan brown 21.58666666666667 +ethan brown 21.799999999999997 +ethan brown 29.099999999999998 +ethan brown 32.43666666666667 +ethan brown 39.84 +ethan carson 24.15666666666667 +ethan ellison 27.80777777777778 +ethan ellison 48.71 +ethan falkner 17.993333333333336 +ethan falkner 26.775000000000002 +ethan garcia 19.15 +ethan hernandez 25.081111111111113 +ethan johnson 32.81875 +ethan king 19.51 +ethan laertes 16.463 +ethan laertes 17.625999999999998 +ethan laertes 25.020714285714288 +ethan laertes 26.697142857142858 +ethan laertes 28.14 +ethan laertes 29.668571428571425 +ethan laertes 36.589999999999996 +ethan miller 24.326666666666664 +ethan nixon 34.78666666666667 +ethan ovid 20.642857142857142 +ethan polk 6.98 +ethan polk 12.756666666666666 +ethan polk 30.324 +ethan polk 40.46 +ethan quirinius 23.419999999999998 +ethan quirinius 24.36 +ethan quirinius 29.068 +ethan robinson 24.463750000000005 +ethan robinson 31.630000000000003 +ethan underhill 19.86 +ethan van buren 22.241999999999997 +ethan white 31.3175 +ethan white 32.87 +ethan xylophone 30.996000000000002 +ethan zipper 22.728333333333335 +ethan zipper 29.66 +fred davidson 30.116666666666667 +fred davidson 33.55200000000001 +fred davidson 39.37 +fred ellison 16.72 +fred ellison 17.462 +fred ellison 35.1 +fred falkner 14.51 +fred falkner 27.207000000000004 +fred falkner 27.887500000000003 +fred hernandez 36.045 +fred ichabod 29.017000000000003 +fred ichabod 30.405000000000005 +fred johnson 16.9925 +fred king 20.024 +fred king 32.54666666666667 +fred laertes 25.610000000000003 +fred miller 25.92 +fred nixon 14.915 +fred nixon 21.830000000000002 +fred nixon 24.4125 +fred nixon 31.360000000000003 +fred polk 18.698 +fred polk 19.743000000000002 +fred polk 20.96 +fred polk 31.11 +fred quirinius 20.085 +fred quirinius 33.9 +fred robinson 22.502 +fred steinbeck 21.123749999999998 +fred steinbeck 25.572 +fred steinbeck 30.81 +fred underhill 29.198888888888884 +fred van buren 21.34 +fred van buren 23.285 +fred van buren 26.520000000000003 +fred van buren 33.6 +fred white 21.41 +fred young 16.876250000000002 +fred young 20.996666666666666 +fred zipper 23.627499999999998 +gabriella allen 24.113333333333333 +gabriella allen 28.4725 +gabriella brown 29.963333333333335 +gabriella brown 30.65222222222222 +gabriella carson 16.6325 +gabriella davidson 34.52 +gabriella ellison 20.18 +gabriella ellison 29.62 +gabriella falkner 14.37 +gabriella falkner 17.738333333333333 +gabriella falkner 28.61 +gabriella garcia 39.025 +gabriella hernandez 20.818333333333335 +gabriella hernandez 24.601666666666663 +gabriella ichabod 10.4925 +gabriella ichabod 20.686666666666667 +gabriella ichabod 23.185 +gabriella ichabod 23.43 +gabriella ichabod 27.44636363636364 +gabriella king 13.645 +gabriella king 22.23 +gabriella laertes 23.735 +gabriella miller 17.165 +gabriella ovid 22.884545454545453 +gabriella ovid 25.29 +gabriella polk 20.38714285714286 +gabriella polk 25.832000000000004 +gabriella steinbeck 6.226666666666667 +gabriella steinbeck 29.683333333333337 +gabriella thompson 25.565454545454546 +gabriella thompson 29.031 +gabriella thompson 29.122500000000006 +gabriella van buren 24.353 +gabriella van buren 34.21666666666667 +gabriella white 36.5175 +gabriella young 21.28142857142857 +gabriella young 21.32 +gabriella zipper 21.798461538461545 +gabriella zipper 28.676666666666666 +holly allen 27.18 +holly brown 22.76 +holly brown 30.950000000000003 +holly falkner 29.666666666666668 +holly hernandez 19.875 +holly hernandez 23.7075 +holly hernandez 24.5 +holly hernandez 26.50333333333333 +holly ichabod 23.262857142857143 +holly ichabod 25.85090909090909 +holly ichabod 29.521666666666665 +holly johnson 18.939999999999998 +holly johnson 23.2625 +holly johnson 26.49285714285714 +holly king 20.61333333333333 +holly king 30.95888888888889 +holly laertes 17.509999999999998 +holly miller 40.8975 +holly nixon 27.775714285714287 +holly nixon 30.642500000000002 +holly polk 21.02 +holly polk 24.446666666666665 +holly robinson 26.083750000000006 +holly thompson 18.801428571428573 +holly thompson 23.91 +holly thompson 29.97125 +holly underhill 18.19 +holly underhill 22.22888888888889 +holly underhill 22.813333333333336 +holly underhill 30.613999999999997 +holly van buren 20.113333333333333 +holly white 25.284999999999997 +holly white 41.0125 +holly xylophone 26.88571428571429 +holly young 30.8425 +holly young 33.24333333333334 +holly zipper 27.784000000000002 +holly zipper 28.384285714285713 +irene allen 34.605000000000004 +irene brown 18.740000000000002 +irene brown 28.974999999999998 +irene brown 32.230000000000004 +irene carson 25.665833333333335 +irene ellison 10.225000000000001 +irene ellison 26.119999999999997 +irene falkner 9.94 +irene falkner 19.41 +irene garcia 9.790000000000001 +irene garcia 19.666666666666668 +irene garcia 21.22666666666667 +irene ichabod 20.956666666666667 +irene ichabod 24.488333333333333 +irene johnson 25.34 +irene laertes 15.85 +irene laertes 21.573333333333334 +irene laertes 22.041999999999998 +irene miller 34.994285714285716 +irene nixon 22.52 +irene nixon 32.485 +irene nixon 33.165 +irene ovid 17.73 +irene ovid 22.96 +irene ovid 30.92 +irene polk 5.35 +irene polk 25.535 +irene polk 33.76 +irene polk 35.05 +irene polk 45.14 +irene quirinius 38.36 +irene quirinius 41.864999999999995 +irene quirinius 42.0 +irene robinson 30.86 +irene steinbeck 15.08 +irene thompson 28.419999999999998 +irene underhill 27.977999999999998 +irene underhill 28.438 +irene van buren 26.93625 +irene van buren 27.797999999999995 +irene xylophone 29.10454545454546 +jessica brown 38.325 +jessica carson 16.038 +jessica carson 29.668333333333337 +jessica carson 33.06 +jessica davidson 18.926 +jessica davidson 26.2975 +jessica davidson 27.611428571428572 +jessica davidson 29.86 +jessica ellison 26.873333333333335 +jessica ellison 27.123333333333335 +jessica falkner 21.75142857142858 +jessica garcia 16.939090909090908 +jessica garcia 26.48 +jessica ichabod 28.971666666666664 +jessica johnson 21.601428571428574 +jessica johnson 24.42 +jessica miller 26.90571428571429 +jessica nixon 19.15 +jessica nixon 27.025000000000002 +jessica ovid 30.72285714285714 +jessica ovid 30.895 +jessica polk 27.912857142857145 +jessica quirinius 17.05 +jessica quirinius 21.529999999999998 +jessica quirinius 25.16 +jessica quirinius 26.347999999999995 +jessica robinson 24.322857142857142 +jessica thompson 28.658000000000005 +jessica thompson 30.873636363636365 +jessica underhill 14.6725 +jessica underhill 25.831666666666667 +jessica underhill 31.345000000000002 +jessica van buren 19.575 +jessica white 18.35 +jessica white 19.175 +jessica white 20.812 +jessica white 26.0 +jessica white 29.307142857142857 +jessica xylophone 22.26 +jessica young 27.9525 +jessica young 37.61333333333334 +jessica zipper 7.03 +jessica zipper 15.794999999999998 +jessica zipper 19.95 +katie allen 27.283846153846152 +katie brown 24.156666666666666 +katie davidson 13.498000000000001 +katie ellison 19.2 +katie ellison 24.888571428571428 +katie falkner 28.959999999999997 +katie garcia 28.287142857142857 +katie garcia 36.196666666666665 +katie hernandez 25.14428571428572 +katie ichabod 19.363333333333333 +katie ichabod 20.458571428571428 +katie ichabod 28.924999999999997 +katie king 21.64125 +katie king 21.855 +katie king 22.895 +katie miller 16.263333333333335 +katie miller 30.274285714285718 +katie nixon 25.022499999999997 +katie ovid 24.055000000000003 +katie polk 21.296666666666667 +katie polk 32.03 +katie robinson 36.26 +katie van buren 28.332 +katie van buren 31.408000000000005 +katie white 23.48 +katie white 26.236666666666665 +katie xylophone 32.415 +katie young 18.209999999999997 +katie young 22.88125 +katie young 28.39888888888889 +katie zipper 10.285 +katie zipper 27.495 +luke allen 9.42 +luke allen 21.374615384615385 +luke allen 25.32 +luke allen 27.174999999999997 +luke allen 35.434 +luke brown 25.08 +luke davidson 28.205 +luke davidson 28.790000000000003 +luke ellison 7.8 +luke ellison 16.04 +luke ellison 23.426666666666666 +luke falkner 18.0 +luke falkner 22.19 +luke garcia 29.619999999999997 +luke garcia 32.722 +luke ichabod 21.150000000000002 +luke ichabod 32.78142857142857 +luke johnson 21.58666666666667 +luke johnson 23.03 +luke johnson 23.054 +luke laertes 20.264 +luke laertes 33.72 +luke laertes 39.8 +luke laertes 41.36 +luke laertes 42.254999999999995 +luke miller 20.054444444444446 +luke ovid 19.819999999999997 +luke ovid 30.832857142857147 +luke polk 24.348750000000003 +luke polk 26.57625 +luke quirinius 38.07 +luke robinson 30.119999999999994 +luke robinson 30.31375 +luke thompson 29.026874999999997 +luke underhill 21.735714285714288 +luke underhill 22.175 +luke underhill 26.785714285714285 +luke van buren 17.072222222222223 +luke white 29.063333333333333 +luke xylophone 28.994 +luke zipper 33.995 +mike allen 32.78 +mike brown 27.592222222222222 +mike carson 28.8675 +mike carson 29.88 +mike carson 32.07142857142857 +mike davidson 21.240000000000002 +mike davidson 46.31 +mike ellison 20.5275 +mike ellison 21.99 +mike ellison 24.36 +mike ellison 24.511111111111113 +mike ellison 27.703333333333337 +mike falkner 40.335 +mike garcia 24.3525 +mike garcia 24.582 +mike garcia 35.12 +mike hernandez 8.783333333333333 +mike hernandez 19.40666666666667 +mike ichabod 29.120000000000005 +mike king 14.256666666666668 +mike king 17.889999999999997 +mike king 20.493333333333336 +mike king 23.86 +mike king 26.081 +mike king 30.974 +mike miller 29.275 +mike nixon 17.306 +mike nixon 25.572 +mike polk 18.96 +mike polk 23.75142857142857 +mike polk 33.42 +mike quirinius 19.37375 +mike steinbeck 14.155 +mike steinbeck 19.305833333333332 +mike steinbeck 20.721249999999998 +mike steinbeck 31.75 +mike van buren 15.520000000000001 +mike van buren 25.828333333333333 +mike white 19.13111111111111 +mike white 22.4025 +mike white 24.7725 +mike white 35.235 +mike young 1.5 +mike young 24.679 +mike young 34.02833333333333 +mike zipper 17.97 +mike zipper 26.247333333333337 +mike zipper 44.169999999999995 +nick allen 23.744999999999997 +nick allen 36.93 +nick brown 27.669999999999998 +nick davidson 31.97285714285714 +nick ellison 23.061666666666667 +nick ellison 27.676666666666666 +nick falkner 22.555714285714284 +nick falkner 27.46 +nick garcia 17.465 +nick garcia 18.854 +nick garcia 33.60333333333333 +nick ichabod 19.231428571428573 +nick ichabod 27.645000000000003 +nick ichabod 35.836666666666666 +nick johnson 5.58 +nick johnson 25.274 +nick laertes 26.57857142857143 +nick miller 22.208333333333332 +nick nixon 16.107499999999998 +nick ovid 31.350000000000005 +nick polk 35.70333333333334 +nick quirinius 20.753333333333334 +nick quirinius 30.573333333333334 +nick robinson 21.48 +nick robinson 23.185 +nick steinbeck 19.56555555555556 +nick thompson 31.474999999999998 +nick underhill 38.24 +nick van buren 20.77375 +nick xylophone 30.909999999999997 +nick young 10.725000000000001 +nick young 24.95 +nick zipper 16.185000000000002 +nick zipper 34.72 +oscar allen 24.645 +oscar brown 39.55 +oscar carson 21.893333333333334 +oscar carson 22.868 +oscar carson 27.4875 +oscar carson 28.09428571428571 +oscar carson 30.373333333333335 +oscar davidson 9.046666666666667 +oscar ellison 24.185000000000002 +oscar ellison 30.1675 +oscar falkner 19.295 +oscar garcia 22.495833333333334 +oscar hernandez 16.6825 +oscar hernandez 25.736 +oscar ichabod 17.64 +oscar ichabod 21.11 +oscar ichabod 23.508000000000003 +oscar ichabod 30.392222222222227 +oscar johnson 19.9375 +oscar johnson 21.114444444444445 +oscar king 24.590000000000003 +oscar king 26.675 +oscar king 39.6 +oscar laertes 14.975 +oscar laertes 15.525 +oscar laertes 22.6 +oscar laertes 41.6 +oscar nixon 25.4025 +oscar ovid 24.854285714285712 +oscar ovid 25.309 +oscar ovid 29.63 +oscar polk 21.235999999999997 +oscar polk 21.27 +oscar quirinius 24.200000000000003 +oscar quirinius 24.391428571428573 +oscar quirinius 27.83285714285714 +oscar quirinius 27.853333333333328 +oscar robinson 12.3625 +oscar robinson 12.545 +oscar robinson 20.234 +oscar robinson 28.071666666666673 +oscar steinbeck 31.101111111111113 +oscar thompson 19.4875 +oscar thompson 19.975714285714286 +oscar thompson 21.1425 +oscar thompson 21.166363636363638 +oscar underhill 27.644 +oscar van buren 25.843333333333334 +oscar van buren 29.073333333333334 +oscar van buren 29.682727272727274 +oscar white 19.0775 +oscar white 23.483333333333334 +oscar white 24.705000000000002 +oscar white 28.0075 +oscar xylophone 30.020000000000003 +oscar xylophone 30.46833333333333 +oscar xylophone 33.64 +oscar zipper 21.69 +oscar zipper 23.478 +oscar zipper 31.36 +priscilla brown 14.222 +priscilla brown 27.044999999999998 +priscilla brown 31.14769230769231 +priscilla carson 14.33 +priscilla carson 18.951428571428572 +priscilla carson 27.084999999999997 +priscilla ichabod 28.160999999999994 +priscilla ichabod 49.46 +priscilla johnson 8.365 +priscilla johnson 18.176666666666666 +priscilla johnson 25.02666666666667 +priscilla johnson 26.918333333333337 +priscilla johnson 30.695999999999998 +priscilla king 19.747142857142855 +priscilla nixon 29.035555555555554 +priscilla nixon 30.27333333333333 +priscilla ovid 13.591999999999999 +priscilla ovid 35.879999999999995 +priscilla polk 23.12 +priscilla quirinius 21.826666666666668 +priscilla thompson 20.44 +priscilla underhill 28.23 +priscilla underhill 34.33200000000001 +priscilla van buren 18.122857142857143 +priscilla van buren 20.16 +priscilla van buren 26.447999999999997 +priscilla white 26.37769230769231 +priscilla xylophone 13.95 +priscilla xylophone 20.596666666666668 +priscilla xylophone 27.22 +priscilla young 29.19 +priscilla young 46.28 +priscilla zipper 11.64 +priscilla zipper 31.159999999999997 +quinn allen 26.347272727272728 +quinn allen 26.85833333333333 +quinn brown 26.822857142857146 +quinn brown 30.406000000000006 +quinn brown 41.53 +quinn davidson 17.375714285714288 +quinn davidson 20.22666666666667 +quinn davidson 25.6375 +quinn davidson 30.173333333333332 +quinn ellison 23.052 +quinn ellison 40.565 +quinn garcia 20.544 +quinn garcia 24.104999999999997 +quinn garcia 25.174 +quinn garcia 28.446000000000005 +quinn ichabod 15.12 +quinn king 12.73 +quinn king 15.12125 +quinn laertes 17.29 +quinn laertes 28.221666666666668 +quinn laertes 32.96 +quinn nixon 26.034000000000002 +quinn ovid 28.71 +quinn quirinius 8.61 +quinn robinson 16.852 +quinn steinbeck 30.093333333333334 +quinn steinbeck 49.21 +quinn thompson 7.365 +quinn thompson 33.43125 +quinn underhill 24.045 +quinn underhill 27.905454545454543 +quinn underhill 31.21 +quinn van buren 27.807692307692314 +quinn young 30.56 +quinn zipper 18.31 +quinn zipper 21.380000000000003 +rachel allen 32.501666666666665 +rachel allen 46.57 +rachel brown 23.08 +rachel brown 23.880000000000003 +rachel brown 24.43 +rachel brown 34.11 +rachel brown 35.345 +rachel carson 27.468125 +rachel carson 37.446666666666665 +rachel davidson 22.75 +rachel ellison 22.848333333333333 +rachel falkner 18.78125 +rachel falkner 28.876250000000002 +rachel falkner 29.577777777777776 +rachel falkner 31.831249999999997 +rachel johnson 31.108000000000004 +rachel king 17.4175 +rachel king 30.873749999999998 +rachel laertes 17.470000000000002 +rachel laertes 33.51 +rachel ovid 3.03 +rachel ovid 15.38 +rachel polk 18.564285714285713 +rachel quirinius 31.692500000000003 +rachel robinson 0.6 +rachel robinson 23.953333333333333 +rachel robinson 37.645 +rachel thompson 11.96 +rachel thompson 29.484 +rachel thompson 38.43 +rachel underhill 27.55333333333333 +rachel white 23.511428571428574 +rachel white 33.7 +rachel young 24.85166666666667 +rachel zipper 22.85 +rachel zipper 37.382 +sarah carson 10.38 +sarah carson 22.639 +sarah carson 44.92 +sarah ellison 16.36 +sarah falkner 29.34875 +sarah falkner 29.64125 +sarah garcia 11.296666666666667 +sarah garcia 20.723333333333333 +sarah garcia 24.115 +sarah ichabod 26.948333333333327 +sarah ichabod 33.80428571428571 +sarah johnson 18.3925 +sarah johnson 23.087500000000002 +sarah johnson 26.57857142857143 +sarah johnson 37.01 +sarah king 9.556666666666667 +sarah king 25.6125 +sarah miller 19.14875 +sarah ovid 29.205 +sarah robinson 11.326666666666668 +sarah robinson 35.809999999999995 +sarah steinbeck 23.26 +sarah white 21.75111111111111 +sarah white 26.850000000000005 +sarah xylophone 33.40571428571429 +sarah young 30.66 +sarah zipper 29.521666666666672 +tom brown 16.38 +tom brown 23.645 +tom carson 23.630000000000003 +tom carson 31.935 +tom carson 41.83 +tom davidson 30.404285714285717 +tom ellison 27.056 +tom ellison 27.401999999999997 +tom ellison 29.812 +tom falkner 15.901999999999997 +tom falkner 25.49857142857143 +tom hernandez 11.418000000000001 +tom hernandez 30.705000000000002 +tom ichabod 14.83 +tom johnson 30.748571428571434 +tom johnson 37.086666666666666 +tom king 17.923333333333332 +tom laertes 19.201666666666668 +tom laertes 22.276666666666667 +tom miller 17.9925 +tom miller 19.791666666666668 +tom miller 19.9225 +tom nixon 25.70625 +tom ovid 29.66 +tom polk 27.0975 +tom polk 28.646666666666672 +tom quirinius 37.68333333333333 +tom quirinius 38.28 +tom robinson 18.07 +tom robinson 19.094 +tom robinson 27.34125 +tom robinson 31.135714285714283 +tom steinbeck 32.70333333333333 +tom van buren 20.723333333333333 +tom van buren 24.8525 +tom van buren 31.631666666666664 +tom white 25.646000000000004 +tom young 3.12 +tom young 19.588333333333335 +tom zipper 23.317272727272726 +ulysses brown 16.196666666666665 +ulysses carson 16.3475 +ulysses carson 22.448181818181823 +ulysses carson 28.258 +ulysses carson 32.10833333333333 +ulysses davidson 37.775 +ulysses ellison 30.517000000000003 +ulysses garcia 32.92 +ulysses hernandez 13.877500000000001 +ulysses hernandez 20.856666666666666 +ulysses hernandez 21.32625 +ulysses ichabod 3.29 +ulysses ichabod 24.629999999999995 +ulysses johnson 32.208333333333336 +ulysses king 25.29111111111111 +ulysses laertes 14.936666666666667 +ulysses laertes 25.89 +ulysses laertes 26.63 +ulysses miller 2.36 +ulysses miller 26.403333333333336 +ulysses nixon 34.4575 +ulysses ovid 23.810000000000002 +ulysses polk 22.4075 +ulysses polk 26.778000000000002 +ulysses polk 38.73166666666667 +ulysses polk 47.68 +ulysses quirinius 33.07833333333333 +ulysses robinson 17.386666666666667 +ulysses steinbeck 22.2675 +ulysses steinbeck 24.904000000000003 +ulysses thompson 22.687142857142856 +ulysses underhill 6.66 +ulysses underhill 22.539 +ulysses underhill 24.853333333333335 +ulysses underhill 27.314 +ulysses underhill 29.424999999999997 +ulysses underhill 32.905 +ulysses underhill 41.653333333333336 +ulysses van buren 21.868181818181817 +ulysses white 15.296666666666667 +ulysses white 28.343333333333334 +ulysses xylophone 24.718 +ulysses xylophone 30.205 +ulysses xylophone 35.61 +ulysses young 21.56 +ulysses young 32.28125 +ulysses young 37.275 +victor allen 23.548000000000002 +victor allen 24.759999999999998 +victor brown 22.10181818181818 +victor brown 23.73 +victor brown 25.427272727272726 +victor brown 26.218571428571433 +victor davidson 20.55 +victor davidson 22.21666666666667 +victor davidson 29.778 +victor ellison 13.0775 +victor ellison 33.666 +victor hernandez 10.896 +victor hernandez 18.922 +victor hernandez 24.908888888888892 +victor hernandez 27.426666666666666 +victor hernandez 35.6675 +victor johnson 20.02 +victor johnson 27.070000000000004 +victor johnson 29.0775 +victor king 18.066666666666666 +victor king 21.488 +victor laertes 26.77777777777778 +victor laertes 28.095000000000002 +victor miller 5.3100000000000005 +victor nixon 21.395714285714288 +victor nixon 28.33 +victor ovid 35.225 +victor polk 21.990000000000002 +victor quirinius 24.62833333333333 +victor quirinius 29.742500000000003 +victor robinson 14.575 +victor robinson 25.92 +victor steinbeck 26.136666666666667 +victor steinbeck 26.485 +victor steinbeck 34.745999999999995 +victor thompson 18.735 +victor van buren 27.758333333333336 +victor van buren 37.38333333333333 +victor white 24.607999999999997 +victor white 30.66 +victor xylophone 2.775 +victor xylophone 8.356666666666667 +victor xylophone 24.259999999999998 +victor xylophone 25.636666666666667 +victor xylophone 31.610000000000003 +victor young 22.264444444444443 +victor zipper 39.84 +wendy allen 3.4 +wendy allen 24.695000000000004 +wendy allen 29.912 +wendy brown 28.22 +wendy brown 36.74 +wendy ellison 17.549999999999997 +wendy ellison 22.720000000000002 +wendy falkner 13.765 +wendy falkner 24.424444444444443 +wendy falkner 27.86733333333333 +wendy garcia 12.3 +wendy garcia 22.396666666666665 +wendy garcia 26.8325 +wendy garcia 28.596666666666664 +wendy hernandez 21.111428571428572 +wendy ichabod 4.44 +wendy king 23.654285714285713 +wendy king 29.325714285714287 +wendy king 34.21666666666667 +wendy laertes 31.160714285714285 +wendy laertes 31.46666666666667 +wendy laertes 39.22 +wendy miller 12.73 +wendy miller 30.343333333333334 +wendy nixon 19.92714285714286 +wendy nixon 29.675714285714285 +wendy ovid 21.193749999999998 +wendy ovid 28.49846153846154 +wendy polk 20.94 +wendy polk 22.999999999999996 +wendy quirinius 21.05 +wendy quirinius 26.8425 +wendy robinson 8.39 +wendy robinson 24.05 +wendy robinson 26.974285714285713 +wendy steinbeck 26.765 +wendy thompson 24.14 +wendy thompson 28.995384615384616 +wendy underhill 23.118333333333336 +wendy underhill 25.581666666666667 +wendy underhill 32.985 +wendy van buren 25.151666666666667 +wendy van buren 27.077142857142857 +wendy white 24.4025 +wendy xylophone 22.85181818181818 +wendy xylophone 26.96 +wendy young 4.83 +wendy young 21.325 +xavier allen 19.133333333333333 +xavier allen 26.11466666666667 +xavier allen 34.58 +xavier brown 2.63 +xavier brown 24.764285714285712 +xavier brown 30.166666666666668 +xavier carson 29.006666666666664 +xavier carson 32.106 +xavier davidson 14.094999999999999 +xavier davidson 15.906666666666666 +xavier davidson 27.353333333333335 +xavier ellison 22.174166666666668 +xavier ellison 35.01 +xavier garcia 30.357500000000005 +xavier hernandez 19.87 +xavier hernandez 20.805 +xavier hernandez 33.497499999999995 +xavier ichabod 12.34 +xavier ichabod 26.166249999999998 +xavier johnson 20.33222222222222 +xavier johnson 22.503333333333334 +xavier king 1.3 +xavier king 31.348571428571425 +xavier laertes 7.420000000000001 +xavier ovid 25.576 +xavier polk 11.094285714285714 +xavier polk 19.93 +xavier polk 23.63125 +xavier polk 30.194 +xavier quirinius 13.776666666666666 +xavier quirinius 22.27 +xavier quirinius 24.977692307692312 +xavier quirinius 34.95 +xavier thompson 16.47 +xavier underhill 1.31 +xavier white 19.331666666666667 +xavier white 34.68 +xavier xylophone 21.09625 +xavier zipper 14.89 +yuri allen 18.490000000000002 +yuri allen 22.689999999999998 +yuri brown 15.502857142857142 +yuri brown 22.934285714285714 +yuri carson 27.139999999999997 +yuri carson 35.27 +yuri ellison 10.52 +yuri ellison 25.2025 +yuri falkner 24.633076923076924 +yuri falkner 28.52 +yuri garcia 25.545 +yuri hernandez 16.35 +yuri johnson 19.9525 +yuri johnson 27.636000000000003 +yuri johnson 39.92 +yuri king 15.450000000000001 +yuri laertes 0.41000000000000003 +yuri laertes 33.15 +yuri nixon 27.795 +yuri nixon 39.145 +yuri polk 0.8 +yuri polk 9.705 +yuri polk 25.513333333333332 +yuri quirinius 16.29 +yuri quirinius 19.254999999999995 +yuri quirinius 37.878 +yuri steinbeck 27.6275 +yuri steinbeck 48.89 +yuri thompson 23.330000000000002 +yuri underhill 20.504444444444445 +yuri underhill 21.66 +yuri white 31.205 +yuri xylophone 18.790000000000003 +zach allen 13.06 +zach brown 19.985 +zach brown 26.52333333333333 +zach brown 34.66 +zach brown 34.972857142857144 +zach brown 37.45399999999999 +zach carson 26.195999999999998 +zach ellison 17.55 +zach falkner 3.42 +zach falkner 16.18 +zach garcia 20.062 +zach garcia 25.935 +zach garcia 28.974285714285717 +zach garcia 35.449999999999996 +zach ichabod 10.59 +zach ichabod 31.691999999999997 +zach king 6.81 +zach king 20.817 +zach king 32.542500000000004 +zach miller 13.23 +zach miller 26.30666666666667 +zach miller 26.73 +zach ovid 21.122500000000002 +zach ovid 26.983999999999998 +zach ovid 33.15 +zach ovid 40.59 +zach quirinius 13.38 +zach robinson 20.451999999999998 +zach steinbeck 20.358333333333334 +zach steinbeck 29.65 +zach thompson 16.45 +zach thompson 21.430000000000003 +zach underhill 31.438333333333333 +zach white 23.111428571428572 +zach xylophone 21.221428571428568 +zach xylophone 23.156666666666666 +zach young 24.72666666666667 +zach zipper 19.878888888888884 +zach zipper 34.84571428571429 +zach zipper 35.36 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf2-0-96659fde37d7a38ea15b367b47f59ce2 b/sql/hive/src/test/resources/golden/windowing_udaf2-0-96659fde37d7a38ea15b367b47f59ce2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_udaf2-1-b4bdee4908b1cb8e240c549ae5cfe4c0 b/sql/hive/src/test/resources/golden/windowing_udaf2-1-b4bdee4908b1cb8e240c549ae5cfe4c0 new file mode 100644 index 0000000000000..17c31c0f04592 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf2-1-b4bdee4908b1cb8e240c549ae5cfe4c0 @@ -0,0 +1 @@ +130091 130091 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-0-f498cccf82480be03022d2a36f87651e b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-0-f498cccf82480be03022d2a36f87651e new file mode 100644 index 0000000000000..31b1f85a5eb5a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-0-f498cccf82480be03022d2a36f87651e @@ -0,0 +1,1049 @@ + 4294967354 + 4294967416 + 4294967457 +alice allen 4294967487 +alice allen 4294967525 +alice allen 4294967531 +alice brown 4294967355 +alice carson 4294967370 +alice davidson 4294967517 +alice falkner 4294967316 +alice garcia 4294967369 +alice hernandez 4294967299 +alice hernandez 4294967314 +alice johnson 4294967424 +alice king 4294967387 +alice king 4294967516 +alice king 4294967546 +alice laertes 4294967519 +alice laertes 8589934835 +alice miller 4294967324 +alice nixon 4294967410 +alice nixon 4294967413 +alice nixon 4294967484 +alice ovid 8589934726 +alice polk 4294967366 +alice quirinius 4294967505 +alice quirinius 4294967549 +alice robinson 4294967445 +alice robinson 4294967502 +alice steinbeck 4294967364 +alice steinbeck 4294967474 +alice steinbeck 4294967549 +alice underhill 4294967441 +alice van buren 4294967428 +alice xylophone 4294967363 +alice xylophone 4294967519 +alice xylophone 8589934832 +alice zipper 4294967380 +alice zipper 4294967520 +alice zipper 8589935026 +bob brown 4294967422 +bob brown 4294967427 +bob brown 4294967431 +bob carson 4294967408 +bob davidson 4294967354 +bob davidson 4294967435 +bob davidson 4294967504 +bob ellison 4294967344 +bob ellison 4294967362 +bob ellison 4294967436 +bob ellison 4294967530 +bob falkner 8589934966 +bob garcia 4294967369 +bob garcia 4294967435 +bob garcia 4294967439 +bob garcia 8589934707 +bob garcia 8589934867 +bob hernandez 4294967500 +bob ichabod 4294967424 +bob king 4294967297 +bob king 4294967539 +bob king 8589934870 +bob laertes 4294967380 +bob laertes 4294967472 +bob miller 4294967349 +bob ovid 4294967395 +bob ovid 4294967400 +bob ovid 4294967401 +bob ovid 4294967512 +bob polk 4294967337 +bob quirinius 4294967346 +bob steinbeck 4294967342 +bob van buren 4294967422 +bob white 4294967362 +bob white 4294967493 +bob xylophone 4294967407 +bob xylophone 4294967465 +bob young 4294967413 +bob zipper 4294967299 +bob zipper 8589934723 +bob zipper 8589934840 +calvin allen 12884902208 +calvin brown 4294967411 +calvin brown 4294967437 +calvin brown 4294967530 +calvin carson 8589934876 +calvin davidson 4294967468 +calvin davidson 8589934837 +calvin ellison 4294967480 +calvin falkner 4294967300 +calvin falkner 4294967305 +calvin falkner 4294967345 +calvin falkner 8589934749 +calvin falkner 8589934840 +calvin falkner 8589934978 +calvin garcia 8589934927 +calvin hernandez 12884902173 +calvin johnson 4294967546 +calvin laertes 4294967431 +calvin laertes 4294967499 +calvin nixon 4294967300 +calvin nixon 4294967412 +calvin nixon 4294967488 +calvin ovid 4294967329 +calvin ovid 4294967349 +calvin ovid 8589934723 +calvin ovid 8589934835 +calvin polk 8589934962 +calvin quirinius 4294967521 +calvin quirinius 4294967532 +calvin robinson 4294967326 +calvin steinbeck 4294967474 +calvin steinbeck 4294967505 +calvin steinbeck 8589934722 +calvin thompson 4294967513 +calvin thompson 8589934700 +calvin underhill 4294967478 +calvin van buren 4294967300 +calvin van buren 4294967508 +calvin white 4294967304 +calvin white 8589934924 +calvin xylophone 4294967376 +calvin xylophone 8589934599 +calvin xylophone 8589934866 +calvin young 4294967342 +calvin young 8589934817 +calvin zipper 12884902359 +calvin zipper 17179869649 +david allen 4294967371 +david allen 4294967381 +david brown 8589934762 +david brown 12884902420 +david davidson 4294967522 +david davidson 8589934819 +david davidson 12884902188 +david davidson 12884902327 +david ellison 4294967463 +david ellison 8589934777 +david ellison 12884902263 +david hernandez 4294967324 +david ichabod 4294967487 +david ichabod 12884902220 +david laertes 12884902107 +david nixon 4294967381 +david ovid 4294967396 +david ovid 4294967443 +david quirinius 4294967457 +david quirinius 4294967530 +david quirinius 12884902194 +david robinson 4294967465 +david robinson 17179869575 +david thompson 4294967361 +david underhill 4294967384 +david underhill 8589934942 +david underhill 12884902357 +david van buren 4294967309 +david van buren 8589934901 +david white 4294967428 +david xylophone 4294967479 +david xylophone 4294967480 +david xylophone 8589934856 +david young 4294967296 +david young 4294967305 +ethan allen 4294967351 +ethan brown 4294967320 +ethan brown 4294967331 +ethan brown 4294967403 +ethan brown 4294967420 +ethan brown 8589934797 +ethan brown 8589934805 +ethan carson 4294967352 +ethan ellison 4294967514 +ethan ellison 8589934887 +ethan falkner 4294967318 +ethan falkner 4294967461 +ethan garcia 4294967310 +ethan hernandez 4294967349 +ethan johnson 8589934738 +ethan king 8589934731 +ethan laertes 4294967422 +ethan laertes 4294967531 +ethan laertes 8589934767 +ethan laertes 8589934806 +ethan laertes 8589934830 +ethan laertes 8589934995 +ethan laertes 12884902063 +ethan miller 4294967352 +ethan nixon 8589935019 +ethan ovid 8589934909 +ethan polk 4294967329 +ethan polk 4294967382 +ethan polk 4294967479 +ethan polk 8589935021 +ethan quirinius 4294967348 +ethan quirinius 4294967501 +ethan quirinius 8589934695 +ethan robinson 4294967353 +ethan robinson 8589935019 +ethan underhill 8589934897 +ethan van buren 4294967511 +ethan white 4294967427 +ethan white 8589934975 +ethan xylophone 8589934956 +ethan zipper 4294967462 +ethan zipper 12884902348 +fred davidson 8589934724 +fred davidson 8589934850 +fred davidson 12884902468 +fred ellison 4294967393 +fred ellison 8589934797 +fred ellison 8589934978 +fred falkner 4294967547 +fred falkner 12884902170 +fred falkner 17179869760 +fred hernandez 8589934833 +fred ichabod 8589934853 +fred ichabod 12884902455 +fred johnson 8589934904 +fred king 8589934651 +fred king 8589934951 +fred laertes 8589934883 +fred miller 12884902228 +fred nixon 4294967297 +fred nixon 4294967375 +fred nixon 4294967514 +fred nixon 12884902182 +fred polk 4294967332 +fred polk 4294967458 +fred polk 4294967507 +fred polk 8589934944 +fred quirinius 8589934894 +fred quirinius 12884902335 +fred robinson 8589934904 +fred steinbeck 4294967329 +fred steinbeck 4294967411 +fred steinbeck 4294967472 +fred underhill 4294967387 +fred van buren 8589934830 +fred van buren 12884902319 +fred van buren 12884902382 +fred van buren 17179869836 +fred white 8589934763 +fred young 4294967485 +fred young 8589934832 +fred zipper 12884902371 +gabriella allen 4294967405 +gabriella allen 12884902509 +gabriella brown 4294967403 +gabriella brown 4294967543 +gabriella carson 8589934950 +gabriella davidson 4294967507 +gabriella ellison 4294967393 +gabriella ellison 12884902284 +gabriella falkner 4294967378 +gabriella falkner 4294967523 +gabriella falkner 12884902338 +gabriella garcia 4294967419 +gabriella hernandez 4294967462 +gabriella hernandez 4294967481 +gabriella ichabod 4294967337 +gabriella ichabod 8589934740 +gabriella ichabod 8589934797 +gabriella ichabod 8589934818 +gabriella ichabod 17179869508 +gabriella king 4294967393 +gabriella king 8589934906 +gabriella laertes 4294967410 +gabriella miller 8589934768 +gabriella ovid 4294967522 +gabriella ovid 8589934895 +gabriella polk 4294967302 +gabriella polk 8589934868 +gabriella steinbeck 4294967435 +gabriella steinbeck 4294967500 +gabriella thompson 4294967412 +gabriella thompson 8589934814 +gabriella thompson 12884902318 +gabriella van buren 4294967470 +gabriella van buren 8589934783 +gabriella white 4294967335 +gabriella young 4294967431 +gabriella young 8589934980 +gabriella zipper 4294967510 +gabriella zipper 8589934792 +holly allen 12884901926 +holly brown 8589934722 +holly brown 8589934857 +holly falkner 8589934849 +holly hernandez 8589934749 +holly hernandez 8589934805 +holly hernandez 8589935056 +holly hernandez 12884902485 +holly ichabod 4294967329 +holly ichabod 8589934754 +holly ichabod 8589934981 +holly johnson 4294967535 +holly johnson 12884902194 +holly johnson 17179869874 +holly king 8589934785 +holly king 8589934939 +holly laertes 12884902333 +holly miller 8589934823 +holly nixon 4294967383 +holly nixon 8589934744 +holly polk 4294967434 +holly polk 8589934782 +holly robinson 12884902369 +holly thompson 4294967339 +holly thompson 12884902395 +holly thompson 17179869547 +holly underhill 8589934913 +holly underhill 8589934924 +holly underhill 12884902376 +holly underhill 12884902412 +holly van buren 4294967539 +holly white 17179869548 +holly white 17179869900 +holly xylophone 8589934846 +holly young 4294967500 +holly young 8589934932 +holly zipper 4294967509 +holly zipper 17179869531 +irene allen 12884902413 +irene brown 4294967428 +irene brown 8589934934 +irene brown 12884902207 +irene carson 8589934797 +irene ellison 8589934732 +irene ellison 8589934773 +irene falkner 4294967404 +irene falkner 4294967548 +irene garcia 4294967323 +irene garcia 8589934887 +irene garcia 12884902479 +irene ichabod 4294967509 +irene ichabod 8589934860 +irene johnson 8589934990 +irene laertes 4294967481 +irene laertes 12884902196 +irene laertes 17179869632 +irene miller 4294967387 +irene nixon 4294967538 +irene nixon 12884902129 +irene nixon 12884902324 +irene ovid 8589934764 +irene ovid 8589934886 +irene ovid 8589934903 +irene polk 4294967465 +irene polk 4294967521 +irene polk 8589934672 +irene polk 8589934842 +irene polk 17179869877 +irene quirinius 8589934875 +irene quirinius 12884902269 +irene quirinius 17179869628 +irene robinson 8589934676 +irene steinbeck 4294967549 +irene thompson 4294967479 +irene underhill 8589934694 +irene underhill 12884902077 +irene van buren 8589934932 +irene van buren 12884902202 +irene xylophone 8589934901 +jessica brown 8589934867 +jessica carson 4294967508 +jessica carson 8589934740 +jessica carson 17179869819 +jessica davidson 4294967384 +jessica davidson 8589934864 +jessica davidson 12884902256 +jessica davidson 12884902321 +jessica ellison 4294967316 +jessica ellison 12884902128 +jessica falkner 8589934980 +jessica garcia 4294967540 +jessica garcia 21474837337 +jessica ichabod 8589934816 +jessica johnson 8589935006 +jessica johnson 12884902222 +jessica miller 8589934898 +jessica nixon 8589934742 +jessica nixon 12884902240 +jessica ovid 8589934830 +jessica ovid 12884902307 +jessica polk 21474837163 +jessica quirinius 8589934701 +jessica quirinius 8589934872 +jessica quirinius 12884902159 +jessica quirinius 12884902276 +jessica robinson 4294967542 +jessica thompson 8589934698 +jessica thompson 12884902232 +jessica underhill 8589934810 +jessica underhill 8589934878 +jessica underhill 17179869479 +jessica van buren 8589934726 +jessica white 12884902155 +jessica white 12884902281 +jessica white 12884902296 +jessica white 12884902314 +jessica white 17179869676 +jessica xylophone 17179869697 +jessica young 17179869859 +jessica young 17179869861 +jessica zipper 4294967372 +jessica zipper 8589934727 +jessica zipper 17179869778 +katie allen 8589934791 +katie brown 17179869660 +katie davidson 12884902181 +katie ellison 12884902184 +katie ellison 12884902355 +katie falkner 8589934911 +katie garcia 8589934683 +katie garcia 12884902046 +katie hernandez 8589934812 +katie ichabod 8589934795 +katie ichabod 8589934862 +katie ichabod 8589934869 +katie king 4294967339 +katie king 4294967421 +katie king 8589934826 +katie miller 8589934829 +katie miller 12884902267 +katie nixon 21474837149 +katie ovid 4294967519 +katie polk 8589934726 +katie polk 12884902291 +katie robinson 17179869645 +katie van buren 8589934722 +katie van buren 17179869441 +katie white 4294967306 +katie white 8589934885 +katie xylophone 12884902193 +katie young 8589934819 +katie young 8589935024 +katie young 12884902058 +katie zipper 4294967354 +katie zipper 12884902310 +luke allen 8589934864 +luke allen 8589934931 +luke allen 8589935059 +luke allen 12884902257 +luke allen 12884902322 +luke brown 8589934779 +luke davidson 4294967354 +luke davidson 12884902360 +luke ellison 12884902183 +luke ellison 21474836998 +luke ellison 21474837060 +luke falkner 8589934772 +luke falkner 17179869561 +luke garcia 4294967304 +luke garcia 21474837157 +luke ichabod 12884902150 +luke ichabod 12884902366 +luke johnson 4294967527 +luke johnson 8589934812 +luke johnson 12884902161 +luke laertes 8589935027 +luke laertes 12884902031 +luke laertes 12884902184 +luke laertes 12884902213 +luke laertes 12884902378 +luke miller 8589934826 +luke ovid 4294967492 +luke ovid 8589934913 +luke polk 8589934837 +luke polk 12884902340 +luke quirinius 8589934855 +luke robinson 4294967307 +luke robinson 17179869711 +luke thompson 4294967521 +luke underhill 8589934829 +luke underhill 12884902299 +luke underhill 21474837138 +luke van buren 8589934852 +luke white 12884902418 +luke xylophone 8589934804 +luke zipper 4294967353 +mike allen 17179869750 +mike brown 17179869735 +mike carson 4294967477 +mike carson 8589934803 +mike carson 17179869855 +mike davidson 12884902377 +mike davidson 17179869841 +mike ellison 8589934833 +mike ellison 12884902165 +mike ellison 12884902513 +mike ellison 17179869587 +mike ellison 17179869824 +mike falkner 4294967301 +mike garcia 4294967398 +mike garcia 8589934800 +mike garcia 12884902292 +mike hernandez 8589934824 +mike hernandez 12884902281 +mike ichabod 4294967494 +mike king 4294967347 +mike king 4294967400 +mike king 12884902363 +mike king 12884902475 +mike king 17179869528 +mike king 17179869592 +mike miller 17179869705 +mike nixon 12884902293 +mike nixon 17179869708 +mike polk 17179869752 +mike polk 21474837097 +mike polk 21474837344 +mike quirinius 12884902240 +mike steinbeck 8589934653 +mike steinbeck 12884902273 +mike steinbeck 12884902301 +mike steinbeck 17179869903 +mike van buren 8589934942 +mike van buren 12884902402 +mike white 12884902485 +mike white 17179869676 +mike white 21474836928 +mike white 25769804626 +mike young 8589934704 +mike young 8589934878 +mike young 17179869685 +mike zipper 4294967501 +mike zipper 17179869582 +mike zipper 25769804400 +nick allen 8589934664 +nick allen 8589934860 +nick brown 21474836962 +nick davidson 4294967357 +nick ellison 12884902066 +nick ellison 17179869779 +nick falkner 8589935020 +nick falkner 12884902433 +nick garcia 8589934885 +nick garcia 17179869635 +nick garcia 17179869681 +nick ichabod 12884902193 +nick ichabod 12884902223 +nick ichabod 12884902252 +nick johnson 17179869591 +nick johnson 17179869702 +nick laertes 8589934919 +nick miller 12884902419 +nick nixon 8589934910 +nick ovid 12884902267 +nick polk 17179869712 +nick quirinius 4294967296 +nick quirinius 12884902183 +nick robinson 17179869506 +nick robinson 17179869731 +nick steinbeck 4294967355 +nick thompson 8589934922 +nick underhill 25769804624 +nick van buren 8589934635 +nick xylophone 12884902279 +nick young 12884902399 +nick young 21474837140 +nick zipper 12884902300 +nick zipper 17179869849 +oscar allen 17179869779 +oscar brown 12884902062 +oscar carson 12884902232 +oscar carson 17179869663 +oscar carson 17179869779 +oscar carson 21474837066 +oscar carson 21474837089 +oscar davidson 17179869895 +oscar ellison 4294967304 +oscar ellison 8589934740 +oscar falkner 4294967526 +oscar garcia 21474837156 +oscar hernandez 4294967343 +oscar hernandez 8589935049 +oscar ichabod 8589934837 +oscar ichabod 21474836952 +oscar ichabod 21474837021 +oscar ichabod 25769804491 +oscar johnson 12884902182 +oscar johnson 30064772044 +oscar king 12884902159 +oscar king 17179869738 +oscar king 17179869834 +oscar laertes 4294967550 +oscar laertes 8589934727 +oscar laertes 12884902043 +oscar laertes 12884902478 +oscar nixon 17179869458 +oscar ovid 12884902128 +oscar ovid 12884902240 +oscar ovid 25769804460 +oscar polk 21474836829 +oscar polk 21474837063 +oscar quirinius 8589934728 +oscar quirinius 17179869698 +oscar quirinius 21474837051 +oscar quirinius 25769804521 +oscar robinson 8589934656 +oscar robinson 12884902249 +oscar robinson 21474837105 +oscar robinson 25769804694 +oscar steinbeck 4294967548 +oscar thompson 8589934776 +oscar thompson 12884902164 +oscar thompson 12884902317 +oscar thompson 17179869884 +oscar underhill 8589934895 +oscar van buren 4294967500 +oscar van buren 8589934984 +oscar van buren 21474837205 +oscar white 4294967454 +oscar white 8589934826 +oscar white 21474836931 +oscar white 21474837305 +oscar xylophone 12884902193 +oscar xylophone 12884902307 +oscar xylophone 17179869593 +oscar zipper 8589934865 +oscar zipper 8589934874 +oscar zipper 8589934911 +priscilla brown 8589934848 +priscilla brown 8589935013 +priscilla brown 17179869801 +priscilla carson 12884902145 +priscilla carson 21474836880 +priscilla carson 30064772126 +priscilla ichabod 4294967547 +priscilla ichabod 17179869756 +priscilla johnson 4294967468 +priscilla johnson 8589934667 +priscilla johnson 17179869667 +priscilla johnson 17179869787 +priscilla johnson 25769804279 +priscilla king 12884902153 +priscilla nixon 12884902188 +priscilla nixon 25769804766 +priscilla ovid 12884902234 +priscilla ovid 30064772049 +priscilla polk 17179869480 +priscilla quirinius 12884902171 +priscilla thompson 25769804637 +priscilla underhill 4294967333 +priscilla underhill 17179869740 +priscilla van buren 12884902324 +priscilla van buren 21474837167 +priscilla van buren 21474837343 +priscilla white 4294967419 +priscilla xylophone 8589934792 +priscilla xylophone 12884902245 +priscilla xylophone 12884902287 +priscilla young 21474836992 +priscilla young 34359739656 +priscilla zipper 12884902296 +priscilla zipper 12884902537 +quinn allen 4294967542 +quinn allen 17179869552 +quinn brown 12884902251 +quinn brown 17179869401 +quinn brown 17179869626 +quinn davidson 8589934992 +quinn davidson 17179869690 +quinn davidson 25769804455 +quinn davidson 30064771771 +quinn ellison 12884902376 +quinn ellison 34359739559 +quinn garcia 8589934828 +quinn garcia 12884902387 +quinn garcia 12884902460 +quinn garcia 21474837066 +quinn ichabod 30064772171 +quinn king 4294967458 +quinn king 4294967538 +quinn laertes 8589935080 +quinn laertes 17179869711 +quinn laertes 21474837142 +quinn nixon 17179869672 +quinn ovid 17179869695 +quinn quirinius 21474836827 +quinn robinson 12884902445 +quinn steinbeck 17179869739 +quinn steinbeck 21474836905 +quinn thompson 17179869645 +quinn thompson 25769804317 +quinn underhill 8589934815 +quinn underhill 12884902185 +quinn underhill 30064771762 +quinn van buren 4294967362 +quinn young 8589934731 +quinn zipper 12884902453 +quinn zipper 17179869841 +rachel allen 8589934882 +rachel allen 12884902208 +rachel brown 8589934768 +rachel brown 12884902075 +rachel brown 17179869910 +rachel brown 17179869911 +rachel brown 21474837280 +rachel carson 8589934728 +rachel carson 17179869970 +rachel davidson 30064771666 +rachel ellison 4294967423 +rachel falkner 4294967348 +rachel falkner 12884902482 +rachel falkner 21474837331 +rachel falkner 25769804739 +rachel johnson 38654707197 +rachel king 12884902157 +rachel king 30064771759 +rachel laertes 17179869678 +rachel laertes 25769804379 +rachel ovid 12884902055 +rachel ovid 17179869857 +rachel polk 12884902391 +rachel quirinius 17179869456 +rachel robinson 17179869499 +rachel robinson 17179869703 +rachel robinson 25769804290 +rachel thompson 17179869910 +rachel thompson 21474836989 +rachel thompson 21474837392 +rachel underhill 8589934862 +rachel white 17179869585 +rachel white 21474837039 +rachel young 17179869708 +rachel zipper 4294967434 +rachel zipper 21474837228 +sarah carson 4294967319 +sarah carson 17179869688 +sarah carson 30064772084 +sarah ellison 4294967542 +sarah falkner 17179869797 +sarah falkner 21474837349 +sarah garcia 8589934733 +sarah garcia 8589934858 +sarah garcia 17179869599 +sarah ichabod 12884902196 +sarah ichabod 12884902401 +sarah johnson 12884902455 +sarah johnson 21474836981 +sarah johnson 21474837145 +sarah johnson 25769804480 +sarah king 12884902453 +sarah king 21474837191 +sarah miller 8589934958 +sarah ovid 21474837184 +sarah robinson 21474837237 +sarah robinson 21474837389 +sarah steinbeck 21474837313 +sarah white 17179869905 +sarah white 25769804341 +sarah xylophone 12884902207 +sarah young 21474837319 +sarah zipper 25769804616 +tom brown 8589934894 +tom brown 21474837024 +tom carson 4294967388 +tom carson 12884902278 +tom carson 21474836983 +tom davidson 8589934895 +tom ellison 12884902192 +tom ellison 17179869965 +tom ellison 25769804262 +tom falkner 12884902272 +tom falkner 17179869815 +tom hernandez 4294967296 +tom hernandez 12884902109 +tom ichabod 17179869628 +tom johnson 25769804829 +tom johnson 30064771891 +tom king 12884902390 +tom laertes 12884902181 +tom laertes 12884902236 +tom miller 12884901992 +tom miller 17179869647 +tom miller 21474837107 +tom nixon 17179869677 +tom ovid 12884902279 +tom polk 8589934748 +tom polk 8589934892 +tom quirinius 12884902174 +tom quirinius 21474836986 +tom robinson 8589934753 +tom robinson 12884902203 +tom robinson 12884902358 +tom robinson 21474836952 +tom steinbeck 8589934912 +tom van buren 8589934823 +tom van buren 12884902122 +tom van buren 25769804641 +tom white 21474837076 +tom young 4294967535 +tom young 21474837038 +tom zipper 30064772355 +ulysses brown 8589934991 +ulysses carson 8589934789 +ulysses carson 21474837258 +ulysses carson 25769804457 +ulysses carson 34359739082 +ulysses davidson 12884902216 +ulysses ellison 17179869551 +ulysses garcia 12884902382 +ulysses hernandez 12884902210 +ulysses hernandez 12884902276 +ulysses hernandez 17179869748 +ulysses ichabod 4294967353 +ulysses ichabod 12884902217 +ulysses johnson 21474837122 +ulysses king 8589934995 +ulysses laertes 8589934801 +ulysses laertes 21474837354 +ulysses laertes 25769804499 +ulysses miller 21474837284 +ulysses miller 30064771926 +ulysses nixon 17179869288 +ulysses ovid 17179869754 +ulysses polk 8589934855 +ulysses polk 8589934862 +ulysses polk 12884902420 +ulysses polk 17179869479 +ulysses quirinius 17179869659 +ulysses robinson 4294967531 +ulysses steinbeck 8589935027 +ulysses steinbeck 21474837100 +ulysses thompson 12884902194 +ulysses underhill 8589934760 +ulysses underhill 8589934799 +ulysses underhill 12884902240 +ulysses underhill 17179869759 +ulysses underhill 17179869760 +ulysses underhill 17179869939 +ulysses underhill 21474837264 +ulysses van buren 8589934938 +ulysses white 25769804453 +ulysses white 30064772086 +ulysses xylophone 8589935029 +ulysses xylophone 12884902249 +ulysses xylophone 25769804765 +ulysses young 4294967427 +ulysses young 17179869391 +ulysses young 30064771844 +victor allen 8589934793 +victor allen 12884902264 +victor brown 4294967455 +victor brown 17179869657 +victor brown 21474837426 +victor brown 30064771922 +victor davidson 17179869715 +victor davidson 17179869872 +victor davidson 25769804287 +victor ellison 17179869611 +victor ellison 17179869709 +victor hernandez 8589934847 +victor hernandez 12884902463 +victor hernandez 17179869647 +victor hernandez 17179869720 +victor hernandez 25769804310 +victor johnson 17179869652 +victor johnson 21474837148 +victor johnson 25769804771 +victor king 8589934917 +victor king 25769804714 +victor laertes 12884902188 +victor laertes 21474837186 +victor miller 21474837170 +victor nixon 8589934778 +victor nixon 12884902261 +victor ovid 12884902350 +victor polk 17179869376 +victor quirinius 21474837074 +victor quirinius 21474837279 +victor robinson 21474836948 +victor robinson 21474837097 +victor steinbeck 12884902162 +victor steinbeck 17179869721 +victor steinbeck 21474836916 +victor thompson 25769804395 +victor van buren 21474837010 +victor van buren 25769804601 +victor white 8589934816 +victor white 30064771798 +victor xylophone 17179869560 +victor xylophone 25769804719 +victor xylophone 25769804760 +victor xylophone 34359739093 +victor xylophone 34359739095 +victor young 21474837052 +victor zipper 12884902345 +wendy allen 21474837127 +wendy allen 25769804525 +wendy allen 25769804732 +wendy brown 12884902342 +wendy brown 21474836889 +wendy ellison 12884902392 +wendy ellison 21474836763 +wendy falkner 8589934926 +wendy falkner 17179869470 +wendy falkner 25769804816 +wendy garcia 17179869439 +wendy garcia 17179869732 +wendy garcia 30064771654 +wendy garcia 30064771704 +wendy hernandez 17179869752 +wendy ichabod 17179869547 +wendy king 17179869612 +wendy king 21474837301 +wendy king 30064772042 +wendy laertes 8589934872 +wendy laertes 12884902469 +wendy laertes 21474837084 +wendy miller 17179869661 +wendy miller 17179869682 +wendy nixon 12884902521 +wendy nixon 21474836846 +wendy ovid 21474837025 +wendy ovid 38654706512 +wendy polk 8589934960 +wendy polk 21474837144 +wendy quirinius 12884902263 +wendy quirinius 17179869652 +wendy robinson 21474837104 +wendy robinson 25769804321 +wendy robinson 25769804728 +wendy steinbeck 12884902299 +wendy thompson 17179869494 +wendy thompson 21474837072 +wendy underhill 17179869898 +wendy underhill 21474837064 +wendy underhill 25769804845 +wendy van buren 25769804447 +wendy van buren 25769804679 +wendy white 17179869866 +wendy xylophone 17179869596 +wendy xylophone 25769804554 +wendy young 4294967313 +wendy young 25769804562 +xavier allen 12884902364 +xavier allen 17179869960 +xavier allen 21474836864 +xavier brown 8589934824 +xavier brown 17179869646 +xavier brown 25769804653 +xavier carson 17179869770 +xavier carson 21474837445 +xavier davidson 30064772118 +xavier davidson 34359739403 +xavier davidson 38654706539 +xavier ellison 34359739490 +xavier ellison 34359739559 +xavier garcia 21474837142 +xavier hernandez 21474837012 +xavier hernandez 25769804421 +xavier hernandez 38654707021 +xavier ichabod 12884902315 +xavier ichabod 17179869567 +xavier johnson 8589934922 +xavier johnson 38654707066 +xavier king 12884902272 +xavier king 21474836962 +xavier laertes 17179869795 +xavier ovid 17179869597 +xavier polk 12884902254 +xavier polk 17179869581 +xavier polk 17179869743 +xavier polk 34359739344 +xavier quirinius 12884902240 +xavier quirinius 21474836996 +xavier quirinius 25769804437 +xavier quirinius 25769804456 +xavier thompson 17179869822 +xavier underhill 8589934813 +xavier white 12884902262 +xavier white 12884902366 +xavier xylophone 17179869722 +xavier zipper 12884902377 +yuri allen 8589935035 +yuri allen 12884902279 +yuri brown 8589934912 +yuri brown 12884902319 +yuri carson 21474837146 +yuri carson 25769804245 +yuri ellison 25769804504 +yuri ellison 25769804568 +yuri falkner 25769804699 +yuri falkner 42949674720 +yuri garcia 4294967362 +yuri hernandez 21474837117 +yuri johnson 21474837002 +yuri johnson 21474837165 +yuri johnson 25769804545 +yuri king 30064772090 +yuri laertes 30064772076 +yuri laertes 34359739328 +yuri nixon 12884902232 +yuri nixon 12884902265 +yuri polk 12884902362 +yuri polk 21474837245 +yuri polk 25769804539 +yuri quirinius 12884902198 +yuri quirinius 17179869606 +yuri quirinius 30064771819 +yuri steinbeck 4294967535 +yuri steinbeck 8589934657 +yuri thompson 12884902467 +yuri underhill 17179869566 +yuri underhill 17179869715 +yuri white 34359739045 +yuri xylophone 12884902412 +zach allen 17179869908 +zach brown 21474836879 +zach brown 21474836891 +zach brown 21474837040 +zach brown 21474837073 +zach brown 30064771852 +zach carson 21474837185 +zach ellison 8589934898 +zach falkner 17179869807 +zach falkner 25769804634 +zach garcia 17179869536 +zach garcia 21474837142 +zach garcia 30064772246 +zach garcia 34359739192 +zach ichabod 17179869613 +zach ichabod 17179869838 +zach king 17179869700 +zach king 21474837427 +zach king 34359739578 +zach miller 4294967391 +zach miller 12884902310 +zach miller 17179869709 +zach ovid 17179869731 +zach ovid 21474837032 +zach ovid 21474837127 +zach ovid 30064771625 +zach quirinius 34359739151 +zach robinson 21474836938 +zach steinbeck 17179869667 +zach steinbeck 25769804623 +zach thompson 12884902354 +zach thompson 17179869659 +zach underhill 12884902149 +zach white 25769804490 +zach xylophone 12884902198 +zach xylophone 21474837163 +zach young 17179869687 +zach zipper 17179869708 +zach zipper 17179869834 +zach zipper 21474837369 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-1-6378faf36ffd3f61e61cee6c0cb70e6 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-1-6378faf36ffd3f61e61cee6c0cb70e6 new file mode 100644 index 0000000000000..1436509e4ec17 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-1-6378faf36ffd3f61e61cee6c0cb70e6 @@ -0,0 +1,1049 @@ + 9.220000267028809 + 43.72999954223633 + 89.52999877929688 +alice allen 2.7899999618530273 +alice allen 21.450000762939453 +alice allen 73.62999725341797 +alice brown 71.30999755859375 +alice carson 39.029998779296875 +alice davidson 70.3499984741211 +alice falkner 90.25 +alice garcia 48.45000076293945 +alice hernandez 88.16999816894531 +alice hernandez 90.55999755859375 +alice johnson 4.46999979019165 +alice king 19.139999389648438 +alice king 23.170000076293945 +alice king 52.22999954223633 +alice laertes 68.94999694824219 +alice laertes 69.52999877929688 +alice miller 68.95999908447266 +alice nixon 40.0 +alice nixon 48.150001525878906 +alice nixon 79.83000183105469 +alice ovid 9.039999961853027 +alice polk 62.900001525878906 +alice quirinius 37.13999938964844 +alice quirinius 62.29999923706055 +alice robinson 5.079999923706055 +alice robinson 56.099998474121094 +alice steinbeck 38.619998931884766 +alice steinbeck 55.5099983215332 +alice steinbeck 92.37000274658203 +alice underhill 98.18000030517578 +alice van buren 38.939998626708984 +alice xylophone 33.58000183105469 +alice xylophone 43.15999984741211 +alice xylophone 78.20999908447266 +alice zipper 26.43000030517578 +alice zipper 42.47999954223633 +alice zipper 89.93000030517578 +bob brown 8.069999694824219 +bob brown 70.93000030517578 +bob brown 93.08999633789062 +bob carson 50.09000015258789 +bob davidson 1.2899999618530273 +bob davidson 53.93000030517578 +bob davidson 74.72000122070312 +bob ellison 41.34000015258789 +bob ellison 65.0199966430664 +bob ellison 75.02999877929688 +bob ellison 80.30000305175781 +bob falkner 16.989999771118164 +bob garcia 4.460000038146973 +bob garcia 5.400000095367432 +bob garcia 45.59000015258789 +bob garcia 80.30000305175781 +bob garcia 87.56999969482422 +bob hernandez 22.68000030517578 +bob ichabod 82.55999755859375 +bob king 8.789999961853027 +bob king 12.539999961853027 +bob king 39.0099983215332 +bob laertes 0.7900000214576721 +bob laertes 10.670000076293945 +bob miller 61.91999816894531 +bob ovid 46.86000061035156 +bob ovid 62.849998474121094 +bob ovid 88.77999877929688 +bob ovid 97.08999633789062 +bob polk 7.980000019073486 +bob quirinius 46.099998474121094 +bob steinbeck 9.699999809265137 +bob van buren 33.66999816894531 +bob white 45.34000015258789 +bob white 45.349998474121094 +bob xylophone 19.690000534057617 +bob xylophone 107.93000221252441 +bob young 35.16999816894531 +bob zipper 1.25 +bob zipper 3.819999933242798 +bob zipper 34.349998474121094 +calvin allen 63.119998931884766 +calvin brown 28.110000610351562 +calvin brown 85.9000015258789 +calvin brown 90.19999694824219 +calvin carson 80.2300033569336 +calvin davidson 31.770000457763672 +calvin davidson 85.51000213623047 +calvin ellison 26.489999771118164 +calvin falkner 2.9700000286102295 +calvin falkner 56.040000915527344 +calvin falkner 56.33000183105469 +calvin falkner 80.5999984741211 +calvin falkner 93.61000061035156 +calvin falkner 94.30999755859375 +calvin garcia 41.849998474121094 +calvin hernandez 33.869998931884766 +calvin johnson 66.61000061035156 +calvin laertes 23.1299991607666 +calvin laertes 50.310001373291016 +calvin nixon 9.8100004196167 +calvin nixon 41.20000076293945 +calvin nixon 69.73999786376953 +calvin ovid 69.95999908447266 +calvin ovid 71.26000213623047 +calvin ovid 79.12000274658203 +calvin ovid 84.72000122070312 +calvin polk 65.72000122070312 +calvin quirinius 29.540000915527344 +calvin quirinius 53.02000045776367 +calvin robinson 40.439998626708984 +calvin steinbeck 15.220000267028809 +calvin steinbeck 22.850000381469727 +calvin steinbeck 93.30000305175781 +calvin thompson 8.90999984741211 +calvin thompson 93.7300033569336 +calvin underhill 59.70000076293945 +calvin van buren 34.209999084472656 +calvin van buren 64.0 +calvin white 50.279998779296875 +calvin white 90.69000244140625 +calvin xylophone 21.700000762939453 +calvin xylophone 25.420000076293945 +calvin xylophone 56.810001373291016 +calvin young 24.489999771118164 +calvin young 39.810001373291016 +calvin zipper 9.1899995803833 +calvin zipper 95.37999725341797 +david allen 51.25 +david allen 64.87000274658203 +david brown 3.2100000381469727 +david brown 93.63999938964844 +david davidson 1.0800000429153442 +david davidson 62.720001220703125 +david davidson 74.1500015258789 +david davidson 95.80999755859375 +david ellison 47.689998626708984 +david ellison 85.2300033569336 +david ellison 94.1500015258789 +david hernandez 99.91000366210938 +david ichabod 5.28000020980835 +david ichabod 82.55000305175781 +david laertes 76.70999908447266 +david nixon 50.31999969482422 +david ovid 25.110000610351562 +david ovid 61.70000076293945 +david quirinius 20.639999389648438 +david quirinius 29.239999771118164 +david quirinius 79.97000122070312 +david robinson 25.280000686645508 +david robinson 161.1199951171875 +david thompson 80.89999771118164 +david underhill 8.319999694824219 +david underhill 88.7699966430664 +david underhill 97.55999755859375 +david van buren 83.56999969482422 +david van buren 115.43999862670898 +david white 45.189998626708984 +david xylophone 8.069999694824219 +david xylophone 54.34000015258789 +david xylophone 72.9800033569336 +david young 10.25 +david young 35.650001525878906 +ethan allen 32.75 +ethan brown 7.110000133514404 +ethan brown 10.09000015258789 +ethan brown 15.630000114440918 +ethan brown 61.86000061035156 +ethan brown 73.18000030517578 +ethan brown 82.30000305175781 +ethan carson 76.33000183105469 +ethan ellison 0.2800000011920929 +ethan ellison 81.47000122070312 +ethan falkner 50.02000045776367 +ethan falkner 59.43000030517578 +ethan garcia 43.189998626708984 +ethan hernandez 49.779998779296875 +ethan johnson 90.05000305175781 +ethan king 4.349999904632568 +ethan laertes 15.449999809265137 +ethan laertes 54.75 +ethan laertes 59.209999084472656 +ethan laertes 70.38999938964844 +ethan laertes 80.70999908447266 +ethan laertes 95.06999969482422 +ethan laertes 96.29000091552734 +ethan miller 25.3700008392334 +ethan nixon 37.779998779296875 +ethan ovid 57.290000915527344 +ethan polk 2.3499999046325684 +ethan polk 21.31999969482422 +ethan polk 23.440000534057617 +ethan polk 122.71999740600586 +ethan quirinius 3.859999895095825 +ethan quirinius 51.84000015258789 +ethan quirinius 97.23999786376953 +ethan robinson 67.94000244140625 +ethan robinson 78.62000274658203 +ethan underhill 55.630001068115234 +ethan van buren 36.70000076293945 +ethan white 60.849998474121094 +ethan white 63.41999816894531 +ethan xylophone 57.11000061035156 +ethan zipper 2.9200000762939453 +ethan zipper 97.51000213623047 +fred davidson 18.860000610351562 +fred davidson 37.2400016784668 +fred davidson 78.30999755859375 +fred ellison 31.179998874664307 +fred ellison 48.59000015258789 +fred ellison 96.77999877929688 +fred falkner 10.289999961853027 +fred falkner 72.04000091552734 +fred falkner 85.0 +fred hernandez 55.9900016784668 +fred ichabod 47.359999656677246 +fred ichabod 81.31999969482422 +fred johnson 96.08999633789062 +fred king 48.369998931884766 +fred king 72.13999843597412 +fred laertes 57.63999938964844 +fred miller 46.970001220703125 +fred nixon 28.690000534057617 +fred nixon 38.04999923706055 +fred nixon 70.5199966430664 +fred nixon 93.02999877929688 +fred polk 23.959999084472656 +fred polk 39.18000030517578 +fred polk 47.31999969482422 +fred polk 90.12000274658203 +fred quirinius 15.300000190734863 +fred quirinius 29.399999618530273 +fred robinson 89.02999877929688 +fred steinbeck 32.22999954223633 +fred steinbeck 41.310001373291016 +fred steinbeck 91.05000305175781 +fred underhill 90.7699966430664 +fred van buren 1.0199999809265137 +fred van buren 21.940000534057617 +fred van buren 52.869998931884766 +fred van buren 83.58000183105469 +fred white 37.79999923706055 +fred young 46.79999923706055 +fred young 97.70999908447266 +fred zipper 29.020000457763672 +gabriella allen 46.27000045776367 +gabriella allen 64.22000122070312 +gabriella brown 15.260000228881836 +gabriella brown 84.83000183105469 +gabriella carson 42.7599983215332 +gabriella davidson 6.550000190734863 +gabriella ellison 48.08000183105469 +gabriella ellison 71.54000091552734 +gabriella falkner 10.170000076293945 +gabriella falkner 51.720001220703125 +gabriella falkner 87.61000061035156 +gabriella garcia 43.0099983215332 +gabriella hernandez 76.91999816894531 +gabriella hernandez 92.9800033569336 +gabriella ichabod 10.729999542236328 +gabriella ichabod 26.639999389648438 +gabriella ichabod 66.36000061035156 +gabriella ichabod 71.12999725341797 +gabriella ichabod 90.3499984741211 +gabriella king 20.670000076293945 +gabriella king 80.45999908447266 +gabriella laertes 65.37999725341797 +gabriella miller 50.83000183105469 +gabriella ovid 77.7400016784668 +gabriella ovid 92.4000015258789 +gabriella polk 35.68000030517578 +gabriella polk 88.05000305175781 +gabriella steinbeck 46.45000076293945 +gabriella steinbeck 78.63999938964844 +gabriella thompson 73.31999969482422 +gabriella thompson 88.36000061035156 +gabriella thompson 94.25 +gabriella van buren 69.80000305175781 +gabriella van buren 70.05999755859375 +gabriella white 55.18000030517578 +gabriella young 9.25 +gabriella young 59.709999084472656 +gabriella zipper 36.2599983215332 +gabriella zipper 91.62999725341797 +holly allen 44.56999969482422 +holly brown 77.80999755859375 +holly brown 78.7300033569336 +holly falkner 80.73999786376953 +holly hernandez 20.81999969482422 +holly hernandez 21.190000534057617 +holly hernandez 24.790000915527344 +holly hernandez 30.25 +holly ichabod 83.2699966430664 +holly ichabod 84.69000244140625 +holly ichabod 90.51000213623047 +holly johnson 36.95000076293945 +holly johnson 64.36000061035156 +holly johnson 65.62000274658203 +holly king 42.310001373291016 +holly king 55.38999938964844 +holly laertes 52.5 +holly miller 50.400001525878906 +holly nixon 53.779998779296875 +holly nixon 88.0199966430664 +holly polk 3.619999885559082 +holly polk 98.30999755859375 +holly robinson 69.31999969482422 +holly thompson 0.07999999821186066 +holly thompson 86.69000244140625 +holly thompson 145.93999481201172 +holly underhill 42.54999923706055 +holly underhill 50.40999984741211 +holly underhill 79.95999908447266 +holly underhill 96.68000030517578 +holly van buren 68.80999755859375 +holly white 7.960000038146973 +holly white 32.91999816894531 +holly xylophone 93.11000061035156 +holly young 60.220001220703125 +holly young 66.16999816894531 +holly zipper 99.12999725341797 +holly zipper 99.29000091552734 +irene allen 38.849998474121094 +irene brown 4.789999961853027 +irene brown 53.939998626708984 +irene brown 87.66999816894531 +irene carson 94.54000091552734 +irene ellison 45.2400016784668 +irene ellison 50.08000183105469 +irene falkner 22.079999923706055 +irene falkner 99.91999816894531 +irene garcia 15.369999885559082 +irene garcia 58.43000030517578 +irene garcia 86.93000030517578 +irene ichabod 41.439998626708984 +irene ichabod 99.62000274658203 +irene johnson 5.880000114440918 +irene laertes 9.569999694824219 +irene laertes 42.66999816894531 +irene laertes 44.43000030517578 +irene miller 65.44000244140625 +irene nixon 15.100000381469727 +irene nixon 29.780000686645508 +irene nixon 42.560001373291016 +irene ovid 5.239999771118164 +irene ovid 35.130001068115234 +irene ovid 79.75 +irene polk 0.9800000190734863 +irene polk 24.020000457763672 +irene polk 42.2400016784668 +irene polk 47.08000183105469 +irene polk 95.83999633789062 +irene quirinius 12.899999618530273 +irene quirinius 58.86000061035156 +irene quirinius 70.0 +irene robinson 94.2699966430664 +irene steinbeck 94.33000183105469 +irene thompson 78.30000305175781 +irene underhill 28.309999465942383 +irene underhill 57.349998474121094 +irene van buren 54.439998626708984 +irene van buren 54.9900016784668 +irene xylophone 74.19000244140625 +jessica brown 51.290000915527344 +jessica carson 25.549999237060547 +jessica carson 31.860000610351562 +jessica carson 62.20000076293945 +jessica davidson 33.54999923706055 +jessica davidson 49.77000045776367 +jessica davidson 95.33999633789062 +jessica davidson 99.20999908447266 +jessica ellison 11.180000305175781 +jessica ellison 22.780000686645508 +jessica falkner 99.6500015258789 +jessica garcia 5.539999961853027 +jessica garcia 87.92999941110611 +jessica ichabod 59.15999984741211 +jessica johnson 9.5600004196167 +jessica johnson 40.79999923706055 +jessica miller 151.0199966430664 +jessica nixon 77.0999984741211 +jessica nixon 90.06999969482422 +jessica ovid 71.68000030517578 +jessica ovid 119.9000015258789 +jessica polk 49.68000030517578 +jessica quirinius 22.940000534057617 +jessica quirinius 32.470001220703125 +jessica quirinius 35.619998931884766 +jessica quirinius 46.869998931884766 +jessica robinson 112.36000442504883 +jessica thompson 38.33000183105469 +jessica thompson 89.55000305175781 +jessica underhill 26.079999923706055 +jessica underhill 45.41999816894531 +jessica underhill 46.209999084472656 +jessica van buren 9.739999771118164 +jessica white 11.550000190734863 +jessica white 36.58000183105469 +jessica white 73.93000030517578 +jessica white 74.30000305175781 +jessica white 96.62000274658203 +jessica xylophone 53.060001373291016 +jessica young 11.1899995803833 +jessica young 43.369998931884766 +jessica zipper 6.630000114440918 +jessica zipper 12.020000457763672 +jessica zipper 92.43999862670898 +katie allen 64.66999816894531 +katie brown 27.719999313354492 +katie davidson 170.84000396728516 +katie ellison 3.609999895095825 +katie ellison 80.97000122070312 +katie falkner 18.5 +katie garcia 24.729999542236328 +katie garcia 84.4000015258789 +katie hernandez 38.61999988555908 +katie ichabod 30.709999084472656 +katie ichabod 39.97999954223633 +katie ichabod 43.16999816894531 +katie king 39.34000015258789 +katie king 39.83000183105469 +katie king 97.80999755859375 +katie miller 31.399999618530273 +katie miller 74.77999877929688 +katie nixon 121.3700008392334 +katie ovid 50.65999984741211 +katie polk 11.680000305175781 +katie polk 40.2400016784668 +katie robinson 13.890000343322754 +katie van buren 17.739999771118164 +katie van buren 52.529998779296875 +katie white 1.309999942779541 +katie white 34.72999954223633 +katie xylophone 14.130000114440918 +katie young 31.010000228881836 +katie young 72.51000213623047 +katie young 97.56999969482422 +katie zipper 18.93000030517578 +katie zipper 58.75 +luke allen 15.180000305175781 +luke allen 50.959999084472656 +luke allen 66.61000061035156 +luke allen 89.55000305175781 +luke allen 99.38999938964844 +luke brown 51.790000915527344 +luke davidson 7.050000190734863 +luke davidson 28.950000762939453 +luke ellison 1.8700000047683716 +luke ellison 16.25 +luke ellison 87.83000183105469 +luke falkner 32.25 +luke falkner 39.60000038146973 +luke garcia 13.350000381469727 +luke garcia 30.3700008392334 +luke ichabod 8.449999809265137 +luke ichabod 97.87000274658203 +luke johnson 11.149999618530273 +luke johnson 14.4399995803833 +luke johnson 31.670000076293945 +luke laertes 0.5199999809265137 +luke laertes 4.800000190734863 +luke laertes 11.819999694824219 +luke laertes 16.690000534057617 +luke laertes 45.9900016784668 +luke miller 97.6500015258789 +luke ovid 38.04999923706055 +luke ovid 159.68000030517578 +luke polk 46.880001068115234 +luke polk 95.27999877929688 +luke quirinius 40.41999816894531 +luke robinson 55.099998474121094 +luke robinson 65.69999694824219 +luke thompson 94.37999725341797 +luke underhill 59.68000030517578 +luke underhill 95.52999877929688 +luke underhill 96.94000244140625 +luke van buren 148.62999725341797 +luke white 67.12000274658203 +luke xylophone 48.279998779296875 +luke zipper 24.829999923706055 +mike allen 48.53999900817871 +mike brown 48.22999954223633 +mike carson 20.06999969482422 +mike carson 47.56999969482422 +mike carson 81.66000366210938 +mike davidson 27.309999465942383 +mike davidson 54.83000183105469 +mike ellison 28.559999465942383 +mike ellison 37.099998474121094 +mike ellison 62.13999938964844 +mike ellison 79.37999725341797 +mike ellison 85.73999786376953 +mike falkner 16.479999542236328 +mike garcia 70.8499984741211 +mike garcia 75.83000183105469 +mike garcia 79.20999908447266 +mike hernandez 37.900001525878906 +mike hernandez 59.45000076293945 +mike ichabod 64.7699966430664 +mike king 38.790000915527344 +mike king 62.7400016784668 +mike king 78.26000213623047 +mike king 84.2300033569336 +mike king 85.0999984741211 +mike king 94.68000030517578 +mike miller 3.9600000381469727 +mike nixon 60.119998931884766 +mike nixon 92.95999908447266 +mike polk 12.449999809265137 +mike polk 27.06999969482422 +mike polk 99.68000030517578 +mike quirinius 89.37999725341797 +mike steinbeck 5.849999904632568 +mike steinbeck 85.13999938964844 +mike steinbeck 93.07000207901001 +mike steinbeck 97.45999908447266 +mike van buren 80.83999633789062 +mike van buren 114.56999969482422 +mike white 9.569999694824219 +mike white 28.889999389648438 +mike white 32.0099983215332 +mike white 91.87999725341797 +mike young 7.820000171661377 +mike young 74.58999633789062 +mike young 83.54000091552734 +mike zipper 26.729999542236328 +mike zipper 83.91999816894531 +mike zipper 97.38999938964844 +nick allen 21.830000400543213 +nick allen 35.08000183105469 +nick brown 42.5099983215332 +nick davidson 49.439998626708984 +nick ellison 9.680000305175781 +nick ellison 89.01000213623047 +nick falkner 10.130000114440918 +nick falkner 88.47000122070312 +nick garcia 13.9399995803833 +nick garcia 26.389999389648438 +nick garcia 46.43000030517578 +nick ichabod 23.450000762939453 +nick ichabod 47.59000015258789 +nick ichabod 74.41999816894531 +nick johnson 3.9700000286102295 +nick johnson 94.08000183105469 +nick laertes 96.25 +nick miller 82.97000122070312 +nick nixon 96.37999725341797 +nick ovid 87.98999786376953 +nick polk 59.27000141143799 +nick quirinius 67.44999694824219 +nick quirinius 81.16999816894531 +nick robinson 57.66999816894531 +nick robinson 60.709999084472656 +nick steinbeck 97.83000183105469 +nick thompson 11.90999984741211 +nick underhill 20.809999465942383 +nick van buren 51.290000915527344 +nick xylophone 103.45999908447266 +nick young 0.27000001072883606 +nick young 24.799999237060547 +nick zipper 56.619998931884766 +nick zipper 119.0199966430664 +oscar allen 18.6299991607666 +oscar brown 13.100000381469727 +oscar carson 6.869999885559082 +oscar carson 55.20000076293945 +oscar carson 78.9800033569336 +oscar carson 87.4800033569336 +oscar carson 98.51000213623047 +oscar davidson 64.45999908447266 +oscar ellison 57.88999938964844 +oscar ellison 107.7100019454956 +oscar falkner 98.4800033569336 +oscar garcia 67.4800033569336 +oscar hernandez 95.4800033569336 +oscar hernandez 125.92999649047852 +oscar ichabod 3.3299999237060547 +oscar ichabod 33.52000045776367 +oscar ichabod 71.80000305175781 +oscar ichabod 76.69000244140625 +oscar johnson 16.09000015258789 +oscar johnson 139.69000244140625 +oscar king 19.059999465942383 +oscar king 25.8799991607666 +oscar king 59.5 +oscar laertes 5.510000228881836 +oscar laertes 8.420000076293945 +oscar laertes 9.260000228881836 +oscar laertes 27.1200008392334 +oscar nixon 41.619998931884766 +oscar ovid 37.13999938964844 +oscar ovid 82.23999786376953 +oscar ovid 91.52999877929688 +oscar polk 30.610000610351562 +oscar polk 63.900001525878906 +oscar quirinius 41.45000076293945 +oscar quirinius 65.43000030517578 +oscar quirinius 113.35000228881836 +oscar quirinius 139.10000610351562 +oscar robinson 11.34000015258789 +oscar robinson 42.849998474121094 +oscar robinson 74.52999877929688 +oscar robinson 131.31999969482422 +oscar steinbeck 29.59000015258789 +oscar thompson 31.90999984741211 +oscar thompson 41.34000015258789 +oscar thompson 60.529998779296875 +oscar thompson 70.88999938964844 +oscar underhill 87.4000015258789 +oscar van buren 2.180000066757202 +oscar van buren 61.880001068115234 +oscar van buren 91.77999877929688 +oscar white 19.0 +oscar white 28.450000762939453 +oscar white 51.849998474121094 +oscar white 59.83000183105469 +oscar xylophone 21.799999237060547 +oscar xylophone 57.119998931884766 +oscar xylophone 57.22999954223633 +oscar zipper 13.989999771118164 +oscar zipper 32.88999938964844 +oscar zipper 39.81999969482422 +priscilla brown 70.23999786376953 +priscilla brown 80.5199966430664 +priscilla brown 104.63999938964844 +priscilla carson 7.960000038146973 +priscilla carson 79.80999946594238 +priscilla carson 85.43000316619873 +priscilla ichabod 80.04000091552734 +priscilla ichabod 92.61000061035156 +priscilla johnson 61.939998626708984 +priscilla johnson 67.9800033569336 +priscilla johnson 68.32999992370605 +priscilla johnson 91.4800033569336 +priscilla johnson 92.48000144958496 +priscilla king 43.91999816894531 +priscilla nixon 95.80999755859375 +priscilla nixon 107.69000244140625 +priscilla ovid 52.72999954223633 +priscilla ovid 125.73999643325806 +priscilla polk 15.149999618530273 +priscilla quirinius 9.710000038146973 +priscilla thompson 9.800000190734863 +priscilla underhill 35.720001220703125 +priscilla underhill 68.22000122070312 +priscilla van buren 68.88999938964844 +priscilla van buren 91.61000061035156 +priscilla van buren 170.5500030517578 +priscilla white 78.27999877929688 +priscilla xylophone 0.15000000596046448 +priscilla xylophone 21.489999771118164 +priscilla xylophone 59.61000061035156 +priscilla young 0.4300000071525574 +priscilla young 4.320000171661377 +priscilla zipper 18.6299991607666 +priscilla zipper 25.670000076293945 +quinn allen 54.72999954223633 +quinn allen 83.33000183105469 +quinn brown 24.280000686645508 +quinn brown 52.439998626708984 +quinn brown 80.58000183105469 +quinn davidson 61.57999849319458 +quinn davidson 67.18000030517578 +quinn davidson 83.4000015258789 +quinn davidson 95.11000061035156 +quinn ellison 19.280000686645508 +quinn ellison 30.649999618530273 +quinn garcia 40.97999954223633 +quinn garcia 59.9900016784668 +quinn garcia 74.0199966430664 +quinn garcia 172.8499984741211 +quinn ichabod 36.790000915527344 +quinn king 74.62000274658203 +quinn king 86.2300033569336 +quinn laertes 4.710000038146973 +quinn laertes 41.290000915527344 +quinn laertes 76.5199966430664 +quinn nixon 86.64000129699707 +quinn ovid 52.500000953674316 +quinn quirinius 32.18000030517578 +quinn robinson 38.64999866485596 +quinn steinbeck 8.449999809265137 +quinn steinbeck 66.51000213623047 +quinn thompson 74.9399995803833 +quinn thompson 76.27999877929688 +quinn underhill 17.15999984741211 +quinn underhill 79.4800033569336 +quinn underhill 140.92000198364258 +quinn van buren 82.5199966430664 +quinn young 45.060001373291016 +quinn zipper 22.25 +quinn zipper 58.0 +rachel allen 15.8100004196167 +rachel allen 74.44999694824219 +rachel brown 2.9600000381469727 +rachel brown 30.809999465942383 +rachel brown 33.36000061035156 +rachel brown 34.40999984741211 +rachel brown 52.16999816894531 +rachel carson 37.599998474121094 +rachel carson 98.95999908447266 +rachel davidson 4.920000076293945 +rachel ellison 10.600000381469727 +rachel falkner 46.150001525878906 +rachel falkner 80.91999816894531 +rachel falkner 88.80000305175781 +rachel falkner 99.23999786376953 +rachel johnson 62.22999954223633 +rachel king 36.220001220703125 +rachel king 59.45000076293945 +rachel laertes 44.220001220703125 +rachel laertes 45.45000076293945 +rachel ovid 0.6000000238418579 +rachel ovid 1.0800000429153442 +rachel polk 89.27999877929688 +rachel quirinius 12.4399995803833 +rachel robinson 4.570000171661377 +rachel robinson 30.360000610351562 +rachel robinson 64.94999694824219 +rachel thompson 0.5600000023841858 +rachel thompson 4.170000076293945 +rachel thompson 58.52000045776367 +rachel underhill 48.45000076293945 +rachel white 43.709999084472656 +rachel white 94.72000122070312 +rachel young 43.130001068115234 +rachel zipper 7.059999942779541 +rachel zipper 72.18000030517578 +sarah carson 1.909999966621399 +sarah carson 14.210000038146973 +sarah carson 78.88999938964844 +sarah ellison 16.989999771118164 +sarah falkner 90.27999877929688 +sarah falkner 99.36000061035156 +sarah garcia 41.290000915527344 +sarah garcia 58.010000228881836 +sarah garcia 153.8800048828125 +sarah ichabod 81.31999969482422 +sarah ichabod 97.26000213623047 +sarah johnson 16.239999771118164 +sarah johnson 45.099998474121094 +sarah johnson 73.87999725341797 +sarah johnson 77.66000366210938 +sarah king 41.869998931884766 +sarah king 48.25 +sarah miller 41.709999084472656 +sarah ovid 60.02000045776367 +sarah robinson 33.83000183105469 +sarah robinson 66.88999938964844 +sarah steinbeck 40.16999816894531 +sarah white 37.849998474121094 +sarah white 89.80999755859375 +sarah xylophone 68.31999969482422 +sarah young 45.560001373291016 +sarah zipper 83.08000183105469 +tom brown 8.609999656677246 +tom brown 12.319999694824219 +tom carson 5.440000057220459 +tom carson 16.079999923706055 +tom carson 18.889999389648438 +tom davidson 170.0 +tom ellison 76.73999786376953 +tom ellison 98.2300033569336 +tom ellison 155.99999618530273 +tom falkner 60.130001068115234 +tom falkner 88.22000122070312 +tom hernandez 41.36000061035156 +tom hernandez 81.63999938964844 +tom ichabod 103.29000282287598 +tom johnson 14.920000076293945 +tom johnson 43.56999969482422 +tom king 15.75 +tom laertes 43.310001373291016 +tom laertes 64.6500015258789 +tom miller 21.229999542236328 +tom miller 68.25 +tom miller 139.04000091552734 +tom nixon 153.83999633789062 +tom ovid 8.670000076293945 +tom polk 38.29999923706055 +tom polk 54.43000030517578 +tom quirinius 10.1899995803833 +tom quirinius 75.31999969482422 +tom robinson 90.69000244140625 +tom robinson 98.72000122070312 +tom robinson 99.1500015258789 +tom robinson 123.5199966430664 +tom steinbeck 26.489999771118164 +tom van buren 3.2799999713897705 +tom van buren 40.779998779296875 +tom van buren 63.5099983215332 +tom white 40.040000915527344 +tom young 22.850000381469727 +tom young 84.30999755859375 +tom zipper 122.78000259399414 +ulysses brown 72.79000091552734 +ulysses carson 77.41999816894531 +ulysses carson 79.54000091552734 +ulysses carson 146.7100067138672 +ulysses carson 220.18000030517578 +ulysses davidson 55.16999816894531 +ulysses ellison 96.7300033569336 +ulysses garcia 89.80000305175781 +ulysses hernandez 35.16999816894531 +ulysses hernandez 54.470001220703125 +ulysses hernandez 68.25 +ulysses ichabod 19.1299991607666 +ulysses ichabod 98.56999969482422 +ulysses johnson 102.5999984741211 +ulysses king 74.19000244140625 +ulysses laertes 1.9199999570846558 +ulysses laertes 24.860000610351562 +ulysses laertes 50.1899995803833 +ulysses miller 2.9600000381469727 +ulysses miller 76.27999877929688 +ulysses nixon 80.95999908447266 +ulysses ovid 29.360000610351562 +ulysses polk 8.710000038146973 +ulysses polk 60.060001373291016 +ulysses polk 65.0199966430664 +ulysses polk 97.10000038146973 +ulysses quirinius 112.56999969482422 +ulysses robinson 104.85999870300293 +ulysses steinbeck 32.40999984741211 +ulysses steinbeck 74.0 +ulysses thompson 198.83000564575195 +ulysses underhill 14.119999885559082 +ulysses underhill 22.360000610351562 +ulysses underhill 35.88999938964844 +ulysses underhill 57.369998931884766 +ulysses underhill 81.58000183105469 +ulysses underhill 88.4800033569336 +ulysses underhill 99.66999816894531 +ulysses van buren 95.52999877929688 +ulysses white 59.54999923706055 +ulysses white 170.0800018310547 +ulysses xylophone 39.689998626708984 +ulysses xylophone 54.099998474121094 +ulysses xylophone 57.3100004196167 +ulysses young 14.930000305175781 +ulysses young 32.52000045776367 +ulysses young 114.55999946594238 +victor allen 44.27000045776367 +victor allen 89.5 +victor brown 59.34000015258789 +victor brown 77.88999938964844 +victor brown 90.37999725341797 +victor brown 91.97000122070312 +victor davidson 60.2599983215332 +victor davidson 66.5999984741211 +victor davidson 98.54999923706055 +victor ellison 17.8700008392334 +victor ellison 68.8499984741211 +victor hernandez 19.030000686645508 +victor hernandez 59.619998931884766 +victor hernandez 69.87999725341797 +victor hernandez 71.3499984741211 +victor hernandez 74.5199966430664 +victor johnson 18.200000762939453 +victor johnson 42.89000141620636 +victor johnson 72.55999755859375 +victor king 47.880001068115234 +victor king 66.66999816894531 +victor laertes 62.91999816894531 +victor laertes 67.58999633789062 +victor miller 22.1200008392334 +victor nixon 34.029998779296875 +victor nixon 68.5 +victor ovid 125.84000015258789 +victor polk 17.210000038146973 +victor quirinius 50.70000076293945 +victor quirinius 134.4000015258789 +victor robinson 51.560001373291016 +victor robinson 58.66999816894531 +victor steinbeck 12.460000038146973 +victor steinbeck 46.09000015258789 +victor steinbeck 52.720001220703125 +victor thompson 58.65999984741211 +victor van buren 34.970001220703125 +victor van buren 41.68000030517578 +victor white 5.670000076293945 +victor white 135.02999687194824 +victor xylophone 10.09000015258789 +victor xylophone 11.220000267028809 +victor xylophone 28.5 +victor xylophone 62.38999938964844 +victor xylophone 76.0999984741211 +victor young 88.55000305175781 +victor zipper 26.289999961853027 +wendy allen 56.06999969482422 +wendy allen 93.96999740600586 +wendy allen 220.7900003194809 +wendy brown 27.8700008392334 +wendy brown 50.2599983215332 +wendy ellison 94.66000366210938 +wendy ellison 124.93999481201172 +wendy falkner 22.010000228881836 +wendy falkner 97.68000030517578 +wendy falkner 141.36000061035156 +wendy garcia 30.6200008392334 +wendy garcia 57.25 +wendy garcia 82.1500015258789 +wendy garcia 133.3400001525879 +wendy hernandez 48.11000061035156 +wendy ichabod 13.149999618530273 +wendy king 45.189998626708984 +wendy king 63.33000183105469 +wendy king 183.75 +wendy laertes 46.619998931884766 +wendy laertes 70.37999725341797 +wendy laertes 79.98999786376953 +wendy miller 1.2699999809265137 +wendy miller 12.420000076293945 +wendy nixon 45.91999816894531 +wendy nixon 60.2599983215332 +wendy ovid 86.62999725341797 +wendy ovid 95.33000183105469 +wendy polk 32.369998931884766 +wendy polk 42.04000073671341 +wendy quirinius 12.15999984741211 +wendy quirinius 14.300000190734863 +wendy robinson 26.469999313354492 +wendy robinson 71.06999969482422 +wendy robinson 117.02000045776367 +wendy steinbeck 120.67000007629395 +wendy thompson 67.34000015258789 +wendy thompson 85.76000213623047 +wendy underhill 68.04000091552734 +wendy underhill 79.19000244140625 +wendy underhill 89.77999877929688 +wendy van buren 57.459999084472656 +wendy van buren 92.81999969482422 +wendy white 73.68000030517578 +wendy xylophone 76.69999694824219 +wendy xylophone 90.60000038146973 +wendy young 8.449999809265137 +wendy young 33.7599983215332 +xavier allen 45.68000030517578 +xavier allen 83.93000030517578 +xavier allen 98.22000122070312 +xavier brown 7.789999961853027 +xavier brown 90.7300033569336 +xavier brown 96.2300033569336 +xavier carson 20.790000915527344 +xavier carson 94.68000030517578 +xavier davidson 15.920000076293945 +xavier davidson 82.41000366210938 +xavier davidson 106.5199966430664 +xavier ellison 12.850000381469727 +xavier ellison 77.97000122070312 +xavier garcia 70.04000091552734 +xavier hernandez 6.670000076293945 +xavier hernandez 38.56999969482422 +xavier hernandez 67.26000213623047 +xavier ichabod 4.71999979019165 +xavier ichabod 71.19000244140625 +xavier johnson 27.299999237060547 +xavier johnson 203.65999794006348 +xavier king 8.569999694824219 +xavier king 87.22000122070312 +xavier laertes 15.899999618530273 +xavier ovid 112.91000366210938 +xavier polk 13.869999885559082 +xavier polk 61.209999084472656 +xavier polk 72.62000274658203 +xavier polk 76.93000030517578 +xavier quirinius 62.52000045776367 +xavier quirinius 83.01000022888184 +xavier quirinius 89.55000305175781 +xavier quirinius 97.14999961853027 +xavier thompson 9.930000305175781 +xavier underhill 47.27000045776367 +xavier white 59.20000171661377 +xavier white 75.29000091552734 +xavier xylophone 79.41999816894531 +xavier zipper 8.449999809265137 +yuri allen 52.849998474121094 +yuri allen 94.98999977111816 +yuri brown 75.19000244140625 +yuri brown 84.02999877929688 +yuri carson 6.289999961853027 +yuri carson 91.16000366210938 +yuri ellison 1.1200000047683716 +yuri ellison 98.82999801635742 +yuri falkner 39.6299991607666 +yuri falkner 86.0 +yuri garcia 27.65999984741211 +yuri hernandez 2.069999933242798 +yuri johnson 0.12999999523162842 +yuri johnson 39.900001525878906 +yuri johnson 48.220001220703125 +yuri king 69.59000015258789 +yuri laertes 37.59000015258789 +yuri laertes 61.95000076293945 +yuri nixon 2.200000047683716 +yuri nixon 82.81000328063965 +yuri polk 26.760000228881836 +yuri polk 28.790000915527344 +yuri polk 105.11999702453613 +yuri quirinius 10.260000228881836 +yuri quirinius 54.310001373291016 +yuri quirinius 57.93000030517578 +yuri steinbeck 17.790000915527344 +yuri steinbeck 75.87999725341797 +yuri thompson 14.920000076293945 +yuri underhill 23.770000457763672 +yuri underhill 83.87000274658203 +yuri white 34.58000183105469 +yuri xylophone 20.3799991607666 +zach allen 65.43000030517578 +zach brown 48.0099983215332 +zach brown 49.119998931884766 +zach brown 57.08000183105469 +zach brown 67.37999725341797 +zach brown 100.46000289916992 +zach carson 95.86999893188477 +zach ellison 6.840000152587891 +zach falkner 9.130000114440918 +zach falkner 91.41999816894531 +zach garcia 32.20000076293945 +zach garcia 84.37999725341797 +zach garcia 106.86999893188477 +zach garcia 167.62000274658203 +zach ichabod 64.25 +zach ichabod 106.69000244140625 +zach king 46.18000030517578 +zach king 70.51000213623047 +zach king 86.93000030517578 +zach miller 2.5999999046325684 +zach miller 21.280000686645508 +zach miller 53.27000045776367 +zach ovid 0.10000000149011612 +zach ovid 23.06999969482422 +zach ovid 92.55000305175781 +zach ovid 94.33999633789062 +zach quirinius 39.209999084472656 +zach robinson 122.81000137329102 +zach steinbeck 85.48999786376953 +zach steinbeck 90.05000305175781 +zach thompson 71.5 +zach thompson 91.63999938964844 +zach underhill 86.22000122070312 +zach white 70.52999877929688 +zach xylophone 43.84999942779541 +zach xylophone 71.01000213623047 +zach young 71.31999969482422 +zach zipper 52.60000133514404 +zach zipper 85.87000274658203 +zach zipper 94.43000030517578 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-2-5f0eab306ea3c22b11ace9b542a7ee56 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-2-5f0eab306ea3c22b11ace9b542a7ee56 new file mode 100644 index 0000000000000..e55bede9242e5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-2-5f0eab306ea3c22b11ace9b542a7ee56 @@ -0,0 +1,1049 @@ + 257.04000091552734 + 261.16000175476074 + 284.2699966430664 +alice allen 73.62999725341797 +alice allen 195.0099983215332 +alice allen 196.729998588562 +alice brown 245.52000427246094 +alice carson 424.63000106811523 +alice davidson 319.00999450683594 +alice falkner 90.25 +alice garcia 174.36999893188477 +alice hernandez 185.6699981689453 +alice hernandez 380.1699981689453 +alice johnson 367.2900071144104 +alice king 58.78999900817871 +alice king 294.1199951171875 +alice king 371.23999404907227 +alice laertes 68.94999694824219 +alice laertes 258.3999938964844 +alice miller 154.19000244140625 +alice nixon 209.91000366210938 +alice nixon 246.36000442504883 +alice nixon 260.19000244140625 +alice ovid 49.8199987411499 +alice polk 148.63999938964844 +alice quirinius 239.81999588012695 +alice quirinius 301.4100036621094 +alice robinson 140.47999572753906 +alice robinson 266.4900016784668 +alice steinbeck 169.76000213623047 +alice steinbeck 186.70999908447266 +alice steinbeck 446.8099937438965 +alice underhill 98.18000030517578 +alice van buren 112.42000198364258 +alice xylophone 78.20999908447266 +alice xylophone 91.22000122070312 +alice xylophone 413.1199951171875 +alice zipper 89.93000030517578 +alice zipper 279.54000091552734 +alice zipper 293.25000381469727 +bob brown 188.89999389648438 +bob brown 228.80999946594238 +bob brown 247.37999725341797 +bob carson 207.67000198364258 +bob davidson 53.93000030517578 +bob davidson 113.83999919891357 +bob davidson 259.0899963378906 +bob ellison 65.0199966430664 +bob ellison 80.30000305175781 +bob ellison 243.86000061035156 +bob ellison 245.02999877929688 +bob falkner 208.82000160217285 +bob garcia 33.410000801086426 +bob garcia 87.56999969482422 +bob garcia 120.17999649047852 +bob garcia 148.65999841690063 +bob garcia 178.87000274658203 +bob hernandez 337.23999977111816 +bob ichabod 82.55999755859375 +bob king 114.11000156402588 +bob king 134.81999588012695 +bob king 152.7699956893921 +bob laertes 42.89999961853027 +bob laertes 393.99999433755875 +bob miller 146.1500015258789 +bob ovid 62.849998474121094 +bob ovid 88.77999877929688 +bob ovid 97.08999633789062 +bob ovid 102.93000030517578 +bob polk 261.4599976539612 +bob quirinius 298.7199897766113 +bob steinbeck 103.01999950408936 +bob van buren 174.89999771118164 +bob white 194.25 +bob white 347.7799949645996 +bob xylophone 19.690000534057617 +bob xylophone 191.52999687194824 +bob young 78.17999649047852 +bob zipper 132.86000061035156 +bob zipper 139.6900042295456 +bob zipper 295.59000039100647 +calvin allen 255.68000411987305 +calvin brown 85.9000015258789 +calvin brown 238.02000427246094 +calvin brown 275.8699951171875 +calvin carson 80.2300033569336 +calvin davidson 31.770000457763672 +calvin davidson 181.76000213623047 +calvin ellison 188.0300006866455 +calvin falkner 93.61000061035156 +calvin falkner 94.30999755859375 +calvin falkner 125.91999816894531 +calvin falkner 137.1699981689453 +calvin falkner 140.99999594688416 +calvin falkner 168.81999969482422 +calvin garcia 307.439998626709 +calvin hernandez 303.4599952697754 +calvin johnson 152.8300018310547 +calvin laertes 150.69999885559082 +calvin laertes 216.81000137329102 +calvin nixon 131.57999801635742 +calvin nixon 143.3699951171875 +calvin nixon 196.34000301361084 +calvin ovid 69.95999908447266 +calvin ovid 176.13999938964844 +calvin ovid 176.3800048828125 +calvin ovid 248.65999603271484 +calvin polk 147.04000091552734 +calvin quirinius 226.66999435424805 +calvin quirinius 266.7100009918213 +calvin robinson 289.7900047302246 +calvin steinbeck 92.05000305175781 +calvin steinbeck 118.15000057220459 +calvin steinbeck 333.6000003814697 +calvin thompson 93.7300033569336 +calvin thompson 249.56000137329102 +calvin underhill 208.3400001525879 +calvin van buren 136.51000213623047 +calvin van buren 347.0999946594238 +calvin white 90.69000244140625 +calvin white 112.15999984741211 +calvin xylophone 25.420000076293945 +calvin xylophone 237.71999740600586 +calvin xylophone 315.2099952697754 +calvin young 222.96000289916992 +calvin young 243.3199977874756 +calvin zipper 95.37999725341797 +calvin zipper 531.3600015640259 +david allen 202.43000030517578 +david allen 302.4399948120117 +david brown 93.63999938964844 +david brown 258.05999851226807 +david davidson 74.1500015258789 +david davidson 95.80999755859375 +david davidson 106.50000202655792 +david davidson 149.94000244140625 +david ellison 85.2300033569336 +david ellison 94.1500015258789 +david ellison 208.3900032043457 +david hernandez 99.91000366210938 +david ichabod 82.55000305175781 +david ichabod 320.47999143600464 +david laertes 250.1699981689453 +david nixon 174.58999633789062 +david ovid 198.21000289916992 +david ovid 230.47999954223633 +david quirinius 29.239999771118164 +david quirinius 79.97000122070312 +david quirinius 180.92999649047852 +david robinson 147.65999603271484 +david robinson 168.7100009918213 +david thompson 41.88999938964844 +david underhill 97.55999755859375 +david underhill 277.5999984741211 +david underhill 369.4600009918213 +david van buren 83.56999969482422 +david van buren 289.189998626709 +david white 124.6099967956543 +david xylophone 135.70000076293945 +david xylophone 237.06000137329102 +david xylophone 338.20999908447266 +david young 172.49000549316406 +david young 184.9800033569336 +ethan allen 240.42000198364258 +ethan brown 61.86000061035156 +ethan brown 73.18000030517578 +ethan brown 105.29000043869019 +ethan brown 177.8300018310547 +ethan brown 185.98999691009521 +ethan brown 284.729998588562 +ethan carson 265.22999572753906 +ethan ellison 166.5 +ethan ellison 244.99000671505928 +ethan falkner 59.43000030517578 +ethan falkner 196.17000198364258 +ethan garcia 271.5999946594238 +ethan hernandez 264.50999450683594 +ethan johnson 90.05000305175781 +ethan king 36.49000024795532 +ethan laertes 95.06999969482422 +ethan laertes 96.29000091552734 +ethan laertes 189.66000270843506 +ethan laertes 192.79999923706055 +ethan laertes 249.04000091552734 +ethan laertes 249.76000213623047 +ethan laertes 369.9599952697754 +ethan miller 314.5599994659424 +ethan nixon 493.03000259399414 +ethan ovid 57.290000915527344 +ethan polk 2.3499999046325684 +ethan polk 59.869998931884766 +ethan polk 219.6599998474121 +ethan polk 263.8600025177002 +ethan quirinius 97.23999786376953 +ethan quirinius 111.70999908447266 +ethan quirinius 317.69000363349915 +ethan robinson 78.62000274658203 +ethan robinson 149.5800018310547 +ethan underhill 231.25000381469727 +ethan van buren 152.60000228881836 +ethan white 155.81999969482422 +ethan white 235.55999755859375 +ethan xylophone 414.61000061035156 +ethan zipper 97.51000213623047 +ethan zipper 297.20999908447266 +fred davidson 78.30999755859375 +fred davidson 105.8499984741211 +fred davidson 220.56000137329102 +fred ellison 56.489999771118164 +fred ellison 96.77999877929688 +fred ellison 199.52000045776367 +fred falkner 66.77999973297119 +fred falkner 85.0 +fred falkner 169.91000366210938 +fred hernandez 117.85000228881836 +fred ichabod 81.31999969482422 +fred ichabod 202.45000457763672 +fred johnson 96.08999633789062 +fred king 140.24999618530273 +fred king 343.82000064849854 +fred laertes 57.63999938964844 +fred miller 176.18000030517578 +fred nixon 28.690000534057617 +fred nixon 187.40999603271484 +fred nixon 246.77000045776367 +fred nixon 338.34999084472656 +fred polk 90.12000274658203 +fred polk 323.1899948120117 +fred polk 357.19000244140625 +fred polk 496.16999435424805 +fred quirinius 218.82999801635742 +fred quirinius 224.12000179290771 +fred robinson 89.02999877929688 +fred steinbeck 32.22999954223633 +fred steinbeck 91.05000305175781 +fred steinbeck 231.92000198364258 +fred underhill 183.31999969482422 +fred van buren 83.58000183105469 +fred van buren 318.38000106811523 +fred van buren 346.1400008201599 +fred van buren 391.9999942779541 +fred white 187.38000106811523 +fred young 97.70999908447266 +fred young 141.22999954223633 +fred zipper 163.89999771118164 +gabriella allen 274.8599967956543 +gabriella allen 283.5 +gabriella brown 163.89999961853027 +gabriella brown 465.0 +gabriella carson 147.78999710083008 +gabriella davidson 263.64000415802 +gabriella ellison 71.54000091552734 +gabriella ellison 188.55999755859375 +gabriella falkner 51.720001220703125 +gabriella falkner 87.61000061035156 +gabriella falkner 162.21999835968018 +gabriella garcia 43.0099983215332 +gabriella hernandez 190.5500030517578 +gabriella hernandez 267.4700012207031 +gabriella ichabod 71.12999725341797 +gabriella ichabod 90.3499984741211 +gabriella ichabod 148.6999969482422 +gabriella ichabod 175.70000076293945 +gabriella ichabod 285.72999572753906 +gabriella king 166.75000190734863 +gabriella king 177.6999969482422 +gabriella laertes 65.37999725341797 +gabriella miller 148.4800033569336 +gabriella ovid 92.4000015258789 +gabriella ovid 137.82999801635742 +gabriella polk 244.07000350952148 +gabriella polk 282.00000762939453 +gabriella steinbeck 272.2799987792969 +gabriella steinbeck 461.060001373291 +gabriella thompson 88.36000061035156 +gabriella thompson 94.25 +gabriella thompson 158.80999755859375 +gabriella van buren 146.0800018310547 +gabriella van buren 151.63999938964844 +gabriella white 138.72000122070312 +gabriella young 30.739999771118164 +gabriella young 146.62999725341797 +gabriella zipper 91.62999725341797 +gabriella zipper 357.5099983215332 +holly allen 44.56999969482422 +holly brown 173.64999389648438 +holly brown 174.2100067138672 +holly falkner 166.22999572753906 +holly hernandez 180.0800018310547 +holly hernandez 248.71000480651855 +holly hernandez 336.87000274658203 +holly hernandez 523.2800025939941 +holly ichabod 179.37000274658203 +holly ichabod 180.18000030517578 +holly ichabod 184.66000366210938 +holly johnson 64.36000061035156 +holly johnson 145.61000061035156 +holly johnson 157.12999725341797 +holly king 275.2299995422363 +holly king 288.52000427246094 +holly laertes 246.21000289916992 +holly miller 290.21999740600586 +holly nixon 177.39999389648438 +holly nixon 228.58999633789062 +holly polk 98.30999755859375 +holly polk 307.0799951553345 +holly robinson 219.27999877929688 +holly thompson 75.41999816894531 +holly thompson 86.69000244140625 +holly thompson 523.360002592206 +holly underhill 96.68000030517578 +holly underhill 163.54000091552734 +holly underhill 187.47000122070312 +holly underhill 328.0099983215332 +holly van buren 161.7699966430664 +holly white 122.98999786376953 +holly white 335.93999576568604 +holly xylophone 191.34000396728516 +holly young 60.220001220703125 +holly young 297.20999908447266 +holly zipper 99.12999725341797 +holly zipper 99.29000091552734 +irene allen 234.6400032043457 +irene brown 4.789999961853027 +irene brown 176.4499969482422 +irene brown 338.2099952697754 +irene carson 292.0 +irene ellison 201.06000137329102 +irene ellison 230.79000091552734 +irene falkner 99.91999816894531 +irene falkner 210.11000061035156 +irene garcia 40.78999996185303 +irene garcia 86.93000030517578 +irene garcia 183.02000045776367 +irene ichabod 99.62000274658203 +irene ichabod 281.96999740600586 +irene johnson 243.59999752044678 +irene laertes 112.54000091552734 +irene laertes 227.45000076293945 +irene laertes 246.53000259399414 +irene miller 395.9100036621094 +irene nixon 29.780000686645508 +irene nixon 199.45999908447266 +irene nixon 261.46000480651855 +irene ovid 158.97000122070312 +irene ovid 339.94000244140625 +irene ovid 362.82000732421875 +irene polk 95.83999633789062 +irene polk 183.43000411987305 +irene polk 258.7100033760071 +irene polk 284.6300048828125 +irene polk 507.2400016784668 +irene quirinius 157.5800018310547 +irene quirinius 250.61000061035156 +irene quirinius 431.6499996185303 +irene robinson 191.72999572753906 +irene steinbeck 94.33000183105469 +irene thompson 256.0 +irene underhill 135.55999755859375 +irene underhill 327.0299892425537 +irene van buren 54.439998626708984 +irene van buren 193.71000289916992 +irene xylophone 168.5 +jessica brown 422.5299949645996 +jessica carson 103.66000366210938 +jessica carson 144.92000198364258 +jessica carson 259.1099967956543 +jessica davidson 95.33999633789062 +jessica davidson 99.20999908447266 +jessica davidson 137.17000198364258 +jessica davidson 227.79999923706055 +jessica ellison 207.35000228881836 +jessica ellison 237.4300022125244 +jessica falkner 99.6500015258789 +jessica garcia 174.70999908447266 +jessica garcia 185.62000179290771 +jessica ichabod 124.59000015258789 +jessica johnson 272.0500030517578 +jessica johnson 294.2899990081787 +jessica miller 77.83999633789062 +jessica nixon 77.0999984741211 +jessica nixon 90.06999969482422 +jessica ovid 71.68000030517578 +jessica ovid 309.44000244140625 +jessica polk 472.2099952697754 +jessica quirinius 35.619998931884766 +jessica quirinius 192.7000026702881 +jessica quirinius 208.6500015258789 +jessica quirinius 370.0599937438965 +jessica robinson 254.5300064086914 +jessica thompson 115.9000015258789 +jessica thompson 180.60000610351562 +jessica underhill 199.10999870300293 +jessica underhill 234.29000091552734 +jessica underhill 257.09000396728516 +jessica van buren 9.739999771118164 +jessica white 96.62000274658203 +jessica white 166.54000091552734 +jessica white 240.52999877929688 +jessica white 432.17999362945557 +jessica white 497.6400032043457 +jessica xylophone 385.4799995422363 +jessica young 47.410000801086426 +jessica young 240.6500015258789 +jessica zipper 323.4199962615967 +jessica zipper 344.8399953842163 +jessica zipper 480.06999588012695 +katie allen 312.9700012207031 +katie brown 573.4599933624268 +katie davidson 96.91000366210938 +katie ellison 163.52999877929688 +katie ellison 384.4699947834015 +katie falkner 125.57000160217285 +katie garcia 84.4000015258789 +katie garcia 160.28999710083008 +katie hernandez 257.9600028991699 +katie ichabod 187.63999557495117 +katie ichabod 274.97999572753906 +katie ichabod 362.9200019836426 +katie king 97.80999755859375 +katie king 169.56999969482422 +katie king 314.1999969482422 +katie miller 31.399999618530273 +katie miller 228.40999603271484 +katie nixon 23.190000534057617 +katie ovid 207.1200065612793 +katie polk 143.2599983215332 +katie polk 247.02000045776367 +katie robinson 83.84999942779541 +katie van buren 297.0300064086914 +katie van buren 464.54999351501465 +katie white 344.1700019836426 +katie white 465.8599934577942 +katie xylophone 175.89999675750732 +katie young 31.010000228881836 +katie young 72.51000213623047 +katie young 97.56999969482422 +katie zipper 101.9000015258789 +katie zipper 314.75 +luke allen 89.55000305175781 +luke allen 133.4800033569336 +luke allen 210.8800048828125 +luke allen 392.0300064086914 +luke allen 420.6299934387207 +luke brown 129.20999908447266 +luke davidson 28.950000762939453 +luke davidson 106.41000080108643 +luke ellison 42.09000027179718 +luke ellison 136.52000427246094 +luke ellison 187.51000213623047 +luke falkner 172.8799991607666 +luke falkner 216.0199966430664 +luke garcia 50.94000053405762 +luke garcia 345.1200008392334 +luke ichabod 67.90000057220459 +luke ichabod 97.87000274658203 +luke johnson 59.00999927520752 +luke johnson 105.32000160217285 +luke johnson 187.2899990081787 +luke laertes 105.42000198364258 +luke laertes 147.14999723434448 +luke laertes 158.86000061035156 +luke laertes 167.01999855041504 +luke laertes 281.19999504089355 +luke miller 97.6500015258789 +luke ovid 186.53000259399414 +luke ovid 340.1300048828125 +luke polk 95.27999877929688 +luke polk 277.6700019836426 +luke quirinius 115.83999633789062 +luke robinson 137.33999633789062 +luke robinson 145.23999786376953 +luke thompson 94.37999725341797 +luke underhill 96.94000244140625 +luke underhill 194.73999786376953 +luke underhill 372.6899948120117 +luke van buren 193.93999862670898 +luke white 67.12000274658203 +luke xylophone 102.37999725341797 +luke zipper 223.54000282287598 +mike allen 79.60999870300293 +mike brown 202.81999588012695 +mike carson 81.66000366210938 +mike carson 105.02999877929688 +mike carson 405.4499931335449 +mike davidson 137.74999809265137 +mike davidson 393.17999267578125 +mike ellison 79.37999725341797 +mike ellison 85.73999786376953 +mike ellison 127.15999603271484 +mike ellison 228.07999992370605 +mike ellison 263.8899955749512 +mike falkner 254.50000381469727 +mike garcia 70.8499984741211 +mike garcia 173.63999938964844 +mike garcia 177.5199966430664 +mike hernandez 59.45000076293945 +mike hernandez 327.6900062561035 +mike ichabod 64.7699966430664 +mike king 78.26000213623047 +mike king 84.2300033569336 +mike king 94.68000030517578 +mike king 133.5900001525879 +mike king 134.87999725341797 +mike king 173.45999908447266 +mike miller 57.890000343322754 +mike nixon 92.95999908447266 +mike nixon 203.68999862670898 +mike polk 32.140000343322754 +mike polk 99.68000030517578 +mike polk 306.61000061035156 +mike quirinius 89.37999725341797 +mike steinbeck 85.13999938964844 +mike steinbeck 97.45999908447266 +mike steinbeck 153.86000204086304 +mike steinbeck 221.21999502182007 +mike van buren 80.83999633789062 +mike van buren 174.21000289916992 +mike white 91.87999725341797 +mike white 341.80999755859375 +mike white 341.86000061035156 +mike white 389.20000076293945 +mike young 74.58999633789062 +mike young 83.54000091552734 +mike young 112.19000101089478 +mike zipper 86.98999786376953 +mike zipper 97.38999938964844 +mike zipper 174.61000061035156 +nick allen 173.32000207901 +nick allen 257.7300033569336 +nick brown 192.45000076293945 +nick davidson 258.9799919128418 +nick ellison 183.34000396728516 +nick ellison 193.02000427246094 +nick falkner 10.130000114440918 +nick falkner 182.72000122070312 +nick garcia 142.65999603271484 +nick garcia 183.7699966430664 +nick garcia 277.8299951553345 +nick ichabod 110.43999862670898 +nick ichabod 112.54999923706055 +nick ichabod 241.68999481201172 +nick johnson 192.56000518798828 +nick johnson 325.9499976634979 +nick laertes 96.25 +nick miller 82.97000122070312 +nick nixon 96.37999725341797 +nick ovid 184.3699951171875 +nick polk 199.57000064849854 +nick quirinius 174.80999755859375 +nick quirinius 243.8300018310547 +nick robinson 129.65999603271484 +nick robinson 216.54999923706055 +nick steinbeck 97.83000183105469 +nick thompson 205.4900016784668 +nick underhill 166.42000007629395 +nick van buren 222.6500015258789 +nick xylophone 75.3499984741211 +nick young 332.23999786376953 +nick young 346.41000083088875 +nick zipper 222.9199981689453 +nick zipper 529.7199974060059 +oscar allen 246.42999839782715 +oscar brown 274.01000022888184 +oscar carson 78.9800033569336 +oscar carson 87.4800033569336 +oscar carson 98.51000213623047 +oscar carson 203.86000442504883 +oscar carson 321.82000064849854 +oscar davidson 361.6699981689453 +oscar ellison 146.44000244140625 +oscar ellison 234.32000160217285 +oscar falkner 98.4800033569336 +oscar garcia 231.04000091552734 +oscar hernandez 85.48999786376953 +oscar hernandez 95.4800033569336 +oscar ichabod 71.80000305175781 +oscar ichabod 123.78000068664551 +oscar ichabod 173.31000518798828 +oscar ichabod 251.22000122070312 +oscar johnson 146.27000427246094 +oscar johnson 260.1600036621094 +oscar king 124.2699966430664 +oscar king 249.5399990081787 +oscar king 284.8599910736084 +oscar laertes 15.640000343322754 +oscar laertes 254.8499984741211 +oscar laertes 261.41000175476074 +oscar laertes 261.8400020599365 +oscar nixon 41.619998931884766 +oscar ovid 82.23999786376953 +oscar ovid 187.76000213623047 +oscar ovid 260.6100044250488 +oscar polk 63.900001525878906 +oscar polk 252.71000289916992 +oscar quirinius 73.4800033569336 +oscar quirinius 165.3800048828125 +oscar quirinius 244.2699966430664 +oscar quirinius 248.75 +oscar robinson 93.31999969482422 +oscar robinson 163.55999755859375 +oscar robinson 191.8300018310547 +oscar robinson 315.1999912261963 +oscar steinbeck 376.6899948120117 +oscar thompson 131.1400032043457 +oscar thompson 148.01000213623047 +oscar thompson 325.42000579833984 +oscar thompson 545.7399940490723 +oscar underhill 87.4000015258789 +oscar van buren 61.880001068115234 +oscar van buren 188.8699951171875 +oscar van buren 209.53000235557556 +oscar white 129.73999786376953 +oscar white 148.9800033569336 +oscar white 275.1500015258789 +oscar white 303.8599910736084 +oscar xylophone 115.22999954223633 +oscar xylophone 319.75000381469727 +oscar xylophone 475.3300018310547 +oscar zipper 109.53999710083008 +oscar zipper 214.40999603271484 +oscar zipper 214.6500015258789 +priscilla brown 77.56999969482422 +priscilla brown 165.5199966430664 +priscilla brown 408.4499969482422 +priscilla carson 168.8300018310547 +priscilla carson 195.7900047302246 +priscilla carson 207.5300006866455 +priscilla ichabod 92.61000061035156 +priscilla ichabod 206.16000366210938 +priscilla johnson 89.1500015258789 +priscilla johnson 156.4600067138672 +priscilla johnson 158.88000106811523 +priscilla johnson 190.61000061035156 +priscilla johnson 211.01000022888184 +priscilla king 371.9299964904785 +priscilla nixon 95.80999755859375 +priscilla nixon 278.87999725341797 +priscilla ovid 96.27000284194946 +priscilla ovid 198.3400001525879 +priscilla polk 252.5800018310547 +priscilla quirinius 131.8499994277954 +priscilla thompson 230.36000156402588 +priscilla underhill 143.56999969482422 +priscilla underhill 354.37000274658203 +priscilla van buren 82.72000122070312 +priscilla van buren 145.61000061035156 +priscilla van buren 183.72000122070312 +priscilla white 78.27999877929688 +priscilla xylophone 21.489999771118164 +priscilla xylophone 159.26000213623047 +priscilla xylophone 406.1000007688999 +priscilla young 163.2900013923645 +priscilla young 260.59000366926193 +priscilla zipper 311.399995803833 +priscilla zipper 327.97999572753906 +quinn allen 257.94000244140625 +quinn allen 365.2299995422363 +quinn brown 80.58000183105469 +quinn brown 80.81999969482422 +quinn brown 198.71000289916992 +quinn davidson 83.4000015258789 +quinn davidson 95.11000061035156 +quinn davidson 154.79000091552734 +quinn davidson 227.13999938964844 +quinn ellison 237.17000007629395 +quinn ellison 361.14000129699707 +quinn garcia 92.33000183105469 +quinn garcia 148.63999938964844 +quinn garcia 226.78999710083008 +quinn garcia 246.70000076293945 +quinn ichabod 89.63999938964844 +quinn king 74.62000274658203 +quinn king 86.2300033569336 +quinn laertes 112.36000061035156 +quinn laertes 243.6900019645691 +quinn laertes 265.51000213623047 +quinn nixon 149.3300018310547 +quinn ovid 393.2099943161011 +quinn quirinius 266.8200035095215 +quinn robinson 247.6400032043457 +quinn steinbeck 144.81999969482422 +quinn steinbeck 213.65999507904053 +quinn thompson 156.51000213623047 +quinn thompson 274.1599931716919 +quinn underhill 248.3000030517578 +quinn underhill 252.61999130249023 +quinn underhill 321.9799976348877 +quinn van buren 82.5199966430664 +quinn young 90.97999954223633 +quinn zipper 58.0 +quinn zipper 249.38999938964844 +rachel allen 15.8100004196167 +rachel allen 151.80999755859375 +rachel brown 193.5800018310547 +rachel brown 312.0800018310547 +rachel brown 347.7999954223633 +rachel brown 423.98999214172363 +rachel brown 437.64999771118164 +rachel carson 98.95999908447266 +rachel carson 385.3799934387207 +rachel davidson 396.38999938964844 +rachel ellison 299.12000465393066 +rachel falkner 88.80000305175781 +rachel falkner 99.23999786376953 +rachel falkner 172.54999542236328 +rachel falkner 233.55999755859375 +rachel johnson 197.92999649047852 +rachel king 36.220001220703125 +rachel king 219.8400001525879 +rachel laertes 97.17000198364258 +rachel laertes 109.5999984741211 +rachel ovid 80.20999872684479 +rachel ovid 260.18999683856964 +rachel polk 89.27999877929688 +rachel quirinius 205.1400022506714 +rachel robinson 254.1300015449524 +rachel robinson 286.0400047302246 +rachel robinson 332.4199981689453 +rachel thompson 137.73000198602676 +rachel thompson 213.31000137329102 +rachel thompson 380.85999488830566 +rachel underhill 175.6099967956543 +rachel white 94.72000122070312 +rachel white 196.18000030517578 +rachel young 230.6400032043457 +rachel zipper 148.9000015258789 +rachel zipper 238.98000192642212 +sarah carson 175.62000274658203 +sarah carson 307.70000088214874 +sarah carson 386.8999948501587 +sarah ellison 161.80999946594238 +sarah falkner 99.36000061035156 +sarah falkner 281.62000274658203 +sarah garcia 73.6500015258789 +sarah garcia 153.73000144958496 +sarah garcia 312.8899955749512 +sarah ichabod 81.31999969482422 +sarah ichabod 97.26000213623047 +sarah johnson 140.37999725341797 +sarah johnson 177.57000732421875 +sarah johnson 248.4499969482422 +sarah johnson 309.1800022125244 +sarah king 216.75 +sarah king 268.5399932861328 +sarah miller 222.31000518798828 +sarah ovid 146.25000381469727 +sarah robinson 143.43000030517578 +sarah robinson 310.75 +sarah steinbeck 208.72000122070312 +sarah white 140.22999572753906 +sarah white 181.86000061035156 +sarah xylophone 68.31999969482422 +sarah young 185.80999755859375 +sarah zipper 168.22000122070312 +tom brown 181.1000051498413 +tom brown 404.3500061035156 +tom carson 142.60999822616577 +tom carson 299.57999992370605 +tom carson 592.3499927520752 +tom davidson 180.61000061035156 +tom ellison 98.2300033569336 +tom ellison 154.58999633789062 +tom ellison 173.02999877929688 +tom falkner 88.22000122070312 +tom falkner 139.11000442504883 +tom hernandez 81.63999938964844 +tom hernandez 263.67000579833984 +tom ichabod 214.0699977874756 +tom johnson 405.95000076293945 +tom johnson 438.9099922180176 +tom king 218.18000030517578 +tom laertes 244.37000274658203 +tom laertes 473.0999984741211 +tom miller 68.25 +tom miller 85.59000015258789 +tom miller 127.56999969482422 +tom nixon 85.02999877929688 +tom ovid 217.32000160217285 +tom polk 188.87000274658203 +tom polk 206.52000045776367 +tom quirinius 120.27000427246094 +tom quirinius 232.63000202178955 +tom robinson 90.69000244140625 +tom robinson 98.72000122070312 +tom robinson 99.1500015258789 +tom robinson 209.5399932861328 +tom steinbeck 277.7100009918213 +tom van buren 40.779998779296875 +tom van buren 217.70000076293945 +tom van buren 375.2099964618683 +tom white 223.4700050354004 +tom young 174.36000061035156 +tom young 304.8199977874756 +tom zipper 213.7900047302246 +ulysses brown 247.1500015258789 +ulysses carson 77.41999816894531 +ulysses carson 79.54000091552734 +ulysses carson 150.93000030517578 +ulysses carson 162.24000549316406 +ulysses davidson 414.7100009918213 +ulysses ellison 96.7300033569336 +ulysses garcia 89.80000305175781 +ulysses hernandez 106.29999542236328 +ulysses hernandez 134.44000244140625 +ulysses hernandez 160.22000122070312 +ulysses ichabod 98.56999969482422 +ulysses ichabod 309.34999656677246 +ulysses johnson 152.47000122070312 +ulysses king 244.7100067138672 +ulysses laertes 138.4400042295456 +ulysses laertes 173.55999755859375 +ulysses laertes 256.91999912261963 +ulysses miller 76.27999877929688 +ulysses miller 417.67000102996826 +ulysses nixon 174.56999969482422 +ulysses ovid 130.13000106811523 +ulysses polk 123.9399995803833 +ulysses polk 149.95999908447266 +ulysses polk 205.2400016784668 +ulysses polk 237.5699920654297 +ulysses quirinius 330.4700012207031 +ulysses robinson 79.48999786376953 +ulysses steinbeck 144.8300018310547 +ulysses steinbeck 155.66000366210938 +ulysses thompson 159.92000579833984 +ulysses underhill 81.58000183105469 +ulysses underhill 88.4800033569336 +ulysses underhill 99.66999816894531 +ulysses underhill 135.55999755859375 +ulysses underhill 189.1099977493286 +ulysses underhill 289.6800003051758 +ulysses underhill 385.60000228881836 +ulysses van buren 95.52999877929688 +ulysses white 188.8300018310547 +ulysses white 305.79000091552734 +ulysses xylophone 54.099998474121094 +ulysses xylophone 205.2099952697754 +ulysses xylophone 251.94000148773193 +ulysses young 100.77000045776367 +ulysses young 275.8300018310547 +ulysses young 522.1700019836426 +victor allen 220.1699981689453 +victor allen 222.10000228881836 +victor brown 77.88999938964844 +victor brown 90.37999725341797 +victor brown 91.97000122070312 +victor brown 455.25000381469727 +victor davidson 149.06000137329102 +victor davidson 291.48000717163086 +victor davidson 321.25 +victor ellison 314.37000274658203 +victor ellison 442.50000190734863 +victor hernandez 69.87999725341797 +victor hernandez 99.85000038146973 +victor hernandez 143.02000045776367 +victor hernandez 160.38999938964844 +victor hernandez 391.2999954223633 +victor johnson 145.18000030517578 +victor johnson 190.27000045776367 +victor johnson 308.1900006532669 +victor king 108.10000228881836 +victor king 310.5 +victor laertes 145.42999267578125 +victor laertes 214.72999572753906 +victor miller 173.76000022888184 +victor nixon 68.5 +victor nixon 269.5899963378906 +victor ovid 151.39999771118164 +victor polk 175.8799991607666 +victor quirinius 65.55000305175781 +victor quirinius 168.5500030517578 +victor robinson 177.9100022315979 +victor robinson 204.09999084472656 +victor steinbeck 52.720001220703125 +victor steinbeck 220.45999908447266 +victor steinbeck 309.4900064468384 +victor thompson 58.65999984741211 +victor van buren 206.77999877929688 +victor van buren 222.44000244140625 +victor white 156.36999893188477 +victor white 167.2699966430664 +victor xylophone 158.36999893188477 +victor xylophone 161.54000091552734 +victor xylophone 234.76000308990479 +victor xylophone 267.82999420166016 +victor xylophone 314.95000076293945 +victor young 88.55000305175781 +victor zipper 192.92999649047852 +wendy allen 56.06999969482422 +wendy allen 66.16000306606293 +wendy allen 267.3199996948242 +wendy brown 453.53000259399414 +wendy brown 525.5100040435791 +wendy ellison 193.95000457763672 +wendy ellison 260.9099998474121 +wendy falkner 77.36000061035156 +wendy falkner 97.68000030517578 +wendy falkner 128.30999565124512 +wendy garcia 4.409999847412109 +wendy garcia 76.72000122070312 +wendy garcia 189.42999839782715 +wendy garcia 265.5900001525879 +wendy hernandez 48.11000061035156 +wendy ichabod 104.3700008392334 +wendy king 156.89999771118164 +wendy king 183.31999969482422 +wendy king 403.27000427246094 +wendy laertes 79.98999786376953 +wendy laertes 165.0999984741211 +wendy laertes 365.0 +wendy miller 72.9500002861023 +wendy miller 313.8300037384033 +wendy nixon 45.91999816894531 +wendy nixon 60.2599983215332 +wendy ovid 95.33000183105469 +wendy ovid 180.36000061035156 +wendy polk 386.7400016784668 +wendy polk 443.3400018811226 +wendy quirinius 152.04999828338623 +wendy quirinius 240.23999977111816 +wendy robinson 71.06999969482422 +wendy robinson 249.35000610351562 +wendy robinson 391.4699993133545 +wendy steinbeck 92.11000061035156 +wendy thompson 136.35000228881836 +wendy thompson 183.1500015258789 +wendy underhill 318.6500015258789 +wendy underhill 320.75000190734863 +wendy underhill 328.2300033569336 +wendy van buren 57.459999084472656 +wendy van buren 92.81999969482422 +wendy white 171.36000061035156 +wendy xylophone 153.62999725341797 +wendy xylophone 223.94999885559082 +wendy young 40.22000026702881 +wendy young 513.8299942016602 +xavier allen 102.97000122070312 +xavier allen 168.3300018310547 +xavier allen 197.45999908447266 +xavier brown 55.20000076293945 +xavier brown 90.7300033569336 +xavier brown 96.2300033569336 +xavier carson 193.63999938964844 +xavier carson 265.1600036621094 +xavier davidson 63.349998474121094 +xavier davidson 264.27000427246094 +xavier davidson 288.1999988555908 +xavier ellison 138.42000198364258 +xavier ellison 262.6300048828125 +xavier garcia 148.66000366210938 +xavier hernandez 122.13999938964844 +xavier hernandez 164.97000122070312 +xavier hernandez 306.25 +xavier ichabod 211.84000635147095 +xavier ichabod 244.50000762939453 +xavier johnson 56.53999900817871 +xavier johnson 89.0999984741211 +xavier king 87.22000122070312 +xavier king 151.22999572753906 +xavier laertes 183.65999794006348 +xavier ovid 398.2100067138672 +xavier polk 72.62000274658203 +xavier polk 76.93000030517578 +xavier polk 261.5100030899048 +xavier polk 318.01000213623047 +xavier quirinius 22.1200008392334 +xavier quirinius 89.55000305175781 +xavier quirinius 246.2400016784668 +xavier quirinius 402.2100009918213 +xavier thompson 283.9400005340576 +xavier underhill 120.45000076293945 +xavier white 138.02999591827393 +xavier white 172.06999969482422 +xavier xylophone 79.41999816894531 +xavier zipper 373.67999935150146 +yuri allen 52.849998474121094 +yuri allen 417.3700008392334 +yuri brown 170.52000427246094 +yuri brown 180.70999908447266 +yuri carson 188.99000549316406 +yuri carson 537.6500015258789 +yuri ellison 86.91999816894531 +yuri ellison 376.32999646663666 +yuri falkner 152.99000358581543 +yuri falkner 181.06999969482422 +yuri garcia 274.6800003051758 +yuri hernandez 153.46999764442444 +yuri johnson 197.28000259399414 +yuri johnson 236.0800018310547 +yuri johnson 258.1899985074997 +yuri king 551.9899978637695 +yuri laertes 37.59000015258789 +yuri laertes 253.4799976348877 +yuri nixon 95.54999732971191 +yuri nixon 248.9700005054474 +yuri polk 82.33999633789062 +yuri polk 275.3200035095215 +yuri polk 305.6399974822998 +yuri quirinius 112.97000122070312 +yuri quirinius 148.27999877929688 +yuri quirinius 449.1699924468994 +yuri steinbeck 292.94000244140625 +yuri steinbeck 357.5 +yuri thompson 428.03999519348145 +yuri underhill 83.87000274658203 +yuri underhill 350.7999897003174 +yuri white 132.09000396728516 +yuri xylophone 107.07000160217285 +zach allen 65.43000030517578 +zach brown 135.6999969482422 +zach brown 247.04999542236328 +zach brown 256.8000030517578 +zach brown 362.38000106811523 +zach brown 418.75 +zach carson 291.7700004577637 +zach ellison 135.149995803833 +zach falkner 91.41999816894531 +zach falkner 196.41999912261963 +zach garcia 84.37999725341797 +zach garcia 160.70000457763672 +zach garcia 167.7599983215332 +zach garcia 205.36999893188477 +zach ichabod 116.2699966430664 +zach ichabod 151.18000030517578 +zach king 127.63000106811523 +zach king 182.2699966430664 +zach king 269.0999984741211 +zach miller 199.71000289916992 +zach miller 220.73999977111816 +zach miller 264.0600047111511 +zach ovid 92.55000305175781 +zach ovid 94.33999633789062 +zach ovid 105.94999847561121 +zach ovid 136.04000091552734 +zach quirinius 103.11000061035156 +zach robinson 76.72000122070312 +zach steinbeck 85.48999786376953 +zach steinbeck 182.87000274658203 +zach thompson 116.93999862670898 +zach thompson 319.9499969482422 +zach underhill 86.22000122070312 +zach white 70.52999877929688 +zach xylophone 227.52000427246094 +zach xylophone 286.45000076293945 +zach young 313.00999450683594 +zach zipper 85.87000274658203 +zach zipper 94.43000030517578 +zach zipper 139.38999938964844 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-3-6f104992e0050576085064815de43194 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-3-6f104992e0050576085064815de43194 new file mode 100644 index 0000000000000..ae2a1e9dd7d3f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-3-6f104992e0050576085064815de43194 @@ -0,0 +1,1049 @@ + 43.52666695912679 + 48.735000133514404 + 57.96666590372721 +alice allen 39.21833221117655 +alice allen 42.813999557495116 +alice allen 49.1824996471405 +alice brown 59.68166707456112 +alice carson 46.703333189090095 +alice davidson 59.51833279927572 +alice falkner 64.48333247502644 +alice garcia 55.114999771118164 +alice hernandez 49.61333228151003 +alice hernandez 69.70166667302449 +alice johnson 49.25166805585226 +alice king 20.052499681711197 +alice king 49.019999186197914 +alice king 56.0733331044515 +alice laertes 35.24999898672104 +alice laertes 68.85333251953125 +alice miller 55.542000198364256 +alice nixon 44.010000785191856 +alice nixon 50.900000762939456 +alice nixon 64.78333409627278 +alice ovid 24.90999937057495 +alice polk 47.426666259765625 +alice quirinius 52.80166610081991 +alice quirinius 52.94833393891653 +alice robinson 44.41500027974447 +alice robinson 55.04249954223633 +alice steinbeck 37.735000928243004 +alice steinbeck 41.02800045013428 +alice steinbeck 53.83499916394552 +alice underhill 52.64500021934509 +alice van buren 48.27666727701823 +alice xylophone 28.047500252723694 +alice xylophone 37.82199954986572 +alice xylophone 59.75166606903076 +alice zipper 48.875000635782875 +alice zipper 51.365000108877815 +alice zipper 89.93000030517578 +bob brown 38.134999910990395 +bob brown 56.20666662851969 +bob brown 77.51166598002116 +bob carson 52.77200050354004 +bob davidson 28.945000171661377 +bob davidson 37.946666399637856 +bob davidson 53.90333271026611 +bob ellison 51.383999824523926 +bob ellison 57.30333344141642 +bob ellison 58.53666559855143 +bob ellison 66.7400016784668 +bob falkner 39.053333600362144 +bob garcia 16.705000400543213 +bob garcia 37.16499960422516 +bob garcia 51.17999919255575 +bob garcia 52.37666575113932 +bob garcia 67.39166768391927 +bob hernandez 53.261999893188474 +bob ichabod 43.96999979019165 +bob king 38.03666718800863 +bob king 38.19249892234802 +bob king 67.40999794006348 +bob laertes 21.449999809265137 +bob laertes 50.37166612346967 +bob miller 41.906000471115114 +bob ovid 27.836666425069172 +bob ovid 39.3833335240682 +bob ovid 60.90749979019165 +bob ovid 63.069999313354494 +bob polk 41.88333296775818 +bob quirinius 54.504998207092285 +bob steinbeck 34.33999983469645 +bob van buren 58.29999923706055 +bob white 30.22333288192749 +bob white 41.44000005722046 +bob xylophone 12.163333415985107 +bob xylophone 47.88249921798706 +bob young 27.38499927520752 +bob zipper 27.93800084590912 +bob zipper 59.11800007820129 +bob zipper 66.43000030517578 +calvin allen 71.51000118255615 +calvin brown 50.44166612625122 +calvin brown 53.625000953674316 +calvin brown 67.48333485921223 +calvin carson 62.17750120162964 +calvin davidson 14.03000009059906 +calvin davidson 43.640000343322754 +calvin ellison 52.52750015258789 +calvin falkner 24.016666332880657 +calvin falkner 46.87333329518636 +calvin falkner 47.53666607538859 +calvin falkner 56.47499918937683 +calvin falkner 57.336666107177734 +calvin falkner 72.25 +calvin garcia 46.484999338785805 +calvin hernandez 43.90199909210205 +calvin johnson 76.41500091552734 +calvin laertes 39.09249973297119 +calvin laertes 47.323333422342934 +calvin nixon 30.113332668940227 +calvin nixon 47.7533327738444 +calvin nixon 49.08500075340271 +calvin ovid 41.924999713897705 +calvin ovid 49.10499978065491 +calvin ovid 62.26499938964844 +calvin ovid 62.27999989191691 +calvin polk 52.95333353678385 +calvin quirinius 53.34200019836426 +calvin quirinius 54.831998634338376 +calvin robinson 60.470001220703125 +calvin steinbeck 35.644999980926514 +calvin steinbeck 53.75800037384033 +calvin steinbeck 56.11000084877014 +calvin thompson 42.355000257492065 +calvin thompson 65.91666666666667 +calvin underhill 47.41199951171875 +calvin van buren 31.591666102409363 +calvin van buren 43.18200063705444 +calvin white 45.27500128746033 +calvin white 56.079999923706055 +calvin xylophone 20.394999980926514 +calvin xylophone 40.59999958674113 +calvin xylophone 54.56500005722046 +calvin young 42.246666272481285 +calvin young 64.49000072479248 +calvin zipper 57.49500036239624 +calvin zipper 57.59000039100647 +david allen 40.14999977747599 +david allen 54.545000076293945 +david brown 35.4516666730245 +david brown 62.83999938964844 +david davidson 35.50000067551931 +david davidson 52.75250005722046 +david davidson 74.1500015258789 +david davidson 95.80999755859375 +david ellison 43.43166727821032 +david ellison 52.18750047683716 +david ellison 72.58400039672851 +david hernandez 64.47600173950195 +david ichabod 29.518332719802856 +david ichabod 34.6100010573864 +david laertes 61.127999496459964 +david nixon 53.60249900817871 +david ovid 37.977500796318054 +david ovid 41.58999983469645 +david quirinius 24.96250009536743 +david quirinius 45.23249912261963 +david quirinius 52.282000350952146 +david robinson 42.17750024795532 +david robinson 62.54666519165039 +david thompson 41.88999938964844 +david underhill 48.143333752950035 +david underhill 62.53499941031138 +david underhill 97.55999755859375 +david van buren 34.84749984741211 +david van buren 51.61666679382324 +david white 62.30499839782715 +david xylophone 33.92500019073486 +david xylophone 53.142000222206114 +david xylophone 72.77166684468587 +david young 30.13666756947835 +david young 51.540000915527344 +ethan allen 53.442500591278076 +ethan brown 7.110000133514404 +ethan brown 41.260000228881836 +ethan brown 41.495000084241234 +ethan brown 41.83833312988281 +ethan brown 46.497499227523804 +ethan brown 63.17750072479248 +ethan carson 64.31999969482422 +ethan ellison 46.72599983215332 +ethan ellison 61.24750167876482 +ethan falkner 36.90250104665756 +ethan falkner 52.71000099182129 +ethan garcia 32.771666407585144 +ethan hernandez 56.239999008178714 +ethan johnson 82.38333384195964 +ethan king 8.399999856948853 +ethan laertes 37.93200054168701 +ethan laertes 40.4883329073588 +ethan laertes 53.396667132774986 +ethan laertes 53.81666628519694 +ethan laertes 55.987499713897705 +ethan laertes 66.36999956766765 +ethan laertes 68.83400039672851 +ethan miller 56.519999504089355 +ethan nixon 50.13333353648583 +ethan ovid 37.51333363850912 +ethan polk 2.3499999046325684 +ethan polk 44.147999954223636 +ethan polk 54.91499996185303 +ethan polk 60.340000788370766 +ethan quirinius 40.21750020980835 +ethan quirinius 47.125000298023224 +ethan quirinius 57.69000013669332 +ethan robinson 41.5060001373291 +ethan robinson 55.0640007019043 +ethan underhill 68.01250076293945 +ethan van buren 43.55250072479248 +ethan white 46.40999913215637 +ethan white 53.03200073242188 +ethan xylophone 70.29333432515462 +ethan zipper 46.92999982833862 +ethan zipper 66.04500198364258 +fred davidson 26.487499618902802 +fred davidson 44.99333477020264 +fred davidson 53.93666648864746 +fred ellison 22.2599999109904 +fred ellison 48.047999954223634 +fred ellison 63.423333485921226 +fred falkner 20.035000324249268 +fred falkner 44.243999004364014 +fred falkner 62.130001068115234 +fred hernandez 37.42199997901916 +fred ichabod 46.56999969482422 +fred ichabod 58.275001525878906 +fred johnson 54.63333257039388 +fred king 46.30499919255575 +fred king 61.48500037193298 +fred laertes 26.203333059946697 +fred miller 43.46400032043457 +fred nixon 28.690000534057617 +fred nixon 32.77999955415726 +fred nixon 52.03799936771393 +fred nixon 60.468332290649414 +fred polk 26.81166632970174 +fred polk 42.48999996185303 +fred polk 46.95200061798096 +fred polk 60.465998840332034 +fred quirinius 43.466000366210935 +fred quirinius 45.79799957275391 +fred robinson 62.42833296457926 +fred steinbeck 32.22999954223633 +fred steinbeck 48.73800039291382 +fred steinbeck 65.91750144958496 +fred underhill 72.94250011444092 +fred van buren 41.28250002861023 +fred van buren 41.5283338278532 +fred van buren 50.871665954589844 +fred van buren 81.77000045776367 +fred white 34.897499561309814 +fred young 58.56666692097982 +fred young 70.61499977111816 +fred zipper 33.90500068664551 +gabriella allen 51.041666666666664 +gabriella allen 55.13999938964844 +gabriella brown 54.63333320617676 +gabriella brown 72.33333396911621 +gabriella carson 49.26333236694336 +gabriella davidson 52.72800083160401 +gabriella ellison 54.68999989827474 +gabriella ellison 71.54000091552734 +gabriella falkner 20.834000015258788 +gabriella falkner 46.348333517710365 +gabriella falkner 48.58500099182129 +gabriella garcia 24.78999964396159 +gabriella hernandez 57.61000029246012 +gabriella hernandez 65.11166667938232 +gabriella ichabod 33.78749895095825 +gabriella ichabod 38.9950008392334 +gabriella ichabod 48.15499925613403 +gabriella ichabod 49.739999008178714 +gabriella ichabod 52.789999643961586 +gabriella king 35.58200044631958 +gabriella king 49.63000059723854 +gabriella laertes 47.81000010172526 +gabriella miller 62.17666753133138 +gabriella ovid 45.94333267211914 +gabriella ovid 50.435001373291016 +gabriella polk 42.58000100851059 +gabriella polk 72.87000179290771 +gabriella steinbeck 65.42000102996826 +gabriella steinbeck 72.0499997138977 +gabriella thompson 49.897499561309814 +gabriella thompson 52.84250023961067 +gabriella thompson 57.23249959945679 +gabriella van buren 39.31000053882599 +gabriella van buren 57.920000076293945 +gabriella white 49.85333410898844 +gabriella young 15.369999885559082 +gabriella young 49.04999907811483 +gabriella zipper 42.82499901453654 +gabriella zipper 59.58499972025553 +holly allen 29.50499963760376 +holly brown 55.284998178482056 +holly brown 55.79833386838436 +holly falkner 40.07666663328806 +holly hernandez 46.40500044822693 +holly hernandez 56.14500045776367 +holly hernandez 56.160000483194985 +holly hernandez 58.95166748017073 +holly ichabod 67.19250011444092 +holly ichabod 68.44500064849854 +holly ichabod 90.09000015258789 +holly johnson 42.795000076293945 +holly johnson 55.76999855041504 +holly johnson 66.11333338419597 +holly king 48.788000869750974 +holly king 64.25 +holly laertes 50.40750074386597 +holly miller 45.60666608810425 +holly nixon 58.096665700276695 +holly nixon 69.59249877929688 +holly polk 41.59666601816813 +holly polk 44.64799900054932 +holly robinson 48.405999755859376 +holly thompson 34.605000495910645 +holly thompson 38.36749941110611 +holly thompson 65.99833394338687 +holly underhill 46.52600040435791 +holly underhill 47.92666663726171 +holly underhill 62.41600036621094 +holly underhill 79.95999908447266 +holly van buren 58.63333225250244 +holly white 31.232499361038208 +holly white 61.494998931884766 +holly xylophone 70.32500044504802 +holly young 54.05000114440918 +holly young 57.103333473205566 +holly zipper 50.59500014781952 +holly zipper 67.81000137329102 +irene allen 53.364000701904295 +irene brown 4.789999961853027 +irene brown 51.65399932861328 +irene brown 87.66999816894531 +irene carson 59.2433336575826 +irene ellison 36.446667989095054 +irene ellison 53.85000038146973 +irene falkner 47.429999995231626 +irene falkner 61.20666694641113 +irene garcia 15.369999885559082 +irene garcia 48.28000005086263 +irene garcia 55.643333435058594 +irene ichabod 40.868333299954735 +irene ichabod 64.45666694641113 +irene johnson 44.37999935150147 +irene laertes 27.625 +irene laertes 46.25 +irene laertes 49.17500019073486 +irene miller 75.87500063578288 +irene nixon 29.780000686645508 +irene nixon 30.070000807444256 +irene nixon 36.34333356221517 +irene ovid 27.21500023206075 +irene ovid 55.6175012588501 +irene ovid 60.353999328613284 +irene polk 47.438334147135414 +irene polk 47.77666505177816 +irene polk 51.74200067520142 +irene polk 52.12200088500977 +irene polk 52.184000205993655 +irene quirinius 52.958333015441895 +irene quirinius 55.29000053405762 +irene quirinius 78.79000091552734 +irene robinson 61.398332595825195 +irene steinbeck 64.34000142415364 +irene thompson 41.92250097543001 +irene underhill 27.72499966621399 +irene underhill 50.783331871032715 +irene van buren 46.65999984741211 +irene van buren 49.71000099182129 +irene xylophone 61.220001220703125 +jessica brown 63.449999491373696 +jessica carson 41.78999951481819 +jessica carson 47.52000069618225 +jessica carson 51.83000183105469 +jessica davidson 34.938333332538605 +jessica davidson 45.91000066200892 +jessica davidson 51.082499504089355 +jessica davidson 64.3099988301595 +jessica ellison 32.53000124295553 +jessica ellison 35.265999984741214 +jessica falkner 54.933334032694496 +jessica garcia 32.575000286102295 +jessica garcia 60.584999084472656 +jessica ichabod 46.704999923706055 +jessica johnson 55.73199977874756 +jessica johnson 72.14000066121419 +jessica miller 55.27799835205078 +jessica nixon 58.53999900817871 +jessica nixon 90.06999969482422 +jessica ovid 36.47500014305115 +jessica ovid 63.03000005086263 +jessica polk 69.52999877929688 +jessica quirinius 19.01333288351695 +jessica quirinius 41.630000829696655 +jessica quirinius 42.58750009536743 +jessica quirinius 47.72999986012777 +jessica robinson 64.81000061035157 +jessica thompson 45.83666737874349 +jessica thompson 57.540000915527344 +jessica underhill 45.360000133514404 +jessica underhill 51.40999889373779 +jessica underhill 64.27250099182129 +jessica van buren 9.739999771118164 +jessica white 38.38999891281128 +jessica white 40.75400023460388 +jessica white 61.89800128936768 +jessica white 62.34749984741211 +jessica white 65.7680004119873 +jessica xylophone 50.808332761128746 +jessica young 18.400000254313152 +jessica young 49.04400033950806 +jessica zipper 35.039999643961586 +jessica zipper 52.78999869028727 +jessica zipper 55.51499891281128 +katie allen 56.10800056457519 +katie brown 48.86833222707113 +katie davidson 96.91000366210938 +katie ellison 31.106667200724285 +katie ellison 38.356666127840676 +katie falkner 17.24333318074544 +katie garcia 53.429999033610024 +katie garcia 53.933334002892174 +katie hernandez 72.71666717529297 +katie ichabod 39.97999954223633 +katie ichabod 54.99599914550781 +katie ichabod 59.41999944051107 +katie king 44.88599967956543 +katie king 60.2549991607666 +katie king 67.27499961853027 +katie miller 31.399999618530273 +katie miller 50.57666619618734 +katie nixon 23.190000534057617 +katie ovid 52.96000158786774 +katie polk 51.029998779296875 +katie polk 54.93600006103516 +katie robinson 13.890000343322754 +katie van buren 52.78999908765157 +katie van buren 53.21750092506409 +katie white 59.799998950958255 +katie white 64.70499992370605 +katie xylophone 53.54499959945679 +katie young 31.010000228881836 +katie young 37.55500102043152 +katie young 49.87499872843424 +katie zipper 29.79666694998741 +katie zipper 50.95000076293945 +luke allen 43.757998657226565 +luke allen 53.18000030517578 +luke allen 62.09749984741211 +luke allen 70.2933349609375 +luke allen 89.55000305175781 +luke brown 45.95999972025553 +luke davidson 28.950000762939453 +luke davidson 53.20500040054321 +luke ellison 5.159999907016754 +luke ellison 18.630000392595928 +luke ellison 76.88000106811523 +luke falkner 6.175000190734863 +luke falkner 54.0049991607666 +luke garcia 15.320000424981117 +luke garcia 25.47000026702881 +luke ichabod 33.950000286102295 +luke ichabod 57.17500114440918 +luke johnson 14.4399995803833 +luke johnson 37.23333263397217 +luke johnson 52.660000801086426 +luke laertes 11.819999694824219 +luke laertes 23.5 +luke laertes 43.71999907493591 +luke laertes 45.9900016784668 +luke laertes 57.85499906539917 +luke miller 67.85000038146973 +luke ovid 38.04999923706055 +luke ovid 79.64200134277344 +luke polk 55.106666564941406 +luke polk 70.18999862670898 +luke quirinius 26.016666491826374 +luke robinson 61.256665547688804 +luke robinson 72.61999893188477 +luke thompson 47.203332940737404 +luke underhill 49.244999408721924 +luke underhill 59.32999897003174 +luke underhill 72.18333307902019 +luke van buren 64.646666208903 +luke white 67.12000274658203 +luke xylophone 46.74333190917969 +luke zipper 36.16000008583069 +mike allen 10.709999859333038 +mike brown 61.0674991607666 +mike carson 50.90249824523926 +mike carson 52.30200061798096 +mike carson 52.51499938964844 +mike davidson 29.733333269755047 +mike davidson 47.396666526794434 +mike ellison 37.807498931884766 +mike ellison 44.339999198913574 +mike ellison 45.89199924468994 +mike ellison 47.91249990463257 +mike ellison 55.295000076293945 +mike falkner 16.479999542236328 +mike garcia 38.98999913533529 +mike garcia 54.91333325703939 +mike garcia 57.096666971842446 +mike hernandez 59.45000076293945 +mike hernandez 62.446667989095054 +mike ichabod 54.69666544596354 +mike king 38.790000915527344 +mike king 43.2050017118454 +mike king 46.85666592915853 +mike king 46.945000648498535 +mike king 53.382500648498535 +mike king 94.68000030517578 +mike miller 3.9600000381469727 +mike nixon 53.894999742507935 +mike nixon 92.95999908447266 +mike polk 12.449999809265137 +mike polk 39.040000319480896 +mike polk 71.40500068664551 +mike quirinius 89.37999725341797 +mike steinbeck 34.05499875545502 +mike steinbeck 45.909999465942384 +mike steinbeck 51.286667346954346 +mike steinbeck 59.292500019073486 +mike van buren 43.13999819755554 +mike van buren 49.59000015258789 +mike white 30.53999964396159 +mike white 34.61499961217245 +mike white 48.08999943733215 +mike white 53.967501163482666 +mike young 10.484999895095825 +mike young 47.070000648498535 +mike young 74.58999633789062 +mike zipper 29.029999288419884 +mike zipper 77.96200027465821 +mike zipper 91.57500076293945 +nick allen 29.146666367848713 +nick allen 64.4325008392334 +nick brown 49.429999669392906 +nick davidson 34.36599922180176 +nick ellison 49.345001220703125 +nick ellison 89.01000213623047 +nick falkner 7.820000171661377 +nick falkner 45.025000631809235 +nick garcia 23.9499994913737 +nick garcia 33.49333349863688 +nick garcia 64.33499908447266 +nick ichabod 30.945000171661377 +nick ichabod 56.27499961853027 +nick ichabod 59.24249863624573 +nick johnson 32.3674995303154 +nick johnson 74.30666859944661 +nick laertes 38.38749980926514 +nick miller 82.97000122070312 +nick nixon 70.01333173116048 +nick ovid 56.82999897003174 +nick polk 33.929999669392906 +nick quirinius 58.91199932098389 +nick quirinius 81.16999816894531 +nick robinson 34.53999948501587 +nick robinson 59.80499839782715 +nick steinbeck 57.25333340962728 +nick thompson 51.3725004196167 +nick underhill 55.47333335876465 +nick van buren 74.21666717529297 +nick xylophone 51.82000001271566 +nick young 0.27000001072883606 +nick young 41.02499961853027 +nick zipper 47.72999954223633 +nick zipper 61.917999267578125 +oscar allen 35.21600015163422 +oscar brown 38.03999948501587 +oscar carson 38.09600009918213 +oscar carson 51.34333419799805 +oscar carson 56.9925012588501 +oscar carson 74.00500106811523 +oscar carson 98.51000213623047 +oscar davidson 65.88750076293945 +oscar ellison 50.507500410079956 +oscar ellison 66.57000096638997 +oscar falkner 64.42000198364258 +oscar garcia 66.36333465576172 +oscar hernandez 42.089999516805015 +oscar hernandez 51.21199997067451 +oscar ichabod 25.300000190734863 +oscar ichabod 41.173332850138344 +oscar ichabod 53.44666830698649 +oscar ichabod 71.80000305175781 +oscar johnson 39.9366668065389 +oscar johnson 44.30500118434429 +oscar king 30.59749937057495 +oscar king 46.149999141693115 +oscar king 49.65999984741211 +oscar laertes 5.510000228881836 +oscar laertes 19.099999745686848 +oscar laertes 39.36250039935112 +oscar laertes 45.340000788370766 +oscar nixon 41.619998931884766 +oscar ovid 45.09000015258789 +oscar ovid 82.23999786376953 +oscar ovid 83.47750091552734 +oscar polk 42.54333368937174 +oscar polk 56.8033332824707 +oscar quirinius 52.94500160217285 +oscar quirinius 65.34666570027669 +oscar quirinius 67.00000127156575 +oscar quirinius 79.4800033569336 +oscar robinson 32.16999944051107 +oscar robinson 38.880001068115234 +oscar robinson 63.9433339436849 +oscar robinson 65.80500030517578 +oscar steinbeck 45.73599967956543 +oscar thompson 40.33599853515625 +oscar thompson 47.860000928243004 +oscar thompson 60.529998779296875 +oscar thompson 60.59333292643229 +oscar underhill 43.980000764131546 +oscar van buren 2.180000066757202 +oscar van buren 53.34999942779541 +oscar van buren 61.880001068115234 +oscar white 38.0633331934611 +oscar white 42.584999084472656 +oscar white 55.179999033610024 +oscar white 74.4900016784668 +oscar xylophone 41.3133331934611 +oscar xylophone 55.5049991607666 +oscar xylophone 67.54500007629395 +oscar zipper 15.680000305175781 +oscar zipper 24.019999504089355 +oscar zipper 39.81999969482422 +priscilla brown 42.88666502634684 +priscilla brown 63.242499351501465 +priscilla brown 77.56999969482422 +priscilla carson 44.799999713897705 +priscilla carson 49.663333892822266 +priscilla carson 78.57333374023438 +priscilla ichabod 56.83666737874349 +priscilla ichabod 58.48666636149088 +priscilla johnson 34.416667779286705 +priscilla johnson 52.890000343322754 +priscilla johnson 53.72666883468628 +priscilla johnson 61.939998626708984 +priscilla johnson 89.1500015258789 +priscilla king 34.30750045180321 +priscilla nixon 27.734999656677246 +priscilla nixon 60.90999984741211 +priscilla ovid 48.13500142097473 +priscilla ovid 66.35999870300293 +priscilla polk 15.149999618530273 +priscilla quirinius 18.606666564941406 +priscilla thompson 48.87000131607056 +priscilla underhill 40.05500078201294 +priscilla underhill 49.54999961853027 +priscilla van buren 42.62666734059652 +priscilla van buren 61.54000017642975 +priscilla van buren 72.80500030517578 +priscilla white 43.177499771118164 +priscilla xylophone 21.489999771118164 +priscilla xylophone 40.144999124109745 +priscilla xylophone 59.61000061035156 +priscilla young 31.610000324249267 +priscilla young 53.71000152826309 +priscilla zipper 18.8799991607666 +priscilla zipper 25.670000076293945 +quinn allen 56.77749991416931 +quinn allen 76.47250080108643 +quinn brown 23.536666870117188 +quinn brown 31.829999446868896 +quinn brown 50.388000297546384 +quinn davidson 41.42499923706055 +quinn davidson 45.90250015258789 +quinn davidson 48.16000032424927 +quinn davidson 71.51000022888184 +quinn ellison 50.6766668955485 +quinn ellison 62.32666842142741 +quinn garcia 39.98599967956543 +quinn garcia 43.27000045776367 +quinn garcia 74.31999969482422 +quinn garcia 92.33000183105469 +quinn ichabod 44.81999969482422 +quinn king 50.99666786193848 +quinn king 74.62000274658203 +quinn laertes 5.884999990463257 +quinn laertes 49.1379997253418 +quinn laertes 56.18000030517578 +quinn nixon 74.66500091552734 +quinn ovid 1.2100000381469727 +quinn quirinius 46.15500068664551 +quinn robinson 44.96249985694885 +quinn steinbeck 24.06999921798706 +quinn steinbeck 41.750000953674316 +quinn thompson 23.744999408721924 +quinn thompson 73.64500045776367 +quinn underhill 41.83333269755045 +quinn underhill 49.63749885559082 +quinn underhill 83.85000228881836 +quinn van buren 54.36333338419596 +quinn young 45.489999771118164 +quinn zipper 22.25 +quinn zipper 33.355000019073486 +rachel allen 15.8100004196167 +rachel allen 71.57666524251302 +rachel brown 2.9600000381469727 +rachel brown 33.022499561309814 +rachel brown 42.442498207092285 +rachel brown 59.21999931335449 +rachel brown 64.52666727701823 +rachel carson 61.17999776204427 +rachel carson 69.85333315531413 +rachel davidson 42.292500495910645 +rachel ellison 10.600000381469727 +rachel falkner 23.615000784397125 +rachel falkner 50.46250069141388 +rachel falkner 54.669999837875366 +rachel falkner 72.96999740600586 +rachel johnson 61.76249885559082 +rachel king 22.005000591278076 +rachel king 66.46500015258789 +rachel laertes 39.025001525878906 +rachel laertes 45.45000076293945 +rachel ovid 0.6000000238418579 +rachel ovid 1.0800000429153442 +rachel polk 78.08499908447266 +rachel quirinius 51.12000131607056 +rachel robinson 30.360000610351562 +rachel robinson 37.683333237965904 +rachel robinson 39.199998219807945 +rachel thompson 0.5600000023841858 +rachel thompson 38.81000010172526 +rachel thompson 49.772499561309814 +rachel underhill 48.45000076293945 +rachel white 42.83999991416931 +rachel white 58.609999656677246 +rachel young 43.130001068115234 +rachel zipper 7.059999942779541 +rachel zipper 49.676667173703514 +sarah carson 54.022500067949295 +sarah carson 54.18333212534586 +sarah carson 87.81000137329102 +sarah ellison 16.989999771118164 +sarah falkner 73.06500005722046 +sarah falkner 99.36000061035156 +sarah garcia 38.43250036239624 +sarah garcia 64.97333272298177 +sarah garcia 73.6500015258789 +sarah ichabod 57.46000003814697 +sarah ichabod 81.31999969482422 +sarah johnson 34.98499917984009 +sarah johnson 45.099998474121094 +sarah johnson 74.42249870300293 +sarah johnson 74.72000122070312 +sarah king 41.869998931884766 +sarah king 48.25 +sarah miller 41.53499984741211 +sarah ovid 33.38000011444092 +sarah robinson 33.83000183105469 +sarah robinson 66.88999938964844 +sarah steinbeck 31.023332993189495 +sarah white 45.974998474121094 +sarah white 61.54666709899902 +sarah xylophone 61.60666529337565 +sarah young 45.560001373291016 +sarah zipper 60.69000053405762 +tom brown 40.08000135421753 +tom brown 55.44499969482422 +tom carson 5.440000057220459 +tom carson 26.32499885559082 +tom carson 31.476666768391926 +tom davidson 53.00749921798706 +tom ellison 67.32666714986165 +tom ellison 76.73999786376953 +tom ellison 77.29499816894531 +tom falkner 60.130001068115234 +tom falkner 88.22000122070312 +tom hernandez 41.36000061035156 +tom hernandez 81.63999938964844 +tom ichabod 42.08666547139486 +tom johnson 33.75999959309896 +tom johnson 53.47666549682617 +tom king 40.0 +tom laertes 32.05000114440918 +tom laertes 43.46000099182129 +tom miller 21.229999542236328 +tom miller 43.37666702270508 +tom miller 44.41333325703939 +tom nixon 46.35333251953125 +tom ovid 43.04499912261963 +tom polk 38.29999923706055 +tom polk 54.45000076293945 +tom quirinius 19.82000058889389 +tom quirinius 22.580000400543213 +tom robinson 66.16999816894531 +tom robinson 74.18666712443034 +tom robinson 80.77000045776367 +tom robinson 98.72000122070312 +tom steinbeck 44.999999046325684 +tom van buren 31.103334546089172 +tom van buren 40.779998779296875 +tom van buren 63.5099983215332 +tom white 49.06500053405762 +tom young 54.16999912261963 +tom young 78.54999923706055 +tom zipper 48.666666984558105 +ulysses brown 72.79000091552734 +ulysses carson 40.28500175476074 +ulysses carson 71.55000305175781 +ulysses carson 77.41999816894531 +ulysses carson 79.54000091552734 +ulysses davidson 41.90166711807251 +ulysses ellison 96.7300033569336 +ulysses garcia 89.80000305175781 +ulysses hernandez 21.339999516805012 +ulysses hernandez 54.470001220703125 +ulysses hernandez 80.11000061035156 +ulysses ichabod 19.1299991607666 +ulysses ichabod 98.56999969482422 +ulysses johnson 42.55000019073486 +ulysses king 81.57000223795573 +ulysses laertes 1.9199999570846558 +ulysses laertes 11.890000343322754 +ulysses laertes 53.599998474121094 +ulysses miller 40.43499946594238 +ulysses miller 47.78200054168701 +ulysses nixon 74.60333251953125 +ulysses ovid 30.940000534057617 +ulysses polk 8.710000038146973 +ulysses polk 39.17500019073486 +ulysses polk 65.0199966430664 +ulysses polk 68.41333389282227 +ulysses quirinius 65.93499946594238 +ulysses robinson 79.48999786376953 +ulysses steinbeck 32.40999984741211 +ulysses steinbeck 43.93499994277954 +ulysses thompson 79.96000289916992 +ulysses underhill 17.85000006357829 +ulysses underhill 42.5533332824707 +ulysses underhill 46.60000157356262 +ulysses underhill 47.84000049829483 +ulysses underhill 51.85000133514404 +ulysses underhill 67.77999877929688 +ulysses underhill 99.66999816894531 +ulysses van buren 69.89999961853027 +ulysses white 45.01500183343887 +ulysses white 71.3933334350586 +ulysses xylophone 27.880000591278076 +ulysses xylophone 39.689998626708984 +ulysses xylophone 54.099998474121094 +ulysses young 32.52000045776367 +ulysses young 39.81333382924398 +ulysses young 80.7933349609375 +victor allen 44.27000045776367 +victor allen 57.994998931884766 +victor brown 56.84499931335449 +victor brown 59.34000015258789 +victor brown 90.37999725341797 +victor brown 91.97000122070312 +victor davidson 52.07000160217285 +victor davidson 54.239999771118164 +victor davidson 68.8033332824707 +victor ellison 45.45750088989735 +victor ellison 58.42999839782715 +victor hernandez 23.164999961853027 +victor hernandez 57.98499870300293 +victor hernandez 59.619998931884766 +victor hernandez 71.42499923706055 +victor hernandez 80.19499969482422 +victor johnson 1.5800000429153442 +victor johnson 46.7450008392334 +victor johnson 72.59000015258789 +victor king 37.559998989105225 +victor king 47.880001068115234 +victor laertes 50.0099983215332 +victor laertes 70.13999938964844 +victor miller 22.1200008392334 +victor nixon 34.029998779296875 +victor nixon 37.08500003814697 +victor ovid 18.815000653266907 +victor polk 3.0 +victor quirinius 26.450000405311584 +victor quirinius 33.080001533031464 +victor robinson 4.590000152587891 +victor robinson 41.21999931335449 +victor steinbeck 41.82500123977661 +victor steinbeck 46.09000015258789 +victor steinbeck 52.720001220703125 +victor thompson 45.346666971842446 +victor van buren 34.970001220703125 +victor van buren 46.57333246866862 +victor white 5.670000076293945 +victor white 74.16999816894531 +victor xylophone 11.220000267028809 +victor xylophone 28.954999923706055 +victor xylophone 34.010000228881836 +victor xylophone 43.179999351501465 +victor xylophone 62.38999938964844 +victor young 70.91000175476074 +victor zipper 48.795000076293945 +wendy allen 0.6100000143051147 +wendy allen 52.64999961853027 +wendy allen 56.06999969482422 +wendy brown 51.874999046325684 +wendy brown 66.73999913533528 +wendy ellison 27.014999389648438 +wendy ellison 94.66000366210938 +wendy falkner 14.425000190734863 +wendy falkner 77.36000061035156 +wendy falkner 85.68000030517578 +wendy garcia 4.409999847412109 +wendy garcia 20.390000343322754 +wendy garcia 38.42500060796738 +wendy garcia 57.25 +wendy hernandez 48.11000061035156 +wendy ichabod 13.149999618530273 +wendy king 33.234999656677246 +wendy king 74.97999954223633 +wendy king 87.94000244140625 +wendy laertes 49.01999855041504 +wendy laertes 54.750000635782875 +wendy laertes 79.98999786376953 +wendy miller 1.2699999809265137 +wendy miller 47.41500186920166 +wendy nixon 30.17999916151166 +wendy nixon 45.91999816894531 +wendy ovid 85.26000213623047 +wendy ovid 86.62999725341797 +wendy polk 43.679999669392906 +wendy polk 54.65333424011866 +wendy quirinius 12.15999984741211 +wendy quirinius 14.300000190734863 +wendy robinson 58.815001487731934 +wendy robinson 71.06999969482422 +wendy robinson 71.77999877929688 +wendy steinbeck 77.31500053405762 +wendy thompson 58.09000015258789 +wendy thompson 85.76000213623047 +wendy underhill 41.23333231608073 +wendy underhill 49.33500003814697 +wendy underhill 81.56000137329102 +wendy van buren 57.459999084472656 +wendy van buren 91.43500137329102 +wendy white 73.68000030517578 +wendy xylophone 31.149999618530273 +wendy xylophone 76.81499862670898 +wendy young 8.449999809265137 +wendy young 33.7599983215332 +xavier allen 45.68000030517578 +xavier allen 52.255000591278076 +xavier allen 83.93000030517578 +xavier brown 7.789999961853027 +xavier brown 77.15500259399414 +xavier brown 80.35000228881836 +xavier carson 20.790000915527344 +xavier carson 55.30000019073486 +xavier davidson 15.920000076293945 +xavier davidson 58.46999931335449 +xavier davidson 82.41000366210938 +xavier ellison 16.614999771118164 +xavier ellison 77.97000122070312 +xavier garcia 49.415000915527344 +xavier hernandez 6.670000076293945 +xavier hernandez 23.054999828338623 +xavier hernandez 67.26000213623047 +xavier ichabod 4.71999979019165 +xavier ichabod 71.19000244140625 +xavier johnson 27.299999237060547 +xavier johnson 89.0999984741211 +xavier king 22.729999542236328 +xavier king 87.22000122070312 +xavier laertes 24.050000190734863 +xavier ovid 58.08000183105469 +xavier polk 13.869999885559082 +xavier polk 58.98750066757202 +xavier polk 72.62000274658203 +xavier polk 76.93000030517578 +xavier quirinius 22.1200008392334 +xavier quirinius 58.24666786193848 +xavier quirinius 62.52000045776367 +xavier quirinius 89.55000305175781 +xavier thompson 9.930000305175781 +xavier underhill 47.27000045776367 +xavier white 8.369999885559082 +xavier white 75.29000091552734 +xavier xylophone 79.41999816894531 +xavier zipper 8.449999809265137 +yuri allen 30.6299991607666 +yuri allen 52.849998474121094 +yuri brown 75.19000244140625 +yuri brown 84.02999877929688 +yuri carson 6.289999961853027 +yuri carson 91.16000366210938 +yuri ellison 1.1200000047683716 +yuri ellison 86.91999816894531 +yuri falkner 6.739999771118164 +yuri falkner 80.8650016784668 +yuri garcia 27.65999984741211 +yuri hernandez 2.069999933242798 +yuri johnson 0.12999999523162842 +yuri johnson 39.900001525878906 +yuri johnson 48.220001220703125 +yuri king 22.270000457763672 +yuri laertes 10.15999984741211 +yuri laertes 37.59000015258789 +yuri nixon 2.200000047683716 +yuri nixon 17.3700008392334 +yuri polk 26.760000228881836 +yuri polk 28.790000915527344 +yuri polk 82.33999633789062 +yuri quirinius 10.260000228881836 +yuri quirinius 38.69000053405762 +yuri quirinius 57.93000030517578 +yuri steinbeck 56.064998626708984 +yuri steinbeck 75.87999725341797 +yuri thompson 14.920000076293945 +yuri underhill 23.770000457763672 +yuri underhill 83.87000274658203 +yuri white 34.58000183105469 +yuri xylophone 20.3799991607666 +zach allen 65.43000030517578 +zach brown 48.0099983215332 +zach brown 57.08000183105469 +zach brown 58.24999809265137 +zach brown 67.37999725341797 +zach brown 75.7300033569336 +zach carson 67.78500175476074 +zach ellison 6.840000152587891 +zach falkner 50.274999141693115 +zach falkner 91.41999816894531 +zach garcia 32.20000076293945 +zach garcia 35.79999923706055 +zach garcia 69.97000122070312 +zach garcia 84.37999725341797 +zach ichabod 36.88999938964844 +zach ichabod 64.25 +zach king 31.864999771118164 +zach king 46.18000030517578 +zach king 86.93000030517578 +zach miller 2.5999999046325684 +zach miller 21.280000686645508 +zach miller 53.27000045776367 +zach ovid 0.10000000149011612 +zach ovid 23.06999969482422 +zach ovid 92.55000305175781 +zach ovid 94.33999633789062 +zach quirinius 39.209999084472656 +zach robinson 76.72000122070312 +zach steinbeck 85.48999786376953 +zach steinbeck 90.05000305175781 +zach thompson 53.59000015258789 +zach thompson 71.5 +zach underhill 86.22000122070312 +zach white 70.52999877929688 +zach xylophone 29.40999984741211 +zach xylophone 71.01000213623047 +zach young 71.31999969482422 +zach zipper 7.539999961853027 +zach zipper 85.87000274658203 +zach zipper 94.43000030517578 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-4-cd2e3d2344810cb3ba843d4c01c81d7e b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-4-cd2e3d2344810cb3ba843d4c01c81d7e new file mode 100644 index 0000000000000..ee1c26e331a1e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-4-cd2e3d2344810cb3ba843d4c01c81d7e @@ -0,0 +1,1049 @@ + 17.601666666666667 + 30.72 + 33.07833333333334 +alice allen 23.081666666666667 +alice allen 23.263333333333332 +alice allen 31.38 +alice brown 11.518333333333333 +alice carson 31.99166666666667 +alice davidson 26.986666666666665 +alice falkner 30.513333333333335 +alice garcia 17.13 +alice hernandez 28.126666666666665 +alice hernandez 28.69666666666667 +alice johnson 28.30333333333333 +alice king 23.451666666666668 +alice king 25.20333333333333 +alice king 30.613999999999997 +alice laertes 23.633333333333336 +alice laertes 30.16428571428571 +alice miller 32.50833333333333 +alice nixon 25.278333333333336 +alice nixon 35.345 +alice nixon 36.458333333333336 +alice ovid 26.04714285714286 +alice polk 23.185 +alice quirinius 20.388333333333335 +alice quirinius 23.064999999999998 +alice robinson 30.296666666666667 +alice robinson 33.15 +alice steinbeck 27.894999999999996 +alice steinbeck 31.47833333333334 +alice steinbeck 36.089999999999996 +alice underhill 22.513333333333335 +alice van buren 36.32000000000001 +alice xylophone 27.355000000000004 +alice xylophone 30.505714285714284 +alice xylophone 30.613999999999997 +alice zipper 20.113333333333333 +alice zipper 28.058333333333337 +alice zipper 29.705000000000002 +bob brown 13.546666666666669 +bob brown 16.50333333333333 +bob brown 27.59 +bob carson 25.781666666666666 +bob davidson 18.073333333333334 +bob davidson 23.573333333333334 +bob davidson 31.894999999999996 +bob ellison 16.493333333333336 +bob ellison 17.889999999999997 +bob ellison 25.84142857142857 +bob ellison 33.07833333333334 +bob falkner 21.783333333333335 +bob garcia 16.492857142857144 +bob garcia 23.088333333333335 +bob garcia 26.42833333333333 +bob garcia 32.709999999999994 +bob garcia 33.91833333333334 +bob hernandez 22.303333333333338 +bob ichabod 23.336666666666662 +bob king 20.591666666666665 +bob king 21.244999999999997 +bob king 33.15333333333333 +bob laertes 21.240000000000002 +bob laertes 28.88 +bob miller 32.158750000000005 +bob ovid 21.83666666666667 +bob ovid 23.678571428571427 +bob ovid 25.12333333333333 +bob ovid 31.46 +bob polk 16.3475 +bob quirinius 28.465714285714284 +bob steinbeck 26.99 +bob van buren 26.127142857142854 +bob white 23.021666666666672 +bob white 23.582857142857144 +bob xylophone 26.18166666666667 +bob xylophone 27.995 +bob young 25.495714285714286 +bob zipper 26.435000000000002 +bob zipper 28.384285714285713 +bob zipper 30.65833333333333 +calvin allen 20.812 +calvin brown 20.808 +calvin brown 28.26 +calvin brown 28.37875 +calvin carson 20.38714285714286 +calvin davidson 22.65142857142857 +calvin davidson 23.585714285714285 +calvin ellison 24.6875 +calvin falkner 16.073333333333334 +calvin falkner 17.054285714285715 +calvin falkner 25.2025 +calvin falkner 28.525714285714283 +calvin falkner 33.382 +calvin falkner 37.29666666666667 +calvin garcia 19.924285714285713 +calvin hernandez 21.759999999999998 +calvin johnson 24.03222222222222 +calvin laertes 24.702857142857145 +calvin laertes 28.788333333333338 +calvin nixon 15.631250000000001 +calvin nixon 25.64428571428571 +calvin nixon 29.084999999999997 +calvin ovid 16.243333333333332 +calvin ovid 25.2025 +calvin ovid 25.935555555555553 +calvin ovid 29.299999999999997 +calvin polk 29.615 +calvin quirinius 19.294999999999998 +calvin quirinius 31.64625 +calvin robinson 27.811428571428575 +calvin steinbeck 17.14 +calvin steinbeck 17.535 +calvin steinbeck 21.551666666666666 +calvin thompson 28.592857142857145 +calvin thompson 33.382 +calvin underhill 20.113333333333333 +calvin van buren 28.384285714285713 +calvin van buren 32.106 +calvin white 26.948333333333334 +calvin white 28.256249999999998 +calvin xylophone 25.33666666666667 +calvin xylophone 27.061428571428568 +calvin xylophone 28.384285714285713 +calvin young 19.331666666666667 +calvin young 24.63 +calvin zipper 16.08125 +calvin zipper 28.80555555555556 +david allen 26.21375 +david allen 26.948333333333334 +david brown 14.222 +david brown 27.1 +david davidson 21.062857142857144 +david davidson 26.12 +david davidson 30.951428571428576 +david davidson 31.96142857142857 +david ellison 22.687142857142856 +david ellison 23.435 +david ellison 25.442999999999998 +david hernandez 28.279999999999998 +david ichabod 16.3475 +david ichabod 23.751428571428573 +david laertes 29.07 +david nixon 25.535714285714285 +david ovid 26.75714285714286 +david ovid 29.912857142857142 +david quirinius 17.179999999999996 +david quirinius 20.808 +david quirinius 22.90625 +david robinson 25.274 +david robinson 25.885 +david thompson 22.19125 +david underhill 20.812 +david underhill 21.546666666666667 +david underhill 28.26 +david van buren 24.472857142857148 +david van buren 32.75125 +david white 22.861428571428576 +david xylophone 19.3325 +david xylophone 26.930000000000003 +david xylophone 30.093333333333334 +david young 21.12375 +david young 25.03857142857143 +ethan allen 27.038333333333338 +ethan brown 16.3475 +ethan brown 19.37 +ethan brown 23.73 +ethan brown 25.57166666666667 +ethan brown 26.168333333333333 +ethan brown 31.893333333333334 +ethan carson 25.655714285714286 +ethan ellison 27.807777777777776 +ethan ellison 32.244285714285716 +ethan falkner 28.287142857142857 +ethan falkner 31.411428571428576 +ethan garcia 24.153750000000002 +ethan hernandez 17.986 +ethan johnson 31.54571428571429 +ethan king 22.62 +ethan laertes 15.045555555555556 +ethan laertes 17.889999999999997 +ethan laertes 24.28 +ethan laertes 25.2025 +ethan laertes 26.56888888888889 +ethan laertes 28.37875 +ethan laertes 34.84571428571429 +ethan miller 28.592857142857145 +ethan nixon 29.86833333333333 +ethan ovid 17.85166666666667 +ethan polk 16.463 +ethan polk 18.184545454545457 +ethan polk 25.737142857142857 +ethan polk 29.895000000000003 +ethan quirinius 21.827777777777776 +ethan quirinius 23.42 +ethan quirinius 35.54333333333333 +ethan robinson 24.03222222222222 +ethan robinson 36.35333333333333 +ethan underhill 21.545555555555556 +ethan van buren 15.21142857142857 +ethan white 29.702857142857145 +ethan white 33.33 +ethan xylophone 29.521666666666665 +ethan zipper 23.994 +ethan zipper 26.765 +fred davidson 27.729999999999997 +fred davidson 29.90625 +fred davidson 30.804999999999996 +fred ellison 16.720000000000002 +fred ellison 22.17125 +fred ellison 33.15833333333334 +fred falkner 17.13142857142857 +fred falkner 26.82 +fred falkner 31.925555555555555 +fred hernandez 28.094285714285718 +fred ichabod 23.352500000000003 +fred ichabod 32.906666666666666 +fred johnson 16.9925 +fred king 20.125 +fred king 30.377142857142854 +fred laertes 26.458571428571428 +fred miller 29.66666666666667 +fred nixon 19.565555555555555 +fred nixon 21.830000000000002 +fred nixon 25.828333333333333 +fred nixon 28.094285714285718 +fred polk 21.744999999999997 +fred polk 22.301666666666666 +fred polk 24.156666666666666 +fred polk 28.217142857142857 +fred quirinius 27.495 +fred quirinius 29.615714285714283 +fred robinson 24.243636363636366 +fred steinbeck 17.91333333333333 +fred steinbeck 21.12375 +fred steinbeck 26.47125 +fred underhill 26.43166666666667 +fred van buren 16.165714285714284 +fred van buren 23.285 +fred van buren 24.875714285714288 +fred van buren 27.878333333333334 +fred white 29.498571428571427 +fred young 17.889999999999997 +fred young 19.565555555555555 +fred zipper 21.581428571428575 +gabriella allen 22.03142857142857 +gabriella allen 26.87375 +gabriella brown 29.675714285714285 +gabriella brown 32.44 +gabriella carson 28.75 +gabriella davidson 27.531250000000004 +gabriella ellison 19.82 +gabriella ellison 27.353333333333335 +gabriella falkner 17.535 +gabriella falkner 19.487500000000004 +gabriella falkner 30.964999999999996 +gabriella garcia 20.544 +gabriella hernandez 20.818333333333335 +gabriella hernandez 28.094285714285718 +gabriella ichabod 10.58 +gabriella ichabod 18.64272727272727 +gabriella ichabod 20.504444444444445 +gabriella ichabod 23.185 +gabriella ichabod 23.35375 +gabriella king 16.18 +gabriella king 27.887500000000003 +gabriella laertes 23.799090909090907 +gabriella miller 15.695714285714283 +gabriella ovid 23.515454545454546 +gabriella ovid 33.33 +gabriella polk 20.38714285714286 +gabriella polk 35.77285714285715 +gabriella steinbeck 16.18 +gabriella steinbeck 32.464999999999996 +gabriella thompson 26.477777777777778 +gabriella thompson 27.29777777777778 +gabriella thompson 30.87666666666667 +gabriella van buren 28.513333333333335 +gabriella van buren 32.41111111111111 +gabriella white 26.765 +gabriella young 24.022499999999997 +gabriella young 29.521666666666665 +gabriella zipper 21.43727272727273 +gabriella zipper 32.106 +holly allen 24.271428571428572 +holly brown 22.959 +holly brown 27.498 +holly falkner 29.66666666666667 +holly hernandez 14.179999999999998 +holly hernandez 22.396666666666665 +holly hernandez 27.434000000000005 +holly hernandez 27.887500000000003 +holly ichabod 27.87375 +holly ichabod 32.525 +holly ichabod 34.042857142857144 +holly johnson 20.808 +holly johnson 25.024285714285718 +holly johnson 30.487142857142857 +holly king 23.185 +holly king 29.008888888888894 +holly laertes 19.41 +holly miller 29.89333333333333 +holly nixon 23.162857142857142 +holly nixon 28.876250000000002 +holly polk 22.7825 +holly polk 26.297499999999996 +holly robinson 24.160000000000004 +holly thompson 19.565555555555555 +holly thompson 27.048999999999996 +holly thompson 29.70555555555556 +holly underhill 17.876250000000002 +holly underhill 27.820000000000004 +holly underhill 30.613999999999997 +holly underhill 30.708 +holly van buren 20.113333333333333 +holly white 23.185 +holly white 29.64272727272727 +holly xylophone 26.400909090909092 +holly young 27.807777777777776 +holly young 31.63 +holly zipper 27.401999999999997 +holly zipper 28.384285714285713 +irene allen 35.345 +irene brown 22.527500000000003 +irene brown 28.384285714285713 +irene brown 32.81875 +irene carson 27.10666666666667 +irene ellison 16.720000000000002 +irene ellison 28.592857142857145 +irene falkner 19.41 +irene falkner 30.564999999999998 +irene garcia 16.9925 +irene garcia 24.03222222222222 +irene garcia 26.297499999999996 +irene ichabod 27.540000000000003 +irene ichabod 29.34875 +irene johnson 25.418181818181814 +irene laertes 22.124285714285712 +irene laertes 22.200000000000003 +irene laertes 24.446666666666665 +irene miller 30.166666666666668 +irene nixon 18.922222222222224 +irene nixon 25.2025 +irene nixon 33.382 +irene ovid 28.256249999999998 +irene ovid 31.63 +irene ovid 32.088750000000005 +irene polk 24.63 +irene polk 25.296363636363637 +irene polk 27.137142857142855 +irene polk 30.65222222222222 +irene polk 33.760000000000005 +irene quirinius 27.044999999999998 +irene quirinius 33.07833333333334 +irene quirinius 41.865 +irene robinson 32.18875 +irene steinbeck 16.463 +irene thompson 25.281666666666666 +irene underhill 24.4025 +irene underhill 28.531 +irene van buren 25.532222222222224 +irene van buren 32.50142857142857 +irene xylophone 26.288181818181815 +jessica brown 28.968181818181822 +jessica carson 19.41 +jessica carson 24.854285714285716 +jessica carson 25.406363636363633 +jessica davidson 22.19625 +jessica davidson 23.888 +jessica davidson 26.297499999999996 +jessica davidson 26.825454545454537 +jessica ellison 22.07777777777778 +jessica ellison 33.33 +jessica falkner 22.637272727272727 +jessica garcia 14.749999999999996 +jessica garcia 29.675714285714285 +jessica ichabod 31.831249999999997 +jessica johnson 21.546666666666667 +jessica johnson 29.986363636363638 +jessica miller 28.735000000000003 +jessica nixon 19.13111111111111 +jessica nixon 26.244999999999997 +jessica ovid 25.274 +jessica ovid 33.181666666666665 +jessica polk 26.79222222222222 +jessica quirinius 20.38714285714286 +jessica quirinius 25.776000000000003 +jessica quirinius 28.26 +jessica quirinius 29.605000000000008 +jessica robinson 24.5625 +jessica thompson 25.736 +jessica thompson 30.87363636363636 +jessica underhill 16.400000000000002 +jessica underhill 25.529090909090915 +jessica underhill 31.63 +jessica van buren 24.446666666666665 +jessica white 20.812 +jessica white 23.26 +jessica white 27.807777777777776 +jessica white 29.031000000000006 +jessica white 30.654545454545453 +jessica xylophone 15.296666666666667 +jessica young 26.718333333333334 +jessica young 27.853749999999998 +jessica zipper 20.3575 +jessica zipper 24.446666666666665 +jessica zipper 29.276363636363637 +katie allen 29.029090909090915 +katie brown 24.156666666666666 +katie davidson 15.383749999999997 +katie ellison 20.978333333333335 +katie ellison 26.96 +katie falkner 24.5625 +katie garcia 27.807777777777776 +katie garcia 28.287142857142857 +katie hernandez 23.667272727272724 +katie ichabod 14.222 +katie ichabod 20.553749999999997 +katie ichabod 31.831249999999997 +katie king 20.05444444444445 +katie king 20.242222222222225 +katie king 23.342857142857145 +katie miller 26.21857142857143 +katie miller 27.675000000000004 +katie nixon 14.476999999999999 +katie ovid 28.37875 +katie polk 20.99 +katie polk 25.090000000000003 +katie robinson 30.65222222222222 +katie van buren 27.133636363636366 +katie van buren 29.675714285714285 +katie white 22.555714285714288 +katie white 24.463749999999997 +katie xylophone 25.74142857142857 +katie young 23.011250000000004 +katie young 26.650000000000002 +katie young 29.301428571428573 +katie zipper 28.26 +katie zipper 29.675714285714285 +luke allen 16.9925 +luke allen 20.595000000000002 +luke allen 27.54181818181818 +luke allen 27.887500000000003 +luke allen 33.07833333333334 +luke brown 29.34875 +luke davidson 27.3575 +luke davidson 31.473333333333333 +luke ellison 14.024444444444443 +luke ellison 22.555714285714288 +luke ellison 28.592857142857145 +luke falkner 21.855 +luke falkner 27.044999999999998 +luke garcia 27.887500000000003 +luke garcia 31.237000000000002 +luke ichabod 28.198571428571427 +luke ichabod 34.345000000000006 +luke johnson 21.239999999999995 +luke johnson 25.462727272727275 +luke johnson 30.188888888888894 +luke laertes 18.344 +luke laertes 20.817 +luke laertes 27.401999999999997 +luke laertes 30.72285714285714 +luke laertes 41.865 +luke miller 22.539000000000005 +luke ovid 16.615454545454543 +luke ovid 26.06625 +luke polk 28.163000000000004 +luke polk 28.840909090909097 +luke quirinius 27.077142857142857 +luke robinson 24.816363636363644 +luke robinson 27.110909090909093 +luke thompson 28.44454545454545 +luke underhill 22.175 +luke underhill 25.518888888888892 +luke underhill 27.34125 +luke van buren 16.54 +luke white 20.544 +luke xylophone 24.5625 +luke zipper 24.764285714285712 +mike allen 23.860000000000003 +mike brown 31.016363636363643 +mike carson 26.066363636363636 +mike carson 28.947142857142858 +mike carson 33.382 +mike davidson 20.544 +mike davidson 21.239999999999995 +mike ellison 18.922 +mike ellison 21.4175 +mike ellison 25.45272727272727 +mike ellison 27.26 +mike ellison 28.39888888888889 +mike falkner 29.397777777777776 +mike garcia 20.544 +mike garcia 24.582 +mike garcia 34.84571428571429 +mike hernandez 10.4925 +mike hernandez 17.7 +mike ichabod 26.772727272727273 +mike king 17.889999999999997 +mike king 19.294999999999998 +mike king 20.004285714285714 +mike king 23.197999999999997 +mike king 23.285 +mike king 27.401999999999997 +mike miller 31.587272727272726 +mike nixon 17.775555555555556 +mike nixon 27.044999999999998 +mike polk 22.175 +mike polk 23.751428571428573 +mike polk 28.095 +mike quirinius 19.13111111111111 +mike steinbeck 14.222 +mike steinbeck 18.100909090909088 +mike steinbeck 18.344 +mike steinbeck 33.760000000000005 +mike van buren 23.42 +mike van buren 25.828333333333333 +mike white 19.13111111111111 +mike white 25.755714285714284 +mike white 29.031000000000006 +mike white 30.516999999999996 +mike young 26.765 +mike young 27.766 +mike young 28.409090909090914 +mike zipper 17.306 +mike zipper 33.23 +mike zipper 41.865 +nick allen 19.331666666666667 +nick allen 32.106 +nick brown 27.578181818181818 +nick davidson 29.100000000000005 +nick ellison 24.764285714285712 +nick ellison 29.521666666666665 +nick falkner 22.555714285714288 +nick falkner 23.15888888888888 +nick garcia 21.546666666666667 +nick garcia 26.25090909090909 +nick garcia 30.166666666666668 +nick ichabod 21.855 +nick ichabod 23.479000000000003 +nick ichabod 29.100000000000005 +nick johnson 25.274 +nick johnson 29.994 +nick laertes 25.820909090909094 +nick miller 19.87888888888889 +nick nixon 17.082 +nick ovid 33.597777777777786 +nick polk 25.736 +nick quirinius 20.707500000000003 +nick quirinius 28.094285714285718 +nick robinson 22.396666666666665 +nick robinson 25.298749999999995 +nick steinbeck 21.192857142857143 +nick thompson 30.72285714285714 +nick underhill 29.345 +nick van buren 25.152727272727272 +nick xylophone 26.948333333333334 +nick young 23.751428571428573 +nick young 24.810000000000002 +nick zipper 24.854285714285716 +nick zipper 27.353333333333335 +oscar allen 18.815 +oscar brown 26.948333333333334 +oscar carson 24.764285714285712 +oscar carson 27.766 +oscar carson 28.094285714285718 +oscar carson 28.31555555555556 +oscar carson 35.22818181818182 +oscar davidson 17.535 +oscar ellison 22.121428571428574 +oscar ellison 28.735000000000003 +oscar falkner 19.294999999999998 +oscar garcia 20.62636363636364 +oscar hernandez 22.539000000000005 +oscar hernandez 23.31888888888889 +oscar ichabod 20.818333333333335 +oscar ichabod 21.546666666666667 +oscar ichabod 26.914545454545454 +oscar ichabod 28.811111111111106 +oscar johnson 22.381818181818183 +oscar johnson 24.266363636363643 +oscar king 15.296666666666667 +oscar king 25.580000000000002 +oscar king 28.37875 +oscar laertes 21.51818181818182 +oscar laertes 23.285 +oscar laertes 24.4025 +oscar laertes 25.345454545454547 +oscar nixon 18.88111111111111 +oscar ovid 24.854285714285716 +oscar ovid 25.274 +oscar ovid 33.29636363636364 +oscar polk 19.331666666666667 +oscar polk 29.34875 +oscar quirinius 22.928 +oscar quirinius 25.66727272727273 +oscar quirinius 25.970909090909092 +oscar quirinius 29.66666666666667 +oscar robinson 20.90666666666667 +oscar robinson 21.855 +oscar robinson 23.42 +oscar robinson 32.90500000000001 +oscar steinbeck 32.02818181818182 +oscar thompson 20.817 +oscar thompson 21.477000000000004 +oscar thompson 21.843636363636367 +oscar thompson 23.559000000000005 +oscar underhill 22.555714285714288 +oscar van buren 27.210000000000008 +oscar van buren 28.592857142857145 +oscar van buren 31.375454545454545 +oscar white 20.818333333333335 +oscar white 21.748 +oscar white 24.582 +oscar white 28.287142857142857 +oscar xylophone 25.845 +oscar xylophone 28.735000000000003 +oscar xylophone 30.72285714285714 +oscar zipper 24.511111111111113 +oscar zipper 25.067777777777778 +oscar zipper 26.21857142857143 +priscilla brown 14.222 +priscilla brown 27.044999999999998 +priscilla brown 30.19909090909091 +priscilla carson 18.07 +priscilla carson 20.70875 +priscilla carson 26.297499999999996 +priscilla ichabod 29.451111111111118 +priscilla ichabod 29.878888888888884 +priscilla johnson 16.9925 +priscilla johnson 22.050000000000004 +priscilla johnson 24.093000000000004 +priscilla johnson 29.200000000000003 +priscilla johnson 29.246 +priscilla king 15.536666666666669 +priscilla nixon 18.9 +priscilla nixon 30.516999999999996 +priscilla ovid 16.005000000000003 +priscilla ovid 29.88111111111111 +priscilla polk 28.018888888888892 +priscilla quirinius 23.064999999999998 +priscilla thompson 27.077142857142857 +priscilla underhill 28.160999999999994 +priscilla underhill 28.56571428571429 +priscilla van buren 20.7 +priscilla van buren 21.830000000000002 +priscilla van buren 24.665 +priscilla white 29.23375 +priscilla xylophone 19.331666666666667 +priscilla xylophone 23.185 +priscilla xylophone 27.34125 +priscilla young 26.32777777777778 +priscilla young 30.613999999999997 +priscilla zipper 13.498 +priscilla zipper 31.972727272727266 +quinn allen 28.786666666666672 +quinn allen 29.471818181818175 +quinn brown 26.314285714285713 +quinn brown 27.38 +quinn brown 28.39888888888889 +quinn davidson 20.808 +quinn davidson 22.71285714285714 +quinn davidson 23.306250000000002 +quinn davidson 27.34125 +quinn ellison 25.002857142857142 +quinn ellison 33.760000000000005 +quinn garcia 20.544 +quinn garcia 27.055999999999994 +quinn garcia 29.183333333333334 +quinn garcia 31.831249999999997 +quinn ichabod 22.101818181818185 +quinn king 17.535 +quinn king 18.035714285714285 +quinn laertes 24.511111111111113 +quinn laertes 28.876250000000002 +quinn laertes 29.202857142857145 +quinn nixon 21.75111111111111 +quinn ovid 29.64125 +quinn quirinius 18.922 +quinn robinson 23.985 +quinn steinbeck 27.077142857142857 +quinn steinbeck 28.160999999999994 +quinn thompson 21.99888888888889 +quinn thompson 34.46857142857143 +quinn underhill 20.113333333333333 +quinn underhill 23.107272727272726 +quinn underhill 26.224285714285717 +quinn van buren 23.612222222222222 +quinn young 24.5625 +quinn zipper 17.889999999999997 +quinn zipper 20.163333333333338 +rachel allen 28.446666666666665 +rachel allen 41.865 +rachel brown 20.92875 +rachel brown 26.21857142857143 +rachel brown 27.905 +rachel brown 30.166666666666668 +rachel brown 35.345 +rachel carson 28.735000000000003 +rachel carson 31.715999999999998 +rachel davidson 29.100000000000005 +rachel ellison 27.055454545454552 +rachel falkner 14.812499999999998 +rachel falkner 28.876250000000002 +rachel falkner 29.308888888888887 +rachel falkner 31.831249999999997 +rachel johnson 31.63 +rachel king 24.511111111111113 +rachel king 30.873749999999998 +rachel laertes 17.306 +rachel laertes 26.765 +rachel ovid 24.042727272727276 +rachel ovid 28.01181818181818 +rachel polk 21.12375 +rachel quirinius 29.831249999999997 +rachel robinson 10.4925 +rachel robinson 22.264444444444447 +rachel robinson 33.43125 +rachel thompson 16.720000000000002 +rachel thompson 26.905714285714286 +rachel thompson 28.876250000000002 +rachel underhill 21.75111111111111 +rachel white 22.175 +rachel white 31.580000000000002 +rachel young 28.150000000000002 +rachel zipper 22.187142857142856 +rachel zipper 33.760000000000005 +sarah carson 21.86818181818182 +sarah carson 22.175 +sarah carson 33.43125 +sarah ellison 17.535 +sarah falkner 29.34875 +sarah falkner 29.64125 +sarah garcia 10.4925 +sarah garcia 20.812 +sarah garcia 28.2175 +sarah ichabod 26.948333333333334 +sarah ichabod 33.62375 +sarah johnson 21.546666666666667 +sarah johnson 24.978000000000005 +sarah johnson 29.608000000000004 +sarah johnson 33.760000000000005 +sarah king 19.41 +sarah king 27.055999999999994 +sarah miller 24.815454545454543 +sarah ovid 28.31625 +sarah robinson 13.498 +sarah robinson 28.256249999999998 +sarah steinbeck 23.26 +sarah white 21.75111111111111 +sarah white 31.63 +sarah xylophone 21.964545454545455 +sarah young 29.335555555555555 +sarah zipper 29.521666666666665 +tom brown 22.873333333333335 +tom brown 30.415555555555557 +tom carson 22.4025 +tom carson 28.39888888888889 +tom carson 29.64125 +tom davidson 30.61142857142857 +tom ellison 23.568 +tom ellison 27.884999999999998 +tom ellison 32.02625 +tom falkner 13.72 +tom falkner 19.849999999999998 +tom hernandez 16.720000000000002 +tom hernandez 29.974285714285713 +tom ichabod 20.113333333333333 +tom johnson 27.077142857142857 +tom johnson 32.90500000000001 +tom king 21.855 +tom laertes 17.981666666666666 +tom laertes 21.80857142857143 +tom miller 18.922 +tom miller 21.239999999999995 +tom miller 22.396666666666665 +tom nixon 27.005000000000003 +tom ovid 34.84571428571429 +tom polk 29.521666666666665 +tom polk 29.805 +tom quirinius 24.764285714285712 +tom quirinius 36.46857142857143 +tom robinson 16.18 +tom robinson 18.07 +tom robinson 27.34125 +tom robinson 34.958571428571425 +tom steinbeck 30.613999999999997 +tom van buren 22.6475 +tom van buren 23.13555555555556 +tom van buren 24.4025 +tom white 27.715714285714284 +tom young 19.41 +tom young 24.63 +tom zipper 22.902 +ulysses brown 16.223333333333333 +ulysses carson 16.3475 +ulysses carson 22.057500000000005 +ulysses carson 28.256249999999998 +ulysses carson 28.27285714285714 +ulysses davidson 24.701249999999998 +ulysses ellison 29.52333333333333 +ulysses garcia 33.382 +ulysses hernandez 18.421818181818185 +ulysses hernandez 20.443749999999998 +ulysses hernandez 22.365 +ulysses ichabod 24.63 +ulysses ichabod 33.24333333333333 +ulysses johnson 33.43125 +ulysses king 27.083333333333332 +ulysses laertes 26.915000000000003 +ulysses laertes 27.305 +ulysses laertes 28.501111111111115 +ulysses miller 18.22 +ulysses miller 26.21857142857143 +ulysses nixon 30.65222222222222 +ulysses ovid 21.366666666666667 +ulysses polk 22.555714285714288 +ulysses polk 22.66625 +ulysses polk 25.11777777777778 +ulysses polk 27.141666666666666 +ulysses quirinius 33.07833333333334 +ulysses robinson 21.12375 +ulysses steinbeck 23.751428571428573 +ulysses steinbeck 25.931428571428572 +ulysses thompson 22.264444444444447 +ulysses underhill 20.812 +ulysses underhill 23.751428571428573 +ulysses underhill 25.071666666666662 +ulysses underhill 25.828333333333333 +ulysses underhill 25.865 +ulysses underhill 28.722499999999997 +ulysses underhill 35.268888888888895 +ulysses van buren 22.134999999999998 +ulysses white 15.296666666666667 +ulysses white 32.093333333333334 +ulysses xylophone 20.38714285714286 +ulysses xylophone 25.274 +ulysses xylophone 29.64125 +ulysses young 22.213333333333335 +ulysses young 22.90285714285714 +ulysses young 32.93125 +victor allen 24.82875 +victor allen 27.51 +victor brown 21.621250000000003 +victor brown 23.73 +victor brown 26.21857142857143 +victor brown 27.548571428571428 +victor davidson 22.391666666666666 +victor davidson 33.16428571428572 +victor davidson 35.197500000000005 +victor ellison 11.100000000000001 +victor ellison 30.96857142857143 +victor hernandez 10.4925 +victor hernandez 18.922 +victor hernandez 24.301250000000003 +victor hernandez 26.69857142857143 +victor hernandez 35.358333333333334 +victor johnson 16.580000000000002 +victor johnson 27.516666666666666 +victor johnson 32.106 +victor king 19.962857142857143 +victor king 33.01857142857143 +victor laertes 21.78142857142857 +victor laertes 33.10999999999999 +victor miller 21.93285714285714 +victor nixon 20.419999999999998 +victor nixon 33.69 +victor ovid 28.75857142857143 +victor polk 18.43111111111111 +victor quirinius 17.84777777777778 +victor quirinius 27.53 +victor robinson 19.37 +victor robinson 20.38714285714286 +victor steinbeck 20.818333333333335 +victor steinbeck 25.16 +victor steinbeck 30.503749999999997 +victor thompson 23.987142857142857 +victor van buren 27.009999999999998 +victor van buren 33.43125 +victor white 24.322857142857142 +victor white 28.287142857142857 +victor xylophone 11.807142857142859 +victor xylophone 13.988571428571428 +victor xylophone 16.720000000000002 +victor xylophone 19.686666666666667 +victor xylophone 37.20428571428572 +victor young 22.264444444444447 +victor zipper 24.854285714285716 +wendy allen 28.24142857142857 +wendy allen 29.675714285714285 +wendy allen 34.275 +wendy brown 22.482857142857142 +wendy brown 27.79714285714286 +wendy ellison 16.80888888888889 +wendy ellison 18.135 +wendy falkner 22.628888888888884 +wendy falkner 23.325000000000003 +wendy falkner 24.0375 +wendy garcia 19.307142857142853 +wendy garcia 21.761428571428574 +wendy garcia 24.63 +wendy garcia 24.854285714285716 +wendy hernandez 16.60875 +wendy ichabod 28.26 +wendy king 22.5 +wendy king 24.793333333333333 +wendy king 28.252857142857145 +wendy laertes 25.881428571428568 +wendy laertes 30.338333333333328 +wendy laertes 30.52857142857143 +wendy miller 15.478333333333332 +wendy miller 25.34333333333333 +wendy nixon 19.54714285714286 +wendy nixon 27.003333333333334 +wendy ovid 14.283750000000001 +wendy ovid 30.878333333333334 +wendy polk 21.69375 +wendy polk 24.63 +wendy quirinius 28.731428571428573 +wendy quirinius 29.74333333333333 +wendy robinson 16.720000000000002 +wendy robinson 23.834285714285716 +wendy robinson 29.911666666666672 +wendy steinbeck 29.272857142857145 +wendy thompson 18.17875 +wendy thompson 22.544285714285714 +wendy underhill 21.69625 +wendy underhill 27.077142857142857 +wendy underhill 30.03333333333333 +wendy van buren 28.624285714285715 +wendy van buren 29.28333333333333 +wendy white 24.4025 +wendy xylophone 16.84 +wendy xylophone 23.426666666666666 +wendy young 20.80125 +wendy young 32.693333333333335 +xavier allen 16.535 +xavier allen 17.398333333333337 +xavier allen 35.708333333333336 +xavier brown 20.787142857142857 +xavier brown 24.764285714285712 +xavier brown 31.784999999999997 +xavier carson 20.818333333333335 +xavier carson 32.106 +xavier davidson 16.862857142857145 +xavier davidson 20.53625 +xavier davidson 27.353333333333335 +xavier ellison 17.991666666666667 +xavier ellison 23.976666666666663 +xavier garcia 35.84428571428572 +xavier hernandez 22.654285714285713 +xavier hernandez 26.948333333333334 +xavier hernandez 28.075 +xavier ichabod 20.344285714285714 +xavier ichabod 20.818333333333335 +xavier johnson 15.754285714285714 +xavier johnson 19.490000000000002 +xavier king 29.246666666666666 +xavier king 29.521666666666665 +xavier laertes 19.294999999999998 +xavier ovid 28.51 +xavier polk 12.728333333333333 +xavier polk 19.37 +xavier polk 22.548333333333332 +xavier polk 28.465714285714284 +xavier quirinius 9.991428571428571 +xavier quirinius 24.156666666666666 +xavier quirinius 25.69666666666667 +xavier quirinius 25.828333333333333 +xavier thompson 23.961428571428574 +xavier underhill 21.830000000000002 +xavier white 19.331666666666667 +xavier white 35.345 +xavier xylophone 21.187142857142856 +xavier zipper 17.488333333333333 +yuri allen 15.705714285714285 +yuri allen 20.808 +yuri brown 19.53 +yuri brown 22.457142857142856 +yuri carson 25.699999999999996 +yuri carson 27.216666666666665 +yuri ellison 15.034999999999998 +yuri ellison 28.463333333333335 +yuri falkner 17.81833333333333 +yuri falkner 19.294999999999998 +yuri garcia 28.287142857142857 +yuri hernandez 32.395 +yuri johnson 25.828333333333333 +yuri johnson 27.301666666666666 +yuri johnson 29.578333333333337 +yuri king 19.921666666666663 +yuri laertes 16.18 +yuri laertes 30.519999999999996 +yuri nixon 16.383333333333333 +yuri nixon 25.828333333333333 +yuri polk 16.18 +yuri polk 20.503333333333334 +yuri polk 30.16333333333333 +yuri quirinius 20.311666666666667 +yuri quirinius 23.185 +yuri quirinius 24.828333333333333 +yuri steinbeck 19.331666666666667 +yuri steinbeck 28.50666666666667 +yuri thompson 35.27 +yuri underhill 23.042857142857144 +yuri underhill 28.786666666666665 +yuri white 30.72285714285714 +yuri xylophone 24.173333333333332 +zach allen 8.983333333333333 +zach brown 18.922 +zach brown 23.036666666666665 +zach brown 29.72666666666667 +zach brown 31.58285714285714 +zach brown 33.07833333333334 +zach carson 27.110000000000003 +zach ellison 18.168333333333333 +zach falkner 16.18 +zach falkner 30.83285714285714 +zach garcia 16.586666666666666 +zach garcia 22.53333333333333 +zach garcia 28.13166666666667 +zach garcia 34.84571428571429 +zach ichabod 17.535 +zach ichabod 30.72285714285714 +zach king 19.878333333333334 +zach king 25.643333333333334 +zach king 28.646666666666665 +zach miller 23.285 +zach miller 23.366666666666664 +zach miller 30.46833333333333 +zach ovid 23.94666666666667 +zach ovid 28.75166666666667 +zach ovid 28.763333333333335 +zach ovid 34.84571428571429 +zach quirinius 20.755 +zach robinson 21.546666666666667 +zach steinbeck 27.243333333333336 +zach steinbeck 30.073333333333334 +zach thompson 14.222 +zach thompson 24.755 +zach underhill 31.885 +zach white 20.208333333333332 +zach xylophone 10.485 +zach xylophone 20.113333333333333 +zach young 20.176666666666666 +zach zipper 21.709999999999997 +zach zipper 22.264999999999997 +zach zipper 34.01166666666667 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-5-ee44c5cdc80e1c832b702f9fb76d8145 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-5-ee44c5cdc80e1c832b702f9fb76d8145 new file mode 100644 index 0000000000000..a9ae190825a01 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-5-ee44c5cdc80e1c832b702f9fb76d8145 @@ -0,0 +1,1049 @@ + 65560 + 65718 + 65740 +alice allen 65662 +alice allen 65720 +alice allen 65758 +alice brown 65696 +alice carson 65559 +alice davidson 65547 +alice falkner 65669 +alice garcia 65613 +alice hernandez 65737 +alice hernandez 65784 +alice johnson 65739 +alice king 65660 +alice king 65738 +alice king 131281 +alice laertes 65669 +alice laertes 65671 +alice miller 65590 +alice nixon 65586 +alice nixon 65595 +alice nixon 65604 +alice ovid 65737 +alice polk 65548 +alice quirinius 65636 +alice quirinius 65728 +alice robinson 65606 +alice robinson 65789 +alice steinbeck 65578 +alice steinbeck 65673 +alice steinbeck 65786 +alice underhill 65750 +alice van buren 65562 +alice xylophone 65585 +alice xylophone 65599 +alice xylophone 131247 +alice zipper 65553 +alice zipper 65662 +alice zipper 65766 +bob brown 65584 +bob brown 65777 +bob brown 65783 +bob carson 65713 +bob davidson 65664 +bob davidson 65693 +bob davidson 65768 +bob ellison 65591 +bob ellison 65624 +bob ellison 65721 +bob ellison 65760 +bob falkner 65789 +bob garcia 65585 +bob garcia 65598 +bob garcia 65673 +bob garcia 65754 +bob garcia 65782 +bob hernandez 131340 +bob ichabod 65549 +bob king 65715 +bob king 65757 +bob king 65783 +bob laertes 65602 +bob laertes 131447 +bob miller 65608 +bob ovid 65564 +bob ovid 65686 +bob ovid 196959 +bob ovid 196973 +bob polk 65594 +bob quirinius 65700 +bob steinbeck 65637 +bob van buren 65778 +bob white 65543 +bob white 65605 +bob xylophone 65574 +bob xylophone 65666 +bob young 65556 +bob zipper 65559 +bob zipper 65633 +bob zipper 65739 +calvin allen 65669 +calvin brown 65537 +calvin brown 131272 +calvin brown 197027 +calvin carson 65637 +calvin davidson 65541 +calvin davidson 65564 +calvin ellison 65667 +calvin falkner 65573 +calvin falkner 65596 +calvin falkner 65778 +calvin falkner 131397 +calvin falkner 131411 +calvin falkner 131433 +calvin garcia 131212 +calvin hernandez 131251 +calvin johnson 65731 +calvin laertes 65570 +calvin laertes 65684 +calvin nixon 65654 +calvin nixon 131386 +calvin nixon 131503 +calvin ovid 65554 +calvin ovid 65643 +calvin ovid 65715 +calvin ovid 196944 +calvin polk 65731 +calvin quirinius 65741 +calvin quirinius 65769 +calvin robinson 131320 +calvin steinbeck 131271 +calvin steinbeck 131326 +calvin steinbeck 131415 +calvin thompson 65560 +calvin thompson 131244 +calvin underhill 196944 +calvin van buren 65771 +calvin van buren 131138 +calvin white 65553 +calvin white 65561 +calvin xylophone 65575 +calvin xylophone 65596 +calvin xylophone 262686 +calvin young 65746 +calvin young 131168 +calvin zipper 65669 +calvin zipper 131476 +david allen 65588 +david allen 131222 +david brown 65637 +david brown 131303 +david davidson 65756 +david davidson 65778 +david davidson 65779 +david davidson 131342 +david ellison 65724 +david ellison 65724 +david ellison 131224 +david hernandez 197083 +david ichabod 131454 +david ichabod 197085 +david laertes 65762 +david nixon 65536 +david ovid 65623 +david ovid 196766 +david quirinius 65759 +david quirinius 65779 +david quirinius 131303 +david robinson 65762 +david robinson 65775 +david thompson 65550 +david underhill 65662 +david underhill 65751 +david underhill 131198 +david van buren 65634 +david van buren 262584 +david white 65678 +david xylophone 65537 +david xylophone 131426 +david xylophone 131447 +david young 65551 +david young 131255 +ethan allen 131460 +ethan brown 65539 +ethan brown 65617 +ethan brown 65685 +ethan brown 65685 +ethan brown 65722 +ethan brown 131483 +ethan carson 197189 +ethan ellison 65714 +ethan ellison 131302 +ethan falkner 131222 +ethan falkner 131333 +ethan garcia 131507 +ethan hernandez 65618 +ethan johnson 65536 +ethan king 131280 +ethan laertes 65562 +ethan laertes 65597 +ethan laertes 65628 +ethan laertes 65680 +ethan laertes 65760 +ethan laertes 131304 +ethan laertes 328329 +ethan miller 328296 +ethan nixon 65766 +ethan ovid 65697 +ethan polk 65589 +ethan polk 65615 +ethan polk 131206 +ethan polk 197082 +ethan quirinius 65591 +ethan quirinius 196912 +ethan quirinius 196957 +ethan robinson 65547 +ethan robinson 65659 +ethan underhill 65570 +ethan van buren 131252 +ethan white 65677 +ethan white 197039 +ethan xylophone 65595 +ethan zipper 65593 +ethan zipper 131365 +fred davidson 65595 +fred davidson 65721 +fred davidson 131221 +fred ellison 65548 +fred ellison 65691 +fred ellison 65771 +fred falkner 65637 +fred falkner 131474 +fred falkner 196920 +fred hernandez 131226 +fred ichabod 131109 +fred ichabod 131520 +fred johnson 131332 +fred king 65694 +fred king 197016 +fred laertes 131354 +fred miller 65536 +fred nixon 65560 +fred nixon 65612 +fred nixon 65705 +fred nixon 196929 +fred polk 65656 +fred polk 131231 +fred polk 262645 +fred polk 262733 +fred quirinius 131486 +fred quirinius 196950 +fred robinson 65623 +fred steinbeck 65544 +fred steinbeck 65755 +fred steinbeck 131253 +fred underhill 131188 +fred van buren 65561 +fred van buren 65745 +fred van buren 131380 +fred van buren 328270 +fred white 131136 +fred young 65594 +fred young 131551 +fred zipper 196885 +gabriella allen 65677 +gabriella allen 131283 +gabriella brown 65753 +gabriella brown 197180 +gabriella carson 65586 +gabriella davidson 65565 +gabriella ellison 65706 +gabriella ellison 131505 +gabriella falkner 65767 +gabriella falkner 131183 +gabriella falkner 131397 +gabriella garcia 131127 +gabriella hernandez 131304 +gabriella hernandez 131304 +gabriella ichabod 65559 +gabriella ichabod 65712 +gabriella ichabod 131297 +gabriella ichabod 131311 +gabriella ichabod 131460 +gabriella king 65657 +gabriella king 197031 +gabriella laertes 131543 +gabriella miller 131300 +gabriella ovid 65556 +gabriella ovid 131260 +gabriella polk 65790 +gabriella polk 131425 +gabriella steinbeck 65582 +gabriella steinbeck 131248 +gabriella thompson 131528 +gabriella thompson 197181 +gabriella thompson 262632 +gabriella van buren 65644 +gabriella van buren 131238 +gabriella white 65638 +gabriella young 65699 +gabriella young 65774 +gabriella zipper 65754 +gabriella zipper 196762 +holly allen 65596 +holly brown 131315 +holly brown 131368 +holly falkner 65720 +holly hernandez 65602 +holly hernandez 65686 +holly hernandez 131387 +holly hernandez 131554 +holly ichabod 65752 +holly ichabod 131308 +holly ichabod 131473 +holly johnson 65755 +holly johnson 131240 +holly johnson 131277 +holly king 131286 +holly king 131303 +holly laertes 196950 +holly miller 131381 +holly nixon 196941 +holly nixon 328184 +holly polk 197132 +holly polk 262782 +holly robinson 131241 +holly thompson 65578 +holly thompson 65713 +holly thompson 197092 +holly underhill 65654 +holly underhill 131323 +holly underhill 131385 +holly underhill 131504 +holly van buren 131449 +holly white 131092 +holly white 262734 +holly xylophone 196792 +holly young 65765 +holly young 131229 +holly zipper 131151 +holly zipper 131545 +irene allen 131109 +irene brown 65765 +irene brown 131368 +irene brown 393929 +irene carson 262770 +irene ellison 196956 +irene ellison 196982 +irene falkner 131287 +irene falkner 197046 +irene garcia 65660 +irene garcia 131286 +irene garcia 131375 +irene ichabod 65645 +irene ichabod 131442 +irene johnson 131179 +irene laertes 131324 +irene laertes 131381 +irene laertes 131407 +irene miller 262822 +irene nixon 197105 +irene nixon 262409 +irene nixon 262565 +irene ovid 65734 +irene ovid 196935 +irene ovid 262836 +irene polk 65551 +irene polk 131189 +irene polk 131189 +irene polk 196943 +irene polk 328365 +irene quirinius 131369 +irene quirinius 196998 +irene quirinius 262855 +irene robinson 131259 +irene steinbeck 65683 +irene thompson 262719 +irene underhill 131291 +irene underhill 131386 +irene van buren 131216 +irene van buren 262539 +irene xylophone 131348 +jessica brown 393772 +jessica carson 65747 +jessica carson 131207 +jessica carson 131232 +jessica davidson 65606 +jessica davidson 65675 +jessica davidson 196917 +jessica davidson 197030 +jessica ellison 131108 +jessica ellison 196885 +jessica falkner 131270 +jessica garcia 197059 +jessica garcia 328458 +jessica ichabod 197028 +jessica johnson 131177 +jessica johnson 197085 +jessica miller 197024 +jessica nixon 131549 +jessica nixon 196682 +jessica ovid 65751 +jessica ovid 196890 +jessica polk 459409 +jessica quirinius 131222 +jessica quirinius 131248 +jessica quirinius 131294 +jessica quirinius 393878 +jessica robinson 131174 +jessica thompson 131336 +jessica thompson 196927 +jessica underhill 131218 +jessica underhill 131267 +jessica underhill 197086 +jessica van buren 65615 +jessica white 65544 +jessica white 65594 +jessica white 197012 +jessica white 262435 +jessica white 262571 +jessica xylophone 196866 +jessica young 65711 +jessica young 131183 +jessica zipper 196897 +jessica zipper 262523 +jessica zipper 262695 +katie allen 196740 +katie brown 328113 +katie davidson 131371 +katie ellison 131248 +katie ellison 197182 +katie falkner 131441 +katie garcia 131384 +katie garcia 197051 +katie hernandez 131296 +katie ichabod 131495 +katie ichabod 197131 +katie ichabod 197275 +katie king 131252 +katie king 262588 +katie king 262861 +katie miller 65661 +katie miller 262723 +katie nixon 65669 +katie ovid 65681 +katie polk 65784 +katie polk 197249 +katie robinson 131251 +katie van buren 131237 +katie van buren 197141 +katie white 262510 +katie white 262860 +katie xylophone 197034 +katie young 65644 +katie young 328173 +katie young 393859 +katie zipper 65733 +katie zipper 328287 +luke allen 65776 +luke allen 131268 +luke allen 196819 +luke allen 196855 +luke allen 328011 +luke brown 196967 +luke davidson 65656 +luke davidson 131573 +luke ellison 65582 +luke ellison 131343 +luke ellison 197118 +luke falkner 196797 +luke falkner 196837 +luke garcia 65778 +luke garcia 393974 +luke ichabod 65629 +luke ichabod 262574 +luke johnson 131302 +luke johnson 131312 +luke johnson 131361 +luke laertes 131226 +luke laertes 131504 +luke laertes 197018 +luke laertes 197153 +luke laertes 197177 +luke miller 197052 +luke ovid 65569 +luke ovid 262745 +luke polk 65658 +luke polk 262627 +luke quirinius 131233 +luke robinson 65634 +luke robinson 262569 +luke thompson 196858 +luke underhill 65651 +luke underhill 131240 +luke underhill 328248 +luke van buren 131398 +luke white 65693 +luke xylophone 131312 +luke zipper 131297 +mike allen 196928 +mike brown 197149 +mike carson 65751 +mike carson 131284 +mike carson 393711 +mike davidson 196917 +mike davidson 262912 +mike ellison 65598 +mike ellison 131366 +mike ellison 131412 +mike ellison 131509 +mike ellison 262704 +mike falkner 328183 +mike garcia 131530 +mike garcia 328305 +mike garcia 328461 +mike hernandez 131301 +mike hernandez 328384 +mike ichabod 131157 +mike king 196965 +mike king 197091 +mike king 197121 +mike king 262471 +mike king 262527 +mike king 328279 +mike miller 131317 +mike nixon 131328 +mike nixon 262653 +mike polk 131240 +mike polk 196899 +mike polk 262885 +mike quirinius 525126 +mike steinbeck 65550 +mike steinbeck 131201 +mike steinbeck 131490 +mike steinbeck 262490 +mike van buren 131548 +mike van buren 262547 +mike white 197000 +mike white 197060 +mike white 262425 +mike white 328482 +mike young 196935 +mike young 196976 +mike young 328084 +mike zipper 131147 +mike zipper 197075 +mike zipper 328517 +nick allen 131192 +nick allen 197024 +nick brown 131503 +nick davidson 262686 +nick ellison 197119 +nick ellison 197119 +nick falkner 65583 +nick falkner 328561 +nick garcia 131318 +nick garcia 262755 +nick garcia 328281 +nick ichabod 131430 +nick ichabod 196812 +nick ichabod 328593 +nick johnson 131453 +nick johnson 262597 +nick laertes 196732 +nick miller 131490 +nick nixon 262547 +nick ovid 328266 +nick polk 196852 +nick quirinius 131438 +nick quirinius 328176 +nick robinson 131326 +nick robinson 196980 +nick steinbeck 131250 +nick thompson 65610 +nick underhill 65619 +nick van buren 196795 +nick xylophone 196972 +nick young 394136 +nick young 459634 +nick zipper 262954 +nick zipper 394218 +oscar allen 262674 +oscar brown 196916 +oscar carson 131099 +oscar carson 131330 +oscar carson 196731 +oscar carson 196733 +oscar carson 196918 +oscar davidson 262554 +oscar ellison 65630 +oscar ellison 197116 +oscar falkner 197145 +oscar garcia 328305 +oscar hernandez 197022 +oscar hernandez 328315 +oscar ichabod 131302 +oscar ichabod 131309 +oscar ichabod 196760 +oscar ichabod 196874 +oscar johnson 196942 +oscar johnson 197203 +oscar king 196793 +oscar king 196944 +oscar king 328236 +oscar laertes 131208 +oscar laertes 262522 +oscar laertes 262842 +oscar laertes 328364 +oscar nixon 65596 +oscar ovid 131228 +oscar ovid 262580 +oscar ovid 393817 +oscar polk 131078 +oscar polk 131260 +oscar quirinius 131103 +oscar quirinius 196748 +oscar quirinius 196829 +oscar quirinius 262838 +oscar robinson 196874 +oscar robinson 262803 +oscar robinson 393773 +oscar robinson 394087 +oscar steinbeck 328432 +oscar thompson 196826 +oscar thompson 196992 +oscar thompson 262593 +oscar thompson 459401 +oscar underhill 131301 +oscar van buren 131134 +oscar van buren 328162 +oscar van buren 394034 +oscar white 131457 +oscar white 262345 +oscar white 328538 +oscar white 459337 +oscar xylophone 65773 +oscar xylophone 262708 +oscar xylophone 262906 +oscar zipper 196904 +oscar zipper 262512 +oscar zipper 328262 +priscilla brown 196950 +priscilla brown 328237 +priscilla brown 328624 +priscilla carson 262488 +priscilla carson 262510 +priscilla carson 262703 +priscilla ichabod 131178 +priscilla ichabod 131303 +priscilla johnson 131224 +priscilla johnson 196906 +priscilla johnson 196994 +priscilla johnson 197184 +priscilla johnson 394171 +priscilla king 262692 +priscilla nixon 262691 +priscilla nixon 394188 +priscilla ovid 65541 +priscilla ovid 197067 +priscilla polk 394009 +priscilla quirinius 131306 +priscilla thompson 196875 +priscilla underhill 197084 +priscilla underhill 262701 +priscilla van buren 65685 +priscilla van buren 131368 +priscilla van buren 196814 +priscilla white 196893 +priscilla xylophone 131473 +priscilla xylophone 262597 +priscilla xylophone 262785 +priscilla young 131392 +priscilla young 262788 +priscilla zipper 393888 +priscilla zipper 394031 +quinn allen 197095 +quinn allen 394225 +quinn brown 131470 +quinn brown 131473 +quinn brown 262642 +quinn davidson 197079 +quinn davidson 197112 +quinn davidson 262510 +quinn davidson 459427 +quinn ellison 197268 +quinn ellison 328130 +quinn garcia 65604 +quinn garcia 131321 +quinn garcia 197067 +quinn garcia 328528 +quinn ichabod 65564 +quinn king 65649 +quinn king 196879 +quinn laertes 65542 +quinn laertes 196877 +quinn laertes 262466 +quinn nixon 196837 +quinn ovid 525126 +quinn quirinius 328235 +quinn robinson 131378 +quinn steinbeck 131484 +quinn steinbeck 262528 +quinn thompson 197030 +quinn thompson 262717 +quinn underhill 262791 +quinn underhill 328146 +quinn underhill 393824 +quinn van buren 197234 +quinn young 65647 +quinn zipper 131466 +quinn zipper 262658 +rachel allen 65661 +rachel allen 196935 +rachel brown 131220 +rachel brown 328076 +rachel brown 328320 +rachel brown 393835 +rachel brown 524988 +rachel carson 131259 +rachel carson 459393 +rachel davidson 262632 +rachel ellison 393845 +rachel falkner 196947 +rachel falkner 262474 +rachel falkner 394046 +rachel falkner 525086 +rachel johnson 65658 +rachel king 131354 +rachel king 196907 +rachel laertes 131391 +rachel laertes 197105 +rachel ovid 262664 +rachel ovid 328195 +rachel polk 328389 +rachel quirinius 262779 +rachel robinson 262491 +rachel robinson 262862 +rachel robinson 590712 +rachel thompson 197034 +rachel thompson 328158 +rachel thompson 394094 +rachel underhill 197033 +rachel white 131399 +rachel white 197190 +rachel young 196967 +rachel zipper 328223 +rachel zipper 394149 +sarah carson 131379 +sarah carson 196870 +sarah carson 262491 +sarah ellison 197095 +sarah falkner 131262 +sarah falkner 328251 +sarah garcia 196963 +sarah garcia 197030 +sarah garcia 459657 +sarah ichabod 262504 +sarah ichabod 262766 +sarah johnson 131409 +sarah johnson 262783 +sarah johnson 328591 +sarah johnson 394043 +sarah king 196998 +sarah king 328416 +sarah miller 196893 +sarah ovid 131199 +sarah robinson 262868 +sarah robinson 394066 +sarah steinbeck 262650 +sarah white 197059 +sarah white 262579 +sarah xylophone 131336 +sarah young 394123 +sarah zipper 262818 +tom brown 196848 +tom brown 328268 +tom carson 197328 +tom carson 262517 +tom carson 656251 +tom davidson 262864 +tom ellison 196974 +tom ellison 328416 +tom ellison 393921 +tom falkner 393809 +tom falkner 459407 +tom hernandez 262525 +tom hernandez 328085 +tom ichabod 197048 +tom johnson 328321 +tom johnson 393865 +tom king 196951 +tom laertes 262657 +tom laertes 459805 +tom miller 131278 +tom miller 131459 +tom miller 262633 +tom nixon 262588 +tom ovid 262595 +tom polk 328470 +tom polk 328584 +tom quirinius 262597 +tom quirinius 262681 +tom robinson 196978 +tom robinson 328481 +tom robinson 459857 +tom robinson 525095 +tom steinbeck 262426 +tom van buren 131389 +tom van buren 328095 +tom van buren 328313 +tom white 328128 +tom young 131080 +tom young 393692 +tom zipper 197167 +ulysses brown 196815 +ulysses carson 131277 +ulysses carson 262450 +ulysses carson 262937 +ulysses carson 328311 +ulysses davidson 262750 +ulysses ellison 262445 +ulysses garcia 328445 +ulysses hernandez 131414 +ulysses hernandez 196871 +ulysses hernandez 394370 +ulysses ichabod 393834 +ulysses ichabod 459582 +ulysses johnson 262966 +ulysses king 131363 +ulysses laertes 262739 +ulysses laertes 328412 +ulysses laertes 328462 +ulysses miller 262661 +ulysses miller 328360 +ulysses nixon 394194 +ulysses ovid 328289 +ulysses polk 65563 +ulysses polk 197046 +ulysses polk 328294 +ulysses polk 590698 +ulysses quirinius 525643 +ulysses robinson 394160 +ulysses steinbeck 196783 +ulysses steinbeck 262778 +ulysses thompson 262607 +ulysses underhill 131214 +ulysses underhill 196937 +ulysses underhill 197027 +ulysses underhill 262623 +ulysses underhill 262623 +ulysses underhill 262648 +ulysses underhill 262836 +ulysses van buren 196944 +ulysses white 197033 +ulysses white 393988 +ulysses xylophone 262695 +ulysses xylophone 328151 +ulysses xylophone 328747 +ulysses young 196903 +ulysses young 394037 +ulysses young 459782 +victor allen 197189 +victor allen 262651 +victor brown 262544 +victor brown 262799 +victor brown 327900 +victor brown 591265 +victor davidson 197173 +victor davidson 262486 +victor davidson 328274 +victor ellison 328618 +victor ellison 393962 +victor hernandez 197041 +victor hernandez 197132 +victor hernandez 262771 +victor hernandez 328261 +victor hernandez 459902 +victor johnson 131155 +victor johnson 131169 +victor johnson 394168 +victor king 131486 +victor king 328509 +victor laertes 262573 +victor laertes 328435 +victor miller 196784 +victor nixon 196987 +victor nixon 394249 +victor ovid 196882 +victor polk 262462 +victor quirinius 65620 +victor quirinius 328301 +victor robinson 328334 +victor robinson 394031 +victor steinbeck 65661 +victor steinbeck 262560 +victor steinbeck 262750 +victor thompson 65548 +victor van buren 197173 +victor van buren 328261 +victor white 262588 +victor white 328039 +victor xylophone 131203 +victor xylophone 262596 +victor xylophone 328191 +victor xylophone 393913 +victor xylophone 459542 +victor young 131258 +victor zipper 131349 +wendy allen 131402 +wendy allen 196954 +wendy allen 328359 +wendy brown 328365 +wendy brown 459501 +wendy ellison 262718 +wendy ellison 328191 +wendy falkner 197009 +wendy falkner 262430 +wendy falkner 328177 +wendy garcia 65746 +wendy garcia 393974 +wendy garcia 459883 +wendy garcia 459926 +wendy hernandez 65650 +wendy ichabod 262665 +wendy king 262545 +wendy king 328229 +wendy king 393951 +wendy laertes 262739 +wendy laertes 262794 +wendy laertes 328315 +wendy miller 131377 +wendy miller 328161 +wendy nixon 131258 +wendy nixon 196893 +wendy ovid 196952 +wendy ovid 459594 +wendy polk 328520 +wendy polk 394310 +wendy quirinius 328703 +wendy quirinius 394360 +wendy robinson 131316 +wendy robinson 394030 +wendy robinson 459665 +wendy steinbeck 262645 +wendy thompson 262725 +wendy thompson 393865 +wendy underhill 328445 +wendy underhill 394295 +wendy underhill 460068 +wendy van buren 65699 +wendy van buren 196964 +wendy white 328135 +wendy xylophone 262894 +wendy xylophone 525344 +wendy young 197017 +wendy young 721936 +xavier allen 197025 +xavier allen 525393 +xavier allen 525839 +xavier brown 197058 +xavier brown 262626 +xavier brown 328388 +xavier carson 196990 +xavier carson 328415 +xavier davidson 65644 +xavier davidson 262745 +xavier davidson 393825 +xavier ellison 197095 +xavier ellison 328447 +xavier garcia 262590 +xavier hernandez 196847 +xavier hernandez 197077 +xavier hernandez 393838 +xavier ichabod 262600 +xavier ichabod 328157 +xavier johnson 197084 +xavier johnson 262785 +xavier king 196919 +xavier king 262774 +xavier laertes 262770 +xavier ovid 328414 +xavier polk 196844 +xavier polk 328474 +xavier polk 394013 +xavier polk 590931 +xavier quirinius 65650 +xavier quirinius 131140 +xavier quirinius 328382 +xavier quirinius 459669 +xavier thompson 393799 +xavier underhill 197012 +xavier white 196858 +xavier white 262712 +xavier xylophone 131250 +xavier zipper 394070 +yuri allen 131129 +yuri allen 459977 +yuri brown 262640 +yuri brown 393858 +yuri carson 459799 +yuri carson 591063 +yuri ellison 197085 +yuri ellison 459558 +yuri falkner 196857 +yuri falkner 525350 +yuri garcia 328378 +yuri hernandez 262588 +yuri johnson 393861 +yuri johnson 394444 +yuri johnson 525638 +yuri king 525526 +yuri laertes 131551 +yuri laertes 459611 +yuri nixon 262644 +yuri nixon 393936 +yuri polk 328197 +yuri polk 328404 +yuri polk 328481 +yuri quirinius 131092 +yuri quirinius 196898 +yuri quirinius 525159 +yuri steinbeck 394037 +yuri steinbeck 525180 +yuri thompson 459710 +yuri underhill 328325 +yuri underhill 459781 +yuri white 131252 +yuri xylophone 262809 +zach allen 394026 +zach brown 262789 +zach brown 262789 +zach brown 459521 +zach brown 459846 +zach brown 590938 +zach carson 262320 +zach ellison 262757 +zach falkner 262608 +zach falkner 262608 +zach garcia 262818 +zach garcia 328314 +zach garcia 393686 +zach garcia 394011 +zach ichabod 262518 +zach ichabod 262563 +zach king 196780 +zach king 196905 +zach king 459991 +zach miller 196923 +zach miller 393813 +zach miller 393892 +zach ovid 196876 +zach ovid 262643 +zach ovid 328023 +zach ovid 459615 +zach quirinius 262471 +zach robinson 196967 +zach steinbeck 131394 +zach steinbeck 459294 +zach thompson 131340 +zach thompson 525538 +zach underhill 131304 +zach white 65733 +zach xylophone 262810 +zach xylophone 459455 +zach young 393615 +zach zipper 197130 +zach zipper 262496 +zach zipper 393937 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-6-4d78f7b1d172d20c91f5867bc13a42a0 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-6-4d78f7b1d172d20c91f5867bc13a42a0 new file mode 100644 index 0000000000000..b3f08818f491a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-6-4d78f7b1d172d20c91f5867bc13a42a0 @@ -0,0 +1,1049 @@ +0.08 0.07999999821186066 +0.1 0.10000000149011612 +0.13 0.12999999523162842 +0.15 0.15000000596046448 +0.27 0.27000001072883606 +0.28 0.2800000011920929 +0.43 0.4300000071525574 +0.52 0.5199999809265137 +0.56 0.5600000023841858 +0.6 0.6000000238418579 +0.61 0.6100000143051147 +0.79 0.7900000214576721 +0.84 0.8399999737739563 +0.98 0.9800000190734863 +1.02 1.2899999916553497 +1.08 1.0800000429153442 +1.08 1.0800000429153442 +1.12 1.1200000047683716 +1.21 2.0000000596046448 +1.25 1.25 +1.27 1.2699999809265137 +1.29 1.2899999618530273 +1.31 1.309999942779541 +1.58 1.5800000429153442 +1.87 1.8700000047683716 +1.91 1.909999966621399 +1.92 3.1699999570846558 +2.07 2.069999933242798 +2.18 2.180000066757202 +2.2 2.200000047683716 +2.35 2.3499999046325684 +2.6 2.5999999046325684 +2.79 2.7899999618530273 +2.92 2.9200000762939453 +2.96 2.9600000381469727 +2.96 2.9600000381469727 +2.97 2.9700000286102295 +3.0 3.0 +3.21 3.340000033378601 +3.28 4.399999976158142 +3.33 3.3299999237060547 +3.61 3.609999895095825 +3.62 3.619999885559082 +3.82 3.819999933242798 +3.86 3.859999895095825 +3.96 3.9600000381469727 +3.97 3.9700000286102295 +4.17 7.7799999713897705 +4.32 4.320000171661377 +4.35 4.349999904632568 +4.41 4.409999847412109 +4.46 4.460000038146973 +4.47 4.46999979019165 +4.57 4.570000171661377 +4.59 4.590000152587891 +4.71 4.710000038146973 +4.72 4.71999979019165 +4.79 4.789999961853027 +4.8 4.800000190734863 +4.92 4.920000076293945 +5.08 5.079999923706055 +5.24 9.559999942779541 +5.28 5.28000020980835 +5.4 5.400000095367432 +5.44 5.440000057220459 +5.45 5.449999809265137 +5.51 5.510000228881836 +5.54 5.539999961853027 +5.62 5.619999885559082 +5.67 5.670000076293945 +5.85 5.849999904632568 +5.88 5.880000114440918 +6.29 6.289999961853027 +6.55 6.550000190734863 +6.57 11.160000324249268 +6.63 9.59000015258789 +6.67 6.670000076293945 +6.72 6.71999979019165 +6.74 6.739999771118164 +6.84 6.840000152587891 +6.87 6.869999885559082 +7.05 7.050000190734863 +7.06 11.769999980926514 +7.11 7.110000133514404 +7.54 7.539999961853027 +7.56 7.559999942779541 +7.79 7.789999961853027 +7.82 7.820000171661377 +7.96 7.960000038146973 +7.96 7.960000038146973 +7.98 7.980000019073486 +8.07 8.069999694824219 +8.07 8.069999694824219 +8.32 8.319999694824219 +8.37 11.339999914169312 +8.42 11.760000109672546 +8.45 8.449999809265137 +8.45 8.449999809265137 +8.45 10.319999814033508 +8.45 16.009999752044678 +8.57 8.569999694824219 +8.61 8.609999656677246 +8.67 8.670000076293945 +8.71 8.710000038146973 +8.79 8.789999961853027 +8.91 13.480000019073486 +9.04 9.039999961853027 +9.13 9.130000114440918 +9.19 15.479999542236328 +9.22 9.220000267028809 +9.25 9.25 +9.26 9.260000228881836 +9.35 12.350000381469727 +9.48 9.479999542236328 +9.56 12.480000495910645 +9.57 9.569999694824219 +9.57 9.569999694824219 +9.68 9.680000305175781 +9.7 9.699999809265137 +9.71 17.25 +9.74 9.739999771118164 +9.8 9.800000190734863 +9.81 9.8100004196167 +9.93 9.930000305175781 +10.09 10.09000015258789 +10.09 10.09000015258789 +10.13 15.640000343322754 +10.16 18.139999866485596 +10.17 14.970000267028809 +10.19 10.1899995803833 +10.2 10.199999809265137 +10.22 10.220000267028809 +10.25 18.859999656677246 +10.26 10.260000228881836 +10.29 10.289999961853027 +10.6 10.600000381469727 +10.66 10.65999984741211 +10.67 10.670000076293945 +10.73 10.729999542236328 +11.15 20.27999973297119 +11.18 13.360000371932983 +11.19 18.979999542236328 +11.22 11.220000267028809 +11.34 16.62000036239624 +11.55 11.550000190734863 +11.57 11.569999694824219 +11.68 17.080000400543213 +11.82 11.819999694824219 +11.89 11.890000343322754 +11.91 11.90999984741211 +12.02 12.020000457763672 +12.16 12.15999984741211 +12.19 20.149999618530273 +12.32 12.319999694824219 +12.42 16.27999997138977 +12.44 12.4399995803833 +12.45 16.799999713897705 +12.46 12.460000038146973 +12.5 15.460000038146973 +12.54 12.539999961853027 +12.85 12.850000381469727 +12.9 12.899999618530273 +13.01 28.47000026702881 +13.1 23.030000686645508 +13.15 20.96999979019165 +13.35 13.350000381469727 +13.87 13.869999885559082 +13.89 13.890000343322754 +13.94 13.9399995803833 +13.99 13.989999771118164 +14.13 24.22000026702881 +14.21 14.210000038146973 +14.3 29.270000457763672 +14.44 14.4399995803833 +14.84 14.84000015258789 +14.92 14.920000076293945 +14.92 25.18000030517578 +14.93 30.40999984741211 +15.1 17.700000286102295 +15.15 24.40999984741211 +15.18 26.730000495910645 +15.22 15.220000267028809 +15.26 15.260000228881836 +15.3 25.5 +15.37 15.369999885559082 +15.45 15.449999809265137 +15.63 28.110000610351562 +15.75 15.75 +15.81 15.8100004196167 +15.86 25.079999923706055 +15.9 21.34999942779541 +15.92 15.920000076293945 +16.08 22.75 +16.09 16.520000159740448 +16.24 27.809999465942383 +16.25 19.419999957084656 +16.48 16.479999542236328 +16.69 16.690000534057617 +16.99 16.989999771118164 +16.99 42.489999771118164 +17.16 21.12999987602234 +17.37 31.360000610351562 +17.74 19.049999713897705 +17.79 45.60000038146973 +17.87 18.710000813007355 +18.2 18.200000762939453 +18.5 31.350000381469727 +18.56 18.559999465942383 +18.63 26.589999198913574 +18.63 30.389999270439148 +18.86 18.96000061184168 +18.89 18.889999389648438 +18.93 18.93000030517578 +19.0 35.62000036239624 +19.03 19.030000686645508 +19.06 19.059999465942383 +19.06 19.059999465942383 +19.13 45.719998359680176 +19.14 19.139999389648438 +19.28 27.600000381469727 +19.69 36.49000024795532 +20.07 46.80000019073486 +20.38 51.72999954223633 +20.64 20.639999389648438 +20.67 31.830000400543213 +20.79 20.790000915527344 +20.81 20.809999465942383 +20.82 21.419999718666077 +20.82 26.359999656677246 +21.18 21.18000030517578 +21.19 21.190000534057617 +21.23 21.229999542236328 +21.28 29.350000381469727 +21.32 21.31999969482422 +21.45 40.510000228881836 +21.49 30.739999771118164 +21.61 37.0600004196167 +21.7 27.58000087738037 +21.8 21.799999237060547 +21.94 23.940000593662262 +22.01 28.850000381469727 +22.08 22.079999923706055 +22.12 22.1200008392334 +22.12 22.1200008392334 +22.25 22.25 +22.27 22.270000457763672 +22.36 22.360000610351562 +22.68 22.68000030517578 +22.78 47.19000053405762 +22.85 33.070000648498535 +22.85 43.980000257492065 +22.94 35.38000011444092 +23.07 23.06999969482422 +23.13 28.799999237060547 +23.17 44.58999979496002 +23.19 23.190000534057617 +23.44 23.440000534057617 +23.45 24.74000072479248 +23.6 33.16000032424927 +23.77 23.770000457763672 +23.96 23.959999084472656 +24.02 24.020000457763672 +24.28 43.310001373291016 +24.49 42.62999963760376 +24.52 32.59000015258789 +24.73 45.369998931884766 +24.79 24.790000915527344 +24.8 34.369998931884766 +24.83 36.05000019073486 +24.86 65.3700008392334 +25.11 44.170000076293945 +25.28 25.280000686645508 +25.37 48.05000114440918 +25.42 40.78999996185303 +25.55 26.62999927997589 +25.67 37.69000053405762 +25.88 61.49999952316284 +26.08 26.079999923706055 +26.39 34.959999084472656 +26.43 26.43000030517578 +26.47 31.389999389648438 +26.49 26.489999771118164 +26.49 48.56999969482422 +26.64 64.32999992370605 +26.71 36.999999046325684 +26.73 45.69000015407801 +26.76 26.760000228881836 +27.07 28.649999737739563 +27.12 32.20000076293945 +27.3 70.61000061035156 +27.31 56.579999923706055 +27.63 27.6299991607666 +27.66 27.65999984741211 +27.72 46.60999870300293 +27.87 27.8700008392334 +28.11 44.59000015258789 +28.31 52.079999923706055 +28.45 74.05000114440918 +28.5 35.36999988555908 +28.56 40.71999931335449 +28.69 28.690000534057617 +28.71 55.46999931335449 +28.79 28.790000915527344 +28.89 56.489999771118164 +28.95 33.410000801086426 +29.02 56.64999961853027 +29.24 99.85000038146973 +29.36 62.52000093460083 +29.4 72.02999925613403 +29.41 64.77999973297119 +29.54 29.540000915527344 +29.59 37.37000012397766 +29.78 66.77999973297119 +30.25 30.32999999821186 +30.36 30.360000610351562 +30.37 31.660000830888748 +30.61 30.610000610351562 +30.62 102.65000009536743 +30.63 30.6299991607666 +30.65 60.19000053405762 +30.71 49.849998474121094 +30.81 55.989999771118164 +31.01 31.010000228881836 +31.15 31.149999618530273 +31.4 31.399999618530273 +31.61 31.610000610351562 +31.67 40.46000003814697 +31.77 42.09000027179718 +31.86 31.860000610351562 +31.91 78.51999855041504 +32.01 60.47999858856201 +32.18 58.61000061035156 +32.2 53.55000019073486 +32.23 42.89999961853027 +32.25 59.83000087738037 +32.37 62.99999809265137 +32.41 32.40999984741211 +32.47 41.14000129699707 +32.52 95.0400013923645 +32.75 56.19000053405762 +32.89 80.07999992370605 +32.92 47.7599983215332 +33.36 45.27000045776367 +33.52 60.010000228881836 +33.55 63.939998507499695 +33.58 54.55000162124634 +33.67 33.66999816894531 +33.76 112.27999687194824 +33.83 59.110002517700195 +33.85 47.719998359680176 +33.87 37.48999881744385 +34.03 71.51999759674072 +34.21 71.57999920845032 +34.35 34.349998474121094 +34.41 59.20000076293945 +34.58 34.58000183105469 +34.73 34.72999954223633 +34.97 45.160000801086426 +35.0 35.0 +35.08 36.060001850128174 +35.13 39.600000858306885 +35.17 64.01999855041504 +35.17 66.52999877929688 +35.56 37.63000130653381 +35.62 80.20999872684479 +35.65 56.83000183105469 +35.68 52.20000046491623 +35.72 98.71999931335449 +35.8 79.96999931335449 +35.89 81.2599983215332 +36.22 55.20000076293945 +36.26 36.2599983215332 +36.58 64.45000267028809 +36.7 73.76000118255615 +36.79 36.790000915527344 +36.89 71.8499984741211 +36.95 36.95000076293945 +37.07 37.06999969482422 +37.1 51.039998054504395 +37.14 53.41999936103821 +37.14 61.15999984741211 +37.24 47.04000186920166 +37.59 50.94000053405762 +37.6 84.39999866485596 +37.72 57.14000117778778 +37.78 68.10999877750874 +37.8 57.94999885559082 +37.85 50.38999843597412 +37.9 77.50000238418579 +38.05 40.24999928474426 +38.05 47.859999656677246 +38.3 98.48999977111816 +38.33 112.09000301361084 +38.57 55.81999969482422 +38.62 73.99999904632568 +38.79 95.44000053405762 +38.85 97.45999908447266 +38.88 48.58000087738037 +38.94 71.3499984741211 +39.01 39.0099983215332 +39.03 57.73999959230423 +39.18 99.6599988937378 +39.21 71.79999923706055 +39.34 39.34000015258789 +39.69 55.69999837875366 +39.81 74.81000137329102 +39.82 39.81999969482422 +39.83 119.80000114440918 +39.87 62.22999954223633 +39.9 64.98000144958496 +39.98 39.97999954223633 +40.0 84.59000015258789 +40.04 101.20000076293945 +40.17 80.41999745368958 +40.24 67.9000015258789 +40.42 78.04999947547913 +40.44 117.94000101089478 +40.78 49.8199987411499 +40.8 40.79999923706055 +40.98 92.01999759674072 +41.2 58.280001163482666 +41.29 41.290000915527344 +41.29 112.87000012397766 +41.31 53.08000135421753 +41.34 53.230000495910645 +41.34 115.33999919891357 +41.36 41.36000061035156 +41.44 85.41999888420105 +41.45 91.29999923706055 +41.62 41.619998931884766 +41.68 109.58000183105469 +41.71 83.06999969482422 +41.81 89.8600025177002 +41.85 76.21999740600586 +41.87 47.48999881744385 +41.89 41.88999938964844 +42.24 72.6500015258789 +42.31 52.91000175476074 +42.42 154.69999504089355 +42.48 71.12999927997589 +42.51 61.069997787475586 +42.55 87.71000003814697 +42.56 71.91000175476074 +42.67 71.45999908447266 +42.76 42.7599983215332 +42.85 85.33999824523926 +43.01 109.53999710083008 +43.02 46.84000039100647 +43.13 43.130001068115234 +43.16 58.079999923706055 +43.17 43.16999816894531 +43.19 156.05999875068665 +43.31 64.10000228881836 +43.37 56.84999895095825 +43.57 43.71999970078468 +43.71 108.69000053405762 +43.73 108.50999927520752 +43.92 48.319998145103455 +44.1 67.1299991607666 +44.22 103.33000373840332 +44.27 74.88000106811523 +44.43 106.65999984741211 +44.57 59.00999927520752 +45.06 45.060001373291016 +45.1 45.099998474121094 +45.19 45.189998626708984 +45.19 117.10000038146973 +45.24 109.34000396728516 +45.34 129.73999881744385 +45.35 109.28999698162079 +45.42 77.61999893188477 +45.45 45.45000076293945 +45.56 137.57999897003174 +45.59 82.54000091552734 +45.68 55.25 +45.92 90.97999954223633 +45.99 47.07000172138214 +46.02 82.27999877929688 +46.09 46.09000015258789 +46.1 98.17999839782715 +46.15 72.7800008058548 +46.18 74.29000091552734 +46.21 52.75999927520752 +46.27 85.61000061035156 +46.43 106.26000118255615 +46.45 110.90000343322754 +46.62 78.0099983215332 +46.8 80.46999740600586 +46.86 62.08000087738037 +46.87 70.80999952554703 +46.88 106.08000183105469 +46.97 88.1100025177002 +47.08 148.28000259399414 +47.27 50.60000038146973 +47.32 118.12999922037125 +47.57 90.32999801635742 +47.59 104.17000007629395 +47.69 99.88999909162521 +47.88 47.880001068115234 +48.01 91.72999802231789 +48.08 79.69000244140625 +48.11 48.11000061035156 +48.15 65.8500018119812 +48.22 105.07000017166138 +48.23 139.52999877929688 +48.25 48.25 +48.28 98.669997215271 +48.37 185.9499979019165 +48.45 48.45000076293945 +48.45 94.54000091552734 +48.52 146.69999885559082 +48.59 89.30999946594238 +49.12 49.119998931884766 +49.28 123.56999969482422 +49.44 110.93999814987183 +49.68 73.63999938964844 +49.77 50.33000046014786 +49.78 66.46999931335449 +50.02 63.380000829696655 +50.08 156.16000366210938 +50.09 106.28000068664551 +50.26 72.05999755859375 +50.28 50.279998779296875 +50.31 117.44000053405762 +50.32 90.13999938964844 +50.4 96.11999988555908 +50.41 98.72999799251556 +50.66 55.37999963760376 +50.7 131.11999821662903 +50.83 98.69000148773193 +50.92 53.70999813079834 +50.96 103.71999835968018 +51.25 67.0 +51.29 87.35000276565552 +51.29 124.93000030517578 +51.72 97.17000198364258 +51.79 139.90000343322754 +51.84 168.94000053405762 +51.85 171.64999961853027 +52.17 206.86999320983887 +52.23 177.1599998474121 +52.44 88.48999881744385 +52.5 105.41000175476074 +52.53 64.98999881744385 +52.72 52.720001220703125 +52.73 74.04999923706055 +52.85 89.63999938964844 +52.87 130.87999725341797 +53.02 100.50999927520752 +53.06 259.9299945831299 +53.18 53.18000030517578 +53.27 53.27000045776367 +53.59 53.59000015258789 +53.78 139.38999938964844 +53.93 57.890000343322754 +53.94 63.529998779296875 +54.1 152.7699956893921 +54.31 77.38000106811523 +54.34 125.46999943256378 +54.43 132.04999923706055 +54.44 103.01999950408936 +54.47 186.52000045776367 +54.73 63.179999351501465 +54.75 112.82999992370605 +54.83 110.82000160217285 +54.99 160.40000343322754 +55.1 161.35999965667725 +55.18 215.58000373840332 +55.2 126.65999984741211 +55.39 137.6699981689453 +55.51 74.55999803543091 +55.63 96.43000030517578 +55.99 187.10999989509583 +56.04 150.5800018310547 +56.07 118.15000057220459 +56.1 135.79000091552734 +56.15 144.64000034332275 +56.33 61.77000188827515 +56.62 78.88999938964844 +56.68 154.13999938964844 +56.81 169.64000129699707 +57.08 69.98000144958496 +57.11 168.0100040435791 +57.12 100.28999710083008 +57.23 65.9399995803833 +57.25 133.46999740600586 +57.29 112.54000091552734 +57.35 110.89999866485596 +57.37 115.109998524189 +57.46 147.78999710083008 +57.64 112.19000101089478 +57.67 57.66999816894531 +57.89 111.15999984741211 +57.93 68.02000045776367 +58.0 123.9399995803833 +58.08 58.08000183105469 +58.09 206.37000274658203 +58.13 105.84999942779541 +58.43 165.0900001525879 +58.52 167.0299997329712 +58.66 136.04000091552734 +58.67 205.36999702453613 +58.75 90.41000083088875 +58.86 165.14000129699707 +59.07 87.86999893188477 +59.16 224.25 +59.21 90.35999870300293 +59.34 127.44999893009663 +59.43 106.50000202655792 +59.45 67.90000057220459 +59.45 197.11999893188477 +59.5 149.63999938964844 +59.55 61.459999203681946 +59.61 85.97000026702881 +59.62 113.3299970626831 +59.68 73.89000034332275 +59.68 94.40999984741211 +59.7 193.1699981689453 +59.71 60.22999906539917 +59.83 145.17000007629395 +59.87 228.80999946594238 +59.99 134.04000282287598 +60.02 66.76000022888184 +60.06 60.060001373291016 +60.12 113.34999942779541 +60.13 214.27000045776367 +60.22 108.10000228881836 +60.26 105.94999847561121 +60.26 165.32999849319458 +60.53 66.37999868392944 +60.6 82.8499984741211 +60.71 72.04999899864197 +60.85 132.36999607086182 +61.21 160.86999797821045 +61.7 127.55000257492065 +61.86 248.9700005054474 +61.88 112.15999984741211 +61.92 125.29999899864197 +61.94 119.6099967956543 +62.14 110.59000015258789 +62.2 149.91000080108643 +62.23 111.3499984741211 +62.3 158.41999912261963 +62.39 110.95999908447266 +62.52 123.97999966144562 +62.72 123.78999900817871 +62.74 153.10000038146973 +62.85 167.01999855041504 +62.9 256.0699996948242 +62.92 129.3899974822998 +63.12 93.47999954223633 +63.33 135.38999938964844 +63.35 116.93999862670898 +63.42 172.76000213623047 +63.51 123.51999855041504 +63.9 135.70000076293945 +64.0 191.55000257492065 +64.22 86.97000122070312 +64.25 131.25 +64.3 122.3800048828125 +64.36 85.59000015258789 +64.46 134.44000053405762 +64.65 143.54000091552734 +64.67 121.15999794006348 +64.77 214.40999603271484 +64.87 194.61000156402588 +64.95 324.87999153137207 +65.02 175.6099967956543 +65.02 259.6299982070923 +65.38 168.7100009918213 +65.43 112.27000069618225 +65.43 289.6800003051758 +65.44 192.89000137150288 +65.55 66.16000306606293 +65.62 139.67000198364258 +65.7 65.69999694824219 +65.72 77.54000091552734 +66.17 177.10999631881714 +66.17 200.60999870300293 +66.36 131.73000144958496 +66.51 83.50000190734863 +66.61 66.61000061035156 +66.61 78.93000030517578 +66.67 129.84999752044678 +66.89 99.96000003814697 +67.12 67.12000274658203 +67.18 234.21000003814697 +67.26 77.9900016784668 +67.38 178.72999572753906 +67.45 197.29999446868896 +67.48 268.0900020599365 +67.59 272.95999336242676 +67.94 125.89000129699707 +67.98 123.36000299453735 +68.01 124.84000396728516 +68.04 166.76000022888184 +68.22 181.57000064849854 +68.25 113.52000045776367 +68.25 163.2900013923645 +68.32 247.04999542236328 +68.41 157.72000312805176 +68.5 156.36999893188477 +68.81 93.02999782562256 +68.85 160.57999649643898 +68.89 89.69999885559082 +68.95 140.99999594688416 +68.96 192.4799976348877 +69.32 156.29000091552734 +69.53 239.17000007629395 +69.74 246.84999418258667 +69.8 101.63000345230103 +69.88 220.45999908447266 +69.96 83.84999942779541 +69.97 169.86000031232834 +70.0 236.76000022888184 +70.04 196.70000076293945 +70.06 92.17999839782715 +70.24 213.77999877929688 +70.35 247.5099983215332 +70.38 179.95999908447266 +70.39 164.79999923706055 +70.52 181.33999824523926 +70.53 70.52999877929688 +70.56 110.53999710083008 +70.85 223.94999885559082 +70.89 149.81999969482422 +70.93 160.79000282287598 +71.01 92.20000267028809 +71.07 112.36000061035156 +71.13 135.149995803833 +71.19 136.1800012588501 +71.26 318.7700004577637 +71.31 231.88999405503273 +71.32 145.21000003814697 +71.35 145.909996509552 +71.5 217.409996509552 +71.54 71.54000091552734 +71.55 90.41000270843506 +71.68 72.9500002861023 +71.68 227.97000122070312 +71.78 189.71999979019165 +71.8 103.66000366210938 +71.89 180.57999992370605 +72.04 156.63000106811523 +72.18 181.46999728679657 +72.51 264.0600047111511 +72.53 265.42000015079975 +72.56 132.61999893188477 +72.62 205.2400016784668 +72.79 72.79000091552734 +72.98 286.76000213623047 +73.18 123.78000068664551 +73.32 175.96999979019165 +73.48 144.8300018310547 +73.63 320.47999143600464 +73.65 114.11000156402588 +73.68 161.0300030708313 +73.88 291.28999376296997 +73.93 156.77999877929688 +74.0 179.8499994277954 +74.02 89.27999687194824 +74.15 74.1500015258789 +74.19 74.47000244259834 +74.19 122.44000244140625 +74.3 159.72000193595886 +74.42 219.62999820709229 +74.45 203.839994430542 +74.52 271.6399955749512 +74.53 342.6200008392334 +74.59 157.12999725341797 +74.62 163.89999961853027 +74.72 139.05000114440918 +74.78 230.83999752998352 +75.03 174.98999881744385 +75.1 214.62999725341797 +75.19 149.6600048840046 +75.29 93.4900016784668 +75.35 256.91999912261963 +75.42 153.46999764442444 +75.66 225.4800033569336 +75.73 236.60000133514404 +75.83 199.4000015258789 +75.88 243.89000129699707 +76.05 179.770001411438 +76.1 257.43999671936035 +76.28 168.48000144958496 +76.28 177.9100022315979 +76.33 363.09000396728516 +76.52 207.39999389648438 +76.69 212.87000370025635 +76.7 307.5399944782257 +76.71 132.17999839782715 +76.72 166.42000007629395 +76.72 258.1899985074997 +76.74 102.81999778747559 +76.92 401.7999897003174 +76.93 384.4699947834015 +77.02 296.6499948501587 +77.1 187.63999557495117 +77.36 281.19999504089355 +77.42 217.32000160217285 +77.57 189.66000270843506 +77.66 267.380003452301 +77.81 178.31999683380127 +77.84 350.7999897003174 +77.89 249.5399990081787 +77.97 178.2599983215332 +78.21 189.1099977493286 +78.26 284.6300048828125 +78.28 306.25 +78.3 168.71000388264656 +78.31 161.80999946594238 +78.62 275.3200035095215 +78.64 94.55999946594238 +78.73 310.6199974119663 +78.89 175.31999969482422 +78.98 293.25000381469727 +79.12 276.419997215271 +79.19 194.30000096559525 +79.21 237.62999820709229 +79.38 151.22999572753906 +79.42 124.6099967956543 +79.48 116.55000305175781 +79.48 200.64000129699707 +79.49 294.1199951171875 +79.54 145.23999786376953 +79.55 254.54000186920166 +79.75 215.13999938964844 +79.83 294.9700012207031 +79.96 79.95999908447266 +79.97 266.4900016784668 +79.99 219.6599998474121 +80.23 248.71000480651855 +80.3 133.4800033569336 +80.3 305.7800064086914 +80.46 249.17000296711922 +80.52 136.21999502182007 +80.58 261.16000175476074 +80.6 281.23999977111816 +80.71 275.0100000500679 +80.74 240.4599997997284 +80.84 142.60999822616577 +80.92 340.5499963760376 +80.96 372.2499928474426 +80.97 93.32000160217285 +80.99 317.74999809265137 +81.17 220.55999755859375 +81.32 158.86000061035156 +81.32 206.16000366210938 +81.47 198.91000175476074 +81.58 173.76000022888184 +81.64 207.5300006866455 +81.66 261.5100030899048 +82.24 243.59999752044678 +82.3 157.18000411987305 +82.34 214.0699977874756 +82.41 135.83000302314758 +82.52 240.23999977111816 +82.55 139.6900042295456 +82.56 175.8799991607666 +82.72 232.63000202178955 +82.97 101.9000015258789 +83.08 181.57000160217285 +83.27 83.2699966430664 +83.33 348.75000198185444 +83.4 196.729998588562 +83.54 299.12000465393066 +83.57 139.38999938964844 +83.58 163.54000091552734 +83.87 263.64000415802 +83.92 432.67000015079975 +83.93 358.9400003552437 +84.03 240.19000244140625 +84.23 209.53000235557556 +84.31 157.0999984741211 +84.38 220.1699981689453 +84.4 443.3400018811226 +84.69 249.4900016784668 +84.72 105.00000095367432 +84.83 157.4800033569336 +85.0 221.21999502182007 +85.03 283.9400005340576 +85.1 217.27999687194824 +85.14 266.7100009918213 +85.23 277.7100009918213 +85.49 261.4599976539612 +85.49 325.9499976634979 +85.51 165.59000205993652 +85.74 341.80999755859375 +85.76 160.57000350952148 +85.87 357.5099983215332 +85.9 202.45000457763672 +86.0 322.60000133514404 +86.22 152.8300018310547 +86.23 152.99000358581543 +86.63 381.5999984741211 +86.69 138.42000198364258 +86.92 147.14999723434448 +86.93 97.59000015258789 +86.93 218.18000030517578 +87.14 219.50999546051025 +87.22 211.01000022888184 +87.4 137.73000198602676 +87.48 153.86000204086304 +87.57 307.0799951553345 +87.61 321.82000064849854 +87.67 248.4600009918213 +87.83 130.96000289916992 +87.94 134.9800043106079 +87.99 227.0399990081787 +88.02 406.7899971008301 +88.05 97.53000259399414 +88.07 210.45000457763672 +88.17 245.6500015258789 +88.22 369.4600009918213 +88.36 305.6399974822998 +88.47 213.9400006532669 +88.48 211.84000635147095 +88.55 199.71000289916992 +88.77 187.49999463558197 +88.78 337.23999977111816 +88.8 254.1300015449524 +88.91 276.40999829769135 +89.01 98.69000244140625 +89.03 431.6499996185303 +89.1 113.83999919891357 +89.15 234.32000160217285 +89.28 343.82000064849854 +89.38 496.16999435424805 +89.53 153.05999755859375 +89.55 89.55000305175781 +89.55 96.27000284194946 +89.55 172.62000274658203 +89.8 205.1400022506714 +89.81 225.64000058174133 +89.93 89.93000030517578 +90.05 90.05000305175781 +90.05 247.1500015258789 +90.07 137.82999801635742 +90.12 234.76000308990479 +90.2 208.32999616861343 +90.25 386.8999948501587 +90.28 334.17000007629395 +90.35 158.36999893188477 +90.38 148.65999841690063 +90.51 268.7700004577637 +90.56 298.8899937272072 +90.69 181.1000051498413 +90.69 523.360002592206 +90.73 260.59000366926193 +90.77 203.03999733924866 +91.05 263.67000579833984 +91.16 298.55999755859375 +91.42 196.41999912261963 +91.48 144.56000471115112 +91.53 301.9800033569336 +91.61 215.59000027179718 +91.63 432.17999362945557 +91.78 330.9499988555908 +91.88 277.8299951553345 +91.97 205.4900016784668 +92.05 317.69000363349915 +92.11 307.70000088214874 +92.33 92.33000183105469 +92.37 226.410005569458 +92.4 265.1600036621094 +92.55 295.59000039100647 +92.61 249.38999938964844 +92.82 182.87000274658203 +92.96 185.98999691009521 +92.98 494.779993057251 +93.03 165.80999958515167 +93.09 456.1800003051758 +93.11 427.2800006866455 +93.61 465.8599934577942 +93.64 314.1999969482422 +93.73 475.3300018310547 +94.08 187.56000137329102 +94.15 362.9200019836426 +94.25 308.1900006532669 +94.27 351.70999336242676 +94.31 216.75 +94.33 193.02000427246094 +94.34 320.75000190734863 +94.38 260.18999683856964 +94.43 174.89999771118164 +94.54 340.19000244140625 +94.66 192.19000625610352 +94.68 189.23999977111816 +94.68 344.1700019836426 +94.72 274.6800003051758 +95.07 417.67000102996826 +95.11 393.99999433755875 +95.28 140.37999725341797 +95.33 244.99000671505928 +95.34 192.92999649047852 +95.38 230.36000156402588 +95.48 406.1000007688999 +95.53 248.58999633789062 +95.53 252.71000289916992 +95.81 134.81999588012695 +95.81 551.9899978637695 +95.84 274.1599931716919 +96.09 191.52999687194824 +96.23 398.2100067138672 +96.25 261.8400020599365 +96.29 199.10999870300293 +96.38 323.4199962615967 +96.62 309.4900064468384 +96.68 336.87000274658203 +96.73 272.0500030517578 +96.78 190.27000045776367 +96.91 180.18000030517578 +96.94 216.54999923706055 +97.09 428.03999519348145 +97.24 346.41000083088875 +97.26 373.67999935150146 +97.39 257.9600028991699 +97.46 449.1699924468994 +97.51 132.09000396728516 +97.56 97.55999755859375 +97.57 592.3499927520752 +97.65 196.34000301361084 +97.68 258.7100033760071 +97.71 175.70000076293945 +97.81 297.20999908447266 +97.83 396.38999938964844 +97.87 254.50000381469727 +98.18 105.29000043869019 +98.22 438.4100036621094 +98.23 525.5100040435791 +98.31 335.93999576568604 +98.48 286.0400047302246 +98.51 132.86000061035156 +98.57 404.3500061035156 +98.72 263.8600025177002 +98.96 288.1999988555908 +99.13 243.6900019645691 +99.15 210.11000061035156 +99.21 347.7999954223633 +99.24 537.6500015258789 +99.29 291.48000717163086 +99.36 106.41000080108643 +99.62 417.3700008392334 +99.65 185.62000179290771 +99.67 180.92999649047852 +99.68 230.6400032043457 +99.91 367.2900071144104 +99.92 376.32999646663666 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-7-20fdc99aa046b2c41d9b85ab338c749c b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-7-20fdc99aa046b2c41d9b85ab338c749c new file mode 100644 index 0000000000000..1a4528978b74f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-7-20fdc99aa046b2c41d9b85ab338c749c @@ -0,0 +1,1049 @@ + 65560 3.1 + 65718 2.38 + 65740 3.04 +alice allen 65662 1.55 +alice allen 65720 1.79 +alice allen 65758 1.98 +alice brown 65696 0.13 +alice carson 65559 4.2 +alice davidson 65547 1.51 +alice falkner 65669 4.19 +alice garcia 65613 0.72 +alice hernandez 65737 0.92 +alice hernandez 65784 2.09 +alice johnson 65739 2.55 +alice king 65660 3.84 +alice king 65734 2.96 +alice king 65738 2.14 +alice laertes 65669 0.28 +alice laertes 65671 1.16 +alice miller 65590 4.95 +alice nixon 65586 2.98 +alice nixon 65595 2.36 +alice nixon 65604 2.45 +alice ovid 65737 3.2 +alice polk 65548 1.23 +alice quirinius 65636 0.23 +alice quirinius 65728 0.82 +alice robinson 65606 3.99 +alice robinson 65789 4.35 +alice steinbeck 65578 4.72 +alice steinbeck 65673 3.97 +alice steinbeck 65786 3.92 +alice underhill 65750 2.06 +alice van buren 65562 2.43 +alice xylophone 65578 2.22 +alice xylophone 65585 2.11 +alice xylophone 65599 2.92 +alice zipper 65553 3.78 +alice zipper 65662 2.61 +alice zipper 65766 3.12 +bob brown 65584 2.09 +bob brown 65777 1.62 +bob brown 65783 2.4 +bob carson 65713 3.87 +bob davidson 65664 4.25 +bob davidson 65693 3.54 +bob davidson 65768 2.91 +bob ellison 65591 2.23 +bob ellison 65624 1.69 +bob ellison 65721 1.69 +bob ellison 65760 1.69 +bob falkner 65789 0.21 +bob garcia 65585 3.91 +bob garcia 65598 2.64 +bob garcia 65673 3.26 +bob garcia 65754 3.29 +bob garcia 65782 2.86 +bob hernandez 65557 3.72 +bob ichabod 65549 1.39 +bob king 65715 2.76 +bob king 65757 1.71 +bob king 65783 2.24 +bob laertes 65602 4.98 +bob laertes 65663 3.56 +bob miller 65608 4.95 +bob ovid 65564 1.23 +bob ovid 65619 1.53 +bob ovid 65686 1.84 +bob ovid 65726 2.38 +bob polk 65594 0.28 +bob quirinius 65700 3.82 +bob steinbeck 65637 0.22 +bob van buren 65778 2.89 +bob white 65543 4.75 +bob white 65605 2.89 +bob xylophone 65574 1.7 +bob xylophone 65666 2.51 +bob young 65556 0.95 +bob zipper 65559 3.18 +bob zipper 65633 3.2 +bob zipper 65739 3.24 +calvin allen 65669 2.3 +calvin brown 65537 1.1 +calvin brown 65580 2.82 +calvin brown 65677 2.57 +calvin carson 65637 1.33 +calvin davidson 65541 1.98 +calvin davidson 65564 1.6 +calvin ellison 65667 1.85 +calvin falkner 65573 3.52 +calvin falkner 65596 3.31 +calvin falkner 65738 2.36 +calvin falkner 65762 2.26 +calvin falkner 65778 2.7 +calvin falkner 65784 2.98 +calvin garcia 65664 2.9 +calvin hernandez 65578 2.08 +calvin johnson 65731 4.69 +calvin laertes 65570 4.7 +calvin laertes 65684 3.68 +calvin nixon 65654 2.74 +calvin nixon 65724 3.27 +calvin nixon 65749 2.88 +calvin ovid 65554 3.34 +calvin ovid 65643 2.38 +calvin ovid 65663 2.74 +calvin ovid 65715 2.47 +calvin polk 65731 4.36 +calvin quirinius 65741 4.0 +calvin quirinius 65769 2.2 +calvin robinson 65581 3.32 +calvin steinbeck 65680 1.29 +calvin steinbeck 65762 1.3 +calvin steinbeck 65779 1.98 +calvin thompson 65560 4.08 +calvin thompson 65640 3.33 +calvin underhill 65732 2.41 +calvin van buren 65552 1.05 +calvin van buren 65771 1.33 +calvin white 65553 4.7 +calvin white 65561 2.68 +calvin xylophone 65575 4.6 +calvin xylophone 65596 4.77 +calvin xylophone 65713 3.23 +calvin young 65574 0.27 +calvin young 65746 0.9 +calvin zipper 65669 4.4 +calvin zipper 65739 2.29 +david allen 65588 3.86 +david allen 65617 3.18 +david brown 65637 1.17 +david brown 65760 1.01 +david davidson 65559 1.37 +david davidson 65756 1.57 +david davidson 65778 1.89 +david davidson 65779 2.43 +david ellison 65634 3.23 +david ellison 65724 2.95 +david ellison 65724 2.95 +david hernandez 65763 1.15 +david ichabod 65699 1.67 +david ichabod 65715 1.26 +david laertes 65762 1.38 +david nixon 65536 1.27 +david ovid 65623 0.23 +david ovid 65628 1.15 +david quirinius 65697 1.14 +david quirinius 65759 1.65 +david quirinius 65779 1.93 +david robinson 65762 3.51 +david robinson 65775 3.38 +david thompson 65550 3.3 +david underhill 65602 0.12 +david underhill 65662 2.29 +david underhill 65751 2.43 +david van buren 65625 1.55 +david van buren 65634 3.25 +david white 65678 0.17 +david xylophone 65537 1.07 +david xylophone 65670 0.72 +david xylophone 65764 0.94 +david young 65551 4.51 +david young 65694 2.74 +ethan allen 65747 3.61 +ethan brown 65539 2.9 +ethan brown 65617 1.59 +ethan brown 65685 2.17 +ethan brown 65685 2.17 +ethan brown 65722 2.64 +ethan brown 65733 2.75 +ethan carson 65742 2.84 +ethan ellison 65714 4.87 +ethan ellison 65732 3.9 +ethan falkner 65577 3.61 +ethan falkner 65614 1.95 +ethan garcia 65736 4.63 +ethan hernandez 65618 0.46 +ethan johnson 65536 1.76 +ethan king 65614 0.92 +ethan laertes 65562 2.89 +ethan laertes 65597 3.45 +ethan laertes 65628 3.15 +ethan laertes 65643 3.53 +ethan laertes 65680 3.18 +ethan laertes 65745 3.36 +ethan laertes 65760 3.01 +ethan miller 65712 1.97 +ethan nixon 65766 4.1 +ethan ovid 65697 3.81 +ethan polk 65589 0.7 +ethan polk 65615 1.92 +ethan polk 65622 2.0 +ethan polk 65622 2.0 +ethan quirinius 65542 4.64 +ethan quirinius 65591 3.97 +ethan quirinius 65706 2.88 +ethan robinson 65547 2.2 +ethan robinson 65659 2.17 +ethan underhill 65570 2.45 +ethan van buren 65572 1.11 +ethan white 65677 3.42 +ethan white 65788 4.09 +ethan xylophone 65595 4.66 +ethan zipper 65593 2.1 +ethan zipper 65680 2.53 +fred davidson 65552 0.37 +fred davidson 65595 2.31 +fred davidson 65721 2.65 +fred ellison 65548 4.32 +fred ellison 65691 3.44 +fred ellison 65771 3.1 +fred falkner 65637 4.59 +fred falkner 65648 2.79 +fred falkner 65783 2.35 +fred hernandez 65541 3.87 +fred ichabod 65572 1.45 +fred ichabod 65789 1.68 +fred johnson 65758 3.86 +fred king 65694 4.05 +fred king 65745 4.5 +fred laertes 65769 3.89 +fred miller 65536 2.59 +fred nixon 65560 2.52 +fred nixon 65612 2.0 +fred nixon 65703 1.35 +fred nixon 65705 1.16 +fred polk 65603 2.9 +fred polk 65656 1.65 +fred polk 65701 1.75 +fred polk 65706 1.4 +fred quirinius 65697 1.91 +fred quirinius 65782 3.24 +fred robinson 65623 2.6 +fred steinbeck 65544 2.0 +fred steinbeck 65651 3.32 +fred steinbeck 65755 3.51 +fred underhill 65629 2.56 +fred van buren 65537 4.97 +fred van buren 65561 3.28 +fred van buren 65745 3.24 +fred van buren 65789 3.27 +fred white 65589 3.16 +fred young 65594 0.34 +fred young 65773 1.56 +fred zipper 65553 0.37 +gabriella allen 65646 1.68 +gabriella allen 65677 1.6 +gabriella brown 65704 0.02 +gabriella brown 65753 1.86 +gabriella carson 65586 0.37 +gabriella davidson 65565 3.45 +gabriella ellison 65706 1.15 +gabriella ellison 65716 2.06 +gabriella falkner 65623 2.09 +gabriella falkner 65711 2.48 +gabriella falkner 65767 1.82 +gabriella garcia 65571 3.17 +gabriella hernandez 65587 0.74 +gabriella hernandez 65717 0.96 +gabriella ichabod 65559 0.63 +gabriella ichabod 65633 2.42 +gabriella ichabod 65702 3.27 +gabriella ichabod 65712 3.6 +gabriella ichabod 65717 3.56 +gabriella king 65651 3.59 +gabriella king 65657 2.17 +gabriella laertes 65781 2.81 +gabriella miller 65646 3.47 +gabriella ovid 65556 1.23 +gabriella ovid 65583 1.95 +gabriella polk 65701 3.58 +gabriella polk 65790 2.08 +gabriella steinbeck 65582 3.6 +gabriella steinbeck 65653 2.7 +gabriella thompson 65682 1.78 +gabriella thompson 65755 3.21 +gabriella thompson 65766 2.71 +gabriella van buren 65581 3.36 +gabriella van buren 65644 2.6 +gabriella white 65638 4.55 +gabriella young 65699 4.13 +gabriella young 65774 3.58 +gabriella zipper 65540 0.96 +gabriella zipper 65754 2.13 +holly allen 65596 0.05 +holly brown 65599 3.23 +holly brown 65619 3.4 +holly falkner 65720 4.06 +holly hernandez 65602 3.67 +holly hernandez 65686 3.39 +holly hernandez 65750 3.2 +holly hernandez 65788 2.95 +holly ichabod 65711 4.73 +holly ichabod 65749 3.54 +holly ichabod 65752 3.27 +holly johnson 65655 4.19 +holly johnson 65662 3.84 +holly johnson 65755 2.65 +holly king 65549 3.61 +holly king 65648 2.37 +holly laertes 65664 4.14 +holly miller 65653 3.84 +holly nixon 65539 4.09 +holly nixon 65658 3.04 +holly polk 65743 2.1 +holly polk 65751 2.78 +holly robinson 65564 0.24 +holly thompson 65538 2.39 +holly thompson 65578 1.49 +holly thompson 65713 1.54 +holly underhill 65634 4.69 +holly underhill 65654 3.08 +holly underhill 65721 3.14 +holly underhill 65759 2.61 +holly van buren 65727 0.04 +holly white 65536 4.59 +holly white 65602 4.13 +holly xylophone 65544 1.49 +holly young 65606 4.39 +holly young 65765 3.81 +holly zipper 65607 4.12 +holly zipper 65755 3.3 +irene allen 65556 3.45 +irene brown 65633 4.8 +irene brown 65650 3.77 +irene brown 65765 3.53 +irene carson 65590 2.83 +irene ellison 65659 3.15 +irene ellison 65696 2.0 +irene falkner 65620 0.99 +irene falkner 65661 1.41 +irene garcia 65660 1.55 +irene garcia 65711 1.72 +irene garcia 65787 1.57 +irene ichabod 65645 0.95 +irene ichabod 65722 1.49 +irene johnson 65583 4.51 +irene laertes 65664 2.5 +irene laertes 65710 1.5 +irene laertes 65722 2.01 +irene miller 65730 4.33 +irene nixon 65631 2.36 +irene nixon 65643 3.43 +irene nixon 65653 2.43 +irene ovid 65691 3.24 +irene ovid 65734 3.17 +irene ovid 65753 3.18 +irene polk 65551 4.51 +irene polk 65575 2.97 +irene polk 65579 3.58 +irene polk 65595 2.82 +irene polk 65610 2.99 +irene quirinius 65724 3.5 +irene quirinius 65769 3.85 +irene quirinius 65773 4.21 +irene robinson 65554 2.67 +irene steinbeck 65683 1.48 +irene thompson 65688 0.06 +irene underhill 65591 3.61 +irene underhill 65787 4.01 +irene van buren 65579 4.26 +irene van buren 65589 4.37 +irene xylophone 65775 4.81 +jessica brown 65588 2.87 +jessica carson 65553 1.02 +jessica carson 65672 1.82 +jessica carson 65747 1.91 +jessica davidson 65549 4.48 +jessica davidson 65606 2.72 +jessica davidson 65675 2.23 +jessica davidson 65727 2.12 +jessica ellison 65567 3.0 +jessica ellison 65663 3.15 +jessica falkner 65584 2.11 +jessica garcia 65676 2.13 +jessica garcia 65789 3.54 +jessica ichabod 65704 3.48 +jessica johnson 65607 3.55 +jessica johnson 65720 3.0 +jessica miller 65733 2.9 +jessica nixon 65590 2.18 +jessica nixon 65774 3.1 +jessica ovid 65582 3.23 +jessica ovid 65751 3.14 +jessica polk 65637 1.12 +jessica quirinius 65562 3.06 +jessica quirinius 65608 1.75 +jessica quirinius 65712 1.51 +jessica quirinius 65716 1.37 +jessica robinson 65576 1.11 +jessica thompson 65581 4.94 +jessica thompson 65675 3.56 +jessica underhill 65656 3.97 +jessica underhill 65702 3.01 +jessica underhill 65783 3.5 +jessica van buren 65615 2.15 +jessica white 65544 1.89 +jessica white 65570 1.92 +jessica white 65594 2.67 +jessica white 65673 2.1 +jessica white 65779 2.04 +jessica xylophone 65562 0.49 +jessica young 65623 0.5 +jessica young 65711 2.37 +jessica zipper 65600 1.71 +jessica zipper 65657 1.0 +jessica zipper 65778 0.9 +katie allen 65542 1.3 +katie brown 65590 3.06 +katie davidson 65619 1.89 +katie ellison 65675 1.92 +katie ellison 65699 2.55 +katie falkner 65728 2.42 +katie garcia 65625 4.18 +katie garcia 65747 4.33 +katie hernandez 65550 2.1 +katie ichabod 65658 1.84 +katie ichabod 65726 2.41 +katie ichabod 65757 3.2 +katie king 65629 0.86 +katie king 65647 2.09 +katie king 65776 2.74 +katie miller 65541 0.66 +katie miller 65661 1.39 +katie nixon 65669 2.06 +katie ovid 65681 1.61 +katie polk 65746 4.74 +katie polk 65784 2.57 +katie robinson 65697 4.74 +katie van buren 65643 0.61 +katie van buren 65730 2.79 +katie white 65620 0.42 +katie white 65719 1.38 +katie xylophone 65585 3.02 +katie young 65644 2.64 +katie young 65746 2.45 +katie young 65764 2.66 +katie zipper 65568 2.33 +katie zipper 65733 1.97 +luke allen 65547 2.04 +luke allen 65552 1.49 +luke allen 65576 2.14 +luke allen 65681 2.2 +luke allen 65776 1.92 +luke brown 65719 4.3 +luke davidson 65656 3.37 +luke davidson 65791 3.9 +luke ellison 65582 0.23 +luke ellison 65664 0.51 +luke ellison 65779 0.87 +luke falkner 65589 2.22 +luke falkner 65618 1.22 +luke garcia 65687 4.73 +luke garcia 65778 3.56 +luke ichabod 65629 3.07 +luke ichabod 65654 3.58 +luke johnson 65545 4.33 +luke johnson 65716 3.01 +luke johnson 65718 3.17 +luke laertes 65608 3.79 +luke laertes 65657 3.89 +luke laertes 65685 2.82 +luke laertes 65730 2.96 +luke laertes 65756 3.19 +luke miller 65752 4.7 +luke ovid 65569 4.4 +luke ovid 65693 2.58 +luke polk 65645 0.57 +luke polk 65658 2.73 +luke quirinius 65655 4.1 +luke robinson 65634 4.9 +luke robinson 65772 4.19 +luke thompson 65626 0.15 +luke underhill 65553 1.28 +luke underhill 65571 0.84 +luke underhill 65651 1.14 +luke van buren 65678 0.34 +luke white 65693 0.91 +luke xylophone 65597 2.27 +luke zipper 65641 4.63 +mike allen 65706 3.06 +mike brown 65654 3.57 +mike carson 65698 4.46 +mike carson 65700 3.89 +mike carson 65751 3.58 +mike davidson 65658 2.06 +mike davidson 65759 3.34 +mike ellison 65598 3.96 +mike ellison 65606 3.28 +mike ellison 65718 3.38 +mike ellison 65738 2.56 +mike ellison 65760 3.03 +mike falkner 65609 4.85 +mike garcia 65571 1.82 +mike garcia 65600 1.42 +mike garcia 65770 1.92 +mike hernandez 65548 1.42 +mike hernandez 65672 1.75 +mike ichabod 65621 3.73 +mike king 65563 4.34 +mike king 65586 3.75 +mike king 65591 3.09 +mike king 65642 2.69 +mike king 65769 2.36 +mike king 65776 2.55 +mike miller 65549 3.96 +mike nixon 65619 0.09 +mike nixon 65704 2.15 +mike polk 65619 4.13 +mike polk 65658 4.27 +mike polk 65704 3.77 +mike quirinius 65717 2.81 +mike steinbeck 65550 3.18 +mike steinbeck 65564 2.58 +mike steinbeck 65573 2.12 +mike steinbeck 65749 1.72 +mike van buren 65620 0.09 +mike van buren 65770 0.88 +mike white 65648 1.72 +mike white 65685 1.12 +mike white 65769 1.74 +mike white 65778 2.05 +mike young 65545 1.69 +mike young 65581 0.92 +mike young 65736 1.84 +mike zipper 65552 4.8 +mike zipper 65695 4.16 +mike zipper 65779 4.22 +nick allen 65641 3.78 +nick allen 65786 3.74 +nick brown 65724 4.61 +nick davidson 65601 0.88 +nick ellison 65691 4.04 +nick ellison 65745 3.22 +nick falkner 65583 4.44 +nick falkner 65676 3.08 +nick garcia 65695 1.05 +nick garcia 65712 2.18 +nick garcia 65720 1.94 +nick ichabod 65572 2.62 +nick ichabod 65681 2.99 +nick ichabod 65737 3.55 +nick johnson 65585 0.56 +nick johnson 65784 0.42 +nick laertes 65624 0.16 +nick miller 65757 4.23 +nick nixon 65650 0.7 +nick ovid 65719 3.93 +nick polk 65716 3.66 +nick quirinius 65588 2.88 +nick quirinius 65723 2.42 +nick robinson 65547 0.21 +nick robinson 65675 0.57 +nick steinbeck 65689 4.11 +nick thompson 65610 2.32 +nick underhill 65619 2.73 +nick van buren 65603 1.84 +nick xylophone 65644 2.06 +nick young 65654 2.06 +nick young 65660 2.28 +nick zipper 65757 3.8 +nick zipper 65765 2.04 +oscar allen 65644 3.01 +oscar brown 65614 3.95 +oscar carson 65537 4.29 +oscar carson 65548 2.89 +oscar carson 65549 1.95 +oscar carson 65624 2.56 +oscar carson 65697 3.03 +oscar davidson 65556 0.6 +oscar ellison 65630 1.47 +oscar ellison 65630 1.47 +oscar falkner 65692 3.44 +oscar garcia 65751 3.71 +oscar hernandez 65683 3.32 +oscar hernandez 65707 2.25 +oscar ichabod 65536 1.8 +oscar ichabod 65562 1.18 +oscar ichabod 65637 1.91 +oscar ichabod 65763 1.96 +oscar johnson 65645 1.69 +oscar johnson 65778 1.59 +oscar king 65541 3.96 +oscar king 65550 4.31 +oscar king 65787 3.54 +oscar laertes 65625 2.26 +oscar laertes 65690 2.55 +oscar laertes 65756 2.85 +oscar laertes 65790 2.39 +oscar nixon 65596 3.12 +oscar ovid 65536 2.94 +oscar ovid 65615 2.95 +oscar ovid 65659 3.41 +oscar polk 65541 1.12 +oscar polk 65643 1.62 +oscar quirinius 65541 3.5 +oscar quirinius 65560 2.87 +oscar quirinius 65689 3.35 +oscar quirinius 65720 3.03 +oscar robinson 65537 0.29 +oscar robinson 65658 0.29 +oscar robinson 65687 1.5 +oscar robinson 65782 1.78 +oscar steinbeck 65709 4.96 +oscar thompson 65542 0.48 +oscar thompson 65698 2.07 +oscar thompson 65727 2.01 +oscar thompson 65738 1.8 +oscar underhill 65626 3.49 +oscar van buren 65581 2.33 +oscar van buren 65635 2.45 +oscar van buren 65705 2.68 +oscar white 65552 3.05 +oscar white 65564 2.58 +oscar white 65671 2.56 +oscar white 65735 2.47 +oscar xylophone 65773 1.51 +oscar xylophone 65773 1.51 +oscar xylophone 65775 1.82 +oscar zipper 65568 4.77 +oscar zipper 65740 3.81 +oscar zipper 65777 3.26 +priscilla brown 65670 2.91 +priscilla brown 65690 2.83 +priscilla brown 65749 2.07 +priscilla carson 65658 1.43 +priscilla carson 65687 2.97 +priscilla carson 65755 2.87 +priscilla ichabod 65627 4.95 +priscilla ichabod 65759 3.41 +priscilla johnson 65543 3.85 +priscilla johnson 65633 2.98 +priscilla johnson 65668 2.23 +priscilla johnson 65681 1.98 +priscilla johnson 65755 1.94 +priscilla king 65646 1.93 +priscilla nixon 65564 0.31 +priscilla nixon 65600 2.35 +priscilla ovid 65541 3.8 +priscilla ovid 65790 2.37 +priscilla polk 65747 3.1 +priscilla quirinius 65672 0.81 +priscilla thompson 65654 2.04 +priscilla underhill 65715 0.82 +priscilla underhill 65729 1.33 +priscilla van buren 65607 0.23 +priscilla van buren 65685 0.81 +priscilla van buren 65749 1.84 +priscilla white 65652 4.79 +priscilla xylophone 65538 3.56 +priscilla xylophone 65763 2.48 +priscilla xylophone 65774 1.84 +priscilla young 65585 2.92 +priscilla young 65658 3.77 +priscilla zipper 65622 4.62 +priscilla zipper 65726 2.67 +quinn allen 65657 3.02 +quinn allen 65708 3.35 +quinn brown 65691 4.37 +quinn brown 65700 3.28 +quinn brown 65733 3.27 +quinn davidson 65549 0.03 +quinn davidson 65714 1.02 +quinn davidson 65776 2.19 +quinn davidson 65779 2.66 +quinn ellison 65705 0.54 +quinn ellison 65778 2.74 +quinn garcia 65568 1.04 +quinn garcia 65604 0.79 +quinn garcia 65610 1.85 +quinn garcia 65773 1.67 +quinn ichabod 65564 0.65 +quinn king 65558 2.38 +quinn king 65649 1.53 +quinn laertes 65542 2.33 +quinn laertes 65560 2.03 +quinn laertes 65627 2.08 +quinn nixon 65659 0.72 +quinn ovid 65699 2.87 +quinn quirinius 65747 0.86 +quinn robinson 65627 4.14 +quinn steinbeck 65578 4.92 +quinn steinbeck 65763 3.18 +quinn thompson 65643 4.62 +quinn thompson 65774 2.43 +quinn underhill 65549 3.89 +quinn underhill 65694 2.39 +quinn underhill 65767 2.48 +quinn van buren 65725 0.53 +quinn young 65647 3.54 +quinn zipper 65579 1.83 +quinn zipper 65693 2.1 +rachel allen 65661 4.66 +rachel allen 65709 4.53 +rachel brown 65586 0.43 +rachel brown 65587 2.69 +rachel brown 65587 2.69 +rachel brown 65610 2.67 +rachel brown 65693 3.07 +rachel carson 65677 4.58 +rachel carson 65682 4.54 +rachel davidson 65755 2.28 +rachel ellison 65761 0.05 +rachel falkner 65616 1.2 +rachel falkner 65681 2.01 +rachel falkner 65693 2.56 +rachel falkner 65764 2.91 +rachel johnson 65658 3.02 +rachel king 65604 2.32 +rachel king 65643 2.4 +rachel laertes 65562 4.43 +rachel laertes 65624 2.8 +rachel ovid 65721 0.3 +rachel ovid 65736 0.92 +rachel polk 65686 2.56 +rachel quirinius 65787 2.95 +rachel robinson 65544 0.06 +rachel robinson 65717 1.55 +rachel robinson 65724 2.27 +rachel thompson 65648 2.49 +rachel thompson 65662 3.16 +rachel thompson 65733 2.51 +rachel underhill 65667 2.29 +rachel white 65615 1.99 +rachel white 65717 3.08 +rachel young 65727 1.75 +rachel zipper 65757 2.82 +rachel zipper 65785 3.62 +sarah carson 65679 1.04 +sarah carson 65693 0.85 +sarah carson 65694 2.06 +sarah ellison 65611 1.64 +sarah falkner 65606 0.77 +sarah falkner 65680 2.85 +sarah garcia 65563 4.89 +sarah garcia 65638 4.78 +sarah garcia 65661 3.63 +sarah ichabod 65667 3.94 +sarah ichabod 65671 2.33 +sarah johnson 65659 3.51 +sarah johnson 65716 4.21 +sarah johnson 65731 3.81 +sarah johnson 65751 3.37 +sarah king 65650 1.05 +sarah king 65699 0.99 +sarah miller 65557 0.2 +sarah ovid 65550 3.21 +sarah robinson 65677 4.9 +sarah robinson 65763 2.99 +sarah steinbeck 65721 2.82 +sarah white 65622 0.07 +sarah white 65747 2.29 +sarah xylophone 65678 0.15 +sarah young 65595 2.15 +sarah zipper 65550 2.22 +tom brown 65593 1.64 +tom brown 65675 2.83 +tom carson 65539 4.38 +tom carson 65624 4.28 +tom carson 65780 4.03 +tom davidson 65780 2.4 +tom ellison 65578 3.3 +tom ellison 65670 4.04 +tom ellison 65756 3.51 +tom falkner 65574 1.09 +tom falkner 65583 2.05 +tom hernandez 65575 2.35 +tom hernandez 65632 2.64 +tom ichabod 65588 1.48 +tom johnson 65536 4.68 +tom johnson 65789 4.6 +tom king 65576 2.87 +tom laertes 65617 1.51 +tom laertes 65701 1.93 +tom miller 65594 1.14 +tom miller 65603 1.25 +tom miller 65704 1.26 +tom nixon 65672 0.04 +tom ovid 65628 1.95 +tom polk 65652 2.54 +tom polk 65742 2.26 +tom quirinius 65563 4.51 +tom quirinius 65783 4.17 +tom robinson 65626 3.12 +tom robinson 65632 3.61 +tom robinson 65691 3.13 +tom robinson 65758 3.45 +tom steinbeck 65666 1.34 +tom van buren 65621 0.66 +tom van buren 65652 2.71 +tom van buren 65669 3.47 +tom white 65548 2.13 +tom young 65544 3.73 +tom young 65546 2.02 +tom zipper 65789 1.51 +ulysses brown 65735 2.92 +ulysses carson 65602 0.34 +ulysses carson 65643 2.11 +ulysses carson 65703 1.43 +ulysses carson 65716 1.41 +ulysses davidson 65750 3.04 +ulysses ellison 65575 4.39 +ulysses garcia 65666 4.2 +ulysses hernandez 65651 1.75 +ulysses hernandez 65702 2.42 +ulysses hernandez 65786 2.53 +ulysses ichabod 65551 0.33 +ulysses ichabod 65566 2.19 +ulysses johnson 65776 4.79 +ulysses king 65649 4.46 +ulysses laertes 65691 4.55 +ulysses laertes 65711 3.54 +ulysses laertes 65781 3.66 +ulysses miller 65610 0.24 +ulysses miller 65637 1.08 +ulysses nixon 65603 1.85 +ulysses ovid 65656 3.17 +ulysses polk 65563 1.32 +ulysses polk 65580 3.05 +ulysses polk 65612 3.46 +ulysses polk 65777 3.75 +ulysses quirinius 65786 2.13 +ulysses robinson 65744 1.97 +ulysses steinbeck 65611 2.74 +ulysses steinbeck 65680 2.64 +ulysses thompson 65788 1.51 +ulysses underhill 65570 0.38 +ulysses underhill 65616 0.8 +ulysses underhill 65620 2.09 +ulysses underhill 65623 2.69 +ulysses underhill 65641 2.54 +ulysses underhill 65713 2.9 +ulysses underhill 65785 2.97 +ulysses van buren 65684 1.42 +ulysses white 65654 0.14 +ulysses white 65675 1.51 +ulysses xylophone 65623 2.3 +ulysses xylophone 65636 2.69 +ulysses xylophone 65781 3.22 +ulysses young 65675 1.34 +ulysses young 65736 2.01 +ulysses young 65748 2.24 +victor allen 65684 0.83 +victor allen 65707 2.31 +victor brown 65550 4.57 +victor brown 65555 3.54 +victor brown 65622 2.61 +victor brown 65673 2.34 +victor davidson 65579 0.61 +victor davidson 65628 1.52 +victor davidson 65783 2.25 +victor ellison 65641 1.32 +victor ellison 65782 2.59 +victor hernandez 65571 3.62 +victor hernandez 65659 3.68 +victor hernandez 65708 3.35 +victor hernandez 65735 2.88 +victor hernandez 65775 2.62 +victor johnson 65606 3.03 +victor johnson 65607 2.3 +victor johnson 65607 2.3 +victor king 65721 4.09 +victor king 65743 2.45 +victor laertes 65638 1.46 +victor laertes 65644 2.38 +victor miller 65570 0.1 +victor nixon 65709 0.74 +victor nixon 65791 1.73 +victor ovid 65649 4.93 +victor polk 65625 1.04 +victor quirinius 65620 1.32 +victor quirinius 65651 3.15 +victor robinson 65596 0.92 +victor robinson 65673 1.76 +victor steinbeck 65618 2.87 +victor steinbeck 65661 2.19 +victor steinbeck 65686 2.81 +victor thompson 65548 1.59 +victor van buren 65664 4.44 +victor van buren 65774 4.06 +victor white 65548 4.67 +victor white 65601 3.87 +victor xylophone 65549 3.8 +victor xylophone 65618 2.13 +victor xylophone 65644 1.59 +victor xylophone 65677 1.89 +victor xylophone 65755 2.27 +victor young 65628 3.16 +victor zipper 65743 3.98 +wendy allen 65628 3.8 +wendy allen 65711 3.44 +wendy allen 65782 2.4 +wendy brown 65580 4.67 +wendy brown 65657 4.68 +wendy ellison 65545 1.51 +wendy ellison 65603 1.6 +wendy falkner 65595 0.58 +wendy falkner 65604 0.82 +wendy falkner 65635 1.59 +wendy garcia 65659 3.47 +wendy garcia 65746 2.35 +wendy garcia 65747 1.67 +wendy garcia 65777 1.32 +wendy hernandez 65650 2.26 +wendy ichabod 65730 0.44 +wendy king 65586 4.46 +wendy king 65664 4.25 +wendy king 65670 2.94 +wendy laertes 65566 3.13 +wendy laertes 65683 3.99 +wendy laertes 65727 3.57 +wendy miller 65582 1.53 +wendy miller 65626 1.4 +wendy nixon 65611 0.26 +wendy nixon 65746 2.27 +wendy ovid 65589 4.75 +wendy ovid 65643 3.42 +wendy polk 65656 0.62 +wendy polk 65692 1.36 +wendy quirinius 65766 1.35 +wendy quirinius 65767 2.76 +wendy robinson 65622 0.85 +wendy robinson 65715 2.13 +wendy robinson 65774 1.8 +wendy steinbeck 65612 0.07 +wendy thompson 65650 2.27 +wendy thompson 65737 3.2 +wendy underhill 65662 4.55 +wendy underhill 65758 2.84 +wendy underhill 65775 2.54 +wendy van buren 65680 1.1 +wendy van buren 65699 1.0 +wendy white 65705 0.5 +wendy xylophone 65687 0.46 +wendy xylophone 65773 1.39 +wendy young 65674 0.48 +wendy young 65685 1.31 +xavier allen 65611 1.53 +xavier allen 65618 2.07 +xavier allen 65771 2.33 +xavier brown 65600 0.89 +xavier brown 65704 0.58 +xavier brown 65723 1.55 +xavier carson 65731 4.42 +xavier carson 65758 3.91 +xavier davidson 65644 1.84 +xavier davidson 65664 3.4 +xavier davidson 65755 2.67 +xavier ellison 65541 1.47 +xavier ellison 65654 2.49 +xavier garcia 65672 2.76 +xavier hernandez 65541 0.96 +xavier hernandez 65544 1.47 +xavier hernandez 65766 1.28 +xavier ichabod 65597 4.76 +xavier ichabod 65663 2.99 +xavier johnson 65654 1.34 +xavier johnson 65744 3.06 +xavier king 65590 2.68 +xavier king 65601 1.4 +xavier laertes 65743 0.75 +xavier ovid 65788 2.06 +xavier polk 65587 0.99 +xavier polk 65653 1.15 +xavier polk 65675 1.9 +xavier polk 65696 1.93 +xavier quirinius 65599 1.66 +xavier quirinius 65650 1.94 +xavier quirinius 65656 2.46 +xavier quirinius 65737 1.92 +xavier thompson 65608 1.65 +xavier underhill 65710 0.13 +xavier white 65703 0.98 +xavier white 65732 2.22 +xavier xylophone 65572 1.0 +xavier zipper 65561 0.94 +yuri allen 65565 2.03 +yuri allen 65682 1.14 +yuri brown 65538 2.73 +yuri brown 65688 2.02 +yuri carson 65670 3.06 +yuri carson 65769 3.43 +yuri ellison 65570 1.05 +yuri ellison 65581 1.68 +yuri falkner 65658 2.85 +yuri falkner 65681 2.14 +yuri garcia 65639 3.41 +yuri hernandez 65706 1.64 +yuri johnson 65587 1.27 +yuri johnson 65697 1.44 +yuri johnson 65712 2.29 +yuri king 65721 0.33 +yuri laertes 65637 4.3 +yuri laertes 65773 2.15 +yuri nixon 65635 4.02 +yuri nixon 65740 4.18 +yuri polk 65607 0.08 +yuri polk 65713 0.37 +yuri polk 65742 1.25 +yuri quirinius 65544 2.58 +yuri quirinius 65617 2.1 +yuri quirinius 65695 1.91 +yuri steinbeck 65592 4.89 +yuri steinbeck 65679 3.24 +yuri thompson 65676 2.67 +yuri underhill 65718 2.86 +yuri underhill 65750 2.51 +yuri white 65659 4.59 +yuri xylophone 65714 2.53 +zach allen 65667 0.88 +zach brown 65559 4.88 +zach brown 65588 4.53 +zach brown 65691 3.49 +zach brown 65759 3.4 +zach brown 65762 3.55 +zach carson 65572 2.03 +zach ellison 65748 1.76 +zach falkner 65620 0.34 +zach falkner 65627 0.25 +zach garcia 65544 0.99 +zach garcia 65623 2.84 +zach garcia 65629 3.01 +zach garcia 65786 2.55 +zach ichabod 65599 3.36 +zach ichabod 65612 1.92 +zach king 65556 2.36 +zach king 65702 1.52 +zach king 65773 2.58 +zach miller 65584 1.6 +zach miller 65665 0.99 +zach miller 65719 1.55 +zach ovid 65578 1.51 +zach ovid 65703 1.92 +zach ovid 65750 2.63 +zach ovid 65784 2.72 +zach quirinius 65691 2.95 +zach robinson 65599 2.87 +zach steinbeck 65602 2.45 +zach steinbeck 65695 1.86 +zach thompson 65636 0.25 +zach thompson 65696 0.51 +zach underhill 65573 3.97 +zach white 65733 2.31 +zach xylophone 65542 1.69 +zach xylophone 65780 0.88 +zach young 65576 1.82 +zach zipper 65579 4.5 +zach zipper 65649 4.02 +zach zipper 65676 3.12 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-8-45a1d7c2aba45d761e19ff4dfdf5463e b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-8-45a1d7c2aba45d761e19ff4dfdf5463e new file mode 100644 index 0000000000000..84b934fad85b1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-8-45a1d7c2aba45d761e19ff4dfdf5463e @@ -0,0 +1,1049 @@ + 65560 20.0 + 65718 20.0 + 65740 20.0 +alice allen 65662 20.0 +alice allen 65720 20.0 +alice allen 65758 20.0 +alice brown 65696 20.0 +alice carson 65559 20.0 +alice davidson 65547 20.0 +alice falkner 65669 20.0 +alice garcia 65613 20.0 +alice hernandez 65737 20.0 +alice hernandez 65784 20.0 +alice johnson 65739 20.0 +alice king 65660 20.0 +alice king 65734 20.0 +alice king 65738 20.0 +alice laertes 65669 20.0 +alice laertes 65671 20.0 +alice miller 65590 20.0 +alice nixon 65586 20.0 +alice nixon 65595 20.0 +alice nixon 65604 20.0 +alice ovid 65737 20.0 +alice polk 65548 20.0 +alice quirinius 65636 20.0 +alice quirinius 65728 20.0 +alice robinson 65606 20.0 +alice robinson 65789 20.0 +alice steinbeck 65578 20.0 +alice steinbeck 65673 20.0 +alice steinbeck 65786 20.0 +alice underhill 65750 20.0 +alice van buren 65562 20.0 +alice xylophone 65578 20.0 +alice xylophone 65585 20.0 +alice xylophone 65599 20.0 +alice zipper 65553 20.0 +alice zipper 65662 20.0 +alice zipper 65766 20.0 +bob brown 65584 20.0 +bob brown 65777 20.0 +bob brown 65783 20.0 +bob carson 65713 20.0 +bob davidson 65664 20.0 +bob davidson 65693 20.0 +bob davidson 65768 20.0 +bob ellison 65591 20.0 +bob ellison 65624 20.0 +bob ellison 65721 20.0 +bob ellison 65760 20.0 +bob falkner 65789 20.0 +bob garcia 65585 20.0 +bob garcia 65598 20.0 +bob garcia 65673 20.0 +bob garcia 65754 20.0 +bob garcia 65782 20.0 +bob hernandez 65557 20.0 +bob ichabod 65549 20.0 +bob king 65715 20.0 +bob king 65757 20.0 +bob king 65783 20.0 +bob laertes 65602 20.0 +bob laertes 65663 20.0 +bob miller 65608 20.0 +bob ovid 65564 20.0 +bob ovid 65619 20.0 +bob ovid 65686 20.0 +bob ovid 65726 20.0 +bob polk 65594 20.0 +bob quirinius 65700 20.0 +bob steinbeck 65637 20.0 +bob van buren 65778 20.0 +bob white 65543 20.0 +bob white 65605 20.0 +bob xylophone 65574 20.0 +bob xylophone 65666 20.0 +bob young 65556 20.0 +bob zipper 65559 20.0 +bob zipper 65633 20.0 +bob zipper 65739 20.0 +calvin allen 65669 20.0 +calvin brown 65537 20.0 +calvin brown 65580 20.0 +calvin brown 65677 20.0 +calvin carson 65637 20.0 +calvin davidson 65541 20.0 +calvin davidson 65564 20.0 +calvin ellison 65667 20.0 +calvin falkner 65573 20.0 +calvin falkner 65596 20.0 +calvin falkner 65738 20.0 +calvin falkner 65762 20.0 +calvin falkner 65778 20.0 +calvin falkner 65784 20.0 +calvin garcia 65664 20.0 +calvin hernandez 65578 20.0 +calvin johnson 65731 20.0 +calvin laertes 65570 20.0 +calvin laertes 65684 20.0 +calvin nixon 65654 20.0 +calvin nixon 65724 20.0 +calvin nixon 65749 20.0 +calvin ovid 65554 20.0 +calvin ovid 65643 20.0 +calvin ovid 65663 20.0 +calvin ovid 65715 20.0 +calvin polk 65731 20.0 +calvin quirinius 65741 20.0 +calvin quirinius 65769 20.0 +calvin robinson 65581 20.0 +calvin steinbeck 65680 20.0 +calvin steinbeck 65762 20.0 +calvin steinbeck 65779 20.0 +calvin thompson 65560 20.0 +calvin thompson 65640 20.0 +calvin underhill 65732 20.0 +calvin van buren 65552 20.0 +calvin van buren 65771 20.0 +calvin white 65553 20.0 +calvin white 65561 20.0 +calvin xylophone 65575 20.0 +calvin xylophone 65596 20.0 +calvin xylophone 65713 20.0 +calvin young 65574 20.0 +calvin young 65746 20.0 +calvin zipper 65669 20.0 +calvin zipper 65739 20.0 +david allen 65588 20.0 +david allen 65617 20.0 +david brown 65637 20.0 +david brown 65760 20.0 +david davidson 65559 20.0 +david davidson 65756 20.0 +david davidson 65778 20.0 +david davidson 65779 20.0 +david ellison 65634 20.0 +david ellison 65724 20.0 +david ellison 65724 20.0 +david hernandez 65763 20.0 +david ichabod 65699 20.0 +david ichabod 65715 20.0 +david laertes 65762 20.0 +david nixon 65536 20.0 +david ovid 65623 20.0 +david ovid 65628 20.0 +david quirinius 65697 20.0 +david quirinius 65759 20.0 +david quirinius 65779 20.0 +david robinson 65762 20.0 +david robinson 65775 20.0 +david thompson 65550 20.0 +david underhill 65602 20.0 +david underhill 65662 20.0 +david underhill 65751 20.0 +david van buren 65625 20.0 +david van buren 65634 20.0 +david white 65678 20.0 +david xylophone 65537 20.0 +david xylophone 65670 20.0 +david xylophone 65764 20.0 +david young 65551 20.0 +david young 65694 20.0 +ethan allen 65747 20.0 +ethan brown 65539 20.0 +ethan brown 65617 20.0 +ethan brown 65685 20.0 +ethan brown 65685 20.0 +ethan brown 65722 20.0 +ethan brown 65733 20.0 +ethan carson 65742 20.0 +ethan ellison 65714 20.0 +ethan ellison 65732 20.0 +ethan falkner 65577 20.0 +ethan falkner 65614 20.0 +ethan garcia 65736 20.0 +ethan hernandez 65618 20.0 +ethan johnson 65536 20.0 +ethan king 65614 20.0 +ethan laertes 65562 20.0 +ethan laertes 65597 20.0 +ethan laertes 65628 20.0 +ethan laertes 65643 20.0 +ethan laertes 65680 20.0 +ethan laertes 65745 20.0 +ethan laertes 65760 20.0 +ethan miller 65712 20.0 +ethan nixon 65766 20.0 +ethan ovid 65697 20.0 +ethan polk 65589 20.0 +ethan polk 65615 20.0 +ethan polk 65622 20.0 +ethan polk 65622 20.0 +ethan quirinius 65542 20.0 +ethan quirinius 65591 20.0 +ethan quirinius 65706 20.0 +ethan robinson 65547 20.0 +ethan robinson 65659 20.0 +ethan underhill 65570 20.0 +ethan van buren 65572 20.0 +ethan white 65677 20.0 +ethan white 65788 20.0 +ethan xylophone 65595 20.0 +ethan zipper 65593 20.0 +ethan zipper 65680 20.0 +fred davidson 65552 20.0 +fred davidson 65595 20.0 +fred davidson 65721 20.0 +fred ellison 65548 20.0 +fred ellison 65691 20.0 +fred ellison 65771 20.0 +fred falkner 65637 20.0 +fred falkner 65648 20.0 +fred falkner 65783 20.0 +fred hernandez 65541 20.0 +fred ichabod 65572 20.0 +fred ichabod 65789 20.0 +fred johnson 65758 20.0 +fred king 65694 20.0 +fred king 65745 20.0 +fred laertes 65769 20.0 +fred miller 65536 20.0 +fred nixon 65560 20.0 +fred nixon 65612 20.0 +fred nixon 65703 20.0 +fred nixon 65705 20.0 +fred polk 65603 20.0 +fred polk 65656 20.0 +fred polk 65701 20.0 +fred polk 65706 20.0 +fred quirinius 65697 20.0 +fred quirinius 65782 20.0 +fred robinson 65623 20.0 +fred steinbeck 65544 20.0 +fred steinbeck 65651 20.0 +fred steinbeck 65755 20.0 +fred underhill 65629 20.0 +fred van buren 65537 20.0 +fred van buren 65561 20.0 +fred van buren 65745 20.0 +fred van buren 65789 20.0 +fred white 65589 20.0 +fred young 65594 20.0 +fred young 65773 20.0 +fred zipper 65553 20.0 +gabriella allen 65646 20.0 +gabriella allen 65677 20.0 +gabriella brown 65704 20.0 +gabriella brown 65753 20.0 +gabriella carson 65586 20.0 +gabriella davidson 65565 20.0 +gabriella ellison 65706 20.0 +gabriella ellison 65716 20.0 +gabriella falkner 65623 20.0 +gabriella falkner 65711 20.0 +gabriella falkner 65767 20.0 +gabriella garcia 65571 20.0 +gabriella hernandez 65587 20.0 +gabriella hernandez 65717 20.0 +gabriella ichabod 65559 20.0 +gabriella ichabod 65633 20.0 +gabriella ichabod 65702 20.0 +gabriella ichabod 65712 20.0 +gabriella ichabod 65717 20.0 +gabriella king 65651 20.0 +gabriella king 65657 20.0 +gabriella laertes 65781 20.0 +gabriella miller 65646 20.0 +gabriella ovid 65556 20.0 +gabriella ovid 65583 20.0 +gabriella polk 65701 20.0 +gabriella polk 65790 20.0 +gabriella steinbeck 65582 20.0 +gabriella steinbeck 65653 20.0 +gabriella thompson 65682 20.0 +gabriella thompson 65755 20.0 +gabriella thompson 65766 20.0 +gabriella van buren 65581 20.0 +gabriella van buren 65644 20.0 +gabriella white 65638 20.0 +gabriella young 65699 20.0 +gabriella young 65774 20.0 +gabriella zipper 65540 20.0 +gabriella zipper 65754 20.0 +holly allen 65596 20.0 +holly brown 65599 20.0 +holly brown 65619 20.0 +holly falkner 65720 20.0 +holly hernandez 65602 20.0 +holly hernandez 65686 20.0 +holly hernandez 65750 20.0 +holly hernandez 65788 20.0 +holly ichabod 65711 20.0 +holly ichabod 65749 20.0 +holly ichabod 65752 20.0 +holly johnson 65655 20.0 +holly johnson 65662 20.0 +holly johnson 65755 20.0 +holly king 65549 20.0 +holly king 65648 20.0 +holly laertes 65664 20.0 +holly miller 65653 20.0 +holly nixon 65539 20.0 +holly nixon 65658 20.0 +holly polk 65743 20.0 +holly polk 65751 20.0 +holly robinson 65564 20.0 +holly thompson 65538 20.0 +holly thompson 65578 20.0 +holly thompson 65713 20.0 +holly underhill 65634 20.0 +holly underhill 65654 20.0 +holly underhill 65721 20.0 +holly underhill 65759 20.0 +holly van buren 65727 20.0 +holly white 65536 20.0 +holly white 65602 20.0 +holly xylophone 65544 20.0 +holly young 65606 20.0 +holly young 65765 20.0 +holly zipper 65607 20.0 +holly zipper 65755 20.0 +irene allen 65556 20.0 +irene brown 65633 20.0 +irene brown 65650 20.0 +irene brown 65765 20.0 +irene carson 65590 20.0 +irene ellison 65659 20.0 +irene ellison 65696 20.0 +irene falkner 65620 20.0 +irene falkner 65661 20.0 +irene garcia 65660 20.0 +irene garcia 65711 20.0 +irene garcia 65787 20.0 +irene ichabod 65645 20.0 +irene ichabod 65722 20.0 +irene johnson 65583 20.0 +irene laertes 65664 20.0 +irene laertes 65710 20.0 +irene laertes 65722 20.0 +irene miller 65730 20.0 +irene nixon 65631 20.0 +irene nixon 65643 20.0 +irene nixon 65653 20.0 +irene ovid 65691 20.0 +irene ovid 65734 20.0 +irene ovid 65753 20.0 +irene polk 65551 20.0 +irene polk 65575 20.0 +irene polk 65579 20.0 +irene polk 65595 20.0 +irene polk 65610 20.0 +irene quirinius 65724 20.0 +irene quirinius 65769 20.0 +irene quirinius 65773 20.0 +irene robinson 65554 20.0 +irene steinbeck 65683 20.0 +irene thompson 65688 20.0 +irene underhill 65591 20.0 +irene underhill 65787 20.0 +irene van buren 65579 20.0 +irene van buren 65589 20.0 +irene xylophone 65775 20.0 +jessica brown 65588 20.0 +jessica carson 65553 20.0 +jessica carson 65672 20.0 +jessica carson 65747 20.0 +jessica davidson 65549 20.0 +jessica davidson 65606 20.0 +jessica davidson 65675 20.0 +jessica davidson 65727 20.0 +jessica ellison 65567 20.0 +jessica ellison 65663 20.0 +jessica falkner 65584 20.0 +jessica garcia 65676 20.0 +jessica garcia 65789 20.0 +jessica ichabod 65704 20.0 +jessica johnson 65607 20.0 +jessica johnson 65720 20.0 +jessica miller 65733 20.0 +jessica nixon 65590 20.0 +jessica nixon 65774 20.0 +jessica ovid 65582 20.0 +jessica ovid 65751 20.0 +jessica polk 65637 20.0 +jessica quirinius 65562 20.0 +jessica quirinius 65608 20.0 +jessica quirinius 65712 20.0 +jessica quirinius 65716 20.0 +jessica robinson 65576 20.0 +jessica thompson 65581 20.0 +jessica thompson 65675 20.0 +jessica underhill 65656 20.0 +jessica underhill 65702 20.0 +jessica underhill 65783 20.0 +jessica van buren 65615 20.0 +jessica white 65544 20.0 +jessica white 65570 20.0 +jessica white 65594 20.0 +jessica white 65673 20.0 +jessica white 65779 20.0 +jessica xylophone 65562 20.0 +jessica young 65623 20.0 +jessica young 65711 20.0 +jessica zipper 65600 20.0 +jessica zipper 65657 20.0 +jessica zipper 65778 20.0 +katie allen 65542 20.0 +katie brown 65590 20.0 +katie davidson 65619 20.0 +katie ellison 65675 20.0 +katie ellison 65699 20.0 +katie falkner 65728 20.0 +katie garcia 65625 20.0 +katie garcia 65747 20.0 +katie hernandez 65550 20.0 +katie ichabod 65658 20.0 +katie ichabod 65726 20.0 +katie ichabod 65757 20.0 +katie king 65629 20.0 +katie king 65647 20.0 +katie king 65776 20.0 +katie miller 65541 20.0 +katie miller 65661 20.0 +katie nixon 65669 20.0 +katie ovid 65681 20.0 +katie polk 65746 20.0 +katie polk 65784 20.0 +katie robinson 65697 20.0 +katie van buren 65643 20.0 +katie van buren 65730 20.0 +katie white 65620 20.0 +katie white 65719 20.0 +katie xylophone 65585 20.0 +katie young 65644 20.0 +katie young 65746 20.0 +katie young 65764 20.0 +katie zipper 65568 20.0 +katie zipper 65733 20.0 +luke allen 65547 20.0 +luke allen 65552 20.0 +luke allen 65576 20.0 +luke allen 65681 20.0 +luke allen 65776 20.0 +luke brown 65719 20.0 +luke davidson 65656 20.0 +luke davidson 65791 20.0 +luke ellison 65582 20.0 +luke ellison 65664 20.0 +luke ellison 65779 20.0 +luke falkner 65589 20.0 +luke falkner 65618 20.0 +luke garcia 65687 20.0 +luke garcia 65778 20.0 +luke ichabod 65629 20.0 +luke ichabod 65654 20.0 +luke johnson 65545 20.0 +luke johnson 65716 20.0 +luke johnson 65718 20.0 +luke laertes 65608 20.0 +luke laertes 65657 20.0 +luke laertes 65685 20.0 +luke laertes 65730 20.0 +luke laertes 65756 20.0 +luke miller 65752 20.0 +luke ovid 65569 20.0 +luke ovid 65693 20.0 +luke polk 65645 20.0 +luke polk 65658 20.0 +luke quirinius 65655 20.0 +luke robinson 65634 20.0 +luke robinson 65772 20.0 +luke thompson 65626 20.0 +luke underhill 65553 20.0 +luke underhill 65571 20.0 +luke underhill 65651 20.0 +luke van buren 65678 20.0 +luke white 65693 20.0 +luke xylophone 65597 20.0 +luke zipper 65641 20.0 +mike allen 65706 20.0 +mike brown 65654 20.0 +mike carson 65698 20.0 +mike carson 65700 20.0 +mike carson 65751 20.0 +mike davidson 65658 20.0 +mike davidson 65759 20.0 +mike ellison 65598 20.0 +mike ellison 65606 20.0 +mike ellison 65718 20.0 +mike ellison 65738 20.0 +mike ellison 65760 20.0 +mike falkner 65609 20.0 +mike garcia 65571 20.0 +mike garcia 65600 20.0 +mike garcia 65770 20.0 +mike hernandez 65548 20.0 +mike hernandez 65672 20.0 +mike ichabod 65621 20.0 +mike king 65563 20.0 +mike king 65586 20.0 +mike king 65591 20.0 +mike king 65642 20.0 +mike king 65769 20.0 +mike king 65776 20.0 +mike miller 65549 20.0 +mike nixon 65619 20.0 +mike nixon 65704 20.0 +mike polk 65619 20.0 +mike polk 65658 20.0 +mike polk 65704 20.0 +mike quirinius 65717 20.0 +mike steinbeck 65550 20.0 +mike steinbeck 65564 20.0 +mike steinbeck 65573 20.0 +mike steinbeck 65749 20.0 +mike van buren 65620 20.0 +mike van buren 65770 20.0 +mike white 65648 20.0 +mike white 65685 20.0 +mike white 65769 20.0 +mike white 65778 20.0 +mike young 65545 20.0 +mike young 65581 20.0 +mike young 65736 20.0 +mike zipper 65552 20.0 +mike zipper 65695 20.0 +mike zipper 65779 20.0 +nick allen 65641 20.0 +nick allen 65786 20.0 +nick brown 65724 20.0 +nick davidson 65601 20.0 +nick ellison 65691 20.0 +nick ellison 65745 20.0 +nick falkner 65583 20.0 +nick falkner 65676 20.0 +nick garcia 65695 20.0 +nick garcia 65712 20.0 +nick garcia 65720 20.0 +nick ichabod 65572 20.0 +nick ichabod 65681 20.0 +nick ichabod 65737 20.0 +nick johnson 65585 20.0 +nick johnson 65784 20.0 +nick laertes 65624 20.0 +nick miller 65757 20.0 +nick nixon 65650 20.0 +nick ovid 65719 20.0 +nick polk 65716 20.0 +nick quirinius 65588 20.0 +nick quirinius 65723 20.0 +nick robinson 65547 20.0 +nick robinson 65675 20.0 +nick steinbeck 65689 20.0 +nick thompson 65610 20.0 +nick underhill 65619 20.0 +nick van buren 65603 20.0 +nick xylophone 65644 20.0 +nick young 65654 20.0 +nick young 65660 20.0 +nick zipper 65757 20.0 +nick zipper 65765 20.0 +oscar allen 65644 20.0 +oscar brown 65614 20.0 +oscar carson 65537 20.0 +oscar carson 65548 20.0 +oscar carson 65549 20.0 +oscar carson 65624 20.0 +oscar carson 65697 20.0 +oscar davidson 65556 20.0 +oscar ellison 65630 20.0 +oscar ellison 65630 20.0 +oscar falkner 65692 20.0 +oscar garcia 65751 20.0 +oscar hernandez 65683 20.0 +oscar hernandez 65707 20.0 +oscar ichabod 65536 20.0 +oscar ichabod 65562 20.0 +oscar ichabod 65637 20.0 +oscar ichabod 65763 20.0 +oscar johnson 65645 20.0 +oscar johnson 65778 20.0 +oscar king 65541 20.0 +oscar king 65550 20.0 +oscar king 65787 20.0 +oscar laertes 65625 20.0 +oscar laertes 65690 20.0 +oscar laertes 65756 20.0 +oscar laertes 65790 20.0 +oscar nixon 65596 20.0 +oscar ovid 65536 20.0 +oscar ovid 65615 20.0 +oscar ovid 65659 20.0 +oscar polk 65541 20.0 +oscar polk 65643 20.0 +oscar quirinius 65541 20.0 +oscar quirinius 65560 20.0 +oscar quirinius 65689 20.0 +oscar quirinius 65720 20.0 +oscar robinson 65537 20.0 +oscar robinson 65658 20.0 +oscar robinson 65687 20.0 +oscar robinson 65782 20.0 +oscar steinbeck 65709 20.0 +oscar thompson 65542 20.0 +oscar thompson 65698 20.0 +oscar thompson 65727 20.0 +oscar thompson 65738 20.0 +oscar underhill 65626 20.0 +oscar van buren 65581 20.0 +oscar van buren 65635 20.0 +oscar van buren 65705 20.0 +oscar white 65552 20.0 +oscar white 65564 20.0 +oscar white 65671 20.0 +oscar white 65735 20.0 +oscar xylophone 65773 20.0 +oscar xylophone 65773 20.0 +oscar xylophone 65775 20.0 +oscar zipper 65568 20.0 +oscar zipper 65740 20.0 +oscar zipper 65777 20.0 +priscilla brown 65670 20.0 +priscilla brown 65690 20.0 +priscilla brown 65749 20.0 +priscilla carson 65658 20.0 +priscilla carson 65687 20.0 +priscilla carson 65755 20.0 +priscilla ichabod 65627 20.0 +priscilla ichabod 65759 20.0 +priscilla johnson 65543 20.0 +priscilla johnson 65633 20.0 +priscilla johnson 65668 20.0 +priscilla johnson 65681 20.0 +priscilla johnson 65755 20.0 +priscilla king 65646 20.0 +priscilla nixon 65564 20.0 +priscilla nixon 65600 20.0 +priscilla ovid 65541 20.0 +priscilla ovid 65790 20.0 +priscilla polk 65747 20.0 +priscilla quirinius 65672 20.0 +priscilla thompson 65654 20.0 +priscilla underhill 65715 20.0 +priscilla underhill 65729 20.0 +priscilla van buren 65607 20.0 +priscilla van buren 65685 20.0 +priscilla van buren 65749 20.0 +priscilla white 65652 20.0 +priscilla xylophone 65538 20.0 +priscilla xylophone 65763 20.0 +priscilla xylophone 65774 20.0 +priscilla young 65585 20.0 +priscilla young 65658 20.0 +priscilla zipper 65622 20.0 +priscilla zipper 65726 20.0 +quinn allen 65657 20.0 +quinn allen 65708 20.0 +quinn brown 65691 20.0 +quinn brown 65700 20.0 +quinn brown 65733 20.0 +quinn davidson 65549 20.0 +quinn davidson 65714 20.0 +quinn davidson 65776 20.0 +quinn davidson 65779 20.0 +quinn ellison 65705 20.0 +quinn ellison 65778 20.0 +quinn garcia 65568 20.0 +quinn garcia 65604 20.0 +quinn garcia 65610 20.0 +quinn garcia 65773 20.0 +quinn ichabod 65564 20.0 +quinn king 65558 20.0 +quinn king 65649 20.0 +quinn laertes 65542 20.0 +quinn laertes 65560 20.0 +quinn laertes 65627 20.0 +quinn nixon 65659 20.0 +quinn ovid 65699 20.0 +quinn quirinius 65747 20.0 +quinn robinson 65627 20.0 +quinn steinbeck 65578 20.0 +quinn steinbeck 65763 20.0 +quinn thompson 65643 20.0 +quinn thompson 65774 20.0 +quinn underhill 65549 20.0 +quinn underhill 65694 20.0 +quinn underhill 65767 20.0 +quinn van buren 65725 20.0 +quinn young 65647 20.0 +quinn zipper 65579 20.0 +quinn zipper 65693 20.0 +rachel allen 65661 20.0 +rachel allen 65709 20.0 +rachel brown 65586 20.0 +rachel brown 65587 20.0 +rachel brown 65587 20.0 +rachel brown 65610 20.0 +rachel brown 65693 20.0 +rachel carson 65677 20.0 +rachel carson 65682 20.0 +rachel davidson 65755 20.0 +rachel ellison 65761 20.0 +rachel falkner 65616 20.0 +rachel falkner 65681 20.0 +rachel falkner 65693 20.0 +rachel falkner 65764 20.0 +rachel johnson 65658 20.0 +rachel king 65604 20.0 +rachel king 65643 20.0 +rachel laertes 65562 20.0 +rachel laertes 65624 20.0 +rachel ovid 65721 20.0 +rachel ovid 65736 20.0 +rachel polk 65686 20.0 +rachel quirinius 65787 20.0 +rachel robinson 65544 20.0 +rachel robinson 65717 20.0 +rachel robinson 65724 20.0 +rachel thompson 65648 20.0 +rachel thompson 65662 20.0 +rachel thompson 65733 20.0 +rachel underhill 65667 20.0 +rachel white 65615 20.0 +rachel white 65717 20.0 +rachel young 65727 20.0 +rachel zipper 65757 20.0 +rachel zipper 65785 20.0 +sarah carson 65679 20.0 +sarah carson 65693 20.0 +sarah carson 65694 20.0 +sarah ellison 65611 20.0 +sarah falkner 65606 20.0 +sarah falkner 65680 20.0 +sarah garcia 65563 20.0 +sarah garcia 65638 20.0 +sarah garcia 65661 20.0 +sarah ichabod 65667 20.0 +sarah ichabod 65671 20.0 +sarah johnson 65659 20.0 +sarah johnson 65716 20.0 +sarah johnson 65731 20.0 +sarah johnson 65751 20.0 +sarah king 65650 20.0 +sarah king 65699 20.0 +sarah miller 65557 20.0 +sarah ovid 65550 20.0 +sarah robinson 65677 20.0 +sarah robinson 65763 20.0 +sarah steinbeck 65721 20.0 +sarah white 65622 20.0 +sarah white 65747 20.0 +sarah xylophone 65678 20.0 +sarah young 65595 20.0 +sarah zipper 65550 20.0 +tom brown 65593 20.0 +tom brown 65675 20.0 +tom carson 65539 20.0 +tom carson 65624 20.0 +tom carson 65780 20.0 +tom davidson 65780 20.0 +tom ellison 65578 20.0 +tom ellison 65670 20.0 +tom ellison 65756 20.0 +tom falkner 65574 20.0 +tom falkner 65583 20.0 +tom hernandez 65575 20.0 +tom hernandez 65632 20.0 +tom ichabod 65588 20.0 +tom johnson 65536 20.0 +tom johnson 65789 20.0 +tom king 65576 20.0 +tom laertes 65617 20.0 +tom laertes 65701 20.0 +tom miller 65594 20.0 +tom miller 65603 20.0 +tom miller 65704 20.0 +tom nixon 65672 20.0 +tom ovid 65628 20.0 +tom polk 65652 20.0 +tom polk 65742 20.0 +tom quirinius 65563 20.0 +tom quirinius 65783 20.0 +tom robinson 65626 20.0 +tom robinson 65632 20.0 +tom robinson 65691 20.0 +tom robinson 65758 20.0 +tom steinbeck 65666 20.0 +tom van buren 65621 20.0 +tom van buren 65652 20.0 +tom van buren 65669 20.0 +tom white 65548 20.0 +tom young 65544 20.0 +tom young 65546 20.0 +tom zipper 65789 20.0 +ulysses brown 65735 20.0 +ulysses carson 65602 20.0 +ulysses carson 65643 20.0 +ulysses carson 65703 20.0 +ulysses carson 65716 20.0 +ulysses davidson 65750 20.0 +ulysses ellison 65575 20.0 +ulysses garcia 65666 20.0 +ulysses hernandez 65651 20.0 +ulysses hernandez 65702 20.0 +ulysses hernandez 65786 20.0 +ulysses ichabod 65551 20.0 +ulysses ichabod 65566 20.0 +ulysses johnson 65776 20.0 +ulysses king 65649 20.0 +ulysses laertes 65691 20.0 +ulysses laertes 65711 20.0 +ulysses laertes 65781 20.0 +ulysses miller 65610 20.0 +ulysses miller 65637 20.0 +ulysses nixon 65603 20.0 +ulysses ovid 65656 20.0 +ulysses polk 65563 20.0 +ulysses polk 65580 20.0 +ulysses polk 65612 20.0 +ulysses polk 65777 20.0 +ulysses quirinius 65786 20.0 +ulysses robinson 65744 20.0 +ulysses steinbeck 65611 20.0 +ulysses steinbeck 65680 20.0 +ulysses thompson 65788 20.0 +ulysses underhill 65570 20.0 +ulysses underhill 65616 20.0 +ulysses underhill 65620 20.0 +ulysses underhill 65623 20.0 +ulysses underhill 65641 20.0 +ulysses underhill 65713 20.0 +ulysses underhill 65785 20.0 +ulysses van buren 65684 20.0 +ulysses white 65654 20.0 +ulysses white 65675 20.0 +ulysses xylophone 65623 20.0 +ulysses xylophone 65636 20.0 +ulysses xylophone 65781 20.0 +ulysses young 65675 20.0 +ulysses young 65736 20.0 +ulysses young 65748 20.0 +victor allen 65684 20.0 +victor allen 65707 20.0 +victor brown 65550 20.0 +victor brown 65555 20.0 +victor brown 65622 20.0 +victor brown 65673 20.0 +victor davidson 65579 20.0 +victor davidson 65628 20.0 +victor davidson 65783 20.0 +victor ellison 65641 20.0 +victor ellison 65782 20.0 +victor hernandez 65571 20.0 +victor hernandez 65659 20.0 +victor hernandez 65708 20.0 +victor hernandez 65735 20.0 +victor hernandez 65775 20.0 +victor johnson 65606 20.0 +victor johnson 65607 20.0 +victor johnson 65607 20.0 +victor king 65721 20.0 +victor king 65743 20.0 +victor laertes 65638 20.0 +victor laertes 65644 20.0 +victor miller 65570 20.0 +victor nixon 65709 20.0 +victor nixon 65791 20.0 +victor ovid 65649 20.0 +victor polk 65625 20.0 +victor quirinius 65620 20.0 +victor quirinius 65651 20.0 +victor robinson 65596 20.0 +victor robinson 65673 20.0 +victor steinbeck 65618 20.0 +victor steinbeck 65661 20.0 +victor steinbeck 65686 20.0 +victor thompson 65548 20.0 +victor van buren 65664 20.0 +victor van buren 65774 20.0 +victor white 65548 20.0 +victor white 65601 20.0 +victor xylophone 65549 20.0 +victor xylophone 65618 20.0 +victor xylophone 65644 20.0 +victor xylophone 65677 20.0 +victor xylophone 65755 20.0 +victor young 65628 20.0 +victor zipper 65743 20.0 +wendy allen 65628 20.0 +wendy allen 65711 20.0 +wendy allen 65782 20.0 +wendy brown 65580 20.0 +wendy brown 65657 20.0 +wendy ellison 65545 20.0 +wendy ellison 65603 20.0 +wendy falkner 65595 20.0 +wendy falkner 65604 20.0 +wendy falkner 65635 20.0 +wendy garcia 65659 20.0 +wendy garcia 65746 20.0 +wendy garcia 65747 20.0 +wendy garcia 65777 20.0 +wendy hernandez 65650 20.0 +wendy ichabod 65730 20.0 +wendy king 65586 20.0 +wendy king 65664 20.0 +wendy king 65670 20.0 +wendy laertes 65566 20.0 +wendy laertes 65683 20.0 +wendy laertes 65727 20.0 +wendy miller 65582 20.0 +wendy miller 65626 20.0 +wendy nixon 65611 20.0 +wendy nixon 65746 20.0 +wendy ovid 65589 20.0 +wendy ovid 65643 20.0 +wendy polk 65656 20.0 +wendy polk 65692 20.0 +wendy quirinius 65766 20.0 +wendy quirinius 65767 20.0 +wendy robinson 65622 20.0 +wendy robinson 65715 20.0 +wendy robinson 65774 20.0 +wendy steinbeck 65612 20.0 +wendy thompson 65650 20.0 +wendy thompson 65737 20.0 +wendy underhill 65662 20.0 +wendy underhill 65758 20.0 +wendy underhill 65775 20.0 +wendy van buren 65680 20.0 +wendy van buren 65699 20.0 +wendy white 65705 20.0 +wendy xylophone 65687 20.0 +wendy xylophone 65773 20.0 +wendy young 65674 20.0 +wendy young 65685 20.0 +xavier allen 65611 20.0 +xavier allen 65618 20.0 +xavier allen 65771 20.0 +xavier brown 65600 20.0 +xavier brown 65704 20.0 +xavier brown 65723 20.0 +xavier carson 65731 20.0 +xavier carson 65758 20.0 +xavier davidson 65644 20.0 +xavier davidson 65664 20.0 +xavier davidson 65755 20.0 +xavier ellison 65541 20.0 +xavier ellison 65654 20.0 +xavier garcia 65672 20.0 +xavier hernandez 65541 20.0 +xavier hernandez 65544 20.0 +xavier hernandez 65766 20.0 +xavier ichabod 65597 20.0 +xavier ichabod 65663 20.0 +xavier johnson 65654 20.0 +xavier johnson 65744 20.0 +xavier king 65590 20.0 +xavier king 65601 20.0 +xavier laertes 65743 20.0 +xavier ovid 65788 20.0 +xavier polk 65587 20.0 +xavier polk 65653 20.0 +xavier polk 65675 20.0 +xavier polk 65696 20.0 +xavier quirinius 65599 20.0 +xavier quirinius 65650 20.0 +xavier quirinius 65656 20.0 +xavier quirinius 65737 20.0 +xavier thompson 65608 20.0 +xavier underhill 65710 20.0 +xavier white 65703 20.0 +xavier white 65732 20.0 +xavier xylophone 65572 20.0 +xavier zipper 65561 20.0 +yuri allen 65565 20.0 +yuri allen 65682 20.0 +yuri brown 65538 20.0 +yuri brown 65688 20.0 +yuri carson 65670 20.0 +yuri carson 65769 20.0 +yuri ellison 65570 20.0 +yuri ellison 65581 20.0 +yuri falkner 65658 20.0 +yuri falkner 65681 20.0 +yuri garcia 65639 20.0 +yuri hernandez 65706 20.0 +yuri johnson 65587 20.0 +yuri johnson 65697 20.0 +yuri johnson 65712 20.0 +yuri king 65721 20.0 +yuri laertes 65637 20.0 +yuri laertes 65773 20.0 +yuri nixon 65635 20.0 +yuri nixon 65740 20.0 +yuri polk 65607 20.0 +yuri polk 65713 20.0 +yuri polk 65742 20.0 +yuri quirinius 65544 20.0 +yuri quirinius 65617 20.0 +yuri quirinius 65695 20.0 +yuri steinbeck 65592 20.0 +yuri steinbeck 65679 20.0 +yuri thompson 65676 20.0 +yuri underhill 65718 20.0 +yuri underhill 65750 20.0 +yuri white 65659 20.0 +yuri xylophone 65714 20.0 +zach allen 65667 20.0 +zach brown 65559 20.0 +zach brown 65588 20.0 +zach brown 65691 20.0 +zach brown 65759 20.0 +zach brown 65762 20.0 +zach carson 65572 20.0 +zach ellison 65748 20.0 +zach falkner 65620 20.0 +zach falkner 65627 20.0 +zach garcia 65544 20.0 +zach garcia 65623 20.0 +zach garcia 65629 20.0 +zach garcia 65786 20.0 +zach ichabod 65599 20.0 +zach ichabod 65612 20.0 +zach king 65556 20.0 +zach king 65702 20.0 +zach king 65773 20.0 +zach miller 65584 20.0 +zach miller 65665 20.0 +zach miller 65719 20.0 +zach ovid 65578 20.0 +zach ovid 65703 20.0 +zach ovid 65750 20.0 +zach ovid 65784 20.0 +zach quirinius 65691 20.0 +zach robinson 65599 20.0 +zach steinbeck 65602 20.0 +zach steinbeck 65695 20.0 +zach thompson 65636 20.0 +zach thompson 65696 20.0 +zach underhill 65573 20.0 +zach white 65733 20.0 +zach xylophone 65542 20.0 +zach xylophone 65780 20.0 +zach young 65576 20.0 +zach zipper 65579 20.0 +zach zipper 65649 20.0 +zach zipper 65676 20.0 diff --git a/sql/hive/src/test/resources/log4j.properties b/sql/hive/src/test/resources/log4j.properties index 5bc08062d30eb..92eaf1f2795ba 100644 --- a/sql/hive/src/test/resources/log4j.properties +++ b/sql/hive/src/test/resources/log4j.properties @@ -33,7 +33,7 @@ log4j.appender.FA.layout=org.apache.log4j.PatternLayout log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n # Set the logger level of File Appender to WARN -log4j.appender.FA.Threshold = INFO +log4j.appender.FA.Threshold = DEBUG # Some packages are noisy for no good reason. log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala index d960a30e00738..30f5313d2b812 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -17,12 +17,11 @@ package org.apache.spark.sql.hive -import java.io.{OutputStream, PrintStream} - import scala.util.Try import org.scalatest.BeforeAndAfter +import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.{AnalysisException, QueryTest} @@ -109,25 +108,6 @@ class ErrorPositionSuite extends QueryTest with BeforeAndAfter { "SELECT 1 + array(1)", "1 + array") } - /** Hive can be very noisy, messing up the output of our tests. */ - private def quietly[A](f: => A): A = { - val origErr = System.err - val origOut = System.out - try { - System.setErr(new PrintStream(new OutputStream { - def write(b: Int) = {} - })) - System.setOut(new PrintStream(new OutputStream { - def write(b: Int) = {} - })) - - f - } finally { - System.setErr(origErr) - System.setOut(origOut) - } - } - /** * Creates a test that checks to see if the error thrown when analyzing a given query includes * the location of the given token in the query string. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameJoinSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameJoinSuite.scala new file mode 100644 index 0000000000000..52e782768cb75 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameJoinSuite.scala @@ -0,0 +1,38 @@ +/* + * 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 + +import org.apache.spark.sql.{Row, QueryTest} +import org.apache.spark.sql.hive.test.TestHive.implicits._ + + +class HiveDataFrameJoinSuite extends QueryTest { + + // We should move this into SQL package if we make case sensitivity configurable in SQL. + test("join - self join auto resolve ambiguity with case insensitivity") { + val df = Seq((1, "1"), (2, "2")).toDF("key", "value") + checkAnswer( + df.join(df, df("key") === df("Key")), + Row(1, "1", 1, "1") :: Row(2, "2", 2, "2") :: Nil) + + checkAnswer( + df.join(df.filter($"value" === "2"), df("key") === df("Key")), + Row(2, "2", 2, "2") :: Nil) + } + +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 0538aa203c5a0..47c60f651d14c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -31,6 +31,7 @@ import org.apache.hadoop.mapred.InvalidInputException import org.apache.spark.sql._ import org.apache.spark.util.Utils import org.apache.spark.sql.types._ +import org.apache.spark.sql.hive.client.{HiveTable, ManagedTable} import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.parquet.ParquetRelation2 @@ -686,16 +687,21 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { test("SPARK-6655 still support a schema stored in spark.sql.sources.schema") { val tableName = "spark6655" val schema = StructType(StructField("int", IntegerType, true) :: Nil) - // Manually create the metadata in metastore. - val tbl = new Table("default", tableName) - tbl.setProperty("spark.sql.sources.provider", "json") - tbl.setProperty("spark.sql.sources.schema", schema.json) - tbl.setProperty("EXTERNAL", "FALSE") - tbl.setTableType(TableType.MANAGED_TABLE) - tbl.setSerdeParam("path", catalog.hiveDefaultTableFilePath(tableName)) - catalog.synchronized { - catalog.client.createTable(tbl) - } + + val hiveTable = HiveTable( + specifiedDatabase = Some("default"), + name = tableName, + schema = Seq.empty, + partitionColumns = Seq.empty, + properties = Map( + "spark.sql.sources.provider" -> "json", + "spark.sql.sources.schema" -> schema.json, + "EXTERNAL" -> "FALSE"), + tableType = ManagedTable, + serdeProperties = Map( + "path" -> catalog.hiveDefaultTableFilePath(tableName))) + + catalog.client.createTable(hiveTable) invalidateTable(tableName) val actualSchema = table(tableName).schema diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala index d6ddd539d159d..8afe5459d4f1b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala @@ -26,8 +26,10 @@ import org.apache.spark.sql.hive.test.TestHive class SerializationSuite extends FunSuite { test("[SPARK-5840] HiveContext should be serializable") { - val hiveContext = new HiveContext(TestHive.sparkContext) + val hiveContext = TestHive hiveContext.hiveconf - new JavaSerializer(new SparkConf()).newInstance().serialize(hiveContext) + val serializer = new JavaSerializer(new SparkConf()).newInstance() + val bytes = serializer.serialize(hiveContext) + val deSer = serializer.deserialize[AnyRef](bytes) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index 81e77ba257bf1..321dc8d7322b8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -22,9 +22,13 @@ import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.util.Utils import org.scalatest.FunSuite +/** + * A simple set of tests that call the methods of a hive ClientInterface, loading different version + * of hive from maven central. These tests are simple in that they are mostly just testing to make + * sure that reflective calls are not throwing NoSuchMethod error, but the actually functionallity + * is not fully tested. + */ class VersionsSuite extends FunSuite with Logging { - val testType = "derby" - private def buildConf() = { lazy val warehousePath = Utils.createTempDir() lazy val metastorePath = Utils.createTempDir() @@ -50,6 +54,14 @@ class VersionsSuite extends FunSuite with Logging { causes } + private val emptyDir = Utils.createTempDir().getCanonicalPath + + private def partSpec = { + val hashMap = new java.util.LinkedHashMap[String, String] + hashMap.put("key", "1") + hashMap + } + // Its actually pretty easy to mess things up and have all of your tests "pass" by accidentally // connecting to an auto-populated, in-process metastore. Let's make sure we are getting the // versions right by forcing a known compatibility failure. @@ -66,10 +78,9 @@ class VersionsSuite extends FunSuite with Logging { private var client: ClientInterface = null versions.foreach { version => - test(s"$version: listTables") { + test(s"$version: create client") { client = null client = IsolatedClientLoader.forVersion(version, buildConf()).client - client.listTables("default") } test(s"$version: createDatabase") { @@ -101,5 +112,64 @@ class VersionsSuite extends FunSuite with Logging { test(s"$version: getTable") { client.getTable("default", "src") } + + test(s"$version: listTables") { + assert(client.listTables("default") === Seq("src")) + } + + test(s"$version: currentDatabase") { + assert(client.currentDatabase === "default") + } + + test(s"$version: getDatabase") { + client.getDatabase("default") + } + + test(s"$version: alterTable") { + client.alterTable(client.getTable("default", "src")) + } + + test(s"$version: set command") { + client.runSqlHive("SET spark.sql.test.key=1") + } + + test(s"$version: create partitioned table DDL") { + client.runSqlHive("CREATE TABLE src_part (value INT) PARTITIONED BY (key INT)") + client.runSqlHive("ALTER TABLE src_part ADD PARTITION (key = '1')") + } + + test(s"$version: getPartitions") { + client.getAllPartitions(client.getTable("default", "src_part")) + } + + test(s"$version: loadPartition") { + client.loadPartition( + emptyDir, + "default.src_part", + partSpec, + false, + false, + false, + false) + } + + test(s"$version: loadTable") { + client.loadTable( + emptyDir, + "src", + false, + false) + } + + test(s"$version: loadDynamicPartitions") { + client.loadDynamicPartitions( + emptyDir, + "default.src_part", + partSpec, + false, + 1, + false, + false) + } } } 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 027056d4b865f..9c056e493bfde 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 @@ -129,7 +129,7 @@ abstract class HiveComparisonTest } protected def prepareAnswer( - hiveQuery: TestHive.type#HiveQLQueryExecution, + hiveQuery: TestHive.type#QueryExecution, answer: Seq[String]): Seq[String] = { def isSorted(plan: LogicalPlan): Boolean = plan match { @@ -298,9 +298,11 @@ abstract class HiveComparisonTest hiveCachedResults } else { - val hiveQueries = queryList.map(new TestHive.HiveQLQueryExecution(_)) + val hiveQueries = queryList.map(new TestHive.QueryExecution(_)) // Make sure we can at least parse everything before attempting hive execution. - hiveQueries.foreach(_.analyzed) + // Note this must only look at the logical plan as we might not be able to analyze if + // other DDL has not been executed yet. + hiveQueries.foreach(_.logical) val computedResults = (queryList.zipWithIndex, hiveQueries, hiveCacheFiles).zipped.map { case ((queryString, i), hiveQuery, cachedAnswerFile)=> try { @@ -346,7 +348,7 @@ abstract class HiveComparisonTest // Run w/ catalyst val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => - val query = new TestHive.HiveQLQueryExecution(queryString) + val query = new TestHive.QueryExecution(queryString) try { (query, prepareAnswer(query, query.stringResult())) } catch { case e: Throwable => val errorMessage = @@ -402,7 +404,7 @@ abstract class HiveComparisonTest // okay by running a simple query. If this fails then we halt testing since // something must have gone seriously wrong. try { - new TestHive.HiveQLQueryExecution("SELECT key FROM src").stringResult() + new TestHive.QueryExecution("SELECT key FROM src").stringResult() TestHive.runSqlHive("SELECT key FROM src") } catch { case e: Exception => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index ac10b173307d8..7d728fe87bda7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -900,7 +900,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |DROP TABLE IF EXISTS dynamic_part_table; """.stripMargin) - test("Dynamic partition folder layout") { + ignore("Dynamic partition folder layout") { sql("DROP TABLE IF EXISTS dynamic_part_table") sql("CREATE TABLE dynamic_part_table(intcol INT) PARTITIONED BY (partcol1 INT, partcol2 INT)") sql("SET hive.exec.dynamic.partition.mode=nonstrict") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 067b577f1560e..de6a41ce5bfcb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -145,25 +145,26 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { expectedScannedColumns: Seq[String], expectedPartValues: Seq[Seq[String]]): Unit = { test(s"$testCaseName - pruning test") { - val plan = new TestHive.HiveQLQueryExecution(sql).executedPlan + val plan = new TestHive.QueryExecution(sql).executedPlan val actualOutputColumns = plan.output.map(_.name) val (actualScannedColumns, actualPartValues) = plan.collect { case p @ HiveTableScan(columns, relation, _) => val columnNames = columns.map(_.name) - val partValues = p.prunePartitions(relation.hiveQlPartitions).map(_.getValues) + val partValues = if (relation.table.isPartitioned) { + p.prunePartitions(relation.hiveQlPartitions).map(_.getValues) + } else { + Seq.empty + } (columnNames, partValues) }.head assert(actualOutputColumns === expectedOutputColumns, "Output columns mismatch") assert(actualScannedColumns === expectedScannedColumns, "Scanned columns mismatch") - assert( - actualPartValues.length === expectedPartValues.length, - "Partition value count mismatches") + val actualPartitions = actualPartValues.map(_.toSeq.mkString(",")).sorted + val expectedPartitions = expectedPartValues.map(_.mkString(",")).sorted - for ((actual, expected) <- actualPartValues.zip(expectedPartValues)) { - assert(actual sameElements expected, "Partition values mismatch") - } + assert(actualPartitions === expectedPartitions, "Partitions selected do not match") } // Creates a query test to compare query results generated by Hive and Catalyst. 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 630dec8fa05a0..c605f1017547a 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 @@ -48,6 +48,10 @@ case class Order( state: String, month: Int) +case class WindowData( + month: Int, + area: String, + product: Int) /** A SQL Dialect for testing purpose, and it can not be nested type */ class MyDialect extends DefaultDialect @@ -604,4 +608,154 @@ class SQLQuerySuite extends QueryTest { sql("SELECT TRANSFORM (d1, d2, d3) USING 'cat' AS (a,b,c) FROM script_trans") .queryExecution.toRdd.count()) } + + test("window function: udaf with aggregate expressin") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().registerTempTable("windowData") + + checkAnswer( + sql( + """ + |select area, sum(product), sum(sum(product)) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 5, 11), + ("a", 6, 11), + ("b", 7, 15), + ("b", 8, 15), + ("c", 9, 19), + ("c", 10, 19) + ).map(i => Row(i._1, i._2, i._3))) + + checkAnswer( + sql( + """ + |select area, sum(product) - 1, sum(sum(product)) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 4, 11), + ("a", 5, 11), + ("b", 6, 15), + ("b", 7, 15), + ("c", 8, 19), + ("c", 9, 19) + ).map(i => Row(i._1, i._2, i._3))) + + checkAnswer( + sql( + """ + |select area, sum(product), sum(product) / sum(sum(product)) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 5, 5d/11), + ("a", 6, 6d/11), + ("b", 7, 7d/15), + ("b", 8, 8d/15), + ("c", 10, 10d/19), + ("c", 9, 9d/19) + ).map(i => Row(i._1, i._2, i._3))) + + checkAnswer( + sql( + """ + |select area, sum(product), sum(product) / sum(sum(product) - 1) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 5, 5d/9), + ("a", 6, 6d/9), + ("b", 7, 7d/13), + ("b", 8, 8d/13), + ("c", 10, 10d/17), + ("c", 9, 9d/17) + ).map(i => Row(i._1, i._2, i._3))) + } + + test("window function: partition and order expressions") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().registerTempTable("windowData") + + checkAnswer( + sql( + """ + |select month, area, product, sum(product + 1) over (partition by 1 order by 2) + |from windowData + """.stripMargin), + Seq( + (1, "a", 5, 51), + (2, "a", 6, 51), + (3, "b", 7, 51), + (4, "b", 8, 51), + (5, "c", 9, 51), + (6, "c", 10, 51) + ).map(i => Row(i._1, i._2, i._3, i._4))) + + checkAnswer( + sql( + """ + |select month, area, product, sum(product) + |over (partition by month % 2 order by 10 - product) + |from windowData + """.stripMargin), + Seq( + (1, "a", 5, 21), + (2, "a", 6, 24), + (3, "b", 7, 16), + (4, "b", 8, 18), + (5, "c", 9, 9), + (6, "c", 10, 10) + ).map(i => Row(i._1, i._2, i._3, i._4))) + } + + test("window function: expressions in arguments of a window functions") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().registerTempTable("windowData") + + checkAnswer( + sql( + """ + |select month, area, month % 2, + |lag(product, 1 + 1, product) over (partition by month % 2 order by area) + |from windowData + """.stripMargin), + Seq( + (1, "a", 1, 5), + (2, "a", 0, 6), + (3, "b", 1, 7), + (4, "b", 0, 8), + (5, "c", 1, 5), + (6, "c", 0, 6) + ).map(i => Row(i._1, i._2, i._3, i._4))) + } + + test("test case key when") { + (1 to 5).map(i => (i, i.toString)).toDF("k", "v").registerTempTable("t") + checkAnswer( + sql("SELECT CASE k WHEN 2 THEN 22 WHEN 4 THEN 44 ELSE 0 END, v FROM t"), + Row(0, "1") :: Row(22, "2") :: Row(0, "3") :: Row(44, "4") :: Row(0, "5") :: Nil) + } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index b1ad0d42ec8a6..bbdb4e8af036c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -563,13 +563,17 @@ class StreamingContext private[streaming] ( /** * Stop the execution of the streams immediately (does not wait for all received data - * to be processed). + * to be processed). By default, if `stopSparkContext` is not specified, the underlying + * SparkContext will also be stopped. This implicit behavior can be configured using the + * SparkConf configuration spark.streaming.stopSparkContextByDefault. * - * @param stopSparkContext if true, stops the associated SparkContext. The underlying SparkContext + * @param stopSparkContext If true, stops the associated SparkContext. The underlying SparkContext * will be stopped regardless of whether this StreamingContext has been * started. */ - def stop(stopSparkContext: Boolean = true): Unit = synchronized { + def stop( + stopSparkContext: Boolean = conf.getBoolean("spark.streaming.stopSparkContextByDefault", true) + ): Unit = synchronized { stop(stopSparkContext, false) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala index 15d9710d37cd4..5cfe43a1ce726 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala @@ -24,6 +24,7 @@ import org.apache.spark.storage.BlockId import org.apache.spark.streaming._ import org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.streaming.scheduler.InputInfo import org.apache.spark.streaming.util.WriteAheadLogUtils /** @@ -68,6 +69,10 @@ abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingCont val blockInfos = receiverTracker.getBlocksOfBatch(validTime).getOrElse(id, Seq.empty) val blockIds = blockInfos.map { _.blockId.asInstanceOf[BlockId] }.toArray + // Register the input blocks information into InputInfoTracker + val inputInfo = InputInfo(id, blockInfos.map(_.numRecords).sum) + ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo) + // Are WAL record handles present with all the blocks val areWALRecordHandlesPresent = blockInfos.forall { _.walRecordHandleOption.nonEmpty } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala index 3f1cab69068dc..831f60e870f74 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala @@ -19,7 +19,7 @@ package org.apache.spark.streaming.ui import javax.servlet.http.HttpServletRequest -import scala.xml.{NodeSeq, Node} +import scala.xml.{NodeSeq, Node, Text} import org.apache.commons.lang3.StringEscapeUtils @@ -28,6 +28,7 @@ import org.apache.spark.ui.{UIUtils => SparkUIUtils, WebUIPage} import org.apache.spark.streaming.ui.StreamingJobProgressListener.{SparkJobId, OutputOpId} import org.apache.spark.ui.jobs.UIData.JobUIData +private case class SparkJobIdWithUIData(sparkJobId: SparkJobId, jobUIData: Option[JobUIData]) private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { private val streamingListener = parent.listener @@ -44,25 +45,33 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { Error } + private def generateJobRow( + outputOpId: OutputOpId, + outputOpDescription: Seq[Node], + formattedOutputOpDuration: String, + numSparkJobRowsInOutputOp: Int, + isFirstRow: Boolean, + sparkJob: SparkJobIdWithUIData): Seq[Node] = { + if (sparkJob.jobUIData.isDefined) { + generateNormalJobRow(outputOpId, outputOpDescription, formattedOutputOpDuration, + numSparkJobRowsInOutputOp, isFirstRow, sparkJob.jobUIData.get) + } else { + generateDroppedJobRow(outputOpId, outputOpDescription, formattedOutputOpDuration, + numSparkJobRowsInOutputOp, isFirstRow, sparkJob.sparkJobId) + } + } + /** * Generate a row for a Spark Job. Because duplicated output op infos needs to be collapsed into * one cell, we use "rowspan" for the first row of a output op. */ - def generateJobRow( + private def generateNormalJobRow( outputOpId: OutputOpId, + outputOpDescription: Seq[Node], formattedOutputOpDuration: String, numSparkJobRowsInOutputOp: Int, isFirstRow: Boolean, sparkJob: JobUIData): Seq[Node] = { - val lastStageInfo = Option(sparkJob.stageIds) - .filter(_.nonEmpty) - .flatMap { ids => sparkListener.stageIdToInfo.get(ids.max) } - val lastStageData = lastStageInfo.flatMap { s => - sparkListener.stageIdToData.get((s.stageId, s.attemptId)) - } - - val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") - val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("") val duration: Option[Long] = { sparkJob.submissionTime.map { start => val end = sparkJob.completionTime.getOrElse(System.currentTimeMillis()) @@ -83,9 +92,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { if (isFirstRow) { {outputOpId.toString} - - {lastStageDescription} - {lastStageName} + {outputOpDescription} {formattedOutputOpDuration} } else { @@ -122,27 +129,96 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { } - private def generateOutputOpIdRow( - outputOpId: OutputOpId, sparkJobs: Seq[JobUIData]): Seq[Node] = { - val sparkjobDurations = sparkJobs.map(sparkJob => { - sparkJob.submissionTime.map { start => - val end = sparkJob.completionTime.getOrElse(System.currentTimeMillis()) - end - start + /** + * If a job is dropped by sparkListener due to exceeding the limitation, we only show the job id + * with "-" cells. + */ + private def generateDroppedJobRow( + outputOpId: OutputOpId, + outputOpDescription: Seq[Node], + formattedOutputOpDuration: String, + numSparkJobRowsInOutputOp: Int, + isFirstRow: Boolean, + jobId: Int): Seq[Node] = { + // In the first row, output op id and its information needs to be shown. In other rows, these + // cells will be taken up due to "rowspan". + // scalastyle:off + val prefixCells = + if (isFirstRow) { + {outputOpId.toString} + {outputOpDescription} + {formattedOutputOpDuration} + } else { + Nil } - }) + // scalastyle:on + + + {prefixCells} + + {jobId.toString} + + + - + + - + + - + + - + + } + + private def generateOutputOpIdRow( + outputOpId: OutputOpId, sparkJobs: Seq[SparkJobIdWithUIData]): Seq[Node] = { + // We don't count the durations of dropped jobs + val sparkJobDurations = sparkJobs.filter(_.jobUIData.nonEmpty).map(_.jobUIData.get). + map(sparkJob => { + sparkJob.submissionTime.map { start => + val end = sparkJob.completionTime.getOrElse(System.currentTimeMillis()) + end - start + } + }) val formattedOutputOpDuration = - if (sparkjobDurations.exists(_ == None)) { - // If any job does not finish, set "formattedOutputOpDuration" to "-" + if (sparkJobDurations.isEmpty || sparkJobDurations.exists(_ == None)) { + // If no job or any job does not finish, set "formattedOutputOpDuration" to "-" "-" } else { - SparkUIUtils.formatDuration(sparkjobDurations.flatMap(x => x).sum) + SparkUIUtils.formatDuration(sparkJobDurations.flatMap(x => x).sum) } - generateJobRow(outputOpId, formattedOutputOpDuration, sparkJobs.size, true, sparkJobs.head) ++ + + val description = generateOutputOpDescription(sparkJobs) + + generateJobRow( + outputOpId, description, formattedOutputOpDuration, sparkJobs.size, true, sparkJobs.head) ++ sparkJobs.tail.map { sparkJob => - generateJobRow(outputOpId, formattedOutputOpDuration, sparkJobs.size, false, sparkJob) + generateJobRow( + outputOpId, description, formattedOutputOpDuration, sparkJobs.size, false, sparkJob) }.flatMap(x => x) } + private def generateOutputOpDescription(sparkJobs: Seq[SparkJobIdWithUIData]): Seq[Node] = { + val lastStageInfo = + sparkJobs.flatMap(_.jobUIData).headOption. // Get the first JobUIData + flatMap { sparkJob => // For the first job, get the latest Stage info + if (sparkJob.stageIds.isEmpty) { + None + } else { + sparkListener.stageIdToInfo.get(sparkJob.stageIds.max) + } + } + val lastStageData = lastStageInfo.flatMap { s => + sparkListener.stageIdToData.get((s.stageId, s.attemptId)) + } + + val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") + val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("") + + + {lastStageDescription} + ++ Text(lastStageName) + } + private def failureReasonCell(failureReason: String): Seq[Node] = { val isMultiline = failureReason.indexOf('\n') >= 0 // Display the first line by default @@ -187,10 +263,10 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { (outputOpId, outputOpIdAndSparkJobIds.map(_.sparkJobId).sorted) } sparkListener.synchronized { - val outputOpIdWithJobs: Seq[(OutputOpId, Seq[JobUIData])] = + val outputOpIdWithJobs: Seq[(OutputOpId, Seq[SparkJobIdWithUIData])] = outputOpIdToSparkJobIds.map { case (outputOpId, sparkJobIds) => - // Filter out spark Job ids that don't exist in sparkListener - (outputOpId, sparkJobIds.flatMap(getJobData)) + (outputOpId, + sparkJobIds.map(sparkJobId => SparkJobIdWithUIData(sparkJobId, getJobData(sparkJobId)))) } @@ -200,7 +276,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { { outputOpIdWithJobs.map { - case (outputOpId, jobs) => generateOutputOpIdRow(outputOpId, jobs) + case (outputOpId, sparkJobIds) => generateOutputOpIdRow(outputOpId, sparkJobIds) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 60745325029f6..93e6b0cd7c661 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -50,6 +50,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Set up the streaming context and input streams withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => + ssc.addStreamingListener(ssc.progressListener) + val input = Seq(1, 2, 3, 4, 5) // Use "batchCount" to make sure we check the result after all batches finish val batchCounter = new BatchCounter(ssc) @@ -72,6 +74,11 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { if (!batchCounter.waitUntilBatchesCompleted(input.size, 30000)) { fail("Timeout: cannot finish all batches in 30 seconds") } + + // Verify all "InputInfo"s have been reported + assert(ssc.progressListener.numTotalReceivedRecords === input.size) + assert(ssc.progressListener.numTotalProcessedRecords === input.size) + logInfo("Stopping server") testServer.stop() logInfo("Stopping context") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 5207b7109e69b..a589deb1fa579 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -118,6 +118,11 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w assert(ssc.state === ssc.StreamingContextState.Started) ssc.stop() assert(ssc.state === ssc.StreamingContextState.Stopped) + + // Make sure that the SparkContext is also stopped by default + intercept[Exception] { + ssc.sparkContext.makeRDD(1 to 10) + } } test("start multiple times") { @@ -154,16 +159,26 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w } test("stop only streaming context") { - ssc = new StreamingContext(master, appName, batchDuration) + val conf = new SparkConf().setMaster(master).setAppName(appName) + + // Explicitly do not stop SparkContext + ssc = new StreamingContext(conf, batchDuration) sc = ssc.sparkContext addInputStream(ssc).register() ssc.start() ssc.stop(stopSparkContext = false) assert(sc.makeRDD(1 to 100).collect().size === 100) - ssc = new StreamingContext(sc, batchDuration) + sc.stop() + + // Implicitly do not stop SparkContext + conf.set("spark.streaming.stopSparkContextByDefault", "false") + ssc = new StreamingContext(conf, batchDuration) + sc = ssc.sparkContext addInputStream(ssc).register() ssc.start() ssc.stop() + assert(sc.makeRDD(1 to 100).collect().size === 100) + sc.stop() } test("stop(stopSparkContext=true) after stop(stopSparkContext=false)") { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 4d0cd7516f42e..4f70ae7f1f187 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -73,9 +73,11 @@ class TestInputStream[T: ClassTag](ssc_ : StreamingContext, input: Seq[Seq[T]], * * The buffer contains a sequence of RDD's, each containing a sequence of items */ -class TestOutputStream[T: ClassTag](parent: DStream[T], - val output: ArrayBuffer[Seq[T]] = ArrayBuffer[Seq[T]]()) - extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => { +class TestOutputStream[T: ClassTag]( + parent: DStream[T], + val output: SynchronizedBuffer[Seq[T]] = + new ArrayBuffer[Seq[T]] with SynchronizedBuffer[Seq[T]] + ) extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => { val collected = rdd.collect() output += collected }) { @@ -95,8 +97,10 @@ class TestOutputStream[T: ClassTag](parent: DStream[T], * The buffer contains a sequence of RDD's, each containing a sequence of partitions, each * containing a sequence of items. */ -class TestOutputStreamWithPartitions[T: ClassTag](parent: DStream[T], - val output: ArrayBuffer[Seq[Seq[T]]] = ArrayBuffer[Seq[Seq[T]]]()) +class TestOutputStreamWithPartitions[T: ClassTag]( + parent: DStream[T], + val output: SynchronizedBuffer[Seq[Seq[T]]] = + new ArrayBuffer[Seq[Seq[T]]] with SynchronizedBuffer[Seq[Seq[T]]]) extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => { val collected = rdd.glom().collect().map(_.toSeq) output += collected @@ -108,10 +112,6 @@ class TestOutputStreamWithPartitions[T: ClassTag](parent: DStream[T], ois.defaultReadObject() output.clear() } - - def toTestOutputStream: TestOutputStream[T] = { - new TestOutputStream[T](this.parent, this.output.map(_.flatten)) - } } /** @@ -425,12 +425,21 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { logInfo("--------------------------------") // Match the output with the expected output - assert(output.size === expectedOutput.size, "Number of outputs do not match") for (i <- 0 until output.size) { if (useSet) { - assert(output(i).toSet === expectedOutput(i).toSet) + assert( + output(i).toSet === expectedOutput(i).toSet, + s"Set comparison failed\n" + + s"Expected output (${expectedOutput.size} items):\n${expectedOutput.mkString("\n")}\n" + + s"Generated output (${output.size} items): ${output.mkString("\n")}" + ) } else { - assert(output(i).toList === expectedOutput(i).toList) + assert( + output(i).toList === expectedOutput(i).toList, + s"Ordered list comparison failed\n" + + s"Expected output (${expectedOutput.size} items):\n${expectedOutput.mkString("\n")}\n" + + s"Generated output (${output.size} items): ${output.mkString("\n")}" + ) } } logInfo("Output verified successfully") diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSet.java b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSet.java index f72e07fce92fd..28e23da108ebe 100644 --- a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSet.java +++ b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSet.java @@ -102,4 +102,12 @@ public boolean isSet(int index) { public int nextSetBit(int fromIndex) { return BitSetMethods.nextSetBit(baseObject, baseOffset, fromIndex, numWords); } + + /** + * Returns {@code true} if any bit is set. + */ + public boolean anySet() { + return BitSetMethods.anySet(baseObject, baseOffset, numWords); + } + } diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java index f30626d8f4317..0987191c1c636 100644 --- a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java +++ b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java @@ -70,9 +70,10 @@ public static boolean isSet(Object baseObject, long baseOffset, int index) { /** * Returns {@code true} if any bit is set. */ - public static boolean anySet(Object baseObject, long baseOffset, long bitSetWidthInBytes) { - for (int i = 0; i <= bitSetWidthInBytes; i++) { - if (PlatformDependent.UNSAFE.getByte(baseObject, baseOffset + i) != 0) { + public static boolean anySet(Object baseObject, long baseOffset, long bitSetWidthInWords) { + long addr = baseOffset; + for (int i = 0; i < bitSetWidthInWords; i++, addr += WORD_SIZE) { + if (PlatformDependent.UNSAFE.getLong(baseObject, addr) != 0) { return true; } } diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/bitset/BitSetSuite.java b/unsafe/src/test/java/org/apache/spark/unsafe/bitset/BitSetSuite.java index e3a824e29b768..18393db9f382f 100644 --- a/unsafe/src/test/java/org/apache/spark/unsafe/bitset/BitSetSuite.java +++ b/unsafe/src/test/java/org/apache/spark/unsafe/bitset/BitSetSuite.java @@ -39,6 +39,8 @@ public void basicOps() { for (int i = 0; i < bs.capacity(); i++) { Assert.assertFalse(bs.isSet(i)); } + // another form of asserting that the bit set is empty + Assert.assertFalse(bs.anySet()); // Set every bit and check it. for (int i = 0; i < bs.capacity(); i++) { @@ -52,6 +54,11 @@ public void basicOps() { bs.unset(i); Assert.assertFalse(bs.isSet(i)); } + + // Make sure anySet() can detect any set bit + bs = createBitSet(256); + bs.set(64); + Assert.assertTrue(bs.anySet()); } @Test diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 20ecaf092e3f8..d21a7393478ce 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -468,6 +468,17 @@ private[spark] class Client( env("SPARK_YARN_USER_ENV") = userEnvs } + // if spark.submit.pyArchives is in sparkConf, append pyArchives to PYTHONPATH + // that can be passed on to the ApplicationMaster and the executors. + if (sparkConf.contains("spark.submit.pyArchives")) { + var pythonPath = sparkConf.get("spark.submit.pyArchives") + if (env.contains("PYTHONPATH")) { + pythonPath = Seq(env.get("PYTHONPATH"), pythonPath).mkString(File.pathSeparator) + } + env("PYTHONPATH") = pythonPath + sparkConf.setExecutorEnv("PYTHONPATH", pythonPath) + } + // In cluster mode, if the deprecated SPARK_JAVA_OPTS is set, we need to propagate it to // executors. But we can't just set spark.executor.extraJavaOptions, because the driver's // SparkContext will not let that set spark* system properties, which is expected behavior for @@ -1074,7 +1085,7 @@ object Client extends Logging { val hiveConf = hiveClass.getMethod("getConf").invoke(hive) val hiveConfClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf") - val hiveConfGet = (param:String) => Option(hiveConfClass + val hiveConfGet = (param: String) => Option(hiveConfClass .getMethod("get", classOf[java.lang.String]) .invoke(hiveConf, param)) @@ -1096,7 +1107,7 @@ object Client extends Logging { val hive2Token = new Token[DelegationTokenIdentifier]() hive2Token.decodeFromUrlString(tokenStr) - credentials.addToken(new Text("hive.server2.delegation.token"),hive2Token) + credentials.addToken(new Text("hive.server2.delegation.token"), hive2Token) logDebug("Added hive.Server2.delegation.token to conf.") hiveClass.getMethod("closeCurrent").invoke(null) } else { @@ -1141,13 +1152,13 @@ object Client extends Logging { logInfo("Added HBase security token to credentials.") } catch { - case e:java.lang.NoSuchMethodException => + case e: java.lang.NoSuchMethodException => logInfo("HBase Method not found: " + e) - case e:java.lang.ClassNotFoundException => + case e: java.lang.ClassNotFoundException => logDebug("HBase Class not found: " + e) - case e:java.lang.NoClassDefFoundError => + case e: java.lang.NoClassDefFoundError => logDebug("HBase Class not found: " + e) - case e:Exception => + case e: Exception => logError("Exception when obtaining HBase security token: " + e) } }