Skip to content

Commit

Permalink
Fix test cases
Browse files Browse the repository at this point in the history
  • Loading branch information
szehon-ho committed Apr 28, 2024
1 parent 41481b4 commit 9776242
Show file tree
Hide file tree
Showing 2 changed files with 12 additions and 9 deletions.
3 changes: 2 additions & 1 deletion core/src/main/scala/org/apache/spark/Partitioner.scala
Original file line number Diff line number Diff line change
Expand Up @@ -151,7 +151,8 @@ private[spark] class KeyGroupedPartitioner(
override def getPartition(key: Any): Int = {
val keys = key.asInstanceOf[Seq[Any]]
val normalizedKeys = ArraySeq.from(keys)
valueMap.getOrElseUpdate(normalizedKeys, Utils.nonNegativeMod(keys.hashCode, numPartitions))
valueMap.getOrElseUpdate(normalizedKeys,
Utils.nonNegativeMod(normalizedKeys.hashCode, numPartitions))
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,9 @@ package org.apache.spark.sql.connector

import java.sql.Date
import java.util.Collections

import org.apache.spark.sql.{catalyst, AnalysisException, DataFrame, Row}
import org.apache.spark.sql.{AnalysisException, DataFrame, Row, catalyst}
import org.apache.spark.sql.catalyst.expressions.{ApplyFunctionExpression, Cast, Literal}
import org.apache.spark.sql.catalyst.expressions.objects.Invoke
import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke
import org.apache.spark.sql.catalyst.plans.physical
import org.apache.spark.sql.catalyst.plans.physical.{CoalescedBoundary, CoalescedHashPartitioning, HashPartitioning, RangePartitioning, UnknownPartitioning}
import org.apache.spark.sql.connector.catalog.{Column, Identifier}
Expand All @@ -40,7 +39,7 @@ import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream
import org.apache.spark.sql.functions.lit
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.{StreamingQueryException, Trigger}
import org.apache.spark.sql.types.{DateType, IntegerType, LongType, ObjectType, StringType, TimestampType}
import org.apache.spark.sql.types.{DataTypes, DateType, IntegerType, LongType, StringType, TimestampType}
import org.apache.spark.sql.util.QueryExecutionListener
import org.apache.spark.tags.SlowSQLTest

Expand Down Expand Up @@ -1128,13 +1127,16 @@ class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase
Seq.empty
),
catalyst.expressions.SortOrder(
Invoke(
Literal.create(YearsFunction, ObjectType(YearsFunction.getClass)),
StaticInvoke(
YearsFunction.getClass,
DataTypes.LongType,
"invoke",
LongType,
Seq(Cast(attr("day"), TimestampType, Some("America/Los_Angeles"))),
Seq(TimestampType),
propagateNull = false),
propagateNull = false,
returnNullable = true,
isDeterministic = true,
Some(YearsFunction)),
catalyst.expressions.Descending,
catalyst.expressions.NullsFirst,
Seq.empty
Expand Down

0 comments on commit 9776242

Please sign in to comment.