You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@ignite.apache.org by fsapei <fe...@googlemail.com> on 2015/12/01 08:14:11 UTC

Could not get any response from SQL Query

I try to execute SQL Queries, but ignite does not return any response. The
error message is as follows:

[08:15:47,891][WARNING][main][GridCachePartitionExchangeManager] Still
waiting for initial partition map exchange
[fut=GridDhtPartitionsExchangeFuture [dummy=false, forcePreload=false,
reassign=false, discoEvt=DiscoveryEvent [evtNode=TcpDiscoveryNode
[id=6c272370-81e2-456b-8d90-e33628f62d73, addrs=[0:0:0:0:0:0:0:1%lo,
10.15.76.245, 127.0.0.1], sockAddrs=[jmeter/10.15.76.245:0,
/0:0:0:0:0:0:0:1%lo:0, /10.15.76.245:0, /127.0.0.1:0], discPort=0, order=9,
intOrder=0, lastExchangeTime=1448954016255, loc=true,
ver=1.4.1#20150925-sha1:f051e49f, isClient=true], topVer=9,
nodeId8=6c272370, msg=null, type=NODE_JOINED, tstamp=1448954027163],
rcvdIds=GridConcurrentHashSet [elements=[]],
rmtIds=[984b844f-90b0-44b3-b1af-40db5f3e3f41,
90c69cd2-2fd5-4fb2-a607-9805fa148ebb], exchId=GridDhtPartitionExchangeId
[topVer=AffinityTopologyVersion [topVer=9, minorTopVer=0], nodeId=6c272370,
evt=NODE_JOINED], init=true, ready=false, replied=false, added=true,
initFut=GridFutureAdapter [resFlag=2, res=true, startTime=1448954027871,
endTime=1448954028014, ignoreInterrupts=false, lsnr=null, state=DONE],
topSnapshot=null, lastVer=null, partReleaseFut=null, skipPreload=true,
clientOnlyExchange=true, oldest=984b844f-90b0-44b3-b1af-40db5f3e3f41,
oldestOrder=1, evtLatch=0, remaining=[984b844f-90b0-44b3-b1af-40db5f3e3f41,
90c69cd2-2fd5-4fb2-a607-9805fa148ebb], super=GridFutureAdapter [resFlag=0,
res=null, startTime=1448954027871, endTime=0, ignoreInterrupts=false,
lsnr=null, state=INIT]]]
[08:16:17,893][WARNING][main][GridCachePartitionExchangeManager] Still
waiting for initial partition map exchange
[fut=GridDhtPartitionsExchangeFuture [dummy=false, forcePreload=false,
reassign=false, discoEvt=DiscoveryEvent [evtNode=TcpDiscoveryNode
[id=6c272370-8

Can somebody give me a hint what I should do?




--
View this message in context: http://apache-ignite-users.70518.x6.nabble.com/Could-not-get-any-response-from-SQL-Query-tp2103.html
Sent from the Apache Ignite Users mailing list archive at Nabble.com.

Re: Could not get any response from SQL Query

Posted by Denis Magda <dm...@gridgain.com>.
Fsapei,

Seems that the cluster breaks down when you execute the queries over 40 
million rows. As a result you see partition map exchange related 
exceptions. Share the full logs using some file-sharing service.

Analyze your SQL query by paying attention to the following:
- What query do you execute? Share the query string with us;
- How big is the result set? (in number of returned entries and size). 
If the result set is big that it can take significant time to transfer 
all the data back to the client;
- What indexes are used during the query? Execute EXPLAIN statement and 
share the execution plan [1]. There can be a chance that you're doing a 
full-scan over 40 million rows;
- Do you really need the REPLICATED cache? Can't it be substituted with 
PARTITIONED?
- Increase IgniteConfiguration.networkTimeout() and 
TcpDiscoverySpi.networkTimeout() when you're sure that the SQL query is 
already optimized and you're not doing full-scans or returning big sets 
of data to the client.

In addition please get to know articles regarding best practices on 
indexes selection and performance optimizations [2].

[1] https://apacheignite.readme.io/docs/sql-queries#using-explain
[2] https://apacheignite.readme.io/docs/sql-queries#choosing-indexes

--
Denis

On 12/1/2015 12:51 PM, Ferry Syafei Sapei wrote:
> Hi Denis,
>
> my scenario is as follows:
> - start a node server with the following configuration:
>
> JVM_OPTS="$JVM_OPTS -Xms4g -Xmx4g"
> JVM_OPTS="$JVM_OPTS -XX:+UseParNewGC -XX:+UseConcMarkSweepGC 
> -XX:+UseTLAB -XX:NewSize=128m -XX:MaxNewSize=128m"
> JVM_OPTS="$JVM_OPTS -XX:MaxTenuringThreshold=0 -XX:SurvivorRatio=1024 
> -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=60"
>
>
>
>     <bean id="ignite.cfg" 
> class="org.apache.ignite.configuration.IgniteConfiguration">
>         <property name="marshaller">
>             <bean 
> class="org.apache.ignite.marshaller.optimized.OptimizedMarshaller">
>                 <!-- Set to false to allow non-serializable objects in 
> examples, default is true. -->
>                 <property name="requireSerializable" value="false"/>
>             </bean>
>         </property>
>
>
>         <property name="communicationSpi">
>             <bean 
> class="org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi">
>                 <property name="socketWriteTimeout" value="10000" />
>             </bean>
>         </property>
>
>
>         <!-- Explicitly configure TCP discovery SPI to provide list of 
> initial nodes. -->
>         <property name="discoverySpi">
>             <bean 
> class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
>                 <!-- in miliseconds -->
>                 <property name="networkTimeout" value="15000"/>
>
>                 <property name="ackTimeout" value="30000" />
>
>                 <property name="socketTimeout" value="30000"/>
>
>                 <property name="ipFinder">
>                     <bean 
> class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
> <property name="multicastGroup" value="228.10.10.157"/>
> <property name="addresses">
> <list>
> <value>218.46.208.243:47500</value>
> <value>218.46.208.244:47500</value>
> </list>
> </property>
> </bean>
> </property>
>             </bean>
>         </property>
>     </bean>
>
>
>
> - start a client and send data to the node server using ignite streaming:
>
>     @Bean(name = "ignite")
>     public Ignite ignite() {
>         Ignition.setClientMode(true);
>         return Ignition.start(igniteConfigPath);
>     }
>
>     @Bean(name = "kubisCache")
>     public IgniteCache<AffinityUuid, Kubis> kubisCache() {
>         CacheConfiguration<AffinityUuid, Kubis> 
> kubisCacheConfiguration = new CacheConfiguration<>();
>
>         // Set unlimited off-heap memory
> kubisCacheConfiguration.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
> kubisCacheConfiguration.setOffHeapMaxMemory(0);
>
> kubisCacheConfiguration.setCacheMode(CacheMode.REPLICATED);
>
> kubisCacheConfiguration.setName(Kubis.class.getSimpleName());
> kubisCacheConfiguration.setIndexedTypes(AffinityUuid.class, Kubis.class);
>
>         return ignite().getOrCreateCache(kubisCacheConfiguration);
>     }
>
>     @Bean(name = "kubisDataStreamer")
>     public IgniteDataStreamer<AffinityUuid, Kubis> kubisDataStreamer() {
>         return ignite().dataStreamer(kubisCache().getName());
>     }
>
>
> - when the data has all been streamed, I start a second node server on 
> another machine and the data is replicated by ignite automatically. 
> The configuration (JVM_OPTS and IgniteConfiguration Bean) is exactly 
> the same as the first node server.
>
>
>
> - start a JMeter test to execute SQL Query:
>
> IgniteCache<AffinityUuid, Kubis> kubisCache = IgniteCacheProvider.getInstance().getIgniteCache();
>
> // Get variables defined by CSV Data Set Config JMeterVariables jMeterVariables = JMeterContextService.getContext().getVariables();
> List<String> sqlParams =new ArrayList<>();
> for(String sqlParam : context.getParameter("sql_params").split(",")) {
>      sqlParams.add(jMeterVariables.get(sqlParam));
> }
>
> SqlQuery<AffinityUuid, Kubis> sqlQuery =
>      new SqlQuery<AffinityUuid, Kubis>(
>          Kubis.class,
>          context.getParameter("sql_query")
>      ).setArgs(sqlParams.toArray(new String[sqlParams.size()]));
>
> result.sampleStart();
>
> int totalResult = kubisCache.query(sqlQuery).getAll().size();
>
> result.sampleEnd();
>
>
> JMeter will act as a client with the following configuration:
>
> public class IgniteCacheProvider {
>      private IgniteCache<AffinityUuid, Kubis>cache;
>      private Igniteignite;
>
>
>      private IgniteCacheProvider() {
>          String igniteConfigPath = System.getProperty("ignite.config.path");
>          if (igniteConfigPath ==null) {
>              throw new RuntimeException("System property ignite.config.path not set! Set it using 
> -Dignite.config.path on running the application.");
>          }
>          ignite = ignite(igniteConfigPath);
>          cache = kubisCache(ignite);
>      }
>
>
>      private static class IgniteCacheProviderHolder {
>          private static final IgniteCacheProviderINSTANCE =new IgniteCacheProvider();
>      }
>
>      public static IgniteCacheProvider getInstance() {
>          return IgniteCacheProviderHolder.INSTANCE;
>      }
>
>
>      public IgniteCache<AffinityUuid, Kubis> getIgniteCache() {
>          return cache;
>      }
>
>      public Ignite getIgnite() {
>          return ignite;
>      }
>
>      protected Ignite ignite(String configPath) {
>          Ignition.setClientMode(true);
>          return Ignition.start(configPath);
>      }
>
>      protected IgniteCache<AffinityUuid, Kubis> kubisCache(Ignite ignite) {
>          CacheConfiguration<AffinityUuid, Kubis> kubisCacheConfiguration =new CacheConfiguration<>();
>
>          // Set unlimited off-heap memory kubisCacheConfiguration.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
>          kubisCacheConfiguration.setOffHeapMaxMemory(0);
>
>          kubisCacheConfiguration.setCacheMode(CacheMode.REPLICATED);
>
>          kubisCacheConfiguration.setName(Kubis.class.getSimpleName());
>          kubisCacheConfiguration.setIndexedTypes(AffinityUuid.class, Kubis.class);
>
>          return ignite.getOrCreateCache(kubisCacheConfiguration);
>      }
> }
>
>
> The reference to the IgniteCache will be shared between threads.
>
> I setup a parallel request with 2500 requests in JMeter.
>
>
>
> When the data on the server is only 998 entries, JMeter performs the 
> test without any problems.
>
> However, with a really big data (40491131 entries), I do not get any 
> response anymore from Ignite.
>
> Could it be that I have to allocate more resources to both ignite servers?
>
>
>
> fsapei
>
>
>
>> Am 01.12.2015 um 08:33 schrieb Denis Magda <dmagda@gridgain.com 
>> <ma...@gridgain.com>>:
>>
>> Hi,
>>
>> This kind of exception just tells that the partition exchange 
>> procedure hasn't completed yet.
>> There are many reasons for that: slow network connection, failure of 
>> a node, a bug that led to this situation.
>>
>> Please provide the following:
>> - your scenario: number of nodes, their configurations, etc.
>> - full logs from all the nodes;
>>
>> --
>> Denis
>>
>> On 12/1/2015 10:14 AM, fsapei wrote:
>>> I try to execute SQL Queries, but ignite does not return any 
>>> response. The
>>> error message is as follows:
>>>
>>> [08:15:47,891][WARNING][main][GridCachePartitionExchangeManager] Still
>>> waiting for initial partition map exchange
>>> [fut=GridDhtPartitionsExchangeFuture [dummy=false, forcePreload=false,
>>> reassign=false, discoEvt=DiscoveryEvent [evtNode=TcpDiscoveryNode
>>> [id=6c272370-81e2-456b-8d90-e33628f62d73, addrs=[0:0:0:0:0:0:0:1%lo,
>>> 10.15.76.245, 127.0.0.1], sockAddrs=[jmeter/10.15.76.245:0,
>>> /0:0:0:0:0:0:0:1%lo:0, /10.15.76.245:0, /127.0.0.1:0], discPort=0, 
>>> order=9,
>>> intOrder=0, lastExchangeTime=1448954016255, loc=true,
>>> ver=1.4.1#20150925-sha1:f051e49f, isClient=true], topVer=9,
>>> nodeId8=6c272370, msg=null, type=NODE_JOINED, tstamp=1448954027163],
>>> rcvdIds=GridConcurrentHashSet [elements=[]],
>>> rmtIds=[984b844f-90b0-44b3-b1af-40db5f3e3f41,
>>> 90c69cd2-2fd5-4fb2-a607-9805fa148ebb], exchId=GridDhtPartitionExchangeId
>>> [topVer=AffinityTopologyVersion [topVer=9, minorTopVer=0], 
>>> nodeId=6c272370,
>>> evt=NODE_JOINED], init=true, ready=false, replied=false, added=true,
>>> initFut=GridFutureAdapter [resFlag=2, res=true, startTime=1448954027871,
>>> endTime=1448954028014, ignoreInterrupts=false, lsnr=null, state=DONE],
>>> topSnapshot=null, lastVer=null, partReleaseFut=null, skipPreload=true,
>>> clientOnlyExchange=true, oldest=984b844f-90b0-44b3-b1af-40db5f3e3f41,
>>> oldestOrder=1, evtLatch=0, 
>>> remaining=[984b844f-90b0-44b3-b1af-40db5f3e3f41,
>>> 90c69cd2-2fd5-4fb2-a607-9805fa148ebb], super=GridFutureAdapter 
>>> [resFlag=0,
>>> res=null, startTime=1448954027871, endTime=0, ignoreInterrupts=false,
>>> lsnr=null, state=INIT]]]
>>> [08:16:17,893][WARNING][main][GridCachePartitionExchangeManager] Still
>>> waiting for initial partition map exchange
>>> [fut=GridDhtPartitionsExchangeFuture [dummy=false, forcePreload=false,
>>> reassign=false, discoEvt=DiscoveryEvent [evtNode=TcpDiscoveryNode
>>> [id=6c272370-8
>>>
>>> Can somebody give me a hint what I should do?
>>>
>>>
>>>
>>>
>>> --
>>> View this message in context: 
>>> http://apache-ignite-users.70518.x6.nabble.com/Could-not-get-any-response-from-SQL-Query-tp2103.html
>>> Sent from the Apache Ignite Users mailing list archive at Nabble.com 
>>> <http://nabble.com>.
>

Re: Could not get any response from SQL Query

Posted by Ferry Syafei Sapei <fe...@googlemail.com>.
Hi Denis,

my scenario is as follows:
- start a node server with the following configuration:

JVM_OPTS="$JVM_OPTS -Xms4g -Xmx4g"
JVM_OPTS="$JVM_OPTS -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+UseTLAB -XX:NewSize=128m -XX:MaxNewSize=128m"
JVM_OPTS="$JVM_OPTS -XX:MaxTenuringThreshold=0 -XX:SurvivorRatio=1024 -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=60"



    <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
        <property name="marshaller">
            <bean class="org.apache.ignite.marshaller.optimized.OptimizedMarshaller">
                <!-- Set to false to allow non-serializable objects in examples, default is true. -->
                <property name="requireSerializable" value="false"/>
            </bean>
        </property>


        <property name="communicationSpi">
            <bean class="org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi">
                <property name="socketWriteTimeout" value="10000" />
            </bean>
        </property>


        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
        <property name="discoverySpi">
            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
                <!-- in miliseconds -->
                <property name="networkTimeout" value="15000"/>

                <property name="ackTimeout" value="30000" />

                <property name="socketTimeout" value="30000"/>

                <property name="ipFinder">
                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
                        <property name="multicastGroup" value="228.10.10.157"/>
                        <property name="addresses">
                            <list>
                                <value>218.46.208.243:47500</value>
                                <value>218.46.208.244:47500</value>
                            </list>
                        </property>
                    </bean>
                </property>
            </bean>
        </property>
    </bean>



- start a client and send data to the node server using ignite streaming:

    @Bean(name = "ignite")
    public Ignite ignite() {
        Ignition.setClientMode(true);
        return Ignition.start(igniteConfigPath);
    }

    @Bean(name = "kubisCache")
    public IgniteCache<AffinityUuid, Kubis> kubisCache() {
        CacheConfiguration<AffinityUuid, Kubis> kubisCacheConfiguration = new CacheConfiguration<>();

        // Set unlimited off-heap memory
        kubisCacheConfiguration.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
        kubisCacheConfiguration.setOffHeapMaxMemory(0);

        kubisCacheConfiguration.setCacheMode(CacheMode.REPLICATED);

        kubisCacheConfiguration.setName(Kubis.class.getSimpleName());
        kubisCacheConfiguration.setIndexedTypes(AffinityUuid.class, Kubis.class);

        return ignite().getOrCreateCache(kubisCacheConfiguration);
    }

    @Bean(name = "kubisDataStreamer")
    public IgniteDataStreamer<AffinityUuid, Kubis> kubisDataStreamer() {
        return ignite().dataStreamer(kubisCache().getName());
    } 


- when the data has all been streamed, I start a second node server on another machine and the data is replicated by ignite automatically. The configuration (JVM_OPTS and IgniteConfiguration Bean) is exactly the same as the first node server.



- start a JMeter test to execute SQL Query:

IgniteCache<AffinityUuid, Kubis> kubisCache = IgniteCacheProvider.getInstance().getIgniteCache();

// Get variables defined by CSV Data Set Config
JMeterVariables jMeterVariables = JMeterContextService.getContext().getVariables();
List<String> sqlParams = new ArrayList<>();
for(String sqlParam : context.getParameter("sql_params").split(",")) {
    sqlParams.add(jMeterVariables.get(sqlParam));
}

SqlQuery<AffinityUuid, Kubis> sqlQuery =
    new SqlQuery<AffinityUuid, Kubis>(
        Kubis.class,
        context.getParameter("sql_query")
    ).setArgs(sqlParams.toArray(new String[sqlParams.size()]));

result.sampleStart();

int totalResult = kubisCache.query(sqlQuery).getAll().size();

result.sampleEnd();


JMeter will act as a client with the following configuration:

public class IgniteCacheProvider {
    private IgniteCache<AffinityUuid, Kubis> cache;
    private Ignite ignite;


    private IgniteCacheProvider() {
        String igniteConfigPath = System.getProperty("ignite.config.path");
        if (igniteConfigPath == null) {
            throw new RuntimeException("System property ignite.config.path not set! Set it using -Dignite.config.path on running the application.");
        }
        ignite = ignite(igniteConfigPath);
        cache = kubisCache(ignite);
    }


    private static class IgniteCacheProviderHolder {
        private static final IgniteCacheProvider INSTANCE = new IgniteCacheProvider();
    }

    public static IgniteCacheProvider getInstance() {
        return IgniteCacheProviderHolder.INSTANCE;
    }


    public IgniteCache<AffinityUuid, Kubis> getIgniteCache() {
        return cache;
    }

    public Ignite getIgnite() {
        return ignite;
    }

    protected Ignite ignite(String configPath) {
        Ignition.setClientMode(true);
        return Ignition.start(configPath);
    }

    protected IgniteCache<AffinityUuid, Kubis> kubisCache(Ignite ignite) {
        CacheConfiguration<AffinityUuid, Kubis> kubisCacheConfiguration = new CacheConfiguration<>();

        // Set unlimited off-heap memory
        kubisCacheConfiguration.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
        kubisCacheConfiguration.setOffHeapMaxMemory(0);

        kubisCacheConfiguration.setCacheMode(CacheMode.REPLICATED);

        kubisCacheConfiguration.setName(Kubis.class.getSimpleName());
        kubisCacheConfiguration.setIndexedTypes(AffinityUuid.class, Kubis.class);

        return ignite.getOrCreateCache(kubisCacheConfiguration);
    }
}


The reference to the IgniteCache will be shared between threads.

I setup a parallel request with 2500 requests in JMeter.



When the data on the server is only 998 entries, JMeter performs the test without any problems.

However, with a really big data (40491131 entries), I do not get any response anymore from Ignite.

Could it be that I have to allocate more resources to both ignite servers?



fsapei



> Am 01.12.2015 um 08:33 schrieb Denis Magda <dm...@gridgain.com>:
> 
> Hi,
> 
> This kind of exception just tells that the partition exchange procedure hasn't completed yet.
> There are many reasons for that: slow network connection, failure of a node, a bug that led to this situation.
> 
> Please provide the following:
> - your scenario: number of nodes, their configurations, etc.
> - full logs from all the nodes;
> 
> --
> Denis
> 
> On 12/1/2015 10:14 AM, fsapei wrote:
>> I try to execute SQL Queries, but ignite does not return any response. The
>> error message is as follows:
>> 
>> [08:15:47,891][WARNING][main][GridCachePartitionExchangeManager] Still
>> waiting for initial partition map exchange
>> [fut=GridDhtPartitionsExchangeFuture [dummy=false, forcePreload=false,
>> reassign=false, discoEvt=DiscoveryEvent [evtNode=TcpDiscoveryNode
>> [id=6c272370-81e2-456b-8d90-e33628f62d73, addrs=[0:0:0:0:0:0:0:1%lo,
>> 10.15.76.245, 127.0.0.1], sockAddrs=[jmeter/10.15.76.245:0,
>> /0:0:0:0:0:0:0:1%lo:0, /10.15.76.245:0, /127.0.0.1:0], discPort=0, order=9,
>> intOrder=0, lastExchangeTime=1448954016255, loc=true,
>> ver=1.4.1#20150925-sha1:f051e49f, isClient=true], topVer=9,
>> nodeId8=6c272370, msg=null, type=NODE_JOINED, tstamp=1448954027163],
>> rcvdIds=GridConcurrentHashSet [elements=[]],
>> rmtIds=[984b844f-90b0-44b3-b1af-40db5f3e3f41,
>> 90c69cd2-2fd5-4fb2-a607-9805fa148ebb], exchId=GridDhtPartitionExchangeId
>> [topVer=AffinityTopologyVersion [topVer=9, minorTopVer=0], nodeId=6c272370,
>> evt=NODE_JOINED], init=true, ready=false, replied=false, added=true,
>> initFut=GridFutureAdapter [resFlag=2, res=true, startTime=1448954027871,
>> endTime=1448954028014, ignoreInterrupts=false, lsnr=null, state=DONE],
>> topSnapshot=null, lastVer=null, partReleaseFut=null, skipPreload=true,
>> clientOnlyExchange=true, oldest=984b844f-90b0-44b3-b1af-40db5f3e3f41,
>> oldestOrder=1, evtLatch=0, remaining=[984b844f-90b0-44b3-b1af-40db5f3e3f41,
>> 90c69cd2-2fd5-4fb2-a607-9805fa148ebb], super=GridFutureAdapter [resFlag=0,
>> res=null, startTime=1448954027871, endTime=0, ignoreInterrupts=false,
>> lsnr=null, state=INIT]]]
>> [08:16:17,893][WARNING][main][GridCachePartitionExchangeManager] Still
>> waiting for initial partition map exchange
>> [fut=GridDhtPartitionsExchangeFuture [dummy=false, forcePreload=false,
>> reassign=false, discoEvt=DiscoveryEvent [evtNode=TcpDiscoveryNode
>> [id=6c272370-8
>> 
>> Can somebody give me a hint what I should do?
>> 
>> 
>> 
>> 
>> --
>> View this message in context: http://apache-ignite-users.70518.x6.nabble.com/Could-not-get-any-response-from-SQL-Query-tp2103.html
>> Sent from the Apache Ignite Users mailing list archive at Nabble.com.


Re: Could not get any response from SQL Query

Posted by Denis Magda <dm...@gridgain.com>.
Hi,

This kind of exception just tells that the partition exchange procedure 
hasn't completed yet.
There are many reasons for that: slow network connection, failure of a 
node, a bug that led to this situation.

Please provide the following:
- your scenario: number of nodes, their configurations, etc.
- full logs from all the nodes;

--
Denis

On 12/1/2015 10:14 AM, fsapei wrote:
> I try to execute SQL Queries, but ignite does not return any response. The
> error message is as follows:
>
> [08:15:47,891][WARNING][main][GridCachePartitionExchangeManager] Still
> waiting for initial partition map exchange
> [fut=GridDhtPartitionsExchangeFuture [dummy=false, forcePreload=false,
> reassign=false, discoEvt=DiscoveryEvent [evtNode=TcpDiscoveryNode
> [id=6c272370-81e2-456b-8d90-e33628f62d73, addrs=[0:0:0:0:0:0:0:1%lo,
> 10.15.76.245, 127.0.0.1], sockAddrs=[jmeter/10.15.76.245:0,
> /0:0:0:0:0:0:0:1%lo:0, /10.15.76.245:0, /127.0.0.1:0], discPort=0, order=9,
> intOrder=0, lastExchangeTime=1448954016255, loc=true,
> ver=1.4.1#20150925-sha1:f051e49f, isClient=true], topVer=9,
> nodeId8=6c272370, msg=null, type=NODE_JOINED, tstamp=1448954027163],
> rcvdIds=GridConcurrentHashSet [elements=[]],
> rmtIds=[984b844f-90b0-44b3-b1af-40db5f3e3f41,
> 90c69cd2-2fd5-4fb2-a607-9805fa148ebb], exchId=GridDhtPartitionExchangeId
> [topVer=AffinityTopologyVersion [topVer=9, minorTopVer=0], nodeId=6c272370,
> evt=NODE_JOINED], init=true, ready=false, replied=false, added=true,
> initFut=GridFutureAdapter [resFlag=2, res=true, startTime=1448954027871,
> endTime=1448954028014, ignoreInterrupts=false, lsnr=null, state=DONE],
> topSnapshot=null, lastVer=null, partReleaseFut=null, skipPreload=true,
> clientOnlyExchange=true, oldest=984b844f-90b0-44b3-b1af-40db5f3e3f41,
> oldestOrder=1, evtLatch=0, remaining=[984b844f-90b0-44b3-b1af-40db5f3e3f41,
> 90c69cd2-2fd5-4fb2-a607-9805fa148ebb], super=GridFutureAdapter [resFlag=0,
> res=null, startTime=1448954027871, endTime=0, ignoreInterrupts=false,
> lsnr=null, state=INIT]]]
> [08:16:17,893][WARNING][main][GridCachePartitionExchangeManager] Still
> waiting for initial partition map exchange
> [fut=GridDhtPartitionsExchangeFuture [dummy=false, forcePreload=false,
> reassign=false, discoEvt=DiscoveryEvent [evtNode=TcpDiscoveryNode
> [id=6c272370-8
>
> Can somebody give me a hint what I should do?
>
>
>
>
> --
> View this message in context: http://apache-ignite-users.70518.x6.nabble.com/Could-not-get-any-response-from-SQL-Query-tp2103.html
> Sent from the Apache Ignite Users mailing list archive at Nabble.com.