[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15169607#comment-15169607 ] Derek Dagit commented on STORM-1469: [~rudra.sharma], if this was seen on master, was the topology submitted using an old client? I had thought in the latest version of storm the StormSubmitter submits topology jars in chunks of about 30kb at a time. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > at
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15169798#comment-15169798 ] Derek Dagit commented on STORM-1469: OK, looking at this some more: It is not the size of the jar file that matters here, it is the size of the topology config and the topology instance when serialized. If those are too big, then as [~kishorvpatil] noted, there is a 16MB max limit in thrift that we are not correctly overriding with the config mentioned above. Nimbus should launch with a thrift server that has the max buffer size configured correctly, instead of always using the default of 16MB. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.th
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174538#comment-15174538 ] ASF GitHub Bot commented on STORM-1469: --- GitHub user kishorvpatil opened a pull request: https://github.com/apache/storm/pull/1173 [STORM-1469] Adding Plain Sasl Transport Plugin Since `org.apache.storm.security.auth.SimpleTransportPlugin` creates `TFramedTransport` with maxBufferSize, but that conflicts with serialized topology of size exceeded 16MB - default `THsHaServer` allowed max buffer size. So the better option is to implement `PlainSaslTransportPlugin` which avoids using any default values for max-buffer-size. You can merge this pull request into a Git repository by running: $ git pull https://github.com/kishorvpatil/incubator-storm storm1469 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/storm/pull/1173.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #1173 commit b1e4c94269dbcf2ba01aee89e468c744887c53de Author: Kishor Patil Date: 2016-02-29T20:44:52Z Adding Plain Sasl Transport Plugin > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inb
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174554#comment-15174554 ] ASF GitHub Bot commented on STORM-1469: --- Github user d2r commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54649628 --- Diff: conf/defaults.yaml --- @@ -39,7 +39,7 @@ storm.exhibitor.port: 8080 storm.exhibitor.poll.uripath: "/exhibitor/v1/cluster/list" storm.cluster.mode: "distributed" # can be distributed or local storm.local.mode.zmq: false -storm.thrift.transport: "org.apache.storm.security.auth.SimpleTransportPlugin" +storm.thrift.transport: "org.apache.storm.security.auth.plain.PlainSaslTransportPlugin" --- End diff -- What is the rationale for the `.plain` portion of this config? The class already says it is Plain. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-1
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174560#comment-15174560 ] ASF GitHub Bot commented on STORM-1469: --- Github user d2r commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54649974 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java --- @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.security.auth.plain; + +import java.io.IOException; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; + + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * client side callback handler. + */ +public class PlainClientCallbackHandler implements CallbackHandler { +private static final String USERNAME = "username"; +private static final String PASSWORD = "password"; +private static final Logger LOG = LoggerFactory.getLogger(PlainClientCallbackHandler.class); +private String _username = "username"; +private String _password = "password"; + +/** + * This method is invoked by SASL for authentication challenges + * @param callbacks a collection of challenge callbacks + */ +public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { +for (Callback c : callbacks) { +if (c instanceof NameCallback) { +LOG.debug("name callback"); +NameCallback nc = (NameCallback) c; +nc.setName(_username); +} else if (c instanceof PasswordCallback) { +LOG.debug("password callback"); +PasswordCallback pc = (PasswordCallback)c; +if (_password != null) { +pc.setPassword(_password.toCharArray()); +} +} else if (c instanceof AuthorizeCallback) { +LOG.debug("authorization callback"); +AuthorizeCallback ac = (AuthorizeCallback) c; +String authid = ac.getAuthenticationID(); +String authzid = ac.getAuthorizationID(); +if (authid.equals(authzid)) { +ac.setAuthorized(true); +} else { +ac.setAuthorized(false); +} +if (ac.isAuthorized()) { +ac.setAuthorizedID(authzid); +} --- End diff -- We can just `setAuthorizedID` above when we `setAuthorized`. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174562#comment-15174562 ] ASF GitHub Bot commented on STORM-1469: --- Github user d2r commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54650057 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java --- @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.security.auth.plain; + +import java.io.IOException; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; + + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * client side callback handler. + */ +public class PlainClientCallbackHandler implements CallbackHandler { +private static final String USERNAME = "username"; +private static final String PASSWORD = "password"; +private static final Logger LOG = LoggerFactory.getLogger(PlainClientCallbackHandler.class); +private String _username = "username"; +private String _password = "password"; + +/** + * This method is invoked by SASL for authentication challenges + * @param callbacks a collection of challenge callbacks + */ +public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { +for (Callback c : callbacks) { +if (c instanceof NameCallback) { +LOG.debug("name callback"); +NameCallback nc = (NameCallback) c; +nc.setName(_username); +} else if (c instanceof PasswordCallback) { +LOG.debug("password callback"); +PasswordCallback pc = (PasswordCallback)c; +if (_password != null) { +pc.setPassword(_password.toCharArray()); +} +} else if (c instanceof AuthorizeCallback) { +LOG.debug("authorization callback"); +AuthorizeCallback ac = (AuthorizeCallback) c; +String authid = ac.getAuthenticationID(); +String authzid = ac.getAuthorizationID(); +if (authid.equals(authzid)) { +ac.setAuthorized(true); +} else { +ac.setAuthorized(false); +} +if (ac.isAuthorized()) { +ac.setAuthorizedID(authzid); +} +} else if (c instanceof RealmCallback) { --- End diff -- Minor: there is no LOG.debug here as with the other callbacks. Is that intentional? > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174563#comment-15174563 ] ASF GitHub Bot commented on STORM-1469: --- Github user d2r commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54650134 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java --- @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.security.auth.plain; + +import java.io.IOException; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; + + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * client side callback handler. + */ +public class PlainClientCallbackHandler implements CallbackHandler { +private static final String USERNAME = "username"; +private static final String PASSWORD = "password"; +private static final Logger LOG = LoggerFactory.getLogger(PlainClientCallbackHandler.class); +private String _username = "username"; +private String _password = "password"; + +/** + * This method is invoked by SASL for authentication challenges + * @param callbacks a collection of challenge callbacks + */ +public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { +for (Callback c : callbacks) { +if (c instanceof NameCallback) { +LOG.debug("name callback"); +NameCallback nc = (NameCallback) c; +nc.setName(_username); +} else if (c instanceof PasswordCallback) { +LOG.debug("password callback"); +PasswordCallback pc = (PasswordCallback)c; +if (_password != null) { +pc.setPassword(_password.toCharArray()); +} +} else if (c instanceof AuthorizeCallback) { +LOG.debug("authorization callback"); +AuthorizeCallback ac = (AuthorizeCallback) c; +String authid = ac.getAuthenticationID(); +String authzid = ac.getAuthorizationID(); +if (authid.equals(authzid)) { +ac.setAuthorized(true); +} else { +ac.setAuthorized(false); +} +if (ac.isAuthorized()) { +ac.setAuthorizedID(authzid); +} +} else if (c instanceof RealmCallback) { +RealmCallback rc = (RealmCallback) c; +((RealmCallback) c).setText(rc.getDefaultText()); --- End diff -- Can we not reuse `rc` instead of casting twice? > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > a
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174570#comment-15174570 ] ASF GitHub Bot commented on STORM-1469: --- Github user d2r commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54650921 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java --- @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.security.auth.plain; + +import java.io.IOException; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; + + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * client side callback handler. + */ +public class PlainClientCallbackHandler implements CallbackHandler { +private static final String USERNAME = "username"; +private static final String PASSWORD = "password"; +private static final Logger LOG = LoggerFactory.getLogger(PlainClientCallbackHandler.class); +private String _username = "username"; +private String _password = "password"; --- End diff -- Let's get rid of the private members and just use the constants already defined above. We also want to reference these hard-coded user/pass in the other classes. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-S
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174574#comment-15174574 ] ASF GitHub Bot commented on STORM-1469: --- Github user d2r commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54651094 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java --- @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.security.auth.plain; + +import org.apache.storm.security.auth.AuthUtils; +import org.apache.storm.security.auth.SaslTransportPlugin; +import org.apache.storm.utils.ExtendedThreadPoolExecutor; +import org.apache.thrift.TProcessor; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.server.TServer; +import org.apache.thrift.server.TThreadPoolServer; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TSaslServerTransport; +import org.apache.thrift.transport.TServerSocket; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; +import org.apache.thrift.transport.TTransportFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.callback.CallbackHandler; +import java.io.IOException; +import java.security.Security; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +public class PlainSaslTransportPlugin extends SaslTransportPlugin { +public static final String PLAIN = "PLAIN"; +private static final Logger LOG = LoggerFactory.getLogger(PlainSaslTransportPlugin.class); + +@Override +protected TTransportFactory getServerTransportFactory() throws IOException { +//create an authentication callback handler +CallbackHandler serer_callback_handler = new PlainServerCallbackHandler(); +Security.addProvider(new SaslPlainServer.SecurityProvider()); +//create a transport factory that will invoke our auth callback for digest +TSaslServerTransport.Factory factory = new TSaslServerTransport.Factory(); +factory.addServerDefinition(PLAIN, AuthUtils.SERVICE, "localhost", null, serer_callback_handler); + +LOG.info("SASL PLAIN transport factory will be used"); +return factory; +} + +@Override +public TTransport connect(TTransport transport, String serverHost, String asUser) throws IOException, TTransportException { +PlainClientCallbackHandler client_callback_handler = new PlainClientCallbackHandler(); +TSaslClientTransport wrapper_transport = new TSaslClientTransport(PLAIN, +null, +AuthUtils.SERVICE, +serverHost, +null, +client_callback_handler, +transport); + +wrapper_transport.open(); +LOG.debug("SASL PLAIN client transport has been established"); + +return wrapper_transport; + +} + --- End diff -- nit: blank lines > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174577#comment-15174577 ] ASF GitHub Bot commented on STORM-1469: --- Github user d2r commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54651204 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java --- @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.security.auth.plain; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.storm.security.auth.ReqContext; +import org.apache.storm.security.auth.SaslTransportPlugin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; + +/** + * SASL server side callback handler + */ +public class PlainServerCallbackHandler implements CallbackHandler { +private static final Logger LOG = LoggerFactory.getLogger(PlainServerCallbackHandler.class); +private static final String SYSPROP_SUPER_PASSWORD = "storm.SASLAuthenticationProvider.superPassword"; + +private String userName="username"; +private final Map credentials = new HashMap<>(); + +public PlainServerCallbackHandler() throws IOException { +credentials.put("username", "password"); +} --- End diff -- Here we probably want to reference constants. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFram
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174576#comment-15174576 ] ASF GitHub Bot commented on STORM-1469: --- Github user d2r commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54651160 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java --- @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.security.auth.plain; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.storm.security.auth.ReqContext; +import org.apache.storm.security.auth.SaslTransportPlugin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; + +/** + * SASL server side callback handler + */ +public class PlainServerCallbackHandler implements CallbackHandler { +private static final Logger LOG = LoggerFactory.getLogger(PlainServerCallbackHandler.class); +private static final String SYSPROP_SUPER_PASSWORD = "storm.SASLAuthenticationProvider.superPassword"; --- End diff -- Do we need a superuser password? I thought this was all hard-coded to use constants? > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174581#comment-15174581 ] ASF GitHub Bot commented on STORM-1469: --- Github user d2r commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54651338 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java --- @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.security.auth.plain; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.storm.security.auth.ReqContext; +import org.apache.storm.security.auth.SaslTransportPlugin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; + +/** + * SASL server side callback handler + */ +public class PlainServerCallbackHandler implements CallbackHandler { +private static final Logger LOG = LoggerFactory.getLogger(PlainServerCallbackHandler.class); +private static final String SYSPROP_SUPER_PASSWORD = "storm.SASLAuthenticationProvider.superPassword"; + +private String userName="username"; +private final Map credentials = new HashMap<>(); + +public PlainServerCallbackHandler() throws IOException { +credentials.put("username", "password"); +} + +public void handle(Callback[] callbacks) throws UnsupportedCallbackException { +for (Callback callback : callbacks) { +if (callback instanceof NameCallback) { +handleNameCallback((NameCallback) callback); +} else if (callback instanceof PasswordCallback) { +handlePasswordCallback((PasswordCallback) callback); +} else if (callback instanceof RealmCallback) { +handleRealmCallback((RealmCallback) callback); +} else if (callback instanceof AuthorizeCallback) { +handleAuthorizeCallback((AuthorizeCallback) callback); +} +} +} + +private void handleNameCallback(NameCallback nc) { +LOG.debug("handleNameCallback"); +userName = nc.getDefaultName(); +nc.setName(nc.getDefaultName()); +} + +private void handlePasswordCallback(PasswordCallback pc) { +LOG.debug("handlePasswordCallback"); +if ("super".equals(this.userName) && System.getProperty(SYSPROP_SUPER_PASSWORD) != null) { +// superuser: use Java system property for password, if available. + pc.setPassword(System.getProperty(SYSPROP_SUPER_PASSWORD).toCharArray()); +} else if (credentials.containsKey(userName) ) { +pc.setPassword(credentials.get(userName).toCharArray()); +} else { +LOG.warn("No password found for user: " + userName); +} +} --- End diff -- Same question here. Not sure we want the concept of a superuser. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception.
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174582#comment-15174582 ] ASF GitHub Bot commented on STORM-1469: --- Github user d2r commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54651434 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java --- @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.security.auth.plain; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.storm.security.auth.ReqContext; +import org.apache.storm.security.auth.SaslTransportPlugin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; + +/** + * SASL server side callback handler + */ +public class PlainServerCallbackHandler implements CallbackHandler { +private static final Logger LOG = LoggerFactory.getLogger(PlainServerCallbackHandler.class); +private static final String SYSPROP_SUPER_PASSWORD = "storm.SASLAuthenticationProvider.superPassword"; + +private String userName="username"; +private final Map credentials = new HashMap<>(); + +public PlainServerCallbackHandler() throws IOException { +credentials.put("username", "password"); +} + +public void handle(Callback[] callbacks) throws UnsupportedCallbackException { +for (Callback callback : callbacks) { +if (callback instanceof NameCallback) { +handleNameCallback((NameCallback) callback); +} else if (callback instanceof PasswordCallback) { +handlePasswordCallback((PasswordCallback) callback); +} else if (callback instanceof RealmCallback) { +handleRealmCallback((RealmCallback) callback); +} else if (callback instanceof AuthorizeCallback) { +handleAuthorizeCallback((AuthorizeCallback) callback); +} +} +} + +private void handleNameCallback(NameCallback nc) { +LOG.debug("handleNameCallback"); +userName = nc.getDefaultName(); +nc.setName(nc.getDefaultName()); +} + +private void handlePasswordCallback(PasswordCallback pc) { +LOG.debug("handlePasswordCallback"); +if ("super".equals(this.userName) && System.getProperty(SYSPROP_SUPER_PASSWORD) != null) { +// superuser: use Java system property for password, if available. + pc.setPassword(System.getProperty(SYSPROP_SUPER_PASSWORD).toCharArray()); +} else if (credentials.containsKey(userName) ) { +pc.setPassword(credentials.get(userName).toCharArray()); +} else { +LOG.warn("No password found for user: " + userName); +} +} + +private void handleRealmCallback(RealmCallback rc) { +LOG.debug("handleRealmCallback: "+ rc.getDefaultText()); +rc.setText(rc.getDefaultText()); +} + +private void handleAuthorizeCallback(AuthorizeCallback ac) { +String authenticationID = ac.getAuthenticationID(); +LOG.info("Successfully authenticated client: authenticationID = " + authenticationID + " authorizationID = " + ac.getAuthorizationID()); + +//if authorizationId is not set, set it to authenticationId. +if(ac.getAuthorizationID() == null) { +ac.setAuthoriz
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174583#comment-15174583 ] ASF GitHub Bot commented on STORM-1469: --- Github user d2r commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54651499 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java --- @@ -0,0 +1,154 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.storm.security.auth.plain; + +import java.security.Provider; +import java.util.Map; + +import javax.security.auth.callback.*; --- End diff -- I would be nice to expand all imports. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174585#comment-15174585 ] ASF GitHub Bot commented on STORM-1469: --- Github user kishorvpatil commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54651550 --- Diff: conf/defaults.yaml --- @@ -39,7 +39,7 @@ storm.exhibitor.port: 8080 storm.exhibitor.poll.uripath: "/exhibitor/v1/cluster/list" storm.cluster.mode: "distributed" # can be distributed or local storm.local.mode.zmq: false -storm.thrift.transport: "org.apache.storm.security.auth.SimpleTransportPlugin" +storm.thrift.transport: "org.apache.storm.security.auth.plain.PlainSaslTransportPlugin" --- End diff -- Under `org.apache.storm.security.auth` package we have sub-package by mechanism e.g. `digest`, `kerberos`. Hence the package name `plain` > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174586#comment-15174586 ] ASF GitHub Bot commented on STORM-1469: --- Github user d2r commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54651672 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java --- @@ -0,0 +1,154 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.storm.security.auth.plain; + +import java.security.Provider; +import java.util.Map; + +import javax.security.auth.callback.*; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslException; +import javax.security.sasl.SaslServer; +import javax.security.sasl.SaslServerFactory; + +public class SaslPlainServer implements SaslServer { + @SuppressWarnings("serial") + public static class SecurityProvider extends Provider { +public SecurityProvider() { + super("SaslPlainServer", 1.0, "SASL PLAIN Authentication Server"); + put("SaslServerFactory.PLAIN", + SaslPlainServerFactory.class.getName()); --- End diff -- Just `"SaslPlainServerFactory"` ? > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryP
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174591#comment-15174591 ] ASF GitHub Bot commented on STORM-1469: --- Github user d2r commented on the pull request: https://github.com/apache/storm/pull/1173#issuecomment-190954323 @kishorvpatil Some of this looks like borrowed code that is not tailored specifically to storm. Is that the case? > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > at org.apache.th
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15174636#comment-15174636 ] ASF GitHub Bot commented on STORM-1469: --- Github user kishorvpatil commented on the pull request: https://github.com/apache/storm/pull/1173#issuecomment-190969675 @d2r That's right. The `SaslPlainServer` is implementation borrowed from [hadoop](https://github.com/apache/hadoop/blob/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPlainServer.java). And `CallBackHandler` implementations are mostly clones from digest package with minor changes. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15175109#comment-15175109 ] ASF GitHub Bot commented on STORM-1469: --- Github user kishorvpatil commented on the pull request: https://github.com/apache/storm/pull/1173#issuecomment-191083880 @d2r i have tried to address all your comments, including refactoring two abstract classed `AbstractSaslClientCallbackHandler.java` and `AbstractSaslServerCallbackHandler.java` to place common methods for DIGEST and PLAIN in one place. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTra
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15175125#comment-15175125 ] ASF GitHub Bot commented on STORM-1469: --- Github user kishorvpatil closed the pull request at: https://github.com/apache/storm/pull/1173 > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.ja
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15175126#comment-15175126 ] ASF GitHub Bot commented on STORM-1469: --- GitHub user kishorvpatil reopened a pull request: https://github.com/apache/storm/pull/1173 [STORM-1469] Adding Plain Sasl Transport Plugin Since `org.apache.storm.security.auth.SimpleTransportPlugin` creates `TFramedTransport` with maxBufferSize, but that conflicts with serialized topology of size exceeded 16MB - default `THsHaServer` allowed max buffer size. So the better option is to implement `PlainSaslTransportPlugin` which avoids using any default values for max-buffer-size. You can merge this pull request into a Git repository by running: $ git pull https://github.com/kishorvpatil/incubator-storm storm1469 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/storm/pull/1173.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #1173 commit b1e4c94269dbcf2ba01aee89e468c744887c53de Author: Kishor Patil Date: 2016-02-29T20:44:52Z Adding Plain Sasl Transport Plugin commit cccb9766eb6b01477b44cd35e836997811464632 Author: Kishor Patil Date: 2016-03-02T06:12:57Z Refactoring SaslServerCallbackHandler and SaslClientCallbackHandler > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[sto
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15175216#comment-15175216 ] ASF GitHub Bot commented on STORM-1469: --- Github user abhishekagarwal87 commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54689200 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java --- @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.security.auth.plain; + +import org.apache.storm.security.auth.AuthUtils; +import org.apache.storm.security.auth.SaslTransportPlugin; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TSaslServerTransport; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; +import org.apache.thrift.transport.TTransportFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.callback.CallbackHandler; +import java.io.IOException; +import java.security.Security; + +public class PlainSaslTransportPlugin extends SaslTransportPlugin { +public static final String PLAIN = "PLAIN"; +private static final Logger LOG = LoggerFactory.getLogger(PlainSaslTransportPlugin.class); + +@Override +protected TTransportFactory getServerTransportFactory() throws IOException { +//create an authentication callback handler +CallbackHandler server_callback_handler = new PlainServerCallbackHandler(); --- End diff -- minor: could use camelCase convention. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15175650#comment-15175650 ] ASF GitHub Bot commented on STORM-1469: --- Github user revans2 commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54727772 --- Diff: conf/defaults.yaml --- @@ -39,7 +39,7 @@ storm.exhibitor.port: 8080 storm.exhibitor.poll.uripath: "/exhibitor/v1/cluster/list" storm.cluster.mode: "distributed" # can be distributed or local storm.local.mode.zmq: false -storm.thrift.transport: "org.apache.storm.security.auth.SimpleTransportPlugin" +storm.thrift.transport: "org.apache.storm.security.auth.plain.PlainSaslTransportPlugin" --- End diff -- This breaks wire compatibility, so it will not be a rolling upgrade. I am OK with that, but I want to be sure that it is documented. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/i
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15175651#comment-15175651 ] ASF GitHub Bot commented on STORM-1469: --- Github user revans2 commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54727990 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslClientCallbackHandler.java --- @@ -0,0 +1,76 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.security.auth; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; +import java.io.IOException; + +public abstract class AbstractSaslClientCallbackHandler implements CallbackHandler { +protected static final String USERNAME = "username"; +protected static final String PASSWORD = "password"; +private static final Logger LOG = LoggerFactory.getLogger(AbstractSaslClientCallbackHandler.class); +protected String _username = null; +protected String _password = null; + +/** + * This method is invoked by SASL for authentication challenges + * @param callbacks a collection of challenge callbacks + */ +public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { +for (Callback c : callbacks) { +if (c instanceof NameCallback) { +LOG.debug("name callback"); +NameCallback nc = (NameCallback) c; +nc.setName(_username); --- End diff -- So we are always going to set the name to null??? > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTranspo
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15175655#comment-15175655 ] ASF GitHub Bot commented on STORM-1469: --- Github user revans2 commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54728287 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java --- @@ -0,0 +1,77 @@ +package org.apache.storm.security.auth; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; +import java.util.HashMap; +import java.util.Map; + +public abstract class AbstractSaslServerCallbackHandler implements CallbackHandler { +private static final Logger LOG = LoggerFactory.getLogger(AbstractSaslServerCallbackHandler.class); +protected final Map credentials = new HashMap<>(); +protected String userName; + +public void handle(Callback[] callbacks) throws UnsupportedCallbackException { +for (Callback callback : callbacks) { +if (callback instanceof NameCallback) { +handleNameCallback((NameCallback) callback); +} else if (callback instanceof PasswordCallback) { +handlePasswordCallback((PasswordCallback) callback); +} else if (callback instanceof RealmCallback) { +handleRealmCallback((RealmCallback) callback); +} else if (callback instanceof AuthorizeCallback) { +handleAuthorizeCallback((AuthorizeCallback) callback); +} +} +} + +private void handleNameCallback(NameCallback nc) { +LOG.debug("handleNameCallback"); +userName = nc.getDefaultName(); +nc.setName(nc.getDefaultName()); +} + +protected void handlePasswordCallback(PasswordCallback pc) { +LOG.debug("handlePasswordCallback"); +if (credentials.containsKey(userName) ) { +pc.setPassword(credentials.get(userName).toCharArray()); +} else { +LOG.warn("No password found for user: " + userName); +} +} + +private void handleRealmCallback(RealmCallback rc) { +LOG.debug("handleRealmCallback: "+ rc.getDefaultText()); --- End diff -- Lets use the "{}" syntax here. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-S
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15175659#comment-15175659 ] ASF GitHub Bot commented on STORM-1469: --- Github user revans2 commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54728782 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java --- @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.security.auth.plain; + +import org.apache.storm.security.auth.AuthUtils; +import org.apache.storm.security.auth.SaslTransportPlugin; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TSaslServerTransport; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; +import org.apache.thrift.transport.TTransportFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.callback.CallbackHandler; +import java.io.IOException; +import java.security.Security; + +public class PlainSaslTransportPlugin extends SaslTransportPlugin { +public static final String PLAIN = "PLAIN"; +private static final Logger LOG = LoggerFactory.getLogger(PlainSaslTransportPlugin.class); + +@Override +protected TTransportFactory getServerTransportFactory() throws IOException { +//create an authentication callback handler +CallbackHandler server_callback_handler = new PlainServerCallbackHandler(); +Security.addProvider(new SaslPlainServer.SecurityProvider()); --- End diff -- What happens if we add this multiple times? > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apa
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15175660#comment-15175660 ] ASF GitHub Bot commented on STORM-1469: --- Github user revans2 commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54728926 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java --- @@ -0,0 +1,154 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.storm.security.auth.plain; + +import java.security.Provider; +import java.util.Map; + +import javax.security.auth.callback.*; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslException; +import javax.security.sasl.SaslServer; +import javax.security.sasl.SaslServerFactory; + +public class SaslPlainServer implements SaslServer { + @SuppressWarnings("serial") + public static class SecurityProvider extends Provider { +public SecurityProvider() { + super("SaslPlainServer", 1.0, "SASL PLAIN Authentication Server"); + put("SaslServerFactory.PLAIN", + SaslPlainServerFactory.class.getName()); --- End diff -- This ensures that if the package changes or something else like it changes we get a compile error, not a runtime error. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15175656#comment-15175656 ] ASF GitHub Bot commented on STORM-1469: --- Github user revans2 commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54728326 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java --- @@ -0,0 +1,77 @@ +package org.apache.storm.security.auth; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; +import java.util.HashMap; +import java.util.Map; + +public abstract class AbstractSaslServerCallbackHandler implements CallbackHandler { +private static final Logger LOG = LoggerFactory.getLogger(AbstractSaslServerCallbackHandler.class); +protected final Map credentials = new HashMap<>(); +protected String userName; + +public void handle(Callback[] callbacks) throws UnsupportedCallbackException { +for (Callback callback : callbacks) { +if (callback instanceof NameCallback) { +handleNameCallback((NameCallback) callback); +} else if (callback instanceof PasswordCallback) { +handlePasswordCallback((PasswordCallback) callback); +} else if (callback instanceof RealmCallback) { +handleRealmCallback((RealmCallback) callback); +} else if (callback instanceof AuthorizeCallback) { +handleAuthorizeCallback((AuthorizeCallback) callback); +} +} +} + +private void handleNameCallback(NameCallback nc) { +LOG.debug("handleNameCallback"); +userName = nc.getDefaultName(); +nc.setName(nc.getDefaultName()); +} + +protected void handlePasswordCallback(PasswordCallback pc) { +LOG.debug("handlePasswordCallback"); +if (credentials.containsKey(userName) ) { +pc.setPassword(credentials.get(userName).toCharArray()); +} else { +LOG.warn("No password found for user: " + userName); --- End diff -- "{}" syntax > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] >
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15175670#comment-15175670 ] ASF GitHub Bot commented on STORM-1469: --- Github user revans2 commented on the pull request: https://github.com/apache/storm/pull/1173#issuecomment-191261944 Overall it looks fairly good, but you have a problem with RAT somewhere. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > at org.apache.thrift7.transport.TTransport.readAll(TTr
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15175671#comment-15175671 ] ASF GitHub Bot commented on STORM-1469: --- Github user revans2 commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54730042 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java --- @@ -0,0 +1,77 @@ +package org.apache.storm.security.auth; --- End diff -- Needs an Apache header. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15175768#comment-15175768 ] ASF GitHub Bot commented on STORM-1469: --- Github user revans2 commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54736725 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java --- @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.security.auth.plain; + +import org.apache.storm.security.auth.AbstractSaslClientCallbackHandler; + +public class PlainClientCallbackHandler extends AbstractSaslClientCallbackHandler { + +/* + * For plain, using constants for a pair of user name and password. + */ +public PlainClientCallbackHandler() { +_username = USERNAME; --- End diff -- I forgot to ask before is the user this always reports to nimbus/etc now "username"? If so I really would prefer to pass in the user name of the current user on this system, or even better have the server set the user to null like we were doing before. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtoco
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15175813#comment-15175813 ] ASF GitHub Bot commented on STORM-1469: --- Github user kishorvpatil commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54740640 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java --- @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.security.auth.plain; + +import org.apache.storm.security.auth.AuthUtils; +import org.apache.storm.security.auth.SaslTransportPlugin; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TSaslServerTransport; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; +import org.apache.thrift.transport.TTransportFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.callback.CallbackHandler; +import java.io.IOException; +import java.security.Security; + +public class PlainSaslTransportPlugin extends SaslTransportPlugin { +public static final String PLAIN = "PLAIN"; +private static final Logger LOG = LoggerFactory.getLogger(PlainSaslTransportPlugin.class); + +@Override +protected TTransportFactory getServerTransportFactory() throws IOException { +//create an authentication callback handler +CallbackHandler server_callback_handler = new PlainServerCallbackHandler(); +Security.addProvider(new SaslPlainServer.SecurityProvider()); --- End diff -- I will add it to constructor with additional check on if provider is already registered. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15175885#comment-15175885 ] ASF GitHub Bot commented on STORM-1469: --- Github user kishorvpatil commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54749129 --- Diff: conf/defaults.yaml --- @@ -39,7 +39,7 @@ storm.exhibitor.port: 8080 storm.exhibitor.poll.uripath: "/exhibitor/v1/cluster/list" storm.cluster.mode: "distributed" # can be distributed or local storm.local.mode.zmq: false -storm.thrift.transport: "org.apache.storm.security.auth.SimpleTransportPlugin" +storm.thrift.transport: "org.apache.storm.security.auth.plain.PlainSaslTransportPlugin" --- End diff -- Since there is not good place to document this. I am simply going without any chances to default on "storm.thrift.tranport" so we can pull this in 1.x-branch as well. I will create two separate pull requests - 1. to change default and deprecate `SimpleTransportPlugin` and configuration on 1.x-branch. 2. To remove `SimpleTransportPlugin` and configuration in master. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.8
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15176219#comment-15176219 ] ASF GitHub Bot commented on STORM-1469: --- Github user kishorvpatil commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54769352 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java --- @@ -0,0 +1,77 @@ +package org.apache.storm.security.auth; --- End diff -- added > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) >
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15176265#comment-15176265 ] ASF GitHub Bot commented on STORM-1469: --- Github user kishorvpatil commented on a diff in the pull request: https://github.com/apache/storm/pull/1173#discussion_r54772909 --- Diff: storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java --- @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.security.auth.plain; + +import org.apache.storm.security.auth.AbstractSaslClientCallbackHandler; + +public class PlainClientCallbackHandler extends AbstractSaslClientCallbackHandler { + +/* + * For plain, using constants for a pair of user name and password. + */ +public PlainClientCallbackHandler() { +_username = USERNAME; --- End diff -- Fixed by using System.getProperty("user.name"). > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15176386#comment-15176386 ] ASF GitHub Bot commented on STORM-1469: --- Github user revans2 commented on the pull request: https://github.com/apache/storm/pull/1173#issuecomment-191407577 Looks good to me. The test failures look unrelated to this. +1 > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > at org.apache.thrift7.transport.TTransport.readAll(TTransport.
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15176441#comment-15176441 ] ASF GitHub Bot commented on STORM-1469: --- Github user d2r commented on the pull request: https://github.com/apache/storm/pull/1173#issuecomment-191419837 Seems good to me. +1 > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > at > org.apache.thrift7.protoc
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177287#comment-15177287 ] ASF GitHub Bot commented on STORM-1469: --- GitHub user kishorvpatil opened a pull request: https://github.com/apache/storm/pull/1178 [STORM-1469] Remove SimpleTransportPlugin and configuration This pull request awaits merging in #1173 You can merge this pull request into a Git repository by running: $ git pull https://github.com/kishorvpatil/incubator-storm storm1469-removeSimple Alternatively you can review and apply these changes as the patch at: https://github.com/apache/storm/pull/1178.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #1178 commit b1e4c94269dbcf2ba01aee89e468c744887c53de Author: Kishor Patil Date: 2016-02-29T20:44:52Z Adding Plain Sasl Transport Plugin commit cccb9766eb6b01477b44cd35e836997811464632 Author: Kishor Patil Date: 2016-03-02T06:12:57Z Refactoring SaslServerCallbackHandler and SaslClientCallbackHandler commit 4e04ce8dcdc33d488a3d15f7a47ab8af15136db4 Author: Kishor Patil Date: 2016-03-02T16:27:56Z Addressing review comments commit 5e2d44df8c342d29d723ac4ac90d0e1efb6884bb Author: Kishor Patil Date: 2016-03-02T18:32:00Z Using real user-id in the ReqContext instead of username in PlainSaslTransportPlugin commit 1dbdfb1769979a8391348b5275bfd4bd2a4edf18 Author: Kishor Patil Date: 2016-03-02T18:48:48Z Renaming local variable to camelCase commit b133d52043a5f97c019d796a27ded81523120428 Author: Kishor Patil Date: 2016-03-03T04:46:48Z Remove SimpleTransportPlugin and configurations > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Ni
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177288#comment-15177288 ] ASF GitHub Bot commented on STORM-1469: --- GitHub user kishorvpatil opened a pull request: https://github.com/apache/storm/pull/1179 [STORM-1469] Decommissioning SimpleTransportPlugin and configuration This pull request awaits merging in #1173 merged into 1.x-branch as well. You can merge this pull request into a Git repository by running: $ git pull https://github.com/kishorvpatil/incubator-storm storm1469-simpledecom Alternatively you can review and apply these changes as the patch at: https://github.com/apache/storm/pull/1179.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #1179 commit 59b29b218357aaf39ae70e829ad5ff20e0732431 Author: Kishor Patil Date: 2016-03-03T05:36:32Z Decommissioning SimpleTransportPlugin > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-1
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177289#comment-15177289 ] ASF GitHub Bot commented on STORM-1469: --- Github user kishorvpatil commented on the pull request: https://github.com/apache/storm/pull/1173#issuecomment-191596685 Created two pull requests as follow-up #1178 Remove SimpleTransportPlugin and configuration and #1179 Decommissioning SimpleTransportPlugin and configuration ( on 1.x-branch.) > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrif
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15178569#comment-15178569 ] ASF GitHub Bot commented on STORM-1469: --- Github user revans2 commented on the pull request: https://github.com/apache/storm/pull/1179#issuecomment-191949144 +1 not sure why travis failed. Didn't even try to build in most cases. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > at org.apache.thrift7.transport.TTransport.readAll(TTra
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15178570#comment-15178570 ] ASF GitHub Bot commented on STORM-1469: --- Github user revans2 commented on the pull request: https://github.com/apache/storm/pull/1178#issuecomment-191949380 Please rebase > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > at > org.apache.thrift7.protocol.
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15179037#comment-15179037 ] ASF GitHub Bot commented on STORM-1469: --- Github user kishorvpatil closed the pull request at: https://github.com/apache/storm/pull/1179 > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.ja
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15179038#comment-15179038 ] ASF GitHub Bot commented on STORM-1469: --- GitHub user kishorvpatil reopened a pull request: https://github.com/apache/storm/pull/1179 [STORM-1469] Decommissioning SimpleTransportPlugin and configuration This pull request awaits merging in #1173 merged into 1.x-branch as well. You can merge this pull request into a Git repository by running: $ git pull https://github.com/kishorvpatil/incubator-storm storm1469-simpledecom Alternatively you can review and apply these changes as the patch at: https://github.com/apache/storm/pull/1179.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #1179 commit 59b29b218357aaf39ae70e829ad5ff20e0732431 Author: Kishor Patil Date: 2016-03-03T05:36:32Z Decommissioning SimpleTransportPlugin > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15183644#comment-15183644 ] ASF GitHub Bot commented on STORM-1469: --- Github user asfgit closed the pull request at: https://github.com/apache/storm/pull/1173 > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0, 2.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15185819#comment-15185819 ] ASF GitHub Bot commented on STORM-1469: --- Github user ptgoetz commented on the pull request: https://github.com/apache/storm/pull/1179#issuecomment-193971785 +1. The travis failures seem like a red herring. All tests passed in my environment. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > at org.apache.thrift7.transport.TTransport.readA
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15185827#comment-15185827 ] ASF GitHub Bot commented on STORM-1469: --- Github user d2r commented on the pull request: https://github.com/apache/storm/pull/1179#issuecomment-193972813 +1 > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > at > org.apache.thrift7.protocol.TBinaryProtocol.readAl
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15187901#comment-15187901 ] ASF GitHub Bot commented on STORM-1469: --- Github user ptgoetz commented on the pull request: https://github.com/apache/storm/pull/1179#issuecomment-194497883 Merged #1179 to 1.x-branch. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > at > org.apache.thrift7.pr
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15187902#comment-15187902 ] ASF GitHub Bot commented on STORM-1469: --- Github user asfgit closed the pull request at: https://github.com/apache/storm/pull/1179 > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) >
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15188034#comment-15188034 ] ASF GitHub Bot commented on STORM-1469: --- Github user revans2 commented on the pull request: https://github.com/apache/storm/pull/1178#issuecomment-194507679 +1 > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > at > org.apache.thrift7.protocol.TBinaryProtocol.re
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15188036#comment-15188036 ] ASF GitHub Bot commented on STORM-1469: --- Github user revans2 commented on the pull request: https://github.com/apache/storm/pull/1178#issuecomment-194507988 oops I missed that org.apache.storm.security.auth.auth-test / testname: digest-authentication-test is failing thinking that it want plane. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101)
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15226315#comment-15226315 ] ASF GitHub Bot commented on STORM-1469: --- Github user abhishekagarwal87 commented on the pull request: https://github.com/apache/storm/pull/1178#issuecomment-205824333 @kishorvpatil can you rebase it? Also, I see that default transport plugin in 1.x is still SimpleTransportPlugin even though it is deprecated. Is it intended? > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.re
[jira] [Commented] (STORM-1469) Unable to deploy large topologies on apache storm
[ https://issues.apache.org/jira/browse/STORM-1469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15226329#comment-15226329 ] ASF GitHub Bot commented on STORM-1469: --- Github user kishorvpatil commented on the pull request: https://github.com/apache/storm/pull/1178#issuecomment-205829598 @abhishekagarwal87 I supposed we did not merge this PR in time. I think we should change default plugin to `PlainSaslTransportPlugin`. Let me upmerge this. > Unable to deploy large topologies on apache storm > - > > Key: STORM-1469 > URL: https://issues.apache.org/jira/browse/STORM-1469 > Project: Apache Storm > Issue Type: Bug > Components: storm-core >Affects Versions: 1.0.0, 2.0.0 >Reporter: Rudra Sharma >Assignee: Kishor Patil > Fix For: 1.0.0 > > > When deploying to a nimbus a topology which is larger in size >17MB, we get > an exception. In storm 0.9.3 this could be mitigated by using the following > config on the storm.yaml to increse the buffer size to handle the topology > size. i.e. 50MB would be > nimbus.thrift.max_buffer_size: 5000 > This configuration does not resolve the issue in the master branch of storm > and we cannot deploy topologies which are large in size. > Here is the log on the client side when attempting to deploy to the nimbus > node: > java.lang.RuntimeException: org.apache.thrift7.transport.TTransportException > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:251) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:272) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopology(StormSubmitter.java:155) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.deploy(TopologyDeployer.java:149) > [siem-ng-storm-deployer-cloud.jar:] > at > com.trustwave.siem.storm.topology.deployer.TopologyDeployer.main(TopologyDeployer.java:87) > [siem-ng-storm-deployer-cloud.jar:] > Caused by: org.apache.thrift7.transport.TTransportException > at > org.apache.thrift7.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:129) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.transport.TFramedTransport.read(TFramedTransport.java:101) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at org.apache.thrift7.transport.TTransport.readAll(TTransport.java:86) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > org.apache.thrift7.TServiceClient.receiveBase(TServiceClient.java:77) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.recv_submitTopology(Nimbus.java:238) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.generated.Nimbus$Client.submitTopology(Nimbus.java:222) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > at > backtype.storm.StormSubmitter.submitTopologyAs(StormSubmitter.java:237) > ~[storm-core-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT] > ... 4 more > Here is the log on the server side (nimbus.log): > 2016-01-13 10:48:07.206 o.a.s.d.nimbus [INFO] Cleaning inbox ... deleted: > stormjar-c8666220-fa19-426b-a7e4-c62dfb57f1f0.jar > 2016-01-13 10:55:09.823 o.a.s.d.nimbus [INFO] Uploading file from client to > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:11.910 o.a.s.d.nimbus [INFO] Finished uploading file from > client: > /var/storm-data/nimbus/inbox/stormjar-80ecdf05-6a25-4281-8c78-10062ac5e396.jar > 2016-01-13 10:55:12.084 o.a.t.s.AbstractNonblockingServer$FrameBuffer [WARN] > Exception while invoking! > org.apache.thrift7.transport.TTransportException: Frame size (17435758) > larger than max length (16384000)! > at > org.apache.thrift7.transport.TFramedTransport.readFrame(TFramedTransport.java:137) > at > org.apache.thrift7.transport.TFramedTransport.read(TFramed