You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/04/06 23:04:17 UTC

[GitHub] [iceberg] RussellSpitzer opened a new pull request #2428: Fix Antlr Shadowing and add Integration

RussellSpitzer opened a new pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428


   Fix our issue with Antlr Shadowing and add an integration test which checks to make sure that the Spark3 Runtime Jar works. 
   
   Addresses #2382


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2428: Fix Antlr Shadowing and add Integration Tests

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428#discussion_r612573102



##########
File path: build.gradle
##########
@@ -1021,6 +1031,18 @@ project(':iceberg-spark3-runtime') {
     compile(project(':iceberg-nessie')) {
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
+
+    integrationImplementation 'org.apache.spark:spark-hive_2.12'

Review comment:
       Yeah rather than excluding everything from tests (and the projects they depend on) I thought it would be much easier to instead only list the exact things we need explicitly. I think this also makes it clear that this is not a set of unit tests.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2428: Fix Antlr Shadowing and add Integration Tests

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428#discussion_r612621586



##########
File path: build.gradle
##########
@@ -1021,6 +1031,18 @@ project(':iceberg-spark3-runtime') {
     compile(project(':iceberg-nessie')) {
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
+
+    integrationImplementation 'org.apache.spark:spark-hive_2.12'

Review comment:
       Makes sense, thanks for explaining!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2428: Fix Antlr Shadowing and add Integration Tests

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428#discussion_r612587720



##########
File path: build.gradle
##########
@@ -1021,6 +1029,18 @@ project(':iceberg-spark3-runtime') {
     compile(project(':iceberg-nessie')) {
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
+
+    integrationImplementation 'org.apache.spark:spark-hive_2.12'
+    integrationImplementation 'junit:junit'
+    integrationImplementation 'org.slf4j:slf4j-simple'
+    integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts')

Review comment:
       Compiled test classes are ok on the runtime classpath.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2428: Fix Antlr Shadowing and add Integration Tests

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428#discussion_r612587094



##########
File path: build.gradle
##########
@@ -1021,6 +1031,18 @@ project(':iceberg-spark3-runtime') {
     compile(project(':iceberg-nessie')) {
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
+
+    integrationImplementation 'org.apache.spark:spark-hive_2.12'

Review comment:
       (This also made it easier (at least for me) to setup the integration test task and have it depend on the output of the shadow Jar task. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2428: Fix Antlr Shadowing and add Integration Tests

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428#discussion_r612574288



##########
File path: build.gradle
##########
@@ -1038,9 +1060,7 @@ project(':iceberg-spark3-runtime') {
     relocate 'com.google', 'org.apache.iceberg.shaded.com.google'
     relocate 'com.fasterxml', 'org.apache.iceberg.shaded.com.fasterxml'
     relocate 'com.github.benmanes', 'org.apache.iceberg.shaded.com.github.benmanes'
-    relocate 'org.antlr.v4.runtime', 'org.apache.iceberg.shaded.org.antlr.v4.runtime'
     relocate 'org.checkerframework', 'org.apache.iceberg.shaded.org.checkerframework'
-    relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro'

Review comment:
       This original comment was for a missing relocate "avro" lib. The Antlr library I moved to the bottom since I feel like it's another separate category




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2428: Fix Antlr Shadowing and add Integration Tests

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428#discussion_r612102052



##########
File path: build.gradle
##########
@@ -972,7 +972,6 @@ project(":iceberg-spark3-extensions") {
 
   dependencies {
     compileOnly project(':iceberg-spark3')
-

Review comment:
       nit: unneeded change

##########
File path: build.gradle
##########
@@ -984,7 +983,9 @@ project(":iceberg-spark3-extensions") {
     testCompile project(path: ':iceberg-spark', configuration: 'testArtifacts')
     testCompile project(path: ':iceberg-spark3', configuration: 'testArtifacts')
 
+

Review comment:
       nit: extra empty line

##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala
##########
@@ -198,3 +228,71 @@ case object IcebergSqlExtensionsPostProcessor extends IcebergSqlExtensionsBaseLi
     parent.addChild(new TerminalNodeImpl(f(newToken)))
   }
 }
+
+
+/* Partially copied from Apache Spark's Parser to avoid dependency on Spark Internals */
+case object ParseErrorListener extends BaseErrorListener {

Review comment:
       nit: should we call it `IcebergParseErrorListener` to be consistent with other copied classes?

##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala
##########
@@ -198,3 +228,71 @@ case object IcebergSqlExtensionsPostProcessor extends IcebergSqlExtensionsBaseLi
     parent.addChild(new TerminalNodeImpl(f(newToken)))
   }
 }
+

Review comment:
       nit: extra line

##########
File path: build.gradle
##########
@@ -1021,6 +1031,18 @@ project(':iceberg-spark3-runtime') {
     compile(project(':iceberg-nessie')) {
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
+
+    integrationImplementation 'org.apache.spark:spark-hive_2.12'

Review comment:
       Do we need a separate source set as test inherits other things?

##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala
##########
@@ -198,3 +228,71 @@ case object IcebergSqlExtensionsPostProcessor extends IcebergSqlExtensionsBaseLi
     parent.addChild(new TerminalNodeImpl(f(newToken)))
   }
 }
+
+
+/* Partially copied from Apache Spark's Parser to avoid dependency on Spark Internals */
+case object ParseErrorListener extends BaseErrorListener {
+  override def syntaxError(
+                            recognizer: Recognizer[_, _],

Review comment:
       nit: formatting, let's use what we do in Scala that matches Spark's behavior

##########
File path: build.gradle
##########
@@ -1038,9 +1060,7 @@ project(':iceberg-spark3-runtime') {
     relocate 'com.google', 'org.apache.iceberg.shaded.com.google'
     relocate 'com.fasterxml', 'org.apache.iceberg.shaded.com.fasterxml'
     relocate 'com.github.benmanes', 'org.apache.iceberg.shaded.com.github.benmanes'
-    relocate 'org.antlr.v4.runtime', 'org.apache.iceberg.shaded.org.antlr.v4.runtime'
     relocate 'org.checkerframework', 'org.apache.iceberg.shaded.org.checkerframework'
-    relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro'

Review comment:
       Looks like this hasn't been reverted yet.

##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala
##########
@@ -150,22 +148,54 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI
       }
     }
     catch {
-      case e: ParseException if e.command.isDefined =>
+      case e: IcebergParseException if e.command.isDefined =>
         throw e
-      case e: ParseException =>
+      case e: IcebergParseException =>
         throw e.withCommand(command)
       case e: AnalysisException =>
         val position = Origin(e.line, e.startPosition)
-        throw new ParseException(Option(command), e.message, position, position)
+        throw new IcebergParseException(Option(command), e.message, position, position)
     }
   }
+

Review comment:
       nit: extra empty lines

##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala
##########
@@ -198,3 +228,71 @@ case object IcebergSqlExtensionsPostProcessor extends IcebergSqlExtensionsBaseLi
     parent.addChild(new TerminalNodeImpl(f(newToken)))
   }
 }
+
+
+/* Partially copied from Apache Spark's Parser to avoid dependency on Spark Internals */
+case object ParseErrorListener extends BaseErrorListener {
+  override def syntaxError(
+                            recognizer: Recognizer[_, _],
+                            offendingSymbol: scala.Any,
+                            line: Int,
+                            charPositionInLine: Int,
+                            msg: String,
+                            e: RecognitionException): Unit = {
+    val (start, stop) = offendingSymbol match {
+      case token: CommonToken =>
+        val start = Origin(Some(line), Some(token.getCharPositionInLine))
+        val length = token.getStopIndex - token.getStartIndex + 1
+        val stop = Origin(Some(line), Some(token.getCharPositionInLine + length))
+        (start, stop)
+      case _ =>
+        val start = Origin(Some(line), Some(charPositionInLine))
+        (start, start)
+    }
+    throw new IcebergParseException(None, msg, start, stop)
+  }
+}
+
+/**
+ * Copied from Apache Spark
+ * A [[ParseException]] is an [[AnalysisException]] that is thrown during the parse process. It
+ * contains fields and an extended error message that make reporting and diagnosing errors easier.
+ */
+class IcebergParseException(
+                      val command: Option[String],
+                      message: String,

Review comment:
       nit: formatting

##########
File path: build.gradle
##########
@@ -1054,10 +1075,22 @@ project(':iceberg-spark3-runtime') {
     relocate 'org.apache.arrow', 'org.apache.iceberg.shaded.org.apache.arrow'
     relocate 'com.carrotsearch', 'org.apache.iceberg.shaded.com.carrotsearch'
     relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra'
+    // relocate Antlr runtime and related deps to shade Iceberg specific version
+    relocate 'org.antlr.v4.runtime', 'org.apache.iceberg.shaded.org.antlr.v4.runtime'
 
     classifier null
   }
 
+  task integrationTest(type: Test) {
+    description = "Test Spark3 Runtime Jar"
+    group = "verification"
+    testClassesDirs = sourceSets.integration.output.classesDirs
+    classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path)
+    inputs.file(shadowJar.archiveFile.get().asFile.path)
+  }
+  integrationTest.dependsOn(shadowJar)

Review comment:
       nit: I think we mostly use `integrationTest.dependsOn` shadowJar like on the line below




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pan3793 commented on a change in pull request #2428: Fix Antlr Shadowing and add Integration Tests

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428#discussion_r612125544



##########
File path: build.gradle
##########
@@ -1000,6 +1001,15 @@ project(':iceberg-spark3-runtime') {
 
   tasks.jar.dependsOn tasks.shadowJar
 
+  sourceSets {
+    integration {
+      java {
+        srcDir "$projectDir/src/integration/java"
+      }

Review comment:
       how about `java.srcDir "$projectDir/src/integration/java"` align style with below line?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on pull request #2428: Fix Antlr Shadowing and add Integration

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428#issuecomment-814489279


   @aokolnychyi and I were discussing this and we are going to take another run at shading the Antlr dependency now that we have a fail safe integration test.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pan3793 commented on a change in pull request #2428: Fix Antlr Shadowing and add Integration Tests

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428#discussion_r610985471



##########
File path: build.gradle
##########
@@ -1038,9 +1060,7 @@ project(':iceberg-spark3-runtime') {
     relocate 'com.google', 'org.apache.iceberg.shaded.com.google'
     relocate 'com.fasterxml', 'org.apache.iceberg.shaded.com.fasterxml'
     relocate 'com.github.benmanes', 'org.apache.iceberg.shaded.com.github.benmanes'
-    relocate 'org.antlr.v4.runtime', 'org.apache.iceberg.shaded.org.antlr.v4.runtime'
     relocate 'org.checkerframework', 'org.apache.iceberg.shaded.org.checkerframework'
-    relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro'

Review comment:
       why remove this?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #2428: Fix Antlr Shadowing and add Integration Tests

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428#issuecomment-818892805


   LGTM, let's wait for tests.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2428: Fix Antlr Shadowing and add Integration Tests

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428#discussion_r611805661



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala
##########
@@ -159,13 +157,45 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI
         throw new ParseException(Option(command), e.message, position, position)
     }
   }
+
+
+}
+
+/* Copied from Apache Spark's to avoid dependency on Spark Internals */
+class UpperCaseCharStream(wrapped: CodePointCharStream) extends CharStream {
+  override def consume(): Unit = wrapped.consume
+  override def getSourceName(): String = wrapped.getSourceName
+  override def index(): Int = wrapped.index
+  override def mark(): Int = wrapped.mark
+  override def release(marker: Int): Unit = wrapped.release(marker)
+  override def seek(where: Int): Unit = wrapped.seek(where)
+  override def size(): Int = wrapped.size
+
+  override def getText(interval: Interval): String = {
+    // ANTLR 4.7's CodePointCharStream implementations have bugs when
+    // getText() is called with an empty stream, or intervals where
+    // the start > end. See
+    // https://github.com/antlr/antlr4/commit/ac9f7530 for one fix
+    // that is not yet in a released ANTLR artifact.
+    if (size() > 0 && (interval.b - interval.a >= 0)) {
+      wrapped.getText(interval)
+    } else {
+      ""
+    }
+  }

Review comment:
       I think for now we should just leave it as is although I think it could just be wrapped.getText() ... without any ill effects




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pan3793 commented on a change in pull request #2428: Fix Antlr Shadowing and add Integration Tests

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428#discussion_r610985101



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala
##########
@@ -159,13 +157,45 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI
         throw new ParseException(Option(command), e.message, position, position)
     }
   }
+
+
+}
+
+/* Copied from Apache Spark's to avoid dependency on Spark Internals */
+class UpperCaseCharStream(wrapped: CodePointCharStream) extends CharStream {
+  override def consume(): Unit = wrapped.consume
+  override def getSourceName(): String = wrapped.getSourceName
+  override def index(): Int = wrapped.index
+  override def mark(): Int = wrapped.mark
+  override def release(marker: Int): Unit = wrapped.release(marker)
+  override def seek(where: Int): Unit = wrapped.seek(where)
+  override def size(): Int = wrapped.size
+
+  override def getText(interval: Interval): String = {
+    // ANTLR 4.7's CodePointCharStream implementations have bugs when
+    // getText() is called with an empty stream, or intervals where
+    // the start > end. See
+    // https://github.com/antlr/antlr4/commit/ac9f7530 for one fix
+    // that is not yet in a released ANTLR artifact.
+    if (size() > 0 && (interval.b - interval.a >= 0)) {
+      wrapped.getText(interval)
+    } else {
+      ""
+    }
+  }

Review comment:
       Since already decoupled with Spark antlr4 version, so we can upgrade antlr4 and remove this workaround?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2428: Fix Antlr Shadowing and add Integration Tests

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428#discussion_r611804981



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala
##########
@@ -159,13 +157,45 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI
         throw new ParseException(Option(command), e.message, position, position)
     }
   }
+
+
+}
+
+/* Copied from Apache Spark's to avoid dependency on Spark Internals */
+class UpperCaseCharStream(wrapped: CodePointCharStream) extends CharStream {
+  override def consume(): Unit = wrapped.consume
+  override def getSourceName(): String = wrapped.getSourceName
+  override def index(): Int = wrapped.index
+  override def mark(): Int = wrapped.mark
+  override def release(marker: Int): Unit = wrapped.release(marker)
+  override def seek(where: Int): Unit = wrapped.seek(where)
+  override def size(): Int = wrapped.size
+
+  override def getText(interval: Interval): String = {
+    // ANTLR 4.7's CodePointCharStream implementations have bugs when
+    // getText() is called with an empty stream, or intervals where
+    // the start > end. See
+    // https://github.com/antlr/antlr4/commit/ac9f7530 for one fix
+    // that is not yet in a released ANTLR artifact.
+    if (size() > 0 && (interval.b - interval.a >= 0)) {
+      wrapped.getText(interval)
+    } else {
+      ""
+    }
+  }

Review comment:
       This is actually fixed in the version Spark is using 4.7.1 I guess no one just ever removed it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer merged pull request #2428: Fix Antlr Shadowing and add Integration Tests

Posted by GitBox <gi...@apache.org>.
RussellSpitzer merged pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2428: Fix Antlr Shadowing and add Integration Tests

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428#discussion_r610985563



##########
File path: build.gradle
##########
@@ -1038,9 +1060,7 @@ project(':iceberg-spark3-runtime') {
     relocate 'com.google', 'org.apache.iceberg.shaded.com.google'
     relocate 'com.fasterxml', 'org.apache.iceberg.shaded.com.fasterxml'
     relocate 'com.github.benmanes', 'org.apache.iceberg.shaded.com.github.benmanes'
-    relocate 'org.antlr.v4.runtime', 'org.apache.iceberg.shaded.org.antlr.v4.runtime'
     relocate 'org.checkerframework', 'org.apache.iceberg.shaded.org.checkerframework'
-    relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro'

Review comment:
       oops that's a typo, I had alphabetized this list and when I undid it I must have missed this




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2428: Fix Antlr Shadowing and add Integration Tests

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428#discussion_r612588395



##########
File path: build.gradle
##########
@@ -1021,6 +1029,18 @@ project(':iceberg-spark3-runtime') {
     compile(project(':iceberg-nessie')) {
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
+
+    integrationImplementation 'org.apache.spark:spark-hive_2.12'
+    integrationImplementation 'junit:junit'
+    integrationImplementation 'org.slf4j:slf4j-simple'
+    integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts')
+    integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts')
+    integrationImplementation project(path: ':iceberg-spark', configuration: 'testArtifacts')
+    integrationImplementation project(path: ':iceberg-spark3', configuration: 'testArtifacts')
+    integrationImplementation project(path: ':iceberg-spark3-extensions', configuration: 'testArtifacts')
+    // Not allowed on our classpath, only the runtime jar is allowed
+    integrationCompileOnly project(':iceberg-spark3-extensions')

Review comment:
       Only spark-hive, the shadow jar output, and test classes are allowed on our runtime path




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on pull request #2428: Fix Antlr Shadowing and add Integration Tests

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #2428:
URL: https://github.com/apache/iceberg/pull/2428#issuecomment-817026027


   @aokolnychyi  + @pan3793  Rebased and reshaded, if you could please take a look?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org