Skip to content

Conversation

@leanken-zz
Copy link
Contributor

@leanken-zz leanken-zz commented Oct 17, 2018

What is the purpose of the change

See. FLINK-10166.

Version 1. Fix sqlClient org.apache.commons.codec.binary.Base64 classNotFound issue on hadoopless binary.

Version 2. After discussed with dawidwys, we all agree that replace of the org.apache.commons.codec.binary.Base64 with other implementation to get rid of the dependency on commons-codec is a better approach.

Diagnose

The Exception stacktrace is quite straight-forward.

## Exception Message
execute SQL statement. Reason:
org.codehaus.commons.compiler.CompileException: Line 43, Column 10: Unknown variable or type "org.apache.commons.codec.binary.Base64"

As I mentioned above, this error only happens at hadoopless binary because the TaskManagerRunner start without flink-shaded-hadoop2-uber-1.6.0.jar, and as for the flink-table component has a shaded relocation on org.apache.commons prefix. so in this case, neither the TaskManagerRunner nor the FlinkUserJar supply valid class definition for org.apache.commons.codec.binary.Base64

<!-- commons-codec, commons-lang3, and commons-lang (from Calcite) -->
<relocation>
	<pattern>org.apache.commons</pattern>
	<shadedPattern>org.apache.flink.table.shaded.org.apache.commons</shadedPattern>
</relocation>

Proposal

version 1

I've went through the history code of the flink-table relocation. Supposed the purpose of the origin author is that, to avoid version conflict that Calcite might bring.

  • org.apache.commons.codec
  • org.apache.commons.lang
  • org.apache.commons.lang3

But, i have double check on the sub-dependency of calcite-core, only org.apache.commons.lang is sub-dependency of calcite-core, org.apache.commons.codec and org.apache.commons.lang3 in fact are inherited from flink-parent, so i think these two namespace no need to relocation.

In genernal, my proposal is to not relocation org.apache.commons.codec and org.apache.commons.lang3

version 2

Replace org.apache.commons.codec.binary.Base64 with java.util.Base64

Brief change log

  • not relocation org.apache.commons.codec and org.apache.commons.lang3

Verifying this change

  • sqlClient with hadoopless binary on select count(distinct name) from (Values ('a'), ('b')) AS NameTable(name) [SUCCESS]
  • org.apache.flink.table.examples.scala.WordCountTable [SUCCESS]
  • org.apache.flink.table.examples.scala.WordCountSQL [SUCCESS]
  • org.apache.flink.table.examples.scala.StreamTableExample [SUCCESS]
  • org.apache.flink.table.examples.scala.StreamSQLExample [SUCCESS]
  • add TestCase org.apache.flink.runtime.util.Base64CompatibleTest

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): (no)
  • The public API, i.e., is any changed class annotated with @Public(Evolving): (no)
  • The serializers: (no)
  • The runtime per-record code paths (performance sensitive): (no)
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)
  • The S3 file system connector: (no)

Documentation

  • Does this pull request introduce a new feature? (no)
  • If yes, how is the feature documented? (not documented)

… flink-table

Change-Id: I3cebb1dd9fa708c93186e31a3117f05a8059ba8c
@leanken-zz
Copy link
Contributor Author

cc @StefanRRichter @tillrohrmann . Plz help review on the fix of hadoopless sqlClient classNotFound issue. thanks

@dawidwys
Copy link
Contributor

Hi @leanken . Thank you for your contribution. I am afraid though it is not the right way to proceed with this bug. I think we should rather cut off usage of Base64 (replace with our own implementation) than add hard external dependencies.

Also just to clarify why we relocate dependencies. The reason is not just so they do not clash with other dependencies of ours (e.g. from calcite as you said), but also so that Flink's users can use any arbitrary version of popular libraries.

@leanken-zz leanken-zz changed the title [FLINK-10166] skip relocation of commons-codes and commons-lang3 from flink-table [FLINK-10166] skip relocation of commons-codec and commons-lang3 from flink-table Oct 17, 2018
@dawidwys dawidwys changed the title [FLINK-10166] skip relocation of commons-codec and commons-lang3 from flink-table [FLINK-10166][table] skip relocation of commons-codes and commons-lang3 from flink-table Oct 17, 2018
@leanken-zz leanken-zz changed the title [FLINK-10166][table] skip relocation of commons-codes and commons-lang3 from flink-table [FLINK-10166][table] skip relocation of commons-codec and commons-lang3 from flink-table Oct 17, 2018
@leanken-zz
Copy link
Contributor Author

leanken-zz commented Oct 17, 2018

Hi @leanken . Thank you for your contribution. I am afraid though it is not the right way to proceed with this bug. I think we should rather cut off usage of Base64 (replace with our own implementation) than add hard external dependencies.
---- "reply: for Base64 replacement. I think it's ok for just this single case, what if there are even more cases like Base64, in future, we might need to do the replacement case by case"

Also just to clarify why we relocate dependencies. The reason is not just so they do not clash with other dependencies of ours (e.g. from calcite as you said), but also so that Flink's users can use any arbitrary version of popular libraries.
---- "reply: the reason that you mentioned, user can use any version of org.apache.commons prefix library, I dont think it is now working, because in flink-binary-with-hadoop common-codec library already included in classpath, even if FlinkUserJar contains its own common-codec jar, it might be conflict during runtime"

@leanken-zz
Copy link
Contributor Author

@dawidwys I will first have a look at the candidate replacement for <org.apache.commons.codec.Base64> for further discussion.

@dawidwys
Copy link
Contributor

  1. As far as I know Base64 is the only class from those packages used, and therefore cutting it should be the preferred way in this case, as it simplifies our dependencies.

  2. I agree, but I would say this is a bug of flink-binary-with-hadoop, rather than something that we should follow.

@leanken-zz
Copy link
Contributor Author

  1. As far as I know Base64 is the only class from those packages used, and therefore cutting it should be the preferred way in this case, as it simplifies our dependencies.
  2. I agree, but I would say this is a bug of flink-binary-with-hadoop, rather than something that we should follow.

ok. I see your point. I will update in the next commit for Base64 replacement. thanks.

…64 and add TCase

Change-Id: Id32aeee1dea89ca9309c67056f4864186823fca6
@leanken-zz leanken-zz changed the title [FLINK-10166][table] skip relocation of commons-codec and commons-lang3 from flink-table [FLINK-10166][table] Replace commons.codec.binary.Base64 with java.util.Base64 Oct 17, 2018
@leanken-zz
Copy link
Contributor Author

cc @dawidwys Updated commit. Replaced commons.codec.binary.Base64 with java.util.Base64, and basically I've search all reference of commons.codec.binary.Base64, all of them have been replaced with java.util.Base64.

And the origin encodeBase64URLSafeString should use Base64.getUrlEncode as replace candidate.

@leanken-zz
Copy link
Contributor Author

Just to confirm that the change did works. I catch the GeneratedCode from debugging.

function_org$apache$flink$table$functions$aggfunctions$CountAggFunction$f4346869d1df07833c4f2ba55982f5ce.open(new org.apache.flink.table.functions.FunctionContext(ctx));
       

    
    org.apache.flink.api.common.state.StateDescriptor acc0_distinctValueMap_dataview_desc = (org.apache.flink.api.common.state.StateDescriptor)
      org.apache.flink.util.InstantiationUtil.deserializeObject(
      java.util.Base64.getUrlDecoder().decode("rO0ABXNyADRvcmcuYXBhY2hlLmZsaW5rLmFwaS5jb21tb24uc3RhdGUuTWFwU3RhdGVEZXNjcmlwdG9yAAAAAAAAAAECAAB4cgAxb3JnLmFwYWNoZS5mbGluay5hcGkuY29tbW9uLnN0YXRlLlN0YXRlRGVzY3JpcHRvcgAAAAAAAAABAwAFTAAEbmFtZXQAEkxqYXZhL2xhbmcvU3RyaW5nO0wAEnF1ZXJ5YWJsZVN0YXRlTmFtZXEAfgACTAAKc2VyaWFsaXplcnQANkxvcmcvYXBhY2hlL2ZsaW5rL2FwaS9jb21tb24vdHlwZXV0aWxzL1R5cGVTZXJpYWxpe

the origin org.apache.commons.codec.binary.Base64.decodeBase64 did switch into java.util.Base64.getUrlDecoder().decode

@leanken-zz
Copy link
Contributor Author

let me fix TestCase build break first.

…encodeBase64URLSafeString

Change-Id: Ieb6d33a4c233361560137441c8d03a84207ba6fc
Change-Id: I12dda1a3d196e8e1bb630683572c6338f23ed88d
Change-Id: Idaa30b9b34749220951e15344cc54536fbb0f3f8
@leanken-zz
Copy link
Contributor Author

Kindly remind. @dawidwys
Build turned green, ready to go.

@leanken-zz
Copy link
Contributor Author

Calling @dawidwys
^_^

@twalthr
Copy link
Contributor

twalthr commented Oct 31, 2018

I tried to fix this issue once and for all in #6966. This PR had to be rebased to the new flink-table-common module anyway. I will close this PR. Sorry, for the inconvenience.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants