Joe, Before moving forward and make things clearer.
After applying all the suggestions (timeouts, VM settings) how stable is your cluster without any load (without examples execution)? In particular: - how often is the case when a node fails to join the topology and you see topology snapshot containing less than 64 nodes? In the logs you sent me I see that topology assembled without any issue. If this happens please share the such logs with us. - how long does the cluster live (without any load) before you see I/O related issues or some other instability? Sorry if I haven't seen the answers on these questions in your previous responses. For me it's still unclear whether you start seeing the instability when the cluster under load or not. -- Denis On Fri, Oct 30, 2015 at 8:26 PM, <d...@eiler.net> wrote: > Adding the networkTimeouts for 15 seconds did not help, but I'm going to > bump up the networkTimeout to 30 seconds and try it again. (note: 30 > seconds didn't help either) > > adding the additional GC options did not help > > Each ignite node (i.e. each ignite.sh run) is on its own dedicated > physical hardware (2x Intel Xeon w/ 8 cores each..so /proc/cpuinfo shows 32 > processors, all have 64GB RAM some have 128GB, no local disk, Mellanox > ConnectX-3 FDR IB) > > Again just some recap of what I think I currently know: > 8 nodes works fine, as I increase cluster size the probability of success > drops. > > I have occasionally (about 5 times) gotten everything to come up with 64 > nodes and run and I can run pi estimator fine then, I usually kill it when > trying to load data to do something real but that could be issues with my > stuff still. I've also confirmed that using hdfs cli is also affected in > the same way (both put and ls). > > I can predict if a job (either pi estimator or hdfs put) will fail (i.e. > hang). If I see any of the ignite node logs contain a "Closing NIO session > because unhandled exception". I add a couple printStackTrace calls and it > is being thrown from the sockCh.read(readBuf) in the nested > DirectNioClientWorker in GridNioServer.java. Since I have been paying > attention, the exception comes just under 15 minutes after starting the > ignite nodes. (and I am still seeing the "Retrying preload partition > exchange due to timeout" on some nodes when that happens) > > In the cases when stuff comes up and works, it seems to only take a few > minutes (but I've been to excited to remember to check the logs for actual > times and I keep forgetting to save away a "working" set of logs). > > > Other ideas bouncing around in my head (in addition to any suggestions > provided): > Enable tracing. > My ignite framework and my log files all go to a shared filesystem across > the nodes, I'm going to try to push the log files down to a local tmpfs > area in case that is contributing somehow > I might try wrapping the sockCh.read() call in some debug, and maybe even > a retry loop just to see if I can learn anything about the health of the > channel. > I want to take a look at the system logs (/var/log/messages, etc.) and see > if anything is showing up there but I currently don't have read access > Start making thread dumps of processes. > I might try to flip all my caches mode to something else (local probably) > Take a look at rebalancing (can I increase threads, transfer sizes,etc.) > Do slow ramp up of nodes (start 8, wait until they are up, add a couple > more at a time) > > > Thanks again for the suggestions (and any future ones), > > Joe > > > > > Quoting Denis Magda <dma...@gridgain.com>: > > Joe, >> >> After taking a look at the logs I should say that the topology and the >> whole cluster is stable, no any node left the topology. So >> 'failureDetectionTimeout' did what it had been created for. You can keep >> using its current value. >> >> Preloading timeout happens from time to time on some nodes but finally it >> finishes. I'm quite confident that if you increase >> IgniteConfiguration.networkTimeout and TcpDiscoverySpi.networkTimeout >> values then this message will disappear at all. >> Let's try to set it to 15 secs. The example is below. In general, I think >> that we will simplify network timeouts configuration in the nearest >> releases. >> >> <bean id="ignite.cfg" >> class="org.apache.ignite.configuration.IgniteConfiguration"> >> >> <property name="networkTimeout" value="15000"/> >> >> ...... >> <property name="discoverySpi"> >> <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi"> >> <property name="networkTimeout" value="15000"/> >> ............. >> </bean> >> </property> >> </bean> >> >> >> Next. Since you have quite a big cluster I would suggest you tuning VM >> settings a bit. >> >> - server >> -XX:+UseParNewGC >> -XX:+UseConcMarkSweepGC >> -XX:+UseTLAB >> -XX:+UseCMSInitiatingOccupancyOnly >> -XX:CMSInitiatingOccupancyFraction=70 >> -XX:+ScavengeBeforeFullGC >> -XX:+CMSScavengeBeforeRemark >> >> Finally, the only clue I see that may help us to realize why pi estimator >> example hangs is the following line in the logs of some nodes >> >> [07:33:41,704][WARN ][main][IgfsServerManager] Failed to start IGFS >> endpoint (will retry every 3s). Failed to bind to port (is port already in >> use?): 10500 >> >> This can probably lead to the hangs. >> >> Is every node started on its own physical machine? >> I guess that you start several nodes per one single machine. If my >> understanding is correct then please make sure that every node that is >> running on a single machine has unique IGFS endpoint port number. >> >> Joe, if fixing of IGFS endpoints port numbers doesn't help then please >> send us thread dumps from the nodes that are hanging during examples >> execution. >> >> Regards, >> Denis >> >> On 10/29/2015 4:37 PM, d...@eiler.net wrote: >> >>> Nope, not trying to load any data yet. Just starting up all the nodes >>> and trying to run the hadoop mapreduce pi estimator example. >>> >>> I made all the changes below and started all the nodes (I have a script >>> the does a pssh so all the nodes start at about the same time.) I then >>> waited quite a while until all the node logs stopped showing the "Retrying >>> preload" messages. At this point one of the nodes already had the NIO >>> exception. >>> >>> But I then attempted to run the pi estimator example anyways. >>> It prints out the Number of Maps / Samples per Map messages but it just >>> hangs and I never see the "Wrote input for Map" messages (there have been >>> times were I'll see a few and then it will hang. That is what made me think >>> it was the writes using the "hdfs api" that were hanging. >>> >>> Denis, I'll send a tar file with the logs directly to you shortly so as >>> not to clog peoples inboxes. >>> >>> Joe >>> >>> >>> Quoting Denis Magda <dma...@gridgain.com>: >>> >>> Hi Joe, >>>> >>>> No problems, I'll guide you until we get to the bottom. >>>> >>>> Do you start pre-loading the caches with data right after the cluster >>>> is ready? If so let's postpone doing this until you have a stable cluster >>>> with caches rebalanced and ready to be used. >>>> >>>> Please, do the following as the next steps: >>>> >>>> 1) Set 'failureDetectionTimeout' to a bigger value (~ 15 secs); >>>> >>>> 2) Set CacheConfiguration.setRebalanceTimeout to a value that is >>>> approximately equal to the time when all the nodes are joined the topology >>>> (~ 1 minute or so). >>>> >>>> 3) Enable verbose logging for every node by passing >>>> -DIGNITE_QUEIT=false parameter to virtual machine arguments list. If you >>>> use ignite.sh script then just pass '-v' flag. >>>> >>>> 4) Enable garbage collection logs for every node by passing this string >>>> to virtual machine arguments list -Xloggc:./gc.log -XX:+PrintGCDetails >>>> -verbose:gc >>>> >>>> When you did a test run taking into account all the points above please >>>> gather all the logs (including garbage collection logs) and send us for >>>> further investigation. >>>> >>>> Regards, >>>> Denis >>>> >>>> On 10/28/2015 1:40 PM, d...@eiler.net wrote: >>>> >>>>> Thanks for the info Denis. >>>>> >>>>> Removing the failureDetectionTimeout and using the networkTimeout >>>>> seems to allow the nodes to join the topology in about the same amount of >>>>> time. I'm still only having occasional success running anything (even just >>>>> the pi estimator) >>>>> >>>>> I seem to always see a bunch of warnings...a summary is dumped below >>>>> along with my config at the end, any guidance you can provide is >>>>> appreciated. >>>>> >>>>> Thanks, >>>>> Joe >>>>> >>>>> Every node seems to see a bunch of "Retrying preload partition", with >>>>> the lowest locNodeOrder having fewer nodes in the remaining >>>>> >>>>> [14:52:38,979][WARN >>>>> ][ignite-#104%sys-null%][GridDhtPartitionsExchangeFuture] Retrying preload >>>>> partition exchange due to timeout [done=false, dummy=false, >>>>> exchId=GridDhtPartitionExchangeId [topVer=AffinityTopologyVersion >>>>> [topVer=62, minorTopVer=0], nodeId=fd9620f5, evt=NODE_JOINED], rcvdIds=[], >>>>> rmtIds=[0ab29a08, 5216f6ba, f882885f, 0d232f1a, b74f5ebb, 5790761a, >>>>> 55d2082e, b1bf93b3, 2fd79f9f, a899ccce, 3dd74aba, 320d05fd, 0d44a4b3, >>>>> 9a00f235, 4426467e, 7837fdfc, e8778da0, 4a988e3e, f8cabdbb, 494ad6fd, >>>>> 7c05abfb, 5902c851, c406028e, a0b57685, e213b903, c85a0b46, df981c08, >>>>> 187cd54f, f0b7b298, 94ec7576, 041975f5, aecba5d0, 5549256d, f9b5a77a, >>>>> 596d0df7, 26266d8c, 0e664e25, 97d112b2, aac08043, 6b81a2b1, 5a2a1012, >>>>> 534ac94b, b34cb942, 837785eb, 966d70b2, 3aab732e, 4e34ad89, 6df0ffff, >>>>> 4c7c3c47, 85eea5fe, 1c5e2f6b, 3f426f4e, 27a9bef9, cd874e96, dc3256a7, >>>>> 4da50521, 1d370c9e, 19c334eb, 24be15dd, 6c922af3, 01ea2812], >>>>> remaining=[0ab29a08, 5216f6ba, f882885f, 0d232f1a, b74f5ebb, 5790761a, >>>>> 55d2082e, b1bf93b3, 2fd79f9f, a899ccce, 3dd74aba, 320d05fd, 0d44a4b3, >>>>> 9a00f235, 4426467e, 7837fdfc, e8778da0, 4a988e3e, f8cabdbb, 494ad6fd, >>>>> 7c05abfb, 5902c851, c406028e, a0b57685, e213b903, c85a0b46, df981c08, >>>>> 187cd54f, f0b7b298, 94ec7576, 041975f5, aecba5d0, 5549256d, f9b5a77a, >>>>> 596d0df7, 26266d8c, 0e664e25, 97d112b2, aac08043, 6b81a2b1, 5a2a1012, >>>>> 534ac94b, b34cb942, 837785eb, 966d70b2, 3aab732e, 4e34ad89, 6df0ffff, >>>>> 4c7c3c47, 85eea5fe, 1c5e2f6b, 3f426f4e, 27a9bef9, cd874e96, dc3256a7, >>>>> 4da50521, 1d370c9e, 19c334eb, 24be15dd, 6c922af3, 01ea2812], init=true, >>>>> initFut=true, ready=true, replied=false, added=true, oldest=0d44a4b3, >>>>> oldestOrder=1, evtLatch=0, locNodeOrder=62, >>>>> locNodeId=fd9620f5-3ebb-4a71-a482-73d6a81b1688] >>>>> >>>>> >>>>> [14:38:41,893][WARN >>>>> ][ignite-#95%sys-null%][GridDhtPartitionsExchangeFuture] Retrying preload >>>>> partition exchange due to timeout [done=false, dummy=false, >>>>> exchId=GridDhtPartitionExchangeId [topVer=AffinityTopologyVersion >>>>> [topVer=25, minorTopVer=0], nodeId=df981c08, evt=NODE_JOINED], >>>>> rcvdIds=[7c05abfb, b34cb942, e213b903, 320d05fd, 5902c851, f0b7b298, >>>>> 1d370c9e, 0d232f1a, 494ad6fd, 5a2a1012, b1bf93b3, 55d2082e, 7837fdfc, >>>>> 85eea5fe, 4e34ad89, 5790761a, 3f426f4e, aac08043, 187cd54f, 01ea2812, >>>>> c406028e, 24be15dd, 966d70b2], rmtIds=[0d232f1a, 5790761a, 55d2082e, >>>>> b1bf93b3, aac08043, 5a2a1012, b34cb942, 320d05fd, 966d70b2, 4e34ad89, >>>>> 85eea5fe, 7837fdfc, 3f426f4e, 1d370c9e, 494ad6fd, 7c05abfb, 5902c851, >>>>> c406028e, 24be15dd, e213b903, df981c08, 187cd54f, f0b7b298, 01ea2812], >>>>> remaining=[df981c08], init=true, initFut=true, ready=true, replied=false, >>>>> added=true, oldest=0d44a4b3, oldestOrder=1, evtLatch=0, locNodeOrder=1, >>>>> locNodeId=0d44a4b3-4d10-4f67-b8bd-005be226b1df] >>>>> >>>>> >>>>> I also see a little over half the nodes getting "Still waiting for >>>>> initial partition map exchange" warnings like this >>>>> >>>>> >>>>> [14:39:37,848][WARN ][main][GridCachePartitionExchangeManager] Still >>>>> waiting for initial partition map exchange >>>>> [fut=GridDhtPartitionsExchangeFuture [dummy=false, forcePreload=false, >>>>> reassign=false, discoEvt=DiscoveryEvent [evtNode=TcpDiscoveryNode >>>>> [id=27a9bef9-de04-486d-aac0-bfa749e9007d, addrs=[0:0:0:0:0:0:0:1%1, >>>>> 10.148.0.87, 10.159.1.182, 127.0.0.1], sockAddrs=[ >>>>> r1i4n10.redacted.com/10.148.0.87:47500, /0:0:0:0:0:0:0:1%1:47500, / >>>>> 10.159.1.182:47500, /10.148.0.87:47500, /10.159.1.182:47500, / >>>>> 127.0.0.1:47500], discPort=47500, order=48, intOrder=48, >>>>> lastExchangeTime=1445974777828, loc=true, >>>>> ver=1.4.0#19691231-sha1:00000000, >>>>> isClient=false], topVer=48, nodeId8=27a9bef9, msg=null, type=NODE_JOINED, >>>>> tstamp=1445974647187], rcvdIds=GridConcurrentHashSet [elements=[]], >>>>> rmtIds=[0ab29a08-9c95-4054-8035-225f5828b3d4, >>>>> 0d232f1a-0f46-4798-a39a-63a17dc4dc7f, >>>>> f9b5a77a-a4c1-46aa-872e-aeaca9b76ee3, >>>>> 596d0df7-3edf-4078-8f4a-ffa3d96296c6, >>>>> 5790761a-aeeb-44d1-9fce-3fee31ef39b7, >>>>> 55d2082e-3517-4828-8d47-57b4ed5a41bc, >>>>> 26266d8c-cc87-4472-9fa4-c526d6da2233, >>>>> 0e664e25-8dde-4df8-966b-53b60f9a1087, >>>>> b1bf93b3-24bb-4520-ade0-31d05a93558d, >>>>> aac08043-875a-485a-ab2c-cd7e66d68f8f, >>>>> 2fd79f9f-9590-41d2-962e-004a3d7690b5, >>>>> 5a2a1012-0766-448c-9583-25873c305de9, >>>>> 534ac94b-8dd1-4fa8-a481-539fa4f4ce55, >>>>> b34cb942-e960-4a00-b4fb-10add6466a93, >>>>> 320d05fd-e021-40ac-83bc-62f54756771b, >>>>> 0d44a4b3-4d10-4f67-b8bd-005be226b1df, >>>>> 837785eb-24e0-496a-a0cc-f795b64b5929, >>>>> 9a00f235-3b6a-4be5-b0e3-93cd1beacaf4, >>>>> 966d70b2-e1dc-4e20-9876-b63736545abd, >>>>> 3aab732e-a075-4b19-9525-e97a1260a4fe, >>>>> 4e34ad89-fa46-4503-a599-b8c937ca1f47, >>>>> 4c7c3c47-6e5c-4c15-80a9-408192596bc2, >>>>> 85eea5fe-9aff-4821-970c-4ce006ee853a, >>>>> 7837fdfc-6255-4784-8088-09d4e6e37bb9, >>>>> 3f426f4e-2d0c-402a-a4af-9d7656f46484, >>>>> e8778da0-a764-4ad9-afba-8a748564e12a, >>>>> 4a988e3e-3434-4271-acd6-af2a1e30524c, >>>>> cd874e96-63cf-41c9-8e8a-75f3223bfe9d, >>>>> f8cabdbb-875a-480b-8b5e-4b5313c5fcbd, >>>>> dc3256a7-ae23-4c2e-b375-55e2884e045d, >>>>> 4da50521-aad0-48a4-9f79-858bbc2e6b89, >>>>> 1d370c9e-250f-4733-8b8a-7b6f5c6e1b2b, >>>>> 494ad6fd-1637-44b8-8d3a-1fa19681ba64, >>>>> 7c05abfb-dba1-43c3-a8b1-af504762ec60, >>>>> 5902c851-5275-41fd-89c4-cd6390c88670, >>>>> 19c334eb-5661-4697-879d-1082571dfef8, >>>>> c406028e-768e-404e-8417-40d2960c4ba3, >>>>> a0b57685-e5dc-498c-99a4-33b1aef32632, >>>>> 24be15dd-45f7-4980-b4f8-3176ab67e8f6, >>>>> e213b903-107b-4465-8fe1-78b7b393d631, >>>>> df981c08-148d-4266-9ea7-163168012968, >>>>> 187cd54f-396b-4c3c-9bfc-9883ac37f556, >>>>> f0b7b298-6432-477a-85a0-83e29e8c5380, >>>>> 94ec7576-7a02-4c08-8739-4e0fc52a3d3a, >>>>> 041975f5-990a-4792-b384-eded32966783, >>>>> 01ea2812-5005-4152-af2e-2586bf65b4c6, >>>>> aecba5d0-9d9b-4ab6-9018-62f5abb7b809], exchId=GridDhtPartitionExchangeId >>>>> [topVer=AffinityTopologyVersion [topVer=48, minorTopVer=0], >>>>> nodeId=27a9bef9, evt=NODE_JOINED], init=true, ready=true, replied=false, >>>>> added=true, initFut=GridFutureAdapter [resFlag=2, res=true, >>>>> startTime=1445974657836, endTime=1445974658400, ignoreInterrupts=false, >>>>> lsnr=null, state=DONE], topSnapshot=null, lastVer=null, >>>>> partReleaseFut=GridCompoundFuture [lsnrCalls=3, finished=true, rdc=null, >>>>> init=true, >>>>> res=java.util.concurrent.atomic.AtomicMarkableReference@6b58be0e, >>>>> err=null, done=true, cancelled=false, err=null, futs=[true, true, true]], >>>>> skipPreload=false, clientOnlyExchange=false, >>>>> oldest=0d44a4b3-4d10-4f67-b8bd-005be226b1df, oldestOrder=1, evtLatch=0, >>>>> remaining=[0ab29a08-9c95-4054-8035-225f5828b3d4, >>>>> 0d232f1a-0f46-4798-a39a-63a17dc4dc7f, >>>>> f9b5a77a-a4c1-46aa-872e-aeaca9b76ee3, >>>>> 596d0df7-3edf-4078-8f4a-ffa3d96296c6, >>>>> 5790761a-aeeb-44d1-9fce-3fee31ef39b7, >>>>> 55d2082e-3517-4828-8d47-57b4ed5a41bc, >>>>> 26266d8c-cc87-4472-9fa4-c526d6da2233, >>>>> 0e664e25-8dde-4df8-966b-53b60f9a1087, >>>>> b1bf93b3-24bb-4520-ade0-31d05a93558d, >>>>> aac08043-875a-485a-ab2c-cd7e66d68f8f, >>>>> 2fd79f9f-9590-41d2-962e-004a3d7690b5, >>>>> 5a2a1012-0766-448c-9583-25873c305de9, >>>>> 534ac94b-8dd1-4fa8-a481-539fa4f4ce55, >>>>> b34cb942-e960-4a00-b4fb-10add6466a93, >>>>> 320d05fd-e021-40ac-83bc-62f54756771b, >>>>> 0d44a4b3-4d10-4f67-b8bd-005be226b1df, >>>>> 837785eb-24e0-496a-a0cc-f795b64b5929, >>>>> 9a00f235-3b6a-4be5-b0e3-93cd1beacaf4, >>>>> 966d70b2-e1dc-4e20-9876-b63736545abd, >>>>> 3aab732e-a075-4b19-9525-e97a1260a4fe, >>>>> 4e34ad89-fa46-4503-a599-b8c937ca1f47, >>>>> 4c7c3c47-6e5c-4c15-80a9-408192596bc2, >>>>> 85eea5fe-9aff-4821-970c-4ce006ee853a, >>>>> 7837fdfc-6255-4784-8088-09d4e6e37bb9, >>>>> 3f426f4e-2d0c-402a-a4af-9d7656f46484, >>>>> e8778da0-a764-4ad9-afba-8a748564e12a, >>>>> 4a988e3e-3434-4271-acd6-af2a1e30524c, >>>>> cd874e96-63cf-41c9-8e8a-75f3223bfe9d, >>>>> f8cabdbb-875a-480b-8b5e-4b5313c5fcbd, >>>>> dc3256a7-ae23-4c2e-b375-55e2884e045d, >>>>> 4da50521-aad0-48a4-9f79-858bbc2e6b89, >>>>> 1d370c9e-250f-4733-8b8a-7b6f5c6e1b2b, >>>>> 494ad6fd-1637-44b8-8d3a-1fa19681ba64, >>>>> 7c05abfb-dba1-43c3-a8b1-af504762ec60, >>>>> 5902c851-5275-41fd-89c4-cd6390c88670, >>>>> 19c334eb-5661-4697-879d-1082571dfef8, >>>>> c406028e-768e-404e-8417-40d2960c4ba3, >>>>> a0b57685-e5dc-498c-99a4-33b1aef32632, >>>>> 24be15dd-45f7-4980-b4f8-3176ab67e8f6, >>>>> e213b903-107b-4465-8fe1-78b7b393d631, >>>>> df981c08-148d-4266-9ea7-163168012968, >>>>> 187cd54f-396b-4c3c-9bfc-9883ac37f556, >>>>> f0b7b298-6432-477a-85a0-83e29e8c5380, >>>>> 94ec7576-7a02-4c08-8739-4e0fc52a3d3a, >>>>> 041975f5-990a-4792-b384-eded32966783, >>>>> 01ea2812-5005-4152-af2e-2586bf65b4c6, >>>>> aecba5d0-9d9b-4ab6-9018-62f5abb7b809], super=GridFutureAdapter [resFlag=0, >>>>> res=null, startTime=1445974657836, endTime=0, ignoreInterrupts=false, >>>>> lsnr=null, state=INIT]]] >>>>> >>>>> >>>>> >>>>> Then on the occasions when mapreduce jobs fail I will see one node >>>>> with (it isn't always the same node) >>>>> >>>>> >>>>> [14:52:57,080][WARN >>>>> ][grid-nio-worker-3-#135%null%][TcpCommunicationSpi] Closing NIO session >>>>> because of unhandled exception [cls=class >>>>> o.a.i.i.util.nio.GridNioException, msg=Connection timed out] >>>>> [14:52:59,123][WARN >>>>> ][grid-nio-worker-3-#135%null%][TcpCommunicationSpi] Failed to process >>>>> selector key (will close): GridSelectorNioSessionImpl [selectorIdx=3, >>>>> queueSize=0, writeBuf=java.nio.DirectByteBuffer[pos=0 lim=32768 >>>>> cap=32768], >>>>> readBuf=java.nio.DirectByteBuffer[pos=0 lim=32768 cap=32768], >>>>> recovery=GridNioRecoveryDescriptor [acked=3, resendCnt=0, rcvCnt=0, >>>>> reserved=true, lastAck=0, nodeLeft=false, node=TcpDiscoveryNode >>>>> [id=837785eb-24e0-496a-a0cc-f795b64b5929, addrs=[0:0:0:0:0:0:0:1%1, >>>>> 10.148.0.81, 10.159.1.176, 127.0.0.1], sockAddrs=[/10.159.1.176:47500, >>>>> /0:0:0:0:0:0:0:1%1:47500, r1i4n4.redacted.com/10.148.0.81:47500, / >>>>> 10.148.0.81:47500, /10.159.1.176:47500, /127.0.0.1:47500], >>>>> discPort=47500, order=45, intOrder=45, lastExchangeTime=1445974625750, >>>>> loc=false, ver=1.4.0#19691231-sha1:00000000, isClient=false], >>>>> connected=true, connectCnt=1, queueLimit=5120], super=GridNioSessionImpl >>>>> [locAddr=/10.159.1.112:46222, rmtAddr=/10.159.1.176:47100, >>>>> createTime=1445974646591, closeTime=0, bytesSent=30217, bytesRcvd=9, >>>>> sndSchedTime=1445975577912, lastSndTime=1445975577912, >>>>> lastRcvTime=1445974655114, readsPaused=false, >>>>> filterChain=FilterChain[filters=[GridNioCodecFilter >>>>> [parser=o.a.i.i.util.nio.GridDirectParser@44de55ba, directMode=true], >>>>> GridConnectionBytesVerifyFilter], accepted=false]] >>>>> [14:52:59,124][WARN >>>>> ][grid-nio-worker-3-#135%null%][TcpCommunicationSpi] Closing NIO session >>>>> because of unhandled exception [cls=class >>>>> o.a.i.i.util.nio.GridNioException, msg=Connection timed out] >>>>> [14:53:00,105][WARN >>>>> ][grid-nio-worker-3-#135%null%][TcpCommunicationSpi] Failed to process >>>>> selector key (will close): GridSelectorNioSessionImpl [selectorIdx=3, >>>>> queueSize=0, writeBuf=java.nio.DirectByteBuffer[pos=0 lim=32768 >>>>> cap=32768], >>>>> readBuf=java.nio.DirectByteBuffer[pos=0 lim=32768 cap=32768], >>>>> recovery=GridNioRecoveryDescriptor [acked=0, resendCnt=0, rcvCnt=0, >>>>> reserved=true, lastAck=0, nodeLeft=false, node=TcpDiscoveryNode >>>>> [id=4426467e-b4b4-4912-baa1-d7cc839d9188, addrs=[0:0:0:0:0:0:0:1%1, >>>>> 10.148.0.106, 10.159.1.201, 127.0.0.1], sockAddrs=[/10.159.1.201:47500, >>>>> /0:0:0:0:0:0:0:1%1:47500, r1i5n11.redacted.com/10.148.0.106:47500, / >>>>> 10.148.0.106:47500, /10.159.1.201:47500, /127.0.0.1:47500], >>>>> discPort=47500, order=57, intOrder=57, lastExchangeTime=1445974625790, >>>>> loc=false, ver=1.4.0#19691231-sha1:00000000, isClient=false], >>>>> connected=true, connectCnt=1, queueLimit=5120], super=GridNioSessionImpl >>>>> [locAddr=/10.159.1.112:60869, rmtAddr=/10.159.1.201:47100, >>>>> createTime=1445974654478, closeTime=0, bytesSent=22979, bytesRcvd=0, >>>>> sndSchedTime=1445975577912, lastSndTime=1445975577912, >>>>> lastRcvTime=1445974654478, readsPaused=false, >>>>> filterChain=FilterChain[filters=[GridNioCodecFilter >>>>> [parser=o.a.i.i.util.nio.GridDirectParser@44de55ba, directMode=true], >>>>> GridConnectionBytesVerifyFilter], accepted=false]] >>>>> [14:53:00,105][WARN >>>>> ][grid-nio-worker-3-#135%null%][TcpCommunicationSpi] Closing NIO session >>>>> because of unhandled exception [cls=class >>>>> o.a.i.i.util.nio.GridNioException, msg=Connection timed out] >>>>> >>>>> >>>>> I've tried adjusting the timeout settings further but haven't had much >>>>> success. >>>>> >>>>> Here is what my config looks like, it is obviously heavily based off >>>>> the hadoop example config. >>>>> >>>>> >>>>> <?xml version="1.0" encoding="UTF-8"?> >>>>> <beans ns1:schemaLocation="http://www.springframework.org/schema/beans >>>>> http://www.springframework.org/schema/beans/spring-beans.xsd >>>>> http://www.springframework.org/schema/util >>>>> http://www.springframework.org/schema/util/spring-util.xsd" xmlns=" >>>>> http://www.springframework.org/schema/beans" xmlns:ns1=" >>>>> http://www.w3.org/2001/XMLSchema-instance"> >>>>> <description> >>>>> Spring file for Ignite node configuration with IGFS and Apache >>>>> Hadoop map-reduce support enabled. >>>>> Ignite node will start with this configuration by default. >>>>> </description> >>>>> <bean >>>>> class="org.springframework.beans.factory.config.PropertyPlaceholderConfigurer" >>>>> id="propertyConfigurer"> >>>>> <property name="systemPropertiesModeName" >>>>> value="SYSTEM_PROPERTIES_MODE_FALLBACK" /> >>>>> <property name="searchSystemEnvironment" value="true" /> >>>>> </bean> >>>>> <bean abstract="true" >>>>> class="org.apache.ignite.configuration.FileSystemConfiguration" >>>>> id="igfsCfgBase"> >>>>> <property name="blockSize" value="#{128 * 1024}" /> >>>>> <property name="perNodeBatchSize" value="512" /> >>>>> <property name="perNodeParallelBatchCount" value="16" /> >>>>> <property name="prefetchBlocks" value="32" /> >>>>> </bean> >>>>> <bean abstract="true" >>>>> class="org.apache.ignite.configuration.CacheConfiguration" >>>>> id="dataCacheCfgBase"> >>>>> <property name="cacheMode" value="PARTITIONED" /> >>>>> <property name="atomicityMode" value="TRANSACTIONAL" /> >>>>> <property name="writeSynchronizationMode" value="FULL_SYNC" /> >>>>> <property name="backups" value="0" /> >>>>> <property name="affinityMapper"> >>>>> <bean class="org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper"> >>>>> <constructor-arg value="512" /> >>>>> </bean> >>>>> </property> >>>>> <property name="startSize" value="#{100*1024*1024}" /> >>>>> <property name="offHeapMaxMemory" value="0" /> >>>>> </bean> >>>>> <bean abstract="true" >>>>> class="org.apache.ignite.configuration.CacheConfiguration" >>>>> id="metaCacheCfgBase"> >>>>> <property name="cacheMode" value="REPLICATED" /> >>>>> <property name="atomicityMode" value="TRANSACTIONAL" /> >>>>> <property name="writeSynchronizationMode" value="FULL_SYNC" /> >>>>> </bean> >>>>> <bean class="org.apache.ignite.configuration.IgniteConfiguration" >>>>> id="grid.cfg"> >>>>> <property name="failureDetectionTimeout" value="3000" /> >>>>> <property name="hadoopConfiguration"> >>>>> <bean class="org.apache.ignite.configuration.HadoopConfiguration"> >>>>> <property name="finishedJobInfoTtl" value="30000" /> >>>>> </bean> >>>>> </property> >>>>> <property name="connectorConfiguration"> >>>>> <bean >>>>> class="org.apache.ignite.configuration.ConnectorConfiguration"> >>>>> <property name="port" value="11211" /> >>>>> </bean> >>>>> </property> >>>>> <property name="fileSystemConfiguration"> >>>>> <list> >>>>> <bean >>>>> class="org.apache.ignite.configuration.FileSystemConfiguration" >>>>> parent="igfsCfgBase"> >>>>> <property name="name" value="igfs" /> >>>>> <property name="metaCacheName" value="igfs-meta" /> >>>>> <property name="dataCacheName" value="igfs-data" /> >>>>> <property name="ipcEndpointConfiguration"> >>>>> <bean >>>>> class="org.apache.ignite.igfs.IgfsIpcEndpointConfiguration"> >>>>> <property name="type" value="TCP" /> >>>>> <property name="host" value="r1i0n12" /> >>>>> <property name="port" value="10500" /> >>>>> </bean> >>>>> </property> >>>>> </bean> >>>>> </list> >>>>> </property> >>>>> <property name="cacheConfiguration"> >>>>> <list> >>>>> <bean class="org.apache.ignite.configuration.CacheConfiguration" >>>>> parent="metaCacheCfgBase"> >>>>> <property name="name" value="igfs-meta" /> >>>>> </bean> >>>>> <bean class="org.apache.ignite.configuration.CacheConfiguration" >>>>> parent="dataCacheCfgBase"> >>>>> <property name="name" value="igfs-data" /> >>>>> </bean> >>>>> </list> >>>>> </property> >>>>> <property name="includeEventTypes"> >>>>> <list> >>>>> <ns2:constant >>>>> static-field="org.apache.ignite.events.EventType.EVT_TASK_FAILED" >>>>> xmlns:ns2="http://www.springframework.org/schema/util" /> >>>>> <ns2:constant >>>>> static-field="org.apache.ignite.events.EventType.EVT_TASK_FINISHED" >>>>> xmlns:ns2="http://www.springframework.org/schema/util" /> >>>>> <ns2:constant >>>>> static-field="org.apache.ignite.events.EventType.EVT_JOB_MAPPED" >>>>> xmlns:ns2=" >>>>> http://www.springframework.org/schema/util" /> >>>>> </list> >>>>> </property> >>>>> <property name="discoverySpi"> >>>>> <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi"> >>>>> <property name="ipFinder"> >>>>> <bean >>>>> class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder"> >>>>> <property name="addresses"> >>>>> <list> >>>>> <value>r1i0n12:47500</value> >>>>> </list> >>>>> </property> >>>>> </bean> >>>>> </property> >>>>> </bean> >>>>> </property> >>>>> </bean> >>>>> </beans> >>>>> >>>>> >>>>> >>>>> Quoting Denis Magda <dma...@gridgain.com>: >>>>> >>>>> Hi Joe, >>>>>> >>>>>> Great! >>>>>> >>>>>> Please see below >>>>>> >>>>>> On 10/27/2015 9:37 AM, d...@eiler.net wrote: >>>>>> >>>>>>> Reducing the port range (to a single port) and lowering the >>>>>>> IgniteConfiguration.setFailureDetectionTimeout to 1000 helped speed up >>>>>>> everybody joining the topology and I was able to get a pi estimator run >>>>>>> on >>>>>>> 64 nodes. >>>>>>> >>>>>>> >>>>>> I suspect that the reason was in the number of ports specified in the >>>>>> range. By some reason it takes significant time to get a response from >>>>>> TCP/IP stack that a connection can't be established on a particular port >>>>>> number. >>>>>> Please try to reduce the port range, lower >>>>>> TcpDiscoverySpi.setNetworkTimeout, keep >>>>>> IgniteConfiguration.setFailureDetectionTimeout's default value and share >>>>>> results with us. >>>>>> >>>>>> Thanks again for the help, I'm over the current hurdle. >>>>>>> Joe >>>>>>> >>>>>>> >>>>>>> Quoting d...@eiler.net: >>>>>>> >>>>>>> Thanks for the quick response Denis. >>>>>>>> >>>>>>>> I did a port range of 10 ports. I'll take a look at the >>>>>>>> failureDetectionTimeout and networkTimeout. >>>>>>>> >>>>>>>> Side question: Is there an easy way to map between the programmatic >>>>>>>> API and the spring XML properties? For instance I was trying to find >>>>>>>> the >>>>>>>> correct xml incantation for >>>>>>>> TcpDiscoverySpi.setMaxMissedHeartbeats(int) and >>>>>>>> I might have a similar issue finding >>>>>>>> IgniteConfiguration.setFailureDetectionTimeout(long). It seems like I >>>>>>>> can >>>>>>>> usually drop the set and adjust capitalization (setFooBar() == >>>>>>>> <property >>>>>>>> name="fooBar") >>>>>>>> >>>>>>>> Yes, your understanding is correct. >>>>>> >>>>>>> Please pardon my ignorance on terminology: >>>>>>>> Are the nodes I run ignite.sh on considered server nodes or cluster >>>>>>>> nodes (I would have thought they are the same) >>>>>>>> >>>>>>>> Actually we have a notion of server and client nodes. This page >>>>>> contains extensive information on the type of nodes: >>>>>> https://apacheignite.readme.io/docs/clients-vs-servers >>>>>> >>>>>> A cluster node is just a server or client node. >>>>>> >>>>>> Regards, >>>>>> Denis >>>>>> >>>>>>> Thanks, >>>>>>>> Joe >>>>>>>> >>>>>>>> Quoting Denis Magda <dma...@gridgain.com>: >>>>>>>> >>>>>>>> Hi Joe, >>>>>>>>> >>>>>>>>> How big is a port range, that you specified in your discovery >>>>>>>>> configuration, for a every single node? >>>>>>>>> Please take into account that the discovery may iterate over every >>>>>>>>> port from the range before one node connects to the other and >>>>>>>>> depending on >>>>>>>>> the TCP related settings of your network it may take significant time >>>>>>>>> before the cluster is assembled. >>>>>>>>> >>>>>>>>> Here I would recommend you to reduce the port range as much as >>>>>>>>> possible and to play with the following network related parameters: >>>>>>>>> - Try to use the failure detection timeout instead of setting >>>>>>>>> socket, ack and many other timeouts explicitly ( >>>>>>>>> https://apacheignite.readme.io/docs/cluster-config#failure-detection-timeout); >>>>>>>>> - Try to play with TcpDiscoverySpi.networkTimeout because this >>>>>>>>> timeout is >>>>>>>>> considered during the time when a cluster node tries to join a >>>>>>>>> cluster. >>>>>>>>> >>>>>>>>> In order to help you with the hanging compute tasks and to give >>>>>>>>> you more specific recommendations regarding the slow join process >>>>>>>>> please >>>>>>>>> provide us with the following: >>>>>>>>> - config files for server and cluster nodes; >>>>>>>>> - log files from all the nodes. Please start the nodes with >>>>>>>>> -DIGNITE_QUIET=false virtual machine property. If you start the nodes >>>>>>>>> using >>>>>>>>> ignite.sh/bat then just pass '-v' as an argument to the script. >>>>>>>>> - thread dumps for the nodes that are hanging waiting for the >>>>>>>>> compute tasks to be completed. >>>>>>>>> >>>>>>>>> Regards, >>>>>>>>> Denis >>>>>>>>> >>>>>>>>> On 10/26/2015 6:56 AM, d...@eiler.net wrote: >>>>>>>>> >>>>>>>>>> Hi all, >>>>>>>>>> >>>>>>>>>> I have been experimenting with ignite and have run into a problem >>>>>>>>>> scaling up to larger clusters. >>>>>>>>>> >>>>>>>>>> I am playing with only two different use cases, 1) a Hadoop >>>>>>>>>> MapReduce accelerator 2) an in memory data grid (no secondary file >>>>>>>>>> system) >>>>>>>>>> being accessed by frameworks using the HDFS >>>>>>>>>> >>>>>>>>>> Everything works fine with a smaller cluster (8 nodes) but with a >>>>>>>>>> larger cluster (64 nodes) it takes a couple of minutes for all the >>>>>>>>>> nodes to >>>>>>>>>> register with the cluster(which would be ok) and mapreduce jobs just >>>>>>>>>> hang >>>>>>>>>> and never return. >>>>>>>>>> >>>>>>>>>> I've compiled the latest Ignite 1.4 (with ignite.edition=hadoop) >>>>>>>>>> from source, and am using it with Hadoop 2.7.1 just trying to run >>>>>>>>>> things >>>>>>>>>> like the pi estimator and wordcount examples. >>>>>>>>>> >>>>>>>>>> I started with the config/hadoop/default-config.xml >>>>>>>>>> >>>>>>>>>> I can't use multicast so I've configured it to use static IP >>>>>>>>>> based discovery with just a single node/port range. >>>>>>>>>> >>>>>>>>>> I've increased the heartbeat frequency to 10000 and that seemed >>>>>>>>>> to help make things more stable once all the nodes do join the >>>>>>>>>> cluster. >>>>>>>>>> I've also played with increasing both the socket timeout and the ack >>>>>>>>>> timeout but that seemed to just make it take longer for nodes to >>>>>>>>>> attempt to >>>>>>>>>> join the cluster after a failed attempt. >>>>>>>>>> >>>>>>>>>> I have access to a couple of different clusters, we allocate >>>>>>>>>> resources with slurm so I get a piece of a cluster to play with >>>>>>>>>> (hence the >>>>>>>>>> no-multicast restriction). The nodes all have fast networks (FDR >>>>>>>>>> InfiniBand) and a decent amount of memory (64GB-128GB) but no local >>>>>>>>>> storage >>>>>>>>>> (or swap space). >>>>>>>>>> >>>>>>>>>> As mentioned earlier, I disable the secondaryFilesystem. >>>>>>>>>> >>>>>>>>>> Any advice/hints/example xml configs would be extremely welcome. >>>>>>>>>> >>>>>>>>>> >>>>>>>>>> I also haven't been seeing the expected performance using the >>>>>>>>>> hdfs api to access ignite. I've tried both using the hdfs cli to do >>>>>>>>>> some >>>>>>>>>> simple timings of put/get and a little java program that writes then >>>>>>>>>> reads >>>>>>>>>> a file. Even with small files (500MB) that should be kept >>>>>>>>>> completely in a >>>>>>>>>> single node, I only see about 250MB/s for writes and reads are much >>>>>>>>>> slower >>>>>>>>>> than that (4x to 10x). The writes are better than hdfs (our hdfs is >>>>>>>>>> backed >>>>>>>>>> with pretty poor storage) but reads are much slower. Now I haven't >>>>>>>>>> tried >>>>>>>>>> scaling this at all but with an 8 node ignite cluster and a single >>>>>>>>>> "client" >>>>>>>>>> access a single file I would hope for something closer to memory >>>>>>>>>> speeds. >>>>>>>>>> (if you would like me to split this into another message to the list >>>>>>>>>> just >>>>>>>>>> let me know, I'm assuming the cause it the same---I missed a required >>>>>>>>>> config setting ;-) ) >>>>>>>>>> >>>>>>>>>> Thanks in advance for any help, >>>>>>>>>> Joe >>>>>>>>>> >>>>>>>>>> >>>>>>>>>> >>>>>>>>>> >>>>>>>>>> >>>>>>> >>>>>>> >>>>>>> >>>>> >>>>> >>>>> >>> >>> >>> > > >