From user-zh-return-3113-archive-asf-public=cust-asf.ponee.io@flink.apache.org Wed Apr 22 11:00:07 2020 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [207.244.88.153]) by mx-eu-01.ponee.io (Postfix) with SMTP id 98296180181 for ; Wed, 22 Apr 2020 13:00:07 +0200 (CEST) Received: (qmail 51733 invoked by uid 500); 22 Apr 2020 11:00:06 -0000 Mailing-List: contact user-zh-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user-zh@flink.apache.org Delivered-To: mailing list user-zh@flink.apache.org Received: (qmail 51705 invoked by uid 99); 22 Apr 2020 11:00:05 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 22 Apr 2020 11:00:05 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id 263A11812A5 for ; Wed, 22 Apr 2020 11:00:05 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 0.201 X-Spam-Level: X-Spam-Status: No, score=0.201 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, HTML_MESSAGE=0.2, RCVD_IN_DNSWL_NONE=-0.0001, SPF_HELO_NONE=0.001, SPF_PASS=-0.001, URIBL_BLOCKED=0.001] autolearn=disabled Authentication-Results: spamd3-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=akulaku-com.20150623.gappssmtp.com Received: from mx1-he-de.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id EDqHWwz5EAtN for ; Wed, 22 Apr 2020 11:00:02 +0000 (UTC) Received-SPF: Pass (mailfrom) identity=mailfrom; client-ip=2607:f8b0:4864:20::144; helo=mail-il1-x144.google.com; envelope-from=zhangyu@akulaku.com; receiver= Received: from mail-il1-x144.google.com (mail-il1-x144.google.com [IPv6:2607:f8b0:4864:20::144]) by mx1-he-de.apache.org (ASF Mail Server at mx1-he-de.apache.org) with ESMTPS id D16DE7FB3A for ; Wed, 22 Apr 2020 11:00:01 +0000 (UTC) Received: by mail-il1-x144.google.com with SMTP id s10so1400837iln.11 for ; Wed, 22 Apr 2020 04:00:01 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=akulaku-com.20150623.gappssmtp.com; s=20150623; h=mime-version:references:in-reply-to:from:date:message-id:subject:to; bh=PLZzN4AQOfeqnDy53kzaa3QdRC+YhF7C1sBNYbE68KY=; b=fx85ddAt6QXhzfWNJKvuq+1hgA1iYgsNG+K4OLZbU6lDAng50fXm/XMaGIW7jVIv+P 7pWjfH3hRk2Pw00dBESos1QipPXKIvCsRbSav4DJR04yw9+QAt/K3ZoTGuzR37vMf1Ca HG41jj7oAAe9P5dP5qi3ZQSnLqU4wL69sjX/KQxR4fchw2y1lqwZZbydNTi695+OVke0 r9ZtJLxdcmZVVEyJ/UNpmccmeyfqCSEisSuq1VnRe+Itp47Em+2g40n5ioGAVRVeF3YY zTuwCztRjc9rRvVBCvXZ7ppgyd9JbR+8vTt2P3boKBjeMnFDURQioK7/Nw5DAM0zEkZQ 33Hw== X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20161025; h=x-gm-message-state:mime-version:references:in-reply-to:from:date :message-id:subject:to; bh=PLZzN4AQOfeqnDy53kzaa3QdRC+YhF7C1sBNYbE68KY=; b=B7lCHTpQHUYmVNiDfRBhHC0jl9bFM4B+4H8tywJYvgAhcP+5VHhJ1QclXZxZI9tABB 8q5uo9HpciKudISCdO43F06UygkOghbcuAOlauAWphYeBIx3leHD7lvUwRm909+hbg/P jHB/GnkUEkpaMumOjGSSS7pQOyE4GhxmD8xBKr6x1/97La4m+U+sojGS+jCMXcYzPOmf GwVwRmuk7jk8KzNqr3gbLaqQn97EY39hMDD5oqjD0faJ2kMF2xMRYiBeO+77ftjWiHSr UDlwyNMC6fAvHJCNNKBhcDiYN6LeGouexh/+hAHyWF4ZMi5aNnU0Ygy+YHJJstQAhQG3 EZuA== X-Gm-Message-State: AGi0PuYrGW0l9oBa012IXFluyvic+g3uYUIyZso+SKMhKjkTJPrUvgBf VwkxH6Vp+R/i+pk67V0zJ41IeaanBqJIM3qXSkhoWTWyeTs= X-Google-Smtp-Source: APiQypKj7Pp35gFUGGR90qPmVWgbK+y9ZbaF4Jc/9kv7MEaedeitcs70yKCOF4Cw+hRwDNIYee9A8UEvcpWkjp6fsi0= X-Received: by 2002:a92:4086:: with SMTP id d6mr4532448ill.91.1587553200327; Wed, 22 Apr 2020 04:00:00 -0700 (PDT) MIME-Version: 1.0 References: In-Reply-To: From: =?UTF-8?B?5a6H5byg?= Date: Wed, 22 Apr 2020 18:59:49 +0800 Message-ID: Subject: =?UTF-8?B?UmU6IOWFs+S6jkZsaW5rMS4xMCBTdGFuZGFsb25lIOaooeW8j+S7u+WKoeaPkOS6pA==?= To: user-zh@flink.apache.org Content-Type: multipart/alternative; boundary="00000000000091643d05a3df0877" --00000000000091643d05a3df0877 Content-Type: text/plain; charset="UTF-8" Content-Transfer-Encoding: quoted-printable =E7=9C=8B=E4=B8=8B=E4=BD=A0=E6=89=93=E5=8C=85=E7=9A=84 UberJar =E9=87=8C=E6= =9C=89=E6=B2=A1=E4=B8=80=E4=B8=AA=E5=86=85=E5=AE=B9=E5=8C=85=E6=8B=AC 1=E3=80=81=E4=B8=8B=E9=9D=A2=E8=BF=99=E4=B8=AA=E6=96=87=E4=BB=B6=E6=98=AF= =E5=AD=98=E5=9C=A8=E7=9A=84 org.apache.flink.streaming.connectors.kafka.KafkaTableSourceSinkFactory =E7=9A=84=E6=96=87=E4=BB=B6 META-INF/services/org.apache.flink.table.factories.TableFactory 2=E3=80=81flink=E7=89=88=E6=9C=AC1.10=EF=BC=8CStandalone=E6=A8=A1=E5=BC=8F= =E5=90=AF=E5=8A=A8=E6=9C=8D=E5=8A=A1=EF=BC=88start-cluster.sh=EF=BC=89=EF= =BC=8Cflink run=E8=BF=90=E8=A1=8C(/software/flink-1.10.0/bin/flink run -c com.data.main= .StreamMain ./flink_1.10_test-1.0-jar-with-dependencies.jar) 3=E3=80=81=E5=86=8D=E7=A1=AE=E8=AE=A4=E4=B8=8B"TableEnvironmentImpl.sqlQuer= y"=E8=B0=83=E7=94=A8=E6=97=B6=E5=80=99=E7=9A=84ThreadClassLoader=EF=BC=9F =E8=BF=99=E4=B8=AA=E6=8C=87=E7=9A=84=E6=98=AF=E6=89=93=E5=8D=B0=E5=BD=93=E5= =89=8D=E7=BA=BF=E7=A8=8B=E7=9A=84classloader=E5=98=9BThread.currentThread()= .getContextClassLoader() On Wed, Apr 22, 2020 at 6:00 PM Jingsong Li wrote: > Hi, > > =E5=85=88=E7=A1=AE=E8=AE=A4=E4=B8=8B=E4=BD=A0=E7=9A=84Jar=E5=8C=85=E9=87= =8C=E6=9C=89=E6=B2=A1=E6=9C=89 meta-inf-services=E7=9A=84=E6=96=87=E4=BB=B6= =EF=BC=9F=E9=87=8C=E9=9D=A2=E7=A1=AE=E5=AE=9A=E6=9C=89Kafka=EF=BC=9F > > =E5=A6=82=E6=9E=9C=E6=9C=89=EF=BC=8C=E5=86=8D=E7=A1=AE=E8=AE=A4=E4=B8=8B"= TableEnvironmentImpl.sqlQuery"=E8=B0=83=E7=94=A8=E6=97=B6=E5=80=99=E7=9A=84= ThreadClassLoader? > =E5=9B=A0=E4=B8=BA=E7=8E=B0=E5=9C=A8=E9=BB=98=E8=AE=A4=E6=98=AF=E9=80=9A= =E8=BF=87ThreadClassLoader=E6=9D=A5=E8=8E=B7=E5=8F=96Factory=E7=9A=84=E3=80= =82 > > Best, > Jingsong Lee > > On Wed, Apr 22, 2020 at 5:30 PM =E5=AE=87=E5=BC=A0 = wrote: > > > =E6=88=91=E8=BF=99=E9=9D=A2=E4=BD=BF=E7=94=A8Standalone=E6=A8=A1=E5=BC= =8F=E8=BF=90=E8=A1=8CFlink=E4=BB=BB=E5=8A=A1=EF=BC=8C=E4=BD=86=E6=98=AFUber > > Jar=E9=87=8C=E9=9D=A2=E7=9A=84TableSourceFactory=E4=B8=8D=E8=83=BD=E8= =A2=AB=E5=8A=A0=E8=BD=BD=EF=BC=8C=E5=8D=B3=E4=BD=BF=E8=AE=BE=E7=BD=AE=E4=BA= =86classloader.resolve-order: > > child-first=EF=BC=8C=E5=8F=AA=E6=9C=89=E6=94=BE=E5=9C=A8lib=E7=9B=AE=E5= =BD=95=E6=89=8D=E8=83=BD=E5=8A=A0=E8=BD=BD=E5=BE=97=E5=88=B0=EF=BC=8C=E6=88= =91=E7=9C=8B=E5=8F=91=E5=B8=83=E6=96=87=E6=A1=A3=E8=B7=9F=E6=94=B9=E4=BA=86= =E7=B1=BB=E5=8A=A0=E8=BD=BD=E7=AD=96=E7=95=A5=EF=BC=8C=E4=BD=86=E6=98=AF=E6= =88=91=E4=B8=8D=E7=9F=A5=E9=81=93=E4=B8=BA=E4=BB=80=E4=B9=88Uber > Jar=E9=87=8C=E9=9D=A2=E7=9A=84Factory=E4=B8=8D=E8=83=BD=E8=A2=AB=E5=8A=A0= =E8=BD=BD > > Flink Client respects Classloading Policy (FLINK-13749 > > ) > > < > > > https://ci.apache.org/projects/flink/flink-docs-release-1.10/release-note= s/flink-1.10.html#flink-client-respects-classloading-policy-flink-13749 > > > > > > > The Flink client now also respects the configured classloading policy, > > i.e., parent-first or child-first classloading. Previously, only cluste= r > > components such as the job manager or task manager supported this > setting. > > This does mean that users might get different behaviour in their > programs, > > in which case they should configure the classloading policy explicitly = to > > use parent-first classloading, which was the previous (hard-coded) > > behaviour. > > > > =E5=BC=82=E5=B8=B8=E4=BF=A1=E6=81=AF=EF=BC=9A > > > > rg.apache.flink.client.program.ProgramInvocationException: The main > > method caused an error: findAndCreateTableSource failed. > > at > > > > > org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedPr= ogram.java:335) > > at > > > > > org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForE= xecution(PackagedProgram.java:205) > > at > org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:138) > > at > > > > > org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:6= 64) > > at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:213) > > at > > > > > org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:= 895) > > at > > > > > org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:9= 68) > > at > > > > > org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecu= rityContext.java:30) > > at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:968) > > Caused by: org.apache.flink.table.api.TableException: > > findAndCreateTableSource failed. > > at > > > > > org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSourc= e(TableFactoryUtil.java:55) > > at > > > > > org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSourc= e(TableFactoryUtil.java:92) > > at > > > > > org.apache.flink.table.planner.plan.schema.CatalogSourceTable.findAndCrea= teTableSource(CatalogSourceTable.scala:156) > > at > > > > > org.apache.flink.table.planner.plan.schema.CatalogSourceTable.tableSource= $lzycompute(CatalogSourceTable.scala:65) > > at > > > > > org.apache.flink.table.planner.plan.schema.CatalogSourceTable.tableSource= (CatalogSourceTable.scala:65) > > at > > > > > org.apache.flink.table.planner.plan.schema.CatalogSourceTable.toRel(Catal= ogSourceTable.scala:76) > > at > > > > > org.apache.calcite.sql2rel.SqlToRelConverter.toRel(SqlToRelConverter.java= :3328) > > at > > > > > org.apache.calcite.sql2rel.SqlToRelConverter.convertIdentifier(SqlToRelCo= nverter.java:2357) > > at > > > > > org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverte= r.java:2051) > > at > > > > > org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverte= r.java:2005) > > at > > > > > org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverte= r.java:2083) > > at > > > > > org.apache.calcite.sql2rel.SqlToRelConverter.convertSelectImpl(SqlToRelCo= nverter.java:646) > > at > > > > > org.apache.calcite.sql2rel.SqlToRelConverter.convertSelect(SqlToRelConver= ter.java:627) > > at > > > > > org.apache.calcite.sql2rel.SqlToRelConverter.convertQueryRecursive(SqlToR= elConverter.java:3181) > > at > > > > > org.apache.calcite.sql2rel.SqlToRelConverter.convertQuery(SqlToRelConvert= er.java:563) > > at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org > > > > > $apache$flink$table$planner$calcite$FlinkPlannerImpl$$rel(FlinkPlannerImp= l.scala:148) > > at > > > > > org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerI= mpl.scala:135) > > at > > > > > org.apache.flink.table.planner.operations.SqlToOperationConverter.toQuery= Operation(SqlToOperationConverter.java:522) > > at > > > > > org.apache.flink.table.planner.operations.SqlToOperationConverter.convert= SqlQuery(SqlToOperationConverter.java:436) > > at > > > > > org.apache.flink.table.planner.operations.SqlToOperationConverter.convert= (SqlToOperationConverter.java:154) > > at > > > > > org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.jav= a:66) > > at > > > > > org.apache.flink.table.api.internal.TableEnvironmentImpl.sqlQuery(TableEn= vironmentImpl.java:464) > > at com.akulaku.data.main.StreamMain.main(StreamMain.java:87) > > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > > at > > > > > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java= :62) > > at > > > > > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorI= mpl.java:43) > > at java.lang.reflect.Method.invoke(Method.java:498) > > at > > > > > org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedPr= ogram.java:321) > > ... 8 more > > Caused by: org.apache.flink.table.api.NoMatchingTableFactoryException: > > Could not find a suitable table factory for > > 'org.apache.flink.table.factories.TableSourceFactory' in > > the classpath. > > > > Reason: Required context properties mismatch. > > > > The matching candidates: > > org.apache.flink.table.sources.CsvAppendTableSourceFactory > > Mismatched properties: > > 'connector.type' expects 'filesystem', but is 'kafka' > > 'format.type' expects 'csv', but is 'json' > > > > The following properties are requested: > > connector.properties.bootstrap.servers=3Dcentos:9092 > > connector.properties.zookeeper.connect=3Dcentos:2181 > > connector.startup-mode=3Dearliest-offset > > connector.topic=3Dtest > > connector.type=3Dkafka > > connector.version=3D0.11 > > format.type=3Djson > > schema.0.data-type=3DVARCHAR(2147483647) > > schema.0.name=3Dbus > > schema.1.data-type=3DBIGINT > > schema.1.name=3Dts > > schema.2.data-type=3DVARCHAR(2147483647) > > schema.2.name=3Dtype > > schema.3.data-type=3DBIGINT > > schema.3.name=3DputRowNum > > schema.4.data-type=3DTIMESTAMP(3) NOT NULL > > schema.4.expr=3DPROCTIME() > > schema.4.name=3Dproctime > > update-mode=3Dappend > > > > The following factories have been considered: > > org.apache.flink.table.sources.CsvBatchTableSourceFactory > > org.apache.flink.table.sources.CsvAppendTableSourceFactory > > at > > > > > org.apache.flink.table.factories.TableFactoryService.filterByContext(Tabl= eFactoryService.java:322) > > at > > > > > org.apache.flink.table.factories.TableFactoryService.filter(TableFactoryS= ervice.java:190) > > at > > > > > org.apache.flink.table.factories.TableFactoryService.findSingleInternal(T= ableFactoryService.java:143) > > at > > > > > org.apache.flink.table.factories.TableFactoryService.find(TableFactorySer= vice.java:96) > > at > > > > > org.apache.flink.table.factories.TableFactoryUtil.findAndCreateTableSourc= e(TableFactoryUtil.java:52) > > > > > -- > Best, Jingsong Lee > --00000000000091643d05a3df0877--