diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index cb09e847d739a..a1f5c4f8cc18d 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -150,6 +150,56 @@ NULL #' df <- createDataFrame(cbind(model = rownames(mtcars), mtcars))} NULL +#' Miscellaneous functions for Column operations +#' +#' Miscellaneous functions defined for \code{Column}. +#' +#' @param x Column to compute on. In \code{sha2}, it is one of 224, 256, 384, or 512. +#' @param y Column to compute on. +#' @param ... additional Columns. +#' @name column_misc_functions +#' @rdname column_misc_functions +#' @family misc functions +#' @examples +#' \dontrun{ +#' # Dataframe used throughout this doc +#' df <- createDataFrame(cbind(model = rownames(mtcars), mtcars)[, 1:2]) +#' tmp <- mutate(df, v1 = crc32(df$model), v2 = hash(df$model), +#' v3 = hash(df$model, df$mpg), v4 = md5(df$model), +#' v5 = sha1(df$model), v6 = sha2(df$model, 256)) +#' head(tmp) +#' } +NULL + +#' Collection functions for Column operations +#' +#' Collection functions defined for \code{Column}. +#' +#' @param x Column to compute on. Note the difference in the following methods: +#' \itemize{ +#' \item \code{to_json}: it is the column containing the struct or array of the structs. +#' \item \code{from_json}: it is the column containing the JSON string. +#' } +#' @param ... additional argument(s). In \code{to_json} and \code{from_json}, this contains +#' additional named properties to control how it is converted, accepts the same +#' options as the JSON data source. +#' @name column_collection_functions +#' @rdname column_collection_functions +#' @family collection functions +#' @examples +#' \dontrun{ +#' # Dataframe used throughout this doc +#' df <- createDataFrame(cbind(model = rownames(mtcars), mtcars)) +#' df <- createDataFrame(cbind(model = rownames(mtcars), mtcars)) +#' tmp <- mutate(df, v1 = create_array(df$mpg, df$cyl, df$hp)) +#' head(select(tmp, array_contains(tmp$v1, 21), size(tmp$v1))) +#' tmp2 <- mutate(tmp, v2 = explode(tmp$v1)) +#' head(tmp2) +#' head(select(tmp, posexplode(tmp$v1))) +#' head(select(tmp, sort_array(tmp$v1))) +#' head(select(tmp, sort_array(tmp$v1, asc = FALSE)))} +NULL + #' @details #' \code{lit}: A new Column is created to represent the literal value. #' If the parameter is a Column, it is returned unchanged. @@ -569,19 +619,13 @@ setMethod("count", column(jc) }) -#' crc32 -#' -#' Calculates the cyclic redundancy check value (CRC32) of a binary column and -#' returns the value as a bigint. -#' -#' @param x Column to compute on. +#' @details +#' \code{crc32}: Calculates the cyclic redundancy check value (CRC32) of a binary column +#' and returns the value as a bigint. #' -#' @rdname crc32 -#' @name crc32 -#' @family misc functions -#' @aliases crc32,Column-method +#' @rdname column_misc_functions +#' @aliases crc32 crc32,Column-method #' @export -#' @examples \dontrun{crc32(df$c)} #' @note crc32 since 1.5.0 setMethod("crc32", signature(x = "Column"), @@ -590,19 +634,13 @@ setMethod("crc32", column(jc) }) -#' hash -#' -#' Calculates the hash code of given columns, and returns the result as a int column. -#' -#' @param x Column to compute on. -#' @param ... additional Column(s) to be included. +#' @details +#' \code{hash}: Calculates the hash code of given columns, and returns the result +#' as an int column. #' -#' @rdname hash -#' @name hash -#' @family misc functions -#' @aliases hash,Column-method +#' @rdname column_misc_functions +#' @aliases hash hash,Column-method #' @export -#' @examples \dontrun{hash(df$c)} #' @note hash since 2.0.0 setMethod("hash", signature(x = "Column"), @@ -1055,19 +1093,13 @@ setMethod("max", column(jc) }) -#' md5 -#' -#' Calculates the MD5 digest of a binary column and returns the value +#' @details +#' \code{md5}: Calculates the MD5 digest of a binary column and returns the value #' as a 32 character hex string. #' -#' @param x Column to compute on. -#' -#' @rdname md5 -#' @name md5 -#' @family misc functions -#' @aliases md5,Column-method +#' @rdname column_misc_functions +#' @aliases md5 md5,Column-method #' @export -#' @examples \dontrun{md5(df$c)} #' @note md5 since 1.5.0 setMethod("md5", signature(x = "Column"), @@ -1307,19 +1339,13 @@ setMethod("second", column(jc) }) -#' sha1 -#' -#' Calculates the SHA-1 digest of a binary column and returns the value +#' @details +#' \code{sha1}: Calculates the SHA-1 digest of a binary column and returns the value #' as a 40 character hex string. #' -#' @param x Column to compute on. -#' -#' @rdname sha1 -#' @name sha1 -#' @family misc functions -#' @aliases sha1,Column-method +#' @rdname column_misc_functions +#' @aliases sha1 sha1,Column-method #' @export -#' @examples \dontrun{sha1(df$c)} #' @note sha1 since 1.5.0 setMethod("sha1", signature(x = "Column"), @@ -1645,30 +1671,23 @@ setMethod("to_date", column(jc) }) -#' to_json -#' -#' Converts a column containing a \code{structType} or array of \code{structType} into a Column -#' of JSON string. Resolving the Column can fail if an unsupported type is encountered. -#' -#' @param x Column containing the struct or array of the structs -#' @param ... additional named properties to control how it is converted, accepts the same options -#' as the JSON data source. +#' @details +#' \code{to_json}: Converts a column containing a \code{structType} or array of \code{structType} +#' into a Column of JSON string. Resolving the Column can fail if an unsupported type is encountered. #' -#' @family non-aggregate functions -#' @rdname to_json -#' @name to_json -#' @aliases to_json,Column-method +#' @rdname column_collection_functions +#' @aliases to_json to_json,Column-method #' @export #' @examples +#' #' \dontrun{ #' # Converts a struct into a JSON object -#' df <- sql("SELECT named_struct('date', cast('2000-01-01' as date)) as d") -#' select(df, to_json(df$d, dateFormat = 'dd/MM/yyyy')) +#' df2 <- sql("SELECT named_struct('date', cast('2000-01-01' as date)) as d") +#' select(df2, to_json(df2$d, dateFormat = 'dd/MM/yyyy')) #' #' # Converts an array of structs into a JSON array -#' df <- sql("SELECT array(named_struct('name', 'Bob'), named_struct('name', 'Alice')) as people") -#' select(df, to_json(df$people)) -#'} +#' df2 <- sql("SELECT array(named_struct('name', 'Bob'), named_struct('name', 'Alice')) as people") +#' df2 <- mutate(df2, people_json = to_json(df2$people))} #' @note to_json since 2.2.0 setMethod("to_json", signature(x = "Column"), function(x, ...) { @@ -2123,28 +2142,28 @@ setMethod("date_format", signature(y = "Column", x = "character"), column(jc) }) -#' from_json -#' -#' Parses a column containing a JSON string into a Column of \code{structType} with the specified -#' \code{schema} or array of \code{structType} if \code{as.json.array} is set to \code{TRUE}. -#' If the string is unparseable, the Column will contains the value NA. +#' @details +#' \code{from_json}: Parses a column containing a JSON string into a Column of \code{structType} +#' with the specified \code{schema} or array of \code{structType} if \code{as.json.array} is set +#' to \code{TRUE}. If the string is unparseable, the Column will contain the value NA. #' -#' @param x Column containing the JSON string. +#' @rdname column_collection_functions #' @param schema a structType object to use as the schema to use when parsing the JSON string. #' @param as.json.array indicating if input string is JSON array of objects or a single object. -#' @param ... additional named properties to control how the json is parsed, accepts the same -#' options as the JSON data source. -#' -#' @family non-aggregate functions -#' @rdname from_json -#' @name from_json -#' @aliases from_json,Column,structType-method +#' @aliases from_json from_json,Column,structType-method #' @export #' @examples +#' #' \dontrun{ -#' schema <- structType(structField("name", "string"), -#' select(df, from_json(df$value, schema, dateFormat = "dd/MM/yyyy")) -#'} +#' df2 <- sql("SELECT named_struct('date', cast('2000-01-01' as date)) as d") +#' df2 <- mutate(df2, d2 = to_json(df2$d, dateFormat = 'dd/MM/yyyy')) +#' schema <- structType(structField("date", "string")) +#' head(select(df2, from_json(df2$d2, schema, dateFormat = 'dd/MM/yyyy'))) + +#' df2 <- sql("SELECT named_struct('name', 'Bob') as people") +#' df2 <- mutate(df2, people_json = to_json(df2$people)) +#' schema <- structType(structField("name", "string")) +#' head(select(df2, from_json(df2$people_json, schema)))} #' @note from_json since 2.2.0 setMethod("from_json", signature(x = "Column", schema = "structType"), function(x, schema, as.json.array = FALSE, ...) { @@ -2309,19 +2328,14 @@ setMethod("format_number", signature(y = "Column", x = "numeric"), column(jc) }) -#' sha2 -#' -#' Calculates the SHA-2 family of hash functions of a binary column and -#' returns the value as a hex string. +#' @details +#' \code{sha2}: Calculates the SHA-2 family of hash functions of a binary column and +#' returns the value as a hex string. The second argument \code{x} specifies the number +#' of bits, and is one of 224, 256, 384, or 512. #' -#' @param y column to compute SHA-2 on. -#' @param x one of 224, 256, 384, or 512. -#' @family misc functions -#' @rdname sha2 -#' @name sha2 -#' @aliases sha2,Column,numeric-method +#' @rdname column_misc_functions +#' @aliases sha2 sha2,Column,numeric-method #' @export -#' @examples \dontrun{sha2(df$c, 256)} #' @note sha2 since 1.5.0 setMethod("sha2", signature(y = "Column", x = "numeric"), function(y, x) { @@ -3109,18 +3123,14 @@ setMethod("row_number", ###################### Collection functions###################### -#' array_contains -#' -#' Returns null if the array is null, true if the array contains the value, and false otherwise. +#' @details +#' \code{array_contains}: Returns null if the array is null, true if the array contains +#' the value, and false otherwise. #' -#' @param x A Column #' @param value A value to be checked if contained in the column -#' @rdname array_contains -#' @aliases array_contains,Column-method -#' @name array_contains -#' @family collection functions +#' @rdname column_collection_functions +#' @aliases array_contains array_contains,Column-method #' @export -#' @examples \dontrun{array_contains(df$c, 1)} #' @note array_contains since 1.6.0 setMethod("array_contains", signature(x = "Column", value = "ANY"), @@ -3129,18 +3139,12 @@ setMethod("array_contains", column(jc) }) -#' explode -#' -#' Creates a new row for each element in the given array or map column. -#' -#' @param x Column to compute on +#' @details +#' \code{explode}: Creates a new row for each element in the given array or map column. #' -#' @rdname explode -#' @name explode -#' @family collection functions -#' @aliases explode,Column-method +#' @rdname column_collection_functions +#' @aliases explode explode,Column-method #' @export -#' @examples \dontrun{explode(df$c)} #' @note explode since 1.5.0 setMethod("explode", signature(x = "Column"), @@ -3149,18 +3153,12 @@ setMethod("explode", column(jc) }) -#' size -#' -#' Returns length of array or map. -#' -#' @param x Column to compute on +#' @details +#' \code{size}: Returns length of array or map. #' -#' @rdname size -#' @name size -#' @aliases size,Column-method -#' @family collection functions +#' @rdname column_collection_functions +#' @aliases size size,Column-method #' @export -#' @examples \dontrun{size(df$c)} #' @note size since 1.5.0 setMethod("size", signature(x = "Column"), @@ -3169,25 +3167,16 @@ setMethod("size", column(jc) }) -#' sort_array -#' -#' Sorts the input array in ascending or descending order according +#' @details +#' \code{sort_array}: Sorts the input array in ascending or descending order according #' to the natural ordering of the array elements. #' -#' @param x A Column to sort +#' @rdname column_collection_functions #' @param asc A logical flag indicating the sorting order. #' TRUE, sorting is in ascending order. #' FALSE, sorting is in descending order. -#' @rdname sort_array -#' @name sort_array -#' @aliases sort_array,Column-method -#' @family collection functions +#' @aliases sort_array sort_array,Column-method #' @export -#' @examples -#' \dontrun{ -#' sort_array(df$c) -#' sort_array(df$c, FALSE) -#' } #' @note sort_array since 1.6.0 setMethod("sort_array", signature(x = "Column"), @@ -3196,18 +3185,13 @@ setMethod("sort_array", column(jc) }) -#' posexplode -#' -#' Creates a new row for each element with position in the given array or map column. -#' -#' @param x Column to compute on +#' @details +#' \code{posexplode}: Creates a new row for each element with position in the given array +#' or map column. #' -#' @rdname posexplode -#' @name posexplode -#' @family collection functions -#' @aliases posexplode,Column-method +#' @rdname column_collection_functions +#' @aliases posexplode posexplode,Column-method #' @export -#' @examples \dontrun{posexplode(df$c)} #' @note posexplode since 2.1.0 setMethod("posexplode", signature(x = "Column"), @@ -3333,27 +3317,24 @@ setMethod("repeat_string", column(jc) }) -#' explode_outer -#' -#' Creates a new row for each element in the given array or map column. +#' @details +#' \code{explode}: Creates a new row for each element in the given array or map column. #' Unlike \code{explode}, if the array/map is \code{null} or empty #' then \code{null} is produced. #' -#' @param x Column to compute on #' -#' @rdname explode_outer -#' @name explode_outer -#' @family collection functions -#' @aliases explode_outer,Column-method +#' @rdname column_collection_functions +#' @aliases explode_outer explode_outer,Column-method #' @export #' @examples +#' #' \dontrun{ -#' df <- createDataFrame(data.frame( +#' df2 <- createDataFrame(data.frame( #' id = c(1, 2, 3), text = c("a,b,c", NA, "d,e") #' )) #' -#' head(select(df, df$id, explode_outer(split_string(df$text, ",")))) -#' } +#' head(select(df2, df2$id, explode_outer(split_string(df2$text, ",")))) +#' head(select(df2, df2$id, posexplode_outer(split_string(df2$text, ","))))} #' @note explode_outer since 2.3.0 setMethod("explode_outer", signature(x = "Column"), @@ -3362,27 +3343,14 @@ setMethod("explode_outer", column(jc) }) -#' posexplode_outer -#' -#' Creates a new row for each element with position in the given array or map column. -#' Unlike \code{posexplode}, if the array/map is \code{null} or empty +#' @details +#' \code{posexplode_outer}: Creates a new row for each element with position in the given +#' array or map column. Unlike \code{posexplode}, if the array/map is \code{null} or empty #' then the row (\code{null}, \code{null}) is produced. #' -#' @param x Column to compute on -#' -#' @rdname posexplode_outer -#' @name posexplode_outer -#' @family collection functions -#' @aliases posexplode_outer,Column-method +#' @rdname column_collection_functions +#' @aliases posexplode_outer posexplode_outer,Column-method #' @export -#' @examples -#' \dontrun{ -#' df <- createDataFrame(data.frame( -#' id = c(1, 2, 3), text = c("a,b,c", NA, "d,e") -#' )) -#' -#' head(select(df, df$id, posexplode_outer(split_string(df$text, ",")))) -#' } #' @note posexplode_outer since 2.3.0 setMethod("posexplode_outer", signature(x = "Column"), diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 1deb057bb1b82..b901b74e4728d 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -913,8 +913,9 @@ setGeneric("add_months", function(y, x) { standardGeneric("add_months") }) #' @name NULL setGeneric("approxCountDistinct", function(x, ...) { standardGeneric("approxCountDistinct") }) -#' @rdname array_contains +#' @rdname column_collection_functions #' @export +#' @name NULL setGeneric("array_contains", function(x, value) { standardGeneric("array_contains") }) #' @rdname column_string_functions @@ -992,8 +993,9 @@ setGeneric("conv", function(x, fromBase, toBase) { standardGeneric("conv") }) #' @name NULL setGeneric("countDistinct", function(x, ...) { standardGeneric("countDistinct") }) -#' @rdname crc32 +#' @rdname column_misc_functions #' @export +#' @name NULL setGeneric("crc32", function(x) { standardGeneric("crc32") }) #' @rdname column_nonaggregate_functions @@ -1006,8 +1008,9 @@ setGeneric("create_array", function(x, ...) { standardGeneric("create_array") }) #' @name NULL setGeneric("create_map", function(x, ...) { standardGeneric("create_map") }) -#' @rdname hash +#' @rdname column_misc_functions #' @export +#' @name NULL setGeneric("hash", function(x, ...) { standardGeneric("hash") }) #' @param x empty. Should be used with no argument. @@ -1060,12 +1063,14 @@ setGeneric("dense_rank", function(x = "missing") { standardGeneric("dense_rank") #' @name NULL setGeneric("encode", function(x, charset) { standardGeneric("encode") }) -#' @rdname explode +#' @rdname column_collection_functions #' @export +#' @name NULL setGeneric("explode", function(x) { standardGeneric("explode") }) -#' @rdname explode_outer +#' @rdname column_collection_functions #' @export +#' @name NULL setGeneric("explode_outer", function(x) { standardGeneric("explode_outer") }) #' @rdname column_nonaggregate_functions @@ -1088,8 +1093,9 @@ setGeneric("format_number", function(y, x) { standardGeneric("format_number") }) #' @name NULL setGeneric("format_string", function(format, x, ...) { standardGeneric("format_string") }) -#' @rdname from_json +#' @rdname column_collection_functions #' @export +#' @name NULL setGeneric("from_json", function(x, schema, ...) { standardGeneric("from_json") }) #' @rdname column_datetime_functions @@ -1205,8 +1211,9 @@ setGeneric("lpad", function(x, len, pad) { standardGeneric("lpad") }) #' @name NULL setGeneric("ltrim", function(x) { standardGeneric("ltrim") }) -#' @rdname md5 +#' @rdname column_misc_functions #' @export +#' @name NULL setGeneric("md5", function(x) { standardGeneric("md5") }) #' @rdname column_datetime_functions @@ -1272,12 +1279,14 @@ setGeneric("percent_rank", function(x = "missing") { standardGeneric("percent_ra #' @name NULL setGeneric("pmod", function(y, x) { standardGeneric("pmod") }) -#' @rdname posexplode +#' @rdname column_collection_functions #' @export +#' @name NULL setGeneric("posexplode", function(x) { standardGeneric("posexplode") }) -#' @rdname posexplode_outer +#' @rdname column_collection_functions #' @export +#' @name NULL setGeneric("posexplode_outer", function(x) { standardGeneric("posexplode_outer") }) #' @rdname column_datetime_functions @@ -1350,12 +1359,14 @@ setGeneric("sd", function(x, na.rm = FALSE) { standardGeneric("sd") }) #' @name NULL setGeneric("second", function(x) { standardGeneric("second") }) -#' @rdname sha1 +#' @rdname column_misc_functions #' @export +#' @name NULL setGeneric("sha1", function(x) { standardGeneric("sha1") }) -#' @rdname sha2 +#' @rdname column_misc_functions #' @export +#' @name NULL setGeneric("sha2", function(y, x) { standardGeneric("sha2") }) #' @rdname column_math_functions @@ -1378,8 +1389,9 @@ setGeneric("shiftRightUnsigned", function(y, x) { standardGeneric("shiftRightUns #' @name NULL setGeneric("signum", function(x) { standardGeneric("signum") }) -#' @rdname size +#' @rdname column_collection_functions #' @export +#' @name NULL setGeneric("size", function(x) { standardGeneric("size") }) #' @rdname column_aggregate_functions @@ -1387,8 +1399,9 @@ setGeneric("size", function(x) { standardGeneric("size") }) #' @name NULL setGeneric("skewness", function(x) { standardGeneric("skewness") }) -#' @rdname sort_array +#' @rdname column_collection_functions #' @export +#' @name NULL setGeneric("sort_array", function(x, asc = TRUE) { standardGeneric("sort_array") }) #' @rdname column_string_functions @@ -1451,8 +1464,9 @@ setGeneric("toRadians", function(x) { standardGeneric("toRadians") }) #' @name NULL setGeneric("to_date", function(x, format) { standardGeneric("to_date") }) -#' @rdname to_json +#' @rdname column_collection_functions #' @export +#' @name NULL setGeneric("to_json", function(x, ...) { standardGeneric("to_json") }) #' @rdname column_datetime_functions diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java index a6f527c118218..8f354ad78bbaa 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java @@ -179,7 +179,7 @@ public void stream(String streamId, StreamCallback callback) { // written to the socket atomically, so that callbacks are called in the right order // when responses arrive. synchronized (this) { - handler.addStreamCallback(callback); + handler.addStreamCallback(streamId, callback); channel.writeAndFlush(new StreamRequest(streamId)).addListener(future -> { if (future.isSuccess()) { long timeTaken = System.currentTimeMillis() - startTime; diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java index 41bead546cad6..340b8b96aabc6 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java @@ -24,6 +24,8 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicLong; +import scala.Tuple2; + import com.google.common.annotations.VisibleForTesting; import io.netty.channel.Channel; import org.slf4j.Logger; @@ -56,7 +58,7 @@ public class TransportResponseHandler extends MessageHandler { private final Map outstandingRpcs; - private final Queue streamCallbacks; + private final Queue> streamCallbacks; private volatile boolean streamActive; /** Records the time (in system nanoseconds) that the last fetch or RPC request was sent. */ @@ -88,9 +90,9 @@ public void removeRpcRequest(long requestId) { outstandingRpcs.remove(requestId); } - public void addStreamCallback(StreamCallback callback) { + public void addStreamCallback(String streamId, StreamCallback callback) { timeOfLastRequestNs.set(System.nanoTime()); - streamCallbacks.offer(callback); + streamCallbacks.offer(new Tuple2<>(streamId, callback)); } @VisibleForTesting @@ -104,15 +106,31 @@ public void deactivateStream() { */ private void failOutstandingRequests(Throwable cause) { for (Map.Entry entry : outstandingFetches.entrySet()) { - entry.getValue().onFailure(entry.getKey().chunkIndex, cause); + try { + entry.getValue().onFailure(entry.getKey().chunkIndex, cause); + } catch (Exception e) { + logger.warn("ChunkReceivedCallback.onFailure throws exception", e); + } } for (Map.Entry entry : outstandingRpcs.entrySet()) { - entry.getValue().onFailure(cause); + try { + entry.getValue().onFailure(cause); + } catch (Exception e) { + logger.warn("RpcResponseCallback.onFailure throws exception", e); + } + } + for (Tuple2 entry : streamCallbacks) { + try { + entry._2().onFailure(entry._1(), cause); + } catch (Exception e) { + logger.warn("StreamCallback.onFailure throws exception", e); + } } // It's OK if new fetches appear, as they will fail immediately. outstandingFetches.clear(); outstandingRpcs.clear(); + streamCallbacks.clear(); } @Override @@ -190,8 +208,9 @@ public void handle(ResponseMessage message) throws Exception { } } else if (message instanceof StreamResponse) { StreamResponse resp = (StreamResponse) message; - StreamCallback callback = streamCallbacks.poll(); - if (callback != null) { + Tuple2 entry = streamCallbacks.poll(); + if (entry != null) { + StreamCallback callback = entry._2(); if (resp.byteCount > 0) { StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount, callback); @@ -216,8 +235,9 @@ public void handle(ResponseMessage message) throws Exception { } } else if (message instanceof StreamFailure) { StreamFailure resp = (StreamFailure) message; - StreamCallback callback = streamCallbacks.poll(); - if (callback != null) { + Tuple2 entry = streamCallbacks.poll(); + if (entry != null) { + StreamCallback callback = entry._2(); try { callback.onFailure(resp.streamId, new RuntimeException(resp.error)); } catch (IOException ioe) { diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java index 09fc80d12d510..b4032c4c3f031 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java @@ -17,6 +17,7 @@ package org.apache.spark.network; +import java.io.IOException; import java.nio.ByteBuffer; import io.netty.channel.Channel; @@ -127,7 +128,7 @@ public void testActiveStreams() throws Exception { StreamResponse response = new StreamResponse("stream", 1234L, null); StreamCallback cb = mock(StreamCallback.class); - handler.addStreamCallback(cb); + handler.addStreamCallback("stream", cb); assertEquals(1, handler.numOutstandingRequests()); handler.handle(response); assertEquals(1, handler.numOutstandingRequests()); @@ -135,9 +136,35 @@ public void testActiveStreams() throws Exception { assertEquals(0, handler.numOutstandingRequests()); StreamFailure failure = new StreamFailure("stream", "uh-oh"); - handler.addStreamCallback(cb); + handler.addStreamCallback("stream", cb); assertEquals(1, handler.numOutstandingRequests()); handler.handle(failure); assertEquals(0, handler.numOutstandingRequests()); } + + @Test + public void failOutstandingStreamCallbackOnClose() throws Exception { + Channel c = new LocalChannel(); + c.pipeline().addLast(TransportFrameDecoder.HANDLER_NAME, new TransportFrameDecoder()); + TransportResponseHandler handler = new TransportResponseHandler(c); + + StreamCallback cb = mock(StreamCallback.class); + handler.addStreamCallback("stream-1", cb); + handler.channelInactive(); + + verify(cb).onFailure(eq("stream-1"), isA(IOException.class)); + } + + @Test + public void failOutstandingStreamCallbackOnException() throws Exception { + Channel c = new LocalChannel(); + c.pipeline().addLast(TransportFrameDecoder.HANDLER_NAME, new TransportFrameDecoder()); + TransportResponseHandler handler = new TransportResponseHandler(c); + + StreamCallback cb = mock(StreamCallback.class); + handler.addStreamCallback("stream-1", cb); + handler.exceptionCaught(new IOException("Oops!")); + + verify(cb).onFailure(eq("stream-1"), isA(IOException.class)); + } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 40b9fc9534f44..9de4ca71ff6d4 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -1088,6 +1088,12 @@ public UTF8String clone() { return fromBytes(getBytes()); } + public UTF8String copy() { + byte[] bytes = new byte[numBytes]; + copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, numBytes); + return fromBytes(bytes); + } + @Override public int compareTo(@Nonnull final UTF8String other) { int len = Math.min(numBytes, other.numBytes); diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 4bef21b6b4e4d..3b6200e74f1e1 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -160,14 +160,10 @@ public final class BytesToBytesMap extends MemoryConsumer { private final boolean enablePerfMetrics; - private long timeSpentResizingNs = 0; - private long numProbes = 0; private long numKeyLookups = 0; - private long numHashCollisions = 0; - private long peakMemoryUsedBytes = 0L; private final int initialCapacity; @@ -489,10 +485,6 @@ public void safeLookup(Object keyBase, long keyOffset, int keyLength, Location l ); if (areEqual) { return; - } else { - if (enablePerfMetrics) { - numHashCollisions++; - } } } } @@ -859,16 +851,6 @@ public long getPeakMemoryUsedBytes() { return peakMemoryUsedBytes; } - /** - * Returns the total amount of time spent resizing this map (in nanoseconds). - */ - public long getTimeSpentResizingNs() { - if (!enablePerfMetrics) { - throw new IllegalStateException(); - } - return timeSpentResizingNs; - } - /** * Returns the average number of probes per key lookup. */ @@ -879,13 +861,6 @@ public double getAverageProbesPerLookup() { return (1.0 * numProbes) / numKeyLookups; } - public long getNumHashCollisions() { - if (!enablePerfMetrics) { - throw new IllegalStateException(); - } - return numHashCollisions; - } - @VisibleForTesting public int getNumDataPages() { return dataPages.size(); @@ -923,10 +898,6 @@ public void reset() { void growAndRehash() { assert(longArray != null); - long resizeStartTime = -1; - if (enablePerfMetrics) { - resizeStartTime = System.nanoTime(); - } // Store references to the old data structures to be used when we re-hash final LongArray oldLongArray = longArray; final int oldCapacity = (int) oldLongArray.size() / 2; @@ -951,9 +922,5 @@ void growAndRehash() { longArray.set(newPos * 2 + 1, hashcode); } freeArray(oldLongArray); - - if (enablePerfMetrics) { - timeSpentResizingNs += System.nanoTime() - resizeStartTime; - } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index d05ca142b618b..b2a50bd055712 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.history import java.io.{FileNotFoundException, IOException, OutputStream} import java.util.UUID -import java.util.concurrent.{Executors, ExecutorService, Future, TimeUnit} +import java.util.concurrent.{ConcurrentHashMap, Executors, ExecutorService, Future, TimeUnit} import java.util.zip.{ZipEntry, ZipOutputStream} import scala.collection.mutable @@ -122,7 +122,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) @volatile private var applications: mutable.LinkedHashMap[String, FsApplicationHistoryInfo] = new mutable.LinkedHashMap() - val fileToAppInfo = new mutable.HashMap[Path, FsApplicationAttemptInfo]() + val fileToAppInfo = new ConcurrentHashMap[Path, FsApplicationAttemptInfo]() // List of application logs to be deleted by event log cleaner. private var attemptsToClean = new mutable.ListBuffer[FsApplicationAttemptInfo] @@ -321,7 +321,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // scan for modified applications, replay and merge them val logInfos: Seq[FileStatus] = statusList .filter { entry => - val prevFileSize = fileToAppInfo.get(entry.getPath()).map{_.fileSize}.getOrElse(0L) + val fileInfo = fileToAppInfo.get(entry.getPath()) + val prevFileSize = if (fileInfo != null) fileInfo.fileSize else 0L !entry.isDirectory() && // FsHistoryProvider generates a hidden file which can't be read. Accidentally // reading a garbage file is safe, but we would log an error which can be scary to @@ -475,7 +476,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) fileStatus.getLen(), appListener.appSparkVersion.getOrElse("") ) - fileToAppInfo(logPath) = attemptInfo + fileToAppInfo.put(logPath, attemptInfo) logDebug(s"Application log ${attemptInfo.logPath} loaded successfully: $attemptInfo") Some(attemptInfo) } else { diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index be63c637a3a13..8dee0d970c4c6 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -323,10 +323,11 @@ package object config { private[spark] val REDUCER_MAX_REQ_SIZE_SHUFFLE_TO_MEM = ConfigBuilder("spark.reducer.maxReqSizeShuffleToMem") + .internal() .doc("The blocks of a shuffle request will be fetched to disk when size of the request is " + "above this threshold. This is to avoid a giant request takes too much memory.") .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("200m") + .createWithDefault(Long.MaxValue) private[spark] val TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES = ConfigBuilder("spark.taskMetrics.trackUpdatedBlockStatuses") diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapRedCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapRedCommitProtocol.scala new file mode 100644 index 0000000000000..ddbd624b380d4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapRedCommitProtocol.scala @@ -0,0 +1,36 @@ +/* + * 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.internal.io + +import org.apache.hadoop.mapred._ +import org.apache.hadoop.mapreduce.{TaskAttemptContext => NewTaskAttemptContext} + +/** + * An [[FileCommitProtocol]] implementation backed by an underlying Hadoop OutputCommitter + * (from the old mapred API). + * + * Unlike Hadoop's OutputCommitter, this implementation is serializable. + */ +class HadoopMapRedCommitProtocol(jobId: String, path: String) + extends HadoopMapReduceCommitProtocol(jobId, path) { + + override def setupCommitter(context: NewTaskAttemptContext): OutputCommitter = { + val config = context.getConfiguration.asInstanceOf[JobConf] + config.getOutputCommitter + } +} diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopWriteConfigUtil.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopWriteConfigUtil.scala new file mode 100644 index 0000000000000..9b987e0e1bb67 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopWriteConfigUtil.scala @@ -0,0 +1,79 @@ +/* + * 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.internal.io + +import scala.reflect.ClassTag + +import org.apache.hadoop.mapreduce._ + +import org.apache.spark.SparkConf + +/** + * Interface for create output format/committer/writer used during saving an RDD using a Hadoop + * OutputFormat (both from the old mapred API and the new mapreduce API) + * + * Notes: + * 1. Implementations should throw [[IllegalArgumentException]] when wrong hadoop API is + * referenced; + * 2. Implementations must be serializable, as the instance instantiated on the driver + * will be used for tasks on executors; + * 3. Implementations should have a constructor with exactly one argument: + * (conf: SerializableConfiguration) or (conf: SerializableJobConf). + */ +abstract class HadoopWriteConfigUtil[K, V: ClassTag] extends Serializable { + + // -------------------------------------------------------------------------- + // Create JobContext/TaskAttemptContext + // -------------------------------------------------------------------------- + + def createJobContext(jobTrackerId: String, jobId: Int): JobContext + + def createTaskAttemptContext( + jobTrackerId: String, + jobId: Int, + splitId: Int, + taskAttemptId: Int): TaskAttemptContext + + // -------------------------------------------------------------------------- + // Create committer + // -------------------------------------------------------------------------- + + def createCommitter(jobId: Int): HadoopMapReduceCommitProtocol + + // -------------------------------------------------------------------------- + // Create writer + // -------------------------------------------------------------------------- + + def initWriter(taskContext: TaskAttemptContext, splitId: Int): Unit + + def write(pair: (K, V)): Unit + + def closeWriter(taskContext: TaskAttemptContext): Unit + + // -------------------------------------------------------------------------- + // Create OutputFormat + // -------------------------------------------------------------------------- + + def initOutputFormat(jobContext: JobContext): Unit + + // -------------------------------------------------------------------------- + // Verify hadoop config + // -------------------------------------------------------------------------- + + def assertConf(jobContext: JobContext, conf: SparkConf): Unit +} diff --git a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopMapReduceWriter.scala b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopMapReduceWriter.scala deleted file mode 100644 index 376ff9bb19f74..0000000000000 --- a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopMapReduceWriter.scala +++ /dev/null @@ -1,181 +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.internal.io - -import java.text.SimpleDateFormat -import java.util.{Date, Locale} - -import scala.reflect.ClassTag -import scala.util.DynamicVariable - -import org.apache.hadoop.conf.{Configurable, Configuration} -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapred.{JobConf, JobID} -import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl - -import org.apache.spark.{SparkConf, SparkException, TaskContext} -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.OutputMetrics -import org.apache.spark.internal.Logging -import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage -import org.apache.spark.rdd.RDD -import org.apache.spark.util.{SerializableConfiguration, Utils} - -/** - * A helper object that saves an RDD using a Hadoop OutputFormat - * (from the newer mapreduce API, not the old mapred API). - */ -private[spark] -object SparkHadoopMapReduceWriter extends Logging { - - /** - * Basic work flow of this command is: - * 1. Driver side setup, prepare the data source and hadoop configuration for the write job to - * be issued. - * 2. Issues a write job consists of one or more executor side tasks, each of which writes all - * rows within an RDD partition. - * 3. If no exception is thrown in a task, commits that task, otherwise aborts that task; If any - * exception is thrown during task commitment, also aborts that task. - * 4. If all tasks are committed, commit the job, otherwise aborts the job; If any exception is - * thrown during job commitment, also aborts the job. - */ - def write[K, V: ClassTag]( - rdd: RDD[(K, V)], - hadoopConf: Configuration): Unit = { - // Extract context and configuration from RDD. - val sparkContext = rdd.context - val stageId = rdd.id - val sparkConf = rdd.conf - val conf = new SerializableConfiguration(hadoopConf) - - // Set up a job. - val jobTrackerId = SparkHadoopWriterUtils.createJobTrackerID(new Date()) - val jobAttemptId = new TaskAttemptID(jobTrackerId, stageId, TaskType.MAP, 0, 0) - val jobContext = new TaskAttemptContextImpl(conf.value, jobAttemptId) - val format = jobContext.getOutputFormatClass - - if (SparkHadoopWriterUtils.isOutputSpecValidationEnabled(sparkConf)) { - // FileOutputFormat ignores the filesystem parameter - val jobFormat = format.newInstance - jobFormat.checkOutputSpecs(jobContext) - } - - val committer = FileCommitProtocol.instantiate( - className = classOf[HadoopMapReduceCommitProtocol].getName, - jobId = stageId.toString, - outputPath = conf.value.get("mapreduce.output.fileoutputformat.outputdir"), - isAppend = false).asInstanceOf[HadoopMapReduceCommitProtocol] - committer.setupJob(jobContext) - - // Try to write all RDD partitions as a Hadoop OutputFormat. - try { - val ret = sparkContext.runJob(rdd, (context: TaskContext, iter: Iterator[(K, V)]) => { - executeTask( - context = context, - jobTrackerId = jobTrackerId, - sparkStageId = context.stageId, - sparkPartitionId = context.partitionId, - sparkAttemptNumber = context.attemptNumber, - committer = committer, - hadoopConf = conf.value, - outputFormat = format.asInstanceOf[Class[OutputFormat[K, V]]], - iterator = iter) - }) - - committer.commitJob(jobContext, ret) - logInfo(s"Job ${jobContext.getJobID} committed.") - } catch { - case cause: Throwable => - logError(s"Aborting job ${jobContext.getJobID}.", cause) - committer.abortJob(jobContext) - throw new SparkException("Job aborted.", cause) - } - } - - /** Write an RDD partition out in a single Spark task. */ - private def executeTask[K, V: ClassTag]( - context: TaskContext, - jobTrackerId: String, - sparkStageId: Int, - sparkPartitionId: Int, - sparkAttemptNumber: Int, - committer: FileCommitProtocol, - hadoopConf: Configuration, - outputFormat: Class[_ <: OutputFormat[K, V]], - iterator: Iterator[(K, V)]): TaskCommitMessage = { - // Set up a task. - val attemptId = new TaskAttemptID(jobTrackerId, sparkStageId, TaskType.REDUCE, - sparkPartitionId, sparkAttemptNumber) - val taskContext = new TaskAttemptContextImpl(hadoopConf, attemptId) - committer.setupTask(taskContext) - - val (outputMetrics, callback) = SparkHadoopWriterUtils.initHadoopOutputMetrics(context) - - // Initiate the writer. - val taskFormat = outputFormat.newInstance() - // If OutputFormat is Configurable, we should set conf to it. - taskFormat match { - case c: Configurable => c.setConf(hadoopConf) - case _ => () - } - var writer = taskFormat.getRecordWriter(taskContext) - .asInstanceOf[RecordWriter[K, V]] - require(writer != null, "Unable to obtain RecordWriter") - var recordsWritten = 0L - - // Write all rows in RDD partition. - try { - val ret = Utils.tryWithSafeFinallyAndFailureCallbacks { - // Write rows out, release resource and commit the task. - while (iterator.hasNext) { - val pair = iterator.next() - writer.write(pair._1, pair._2) - - // Update bytes written metric every few records - SparkHadoopWriterUtils.maybeUpdateOutputMetrics(outputMetrics, callback, recordsWritten) - recordsWritten += 1 - } - if (writer != null) { - writer.close(taskContext) - writer = null - } - committer.commitTask(taskContext) - }(catchBlock = { - // If there is an error, release resource and then abort the task. - try { - if (writer != null) { - writer.close(taskContext) - writer = null - } - } finally { - committer.abortTask(taskContext) - logError(s"Task ${taskContext.getTaskAttemptID} aborted.") - } - }) - - outputMetrics.setBytesWritten(callback()) - outputMetrics.setRecordsWritten(recordsWritten) - - ret - } catch { - case t: Throwable => - throw new SparkException("Task failed while writing rows", t) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala index acc9c38571007..7d846f9354df6 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala @@ -17,143 +17,374 @@ package org.apache.spark.internal.io -import java.io.IOException -import java.text.{NumberFormat, SimpleDateFormat} +import java.text.NumberFormat import java.util.{Date, Locale} +import scala.reflect.ClassTag + +import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.mapred._ -import org.apache.hadoop.mapreduce.TaskType +import org.apache.hadoop.mapreduce.{JobContext => NewJobContext, +OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, +TaskAttemptContext => NewTaskAttemptContext, TaskAttemptID => NewTaskAttemptID, TaskType} +import org.apache.hadoop.mapreduce.task.{TaskAttemptContextImpl => NewTaskAttemptContextImpl} -import org.apache.spark.SerializableWritable +import org.apache.spark.{SerializableWritable, SparkConf, SparkException, TaskContext} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging -import org.apache.spark.mapred.SparkHadoopMapRedUtil -import org.apache.spark.rdd.HadoopRDD -import org.apache.spark.util.SerializableJobConf +import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage +import org.apache.spark.rdd.{HadoopRDD, RDD} +import org.apache.spark.util.{SerializableConfiguration, SerializableJobConf, Utils} /** - * Internal helper class that saves an RDD using a Hadoop OutputFormat. - * - * Saves the RDD using a JobConf, which should contain an output key class, an output value class, - * a filename to write to, etc, exactly like in a Hadoop MapReduce job. + * A helper object that saves an RDD using a Hadoop OutputFormat. + */ +private[spark] +object SparkHadoopWriter extends Logging { + import SparkHadoopWriterUtils._ + + /** + * Basic work flow of this command is: + * 1. Driver side setup, prepare the data source and hadoop configuration for the write job to + * be issued. + * 2. Issues a write job consists of one or more executor side tasks, each of which writes all + * rows within an RDD partition. + * 3. If no exception is thrown in a task, commits that task, otherwise aborts that task; If any + * exception is thrown during task commitment, also aborts that task. + * 4. If all tasks are committed, commit the job, otherwise aborts the job; If any exception is + * thrown during job commitment, also aborts the job. + */ + def write[K, V: ClassTag]( + rdd: RDD[(K, V)], + config: HadoopWriteConfigUtil[K, V]): Unit = { + // Extract context and configuration from RDD. + val sparkContext = rdd.context + val stageId = rdd.id + + // Set up a job. + val jobTrackerId = createJobTrackerID(new Date()) + val jobContext = config.createJobContext(jobTrackerId, stageId) + config.initOutputFormat(jobContext) + + // Assert the output format/key/value class is set in JobConf. + config.assertConf(jobContext, rdd.conf) + + val committer = config.createCommitter(stageId) + committer.setupJob(jobContext) + + // Try to write all RDD partitions as a Hadoop OutputFormat. + try { + val ret = sparkContext.runJob(rdd, (context: TaskContext, iter: Iterator[(K, V)]) => { + executeTask( + context = context, + config = config, + jobTrackerId = jobTrackerId, + sparkStageId = context.stageId, + sparkPartitionId = context.partitionId, + sparkAttemptNumber = context.attemptNumber, + committer = committer, + iterator = iter) + }) + + committer.commitJob(jobContext, ret) + logInfo(s"Job ${jobContext.getJobID} committed.") + } catch { + case cause: Throwable => + logError(s"Aborting job ${jobContext.getJobID}.", cause) + committer.abortJob(jobContext) + throw new SparkException("Job aborted.", cause) + } + } + + /** Write a RDD partition out in a single Spark task. */ + private def executeTask[K, V: ClassTag]( + context: TaskContext, + config: HadoopWriteConfigUtil[K, V], + jobTrackerId: String, + sparkStageId: Int, + sparkPartitionId: Int, + sparkAttemptNumber: Int, + committer: FileCommitProtocol, + iterator: Iterator[(K, V)]): TaskCommitMessage = { + // Set up a task. + val taskContext = config.createTaskAttemptContext( + jobTrackerId, sparkStageId, sparkPartitionId, sparkAttemptNumber) + committer.setupTask(taskContext) + + val (outputMetrics, callback) = initHadoopOutputMetrics(context) + + // Initiate the writer. + config.initWriter(taskContext, sparkPartitionId) + var recordsWritten = 0L + + // Write all rows in RDD partition. + try { + val ret = Utils.tryWithSafeFinallyAndFailureCallbacks { + while (iterator.hasNext) { + val pair = iterator.next() + config.write(pair) + + // Update bytes written metric every few records + maybeUpdateOutputMetrics(outputMetrics, callback, recordsWritten) + recordsWritten += 1 + } + + config.closeWriter(taskContext) + committer.commitTask(taskContext) + }(catchBlock = { + // If there is an error, release resource and then abort the task. + try { + config.closeWriter(taskContext) + } finally { + committer.abortTask(taskContext) + logError(s"Task ${taskContext.getTaskAttemptID} aborted.") + } + }) + + outputMetrics.setBytesWritten(callback()) + outputMetrics.setRecordsWritten(recordsWritten) + + ret + } catch { + case t: Throwable => + throw new SparkException("Task failed while writing rows", t) + } + } +} + +/** + * A helper class that reads JobConf from older mapred API, creates output Format/Committer/Writer. */ private[spark] -class SparkHadoopWriter(jobConf: JobConf) extends Logging with Serializable { +class HadoopMapRedWriteConfigUtil[K, V: ClassTag](conf: SerializableJobConf) + extends HadoopWriteConfigUtil[K, V] with Logging { - private val now = new Date() - private val conf = new SerializableJobConf(jobConf) + private var outputFormat: Class[_ <: OutputFormat[K, V]] = null + private var writer: RecordWriter[K, V] = null - private var jobID = 0 - private var splitID = 0 - private var attemptID = 0 - private var jID: SerializableWritable[JobID] = null - private var taID: SerializableWritable[TaskAttemptID] = null + private def getConf: JobConf = conf.value - @transient private var writer: RecordWriter[AnyRef, AnyRef] = null - @transient private var format: OutputFormat[AnyRef, AnyRef] = null - @transient private var committer: OutputCommitter = null - @transient private var jobContext: JobContext = null - @transient private var taskContext: TaskAttemptContext = null + // -------------------------------------------------------------------------- + // Create JobContext/TaskAttemptContext + // -------------------------------------------------------------------------- - def preSetup() { - setIDs(0, 0, 0) - HadoopRDD.addLocalConfiguration("", 0, 0, 0, conf.value) + override def createJobContext(jobTrackerId: String, jobId: Int): NewJobContext = { + val jobAttemptId = new SerializableWritable(new JobID(jobTrackerId, jobId)) + new JobContextImpl(getConf, jobAttemptId.value) + } - val jCtxt = getJobContext() - getOutputCommitter().setupJob(jCtxt) + override def createTaskAttemptContext( + jobTrackerId: String, + jobId: Int, + splitId: Int, + taskAttemptId: Int): NewTaskAttemptContext = { + // Update JobConf. + HadoopRDD.addLocalConfiguration(jobTrackerId, jobId, splitId, taskAttemptId, conf.value) + // Create taskContext. + val attemptId = new TaskAttemptID(jobTrackerId, jobId, TaskType.MAP, splitId, taskAttemptId) + new TaskAttemptContextImpl(getConf, attemptId) } + // -------------------------------------------------------------------------- + // Create committer + // -------------------------------------------------------------------------- - def setup(jobid: Int, splitid: Int, attemptid: Int) { - setIDs(jobid, splitid, attemptid) - HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmmss", Locale.US).format(now), - jobid, splitID, attemptID, conf.value) + override def createCommitter(jobId: Int): HadoopMapReduceCommitProtocol = { + // Update JobConf. + HadoopRDD.addLocalConfiguration("", 0, 0, 0, getConf) + // Create commit protocol. + FileCommitProtocol.instantiate( + className = classOf[HadoopMapRedCommitProtocol].getName, + jobId = jobId.toString, + outputPath = getConf.get("mapred.output.dir"), + isAppend = false).asInstanceOf[HadoopMapReduceCommitProtocol] } - def open() { + // -------------------------------------------------------------------------- + // Create writer + // -------------------------------------------------------------------------- + + override def initWriter(taskContext: NewTaskAttemptContext, splitId: Int): Unit = { val numfmt = NumberFormat.getInstance(Locale.US) numfmt.setMinimumIntegerDigits(5) numfmt.setGroupingUsed(false) - val outputName = "part-" + numfmt.format(splitID) - val path = FileOutputFormat.getOutputPath(conf.value) + val outputName = "part-" + numfmt.format(splitId) + val path = FileOutputFormat.getOutputPath(getConf) val fs: FileSystem = { if (path != null) { - path.getFileSystem(conf.value) + path.getFileSystem(getConf) } else { - FileSystem.get(conf.value) + FileSystem.get(getConf) } } - getOutputCommitter().setupTask(getTaskContext()) - writer = getOutputFormat().getRecordWriter(fs, conf.value, outputName, Reporter.NULL) + writer = getConf.getOutputFormat + .getRecordWriter(fs, getConf, outputName, Reporter.NULL) + .asInstanceOf[RecordWriter[K, V]] + + require(writer != null, "Unable to obtain RecordWriter") } - def write(key: AnyRef, value: AnyRef) { + override def write(pair: (K, V)): Unit = { + require(writer != null, "Must call createWriter before write.") + writer.write(pair._1, pair._2) + } + + override def closeWriter(taskContext: NewTaskAttemptContext): Unit = { if (writer != null) { - writer.write(key, value) - } else { - throw new IOException("Writer is null, open() has not been called") + writer.close(Reporter.NULL) } } - def close() { - writer.close(Reporter.NULL) - } + // -------------------------------------------------------------------------- + // Create OutputFormat + // -------------------------------------------------------------------------- - def commit() { - SparkHadoopMapRedUtil.commitTask(getOutputCommitter(), getTaskContext(), jobID, splitID) + override def initOutputFormat(jobContext: NewJobContext): Unit = { + if (outputFormat == null) { + outputFormat = getConf.getOutputFormat.getClass + .asInstanceOf[Class[_ <: OutputFormat[K, V]]] + } } - def commitJob() { - val cmtr = getOutputCommitter() - cmtr.commitJob(getJobContext()) + private def getOutputFormat(): OutputFormat[K, V] = { + require(outputFormat != null, "Must call initOutputFormat first.") + + outputFormat.newInstance() } - // ********* Private Functions ********* + // -------------------------------------------------------------------------- + // Verify hadoop config + // -------------------------------------------------------------------------- + + override def assertConf(jobContext: NewJobContext, conf: SparkConf): Unit = { + val outputFormatInstance = getOutputFormat() + val keyClass = getConf.getOutputKeyClass + val valueClass = getConf.getOutputValueClass + if (outputFormatInstance == null) { + throw new SparkException("Output format class not set") + } + if (keyClass == null) { + throw new SparkException("Output key class not set") + } + if (valueClass == null) { + throw new SparkException("Output value class not set") + } + SparkHadoopUtil.get.addCredentials(getConf) + + logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + + valueClass.getSimpleName + ")") - private def getOutputFormat(): OutputFormat[AnyRef, AnyRef] = { - if (format == null) { - format = conf.value.getOutputFormat() - .asInstanceOf[OutputFormat[AnyRef, AnyRef]] + if (SparkHadoopWriterUtils.isOutputSpecValidationEnabled(conf)) { + // FileOutputFormat ignores the filesystem parameter + val ignoredFs = FileSystem.get(getConf) + getOutputFormat().checkOutputSpecs(ignoredFs, getConf) } - format + } +} + +/** + * A helper class that reads Configuration from newer mapreduce API, creates output + * Format/Committer/Writer. + */ +private[spark] +class HadoopMapReduceWriteConfigUtil[K, V: ClassTag](conf: SerializableConfiguration) + extends HadoopWriteConfigUtil[K, V] with Logging { + + private var outputFormat: Class[_ <: NewOutputFormat[K, V]] = null + private var writer: NewRecordWriter[K, V] = null + + private def getConf: Configuration = conf.value + + // -------------------------------------------------------------------------- + // Create JobContext/TaskAttemptContext + // -------------------------------------------------------------------------- + + override def createJobContext(jobTrackerId: String, jobId: Int): NewJobContext = { + val jobAttemptId = new NewTaskAttemptID(jobTrackerId, jobId, TaskType.MAP, 0, 0) + new NewTaskAttemptContextImpl(getConf, jobAttemptId) + } + + override def createTaskAttemptContext( + jobTrackerId: String, + jobId: Int, + splitId: Int, + taskAttemptId: Int): NewTaskAttemptContext = { + val attemptId = new NewTaskAttemptID( + jobTrackerId, jobId, TaskType.REDUCE, splitId, taskAttemptId) + new NewTaskAttemptContextImpl(getConf, attemptId) + } + + // -------------------------------------------------------------------------- + // Create committer + // -------------------------------------------------------------------------- + + override def createCommitter(jobId: Int): HadoopMapReduceCommitProtocol = { + FileCommitProtocol.instantiate( + className = classOf[HadoopMapReduceCommitProtocol].getName, + jobId = jobId.toString, + outputPath = getConf.get("mapreduce.output.fileoutputformat.outputdir"), + isAppend = false).asInstanceOf[HadoopMapReduceCommitProtocol] } - private def getOutputCommitter(): OutputCommitter = { - if (committer == null) { - committer = conf.value.getOutputCommitter + // -------------------------------------------------------------------------- + // Create writer + // -------------------------------------------------------------------------- + + override def initWriter(taskContext: NewTaskAttemptContext, splitId: Int): Unit = { + val taskFormat = getOutputFormat() + // If OutputFormat is Configurable, we should set conf to it. + taskFormat match { + case c: Configurable => c.setConf(getConf) + case _ => () } - committer + + writer = taskFormat.getRecordWriter(taskContext) + .asInstanceOf[NewRecordWriter[K, V]] + + require(writer != null, "Unable to obtain RecordWriter") + } + + override def write(pair: (K, V)): Unit = { + require(writer != null, "Must call createWriter before write.") + writer.write(pair._1, pair._2) } - private def getJobContext(): JobContext = { - if (jobContext == null) { - jobContext = new JobContextImpl(conf.value, jID.value) + override def closeWriter(taskContext: NewTaskAttemptContext): Unit = { + if (writer != null) { + writer.close(taskContext) + writer = null + } else { + logWarning("Writer has been closed.") } - jobContext } - private def getTaskContext(): TaskAttemptContext = { - if (taskContext == null) { - taskContext = newTaskAttemptContext(conf.value, taID.value) + // -------------------------------------------------------------------------- + // Create OutputFormat + // -------------------------------------------------------------------------- + + override def initOutputFormat(jobContext: NewJobContext): Unit = { + if (outputFormat == null) { + outputFormat = jobContext.getOutputFormatClass + .asInstanceOf[Class[_ <: NewOutputFormat[K, V]]] } - taskContext } - protected def newTaskAttemptContext( - conf: JobConf, - attemptId: TaskAttemptID): TaskAttemptContext = { - new TaskAttemptContextImpl(conf, attemptId) + private def getOutputFormat(): NewOutputFormat[K, V] = { + require(outputFormat != null, "Must call initOutputFormat first.") + + outputFormat.newInstance() } - private def setIDs(jobid: Int, splitid: Int, attemptid: Int) { - jobID = jobid - splitID = splitid - attemptID = attemptid + // -------------------------------------------------------------------------- + // Verify hadoop config + // -------------------------------------------------------------------------- - jID = new SerializableWritable[JobID](SparkHadoopWriterUtils.createJobID(now, jobid)) - taID = new SerializableWritable[TaskAttemptID]( - new TaskAttemptID(new TaskID(jID.value, TaskType.MAP, splitID), attemptID)) + override def assertConf(jobContext: NewJobContext, conf: SparkConf): Unit = { + if (SparkHadoopWriterUtils.isOutputSpecValidationEnabled(conf)) { + getOutputFormat().checkOutputSpecs(jobContext) + } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 58762cc0838cd..4628fa8ba270e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -27,7 +27,6 @@ import scala.reflect.ClassTag import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} @@ -36,13 +35,11 @@ import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewO import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.annotation.Experimental -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.io.{SparkHadoopMapReduceWriter, SparkHadoopWriter, - SparkHadoopWriterUtils} +import org.apache.spark.internal.io._ import org.apache.spark.internal.Logging import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer -import org.apache.spark.util.Utils +import org.apache.spark.util.{SerializableConfiguration, SerializableJobConf, Utils} import org.apache.spark.util.collection.CompactBuffer import org.apache.spark.util.random.StratifiedSamplingUtils @@ -1082,9 +1079,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * result of using direct output committer with speculation enabled. */ def saveAsNewAPIHadoopDataset(conf: Configuration): Unit = self.withScope { - SparkHadoopMapReduceWriter.write( + val config = new HadoopMapReduceWriteConfigUtil[K, V](new SerializableConfiguration(conf)) + SparkHadoopWriter.write( rdd = self, - hadoopConf = conf) + config = config) } /** @@ -1094,62 +1092,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * MapReduce job. */ def saveAsHadoopDataset(conf: JobConf): Unit = self.withScope { - // Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038). - val hadoopConf = conf - val outputFormatInstance = hadoopConf.getOutputFormat - val keyClass = hadoopConf.getOutputKeyClass - val valueClass = hadoopConf.getOutputValueClass - if (outputFormatInstance == null) { - throw new SparkException("Output format class not set") - } - if (keyClass == null) { - throw new SparkException("Output key class not set") - } - if (valueClass == null) { - throw new SparkException("Output value class not set") - } - SparkHadoopUtil.get.addCredentials(hadoopConf) - - logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + - valueClass.getSimpleName + ")") - - if (SparkHadoopWriterUtils.isOutputSpecValidationEnabled(self.conf)) { - // FileOutputFormat ignores the filesystem parameter - val ignoredFs = FileSystem.get(hadoopConf) - hadoopConf.getOutputFormat.checkOutputSpecs(ignoredFs, hadoopConf) - } - - val writer = new SparkHadoopWriter(hadoopConf) - writer.preSetup() - - val writeToFile = (context: TaskContext, iter: Iterator[(K, V)]) => { - // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it - // around by taking a mod. We expect that no task will be attempted 2 billion times. - val taskAttemptId = (context.taskAttemptId % Int.MaxValue).toInt - - val (outputMetrics, callback) = SparkHadoopWriterUtils.initHadoopOutputMetrics(context) - - writer.setup(context.stageId, context.partitionId, taskAttemptId) - writer.open() - var recordsWritten = 0L - - Utils.tryWithSafeFinallyAndFailureCallbacks { - while (iter.hasNext) { - val record = iter.next() - writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) - - // Update bytes written metric every few records - SparkHadoopWriterUtils.maybeUpdateOutputMetrics(outputMetrics, callback, recordsWritten) - recordsWritten += 1 - } - }(finallyBlock = writer.close()) - writer.commit() - outputMetrics.setBytesWritten(callback()) - outputMetrics.setRecordsWritten(recordsWritten) - } - - self.context.runJob(self, writeToFile) - writer.commitJob() + val config = new HadoopMapRedWriteConfigUtil[K, V](new SerializableJobConf(conf)) + SparkHadoopWriter.write( + rdd = self, + config = config) } /** diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index edf328b5ae538..b9371c7ad7b45 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -26,6 +26,8 @@ import scala.language.implicitConversions import scala.xml.Node import org.eclipse.jetty.client.api.Response +import org.eclipse.jetty.client.HttpClient +import org.eclipse.jetty.client.http.HttpClientTransportOverHTTP import org.eclipse.jetty.proxy.ProxyServlet import org.eclipse.jetty.server._ import org.eclipse.jetty.server.handler._ @@ -208,6 +210,16 @@ private[spark] object JettyUtils extends Logging { rewrittenURI.toString() } + override def newHttpClient(): HttpClient = { + // SPARK-21176: Use the Jetty logic to calculate the number of selector threads (#CPUs/2), + // but limit it to 8 max. + // Otherwise, it might happen that we exhaust the threadpool since in reverse proxy mode + // a proxy is instantiated for each executor. If the head node has many processors, this + // can quickly add up to an unreasonably high number of threads. + val numSelectors = math.max(1, math.min(8, Runtime.getRuntime().availableProcessors() / 2)) + new HttpClient(new HttpClientTransportOverHTTP(numSelectors), null) + } + override def filterServerResponseHeader( clientRequest: HttpServletRequest, serverResponse: Response, diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 02df157be377c..44dd955ce8690 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -561,7 +561,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { pairs.saveAsHadoopFile( "ignored", pairs.keyClass, pairs.valueClass, classOf[FakeFormatWithCallback], conf) } - assert(e.getMessage contains "failed to write") + assert(e.getCause.getMessage contains "failed to write") assert(FakeWriterWithCallback.calledBy === "write,callback,close") assert(FakeWriterWithCallback.exception != null, "exception should be captured") diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index e51e6a0d3ff6b..1579b614ea5b0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -18,12 +18,14 @@ package org.apache.spark.scheduler import java.io.File +import java.util.Date import java.util.concurrent.TimeoutException import scala.concurrent.duration._ import scala.language.postfixOps -import org.apache.hadoop.mapred.{JobConf, OutputCommitter, TaskAttemptContext, TaskAttemptID} +import org.apache.hadoop.mapred._ +import org.apache.hadoop.mapreduce.TaskType import org.mockito.Matchers import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock @@ -31,7 +33,7 @@ import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter import org.apache.spark._ -import org.apache.spark.internal.io.SparkHadoopWriter +import org.apache.spark.internal.io.{FileCommitProtocol, HadoopMapRedCommitProtocol, SparkHadoopWriterUtils} import org.apache.spark.rdd.{FakeOutputCommitter, RDD} import org.apache.spark.util.{ThreadUtils, Utils} @@ -214,6 +216,8 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { */ private case class OutputCommitFunctions(tempDirPath: String) { + private val jobId = new SerializableWritable(SparkHadoopWriterUtils.createJobID(new Date, 0)) + // Mock output committer that simulates a successful commit (after commit is authorized) private def successfulOutputCommitter = new FakeOutputCommitter { override def commitTask(context: TaskAttemptContext): Unit = { @@ -256,14 +260,23 @@ private case class OutputCommitFunctions(tempDirPath: String) { def jobConf = new JobConf { override def getOutputCommitter(): OutputCommitter = outputCommitter } - val sparkHadoopWriter = new SparkHadoopWriter(jobConf) { - override def newTaskAttemptContext( - conf: JobConf, - attemptId: TaskAttemptID): TaskAttemptContext = { - mock(classOf[TaskAttemptContext]) - } - } - sparkHadoopWriter.setup(ctx.stageId, ctx.partitionId, ctx.attemptNumber) - sparkHadoopWriter.commit() + + // Instantiate committer. + val committer = FileCommitProtocol.instantiate( + className = classOf[HadoopMapRedCommitProtocol].getName, + jobId = jobId.value.getId.toString, + outputPath = jobConf.get("mapred.output.dir"), + isAppend = false) + + // Create TaskAttemptContext. + // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it + // around by taking a mod. We expect that no task will be attempted 2 billion times. + val taskAttemptId = (ctx.taskAttemptId % Int.MaxValue).toInt + val attemptId = new TaskAttemptID( + new TaskID(jobId.value, TaskType.MAP, ctx.partitionId), taskAttemptId) + val taskContext = new TaskAttemptContextImpl(jobConf, attemptId) + + committer.setupTask(taskContext) + committer.commitTask(taskContext) } } diff --git a/docs/configuration.md b/docs/configuration.md index c8e61537a457c..bd6a1f9e240e2 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -528,14 +528,6 @@ Apart from these, the following properties are also available, and may be useful By allowing it to limit the number of fetch requests, this scenario can be mitigated. - - spark.reducer.maxReqSizeShuffleToMem - 200m - - The blocks of a shuffle request will be fetched to disk when size of the request is above - this threshold. This is to avoid a giant request takes too much memory. - - spark.shuffle.compress true diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala index cce3ca45ccd8f..dd56fbbfa2b63 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala @@ -27,3 +27,24 @@ import org.apache.spark.ml.linalg.Vector * @param features The vector of features for this data point. */ private[ml] case class Instance(label: Double, weight: Double, features: Vector) + +/** + * Case class that represents an instance of data point with + * label, weight, offset and features. + * This is mainly used in GeneralizedLinearRegression currently. + * + * @param label Label for this data point. + * @param weight The weight of this instance. + * @param offset The offset used for this data point. + * @param features The vector of features for this data point. + */ +private[ml] case class OffsetInstance( + label: Double, + weight: Double, + offset: Double, + features: Vector) { + + /** Converts to an [[Instance]] object by leaving out the offset. */ + def toInstance: Instance = Instance(label, weight, features) + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala index 9c495512422ba..6961b45f55e4d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.optim import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.{Instance, OffsetInstance} import org.apache.spark.ml.linalg._ import org.apache.spark.rdd.RDD @@ -43,7 +43,7 @@ private[ml] class IterativelyReweightedLeastSquaresModel( * find M-estimator in robust regression and other optimization problems. * * @param initialModel the initial guess model. - * @param reweightFunc the reweight function which is used to update offsets and weights + * @param reweightFunc the reweight function which is used to update working labels and weights * at each iteration. * @param fitIntercept whether to fit intercept. * @param regParam L2 regularization parameter used by WLS. @@ -57,13 +57,13 @@ private[ml] class IterativelyReweightedLeastSquaresModel( */ private[ml] class IterativelyReweightedLeastSquares( val initialModel: WeightedLeastSquaresModel, - val reweightFunc: (Instance, WeightedLeastSquaresModel) => (Double, Double), + val reweightFunc: (OffsetInstance, WeightedLeastSquaresModel) => (Double, Double), val fitIntercept: Boolean, val regParam: Double, val maxIter: Int, val tol: Double) extends Logging with Serializable { - def fit(instances: RDD[Instance]): IterativelyReweightedLeastSquaresModel = { + def fit(instances: RDD[OffsetInstance]): IterativelyReweightedLeastSquaresModel = { var converged = false var iter = 0 @@ -75,10 +75,10 @@ private[ml] class IterativelyReweightedLeastSquares( oldModel = model - // Update offsets and weights using reweightFunc + // Update working labels and weights using reweightFunc val newInstances = instances.map { instance => - val (newOffset, newWeight) = reweightFunc(instance, oldModel) - Instance(newOffset, newWeight, instance.features) + val (newLabel, newWeight) = reweightFunc(instance, oldModel) + Instance(newLabel, newWeight, instance.features) } // Estimate new model diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala index 56ab9675700a0..32b0af72ba9bb 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.optim import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.{Instance, OffsetInstance} import org.apache.spark.ml.linalg._ import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index bff0d9bbb46ff..ce3460ae43566 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -26,8 +26,8 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.PredictorParams -import org.apache.spark.ml.feature.Instance -import org.apache.spark.ml.linalg.{BLAS, Vector} +import org.apache.spark.ml.feature.{Instance, OffsetInstance} +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} import org.apache.spark.ml.optim._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ @@ -138,6 +138,27 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam @Since("2.0.0") def getLinkPredictionCol: String = $(linkPredictionCol) + /** + * Param for offset column name. If this is not set or empty, we treat all instance offsets + * as 0.0. The feature specified as offset has a constant coefficient of 1.0. + * @group param + */ + @Since("2.3.0") + final val offsetCol: Param[String] = new Param[String](this, "offsetCol", "The offset " + + "column name. If this is not set or empty, we treat all instance offsets as 0.0") + + /** @group getParam */ + @Since("2.3.0") + def getOffsetCol: String = $(offsetCol) + + /** Checks whether weight column is set and nonempty. */ + private[regression] def hasWeightCol: Boolean = + isSet(weightCol) && $(weightCol).nonEmpty + + /** Checks whether offset column is set and nonempty. */ + private[regression] def hasOffsetCol: Boolean = + isSet(offsetCol) && $(offsetCol).nonEmpty + /** Checks whether we should output link prediction. */ private[regression] def hasLinkPredictionCol: Boolean = { isDefined(linkPredictionCol) && $(linkPredictionCol).nonEmpty @@ -172,6 +193,11 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam } val newSchema = super.validateAndTransformSchema(schema, fitting, featuresDataType) + + if (hasOffsetCol) { + SchemaUtils.checkNumericType(schema, $(offsetCol)) + } + if (hasLinkPredictionCol) { SchemaUtils.appendColumn(newSchema, $(linkPredictionCol), DoubleType) } else { @@ -306,6 +332,16 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val @Since("2.0.0") def setWeightCol(value: String): this.type = set(weightCol, value) + /** + * Sets the value of param [[offsetCol]]. + * If this is not set or empty, we treat all instance offsets as 0.0. + * Default is not set, so all instances have offset 0.0. + * + * @group setParam + */ + @Since("2.3.0") + def setOffsetCol(value: String): this.type = set(offsetCol, value) + /** * Sets the solver algorithm used for optimization. * Currently only supports "irls" which is also the default solver. @@ -329,7 +365,7 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val val numFeatures = dataset.select(col($(featuresCol))).first().getAs[Vector](0).size val instr = Instrumentation.create(this, dataset) - instr.logParams(labelCol, featuresCol, weightCol, predictionCol, linkPredictionCol, + instr.logParams(labelCol, featuresCol, weightCol, offsetCol, predictionCol, linkPredictionCol, family, solver, fitIntercept, link, maxIter, regParam, tol) instr.logNumFeatures(numFeatures) @@ -343,15 +379,16 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val "GeneralizedLinearRegression was given data with 0 features, and with Param fitIntercept " + "set to false. To fit a model with 0 features, fitIntercept must be set to true." ) - val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) - val instances: RDD[Instance] = - dataset.select(col($(labelCol)), w, col($(featuresCol))).rdd.map { - case Row(label: Double, weight: Double, features: Vector) => - Instance(label, weight, features) - } + val w = if (!hasWeightCol) lit(1.0) else col($(weightCol)) + val offset = if (!hasOffsetCol) lit(0.0) else col($(offsetCol)).cast(DoubleType) val model = if (familyAndLink.family == Gaussian && familyAndLink.link == Identity) { // TODO: Make standardizeFeatures and standardizeLabel configurable. + val instances: RDD[Instance] = + dataset.select(col($(labelCol)), w, offset, col($(featuresCol))).rdd.map { + case Row(label: Double, weight: Double, offset: Double, features: Vector) => + Instance(label - offset, weight, features) + } val optimizer = new WeightedLeastSquares($(fitIntercept), $(regParam), elasticNetParam = 0.0, standardizeFeatures = true, standardizeLabel = true) val wlsModel = optimizer.fit(instances) @@ -362,6 +399,11 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val wlsModel.diagInvAtWA.toArray, 1, getSolver) model.setSummary(Some(trainingSummary)) } else { + val instances: RDD[OffsetInstance] = + dataset.select(col($(labelCol)), w, offset, col($(featuresCol))).rdd.map { + case Row(label: Double, weight: Double, offset: Double, features: Vector) => + OffsetInstance(label, weight, offset, features) + } // Fit Generalized Linear Model by iteratively reweighted least squares (IRLS). val initialModel = familyAndLink.initialize(instances, $(fitIntercept), $(regParam)) val optimizer = new IterativelyReweightedLeastSquares(initialModel, @@ -425,12 +467,12 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * Get the initial guess model for [[IterativelyReweightedLeastSquares]]. */ def initialize( - instances: RDD[Instance], + instances: RDD[OffsetInstance], fitIntercept: Boolean, regParam: Double): WeightedLeastSquaresModel = { val newInstances = instances.map { instance => val mu = family.initialize(instance.label, instance.weight) - val eta = predict(mu) + val eta = predict(mu) - instance.offset Instance(eta, instance.weight, instance.features) } // TODO: Make standardizeFeatures and standardizeLabel configurable. @@ -441,16 +483,16 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine } /** - * The reweight function used to update offsets and weights + * The reweight function used to update working labels and weights * at each iteration of [[IterativelyReweightedLeastSquares]]. */ - val reweightFunc: (Instance, WeightedLeastSquaresModel) => (Double, Double) = { - (instance: Instance, model: WeightedLeastSquaresModel) => { - val eta = model.predict(instance.features) + val reweightFunc: (OffsetInstance, WeightedLeastSquaresModel) => (Double, Double) = { + (instance: OffsetInstance, model: WeightedLeastSquaresModel) => { + val eta = model.predict(instance.features) + instance.offset val mu = fitted(eta) - val offset = eta + (instance.label - mu) * link.deriv(mu) - val weight = instance.weight / (math.pow(this.link.deriv(mu), 2.0) * family.variance(mu)) - (offset, weight) + val newLabel = eta - instance.offset + (instance.label - mu) * link.deriv(mu) + val newWeight = instance.weight / (math.pow(this.link.deriv(mu), 2.0) * family.variance(mu)) + (newLabel, newWeight) } } } @@ -950,15 +992,22 @@ class GeneralizedLinearRegressionModel private[ml] ( private lazy val familyAndLink = FamilyAndLink(this) override protected def predict(features: Vector): Double = { - val eta = predictLink(features) + predict(features, 0.0) + } + + /** + * Calculates the predicted value when offset is set. + */ + private def predict(features: Vector, offset: Double): Double = { + val eta = predictLink(features, offset) familyAndLink.fitted(eta) } /** - * Calculate the link prediction (linear predictor) of the given instance. + * Calculates the link prediction (linear predictor) of the given instance. */ - private def predictLink(features: Vector): Double = { - BLAS.dot(features, coefficients) + intercept + private def predictLink(features: Vector, offset: Double): Double = { + BLAS.dot(features, coefficients) + intercept + offset } override def transform(dataset: Dataset[_]): DataFrame = { @@ -967,14 +1016,16 @@ class GeneralizedLinearRegressionModel private[ml] ( } override protected def transformImpl(dataset: Dataset[_]): DataFrame = { - val predictUDF = udf { (features: Vector) => predict(features) } - val predictLinkUDF = udf { (features: Vector) => predictLink(features) } + val predictUDF = udf { (features: Vector, offset: Double) => predict(features, offset) } + val predictLinkUDF = udf { (features: Vector, offset: Double) => predictLink(features, offset) } + + val offset = if (!hasOffsetCol) lit(0.0) else col($(offsetCol)).cast(DoubleType) var output = dataset if ($(predictionCol).nonEmpty) { - output = output.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) + output = output.withColumn($(predictionCol), predictUDF(col($(featuresCol)), offset)) } if (hasLinkPredictionCol) { - output = output.withColumn($(linkPredictionCol), predictLinkUDF(col($(featuresCol)))) + output = output.withColumn($(linkPredictionCol), predictLinkUDF(col($(featuresCol)), offset)) } output.toDF() } @@ -1146,9 +1197,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( /** Degrees of freedom. */ @Since("2.0.0") - lazy val degreesOfFreedom: Long = { - numInstances - rank - } + lazy val degreesOfFreedom: Long = numInstances - rank /** The residual degrees of freedom. */ @Since("2.0.0") @@ -1156,18 +1205,20 @@ class GeneralizedLinearRegressionSummary private[regression] ( /** The residual degrees of freedom for the null model. */ @Since("2.0.0") - lazy val residualDegreeOfFreedomNull: Long = if (model.getFitIntercept) { - numInstances - 1 - } else { - numInstances + lazy val residualDegreeOfFreedomNull: Long = { + if (model.getFitIntercept) numInstances - 1 else numInstances } - private def weightCol: Column = { - if (!model.isDefined(model.weightCol) || model.getWeightCol.isEmpty) { - lit(1.0) - } else { - col(model.getWeightCol) - } + private def label: Column = col(model.getLabelCol).cast(DoubleType) + + private def prediction: Column = col(predictionCol) + + private def weight: Column = { + if (!model.hasWeightCol) lit(1.0) else col(model.getWeightCol) + } + + private def offset: Column = { + if (!model.hasOffsetCol) lit(0.0) else col(model.getOffsetCol).cast(DoubleType) } private[regression] lazy val devianceResiduals: DataFrame = { @@ -1175,25 +1226,23 @@ class GeneralizedLinearRegressionSummary private[regression] ( val r = math.sqrt(math.max(family.deviance(y, mu, weight), 0.0)) if (y > mu) r else -1.0 * r } - val w = weightCol predictions.select( - drUDF(col(model.getLabelCol), col(predictionCol), w).as("devianceResiduals")) + drUDF(label, prediction, weight).as("devianceResiduals")) } private[regression] lazy val pearsonResiduals: DataFrame = { val prUDF = udf { mu: Double => family.variance(mu) } - val w = weightCol - predictions.select(col(model.getLabelCol).minus(col(predictionCol)) - .multiply(sqrt(w)).divide(sqrt(prUDF(col(predictionCol)))).as("pearsonResiduals")) + predictions.select(label.minus(prediction) + .multiply(sqrt(weight)).divide(sqrt(prUDF(prediction))).as("pearsonResiduals")) } private[regression] lazy val workingResiduals: DataFrame = { val wrUDF = udf { (y: Double, mu: Double) => (y - mu) * link.deriv(mu) } - predictions.select(wrUDF(col(model.getLabelCol), col(predictionCol)).as("workingResiduals")) + predictions.select(wrUDF(label, prediction).as("workingResiduals")) } private[regression] lazy val responseResiduals: DataFrame = { - predictions.select(col(model.getLabelCol).minus(col(predictionCol)).as("responseResiduals")) + predictions.select(label.minus(prediction).as("responseResiduals")) } /** @@ -1225,16 +1274,35 @@ class GeneralizedLinearRegressionSummary private[regression] ( */ @Since("2.0.0") lazy val nullDeviance: Double = { - val w = weightCol - val wtdmu: Double = if (model.getFitIntercept) { - val agg = predictions.agg(sum(w.multiply(col(model.getLabelCol))), sum(w)).first() - agg.getDouble(0) / agg.getDouble(1) + val intercept: Double = if (!model.getFitIntercept) { + 0.0 } else { - link.unlink(0.0) + /* + Estimate intercept analytically when there is no offset, or when there is offset but + the model is Gaussian family with identity link. Otherwise, fit an intercept only model. + */ + if (!model.hasOffsetCol || + (model.hasOffsetCol && family == Gaussian && link == Identity)) { + val agg = predictions.agg(sum(weight.multiply( + label.minus(offset))), sum(weight)).first() + link.link(agg.getDouble(0) / agg.getDouble(1)) + } else { + // Create empty feature column and fit intercept only model using param setting from model + val featureNull = "feature_" + java.util.UUID.randomUUID.toString + val paramMap = model.extractParamMap() + paramMap.put(model.featuresCol, featureNull) + if (family.name != "tweedie") { + paramMap.remove(model.variancePower) + } + val emptyVectorUDF = udf{ () => Vectors.zeros(0) } + model.parent.fit( + dataset.withColumn(featureNull, emptyVectorUDF()), paramMap + ).intercept + } } - predictions.select(col(model.getLabelCol).cast(DoubleType), w).rdd.map { - case Row(y: Double, weight: Double) => - family.deviance(y, wtdmu, weight) + predictions.select(label, offset, weight).rdd.map { + case Row(y: Double, offset: Double, weight: Double) => + family.deviance(y, link.unlink(intercept + offset), weight) }.sum() } @@ -1243,8 +1311,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( */ @Since("2.0.0") lazy val deviance: Double = { - val w = weightCol - predictions.select(col(model.getLabelCol).cast(DoubleType), col(predictionCol), w).rdd.map { + predictions.select(label, prediction, weight).rdd.map { case Row(label: Double, pred: Double, weight: Double) => family.deviance(label, pred, weight) }.sum() @@ -1269,10 +1336,9 @@ class GeneralizedLinearRegressionSummary private[regression] ( /** Akaike Information Criterion (AIC) for the fitted model. */ @Since("2.0.0") lazy val aic: Double = { - val w = weightCol - val weightSum = predictions.select(w).agg(sum(w)).first().getDouble(0) + val weightSum = predictions.select(weight).agg(sum(weight)).first().getDouble(0) val t = predictions.select( - col(model.getLabelCol).cast(DoubleType), col(predictionCol), w).rdd.map { + label, prediction, weight).rdd.map { case Row(label: Double, pred: Double, weight: Double) => (label, pred, weight) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala index 50260952ecb66..6d143504fcf58 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.optim import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.{Instance, OffsetInstance} import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -26,8 +26,8 @@ import org.apache.spark.rdd.RDD class IterativelyReweightedLeastSquaresSuite extends SparkFunSuite with MLlibTestSparkContext { - private var instances1: RDD[Instance] = _ - private var instances2: RDD[Instance] = _ + private var instances1: RDD[OffsetInstance] = _ + private var instances2: RDD[OffsetInstance] = _ override def beforeAll(): Unit = { super.beforeAll() @@ -39,10 +39,10 @@ class IterativelyReweightedLeastSquaresSuite extends SparkFunSuite with MLlibTes w <- c(1, 2, 3, 4) */ instances1 = sc.parallelize(Seq( - Instance(1.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), - Instance(0.0, 2.0, Vectors.dense(1.0, 2.0)), - Instance(1.0, 3.0, Vectors.dense(2.0, 1.0)), - Instance(0.0, 4.0, Vectors.dense(3.0, 3.0)) + OffsetInstance(1.0, 1.0, 0.0, Vectors.dense(0.0, 5.0).toSparse), + OffsetInstance(0.0, 2.0, 0.0, Vectors.dense(1.0, 2.0)), + OffsetInstance(1.0, 3.0, 0.0, Vectors.dense(2.0, 1.0)), + OffsetInstance(0.0, 4.0, 0.0, Vectors.dense(3.0, 3.0)) ), 2) /* R code: @@ -52,10 +52,10 @@ class IterativelyReweightedLeastSquaresSuite extends SparkFunSuite with MLlibTes w <- c(1, 2, 3, 4) */ instances2 = sc.parallelize(Seq( - Instance(2.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), - Instance(8.0, 2.0, Vectors.dense(1.0, 7.0)), - Instance(3.0, 3.0, Vectors.dense(2.0, 11.0)), - Instance(9.0, 4.0, Vectors.dense(3.0, 13.0)) + OffsetInstance(2.0, 1.0, 0.0, Vectors.dense(0.0, 5.0).toSparse), + OffsetInstance(8.0, 2.0, 0.0, Vectors.dense(1.0, 7.0)), + OffsetInstance(3.0, 3.0, 0.0, Vectors.dense(2.0, 11.0)), + OffsetInstance(9.0, 4.0, 0.0, Vectors.dense(3.0, 13.0)) ), 2) } @@ -156,7 +156,7 @@ class IterativelyReweightedLeastSquaresSuite extends SparkFunSuite with MLlibTes var idx = 0 for (fitIntercept <- Seq(false, true)) { val initial = new WeightedLeastSquares(fitIntercept, regParam = 0.0, elasticNetParam = 0.0, - standardizeFeatures = false, standardizeLabel = false).fit(instances2) + standardizeFeatures = false, standardizeLabel = false).fit(instances2.map(_.toInstance)) val irls = new IterativelyReweightedLeastSquares(initial, L1RegressionReweightFunc, fitIntercept, regParam = 0.0, maxIter = 200, tol = 1e-7).fit(instances2) val actual = Vectors.dense(irls.intercept, irls.coefficients(0), irls.coefficients(1)) @@ -169,29 +169,29 @@ class IterativelyReweightedLeastSquaresSuite extends SparkFunSuite with MLlibTes object IterativelyReweightedLeastSquaresSuite { def BinomialReweightFunc( - instance: Instance, + instance: OffsetInstance, model: WeightedLeastSquaresModel): (Double, Double) = { - val eta = model.predict(instance.features) + val eta = model.predict(instance.features) + instance.offset val mu = 1.0 / (1.0 + math.exp(-1.0 * eta)) - val z = eta + (instance.label - mu) / (mu * (1.0 - mu)) + val z = eta - instance.offset + (instance.label - mu) / (mu * (1.0 - mu)) val w = mu * (1 - mu) * instance.weight (z, w) } def PoissonReweightFunc( - instance: Instance, + instance: OffsetInstance, model: WeightedLeastSquaresModel): (Double, Double) = { - val eta = model.predict(instance.features) + val eta = model.predict(instance.features) + instance.offset val mu = math.exp(eta) - val z = eta + (instance.label - mu) / mu + val z = eta - instance.offset + (instance.label - mu) / mu val w = mu * instance.weight (z, w) } def L1RegressionReweightFunc( - instance: Instance, + instance: OffsetInstance, model: WeightedLeastSquaresModel): (Double, Double) = { - val eta = model.predict(instance.features) + val eta = model.predict(instance.features) + instance.offset val e = math.max(math.abs(eta - instance.label), 1e-7) val w = 1 / e val y = instance.label diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala index f7c7c001a36af..83f1344a7bcb1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala @@ -21,7 +21,7 @@ import scala.util.Random import org.apache.spark.SparkFunSuite import org.apache.spark.ml.classification.LogisticRegressionSuite._ -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.{Instance, OffsetInstance} import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors} import org.apache.spark.ml.param.{ParamMap, ParamsSuite} @@ -797,77 +797,160 @@ class GeneralizedLinearRegressionSuite } } - test("glm summary: gaussian family with weight") { + test("generalized linear regression with weight and offset") { /* - R code: + R code: + library(statmod) - A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) - b <- c(17, 19, 23, 29) - w <- c(1, 2, 3, 4) - df <- as.data.frame(cbind(A, b)) - */ - val datasetWithWeight = Seq( - Instance(17.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), - Instance(19.0, 2.0, Vectors.dense(1.0, 7.0)), - Instance(23.0, 3.0, Vectors.dense(2.0, 11.0)), - Instance(29.0, 4.0, Vectors.dense(3.0, 13.0)) + df <- as.data.frame(matrix(c( + 0.2, 1.0, 2.0, 0.0, 5.0, + 0.5, 2.1, 0.5, 1.0, 2.0, + 0.9, 0.4, 1.0, 2.0, 1.0, + 0.7, 0.7, 0.0, 3.0, 3.0), 4, 5, byrow = TRUE)) + families <- list(gaussian, binomial, poisson, Gamma, tweedie(1.5)) + f1 <- V1 ~ -1 + V4 + V5 + f2 <- V1 ~ V4 + V5 + for (f in c(f1, f2)) { + for (fam in families) { + model <- glm(f, df, family = fam, weights = V2, offset = V3) + print(as.vector(coef(model))) + } + } + [1] 0.5169222 -0.3344444 + [1] 0.9419107 -0.6864404 + [1] 0.1812436 -0.6568422 + [1] -0.2869094 0.7857710 + [1] 0.1055254 0.2979113 + [1] -0.05990345 0.53188982 -0.32118415 + [1] -0.2147117 0.9911750 -0.6356096 + [1] -1.5616130 0.6646470 -0.3192581 + [1] 0.3390397 -0.3406099 0.6870259 + [1] 0.3665034 0.1039416 0.1484616 + */ + val dataset = Seq( + OffsetInstance(0.2, 1.0, 2.0, Vectors.dense(0.0, 5.0)), + OffsetInstance(0.5, 2.1, 0.5, Vectors.dense(1.0, 2.0)), + OffsetInstance(0.9, 0.4, 1.0, Vectors.dense(2.0, 1.0)), + OffsetInstance(0.7, 0.7, 0.0, Vectors.dense(3.0, 3.0)) ).toDF() + + val expected = Seq( + Vectors.dense(0, 0.5169222, -0.3344444), + Vectors.dense(0, 0.9419107, -0.6864404), + Vectors.dense(0, 0.1812436, -0.6568422), + Vectors.dense(0, -0.2869094, 0.785771), + Vectors.dense(0, 0.1055254, 0.2979113), + Vectors.dense(-0.05990345, 0.53188982, -0.32118415), + Vectors.dense(-0.2147117, 0.991175, -0.6356096), + Vectors.dense(-1.561613, 0.664647, -0.3192581), + Vectors.dense(0.3390397, -0.3406099, 0.6870259), + Vectors.dense(0.3665034, 0.1039416, 0.1484616)) + + import GeneralizedLinearRegression._ + + var idx = 0 + + for (fitIntercept <- Seq(false, true)) { + for (family <- Seq("gaussian", "binomial", "poisson", "gamma", "tweedie")) { + val trainer = new GeneralizedLinearRegression().setFamily(family) + .setFitIntercept(fitIntercept).setOffsetCol("offset") + .setWeightCol("weight").setLinkPredictionCol("linkPrediction") + if (family == "tweedie") trainer.setVariancePower(1.5) + val model = trainer.fit(dataset) + val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) + assert(actual ~= expected(idx) absTol 1e-4, s"Model mismatch: GLM with family = $family," + + s" and fitIntercept = $fitIntercept.") + + val familyLink = FamilyAndLink(trainer) + model.transform(dataset).select("features", "offset", "prediction", "linkPrediction") + .collect().foreach { + case Row(features: DenseVector, offset: Double, prediction1: Double, + linkPrediction1: Double) => + val eta = BLAS.dot(features, model.coefficients) + model.intercept + offset + val prediction2 = familyLink.fitted(eta) + val linkPrediction2 = eta + assert(prediction1 ~= prediction2 relTol 1E-5, "Prediction mismatch: GLM with " + + s"family = $family, and fitIntercept = $fitIntercept.") + assert(linkPrediction1 ~= linkPrediction2 relTol 1E-5, "Link Prediction mismatch: " + + s"GLM with family = $family, and fitIntercept = $fitIntercept.") + } + + idx += 1 + } + } + } + + test("glm summary: gaussian family with weight and offset") { /* - R code: + R code: - model <- glm(formula = "b ~ .", family="gaussian", data = df, weights = w) - summary(model) + A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) + b <- c(17, 19, 23, 29) + w <- c(1, 2, 3, 4) + off <- c(2, 3, 1, 4) + df <- as.data.frame(cbind(A, b)) + */ + val dataset = Seq( + OffsetInstance(17.0, 1.0, 2.0, Vectors.dense(0.0, 5.0).toSparse), + OffsetInstance(19.0, 2.0, 3.0, Vectors.dense(1.0, 7.0)), + OffsetInstance(23.0, 3.0, 1.0, Vectors.dense(2.0, 11.0)), + OffsetInstance(29.0, 4.0, 4.0, Vectors.dense(3.0, 13.0)) + ).toDF() + /* + R code: - Deviance Residuals: - 1 2 3 4 - 1.920 -1.358 -1.109 0.960 + model <- glm(formula = "b ~ .", family = "gaussian", data = df, + weights = w, offset = off) + summary(model) - Coefficients: - Estimate Std. Error t value Pr(>|t|) - (Intercept) 18.080 9.608 1.882 0.311 - V1 6.080 5.556 1.094 0.471 - V2 -0.600 1.960 -0.306 0.811 + Deviance Residuals: + 1 2 3 4 + 0.9600 -0.6788 -0.5543 0.4800 - (Dispersion parameter for gaussian family taken to be 7.68) + Coefficients: + Estimate Std. Error t value Pr(>|t|) + (Intercept) 5.5400 4.8040 1.153 0.455 + V1 -0.9600 2.7782 -0.346 0.788 + V2 1.7000 0.9798 1.735 0.333 - Null deviance: 202.00 on 3 degrees of freedom - Residual deviance: 7.68 on 1 degrees of freedom - AIC: 18.783 + (Dispersion parameter for gaussian family taken to be 1.92) - Number of Fisher Scoring iterations: 2 + Null deviance: 152.10 on 3 degrees of freedom + Residual deviance: 1.92 on 1 degrees of freedom + AIC: 13.238 - residuals(model, type="pearson") - 1 2 3 4 - 1.920000 -1.357645 -1.108513 0.960000 + Number of Fisher Scoring iterations: 2 - residuals(model, type="working") + residuals(model, type = "pearson") + 1 2 3 4 + 0.9600000 -0.6788225 -0.5542563 0.4800000 + residuals(model, type = "working") 1 2 3 4 - 1.92 -0.96 -0.64 0.48 - - residuals(model, type="response") + 0.96 -0.48 -0.32 0.24 + residuals(model, type = "response") 1 2 3 4 - 1.92 -0.96 -0.64 0.48 + 0.96 -0.48 -0.32 0.24 */ val trainer = new GeneralizedLinearRegression() - .setWeightCol("weight") + .setWeightCol("weight").setOffsetCol("offset") + + val model = trainer.fit(dataset) - val model = trainer.fit(datasetWithWeight) - - val coefficientsR = Vectors.dense(Array(6.080, -0.600)) - val interceptR = 18.080 - val devianceResidualsR = Array(1.920, -1.358, -1.109, 0.960) - val pearsonResidualsR = Array(1.920000, -1.357645, -1.108513, 0.960000) - val workingResidualsR = Array(1.92, -0.96, -0.64, 0.48) - val responseResidualsR = Array(1.92, -0.96, -0.64, 0.48) - val seCoefR = Array(5.556, 1.960, 9.608) - val tValsR = Array(1.094, -0.306, 1.882) - val pValsR = Array(0.471, 0.811, 0.311) - val dispersionR = 7.68 - val nullDevianceR = 202.00 - val residualDevianceR = 7.68 + val coefficientsR = Vectors.dense(Array(-0.96, 1.7)) + val interceptR = 5.54 + val devianceResidualsR = Array(0.96, -0.67882, -0.55426, 0.48) + val pearsonResidualsR = Array(0.96, -0.67882, -0.55426, 0.48) + val workingResidualsR = Array(0.96, -0.48, -0.32, 0.24) + val responseResidualsR = Array(0.96, -0.48, -0.32, 0.24) + val seCoefR = Array(2.7782, 0.9798, 4.804) + val tValsR = Array(-0.34555, 1.73506, 1.15321) + val pValsR = Array(0.78819, 0.33286, 0.45478) + val dispersionR = 1.92 + val nullDevianceR = 152.1 + val residualDevianceR = 1.92 val residualDegreeOfFreedomNullR = 3 val residualDegreeOfFreedomR = 1 - val aicR = 18.783 + val aicR = 13.23758 assert(model.hasSummary) val summary = model.summary @@ -912,7 +995,7 @@ class GeneralizedLinearRegressionSuite assert(summary.aic ~== aicR absTol 1E-3) assert(summary.solver === "irls") - val summary2: GeneralizedLinearRegressionSummary = model.evaluate(datasetWithWeight) + val summary2: GeneralizedLinearRegressionSummary = model.evaluate(dataset) assert(summary.predictions.columns.toSet === summary2.predictions.columns.toSet) assert(summary.predictionCol === summary2.predictionCol) assert(summary.rank === summary2.rank) @@ -925,79 +1008,79 @@ class GeneralizedLinearRegressionSuite assert(summary.aic === summary2.aic) } - test("glm summary: binomial family with weight") { + test("glm summary: binomial family with weight and offset") { /* - R code: + R code: - A <- matrix(c(0, 1, 2, 3, 5, 2, 1, 3), 4, 2) - b <- c(1, 0.5, 1, 0) - w <- c(1, 2.0, 0.3, 4.7) - df <- as.data.frame(cbind(A, b)) + df <- as.data.frame(matrix(c( + 0.2, 1.0, 2.0, 0.0, 5.0, + 0.5, 2.1, 0.5, 1.0, 2.0, + 0.9, 0.4, 1.0, 2.0, 1.0, + 0.7, 0.7, 0.0, 3.0, 3.0), 4, 5, byrow = TRUE)) */ - val datasetWithWeight = Seq( - Instance(1.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), - Instance(0.5, 2.0, Vectors.dense(1.0, 2.0)), - Instance(1.0, 0.3, Vectors.dense(2.0, 1.0)), - Instance(0.0, 4.7, Vectors.dense(3.0, 3.0)) + val dataset = Seq( + OffsetInstance(0.2, 1.0, 2.0, Vectors.dense(0.0, 5.0)), + OffsetInstance(0.5, 2.1, 0.5, Vectors.dense(1.0, 2.0)), + OffsetInstance(0.9, 0.4, 1.0, Vectors.dense(2.0, 1.0)), + OffsetInstance(0.7, 0.7, 0.0, Vectors.dense(3.0, 3.0)) ).toDF() - /* - R code: - - model <- glm(formula = "b ~ . -1", family="binomial", data = df, weights = w) - summary(model) - - Deviance Residuals: - 1 2 3 4 - 0.2404 0.1965 1.2824 -0.6916 + R code: - Coefficients: - Estimate Std. Error z value Pr(>|z|) - x1 -1.6901 1.2764 -1.324 0.185 - x2 0.7059 0.9449 0.747 0.455 + model <- glm(formula = "V1 ~ V4 + V5", family = "binomial", data = df, + weights = V2, offset = V3) + summary(model) - (Dispersion parameter for binomial family taken to be 1) + Deviance Residuals: + 1 2 3 4 + 0.002584 -0.003800 0.012478 -0.001796 - Null deviance: 8.3178 on 4 degrees of freedom - Residual deviance: 2.2193 on 2 degrees of freedom - AIC: 5.9915 + Coefficients: + Estimate Std. Error z value Pr(>|z|) + (Intercept) -0.2147 3.5687 -0.060 0.952 + V4 0.9912 1.2344 0.803 0.422 + V5 -0.6356 0.9669 -0.657 0.511 - Number of Fisher Scoring iterations: 5 + (Dispersion parameter for binomial family taken to be 1) - residuals(model, type="pearson") - 1 2 3 4 - 0.171217 0.197406 2.085864 -0.495332 + Null deviance: 2.17560881 on 3 degrees of freedom + Residual deviance: 0.00018005 on 1 degrees of freedom + AIC: 10.245 - residuals(model, type="working") - 1 2 3 4 - 1.029315 0.281881 15.502768 -1.052203 + Number of Fisher Scoring iterations: 4 - residuals(model, type="response") - 1 2 3 4 - 0.028480 0.069123 0.935495 -0.049613 + residuals(model, type = "pearson") + 1 2 3 4 + 0.002586113 -0.003799744 0.012372235 -0.001796892 + residuals(model, type = "working") + 1 2 3 4 + 0.006477857 -0.005244163 0.063541250 -0.004691064 + residuals(model, type = "response") + 1 2 3 4 + 0.0010324375 -0.0013110318 0.0060225522 -0.0009832738 */ val trainer = new GeneralizedLinearRegression() .setFamily("Binomial") .setWeightCol("weight") - .setFitIntercept(false) - - val model = trainer.fit(datasetWithWeight) - - val coefficientsR = Vectors.dense(Array(-1.690134, 0.705929)) - val interceptR = 0.0 - val devianceResidualsR = Array(0.2404, 0.1965, 1.2824, -0.6916) - val pearsonResidualsR = Array(0.171217, 0.197406, 2.085864, -0.495332) - val workingResidualsR = Array(1.029315, 0.281881, 15.502768, -1.052203) - val responseResidualsR = Array(0.02848, 0.069123, 0.935495, -0.049613) - val seCoefR = Array(1.276417, 0.944934) - val tValsR = Array(-1.324124, 0.747068) - val pValsR = Array(0.185462, 0.455023) + .setOffsetCol("offset") + + val model = trainer.fit(dataset) + + val coefficientsR = Vectors.dense(Array(0.99117, -0.63561)) + val interceptR = -0.21471 + val devianceResidualsR = Array(0.00258, -0.0038, 0.01248, -0.0018) + val pearsonResidualsR = Array(0.00259, -0.0038, 0.01237, -0.0018) + val workingResidualsR = Array(0.00648, -0.00524, 0.06354, -0.00469) + val responseResidualsR = Array(0.00103, -0.00131, 0.00602, -0.00098) + val seCoefR = Array(1.23439, 0.9669, 3.56866) + val tValsR = Array(0.80297, -0.65737, -0.06017) + val pValsR = Array(0.42199, 0.51094, 0.95202) val dispersionR = 1.0 - val nullDevianceR = 8.3178 - val residualDevianceR = 2.2193 - val residualDegreeOfFreedomNullR = 4 - val residualDegreeOfFreedomR = 2 - val aicR = 5.991537 + val nullDevianceR = 2.17561 + val residualDevianceR = 0.00018 + val residualDegreeOfFreedomNullR = 3 + val residualDegreeOfFreedomR = 1 + val aicR = 10.24453 val summary = model.summary val devianceResiduals = summary.residuals() @@ -1031,7 +1114,7 @@ class GeneralizedLinearRegressionSuite assert(x._1 ~== x._2 absTol 1E-3) } summary.tValues.zip(tValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } summary.pValues.zip(pValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } - assert(summary.dispersion ~== dispersionR absTol 1E-3) + assert(summary.dispersion === dispersionR) assert(summary.nullDeviance ~== nullDevianceR absTol 1E-3) assert(summary.deviance ~== residualDevianceR absTol 1E-3) assert(summary.residualDegreeOfFreedom === residualDegreeOfFreedomR) @@ -1040,81 +1123,79 @@ class GeneralizedLinearRegressionSuite assert(summary.solver === "irls") } - test("glm summary: poisson family with weight") { + test("glm summary: poisson family with weight and offset") { /* - R code: + R code: - A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) - b <- c(2, 8, 3, 9) - w <- c(1, 2, 3, 4) - df <- as.data.frame(cbind(A, b)) + A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) + b <- c(2, 8, 3, 9) + w <- c(1, 2, 3, 4) + off <- c(2, 3, 1, 4) + df <- as.data.frame(cbind(A, b)) */ - val datasetWithWeight = Seq( - Instance(2.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), - Instance(8.0, 2.0, Vectors.dense(1.0, 7.0)), - Instance(3.0, 3.0, Vectors.dense(2.0, 11.0)), - Instance(9.0, 4.0, Vectors.dense(3.0, 13.0)) + val dataset = Seq( + OffsetInstance(2.0, 1.0, 2.0, Vectors.dense(0.0, 5.0).toSparse), + OffsetInstance(8.0, 2.0, 3.0, Vectors.dense(1.0, 7.0)), + OffsetInstance(3.0, 3.0, 1.0, Vectors.dense(2.0, 11.0)), + OffsetInstance(9.0, 4.0, 4.0, Vectors.dense(3.0, 13.0)) ).toDF() /* - R code: - - model <- glm(formula = "b ~ .", family="poisson", data = df, weights = w) - summary(model) - - Deviance Residuals: - 1 2 3 4 - -0.28952 0.11048 0.14839 -0.07268 - - Coefficients: - Estimate Std. Error z value Pr(>|z|) - (Intercept) 6.2999 1.6086 3.916 8.99e-05 *** - V1 3.3241 1.0184 3.264 0.00110 ** - V2 -1.0818 0.3522 -3.071 0.00213 ** - --- - Signif. codes: 0 '***' 0.001 '**' 0.01 '*' 0.05 '.' 0.1 ' ' 1 - - (Dispersion parameter for poisson family taken to be 1) - - Null deviance: 15.38066 on 3 degrees of freedom - Residual deviance: 0.12333 on 1 degrees of freedom - AIC: 41.803 - - Number of Fisher Scoring iterations: 3 + R code: - residuals(model, type="pearson") - 1 2 3 4 - -0.28043145 0.11099310 0.14963714 -0.07253611 + model <- glm(formula = "b ~ .", family = "poisson", data = df, + weights = w, offset = off) + summary(model) - residuals(model, type="working") - 1 2 3 4 - -0.17960679 0.02813593 0.05113852 -0.01201650 + Deviance Residuals: + 1 2 3 4 + -2.0480 1.2315 1.8293 -0.7107 - residuals(model, type="response") - 1 2 3 4 - -0.4378554 0.2189277 0.1459518 -0.1094638 + Coefficients: + Estimate Std. Error z value Pr(>|z|) + (Intercept) -4.5678 1.9625 -2.328 0.0199 + V1 -2.8784 1.1683 -2.464 0.0137 + V2 0.8859 0.4170 2.124 0.0336 + + (Dispersion parameter for poisson family taken to be 1) + + Null deviance: 22.5585 on 3 degrees of freedom + Residual deviance: 9.5622 on 1 degrees of freedom + AIC: 51.242 + + Number of Fisher Scoring iterations: 5 + + residuals(model, type = "pearson") + 1 2 3 4 + -1.7480418 1.3037611 2.0750099 -0.6972966 + residuals(model, type = "working") + 1 2 3 4 + -0.6891489 0.3833588 0.9710682 -0.1096590 + residuals(model, type = "response") + 1 2 3 4 + -4.433948 2.216974 1.477983 -1.108487 */ val trainer = new GeneralizedLinearRegression() .setFamily("Poisson") .setWeightCol("weight") - .setFitIntercept(true) - - val model = trainer.fit(datasetWithWeight) - - val coefficientsR = Vectors.dense(Array(3.3241, -1.0818)) - val interceptR = 6.2999 - val devianceResidualsR = Array(-0.28952, 0.11048, 0.14839, -0.07268) - val pearsonResidualsR = Array(-0.28043145, 0.11099310, 0.14963714, -0.07253611) - val workingResidualsR = Array(-0.17960679, 0.02813593, 0.05113852, -0.01201650) - val responseResidualsR = Array(-0.4378554, 0.2189277, 0.1459518, -0.1094638) - val seCoefR = Array(1.0184, 0.3522, 1.6086) - val tValsR = Array(3.264, -3.071, 3.916) - val pValsR = Array(0.00110, 0.00213, 0.00009) + .setOffsetCol("offset") + + val model = trainer.fit(dataset) + + val coefficientsR = Vectors.dense(Array(-2.87843, 0.88589)) + val interceptR = -4.56784 + val devianceResidualsR = Array(-2.04796, 1.23149, 1.82933, -0.71066) + val pearsonResidualsR = Array(-1.74804, 1.30376, 2.07501, -0.6973) + val workingResidualsR = Array(-0.68915, 0.38336, 0.97107, -0.10966) + val responseResidualsR = Array(-4.43395, 2.21697, 1.47798, -1.10849) + val seCoefR = Array(1.16826, 0.41703, 1.96249) + val tValsR = Array(-2.46387, 2.12428, -2.32757) + val pValsR = Array(0.01374, 0.03365, 0.01993) val dispersionR = 1.0 - val nullDevianceR = 15.38066 - val residualDevianceR = 0.12333 + val nullDevianceR = 22.55853 + val residualDevianceR = 9.5622 val residualDegreeOfFreedomNullR = 3 val residualDegreeOfFreedomR = 1 - val aicR = 41.803 + val aicR = 51.24218 val summary = model.summary val devianceResiduals = summary.residuals() @@ -1148,7 +1229,7 @@ class GeneralizedLinearRegressionSuite assert(x._1 ~== x._2 absTol 1E-3) } summary.tValues.zip(tValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } summary.pValues.zip(pValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } - assert(summary.dispersion ~== dispersionR absTol 1E-3) + assert(summary.dispersion === dispersionR) assert(summary.nullDeviance ~== nullDevianceR absTol 1E-3) assert(summary.deviance ~== residualDevianceR absTol 1E-3) assert(summary.residualDegreeOfFreedom === residualDegreeOfFreedomR) @@ -1157,78 +1238,79 @@ class GeneralizedLinearRegressionSuite assert(summary.solver === "irls") } - test("glm summary: gamma family with weight") { + test("glm summary: gamma family with weight and offset") { /* - R code: + R code: - A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) - b <- c(2, 8, 3, 9) - w <- c(1, 2, 3, 4) - df <- as.data.frame(cbind(A, b)) + A <- matrix(c(0, 5, 1, 2, 2, 1, 3, 3), 4, 2, byrow = TRUE) + b <- c(1, 2, 1, 2) + w <- c(1, 2, 3, 4) + off <- c(0, 0.5, 1, 0) + df <- as.data.frame(cbind(A, b)) */ - val datasetWithWeight = Seq( - Instance(2.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), - Instance(8.0, 2.0, Vectors.dense(1.0, 7.0)), - Instance(3.0, 3.0, Vectors.dense(2.0, 11.0)), - Instance(9.0, 4.0, Vectors.dense(3.0, 13.0)) + val dataset = Seq( + OffsetInstance(1.0, 1.0, 0.0, Vectors.dense(0.0, 5.0)), + OffsetInstance(2.0, 2.0, 0.5, Vectors.dense(1.0, 2.0)), + OffsetInstance(1.0, 3.0, 1.0, Vectors.dense(2.0, 1.0)), + OffsetInstance(2.0, 4.0, 0.0, Vectors.dense(3.0, 3.0)) ).toDF() /* - R code: - - model <- glm(formula = "b ~ .", family="Gamma", data = df, weights = w) - summary(model) + R code: - Deviance Residuals: - 1 2 3 4 - -0.26343 0.05761 0.12818 -0.03484 + model <- glm(formula = "b ~ .", family = "Gamma", data = df, + weights = w, offset = off) + summary(model) - Coefficients: - Estimate Std. Error t value Pr(>|t|) - (Intercept) -0.81511 0.23449 -3.476 0.178 - V1 -0.72730 0.16137 -4.507 0.139 - V2 0.23894 0.05481 4.359 0.144 + Deviance Residuals: + 1 2 3 4 + -0.17095 0.19867 -0.23604 0.03241 - (Dispersion parameter for Gamma family taken to be 0.07986091) + Coefficients: + Estimate Std. Error t value Pr(>|t|) + (Intercept) -0.56474 0.23866 -2.366 0.255 + V1 0.07695 0.06931 1.110 0.467 + V2 0.28068 0.07320 3.835 0.162 - Null deviance: 2.937462 on 3 degrees of freedom - Residual deviance: 0.090358 on 1 degrees of freedom - AIC: 23.202 + (Dispersion parameter for Gamma family taken to be 0.1212174) - Number of Fisher Scoring iterations: 4 + Null deviance: 2.02568 on 3 degrees of freedom + Residual deviance: 0.12546 on 1 degrees of freedom + AIC: 0.93388 - residuals(model, type="pearson") - 1 2 3 4 - -0.24082508 0.05839241 0.13135766 -0.03463621 + Number of Fisher Scoring iterations: 4 - residuals(model, type="working") + residuals(model, type = "pearson") + 1 2 3 4 + -0.16134949 0.20807694 -0.22544551 0.03258777 + residuals(model, type = "working") 1 2 3 4 - 0.091414181 -0.005374314 -0.027196998 0.001890910 - - residuals(model, type="response") - 1 2 3 4 - -0.6344390 0.3172195 0.2114797 -0.1586097 + 0.135315831 -0.084390309 0.113219135 -0.008279688 + residuals(model, type = "response") + 1 2 3 4 + -0.1923918 0.2565224 -0.1496381 0.0320653 */ val trainer = new GeneralizedLinearRegression() .setFamily("Gamma") .setWeightCol("weight") + .setOffsetCol("offset") + + val model = trainer.fit(dataset) - val model = trainer.fit(datasetWithWeight) - - val coefficientsR = Vectors.dense(Array(-0.72730, 0.23894)) - val interceptR = -0.81511 - val devianceResidualsR = Array(-0.26343, 0.05761, 0.12818, -0.03484) - val pearsonResidualsR = Array(-0.24082508, 0.05839241, 0.13135766, -0.03463621) - val workingResidualsR = Array(0.091414181, -0.005374314, -0.027196998, 0.001890910) - val responseResidualsR = Array(-0.6344390, 0.3172195, 0.2114797, -0.1586097) - val seCoefR = Array(0.16137, 0.05481, 0.23449) - val tValsR = Array(-4.507, 4.359, -3.476) - val pValsR = Array(0.139, 0.144, 0.178) - val dispersionR = 0.07986091 - val nullDevianceR = 2.937462 - val residualDevianceR = 0.090358 + val coefficientsR = Vectors.dense(Array(0.07695, 0.28068)) + val interceptR = -0.56474 + val devianceResidualsR = Array(-0.17095, 0.19867, -0.23604, 0.03241) + val pearsonResidualsR = Array(-0.16135, 0.20808, -0.22545, 0.03259) + val workingResidualsR = Array(0.13532, -0.08439, 0.11322, -0.00828) + val responseResidualsR = Array(-0.19239, 0.25652, -0.14964, 0.03207) + val seCoefR = Array(0.06931, 0.0732, 0.23866) + val tValsR = Array(1.11031, 3.83453, -2.3663) + val pValsR = Array(0.46675, 0.16241, 0.25454) + val dispersionR = 0.12122 + val nullDevianceR = 2.02568 + val residualDevianceR = 0.12546 val residualDegreeOfFreedomNullR = 3 val residualDegreeOfFreedomR = 1 - val aicR = 23.202 + val aicR = 0.93388 val summary = model.summary val devianceResiduals = summary.residuals() @@ -1271,77 +1353,81 @@ class GeneralizedLinearRegressionSuite assert(summary.solver === "irls") } - test("glm summary: tweedie family with weight") { + test("glm summary: tweedie family with weight and offset") { /* R code: - library(statmod) df <- as.data.frame(matrix(c( - 1.0, 1.0, 0.0, 5.0, - 0.5, 2.0, 1.0, 2.0, - 1.0, 3.0, 2.0, 1.0, - 0.0, 4.0, 3.0, 3.0), 4, 4, byrow = TRUE)) + 1.0, 1.0, 1.0, 0.0, 5.0, + 0.5, 2.0, 3.0, 1.0, 2.0, + 1.0, 3.0, 2.0, 2.0, 1.0, + 0.0, 4.0, 0.0, 3.0, 3.0), 4, 5, byrow = TRUE)) + */ + val dataset = Seq( + OffsetInstance(1.0, 1.0, 1.0, Vectors.dense(0.0, 5.0)), + OffsetInstance(0.5, 2.0, 3.0, Vectors.dense(1.0, 2.0)), + OffsetInstance(1.0, 3.0, 2.0, Vectors.dense(2.0, 1.0)), + OffsetInstance(0.0, 4.0, 0.0, Vectors.dense(3.0, 3.0)) + ).toDF() + /* + R code: - model <- glm(V1 ~ -1 + V3 + V4, data = df, weights = V2, - family = tweedie(var.power = 1.6, link.power = 0)) + library(statmod) + model <- glm(V1 ~ V4 + V5, data = df, weights = V2, offset = V3, + family = tweedie(var.power = 1.6, link.power = 0.0)) summary(model) Deviance Residuals: 1 2 3 4 - 0.6210 -0.0515 1.6935 -3.2539 + 0.8917 -2.1396 1.2252 -1.7946 Coefficients: - Estimate Std. Error t value Pr(>|t|) - V3 -0.4087 0.5205 -0.785 0.515 - V4 -0.1212 0.4082 -0.297 0.794 + Estimate Std. Error t value Pr(>|t|) + (Intercept) -0.03047 3.65000 -0.008 0.995 + V4 -1.14577 1.41674 -0.809 0.567 + V5 -0.36585 0.97065 -0.377 0.771 - (Dispersion parameter for Tweedie family taken to be 3.830036) + (Dispersion parameter for Tweedie family taken to be 6.334961) - Null deviance: 20.702 on 4 degrees of freedom - Residual deviance: 13.844 on 2 degrees of freedom + Null deviance: 12.784 on 3 degrees of freedom + Residual deviance: 10.095 on 1 degrees of freedom AIC: NA - Number of Fisher Scoring iterations: 11 - - residuals(model, type="pearson") - 1 2 3 4 - 0.7383616 -0.0509458 2.2348337 -1.4552090 - residuals(model, type="working") - 1 2 3 4 - 0.83354150 -0.04103552 1.55676369 -1.00000000 - residuals(model, type="response") - 1 2 3 4 - 0.45460738 -0.02139574 0.60888055 -0.20392801 + Number of Fisher Scoring iterations: 18 + + residuals(model, type = "pearson") + 1 2 3 4 + 1.1472554 -1.4642569 1.4935199 -0.8025842 + residuals(model, type = "working") + 1 2 3 4 + 1.3624928 -0.8322375 0.9894580 -1.0000000 + residuals(model, type = "response") + 1 2 3 4 + 0.57671828 -2.48040354 0.49735052 -0.01040646 */ - val datasetWithWeight = Seq( - Instance(1.0, 1.0, Vectors.dense(0.0, 5.0)), - Instance(0.5, 2.0, Vectors.dense(1.0, 2.0)), - Instance(1.0, 3.0, Vectors.dense(2.0, 1.0)), - Instance(0.0, 4.0, Vectors.dense(3.0, 3.0)) - ).toDF() - val trainer = new GeneralizedLinearRegression() .setFamily("tweedie") .setVariancePower(1.6) .setLinkPower(0.0) .setWeightCol("weight") - .setFitIntercept(false) - - val model = trainer.fit(datasetWithWeight) - val coefficientsR = Vectors.dense(Array(-0.408746, -0.12125)) - val interceptR = 0.0 - val devianceResidualsR = Array(0.621047, -0.051515, 1.693473, -3.253946) - val pearsonResidualsR = Array(0.738362, -0.050946, 2.234834, -1.455209) - val workingResidualsR = Array(0.833541, -0.041036, 1.556764, -1.0) - val responseResidualsR = Array(0.454607, -0.021396, 0.608881, -0.203928) - val seCoefR = Array(0.520519, 0.408215) - val tValsR = Array(-0.785267, -0.297024) - val pValsR = Array(0.514549, 0.794457) - val dispersionR = 3.830036 - val nullDevianceR = 20.702 - val residualDevianceR = 13.844 - val residualDegreeOfFreedomNullR = 4 - val residualDegreeOfFreedomR = 2 + .setOffsetCol("offset") + + val model = trainer.fit(dataset) + + val coefficientsR = Vectors.dense(Array(-1.14577, -0.36585)) + val interceptR = -0.03047 + val devianceResidualsR = Array(0.89171, -2.13961, 1.2252, -1.79463) + val pearsonResidualsR = Array(1.14726, -1.46426, 1.49352, -0.80258) + val workingResidualsR = Array(1.36249, -0.83224, 0.98946, -1) + val responseResidualsR = Array(0.57672, -2.4804, 0.49735, -0.01041) + val seCoefR = Array(1.41674, 0.97065, 3.65) + val tValsR = Array(-0.80873, -0.37691, -0.00835) + val pValsR = Array(0.56707, 0.77053, 0.99468) + val dispersionR = 6.33496 + val nullDevianceR = 12.78358 + val residualDevianceR = 10.09488 + val residualDegreeOfFreedomNullR = 3 + val residualDegreeOfFreedomR = 1 val summary = model.summary diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 9456031736528..7ffa150096333 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -561,6 +561,7 @@ primaryExpression | CASE whenClause+ (ELSE elseExpression=expression)? END #searchedCase | CASE value=expression whenClause+ (ELSE elseExpression=expression)? END #simpleCase | CAST '(' expression AS dataType ')' #cast + | STRUCT '(' (argument+=namedExpression (',' argument+=namedExpression)*)? ')' #struct | FIRST '(' expression (IGNORE NULLS)? ')' #first | LAST '(' expression (IGNORE NULLS)? ')' #last | POSITION '(' substr=valueExpression IN str=valueExpression ')' #position @@ -569,7 +570,7 @@ primaryExpression | qualifiedName '.' ASTERISK #star | '(' namedExpression (',' namedExpression)+ ')' #rowConstructor | '(' query ')' #subqueryExpression - | qualifiedName '(' (setQuantifier? namedExpression (',' namedExpression)*)? ')' + | qualifiedName '(' (setQuantifier? argument+=expression (',' argument+=expression)*)? ')' (OVER windowSpec)? #functionCall | value=primaryExpression '[' index=valueExpression ']' #subscript | identifier #columnReference diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala index 256f64e320be8..29110640d64f2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.types.{DataType, Decimal, StructType} +import org.apache.spark.unsafe.types.UTF8String /** * An abstract class for row used internally in Spark SQL, which only contains the columns as @@ -33,6 +35,10 @@ abstract class InternalRow extends SpecializedGetters with Serializable { def setNullAt(i: Int): Unit + /** + * Updates the value at column `i`. Note that after updating, the given value will be kept in this + * row, and the caller side should guarantee that this value won't be changed afterwards. + */ def update(i: Int, value: Any): Unit // default implementation (slow) @@ -58,7 +64,15 @@ abstract class InternalRow extends SpecializedGetters with Serializable { def copy(): InternalRow /** Returns true if there are any NULL values in this row. */ - def anyNull: Boolean + def anyNull: Boolean = { + val len = numFields + var i = 0 + while (i < len) { + if (isNullAt(i)) { return true } + i += 1 + } + false + } /* ---------------------- utility methods for Scala ---------------------- */ @@ -94,4 +108,15 @@ object InternalRow { /** Returns an empty [[InternalRow]]. */ val empty = apply() + + /** + * Copies the given value if it's string/struct/array/map type. + */ + def copyValue(value: Any): Any = value match { + case v: UTF8String => v.copy() + case v: InternalRow => v.copy() + case v: ArrayData => v.copy() + case v: MapData => v.copy() + case _ => value + } } 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 f6792569b704e..7c100afcd738f 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 @@ -170,6 +170,7 @@ package object dsl { case Seq() => UnresolvedStar(None) case target => UnresolvedStar(Option(target)) } + def namedStruct(e: Expression*): Expression = CreateNamedStruct(e) def callFunction[T, U]( func: T => U, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 43df19ba009a8..3862e64b9d828 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -1047,7 +1047,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String final $rowClass $result = new $rowClass(${fieldsCasts.length}); final InternalRow $tmpRow = $c; $fieldsEvalCode - $evPrim = $result.copy(); + $evPrim = $result; """ } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala index 74e0b4691d4cc..75feaf670c84a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types._ /** @@ -220,17 +219,6 @@ final class SpecificInternalRow(val values: Array[MutableValue]) extends BaseGen override def isNullAt(i: Int): Boolean = values(i).isNull - override def copy(): InternalRow = { - val newValues = new Array[Any](values.length) - var i = 0 - while (i < values.length) { - newValues(i) = values(i).boxed - i += 1 - } - - new GenericInternalRow(newValues) - } - override protected def genericGet(i: Int): Any = values(i).boxed override def update(ordinal: Int, value: Any) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala index 26cd9ab665383..0d2f9889a27d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala @@ -52,7 +52,7 @@ abstract class Collect[T <: Growable[Any] with Iterable[Any]] extends TypedImper // Do not allow null values. We follow the semantics of Hive's collect_list/collect_set here. // See: org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMkCollectionEvaluator if (value != null) { - buffer += value + buffer += InternalRow.copyValue(value) } buffer } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index fffcc7c9ef53a..7af4901435857 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -317,6 +317,9 @@ abstract class ImperativeAggregate extends AggregateFunction with CodegenFallbac * Updates its aggregation buffer, located in `mutableAggBuffer`, based on the given `inputRow`. * * Use `fieldNumber + mutableAggBufferOffset` to access fields of `mutableAggBuffer`. + * + * Note that, the input row may be produced by unsafe projection and it may not be safe to cache + * some fields of the input row, as the values can be changed unexpectedly. */ def update(mutableAggBuffer: InternalRow, inputRow: InternalRow): Unit @@ -326,6 +329,9 @@ abstract class ImperativeAggregate extends AggregateFunction with CodegenFallbac * * Use `fieldNumber + mutableAggBufferOffset` to access fields of `mutableAggBuffer`. * Use `fieldNumber + inputAggBufferOffset` to access fields of `inputAggBuffer`. + * + * Note that, the input row may be produced by unsafe projection and it may not be safe to cache + * some fields of the input row, as the values can be changed unexpectedly. */ def merge(mutableAggBuffer: InternalRow, inputAggBuffer: InternalRow): Unit } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 5158949b95629..b15bf2ca7c116 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -408,9 +408,11 @@ class CodegenContext { dataType match { case _ if isPrimitiveType(jt) => s"$row.set${primitiveTypeName(jt)}($ordinal, $value)" case t: DecimalType => s"$row.setDecimal($ordinal, $value, ${t.precision})" - // The UTF8String may came from UnsafeRow, otherwise clone is cheap (re-use the bytes) - case StringType => s"$row.update($ordinal, $value.clone())" case udt: UserDefinedType[_] => setColumn(row, udt.sqlType, ordinal, value) + // The UTF8String, InternalRow, ArrayData and MapData may came from UnsafeRow, we should copy + // it to avoid keeping a "pointer" to a memory region which may get updated afterwards. + case StringType | _: StructType | _: ArrayType | _: MapType => + s"$row.update($ordinal, $value.copy())" case _ => s"$row.update($ordinal, $value)" } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala index f708aeff2b146..dd0419d2286d1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala @@ -131,8 +131,6 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] case s: StructType => createCodeForStruct(ctx, input, s) case ArrayType(elementType, _) => createCodeForArray(ctx, input, elementType) case MapType(keyType, valueType, _) => createCodeForMap(ctx, input, keyType, valueType) - // UTF8String act as a pointer if it's inside UnsafeRow, so copy it to make it safe. - case StringType => ExprCode("", "false", s"$input.clone()") case udt: UserDefinedType[_] => convertToSafe(ctx, input, udt.sqlType) case _ => ExprCode("", "false", input) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index 751b821e1b009..65539a2f00e6c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -50,16 +50,6 @@ trait BaseGenericInternalRow extends InternalRow { override def getMap(ordinal: Int): MapData = getAs(ordinal) override def getStruct(ordinal: Int, numFields: Int): InternalRow = getAs(ordinal) - override def anyNull: Boolean = { - val len = numFields - var i = 0 - while (i < len) { - if (isNullAt(i)) { return true } - i += 1 - } - false - } - override def toString: String = { if (numFields == 0) { "[empty row]" @@ -79,6 +69,17 @@ trait BaseGenericInternalRow extends InternalRow { } } + override def copy(): GenericInternalRow = { + val len = numFields + val newValues = new Array[Any](len) + var i = 0 + while (i < len) { + newValues(i) = InternalRow.copyValue(genericGet(i)) + i += 1 + } + new GenericInternalRow(newValues) + } + override def equals(o: Any): Boolean = { if (!o.isInstanceOf[BaseGenericInternalRow]) { return false @@ -206,6 +207,4 @@ class GenericInternalRow(val values: Array[Any]) extends BaseGenericInternalRow override def setNullAt(i: Int): Unit = { values(i) = null} override def update(i: Int, value: Any): Unit = { values(i) = value } - - override def copy(): GenericInternalRow = this } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index ef79cbcaa0ce6..8eac3ef2d3568 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1061,6 +1061,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Cast(expression(ctx.expression), visitSparkDataType(ctx.dataType)) } + /** + * Create a [[CreateStruct]] expression. + */ + override def visitStruct(ctx: StructContext): Expression = withOrigin(ctx) { + CreateStruct(ctx.argument.asScala.map(expression)) + } + /** * Create a [[First]] expression. */ @@ -1091,7 +1098,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging // Create the function call. val name = ctx.qualifiedName.getText val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null) - val arguments = ctx.namedExpression().asScala.map(expression) match { + val arguments = ctx.argument.asScala.map(expression) match { case Seq(UnresolvedStar(None)) if name.toLowerCase(Locale.ROOT) == "count" && !isDistinct => // Transform COUNT(*) into COUNT(1). diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala index dd660c80a9c3c..9e39ed9c3a778 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala @@ -49,7 +49,15 @@ class GenericArrayData(val array: Array[Any]) extends ArrayData { def this(seqOrArray: Any) = this(GenericArrayData.anyToSeq(seqOrArray)) - override def copy(): ArrayData = new GenericArrayData(array.clone()) + override def copy(): ArrayData = { + val newValues = new Array[Any](array.length) + var i = 0 + while (i < array.length) { + newValues(i) = InternalRow.copyValue(array(i)) + i += 1 + } + new GenericArrayData(newValues) + } override def numElements(): Int = array.length diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index c641e4d3a23e1..25152f3e32d6b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -774,6 +774,14 @@ object SQLConf { .doubleConf .createWithDefault(0.05) + val AUTO_UPDATE_SIZE = + buildConf("spark.sql.statistics.autoUpdate.size") + .doc("Enables automatic update for table size once table's data is changed. Note that if " + + "the total number of files of the table is very large, this can be expensive and slow " + + "down data change commands.") + .booleanConf + .createWithDefault(false) + val CBO_ENABLED = buildConf("spark.sql.cbo.enabled") .doc("Enables CBO for estimation of plan statistics when set true.") @@ -1083,6 +1091,8 @@ class SQLConf extends Serializable with Logging { def cboEnabled: Boolean = getConf(SQLConf.CBO_ENABLED) + def autoUpdateSize: Boolean = getConf(SQLConf.AUTO_UPDATE_SIZE) + def joinReorderEnabled: Boolean = getConf(SQLConf.JOIN_REORDER_ENABLED) def joinReorderDPThreshold: Int = getConf(SQLConf.JOIN_REORDER_DP_THRESHOLD) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala index c9c9599e7f463..25699de33d717 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala @@ -121,10 +121,6 @@ class RowTest extends FunSpec with Matchers { externalRow should be theSameInstanceAs externalRow.copy() } - it("copy should return same ref for internal rows") { - internalRow should be theSameInstanceAs internalRow.copy() - } - it("toSeq should not expose internal state for external rows") { val modifiedValues = modifyValues(externalRow.toSeq) externalRow.toSeq should not equal modifiedValues diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MapDataSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MapDataSuite.scala deleted file mode 100644 index 25a675a90276d..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MapDataSuite.scala +++ /dev/null @@ -1,57 +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.catalyst.expressions - -import scala.collection._ - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.util.ArrayBasedMapData -import org.apache.spark.sql.types.{DataType, IntegerType, MapType, StringType} -import org.apache.spark.unsafe.types.UTF8String - -class MapDataSuite extends SparkFunSuite { - - test("inequality tests") { - def u(str: String): UTF8String = UTF8String.fromString(str) - - // test data - val testMap1 = Map(u("key1") -> 1) - val testMap2 = Map(u("key1") -> 1, u("key2") -> 2) - val testMap3 = Map(u("key1") -> 1) - val testMap4 = Map(u("key1") -> 1, u("key2") -> 2) - - // ArrayBasedMapData - val testArrayMap1 = ArrayBasedMapData(testMap1.toMap) - val testArrayMap2 = ArrayBasedMapData(testMap2.toMap) - val testArrayMap3 = ArrayBasedMapData(testMap3.toMap) - val testArrayMap4 = ArrayBasedMapData(testMap4.toMap) - assert(testArrayMap1 !== testArrayMap3) - assert(testArrayMap2 !== testArrayMap4) - - // UnsafeMapData - val unsafeConverter = UnsafeProjection.create(Array[DataType](MapType(StringType, IntegerType))) - val row = new GenericInternalRow(1) - def toUnsafeMap(map: ArrayBasedMapData): UnsafeMapData = { - row.update(0, map) - val unsafeRow = unsafeConverter.apply(row) - unsafeRow.getMap(0).copy - } - assert(toUnsafeMap(testArrayMap1) !== toUnsafeMap(testArrayMap3)) - assert(toUnsafeMap(testArrayMap2) !== toUnsafeMap(testArrayMap4)) - } -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala index 58ea5b9cb52d3..0cd0d8859145f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala @@ -172,4 +172,40 @@ class GeneratedProjectionSuite extends SparkFunSuite { assert(unsafe1 === unsafe3) assert(unsafe1.getStruct(1, 7) === unsafe3.getStruct(1, 7)) } + + test("MutableProjection should not cache content from the input row") { + val mutableProj = GenerateMutableProjection.generate( + Seq(BoundReference(0, new StructType().add("i", StringType), true))) + val row = new GenericInternalRow(1) + mutableProj.target(row) + + val unsafeProj = GenerateUnsafeProjection.generate( + Seq(BoundReference(0, new StructType().add("i", StringType), true))) + val unsafeRow = unsafeProj.apply(InternalRow(InternalRow(UTF8String.fromString("a")))) + + mutableProj.apply(unsafeRow) + assert(row.getStruct(0, 1).getString(0) == "a") + + // Even if the input row of the mutable projection has been changed, the target mutable row + // should keep same. + unsafeProj.apply(InternalRow(InternalRow(UTF8String.fromString("b")))) + assert(row.getStruct(0, 1).getString(0).toString == "a") + } + + test("SafeProjection should not cache content from the input row") { + val safeProj = GenerateSafeProjection.generate( + Seq(BoundReference(0, new StructType().add("i", StringType), true))) + + val unsafeProj = GenerateUnsafeProjection.generate( + Seq(BoundReference(0, new StructType().add("i", StringType), true))) + val unsafeRow = unsafeProj.apply(InternalRow(InternalRow(UTF8String.fromString("a")))) + + val row = safeProj.apply(unsafeRow) + assert(row.getStruct(0, 1).getString(0) == "a") + + // Even if the input row of the mutable projection has been changed, the target mutable row + // should keep same. + unsafeProj.apply(InternalRow(InternalRow(UTF8String.fromString("b")))) + assert(row.getStruct(0, 1).getString(0).toString == "a") + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 4d08f016a4a16..45f9f72dccc45 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -231,7 +231,7 @@ class ExpressionParserSuite extends PlanTest { assertEqual("foo(distinct a, b)", 'foo.distinctFunction('a, 'b)) assertEqual("grouping(distinct a, b)", 'grouping.distinctFunction('a, 'b)) assertEqual("`select`(all a, b)", 'select.function('a, 'b)) - assertEqual("foo(a as x, b as e)", 'foo.function('a as 'x, 'b as 'e)) + intercept("foo(a x)", "extraneous input 'x'") } test("window function expressions") { @@ -330,7 +330,9 @@ class ExpressionParserSuite extends PlanTest { assertEqual("a.b", UnresolvedAttribute("a.b")) assertEqual("`select`.b", UnresolvedAttribute("select.b")) assertEqual("(a + b).b", ('a + 'b).getField("b")) // This will fail analysis. - assertEqual("struct(a, b).b", 'struct.function('a, 'b).getField("b")) + assertEqual( + "struct(a, b).b", + namedStruct(NamePlaceholder, 'a, NamePlaceholder, 'b).getField("b")) } test("reference") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index bf15b85d5b510..5b2573fa4d601 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -223,6 +223,12 @@ class PlanParserSuite extends AnalysisTest { assertEqual(s"$sql grouping sets((a, b), (a), ())", GroupingSets(Seq(Seq('a, 'b), Seq('a), Seq()), Seq('a, 'b), table("d"), Seq('a, 'b, 'sum.function('c).as("c")))) + + val m = intercept[ParseException] { + parsePlan("SELECT a, b, count(distinct a, distinct b) as c FROM d GROUP BY a, b") + }.getMessage + assert(m.contains("extraneous input 'b'")) + } test("limit") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ComplexDataSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ComplexDataSuite.scala new file mode 100644 index 0000000000000..9d285916bcf42 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ComplexDataSuite.scala @@ -0,0 +1,107 @@ +/* + * 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.util + +import scala.collection._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow, SpecificInternalRow, UnsafeMapData, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.types.{DataType, IntegerType, MapType, StringType} +import org.apache.spark.unsafe.types.UTF8String + +class ComplexDataSuite extends SparkFunSuite { + def utf8(str: String): UTF8String = UTF8String.fromString(str) + + test("inequality tests for MapData") { + // test data + val testMap1 = Map(utf8("key1") -> 1) + val testMap2 = Map(utf8("key1") -> 1, utf8("key2") -> 2) + val testMap3 = Map(utf8("key1") -> 1) + val testMap4 = Map(utf8("key1") -> 1, utf8("key2") -> 2) + + // ArrayBasedMapData + val testArrayMap1 = ArrayBasedMapData(testMap1.toMap) + val testArrayMap2 = ArrayBasedMapData(testMap2.toMap) + val testArrayMap3 = ArrayBasedMapData(testMap3.toMap) + val testArrayMap4 = ArrayBasedMapData(testMap4.toMap) + assert(testArrayMap1 !== testArrayMap3) + assert(testArrayMap2 !== testArrayMap4) + + // UnsafeMapData + val unsafeConverter = UnsafeProjection.create(Array[DataType](MapType(StringType, IntegerType))) + val row = new GenericInternalRow(1) + def toUnsafeMap(map: ArrayBasedMapData): UnsafeMapData = { + row.update(0, map) + val unsafeRow = unsafeConverter.apply(row) + unsafeRow.getMap(0).copy + } + assert(toUnsafeMap(testArrayMap1) !== toUnsafeMap(testArrayMap3)) + assert(toUnsafeMap(testArrayMap2) !== toUnsafeMap(testArrayMap4)) + } + + test("GenericInternalRow.copy return a new instance that is independent from the old one") { + val project = GenerateUnsafeProjection.generate(Seq(BoundReference(0, StringType, true))) + val unsafeRow = project.apply(InternalRow(utf8("a"))) + + val genericRow = new GenericInternalRow(Array[Any](unsafeRow.getUTF8String(0))) + val copiedGenericRow = genericRow.copy() + assert(copiedGenericRow.getString(0) == "a") + project.apply(InternalRow(UTF8String.fromString("b"))) + // The copied internal row should not be changed externally. + assert(copiedGenericRow.getString(0) == "a") + } + + test("SpecificMutableRow.copy return a new instance that is independent from the old one") { + val project = GenerateUnsafeProjection.generate(Seq(BoundReference(0, StringType, true))) + val unsafeRow = project.apply(InternalRow(utf8("a"))) + + val mutableRow = new SpecificInternalRow(Seq(StringType)) + mutableRow(0) = unsafeRow.getUTF8String(0) + val copiedMutableRow = mutableRow.copy() + assert(copiedMutableRow.getString(0) == "a") + project.apply(InternalRow(UTF8String.fromString("b"))) + // The copied internal row should not be changed externally. + assert(copiedMutableRow.getString(0) == "a") + } + + test("GenericArrayData.copy return a new instance that is independent from the old one") { + val project = GenerateUnsafeProjection.generate(Seq(BoundReference(0, StringType, true))) + val unsafeRow = project.apply(InternalRow(utf8("a"))) + + val genericArray = new GenericArrayData(Array[Any](unsafeRow.getUTF8String(0))) + val copiedGenericArray = genericArray.copy() + assert(copiedGenericArray.getUTF8String(0).toString == "a") + project.apply(InternalRow(UTF8String.fromString("b"))) + // The copied array data should not be changed externally. + assert(copiedGenericArray.getUTF8String(0).toString == "a") + } + + test("copy on nested complex type") { + val project = GenerateUnsafeProjection.generate(Seq(BoundReference(0, StringType, true))) + val unsafeRow = project.apply(InternalRow(utf8("a"))) + + val arrayOfRow = new GenericArrayData(Array[Any](InternalRow(unsafeRow.getUTF8String(0)))) + val copied = arrayOfRow.copy() + assert(copied.getStruct(0, 1).getUTF8String(0).toString == "a") + project.apply(InternalRow(UTF8String.fromString("b"))) + // The copied data should not be changed externally. + assert(copied.getStruct(0, 1).getUTF8String(0).toString == "a") + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java index e23a64350cbc5..34dc3af9b85c8 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java @@ -149,7 +149,7 @@ public InternalRow copy() { } else if (dt instanceof DoubleType) { row.setDouble(i, getDouble(i)); } else if (dt instanceof StringType) { - row.update(i, getUTF8String(i)); + row.update(i, getUTF8String(i).copy()); } else if (dt instanceof BinaryType) { row.update(i, getBinary(i)); } else if (dt instanceof DecimalType) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala index bea2dce1a7657..a5a444b160c63 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala @@ -86,17 +86,6 @@ class SortBasedAggregationIterator( // The aggregation buffer used by the sort-based aggregation. private[this] val sortBasedAggregationBuffer: InternalRow = newBuffer - // This safe projection is used to turn the input row into safe row. This is necessary - // because the input row may be produced by unsafe projection in child operator and all the - // produced rows share one byte array. However, when we update the aggregate buffer according to - // the input row, we may cache some values from input row, e.g. `Max` will keep the max value from - // input row via MutableProjection, `CollectList` will keep all values in an array via - // ImperativeAggregate framework. These values may get changed unexpectedly if the underlying - // unsafe projection update the shared byte array. By applying a safe projection to the input row, - // we can cut down the connection from input row to the shared byte array, and thus it's safe to - // cache values from input row while updating the aggregation buffer. - private[this] val safeProj: Projection = FromUnsafeProjection(valueAttributes.map(_.dataType)) - protected def initialize(): Unit = { if (inputIterator.hasNext) { initializeBuffer(sortBasedAggregationBuffer) @@ -119,7 +108,7 @@ class SortBasedAggregationIterator( // We create a variable to track if we see the next group. var findNextPartition = false // firstRowInNextGroup is the first row of this group. We first process it. - processRow(sortBasedAggregationBuffer, safeProj(firstRowInNextGroup)) + processRow(sortBasedAggregationBuffer, firstRowInNextGroup) // The search will stop when we see the next group or there is no // input row left in the iter. @@ -130,7 +119,7 @@ class SortBasedAggregationIterator( // Check if the current row belongs the current input row. if (currentGroupingKey == groupingKey) { - processRow(sortBasedAggregationBuffer, safeProj(currentRow)) + processRow(sortBasedAggregationBuffer, currentRow) } else { // We find a new group. findNextPartition = true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index d3fa0dcd2d7c3..fc977f2fd5530 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -56,7 +56,6 @@ class MutableUnsafeRow(val writer: UnsafeRowWriter) extends BaseGenericInternalR // all other methods inherited from GenericMutableRow are not need override protected def genericGet(ordinal: Int): Any = throw new UnsupportedOperationException override def numFields: Int = throw new UnsupportedOperationException - override def copy(): InternalRow = throw new UnsupportedOperationException } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala index 92397607f38fd..fce12cc96620c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala @@ -36,7 +36,14 @@ object CommandUtils extends Logging { def updateTableStats(sparkSession: SparkSession, table: CatalogTable): Unit = { if (table.stats.nonEmpty) { val catalog = sparkSession.sessionState.catalog - catalog.alterTableStats(table.identifier, None) + if (sparkSession.sessionState.conf.autoUpdateSize) { + val newTable = catalog.getTableMetadata(table.identifier) + val newSize = CommandUtils.calculateTotalSize(sparkSession.sessionState, newTable) + val newStats = CatalogStatistics(sizeInBytes = newSize) + catalog.alterTableStats(table.identifier, Some(newStats)) + } else { + catalog.alterTableStats(table.identifier, None) + } } } @@ -84,7 +91,9 @@ object CommandUtils extends Logging { size } - locationUri.map { p => + val startTime = System.nanoTime() + logInfo(s"Starting to calculate the total file size under path $locationUri.") + val size = locationUri.map { p => val path = new Path(p) try { val fs = path.getFileSystem(sessionState.newHadoopConf()) @@ -97,6 +106,10 @@ object CommandUtils extends Logging { 0L } }.getOrElse(0L) + val durationInMs = (System.nanoTime() - startTime) / (1000 * 1000) + logInfo(s"It took $durationInMs ms to calculate the total file size under path $locationUri.") + + size } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index ac897c1b22d77..ba7ca84f229fc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -437,7 +437,20 @@ case class AlterTableAddPartitionCommand( } catalog.createPartitions(table.identifier, parts, ignoreIfExists = ifNotExists) - CommandUtils.updateTableStats(sparkSession, table) + if (table.stats.nonEmpty) { + if (sparkSession.sessionState.conf.autoUpdateSize) { + val addedSize = parts.map { part => + CommandUtils.calculateLocationSize(sparkSession.sessionState, table.identifier, + part.storage.locationUri) + }.sum + if (addedSize > 0) { + val newStats = CatalogStatistics(sizeInBytes = table.stats.get.sizeInBytes + addedSize) + catalog.alterTableStats(table.identifier, Some(newStats)) + } + } else { + catalog.alterTableStats(table.identifier, None) + } + } Seq.empty[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index b09edf380c2d4..0396168d3f311 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -215,7 +215,7 @@ trait HashJoin { // At the end of the task, we update the avg hash probe. TaskContext.get().addTaskCompletionListener(_ => - avgHashProbe.set(hashed.getAverageProbesPerLookup())) + avgHashProbe.set(hashed.getAverageProbesPerLookup)) val resultProj = createResultProjection joinedIter.map { r => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 3c702856114f9..2038cb9edb67d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -83,7 +83,7 @@ private[execution] sealed trait HashedRelation extends KnownSizeEstimation { /** * Returns the average number of probes per key lookup. */ - def getAverageProbesPerLookup(): Double + def getAverageProbesPerLookup: Double } private[execution] object HashedRelation { @@ -280,7 +280,7 @@ private[joins] class UnsafeHashedRelation( read(in.readInt, in.readLong, in.readBytes) } - override def getAverageProbesPerLookup(): Double = binaryMap.getAverageProbesPerLookup() + override def getAverageProbesPerLookup: Double = binaryMap.getAverageProbesPerLookup } private[joins] object UnsafeHashedRelation { @@ -776,7 +776,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap /** * Returns the average number of probes per key lookup. */ - def getAverageProbesPerLookup(): Double = numProbes.toDouble / numKeyLookups + def getAverageProbesPerLookup: Double = numProbes.toDouble / numKeyLookups } private[joins] class LongHashedRelation( @@ -829,7 +829,7 @@ private[joins] class LongHashedRelation( map = in.readObject().asInstanceOf[LongToUnsafeRowMap] } - override def getAverageProbesPerLookup(): Double = map.getAverageProbesPerLookup() + override def getAverageProbesPerLookup: Double = map.getAverageProbesPerLookup } /** diff --git a/sql/core/src/test/resources/sql-tests/inputs/struct.sql b/sql/core/src/test/resources/sql-tests/inputs/struct.sql index e56344dc4de80..93a1238ab18c2 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/struct.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/struct.sql @@ -18,3 +18,10 @@ SELECT ID, STRUCT(ST.*,CAST(ID AS STRING) AS E) NST FROM tbl_x; -- Prepend a column to a struct SELECT ID, STRUCT(CAST(ID AS STRING) AS AA, ST.*) NST FROM tbl_x; + +-- Select a column from a struct +SELECT ID, STRUCT(ST.*).C NST FROM tbl_x; +SELECT ID, STRUCT(ST.C, ST.D).D NST FROM tbl_x; + +-- Select an alias from a struct +SELECT ID, STRUCT(ST.C as STC, ST.D as STD).STD FROM tbl_x; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/struct.sql.out b/sql/core/src/test/resources/sql-tests/results/struct.sql.out index 3e32f46195464..1da33bc736f0b 100644 --- a/sql/core/src/test/resources/sql-tests/results/struct.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/struct.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 6 +-- Number of queries: 9 -- !query 0 @@ -58,3 +58,33 @@ struct> 1 {"AA":"1","C":"gamma","D":"delta"} 2 {"AA":"2","C":"epsilon","D":"eta"} 3 {"AA":"3","C":"theta","D":"iota"} + + +-- !query 6 +SELECT ID, STRUCT(ST.*).C NST FROM tbl_x +-- !query 6 schema +struct +-- !query 6 output +1 gamma +2 epsilon +3 theta + + +-- !query 7 +SELECT ID, STRUCT(ST.C, ST.D).D NST FROM tbl_x +-- !query 7 schema +struct +-- !query 7 output +1 delta +2 eta +3 iota + + +-- !query 8 +SELECT ID, STRUCT(ST.C as STC, ST.D as STD).STD FROM tbl_x +-- !query 8 schema +struct +-- !query 8 output +1 delta +2 eta +3 iota diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index 1255c49104718..204858fa29787 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -19,8 +19,9 @@ package org.apache.spark.sql import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction, Window} import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{DataType, LongType, StructType} +import org.apache.spark.sql.types._ /** * Window function testing for DataFrame API. @@ -423,4 +424,48 @@ class DataFrameWindowFunctionsSuite extends QueryTest with SharedSQLContext { df.select(selectList: _*).where($"value" < 2), Seq(Row(3, "1", null, 3.0, 4.0, 3.0), Row(5, "1", false, 4.0, 5.0, 5.0))) } + + test("SPARK-21258: complex object in combination with spilling") { + // Make sure we trigger the spilling path. + withSQLConf(SQLConf.WINDOW_EXEC_BUFFER_SPILL_THRESHOLD.key -> "17") { + val sampleSchema = new StructType(). + add("f0", StringType). + add("f1", LongType). + add("f2", ArrayType(new StructType(). + add("f20", StringType))). + add("f3", ArrayType(new StructType(). + add("f30", StringType))) + + val w0 = Window.partitionBy("f0").orderBy("f1") + val w1 = w0.rowsBetween(Long.MinValue, Long.MaxValue) + + val c0 = first(struct($"f2", $"f3")).over(w0) as "c0" + val c1 = last(struct($"f2", $"f3")).over(w1) as "c1" + + val input = + """{"f1":1497820153720,"f2":[{"f20":"x","f21":0}],"f3":[{"f30":"x","f31":0}]} + |{"f1":1497802179638} + |{"f1":1497802189347} + |{"f1":1497802189593} + |{"f1":1497802189597} + |{"f1":1497802189599} + |{"f1":1497802192103} + |{"f1":1497802193414} + |{"f1":1497802193577} + |{"f1":1497802193709} + |{"f1":1497802202883} + |{"f1":1497802203006} + |{"f1":1497802203743} + |{"f1":1497802203834} + |{"f1":1497802203887} + |{"f1":1497802203893} + |{"f1":1497802203976} + |{"f1":1497820168098} + |""".stripMargin.split("\n").toSeq + + import testImplicits._ + + spark.read.schema(sampleSchema).json(input.toDS()).select(c0, c1).foreach { _ => () } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala index b031c52dad8b5..d9392de37a815 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogStatistics import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.LogicalRelation -import org.apache.spark.sql.internal.StaticSQLConf +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.sql.test.SQLTestData.ArrayData import org.apache.spark.sql.types._ @@ -178,36 +178,63 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared test("change stats after set location command") { val table = "change_stats_set_location_table" - withTable(table) { - spark.range(100).select($"id", $"id" % 5 as "value").write.saveAsTable(table) - // analyze to get initial stats - sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS id, value") - val fetched1 = checkTableStats( - table, hasSizeInBytes = true, expectedRowCounts = Some(100)) - assert(fetched1.get.sizeInBytes > 0) - assert(fetched1.get.colStats.size == 2) - - // set location command - withTempDir { newLocation => - sql(s"ALTER TABLE $table SET LOCATION '${newLocation.toURI.toString}'") - checkTableStats(table, hasSizeInBytes = false, expectedRowCounts = None) + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + spark.range(100).select($"id", $"id" % 5 as "value").write.saveAsTable(table) + // analyze to get initial stats + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS id, value") + val fetched1 = checkTableStats( + table, hasSizeInBytes = true, expectedRowCounts = Some(100)) + assert(fetched1.get.sizeInBytes > 0) + assert(fetched1.get.colStats.size == 2) + + // set location command + val initLocation = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) + .storage.locationUri.get.toString + withTempDir { newLocation => + sql(s"ALTER TABLE $table SET LOCATION '${newLocation.toURI.toString}'") + if (autoUpdate) { + val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched2.get.sizeInBytes == 0) + assert(fetched2.get.colStats.isEmpty) + + // set back to the initial location + sql(s"ALTER TABLE $table SET LOCATION '$initLocation'") + val fetched3 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched3.get.sizeInBytes == fetched1.get.sizeInBytes) + } else { + checkTableStats(table, hasSizeInBytes = false, expectedRowCounts = None) + } + } + } } } } test("change stats after insert command for datasource table") { val table = "change_stats_insert_datasource_table" - withTable(table) { - sql(s"CREATE TABLE $table (i int, j string) USING PARQUET") - // analyze to get initial stats - sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") - val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) - assert(fetched1.get.sizeInBytes == 0) - assert(fetched1.get.colStats.size == 2) - - // insert into command - sql(s"INSERT INTO TABLE $table SELECT 1, 'abc'") - checkTableStats(table, hasSizeInBytes = false, expectedRowCounts = None) + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + sql(s"CREATE TABLE $table (i int, j string) USING PARQUET") + // analyze to get initial stats + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") + val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) + assert(fetched1.get.sizeInBytes == 0) + assert(fetched1.get.colStats.size == 2) + + // insert into command + sql(s"INSERT INTO TABLE $table SELECT 1, 'abc'") + if (autoUpdate) { + val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched2.get.sizeInBytes > 0) + assert(fetched2.get.colStats.isEmpty) + } else { + checkTableStats(table, hasSizeInBytes = false, expectedRowCounts = None) + } + } + } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 5fd266c2d033c..c601038a2b0af 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -444,88 +444,133 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto test("change stats after insert command for hive table") { val table = s"change_stats_insert_hive_table" - withTable(table) { - sql(s"CREATE TABLE $table (i int, j string)") - // analyze to get initial stats - sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") - val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) - assert(fetched1.get.sizeInBytes == 0) - assert(fetched1.get.colStats.size == 2) - - // insert into command - sql(s"INSERT INTO TABLE $table SELECT 1, 'abc'") - assert(getStatsProperties(table).isEmpty) + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + sql(s"CREATE TABLE $table (i int, j string)") + // analyze to get initial stats + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") + val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) + assert(fetched1.get.sizeInBytes == 0) + assert(fetched1.get.colStats.size == 2) + + // insert into command + sql(s"INSERT INTO TABLE $table SELECT 1, 'abc'") + if (autoUpdate) { + val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched2.get.sizeInBytes > 0) + assert(fetched2.get.colStats.isEmpty) + val statsProp = getStatsProperties(table) + assert(statsProp(STATISTICS_TOTAL_SIZE).toLong == fetched2.get.sizeInBytes) + } else { + assert(getStatsProperties(table).isEmpty) + } + } + } } } test("change stats after load data command") { val table = "change_stats_load_table" - withTable(table) { - sql(s"CREATE TABLE $table (i INT, j STRING) STORED AS PARQUET") - // analyze to get initial stats - sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") - val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) - assert(fetched1.get.sizeInBytes == 0) - assert(fetched1.get.colStats.size == 2) - - withTempDir { loadPath => - // load data command - val file = new File(loadPath + "/data") - val writer = new PrintWriter(file) - writer.write("2,xyz") - writer.close() - sql(s"LOAD DATA INPATH '${loadPath.toURI.toString}' INTO TABLE $table") - assert(getStatsProperties(table).isEmpty) + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + sql(s"CREATE TABLE $table (i INT, j STRING) STORED AS PARQUET") + // analyze to get initial stats + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") + val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) + assert(fetched1.get.sizeInBytes == 0) + assert(fetched1.get.colStats.size == 2) + + withTempDir { loadPath => + // load data command + val file = new File(loadPath + "/data") + val writer = new PrintWriter(file) + writer.write("2,xyz") + writer.close() + sql(s"LOAD DATA INPATH '${loadPath.toURI.toString}' INTO TABLE $table") + if (autoUpdate) { + val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched2.get.sizeInBytes > 0) + assert(fetched2.get.colStats.isEmpty) + val statsProp = getStatsProperties(table) + assert(statsProp(STATISTICS_TOTAL_SIZE).toLong == fetched2.get.sizeInBytes) + } else { + assert(getStatsProperties(table).isEmpty) + } + } + } } } } test("change stats after add/drop partition command") { val table = "change_stats_part_table" - withTable(table) { - sql(s"CREATE TABLE $table (i INT, j STRING) PARTITIONED BY (ds STRING, hr STRING)") - // table has two partitions initially - for (ds <- Seq("2008-04-08"); hr <- Seq("11", "12")) { - sql(s"INSERT OVERWRITE TABLE $table PARTITION (ds='$ds',hr='$hr') SELECT 1, 'a'") - } - // analyze to get initial stats - sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") - val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(2)) - assert(fetched1.get.sizeInBytes > 0) - assert(fetched1.get.colStats.size == 2) - - withTempPaths(numPaths = 2) { case Seq(dir1, dir2) => - val file1 = new File(dir1 + "/data") - val writer1 = new PrintWriter(file1) - writer1.write("1,a") - writer1.close() - - val file2 = new File(dir2 + "/data") - val writer2 = new PrintWriter(file2) - writer2.write("1,a") - writer2.close() - - // add partition command - sql( - s""" - |ALTER TABLE $table ADD - |PARTITION (ds='2008-04-09', hr='11') LOCATION '${dir1.toURI.toString}' - |PARTITION (ds='2008-04-09', hr='12') LOCATION '${dir2.toURI.toString}' - """.stripMargin) - assert(getStatsProperties(table).isEmpty) - - // generate stats again - sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") - val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(4)) - assert(fetched2.get.sizeInBytes > 0) - assert(fetched2.get.colStats.size == 2) - - // drop partition command - sql(s"ALTER TABLE $table DROP PARTITION (ds='2008-04-08'), PARTITION (hr='12')") - // only one partition left - assert(spark.sessionState.catalog.listPartitions(TableIdentifier(table)) - .map(_.spec).toSet == Set(Map("ds" -> "2008-04-09", "hr" -> "11"))) - assert(getStatsProperties(table).isEmpty) + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + sql(s"CREATE TABLE $table (i INT, j STRING) PARTITIONED BY (ds STRING, hr STRING)") + // table has two partitions initially + for (ds <- Seq("2008-04-08"); hr <- Seq("11", "12")) { + sql(s"INSERT OVERWRITE TABLE $table PARTITION (ds='$ds',hr='$hr') SELECT 1, 'a'") + } + // analyze to get initial stats + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") + val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(2)) + assert(fetched1.get.sizeInBytes > 0) + assert(fetched1.get.colStats.size == 2) + + withTempPaths(numPaths = 2) { case Seq(dir1, dir2) => + val file1 = new File(dir1 + "/data") + val writer1 = new PrintWriter(file1) + writer1.write("1,a") + writer1.close() + + val file2 = new File(dir2 + "/data") + val writer2 = new PrintWriter(file2) + writer2.write("1,a") + writer2.close() + + // add partition command + sql( + s""" + |ALTER TABLE $table ADD + |PARTITION (ds='2008-04-09', hr='11') LOCATION '${dir1.toURI.toString}' + |PARTITION (ds='2008-04-09', hr='12') LOCATION '${dir2.toURI.toString}' + """.stripMargin) + if (autoUpdate) { + val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched2.get.sizeInBytes > fetched1.get.sizeInBytes) + assert(fetched2.get.colStats.isEmpty) + val statsProp = getStatsProperties(table) + assert(statsProp(STATISTICS_TOTAL_SIZE).toLong == fetched2.get.sizeInBytes) + } else { + assert(getStatsProperties(table).isEmpty) + } + + // now the table has four partitions, generate stats again + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") + val fetched3 = checkTableStats( + table, hasSizeInBytes = true, expectedRowCounts = Some(4)) + assert(fetched3.get.sizeInBytes > 0) + assert(fetched3.get.colStats.size == 2) + + // drop partition command + sql(s"ALTER TABLE $table DROP PARTITION (ds='2008-04-08'), PARTITION (hr='12')") + assert(spark.sessionState.catalog.listPartitions(TableIdentifier(table)) + .map(_.spec).toSet == Set(Map("ds" -> "2008-04-09", "hr" -> "11"))) + // only one partition left + if (autoUpdate) { + val fetched4 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched4.get.sizeInBytes < fetched1.get.sizeInBytes) + assert(fetched4.get.colStats.isEmpty) + val statsProp = getStatsProperties(table) + assert(statsProp(STATISTICS_TOTAL_SIZE).toLong == fetched4.get.sizeInBytes) + } else { + assert(getStatsProperties(table).isEmpty) + } + } + } } } }