Skip to content

Commit fd58cd0

Browse files
authored
change to 9 parameters constructor (#823)
* change to 9 parameters constructor * add 7|8|9 parameters separately to be compatible with different version * update * change to 9 parameters constructor * change to match parameter type length
1 parent a6f63b1 commit fd58cd0

File tree

1 file changed

+21
-8
lines changed

1 file changed

+21
-8
lines changed

core/src/main/scala/za/co/absa/spline/harvester/plugin/embedded/BigQueryPlugin.scala

+21-8
Original file line numberDiff line numberDiff line change
@@ -66,13 +66,15 @@ class BigQueryPlugin(spark: SparkSession)
6666

6767
def bigQueryConfig = {
6868
SparkBigQueryConfig.from(
69-
ImmutableMap.copyOf(cmd.options.asJava),
69+
cmd.options.asJava,
7070
ImmutableMap.copyOf(spark.conf.getAll.asJava),
7171
spark.sparkContext.hadoopConfiguration,
72+
ImmutableMap.of,
7273
0,
7374
spark.sessionState.conf,
7475
spark.version,
75-
Optional.empty()
76+
Optional.empty(),
77+
true
7678
)
7779
}
7880

@@ -145,9 +147,10 @@ object BigQueryPlugin {
145147
.find(
146148
m => m.getName == "from"
147149
&& isStatic(m.getModifiers)
148-
&& m.getParameterTypes.length == 7
150+
&& (m.getParameterTypes.length == 7 || m.getParameterTypes.length == 8 || m.getParameterTypes.length == 9)
149151
&& m.getReturnType.getSimpleName == "SparkBigQueryConfig"
150-
).getOrElse(sys.error(s"Cannot find method `public static SparkBigQueryConfig from(... {7 args} ...)` in the class `$clazz`"))
152+
)
153+
.getOrElse(sys.error(s"Cannot find method `public static SparkBigQueryConfig from(... {7|8|9 args} ...)` in the class `$clazz`"))
151154

152155
object ImmutableMap {
153156
type ImmutableMap = AnyRef
@@ -157,12 +160,22 @@ object BigQueryPlugin {
157160
.getMethod("copyOf", classOf[java.util.Map[_, _]])
158161
.invoke(imClass, _)
159162
.asInstanceOf[ImmutableMap]
163+
val of: ImmutableMap =
164+
imClass
165+
.getMethod("of")
166+
.invoke(imClass)
167+
.asInstanceOf[ImmutableMap]
160168
}
161169

162-
val from: (ImmutableMap.ImmutableMap, ImmutableMap.ImmutableMap, Configuration, Integer, SQLConf, String, Optional[StructType]) => SparkBigQueryConfig =
163-
methodFrom
164-
.invoke(clazz, _, _, _, _, _, _, _)
165-
.asInstanceOf[SparkBigQueryConfig]
170+
val from: (java.util.Map[_, _], ImmutableMap.ImmutableMap, Configuration, ImmutableMap.ImmutableMap, Integer, SQLConf, String, Optional[StructType], java.lang.Boolean) => SparkBigQueryConfig = {
171+
case (a, b, c, d, e, f, g, h, i) => {
172+
methodFrom.getParameterTypes.length match {
173+
case 7 => methodFrom.invoke(clazz, a, b, c, e, f, g, h).asInstanceOf[SparkBigQueryConfig]
174+
case 8 => methodFrom.invoke(clazz, a, b, c, e, f, g, h, i).asInstanceOf[SparkBigQueryConfig]
175+
case 9 => methodFrom.invoke(clazz, a, b, c, d, e, f, g, h, i).asInstanceOf[SparkBigQueryConfig]
176+
}
177+
}
178+
}
166179
}
167180

168181
private object `_: DirectBigQueryRelation` extends SafeTypeMatchingExtractor[AnyRef]("com.google.cloud.spark.bigquery.direct.DirectBigQueryRelation")

0 commit comments

Comments
 (0)