Skip to content

Commit 83db398

Browse files
vladimirg-dbHyukjinKwon
authored andcommitted
[SPARK-51881][SQL] Make AvroOptions comparable
### What changes were proposed in this pull request? Make AvroOptions comparable. Hadoop `Configuration` doesn't have equals, but it's enough to compare `parameters`. ### Why are the changes needed? To properly compare single-pass/fixed-point Analyzer logical plans. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #50683 from vladimirg-db/vladimir-golubev_data/make-avro-options-comparable. Authored-by: Vladimir Golubev <vladimir.golubev@databricks.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
1 parent bab6988 commit 83db398

File tree

1 file changed

+32
-0
lines changed

1 file changed

+32
-0
lines changed

sql/core/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala

Lines changed: 32 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
package org.apache.spark.sql.avro
1919

2020
import java.net.URI
21+
import java.util.HashMap
2122

2223
import org.apache.avro.Schema
2324
import org.apache.hadoop.conf.Configuration
@@ -146,6 +147,37 @@ private[sql] class AvroOptions(
146147
RECURSIVE_FIELD_MAX_DEPTH,
147148
s"Should not be greater than $RECURSIVE_FIELD_MAX_DEPTH_LIMIT.")
148149
}
150+
151+
/**
152+
* [[hadoop.conf.Configuration]] is not comparable so we turn it into a map for [[equals]] and
153+
* [[hashCode]].
154+
*/
155+
@transient private lazy val comparableConf = {
156+
val iter = conf.iterator()
157+
val result = new HashMap[String, String]
158+
while (iter.hasNext()) {
159+
val entry = iter.next()
160+
result.put(entry.getKey(), entry.getValue())
161+
}
162+
result
163+
}
164+
165+
override def equals(other: Any): Boolean = {
166+
other match {
167+
case that: AvroOptions =>
168+
this.parameters == that.parameters &&
169+
this.comparableConf == that.comparableConf
170+
case _ => false
171+
}
172+
}
173+
174+
override def hashCode(): Int = {
175+
val prime = 31
176+
var result = 1
177+
result = prime * result + parameters.hashCode
178+
result = prime * result + comparableConf.hashCode
179+
result
180+
}
149181
}
150182

151183
private[sql] object AvroOptions extends DataSourceOptions {

0 commit comments

Comments
 (0)