You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "vinoyang (JIRA)" <ji...@apache.org> on 2018/11/01 09:22:00 UTC

[jira] [Commented] (FLINK-10704) Fix sql client end to end test failure

    [ https://issues.apache.org/jira/browse/FLINK-10704?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16671329#comment-16671329 ] 

vinoyang commented on FLINK-10704:
----------------------------------

[~pnowojski] [~twalthr] A strange exception : 
{code:java}
Exception in thread "main" org.apache.flink.table.client.SqlClientException: The configured environment is invalid. Please check your environment files again.
 at org.apache.flink.table.client.SqlClient.validateEnvironment(SqlClient.java:140)
 at org.apache.flink.table.client.SqlClient.start(SqlClient.java:99)
 at org.apache.flink.table.client.SqlClient.main(SqlClient.java:187)
Caused by: org.apache.flink.table.client.gateway.SqlExecutionException: Could not create execution context.
 at org.apache.flink.table.client.gateway.local.LocalExecutor.getOrCreateExecutionContext(LocalExecutor.java:485)
 at org.apache.flink.table.client.gateway.local.LocalExecutor.validateSession(LocalExecutor.java:313)
 at org.apache.flink.table.client.SqlClient.validateEnvironment(SqlClient.java:137)
 ... 2 more
Caused by: java.lang.NoClassDefFoundError: org/apache/flink/table/factories/TableFormatFactoryBase
 at java.lang.ClassLoader.defineClass1(Native Method)
 at java.lang.ClassLoader.defineClass(ClassLoader.java:763)
 at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142)
 at java.net.URLClassLoader.defineClass(URLClassLoader.java:467)
 at java.net.URLClassLoader.access$100(URLClassLoader.java:73)
 at java.net.URLClassLoader$1.run(URLClassLoader.java:368)
 at java.net.URLClassLoader$1.run(URLClassLoader.java:362)
 at java.security.AccessController.doPrivileged(Native Method)
 at java.net.URLClassLoader.findClass(URLClassLoader.java:361)
 at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
 at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
 at java.lang.Class.forName0(Native Method)
 at java.lang.Class.forName(Class.java:348)
 at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:370)
 at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404)
 at java.util.ServiceLoader$1.next(ServiceLoader.java:480)
 at scala.collection.convert.Wrappers$JIteratorWrapper.next(Wrappers.scala:43)
 at scala.collection.Iterator$class.toStream(Iterator.scala:1320)
 at scala.collection.AbstractIterator.toStream(Iterator.scala:1334)
 at scala.collection.Iterator$$anonfun$toStream$1.apply(Iterator.scala:1320)
 at scala.collection.Iterator$$anonfun$toStream$1.apply(Iterator.scala:1320)
 at scala.collection.immutable.Stream$Cons.tail(Stream.scala:1233)
 at scala.collection.immutable.Stream$Cons.tail(Stream.scala:1223)
 at scala.collection.immutable.Stream.filter(Stream.scala:519)
 at scala.collection.immutable.Stream$$anonfun$filteredTail$1.apply(Stream.scala:1299)
 at scala.collection.immutable.Stream$$anonfun$filteredTail$1.apply(Stream.scala:1299)
 at scala.collection.immutable.Stream$Cons.tail(Stream.scala:1233)
 at scala.collection.immutable.Stream$Cons.tail(Stream.scala:1223)
 at scala.collection.immutable.Stream.filter(Stream.scala:519)
 at scala.collection.immutable.Stream.filter(Stream.scala:202)
 at org.apache.flink.table.factories.TableFactoryService$.filterByContext(TableFactoryService.scala:197)
 at org.apache.flink.table.factories.TableFactoryService$.findInternal(TableFactoryService.scala:130)
 at org.apache.flink.table.factories.TableFactoryService$.find(TableFactoryService.scala:100)
 at org.apache.flink.table.factories.TableFactoryService.find(TableFactoryService.scala)
 at org.apache.flink.table.client.gateway.local.ExecutionContext.createTableSource(ExecutionContext.java:236)
 at org.apache.flink.table.client.gateway.local.ExecutionContext.lambda$new$0(ExecutionContext.java:121)
 at java.util.LinkedHashMap.forEach(LinkedHashMap.java:684)
 at org.apache.flink.table.client.gateway.local.ExecutionContext.<init>(ExecutionContext.java:119)
 at org.apache.flink.table.client.gateway.local.LocalExecutor.getOrCreateExecutionContext(LocalExecutor.java:481)
 ... 4 more
Caused by: java.lang.ClassNotFoundException: org.apache.flink.table.factories.TableFormatFactoryBase
 at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
 at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
 at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
 ... 43 more
Waiting for CSV results...
{code}
I did not find the usage of TableFormatFactoryBase anywhere. What's the reason for this exception? I have updated specific client and table jar and sql-client-defaults.yaml in *build-target* dir. But it still reports this exception.

> Fix sql client end to end test failure
> --------------------------------------
>
>                 Key: FLINK-10704
>                 URL: https://issues.apache.org/jira/browse/FLINK-10704
>             Project: Flink
>          Issue Type: Bug
>          Components: E2E Tests, Kafka Connector
>            Reporter: vinoyang
>            Assignee: vinoyang
>            Priority: Major
>              Labels: pull-request-available
>
> The log file contains the following sentence:
> {code:java}
> 2018-10-29 03:27:39,209 WARN org.apache.flink.kafka010.shaded.org.apache.kafka.common.utils.AppInfoParser - Error while loading kafka-version.properties :null
> {code}
> The reason for this log is that we explicitly exclude the version description file of the kafka client when packaging the connector:
> {code:java}
> <filters>
>    <filter>
>       <artifact>*:*</artifact>
>       <excludes>
>          <exclude>kafka/kafka-version.properties</exclude>
>       </excludes>
>    </filter>
> </filters>{code}
> When the shell scan the "error" keyword with grep, it will hit, so the test will fail.
> {code:java}
> function check_logs_for_errors {
>   error_count=$(grep -rv "GroupCoordinatorNotAvailableException" $FLINK_DIR/log \
>       | grep -v "RetriableCommitFailedException" \
>       | grep -v "NoAvailableBrokersException" \
>       | grep -v "Async Kafka commit failed" \
>       | grep -v "DisconnectException" \
>       | grep -v "AskTimeoutException" \
>       | grep -v "WARN  akka.remote.transport.netty.NettyTransport" \
>       | grep -v  "WARN  org.apache.flink.shaded.akka.org.jboss.netty.channel.DefaultChannelPipeline" \
>       | grep -v "jvm-exit-on-fatal-error" \
>       | grep -v '^INFO:.*AWSErrorCode=\[400 Bad Request\].*ServiceEndpoint=\[https://.*\.s3\.amazonaws\.com\].*RequestType=\[HeadBucketRequest\]' \
>       | grep -v "RejectedExecutionException" \
>       | grep -v "An exception was thrown by an exception handler" \
>       | grep -v "java.lang.NoClassDefFoundError: org/apache/hadoop/yarn/exceptions/YarnException" \
>       | grep -v "java.lang.NoClassDefFoundError: org/apache/hadoop/conf/Configuration" \
>       | grep -v "org.apache.flink.fs.shaded.hadoop3.org.apache.commons.beanutils.FluentPropertyBeanIntrospector  - Error when creating PropertyDescriptor for public final void org.apache.flink.fs.shaded.hadoop3.org.apache.commons.configuration2.AbstractConfiguration.setProperty(java.lang.String,java.lang.Object)! Ignoring this property." \
>       | grep -ic "error")    //here
>   if [[ ${error_count} -gt 0 ]]; then
>     echo "Found error in log files:"
>     cat $FLINK_DIR/log/*
>     EXIT_CODE=1
>   fi
> }
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)