Skip to content
Snippets Groups Projects
  1. Jun 15, 2017
    • Michael Gummelt's avatar
      [SPARK-20434][YARN][CORE] Move Hadoop delegation token code from yarn to core · a18d6371
      Michael Gummelt authored
      ## What changes were proposed in this pull request?
      
      Move Hadoop delegation token code from `spark-yarn` to `spark-core`, so that other schedulers (such as Mesos), may use it.  In order to avoid exposing Hadoop interfaces in spark-core, the new Hadoop delegation token classes are kept private.  In order to provider backward compatiblity, and to allow YARN users to continue to load their own delegation token providers via Java service loading, the old YARN interfaces, as well as the client code that uses them, have been retained.
      
      Summary:
      - Move registered `yarn.security.ServiceCredentialProvider` classes from `spark-yarn` to `spark-core`.  Moved them into a new, private hierarchy under `HadoopDelegationTokenProvider`.  Client code in `HadoopDelegationTokenManager` now loads credentials from a whitelist of three providers (`HadoopFSDelegationTokenProvider`, `HiveDelegationTokenProvider`, `HBaseDelegationTokenProvider`), instead of service loading, which means that users are not able to implement their own delegation token providers, as they are in the `spark-yarn` module.
      
      - The `yarn.security.ServiceCredentialProvider` interface has been kept for backwards compatibility, and to continue to allow YARN users to implement their own delegation token provider implementations.  Client code in YARN now fetches tokens via the new `YARNHadoopDelegationTokenManager` class, which fetches tokens from the core providers through `HadoopDelegationTokenManager`, as well as service loads them from `yarn.security.ServiceCredentialProvider`.
      
      Old Hierarchy:
      
      ```
      yarn.security.ServiceCredentialProvider (service loaded)
        HadoopFSCredentialProvider
        HiveCredentialProvider
        HBaseCredentialProvider
      yarn.security.ConfigurableCredentialManager
      ```
      
      New Hierarchy:
      
      ```
      HadoopDelegationTokenManager
      HadoopDelegationTokenProvider (not service loaded)
        HadoopFSDelegationTokenProvider
        HiveDelegationTokenProvider
        HBaseDelegationTokenProvider
      
      yarn.security.ServiceCredentialProvider (service loaded)
      yarn.security.YARNHadoopDelegationTokenManager
      ```
      ## How was this patch tested?
      
      unit tests
      
      Author: Michael Gummelt <mgummelt@mesosphere.io>
      Author: Dr. Stefan Schimanski <sttts@mesosphere.io>
      
      Closes #17723 from mgummelt/SPARK-20434-refactor-kerberos.
      a18d6371
    • Xingbo Jiang's avatar
      [SPARK-16251][SPARK-20200][CORE][TEST] Flaky test:... · 7dc3e697
      Xingbo Jiang authored
      [SPARK-16251][SPARK-20200][CORE][TEST] Flaky test: org.apache.spark.rdd.LocalCheckpointSuite.missing checkpoint block fails with informative message
      
      ## What changes were proposed in this pull request?
      
      Currently we don't wait to confirm the removal of the block from the slave's BlockManager, if the removal takes too much time, we will fail the assertion in this test case.
      The failure can be easily reproduced if we sleep for a while before we remove the block in BlockManagerSlaveEndpoint.receiveAndReply().
      
      ## How was this patch tested?
      N/A
      
      Author: Xingbo Jiang <xingbo.jiang@databricks.com>
      
      Closes #18314 from jiangxb1987/LocalCheckpointSuite.
      7dc3e697
    • Felix Cheung's avatar
      [SPARK-20980][DOCS] update doc to reflect multiLine change · 1bf55e39
      Felix Cheung authored
      ## What changes were proposed in this pull request?
      
      doc only change
      
      ## How was this patch tested?
      
      manually
      
      Author: Felix Cheung <felixcheung_m@hotmail.com>
      
      Closes #18312 from felixcheung/sqljsonwholefiledoc.
      1bf55e39
    • ALeksander Eskilson's avatar
      [SPARK-18016][SQL][CATALYST] Code Generation: Constant Pool Limit - Class Splitting · b32b2123
      ALeksander Eskilson authored
      ## What changes were proposed in this pull request?
      
      This pull-request exclusively includes the class splitting feature described in #16648. When code for a given class would grow beyond 1600k bytes, a private, nested sub-class is generated into which subsequent functions are inlined. Additional sub-classes are generated as the code threshold is met subsequent times. This code includes 3 changes:
      
      1. Includes helper maps, lists, and functions for keeping track of sub-classes during code generation (included in the `CodeGenerator` class). These helper functions allow nested classes and split functions to be initialized/declared/inlined to the appropriate locations in the various projection classes.
      2. Changes `addNewFunction` to return a string to support instances where a split function is inlined to a nested class and not the outer class (and so must be invoked using the class-qualified name). Uses of `addNewFunction` throughout the codebase are modified so that the returned name is properly used.
      3. Removes instances of the `this` keyword when used on data inside generated classes. All state declared in the outer class is by default global and accessible to the nested classes. However, if a reference to global state in a nested class is prepended with the `this` keyword, it would attempt to reference state belonging to the nested class (which would not exist), rather than the correct variable belonging to the outer class.
      
      ## How was this patch tested?
      
      Added a test case to the `GeneratedProjectionSuite` that increases the number of columns tested in various projections to a threshold that would previously have triggered a `JaninoRuntimeException` for the Constant Pool.
      
      Note: This PR does not address the second Constant Pool issue with code generation (also mentioned in #16648): excess global mutable state. A second PR may be opened to resolve that issue.
      
      Author: ALeksander Eskilson <alek.eskilson@cerner.com>
      
      Closes #18075 from bdrillard/class_splitting_only.
      b32b2123
    • Xiao Li's avatar
      [SPARK-20980][SQL] Rename `wholeFile` to `multiLine` for both CSV and JSON · 20514281
      Xiao Li authored
      ### What changes were proposed in this pull request?
      The current option name `wholeFile` is misleading for CSV users. Currently, it is not representing a record per file. Actually, one file could have multiple records. Thus, we should rename it. Now, the proposal is `multiLine`.
      
      ### How was this patch tested?
      N/A
      
      Author: Xiao Li <gatorsmile@gmail.com>
      
      Closes #18202 from gatorsmile/renameCVSOption.
      20514281
    • Reynold Xin's avatar
      [SPARK-21092][SQL] Wire SQLConf in logical plan and expressions · fffeb6d7
      Reynold Xin authored
      ## What changes were proposed in this pull request?
      It is really painful to not have configs in logical plan and expressions. We had to add all sorts of hacks (e.g. pass SQLConf explicitly in functions). This patch exposes SQLConf in logical plan, using a thread local variable and a getter closure that's set once there is an active SparkSession.
      
      The implementation is a bit of a hack, since we didn't anticipate this need in the beginning (config was only exposed in physical plan). The implementation is described in `SQLConf.get`.
      
      In terms of future work, we should follow up to clean up CBO (remove the need for passing in config).
      
      ## How was this patch tested?
      Updated relevant tests for constraint propagation.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #18299 from rxin/SPARK-21092.
      fffeb6d7
  2. Jun 14, 2017
  3. Jun 13, 2017
  4. Jun 12, 2017
    • Dongjoon Hyun's avatar
      [SPARK-19910][SQL] `stack` should not reject NULL values due to type mismatch · 2639c3ed
      Dongjoon Hyun authored
      ## What changes were proposed in this pull request?
      
      Since `stack` function generates a table with nullable columns, it should allow mixed null values.
      
      ```scala
      scala> sql("select stack(3, 1, 2, 3)").printSchema
      root
       |-- col0: integer (nullable = true)
      
      scala> sql("select stack(3, 1, 2, null)").printSchema
      org.apache.spark.sql.AnalysisException: cannot resolve 'stack(3, 1, 2, NULL)' due to data type mismatch: Argument 1 (IntegerType) != Argument 3 (NullType); line 1 pos 7;
      ```
      
      ## How was this patch tested?
      
      Pass the Jenkins with a new test case.
      
      Author: Dongjoon Hyun <dongjoon@apache.org>
      
      Closes #17251 from dongjoon-hyun/SPARK-19910.
      2639c3ed
    • Wenchen Fan's avatar
    • Shixiong Zhu's avatar
      [SPARK-20979][SS] Add RateSource to generate values for tests and benchmark · 74a432d3
      Shixiong Zhu authored
      ## What changes were proposed in this pull request?
      
      This PR adds RateSource for Structured Streaming so that the user can use it to generate data for tests and benchmark easily.
      
      This source generates increment long values with timestamps. Each generated row has two columns: a timestamp column for the generated time and an auto increment long column starting with 0L.
      
      It supports the following options:
      - `rowsPerSecond` (e.g. 100, default: 1): How many rows should be generated per second.
      - `rampUpTime` (e.g. 5s, default: 0s): How long to ramp up before the generating speed becomes `rowsPerSecond`. Using finer granularities than seconds will be truncated to integer seconds.
      - `numPartitions` (e.g. 10, default: Spark's default parallelism): The partition number for the generated rows. The source will try its best to reach `rowsPerSecond`, but the query may be resource constrained, and `numPartitions` can be tweaked to help reach the desired speed.
      
      Here is a simple example that prints 10 rows per seconds:
      ```
          spark.readStream
            .format("rate")
            .option("rowsPerSecond", "10")
            .load()
            .writeStream
            .format("console")
            .start()
      ```
      
      The idea came from marmbrus and he did the initial work.
      
      ## How was this patch tested?
      
      The added tests.
      
      Author: Shixiong Zhu <shixiong@databricks.com>
      Author: Michael Armbrust <michael@databricks.com>
      
      Closes #18199 from zsxwing/rate.
      74a432d3
    • Joseph K. Bradley's avatar
      [SPARK-21050][ML] Word2vec persistence overflow bug fix · ff318c0d
      Joseph K. Bradley authored
      ## What changes were proposed in this pull request?
      
      The method calculateNumberOfPartitions() uses Int, not Long (unlike the MLlib version), so it is very easily to have an overflow in calculating the number of partitions for ML persistence.
      
      This modifies the calculations to use Long.
      
      ## How was this patch tested?
      
      New unit test.  I verified that the test fails before this patch.
      
      Author: Joseph K. Bradley <joseph@databricks.com>
      
      Closes #18265 from jkbradley/word2vec-save-fix.
      ff318c0d
    • Reynold Xin's avatar
      [SPARK-21059][SQL] LikeSimplification can NPE on null pattern · b1436c74
      Reynold Xin authored
      ## What changes were proposed in this pull request?
      This patch fixes a bug that can cause NullPointerException in LikeSimplification, when the pattern for like is null.
      
      ## How was this patch tested?
      Added a new unit test case in LikeSimplificationSuite.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #18273 from rxin/SPARK-21059.
      b1436c74
    • Dongjoon Hyun's avatar
      [SPARK-20345][SQL] Fix STS error handling logic on HiveSQLException · 32818d9b
      Dongjoon Hyun authored
      ## What changes were proposed in this pull request?
      
      [SPARK-5100](https://github.com/apache/spark/commit/343d3bfafd449a0371feb6a88f78e07302fa7143) added Spark Thrift Server(STS) UI and the following logic to handle exceptions on case `Throwable`.
      
      ```scala
      HiveThriftServer2.listener.onStatementError(
        statementId, e.getMessage, SparkUtils.exceptionString(e))
      ```
      
      However, there occurred a missed case after implementing [SPARK-6964](https://github.com/apache/spark/commit/eb19d3f75cbd002f7e72ce02017a8de67f562792)'s `Support Cancellation in the Thrift Server` by adding case `HiveSQLException` before case `Throwable`.
      
      ```scala
      case e: HiveSQLException =>
        if (getStatus().getState() == OperationState.CANCELED) {
          return
        } else {
          setState(OperationState.ERROR)
          throw e
        }
        // Actually do need to catch Throwable as some failures don't inherit from Exception and
        // HiveServer will silently swallow them.
      case e: Throwable =>
        val currentState = getStatus().getState()
        logError(s"Error executing query, currentState $currentState, ", e)
        setState(OperationState.ERROR)
        HiveThriftServer2.listener.onStatementError(
          statementId, e.getMessage, SparkUtils.exceptionString(e))
        throw new HiveSQLException(e.toString)
      ```
      
      Logically, we had better add `HiveThriftServer2.listener.onStatementError` on case `HiveSQLException`, too.
      
      ## How was this patch tested?
      
      N/A
      
      Author: Dongjoon Hyun <dongjoon@apache.org>
      
      Closes #17643 from dongjoon-hyun/SPARK-20345.
      32818d9b
    • aokolnychyi's avatar
      [SPARK-17914][SQL] Fix parsing of timestamp strings with nanoseconds · ca4e960a
      aokolnychyi authored
      The PR contains a tiny change to fix the way Spark parses string literals into timestamps. Currently, some timestamps that contain nanoseconds are corrupted during the conversion from internal UTF8Strings into the internal representation of timestamps.
      
      Consider the following example:
      ```
      spark.sql("SELECT cast('2015-01-02 00:00:00.000000001' as TIMESTAMP)").show(false)
      +------------------------------------------------+
      |CAST(2015-01-02 00:00:00.000000001 AS TIMESTAMP)|
      +------------------------------------------------+
      |2015-01-02 00:00:00.000001                      |
      +------------------------------------------------+
      ```
      
      The fix was tested with existing tests. Also, there is a new test to cover cases that did not work previously.
      
      Author: aokolnychyi <anton.okolnychyi@sap.com>
      
      Closes #18252 from aokolnychyi/spark-17914.
      ca4e960a
    • Wenchen Fan's avatar
      [SPARK-21046][SQL] simplify the array offset and length in ColumnVector · 22dd65f5
      Wenchen Fan authored
      ## What changes were proposed in this pull request?
      
      Currently when a `ColumnVector` stores array type elements, we will use 2 arrays for lengths and offsets and implement them individually in on-heap and off-heap column vector.
      
      In this PR, we use one array to represent both offsets and lengths, so that we can treat it as `ColumnVector` and all the logic can go to the base class `ColumnVector`
      
      ## How was this patch tested?
      
      existing tests.
      
      Author: Wenchen Fan <wenchen@databricks.com>
      
      Closes #18260 from cloud-fan/put.
      22dd65f5
    • Dongjoon Hyun's avatar
      [SPARK-21041][SQL] SparkSession.range should be consistent with SparkContext.range · a92e095e
      Dongjoon Hyun authored
      ## What changes were proposed in this pull request?
      
      This PR fixes the inconsistency in `SparkSession.range`.
      
      **BEFORE**
      ```scala
      scala> spark.range(java.lang.Long.MAX_VALUE - 3, java.lang.Long.MIN_VALUE + 2, 1).collect
      res2: Array[Long] = Array(9223372036854775804, 9223372036854775805, 9223372036854775806)
      ```
      
      **AFTER**
      ```scala
      scala> spark.range(java.lang.Long.MAX_VALUE - 3, java.lang.Long.MIN_VALUE + 2, 1).collect
      res2: Array[Long] = Array()
      ```
      
      ## How was this patch tested?
      
      Pass the Jenkins with newly added test cases.
      
      Author: Dongjoon Hyun <dongjoon@apache.org>
      
      Closes #18257 from dongjoon-hyun/SPARK-21041.
      a92e095e
    • Ziyue Huang's avatar
      [DOCS] Fix error: ambiguous reference to overloaded definition · e6eb02df
      Ziyue Huang authored
      ## What changes were proposed in this pull request?
      
      `df.groupBy.count()` should be `df.groupBy().count()` , otherwise there is an error :
      
      ambiguous reference to overloaded definition, both method groupBy in class Dataset of type (col1: String, cols: String*) and method groupBy in class Dataset of type (cols: org.apache.spark.sql.Column*)
      
      ## How was this patch tested?
      
      ```scala
      val df = spark.readStream.schema(...).json(...)
      val dfCounts = df.groupBy().count()
      ```
      
      Author: Ziyue Huang <zyhuang94@gmail.com>
      
      Closes #18272 from ZiyueHuang/master.
      e6eb02df
    • liuxian's avatar
      [SPARK-20665][SQL][FOLLOW-UP] Move test case to MathExpressionsSuite · d1409180
      liuxian authored
      ## What changes were proposed in this pull request?
      
       add test case to MathExpressionsSuite as #17906
      
      ## How was this patch tested?
      
      unit test cases
      
      Author: liuxian <liu.xian3@zte.com.cn>
      
      Closes #18082 from 10110346/wip-lx-0524.
      d1409180
  5. Jun 11, 2017
    • Josh Rosen's avatar
      [SPARK-20715] Store MapStatuses only in MapOutputTracker, not ShuffleMapStage · 3476390c
      Josh Rosen authored
      ## What changes were proposed in this pull request?
      
      This PR refactors `ShuffleMapStage` and `MapOutputTracker` in order to simplify the management of `MapStatuses`, reduce driver memory consumption, and remove a potential source of scheduler correctness bugs.
      
      ### Background
      
      In Spark there are currently two places where MapStatuses are tracked:
      
      - The `MapOutputTracker` maintains an `Array[MapStatus]` storing a single location for each map output. This mapping is used by the `DAGScheduler` for determining reduce-task locality preferences (when locality-aware reduce task scheduling is enabled) and is also used to serve map output locations to executors / tasks.
      - Each `ShuffleMapStage` also contains a mapping of `Array[List[MapStatus]]` which holds the complete set of locations where each map output could be available. This mapping is used to determine which map tasks need to be run when constructing `TaskSets` for the stage.
      
      This duplication adds complexity and creates the potential for certain types of correctness bugs.  Bad things can happen if these two copies of the map output locations get out of sync. For instance, if the `MapOutputTracker` is missing locations for a map output but `ShuffleMapStage` believes that locations are available then tasks will fail with `MetadataFetchFailedException` but `ShuffleMapStage` will not be updated to reflect the missing map outputs, leading to situations where the stage will be reattempted (because downstream stages experienced fetch failures) but no task sets will be launched (because `ShuffleMapStage` thinks all maps are available).
      
      I observed this behavior in a real-world deployment. I'm still not quite sure how the state got out of sync in the first place, but we can completely avoid this class of bug if we eliminate the duplicate state.
      
      ### Why we only need to track a single location for each map output
      
      I think that storing an `Array[List[MapStatus]]` in `ShuffleMapStage` is unnecessary.
      
      First, note that this adds memory/object bloat to the driver we need one extra `List` per task. If you have millions of tasks across all stages then this can add up to be a significant amount of resources.
      
      Secondly, I believe that it's extremely uncommon that these lists will ever contain more than one entry. It's not impossible, but is very unlikely given the conditions which must occur for that to happen:
      
      - In normal operation (no task failures) we'll only run each task once and thus will have at most one output.
      - If speculation is enabled then it's possible that we'll have multiple attempts of a task. The TaskSetManager will [kill duplicate attempts of a task](https://github.com/apache/spark/blob/04901dd03a3f8062fd39ea38d585935ff71a9248/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala#L717) after a task finishes successfully, reducing the likelihood that both the original and speculated task will successfully register map outputs.
      - There is a [comment in `TaskSetManager`](https://github.com/apache/spark/blob/04901dd03a3f8062fd39ea38d585935ff71a9248/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala#L113) which suggests that running tasks are not killed if a task set becomes a zombie. However:
        - If the task set becomes a zombie due to the job being cancelled then it doesn't matter whether we record map outputs.
        - If the task set became a zombie because of a stage failure (e.g. the map stage itself had a fetch failure from an upstream match stage) then I believe that the "failedEpoch" will be updated which may cause map outputs from still-running tasks to [be ignored](https://github.com/apache/spark/blob/04901dd03a3f8062fd39ea38d585935ff71a9248/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1213). (I'm not 100% sure on this point, though).
      - Even if you _do_ manage to record multiple map outputs for a stage, only a single map output is reported to / tracked by the MapOutputTracker. The only situation where the additional output locations could actually be read or used would be if a task experienced a `FetchFailure` exception. The most likely cause of a `FetchFailure` exception is an executor lost, which will have most likely caused the loss of several map tasks' output, so saving on potential re-execution of a single map task isn't a huge win if we're going to have to recompute several other lost map outputs from other tasks which ran on that lost executor. Also note that the re-population of MapOutputTracker state from state in the ShuffleMapTask only happens after the reduce stage has failed; the additional location doesn't help to prevent FetchFailures but, instead, can only reduce the amount of work when recomputing missing parent stages.
      
      Given this, this patch chooses to do away with tracking multiple locations for map outputs and instead stores only a single location. This change removes the main distinction between the `ShuffleMapTask` and `MapOutputTracker`'s copies of this state, paving the way for storing it only in the `MapOutputTracker`.
      
      ### Overview of other changes
      
      - Significantly simplified the cache / lock management inside of the `MapOutputTrackerMaster`:
        - The old code had several parallel `HashMap`s which had to be guarded by maps of `Object`s which were used as locks. This code was somewhat complicated to follow.
        - The new code uses a new `ShuffleStatus` class to group together all of the state associated with a particular shuffle, including cached serialized map statuses, significantly simplifying the logic.
      - Moved more code out of the shared `MapOutputTracker` abstract base class and into the `MapOutputTrackerMaster` and `MapOutputTrackerWorker` subclasses. This makes it easier to reason about which functionality needs to be supported only on the driver or executor.
      - Removed a bunch of code from the `DAGScheduler` which was used to synchronize information from the `MapOutputTracker` to `ShuffleMapStage`.
      - Added comments to clarify the role of `MapOutputTrackerMaster`'s `epoch` in invalidating executor-side shuffle map output caches.
      
      I will comment on these changes via inline GitHub review comments.
      
      /cc hvanhovell and rxin (whom I discussed this with offline), tgravescs (who recently worked on caching of serialized MapOutputStatuses), and kayousterhout and markhamstra (for scheduler changes).
      
      ## How was this patch tested?
      
      Existing tests. I purposely avoided making interface / API which would require significant updates or modifications to test code.
      
      Author: Josh Rosen <joshrosen@databricks.com>
      
      Closes #17955 from JoshRosen/map-output-tracker-rewrite.
      3476390c
    • Michal Senkyr's avatar
      [SPARK-18891][SQL] Support for specific Java List subtypes · f48273c1
      Michal Senkyr authored
      ## What changes were proposed in this pull request?
      
      Add support for specific Java `List` subtypes in deserialization as well as a generic implicit encoder.
      
      All `List` subtypes are supported by using either the size-specifying constructor (one `int` parameter) or the default constructor.
      
      Interfaces/abstract classes use the following implementations:
      
      * `java.util.List`, `java.util.AbstractList` or `java.util.AbstractSequentialList` => `java.util.ArrayList`
      
      ## How was this patch tested?
      
      ```bash
      build/mvn -DskipTests clean package && dev/run-tests
      ```
      
      Additionally in Spark shell:
      
      ```
      scala> val jlist = new java.util.LinkedList[Int]; jlist.add(1)
      jlist: java.util.LinkedList[Int] = [1]
      res0: Boolean = true
      
      scala> Seq(jlist).toDS().map(_.element()).collect()
      res1: Array[Int] = Array(1)
      ```
      
      Author: Michal Senkyr <mike.senkyr@gmail.com>
      
      Closes #18009 from michalsenkyr/dataset-java-lists.
      f48273c1
    • Michal Senkyr's avatar
      [SPARK-18891][SQL] Support for Scala Map collection types · 0538f3b0
      Michal Senkyr authored
      ## What changes were proposed in this pull request?
      
      Add support for arbitrary Scala `Map` types in deserialization as well as a generic implicit encoder.
      
      Used the builder approach as in #16541 to construct any provided `Map` type upon deserialization.
      
      Please note that this PR also adds (ignored) tests for issue [SPARK-19104 CompileException with Map and Case Class in Spark 2.1.0](https://issues.apache.org/jira/browse/SPARK-19104) but doesn't solve it.
      
      Added support for Java Maps in codegen code (encoders will be added in a different PR) with the following default implementations for interfaces/abstract classes:
      
      * `java.util.Map`, `java.util.AbstractMap` => `java.util.HashMap`
      * `java.util.SortedMap`, `java.util.NavigableMap` => `java.util.TreeMap`
      * `java.util.concurrent.ConcurrentMap` => `java.util.concurrent.ConcurrentHashMap`
      * `java.util.concurrent.ConcurrentNavigableMap` => `java.util.concurrent.ConcurrentSkipListMap`
      
      Resulting codegen for `Seq(Map(1 -> 2)).toDS().map(identity).queryExecution.debug.codegen`:
      
      ```
      /* 001 */ public Object generate(Object[] references) {
      /* 002 */   return new GeneratedIterator(references);
      /* 003 */ }
      /* 004 */
      /* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
      /* 006 */   private Object[] references;
      /* 007 */   private scala.collection.Iterator[] inputs;
      /* 008 */   private scala.collection.Iterator inputadapter_input;
      /* 009 */   private boolean CollectObjectsToMap_loopIsNull1;
      /* 010 */   private int CollectObjectsToMap_loopValue0;
      /* 011 */   private boolean CollectObjectsToMap_loopIsNull3;
      /* 012 */   private int CollectObjectsToMap_loopValue2;
      /* 013 */   private UnsafeRow deserializetoobject_result;
      /* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder;
      /* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter;
      /* 016 */   private scala.collection.immutable.Map mapelements_argValue;
      /* 017 */   private UnsafeRow mapelements_result;
      /* 018 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder;
      /* 019 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter;
      /* 020 */   private UnsafeRow serializefromobject_result;
      /* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
      /* 022 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
      /* 023 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter;
      /* 024 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter1;
      /* 025 */
      /* 026 */   public GeneratedIterator(Object[] references) {
      /* 027 */     this.references = references;
      /* 028 */   }
      /* 029 */
      /* 030 */   public void init(int index, scala.collection.Iterator[] inputs) {
      /* 031 */     partitionIndex = index;
      /* 032 */     this.inputs = inputs;
      /* 033 */     wholestagecodegen_init_0();
      /* 034 */     wholestagecodegen_init_1();
      /* 035 */
      /* 036 */   }
      /* 037 */
      /* 038 */   private void wholestagecodegen_init_0() {
      /* 039 */     inputadapter_input = inputs[0];
      /* 040 */
      /* 041 */     deserializetoobject_result = new UnsafeRow(1);
      /* 042 */     this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 32);
      /* 043 */     this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1);
      /* 044 */
      /* 045 */     mapelements_result = new UnsafeRow(1);
      /* 046 */     this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 32);
      /* 047 */     this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1);
      /* 048 */     serializefromobject_result = new UnsafeRow(1);
      /* 049 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 32);
      /* 050 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
      /* 051 */     this.serializefromobject_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
      /* 052 */
      /* 053 */   }
      /* 054 */
      /* 055 */   private void wholestagecodegen_init_1() {
      /* 056 */     this.serializefromobject_arrayWriter1 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
      /* 057 */
      /* 058 */   }
      /* 059 */
      /* 060 */   protected void processNext() throws java.io.IOException {
      /* 061 */     while (inputadapter_input.hasNext() && !stopEarly()) {
      /* 062 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
      /* 063 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
      /* 064 */       MapData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getMap(0));
      /* 065 */
      /* 066 */       boolean deserializetoobject_isNull1 = true;
      /* 067 */       ArrayData deserializetoobject_value1 = null;
      /* 068 */       if (!inputadapter_isNull) {
      /* 069 */         deserializetoobject_isNull1 = false;
      /* 070 */         if (!deserializetoobject_isNull1) {
      /* 071 */           Object deserializetoobject_funcResult = null;
      /* 072 */           deserializetoobject_funcResult = inputadapter_value.keyArray();
      /* 073 */           if (deserializetoobject_funcResult == null) {
      /* 074 */             deserializetoobject_isNull1 = true;
      /* 075 */           } else {
      /* 076 */             deserializetoobject_value1 = (ArrayData) deserializetoobject_funcResult;
      /* 077 */           }
      /* 078 */
      /* 079 */         }
      /* 080 */         deserializetoobject_isNull1 = deserializetoobject_value1 == null;
      /* 081 */       }
      /* 082 */
      /* 083 */       boolean deserializetoobject_isNull3 = true;
      /* 084 */       ArrayData deserializetoobject_value3 = null;
      /* 085 */       if (!inputadapter_isNull) {
      /* 086 */         deserializetoobject_isNull3 = false;
      /* 087 */         if (!deserializetoobject_isNull3) {
      /* 088 */           Object deserializetoobject_funcResult1 = null;
      /* 089 */           deserializetoobject_funcResult1 = inputadapter_value.valueArray();
      /* 090 */           if (deserializetoobject_funcResult1 == null) {
      /* 091 */             deserializetoobject_isNull3 = true;
      /* 092 */           } else {
      /* 093 */             deserializetoobject_value3 = (ArrayData) deserializetoobject_funcResult1;
      /* 094 */           }
      /* 095 */
      /* 096 */         }
      /* 097 */         deserializetoobject_isNull3 = deserializetoobject_value3 == null;
      /* 098 */       }
      /* 099 */       scala.collection.immutable.Map deserializetoobject_value = null;
      /* 100 */
      /* 101 */       if ((deserializetoobject_isNull1 && !deserializetoobject_isNull3) ||
      /* 102 */         (!deserializetoobject_isNull1 && deserializetoobject_isNull3)) {
      /* 103 */         throw new RuntimeException("Invalid state: Inconsistent nullability of key-value");
      /* 104 */       }
      /* 105 */
      /* 106 */       if (!deserializetoobject_isNull1) {
      /* 107 */         if (deserializetoobject_value1.numElements() != deserializetoobject_value3.numElements()) {
      /* 108 */           throw new RuntimeException("Invalid state: Inconsistent lengths of key-value arrays");
      /* 109 */         }
      /* 110 */         int deserializetoobject_dataLength = deserializetoobject_value1.numElements();
      /* 111 */
      /* 112 */         scala.collection.mutable.Builder CollectObjectsToMap_builderValue5 = scala.collection.immutable.Map$.MODULE$.newBuilder();
      /* 113 */         CollectObjectsToMap_builderValue5.sizeHint(deserializetoobject_dataLength);
      /* 114 */
      /* 115 */         int deserializetoobject_loopIndex = 0;
      /* 116 */         while (deserializetoobject_loopIndex < deserializetoobject_dataLength) {
      /* 117 */           CollectObjectsToMap_loopValue0 = (int) (deserializetoobject_value1.getInt(deserializetoobject_loopIndex));
      /* 118 */           CollectObjectsToMap_loopValue2 = (int) (deserializetoobject_value3.getInt(deserializetoobject_loopIndex));
      /* 119 */           CollectObjectsToMap_loopIsNull1 = deserializetoobject_value1.isNullAt(deserializetoobject_loopIndex);
      /* 120 */           CollectObjectsToMap_loopIsNull3 = deserializetoobject_value3.isNullAt(deserializetoobject_loopIndex);
      /* 121 */
      /* 122 */           if (CollectObjectsToMap_loopIsNull1) {
      /* 123 */             throw new RuntimeException("Found null in map key!");
      /* 124 */           }
      /* 125 */
      /* 126 */           scala.Tuple2 CollectObjectsToMap_loopValue4;
      /* 127 */
      /* 128 */           if (CollectObjectsToMap_loopIsNull3) {
      /* 129 */             CollectObjectsToMap_loopValue4 = new scala.Tuple2(CollectObjectsToMap_loopValue0, null);
      /* 130 */           } else {
      /* 131 */             CollectObjectsToMap_loopValue4 = new scala.Tuple2(CollectObjectsToMap_loopValue0, CollectObjectsToMap_loopValue2);
      /* 132 */           }
      /* 133 */
      /* 134 */           CollectObjectsToMap_builderValue5.$plus$eq(CollectObjectsToMap_loopValue4);
      /* 135 */
      /* 136 */           deserializetoobject_loopIndex += 1;
      /* 137 */         }
      /* 138 */
      /* 139 */         deserializetoobject_value = (scala.collection.immutable.Map) CollectObjectsToMap_builderValue5.result();
      /* 140 */       }
      /* 141 */
      /* 142 */       boolean mapelements_isNull = true;
      /* 143 */       scala.collection.immutable.Map mapelements_value = null;
      /* 144 */       if (!false) {
      /* 145 */         mapelements_argValue = deserializetoobject_value;
      /* 146 */
      /* 147 */         mapelements_isNull = false;
      /* 148 */         if (!mapelements_isNull) {
      /* 149 */           Object mapelements_funcResult = null;
      /* 150 */           mapelements_funcResult = ((scala.Function1) references[0]).apply(mapelements_argValue);
      /* 151 */           if (mapelements_funcResult == null) {
      /* 152 */             mapelements_isNull = true;
      /* 153 */           } else {
      /* 154 */             mapelements_value = (scala.collection.immutable.Map) mapelements_funcResult;
      /* 155 */           }
      /* 156 */
      /* 157 */         }
      /* 158 */         mapelements_isNull = mapelements_value == null;
      /* 159 */       }
      /* 160 */
      /* 161 */       MapData serializefromobject_value = null;
      /* 162 */       if (!mapelements_isNull) {
      /* 163 */         final int serializefromobject_length = mapelements_value.size();
      /* 164 */         final Object[] serializefromobject_convertedKeys = new Object[serializefromobject_length];
      /* 165 */         final Object[] serializefromobject_convertedValues = new Object[serializefromobject_length];
      /* 166 */         int serializefromobject_index = 0;
      /* 167 */         final scala.collection.Iterator serializefromobject_entries = mapelements_value.iterator();
      /* 168 */         while(serializefromobject_entries.hasNext()) {
      /* 169 */           final scala.Tuple2 serializefromobject_entry = (scala.Tuple2) serializefromobject_entries.next();
      /* 170 */           int ExternalMapToCatalyst_key1 = (Integer) serializefromobject_entry._1();
      /* 171 */           int ExternalMapToCatalyst_value1 = (Integer) serializefromobject_entry._2();
      /* 172 */
      /* 173 */           boolean ExternalMapToCatalyst_value_isNull1 = false;
      /* 174 */
      /* 175 */           if (false) {
      /* 176 */             throw new RuntimeException("Cannot use null as map key!");
      /* 177 */           } else {
      /* 178 */             serializefromobject_convertedKeys[serializefromobject_index] = (Integer) ExternalMapToCatalyst_key1;
      /* 179 */           }
      /* 180 */
      /* 181 */           if (false) {
      /* 182 */             serializefromobject_convertedValues[serializefromobject_index] = null;
      /* 183 */           } else {
      /* 184 */             serializefromobject_convertedValues[serializefromobject_index] = (Integer) ExternalMapToCatalyst_value1;
      /* 185 */           }
      /* 186 */
      /* 187 */           serializefromobject_index++;
      /* 188 */         }
      /* 189 */
      /* 190 */         serializefromobject_value = new org.apache.spark.sql.catalyst.util.ArrayBasedMapData(new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_convertedKeys), new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_convertedValues));
      /* 191 */       }
      /* 192 */       serializefromobject_holder.reset();
      /* 193 */
      /* 194 */       serializefromobject_rowWriter.zeroOutNullBytes();
      /* 195 */
      /* 196 */       if (mapelements_isNull) {
      /* 197 */         serializefromobject_rowWriter.setNullAt(0);
      /* 198 */       } else {
      /* 199 */         // Remember the current cursor so that we can calculate how many bytes are
      /* 200 */         // written later.
      /* 201 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
      /* 202 */
      /* 203 */         if (serializefromobject_value instanceof UnsafeMapData) {
      /* 204 */           final int serializefromobject_sizeInBytes = ((UnsafeMapData) serializefromobject_value).getSizeInBytes();
      /* 205 */           // grow the global buffer before writing data.
      /* 206 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
      /* 207 */           ((UnsafeMapData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
      /* 208 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
      /* 209 */
      /* 210 */         } else {
      /* 211 */           final ArrayData serializefromobject_keys = serializefromobject_value.keyArray();
      /* 212 */           final ArrayData serializefromobject_values = serializefromobject_value.valueArray();
      /* 213 */
      /* 214 */           // preserve 8 bytes to write the key array numBytes later.
      /* 215 */           serializefromobject_holder.grow(8);
      /* 216 */           serializefromobject_holder.cursor += 8;
      /* 217 */
      /* 218 */           // Remember the current cursor so that we can write numBytes of key array later.
      /* 219 */           final int serializefromobject_tmpCursor1 = serializefromobject_holder.cursor;
      /* 220 */
      /* 221 */           if (serializefromobject_keys instanceof UnsafeArrayData) {
      /* 222 */             final int serializefromobject_sizeInBytes1 = ((UnsafeArrayData) serializefromobject_keys).getSizeInBytes();
      /* 223 */             // grow the global buffer before writing data.
      /* 224 */             serializefromobject_holder.grow(serializefromobject_sizeInBytes1);
      /* 225 */             ((UnsafeArrayData) serializefromobject_keys).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
      /* 226 */             serializefromobject_holder.cursor += serializefromobject_sizeInBytes1;
      /* 227 */
      /* 228 */           } else {
      /* 229 */             final int serializefromobject_numElements = serializefromobject_keys.numElements();
      /* 230 */             serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
      /* 231 */
      /* 232 */             for (int serializefromobject_index1 = 0; serializefromobject_index1 < serializefromobject_numElements; serializefromobject_index1++) {
      /* 233 */               if (serializefromobject_keys.isNullAt(serializefromobject_index1)) {
      /* 234 */                 serializefromobject_arrayWriter.setNullInt(serializefromobject_index1);
      /* 235 */               } else {
      /* 236 */                 final int serializefromobject_element = serializefromobject_keys.getInt(serializefromobject_index1);
      /* 237 */                 serializefromobject_arrayWriter.write(serializefromobject_index1, serializefromobject_element);
      /* 238 */               }
      /* 239 */             }
      /* 240 */           }
      /* 241 */
      /* 242 */           // Write the numBytes of key array into the first 8 bytes.
      /* 243 */           Platform.putLong(serializefromobject_holder.buffer, serializefromobject_tmpCursor1 - 8, serializefromobject_holder.cursor - serializefromobject_tmpCursor1);
      /* 244 */
      /* 245 */           if (serializefromobject_values instanceof UnsafeArrayData) {
      /* 246 */             final int serializefromobject_sizeInBytes2 = ((UnsafeArrayData) serializefromobject_values).getSizeInBytes();
      /* 247 */             // grow the global buffer before writing data.
      /* 248 */             serializefromobject_holder.grow(serializefromobject_sizeInBytes2);
      /* 249 */             ((UnsafeArrayData) serializefromobject_values).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
      /* 250 */             serializefromobject_holder.cursor += serializefromobject_sizeInBytes2;
      /* 251 */
      /* 252 */           } else {
      /* 253 */             final int serializefromobject_numElements1 = serializefromobject_values.numElements();
      /* 254 */             serializefromobject_arrayWriter1.initialize(serializefromobject_holder, serializefromobject_numElements1, 4);
      /* 255 */
      /* 256 */             for (int serializefromobject_index2 = 0; serializefromobject_index2 < serializefromobject_numElements1; serializefromobject_index2++) {
      /* 257 */               if (serializefromobject_values.isNullAt(serializefromobject_index2)) {
      /* 258 */                 serializefromobject_arrayWriter1.setNullInt(serializefromobject_index2);
      /* 259 */               } else {
      /* 260 */                 final int serializefromobject_element1 = serializefromobject_values.getInt(serializefromobject_index2);
      /* 261 */                 serializefromobject_arrayWriter1.write(serializefromobject_index2, serializefromobject_element1);
      /* 262 */               }
      /* 263 */             }
      /* 264 */           }
      /* 265 */
      /* 266 */         }
      /* 267 */
      /* 268 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
      /* 269 */       }
      /* 270 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
      /* 271 */       append(serializefromobject_result);
      /* 272 */       if (shouldStop()) return;
      /* 273 */     }
      /* 274 */   }
      /* 275 */ }
      ```
      
      Codegen for `java.util.Map`:
      
      ```
      /* 001 */ public Object generate(Object[] references) {
      /* 002 */   return new GeneratedIterator(references);
      /* 003 */ }
      /* 004 */
      /* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
      /* 006 */   private Object[] references;
      /* 007 */   private scala.collection.Iterator[] inputs;
      /* 008 */   private scala.collection.Iterator inputadapter_input;
      /* 009 */   private boolean CollectObjectsToMap_loopIsNull1;
      /* 010 */   private int CollectObjectsToMap_loopValue0;
      /* 011 */   private boolean CollectObjectsToMap_loopIsNull3;
      /* 012 */   private int CollectObjectsToMap_loopValue2;
      /* 013 */   private UnsafeRow deserializetoobject_result;
      /* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder;
      /* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter;
      /* 016 */   private java.util.HashMap mapelements_argValue;
      /* 017 */   private UnsafeRow mapelements_result;
      /* 018 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder;
      /* 019 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter;
      /* 020 */   private UnsafeRow serializefromobject_result;
      /* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
      /* 022 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
      /* 023 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter;
      /* 024 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter1;
      /* 025 */
      /* 026 */   public GeneratedIterator(Object[] references) {
      /* 027 */     this.references = references;
      /* 028 */   }
      /* 029 */
      /* 030 */   public void init(int index, scala.collection.Iterator[] inputs) {
      /* 031 */     partitionIndex = index;
      /* 032 */     this.inputs = inputs;
      /* 033 */     wholestagecodegen_init_0();
      /* 034 */     wholestagecodegen_init_1();
      /* 035 */
      /* 036 */   }
      /* 037 */
      /* 038 */   private void wholestagecodegen_init_0() {
      /* 039 */     inputadapter_input = inputs[0];
      /* 040 */
      /* 041 */     deserializetoobject_result = new UnsafeRow(1);
      /* 042 */     this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 32);
      /* 043 */     this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1);
      /* 044 */
      /* 045 */     mapelements_result = new UnsafeRow(1);
      /* 046 */     this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 32);
      /* 047 */     this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1);
      /* 048 */     serializefromobject_result = new UnsafeRow(1);
      /* 049 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 32);
      /* 050 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
      /* 051 */     this.serializefromobject_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
      /* 052 */
      /* 053 */   }
      /* 054 */
      /* 055 */   private void wholestagecodegen_init_1() {
      /* 056 */     this.serializefromobject_arrayWriter1 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
      /* 057 */
      /* 058 */   }
      /* 059 */
      /* 060 */   protected void processNext() throws java.io.IOException {
      /* 061 */     while (inputadapter_input.hasNext() && !stopEarly()) {
      /* 062 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
      /* 063 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
      /* 064 */       MapData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getMap(0));
      /* 065 */
      /* 066 */       boolean deserializetoobject_isNull1 = true;
      /* 067 */       ArrayData deserializetoobject_value1 = null;
      /* 068 */       if (!inputadapter_isNull) {
      /* 069 */         deserializetoobject_isNull1 = false;
      /* 070 */         if (!deserializetoobject_isNull1) {
      /* 071 */           Object deserializetoobject_funcResult = null;
      /* 072 */           deserializetoobject_funcResult = inputadapter_value.keyArray();
      /* 073 */           if (deserializetoobject_funcResult == null) {
      /* 074 */             deserializetoobject_isNull1 = true;
      /* 075 */           } else {
      /* 076 */             deserializetoobject_value1 = (ArrayData) deserializetoobject_funcResult;
      /* 077 */           }
      /* 078 */
      /* 079 */         }
      /* 080 */         deserializetoobject_isNull1 = deserializetoobject_value1 == null;
      /* 081 */       }
      /* 082 */
      /* 083 */       boolean deserializetoobject_isNull3 = true;
      /* 084 */       ArrayData deserializetoobject_value3 = null;
      /* 085 */       if (!inputadapter_isNull) {
      /* 086 */         deserializetoobject_isNull3 = false;
      /* 087 */         if (!deserializetoobject_isNull3) {
      /* 088 */           Object deserializetoobject_funcResult1 = null;
      /* 089 */           deserializetoobject_funcResult1 = inputadapter_value.valueArray();
      /* 090 */           if (deserializetoobject_funcResult1 == null) {
      /* 091 */             deserializetoobject_isNull3 = true;
      /* 092 */           } else {
      /* 093 */             deserializetoobject_value3 = (ArrayData) deserializetoobject_funcResult1;
      /* 094 */           }
      /* 095 */
      /* 096 */         }
      /* 097 */         deserializetoobject_isNull3 = deserializetoobject_value3 == null;
      /* 098 */       }
      /* 099 */       java.util.HashMap deserializetoobject_value = null;
      /* 100 */
      /* 101 */       if ((deserializetoobject_isNull1 && !deserializetoobject_isNull3) ||
      /* 102 */         (!deserializetoobject_isNull1 && deserializetoobject_isNull3)) {
      /* 103 */         throw new RuntimeException("Invalid state: Inconsistent nullability of key-value");
      /* 104 */       }
      /* 105 */
      /* 106 */       if (!deserializetoobject_isNull1) {
      /* 107 */         if (deserializetoobject_value1.numElements() != deserializetoobject_value3.numElements()) {
      /* 108 */           throw new RuntimeException("Invalid state: Inconsistent lengths of key-value arrays");
      /* 109 */         }
      /* 110 */         int deserializetoobject_dataLength = deserializetoobject_value1.numElements();
      /* 111 */         java.util.Map CollectObjectsToMap_builderValue5 = new java.util.HashMap(deserializetoobject_dataLength);
      /* 112 */
      /* 113 */         int deserializetoobject_loopIndex = 0;
      /* 114 */         while (deserializetoobject_loopIndex < deserializetoobject_dataLength) {
      /* 115 */           CollectObjectsToMap_loopValue0 = (int) (deserializetoobject_value1.getInt(deserializetoobject_loopIndex));
      /* 116 */           CollectObjectsToMap_loopValue2 = (int) (deserializetoobject_value3.getInt(deserializetoobject_loopIndex));
      /* 117 */           CollectObjectsToMap_loopIsNull1 = deserializetoobject_value1.isNullAt(deserializetoobject_loopIndex);
      /* 118 */           CollectObjectsToMap_loopIsNull3 = deserializetoobject_value3.isNullAt(deserializetoobject_loopIndex);
      /* 119 */
      /* 120 */           if (CollectObjectsToMap_loopIsNull1) {
      /* 121 */             throw new RuntimeException("Found null in map key!");
      /* 122 */           }
      /* 123 */
      /* 124 */           CollectObjectsToMap_builderValue5.put(CollectObjectsToMap_loopValue0, CollectObjectsToMap_loopValue2);
      /* 125 */
      /* 126 */           deserializetoobject_loopIndex += 1;
      /* 127 */         }
      /* 128 */
      /* 129 */         deserializetoobject_value = (java.util.HashMap) CollectObjectsToMap_builderValue5;
      /* 130 */       }
      /* 131 */
      /* 132 */       boolean mapelements_isNull = true;
      /* 133 */       java.util.HashMap mapelements_value = null;
      /* 134 */       if (!false) {
      /* 135 */         mapelements_argValue = deserializetoobject_value;
      /* 136 */
      /* 137 */         mapelements_isNull = false;
      /* 138 */         if (!mapelements_isNull) {
      /* 139 */           Object mapelements_funcResult = null;
      /* 140 */           mapelements_funcResult = ((scala.Function1) references[0]).apply(mapelements_argValue);
      /* 141 */           if (mapelements_funcResult == null) {
      /* 142 */             mapelements_isNull = true;
      /* 143 */           } else {
      /* 144 */             mapelements_value = (java.util.HashMap) mapelements_funcResult;
      /* 145 */           }
      /* 146 */
      /* 147 */         }
      /* 148 */         mapelements_isNull = mapelements_value == null;
      /* 149 */       }
      /* 150 */
      /* 151 */       MapData serializefromobject_value = null;
      /* 152 */       if (!mapelements_isNull) {
      /* 153 */         final int serializefromobject_length = mapelements_value.size();
      /* 154 */         final Object[] serializefromobject_convertedKeys = new Object[serializefromobject_length];
      /* 155 */         final Object[] serializefromobject_convertedValues = new Object[serializefromobject_length];
      /* 156 */         int serializefromobject_index = 0;
      /* 157 */         final java.util.Iterator serializefromobject_entries = mapelements_value.entrySet().iterator();
      /* 158 */         while(serializefromobject_entries.hasNext()) {
      /* 159 */           final java.util.Map$Entry serializefromobject_entry = (java.util.Map$Entry) serializefromobject_entries.next();
      /* 160 */           int ExternalMapToCatalyst_key1 = (Integer) serializefromobject_entry.getKey();
      /* 161 */           int ExternalMapToCatalyst_value1 = (Integer) serializefromobject_entry.getValue();
      /* 162 */
      /* 163 */           boolean ExternalMapToCatalyst_value_isNull1 = false;
      /* 164 */
      /* 165 */           if (false) {
      /* 166 */             throw new RuntimeException("Cannot use null as map key!");
      /* 167 */           } else {
      /* 168 */             serializefromobject_convertedKeys[serializefromobject_index] = (Integer) ExternalMapToCatalyst_key1;
      /* 169 */           }
      /* 170 */
      /* 171 */           if (false) {
      /* 172 */             serializefromobject_convertedValues[serializefromobject_index] = null;
      /* 173 */           } else {
      /* 174 */             serializefromobject_convertedValues[serializefromobject_index] = (Integer) ExternalMapToCatalyst_value1;
      /* 175 */           }
      /* 176 */
      /* 177 */           serializefromobject_index++;
      /* 178 */         }
      /* 179 */
      /* 180 */         serializefromobject_value = new org.apache.spark.sql.catalyst.util.ArrayBasedMapData(new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_convertedKeys), new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_convertedValues));
      /* 181 */       }
      /* 182 */       serializefromobject_holder.reset();
      /* 183 */
      /* 184 */       serializefromobject_rowWriter.zeroOutNullBytes();
      /* 185 */
      /* 186 */       if (mapelements_isNull) {
      /* 187 */         serializefromobject_rowWriter.setNullAt(0);
      /* 188 */       } else {
      /* 189 */         // Remember the current cursor so that we can calculate how many bytes are
      /* 190 */         // written later.
      /* 191 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
      /* 192 */
      /* 193 */         if (serializefromobject_value instanceof UnsafeMapData) {
      /* 194 */           final int serializefromobject_sizeInBytes = ((UnsafeMapData) serializefromobject_value).getSizeInBytes();
      /* 195 */           // grow the global buffer before writing data.
      /* 196 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
      /* 197 */           ((UnsafeMapData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
      /* 198 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
      /* 199 */
      /* 200 */         } else {
      /* 201 */           final ArrayData serializefromobject_keys = serializefromobject_value.keyArray();
      /* 202 */           final ArrayData serializefromobject_values = serializefromobject_value.valueArray();
      /* 203 */
      /* 204 */           // preserve 8 bytes to write the key array numBytes later.
      /* 205 */           serializefromobject_holder.grow(8);
      /* 206 */           serializefromobject_holder.cursor += 8;
      /* 207 */
      /* 208 */           // Remember the current cursor so that we can write numBytes of key array later.
      /* 209 */           final int serializefromobject_tmpCursor1 = serializefromobject_holder.cursor;
      /* 210 */
      /* 211 */           if (serializefromobject_keys instanceof UnsafeArrayData) {
      /* 212 */             final int serializefromobject_sizeInBytes1 = ((UnsafeArrayData) serializefromobject_keys).getSizeInBytes();
      /* 213 */             // grow the global buffer before writing data.
      /* 214 */             serializefromobject_holder.grow(serializefromobject_sizeInBytes1);
      /* 215 */             ((UnsafeArrayData) serializefromobject_keys).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
      /* 216 */             serializefromobject_holder.cursor += serializefromobject_sizeInBytes1;
      /* 217 */
      /* 218 */           } else {
      /* 219 */             final int serializefromobject_numElements = serializefromobject_keys.numElements();
      /* 220 */             serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
      /* 221 */
      /* 222 */             for (int serializefromobject_index1 = 0; serializefromobject_index1 < serializefromobject_numElements; serializefromobject_index1++) {
      /* 223 */               if (serializefromobject_keys.isNullAt(serializefromobject_index1)) {
      /* 224 */                 serializefromobject_arrayWriter.setNullInt(serializefromobject_index1);
      /* 225 */               } else {
      /* 226 */                 final int serializefromobject_element = serializefromobject_keys.getInt(serializefromobject_index1);
      /* 227 */                 serializefromobject_arrayWriter.write(serializefromobject_index1, serializefromobject_element);
      /* 228 */               }
      /* 229 */             }
      /* 230 */           }
      /* 231 */
      /* 232 */           // Write the numBytes of key array into the first 8 bytes.
      /* 233 */           Platform.putLong(serializefromobject_holder.buffer, serializefromobject_tmpCursor1 - 8, serializefromobject_holder.cursor - serializefromobject_tmpCursor1);
      /* 234 */
      /* 235 */           if (serializefromobject_values instanceof UnsafeArrayData) {
      /* 236 */             final int serializefromobject_sizeInBytes2 = ((UnsafeArrayData) serializefromobject_values).getSizeInBytes();
      /* 237 */             // grow the global buffer before writing data.
      /* 238 */             serializefromobject_holder.grow(serializefromobject_sizeInBytes2);
      /* 239 */             ((UnsafeArrayData) serializefromobject_values).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
      /* 240 */             serializefromobject_holder.cursor += serializefromobject_sizeInBytes2;
      /* 241 */
      /* 242 */           } else {
      /* 243 */             final int serializefromobject_numElements1 = serializefromobject_values.numElements();
      /* 244 */             serializefromobject_arrayWriter1.initialize(serializefromobject_holder, serializefromobject_numElements1, 4);
      /* 245 */
      /* 246 */             for (int serializefromobject_index2 = 0; serializefromobject_index2 < serializefromobject_numElements1; serializefromobject_index2++) {
      /* 247 */               if (serializefromobject_values.isNullAt(serializefromobject_index2)) {
      /* 248 */                 serializefromobject_arrayWriter1.setNullInt(serializefromobject_index2);
      /* 249 */               } else {
      /* 250 */                 final int serializefromobject_element1 = serializefromobject_values.getInt(serializefromobject_index2);
      /* 251 */                 serializefromobject_arrayWriter1.write(serializefromobject_index2, serializefromobject_element1);
      /* 252 */               }
      /* 253 */             }
      /* 254 */           }
      /* 255 */
      /* 256 */         }
      /* 257 */
      /* 258 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
      /* 259 */       }
      /* 260 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
      /* 261 */       append(serializefromobject_result);
      /* 262 */       if (shouldStop()) return;
      /* 263 */     }
      /* 264 */   }
      /* 265 */ }
      ```
      
      ## How was this patch tested?
      
      ```
      build/mvn -DskipTests clean package && dev/run-tests
      ```
      
      Additionally in Spark shell:
      
      ```
      scala> Seq(collection.mutable.HashMap(1 -> 2, 2 -> 3)).toDS().map(_ += (3 -> 4)).collect()
      res0: Array[scala.collection.mutable.HashMap[Int,Int]] = Array(Map(2 -> 3, 1 -> 2, 3 -> 4))
      ```
      
      Author: Michal Senkyr <mike.senkyr@gmail.com>
      Author: Michal Šenkýř <mike.senkyr@gmail.com>
      
      Closes #16986 from michalsenkyr/dataset-map-builder.
      0538f3b0
    • Zhenhua Wang's avatar
      [SPARK-21031][SQL] Add `alterTableStats` to store spark's stats and let... · a7c61c10
      Zhenhua Wang authored
      [SPARK-21031][SQL] Add `alterTableStats` to store spark's stats and let `alterTable` keep existing stats
      
      ## What changes were proposed in this pull request?
      
      Currently, hive's stats are read into `CatalogStatistics`, while spark's stats are also persisted through `CatalogStatistics`. As a result, hive's stats can be unexpectedly propagated into spark' stats.
      
      For example, for a catalog table, we read stats from hive, e.g. "totalSize" and put it into `CatalogStatistics`. Then, by using "ALTER TABLE" command, we will store the stats in `CatalogStatistics` into metastore as spark's stats (because we don't know whether it's from spark or not). But spark's stats should be only generated by "ANALYZE" command. This is unexpected from this command.
      
      Secondly, now that we have spark's stats in metastore, after inserting new data, although hive updated "totalSize" in metastore, we still cannot get the right `sizeInBytes` in `CatalogStatistics`, because we respect spark's stats (should not exist) over hive's stats.
      
      A running example is shown in [JIRA](https://issues.apache.org/jira/browse/SPARK-21031).
      
      To fix this, we add a new method `alterTableStats` to store spark's stats, and let `alterTable` keep existing stats.
      
      ## How was this patch tested?
      
      Added new tests.
      
      Author: Zhenhua Wang <wzh_zju@163.com>
      
      Closes #18248 from wzhfy/separateHiveStats.
      a7c61c10
    • sujithjay's avatar
      Fixed typo in sql.functions · 3a840048
      sujithjay authored
      ## What changes were proposed in this pull request?
      
      I fixed a typo in the Scaladoc for the method `def struct(cols: Column*): Column`. 'retained' was misspelt as 'remained'.
      
      ## How was this patch tested?
      Before:
      
      Creates a new struct column.
         If the input column is a column in a `DataFrame`, or a derived column expression
         that is named (i.e. aliased), its name would be **remained** as the StructField's name,
         otherwise, the newly generated StructField's name would be auto generated as
         `col` with a suffix `index + 1`, i.e. col1, col2, col3, ...
      
      After:
      
         Creates a new struct column.
         If the input column is a column in a `DataFrame`, or a derived column expression
         that is named (i.e. aliased), its name would be **retained** as the StructField's name,
         otherwise, the newly generated StructField's name would be auto generated as
         `col` with a suffix `index + 1`, i.e. col1, col2, col3, ...
      
      Author: sujithjay <sujith@logistimo.com>
      
      Closes #18254 from sujithjay/fix-typo.
      3a840048
Loading