[
https://issues.apache.org/jira/browse/FLINK-6173?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16205906#comment-16205906
]
ASF GitHub Bot commented on FLINK-6173:
---------------------------------------
Github user zentol commented on a diff in the pull request:
https://github.com/apache/flink/pull/4837#discussion_r144850032
--- Diff: flink-libraries/flink-table/pom.xml ---
@@ -213,36 +209,49 @@ under the License.
<filter>
<artifact>*:*</artifact>
<excludes>
+
<!-- excluded all these files for a clean flink-table jar -->
<exclude>org-apache-calcite-jdbc.properties</exclude>
+
<exclude>common.proto</exclude>
+
<exclude>requests.proto</exclude>
+
<exclude>responses.proto</exclude>
<exclude>mozilla/**</exclude>
<exclude>codegen/**</exclude>
<exclude>google/**</exclude>
<exclude>META-INF/*.SF</exclude>
<exclude>META-INF/*.DSA</exclude>
<exclude>META-INF/*.RSA</exclude>
+
<exclude>META-INF/services/**</exclude>
+
<exclude>properties.dtd</exclude>
+
<exclude>PropertyList-1.0.dtd</exclude>
+
<exclude>digesterRules.xml</exclude>
+
<!-- not excluded for now -->
+
<!-- <exclude>org.codehaus.commons.compiler.properties</exclude> -->
</excludes>
</filter>
</filters>
<artifactSet>
<includes
combine.children="append">
+ <!--
Calcite and its dependencies -->
<include>org.apache.calcite:*</include>
<include>org.apache.calcite.avatica:*</include>
+
<include>com.google.guava:guava</include>
+
<include>org.eigenbase:*</include>
<include>net.hydromatic:*</include>
-
<include>org.reflections:*</include>
+
+ <!--
flink-table dependencies -->
+
<include>commons-configuration:*</include>
+
<include>commons-lang:*</include>
+
<include>commons-codec:*</include>
<include>org.codehaus.janino:*</include>
-
<include>com.google.guava:guava</include>
- </includes>
+
<include>org.reflections:*</include>
+
<include>joda-time:*</include>
+ </includes>
</artifactSet>
<relocations>
- <!-- We
currently don't relocate slf4j as we have "logger not found"
-
warnings otherwise during runtime -->
- <!--<relocation>
-
<pattern>org.slf4j</pattern>
-
<shadedPattern>org.apache.flink.shaded.calcite.org.slf4j</shadedPattern>
- </relocation>-->
+ <!-- Calcite
and its dependencies -->
<relocation>
-
<pattern>com.fasterxml.jackson</pattern>
-
<shadedPattern>org.apache.flink.shaded.calcite.com.fasterxml.jackson</shadedPattern>
+
<pattern>org.apache.calcite</pattern>
+
<shadedPattern>org.apache.flink.shaded.calcite.org.apache.calcite</shadedPattern>
--- End diff --
For calcite dependencies, please change the shading pattern to
org.apache.flink.calcite.shaded to a) make them more unique and b) distinguish
them from flink-shaded dependencies.
> flink-table not pack-in com.fasterxml.jackson.* in after #FLINK-5414
> --------------------------------------------------------------------
>
> Key: FLINK-6173
> URL: https://issues.apache.org/jira/browse/FLINK-6173
> Project: Flink
> Issue Type: Bug
> Components: Table API & SQL
> Reporter: Zhenghua Gao
> Assignee: Timo Walther
> Priority: Blocker
> Fix For: 1.4.0
>
>
> Currently, flink-table will pack-in com.fasterxml.jackson.* and rename them
> to org.apache.flink.shaded.calcite.com.fasterxml.jackson.*
> If a project depends on flink-table, and uses fasterxml as follows(function
> explain uses fasterxml indirectly):
> {code:title=WordCount.scala|borderStyle=solid}
> object WordCountWithTable {
> def main(args: Array[String]): Unit = {
> // set up execution environment
> val env = ExecutionEnvironment.getExecutionEnvironment
> val tEnv = TableEnvironment.getTableEnvironment(env)
> val input = env.fromElements(WC("hello", 1), WC("hello", 1), WC("ciao",
> 1))
> val expr = input.toTable(tEnv)
> val result = expr
> .groupBy('word)
> .select('word, 'frequency.sum as 'frequency)
> .filter('frequency === 2)
> println(tEnv.explain(result))
> result.toDataSet[WC].print()
> }
> case class WC(word: String, frequency: Long)
> }
> {code}
> It actually uses org.apache.flink.shaded.calcite.com.fasterxml.jackson.*
> I found after FLINK-5414, flink-table didn't pack-in com.fasterxml.jackson.*
> and the project would throw class not found exception.
> {code:borderStyle=solid}
> Exception in thread "main" java.lang.NoClassDefFoundError:
> org/apache/flink/shaded/calcite/com/fasterxml/jackson/databind/ObjectMapper
> at
> org.apache.flink.table.explain.PlanJsonParser.getSqlExecutionPlan(PlanJsonParser.java:32)
> at
> org.apache.flink.table.api.BatchTableEnvironment.explain(BatchTableEnvironment.scala:143)
> at
> org.apache.flink.table.api.BatchTableEnvironment.explain(BatchTableEnvironment.scala:164)
> at
> org.apache.flink.quickstart.WordCountWithTable$.main(WordCountWithTable.scala:34)
> at
> org.apache.flink.quickstart.WordCountWithTable.main(WordCountWithTable.scala)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at com.intellij.rt.execution.application.AppMain.main(AppMain.java:144)
> Caused by: java.lang.ClassNotFoundException:
> org.apache.flink.shaded.calcite.com.fasterxml.jackson.databind.ObjectMapper
> at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
> at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
> at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:331)
> at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
> ... 10 more
> {code}
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)