From user-return-14442-archive-asf-public=cust-asf.ponee.io@storm.apache.org Mon Jun 3 20:19:12 2019 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 858B718062F for ; Mon, 3 Jun 2019 22:19:12 +0200 (CEST) Received: (qmail 23253 invoked by uid 500); 3 Jun 2019 20:19:10 -0000 Mailing-List: contact user-help@storm.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@storm.apache.org Delivered-To: mailing list user@storm.apache.org Received: (qmail 23243 invoked by uid 99); 3 Jun 2019 20:19:10 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 03 Jun 2019 20:19:10 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 9B5411A4397 for ; Mon, 3 Jun 2019 20:19:09 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -0.505 X-Spam-Level: X-Spam-Status: No, score=-0.505 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, DKIM_VALID_EF=-0.1, RCVD_IN_DNSWL_NONE=-0.0001, RCVD_IN_MSPIKE_H2=-0.305, SPF_HELO_NONE=0.001, SPF_PASS=-0.001] autolearn=disabled Authentication-Results: spamd2-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=gmail.com Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id eaCfFGxPfl1L for ; Mon, 3 Jun 2019 20:19:08 +0000 (UTC) Received: from mail-lj1-f193.google.com (mail-lj1-f193.google.com [209.85.208.193]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTPS id 94B9D5F24A for ; Mon, 3 Jun 2019 20:19:07 +0000 (UTC) Received: by mail-lj1-f193.google.com with SMTP id j24so17533890ljg.1 for ; Mon, 03 Jun 2019 13:19:07 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20161025; h=mime-version:references:in-reply-to:from:date:message-id:subject:to :content-transfer-encoding; bh=ixQtYg3EdtSVd1ZqC4b1mvkZTrNk7aAUobV8WkqLtJE=; b=tNsZEsOnLnFWm8tov1BcdI1rKKqnpLidb6gN84pGPYiDPXlFGYlpYwPkzcdTGp7kbh fgRkH33i4dpA5Gy9AAUf0GrlH6t1sRwM7vuc1TwJqkAewXUm6OHC4fPgfScQzfbSFT7Y 6dsnvOGVZ1+gfjqZJVaaXjylUC2g54UUe4K1aEOESavM0NRDE7RbcOIsRhNc8tPJOQEo x4j0eHOWF9+ksXqidhv7xTs/InXGv1XT2wBKxBDuxRW0srjAAfUN7HchmVHLG2pSmJ6e LIMndfaGE7JfnUuhVaSF/mY1SGKjFCKUGJ2asNJHFYKZIRTkc7NgR8i1FZyBQUZ+w7dm 0Bqw== 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:content-transfer-encoding; bh=ixQtYg3EdtSVd1ZqC4b1mvkZTrNk7aAUobV8WkqLtJE=; b=rHST7boHfkq7/5yefPlrJeHmRuk8Zh0XsPCBEPMIGi+zxt6yM1fLXpIFaEr9pz63Qt EYoP3c5jSEg2gOmjLHq4V3JAP8jR472Kt8OGoPZV3JnFWq8hn9hnGijD48hIPcrss77r YEV3PlOZkeGXZMqm0mzXbi4barVonDXUyMnAQn1kX4yDjytQeB19cP85sZj44l3llKOA oAiClYcSRo+DeCUkGBQ2feIxNegNlE1e3XD9w79AmjKR1p3P/gm/ZCtdidScgXKAKTyY X2pGhP11LjVKs5pUPJDQJxXeng6eNWr310AuAtEGmbG6YV8aiYkyeLzv64kFMgOljOik JCAA== X-Gm-Message-State: APjAAAWDFzBmjR1BVYFhpPuT1DMV0s04HeAKC0M62C0L2ipu7+WhJY6X UUoLJPZmqqbbyAvZfXJj43R4plPCrgSoqgyrRJxLBCWu55A= X-Google-Smtp-Source: APXvYqzeqcgJ9GvndEHpknBUNXUI7+xhyofgdIs1VTF3Zchi70L1IaygnWVvR5ZrLK1/B6qVL8HNXbd61emnFK45A5A= X-Received: by 2002:a2e:7d02:: with SMTP id y2mr15088242ljc.62.1559593145823; Mon, 03 Jun 2019 13:19:05 -0700 (PDT) MIME-Version: 1.0 References: In-Reply-To: From: Alexandre Vermeerbergen Date: Mon, 3 Jun 2019 22:18:54 +0200 Message-ID: Subject: Re: How to setup Kafka authentication on storm-kafka-monitor ? To: user@storm.apache.org Content-Type: text/plain; charset="UTF-8" Content-Transfer-Encoding: quoted-printable Hello Sandip, Thanks for the suggestion, I have created a storm_jaas.conf file with the following content: KafkaClient { org.apache.kafka.common.security.scram.ScramLoginModule required username=3D"foo" password=3D"bar"; }; and i got a different exception on Storm UI at the place where Kafka lag is supposed to be displayed: Unable to get offset lags for kafka. Reason: org.apache.kafka.shaded.common.KafkaException: Failed to construct kafka consumer at org.apache.kafka.shaded.clients.consumer.KafkaConsumer.(KafkaConsumer= .java:703) at org.apache.kafka.shaded.clients.consumer.KafkaConsumer.(KafkaConsu= mer.java:584) at org.apache.kafka.shaded.clients.consumer.KafkaConsumer.(KafkaConsu= mer.java:566) at org.apache.storm.kafka.monitor.KafkaOffsetLagUtil.getOffsetLags(KafkaOff= setLagUtil.java:230) at org.apache.storm.kafka.monitor.KafkaOffsetLagUtil.main(KafkaOffsetLagUti= l.java:144) Caused by: org.apache.kafka.shaded.common.KafkaException: javax.security.auth.login.LoginException: No LoginModule found for org.apache.kafka.common.security.scram.ScramLoginModule at org.apache.kafka.shaded.common.network.SaslChannelBuilder.configure(SaslCha= nnelBuilder.java:86) at org.apache.kafka.shaded.common.network.ChannelBuilders.create(ChannelBui= lders.java:70) at org.apache.kafka.shaded.clients.ClientUtils.createChannelBuilder(ClientU= tils.java:85) at org.apache.kafka.shaded.clients.consumer.KafkaConsumer.(KafkaConsu= mer.java:644) ... 4 more Caused by: javax.security.auth.login.LoginException: No LoginModule found for org.apache.kafka.common.security.scram.ScramLoginModule at java.base/javax.security.auth.login.LoginContext.invoke(LoginContext.java:7= 10) at java.base/javax.security.auth.login.LoginContext$4.run(LoginContext.java= :665) at java.base/javax.security.auth.login.LoginContext$4.run(LoginContext.java= :663) at java.base/java.security.AccessController.doPrivileged(Native Method) at java.base/javax.security.auth.login.LoginContext.invokePriv(Logi= nContext.java:663) at java.base/javax.security.auth.login.LoginContext.login(LoginContext.java= :574) at org.apache.kafka.shaded.common.security.authenticator.AbstractLogin.logi= n(AbstractLogin.java:69) at org.apache.kafka.shaded.common.security.authenticator.LoginManager.(LoginManager.java:46) at org.apache.kafka.shaded.common.security.authenticator.LoginManager.acqui= reLoginManager(LoginManager.java:68) at org.apache.kafka.shaded.common.network.SaslChannelBuilder.configure(Sasl= ChannelBuilder.java:78) ... 7 more moreover, the storm-kafka-monitoring processes are eating a huge CPU. Maybe I'm missing a way to setup storm-kafka-monitoring's consumers with something equivalent to my Kafka Spout's setup (which works fine with my Kafka SSL + auth brokers): .setProp("security.protocol", "SASL_SSL") .setProp("sasl.mechanism", "SCRAM-SHA-512") .setProp(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "") What would be the right way to setup these properties on storm-kafka-client's consumers ? Would the pull requested suggested by Stig help? Note: i'm running with Apache Storm 1.2.3 latest commits, my topologies run fine with Kafka Brokers with SSL + Auth, but I need storm-kafka-monitor to be run & running to track Kafka lag of my Kafka spouts.... Kind regards, Alexandre Le lun. 3 juin 2019 =C3=A0 17:33, Sandeep Nemuri a = =C3=A9crit : > > I guess you are missing the KafkaClient section in ${STORM_CONF_DIR}/stor= m_jaas.conf in UI node. > > On Mon, Jun 3, 2019 at 8:32 PM Stig Rohde D=C3=B8ssing wrote: >> >> There was a recent PR to partially address this at https://github.com/ap= ache/storm/pull/3016, it probably doesn't fix your exact case, but fixing y= our case would probably require doing something similar. >> >> The reason this is broken is that storm-kafka-monitor is a workaround to= us not having generic metrics delivery from spouts to Storm UI. The way st= orm-kafka-monitor works, is that your spout reports some of its configurati= on to Nimbus. When someone requests a Storm UI page, Nimbus then boots up a= new JVM running storm-kafka-monitor, which just starts a KafkaConsumer wit= h the config from Nimbus, checks the current consumer position, and exits. >> >> We want to get rid of storm-kafka-monitor eventually. I've been looking = at it in https://issues.apache.org/jira/browse/STORM-3202, but it will prob= ably take a while to get done. If you'd like to take a look at implementing= that issue, I'd be happy to share the code (it does not currently compile)= . >> >> Den man. 3. jun. 2019 kl. 10.52 skrev Alexandre Vermeerbergen : >>> >>> Hello, >>> >>> I am trying to switch my Storm topologies based on Kafka spout from >>> authenticated & unencrypted Kafka brokers to SSL + Authenticated Kafka >>> Brokers. >>> >>> So I have changed my Kafka spouts build code to something like this >>> (actial user & password have been changed to 'foo' and 'bar'): >>> >>> kafkaSpoutConfig =3D >>> KafkaSpoutConfig.builder(supConfig.kafka_broker_hosts_str, "myTopic") >>> .setProp(ConsumerConfig.GROUP_ID_CONFIG, "myConsume= r") >>> .setProp("security.protocol", "SASL_SSL") >>> .setProp("sasl.jaas.config", >>> "org.apache.kafka.common.security.scram.ScramLoginModule required >>> username=3D\"foo\" password=3D\"bar\";") >>> .setProp("sasl.mechanism", "SCRAM-SHA-512") >>> >>> .setProp(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "") >>> .setFirstPollOffsetStrategy(FirstPollOffsetStrategy= .LATEST) >>> .setRecordTranslator(new PodSyncPodsKafkaRecordTran= slator()) >>> .build(); >>> >>> >>> My problem is that in Storm UI, I see that the usual Kafka Lag >>> monitoring fails with these errors: >>> >>> Topology spouts lag error >>> >>> IdTypeMessage >>> eventFromAdminTopicKAFKAUnable to get offset lags for kafka. Reason: >>> org.apache.kafka.shaded.common.KafkaException: Failed to construct >>> kafka consumer at >>> org.apache.kafka.shaded.clients.consumer.KafkaConsumer.(KafkaCons= umer.java:703) >>> at org.apache.kafka.shaded.clients.consumer.KafkaConsumer.(KafkaC= onsumer.java:584) >>> at org.apache.kafka.shaded.clients.consumer.KafkaConsumer.(KafkaC= onsumer.java:566) >>> at org.apache.storm.kafka.monitor.KafkaOffsetLagUtil.getOffsetLags(Kafk= aOffsetLagUtil.java:230) >>> at org.apache.storm.kafka.monitor.KafkaOffsetLagUtil.main(KafkaOffsetLa= gUtil.java:144) >>> Caused by: org.apache.kafka.shaded.common.KafkaException: >>> java.lang.IllegalArgumentException: Could not find a 'KafkaClient' >>> entry in the JAAS configuration. System property >>> 'java.security.auth.login.config' is not set at >>> org.apache.kafka.shaded.common.network.SaslChannelBuilder.configure(Sas= lChannelBuilder.java:86) >>> at org.apache.kafka.shaded.common.network.ChannelBuilders.create(Channe= lBuilders.java:70) >>> at org.apache.kafka.shaded.clients.ClientUtils.createChannelBuilder(Cli= entUtils.java:85) >>> at org.apache.kafka.shaded.clients.consumer.KafkaConsumer.(KafkaC= onsumer.java:644) >>> ... 4 more Caused by: java.lang.IllegalArgumentException: Could not >>> find a 'KafkaClient' entry in the JAAS configuration. System property >>> 'java.security.auth.login.config' is not set at >>> org.apache.kafka.shaded.common.security.authenticator.AbstractLogin.log= in(AbstractLogin.java:65) >>> at org.apache.kafka.shaded.common.security.authenticator.LoginManager.<= init>(LoginManager.java:46) >>> at org.apache.kafka.shaded.common.security.authenticator.LoginManager.a= cquireLoginManager(LoginManager.java:68) >>> at org.apache.kafka.shaded.common.network.SaslChannelBuilder.configure(= SaslChannelBuilder.java:78) >>> ... 7 more >>> >>> Question: >>> >>> I guess that this issue comes from the fact that storm-kafka-monitor >>> is not aware of the security setup which have been applied to the >>> Spout. >>> =3D> How can I configure storm-kafka-monitor to pass it the Kafka >>> consumer properties that will allow it to query the Kafka lag without >>> failing on Kafka authentication? >>> >>> I have not found a clue about this in documentation... >>> >>> Also, is it normal that Kafka spouts is not automatically inherited by >>> storm-kafka-monitor? >>> >>> Kind regards, >>> Alexandre Vermeerbergen > > > > -- > Regards > Sandeep Nemuri