You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by xubo245 <gi...@git.apache.org> on 2018/04/27 08:57:37 UTC

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

GitHub user xubo245 opened a pull request:

    https://github.com/apache/carbondata/pull/2239

    [CARBONDATA-2408] Fix search mode master SaslException issue in the first time

    
    Before register to master, the master maybe not finished the start service.
    
    Fix method:
       
        Waiting for search mode master thread finished and wait for request. And then start workers.
    
    
    Be sure to do all of the following checklist to help us incorporate 
    your contribution quickly and easily:
    
     - [ ] Any interfaces changed?
     
     - [ ] Any backward compatibility impacted?
     
     - [ ] Document update required?
    
     - [ ] Testing done
            Please provide details on 
            - Whether new unit test cases have been added or why no new tests are required?
            - How it is tested? Please attach test report.
            - Is it a performance related change? Please attach the performance test report.
            - Any additional information to help reviewers in testing this change.
           
     - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA. 
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/xubo245/carbondata CARBONDATA-2408-registerToMaster

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/carbondata/pull/2239.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 #2239
    
----
commit 276a0c97df69dcfa253fa9d453763979d4cb5c13
Author: xubo245 <60...@...>
Date:   2018-04-27T08:38:44Z

    [CARBONDATA-2408] Fix search mode master SaslException issue in first start
    
    Before register to master, the master maybe not finished the start service.
    
    revert the ime

----


---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r185680076
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -64,13 +65,29 @@ class Master(sparkConf: SparkConf, port: Int) {
     
       private val scheduler: Scheduler = new Scheduler
     
    +  /**
    +   * flag is the master status, default value is false, which means master not start now.
    +   * if it is true, then search mode master finished start.
    +   */
    +  private var flag: AtomicBoolean = new AtomicBoolean(false)
    --- End diff --
    
    move this variable into startService


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4708/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    @jackylk Please review it.


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    @jackylk  CI pass, please check it again.


---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xuchuanyin <gi...@git.apache.org>.
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184842350
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -64,13 +64,29 @@ class Master(sparkConf: SparkConf, port: Int) {
     
       private val scheduler: Scheduler = new Scheduler
     
    +  /**
    +   * flag is the master status, default value is false, which means master not start now.
    +   * if it is true, then search mode master finished start.
    +   */
    +  private var flag: Boolean = false
    +
       def this(sparkConf: SparkConf) = {
         this(sparkConf, CarbonProperties.getSearchMasterPort)
       }
     
    +  /**
    +   * get the status flag info
    +   *
    +   * @return whether master started
    +   */
    +  def isStarted(): Boolean = {
    +    this.flag
    +  }
    +
       /** start service and listen on port passed in constructor */
       def startService(): Unit = {
         if (rpcEnv == null) {
    +      LOG.info("Start search mode  master thread")
    --- End diff --
    
    extra blanks found in"mode  master"


---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184850068
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -64,13 +64,29 @@ class Master(sparkConf: SparkConf, port: Int) {
     
       private val scheduler: Scheduler = new Scheduler
     
    +  /**
    +   * flag is the master status, default value is false, which means master not start now.
    +   * if it is true, then search mode master finished start.
    +   */
    +  private var flag: Boolean = false
    +
       def this(sparkConf: SparkConf) = {
         this(sparkConf, CarbonProperties.getSearchMasterPort)
       }
     
    +  /**
    +   * get the status flag info
    +   *
    +   * @return whether master started
    +   */
    +  def isStarted(): Boolean = {
    +    this.flag
    +  }
    +
       /** start service and listen on port passed in constructor */
       def startService(): Unit = {
         if (rpcEnv == null) {
    +      LOG.info("Start search mode  master thread")
    --- End diff --
    
    remove it


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/5467/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/4406/



---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/carbondata/pull/2239


---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r185680045
  
    --- Diff: integration/spark2/src/main/scala/org/apache/carbondata/store/SparkCarbonStore.scala ---
    @@ -110,8 +110,16 @@ class SparkCarbonStore extends MetaCachedCarbonStore {
       }
     
       def startSearchMode(): Unit = {
    +    LOG.info("Starting search mode master")
         master = new Master(session.sparkContext.getConf)
         master.startService()
    +    var count = 0
    +    while (master.isStarted().compareAndSet(false, false) && count < 5000) {
    --- End diff --
    
    I think you can move this while loop into master.startService, so that master.startService is a blocking operation until it is success.  No need to expose master.isStarted method


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/5579/



---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184653882
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -69,20 +69,32 @@ class Master(sparkConf: SparkConf, port: Int) {
       }
     
       /** start service and listen on port passed in constructor */
    -  def startService(): Unit = {
    +  def startService(message: MasterStatus): Unit = {
    --- End diff --
    
    ok, done.


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/5595/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/4299/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/5568/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/5462/



---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184649785
  
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/SearchModeExample.scala ---
    @@ -156,7 +156,6 @@ object SearchModeExample {
             s"""
                  SELECT charField, stringField, intField, dateField
                  FROM carbonsession_table
    -             WHERE stringfield = 'spark' AND decimalField > $i % 37
    --- End diff --
    
    why delete this?


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/4293/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/5493/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4699/



---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184650673
  
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/SearchModeExample.scala ---
    @@ -156,7 +156,6 @@ object SearchModeExample {
             s"""
                  SELECT charField, stringField, intField, dateField
                  FROM carbonsession_table
    -             WHERE stringfield = 'spark' AND decimalField > $i % 37
    --- End diff --
    
    for test it, will keep it.


---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184850376
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -80,9 +96,18 @@ class Master(sparkConf: SparkConf, port: Int) {
               rpcEnv = new NettyRpcEnvFactory().create(config)
               val registryEndpoint: RpcEndpoint = new Registry(rpcEnv, Master.this)
               rpcEnv.setupEndpoint("registry-service", registryEndpoint)
    +          if (!isStarted) {
    +            synchronized {
    +              if (!isStarted) {
    +                flag = true
    +              }
    +            }
    +          }
               rpcEnv.awaitTermination()
             }
           }).start()
    +    } else {
    +      LOG.info("rpcEnv exists:" + rpcEnv.address)
    --- End diff --
    
    Search mode master has already existed before.  maybe by this search mode program, maybe by others.


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4666/



---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xuchuanyin <gi...@git.apache.org>.
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184842333
  
    --- Diff: integration/spark2/src/main/scala/org/apache/carbondata/store/SparkCarbonStore.scala ---
    @@ -110,8 +110,16 @@ class SparkCarbonStore extends MetaCachedCarbonStore {
       }
     
       def startSearchMode(): Unit = {
    +    LOG.info("Search mode master start service")
         master = new Master(session.sparkContext.getConf)
         master.startService()
    +    var count = 0
    +    while (!master.isStarted()) {
    +      Thread.sleep(10)
    --- End diff --
    
    Is 10ms a proper period?
    What if there is a situation that the master starts too late or failed and there will be a log explosion.
    
    So better to add a maximum retrying counts or time and make the sleeping time dynamic by double it.


---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r185718247
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -70,7 +71,9 @@ class Master(sparkConf: SparkConf, port: Int) {
     
       /** start service and listen on port passed in constructor */
       def startService(): Unit = {
    +    val isStarted: AtomicBoolean = new AtomicBoolean(false)
    --- End diff --
    
    move it into next line


---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r185679917
  
    --- Diff: integration/spark2/src/main/scala/org/apache/carbondata/store/SparkCarbonStore.scala ---
    @@ -110,8 +110,16 @@ class SparkCarbonStore extends MetaCachedCarbonStore {
       }
     
       def startSearchMode(): Unit = {
    +    LOG.info("Starting search mode master")
         master = new Master(session.sparkContext.getConf)
         master.startService()
    +    var count = 0
    +    while (master.isStarted().compareAndSet(false, false) && count < 5000) {
    +      LOG.info(s"Waiting search mode master to start, retrying $count times")
    +      Thread.sleep(10)
    +      count = count + 1;
    +    }
    +    LOG.info("Search mode master started")
    --- End diff --
    
    Need one more if check, if (count == 5000), should throw exception because it is failed to start master


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/4384/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/4417/



---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184849996
  
    --- Diff: integration/spark2/src/main/scala/org/apache/carbondata/store/SparkCarbonStore.scala ---
    @@ -110,8 +110,16 @@ class SparkCarbonStore extends MetaCachedCarbonStore {
       }
     
       def startSearchMode(): Unit = {
    +    LOG.info("Search mode master start service")
         master = new Master(session.sparkContext.getConf)
         master.startService()
    +    var count = 0
    +    while (!master.isStarted()) {
    +      Thread.sleep(10)
    +      count = count + 1;
    +      LOG.info("Sleeping and waiting search mode master finish start " + count)
    --- End diff --
    
    ok


---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r185725441
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -80,9 +83,32 @@ class Master(sparkConf: SparkConf, port: Int) {
               rpcEnv = new NettyRpcEnvFactory().create(config)
               val registryEndpoint: RpcEndpoint = new Registry(rpcEnv, Master.this)
               rpcEnv.setupEndpoint("registry-service", registryEndpoint)
    +          if (isStarted.compareAndSet(false, false)) {
    +            synchronized {
    +              if (isStarted.compareAndSet(false, false)) {
    +                isStarted.set(true)
    +              }
    +            }
    +          }
               rpcEnv.awaitTermination()
             }
           }).start()
    +      var count = 0
    +      val countThreshold = 5000
    +      while (isStarted.compareAndSet(false, false) && count < countThreshold) {
    +        LOG.info(s"Waiting search mode master to start, retrying $count times")
    +        Thread.sleep(10)
    +        count = count + 1;
    +      }
    +      if (count >= countThreshold) {
    +        LOG.error(s"Search mode try $countThreshold times to start master but failed")
    +        throw new RuntimeException(
    +          s"Search mode try $countThreshold times to start master but failed")
    +      } else {
    +        LOG.info("Search mode master started")
    +      }
    +    } else {
    +      LOG.info("Search mode master has already existed before.")
    --- End diff --
    
    ok


---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184653962
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/MasterStatus.scala ---
    @@ -0,0 +1,66 @@
    +/*
    + * 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.spark.rpc
    +
    +/**
    + * For search mode master status control
    + *
    + * @param msg status message
    + */
    +class MasterStatus(var msg: String = "Status") {
    --- End diff --
    
    I remove this one


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4607/



---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xuchuanyin <gi...@git.apache.org>.
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184842192
  
    --- Diff: integration/spark2/src/main/scala/org/apache/carbondata/store/SparkCarbonStore.scala ---
    @@ -110,8 +110,16 @@ class SparkCarbonStore extends MetaCachedCarbonStore {
       }
     
       def startSearchMode(): Unit = {
    --- End diff --
    
    I think the log can be optimized like below:
    
    ```
    "Starting search mode master"
    ```
    ```
    s"Waiting search mode master to start, retrying $count times"
    ```
    
    ```
    Search mode master started
    ```



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/5594/



---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184649653
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -69,20 +69,32 @@ class Master(sparkConf: SparkConf, port: Int) {
       }
     
       /** start service and listen on port passed in constructor */
    -  def startService(): Unit = {
    +  def startService(message: MasterStatus): Unit = {
    --- End diff --
    
    instead of adding this parameter, you can add a isStarted method in Master, and use it in SparkCarbonStore


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4593/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/5614/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/5628/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4673/



---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184649709
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/MasterStatus.scala ---
    @@ -0,0 +1,66 @@
    +/*
    + * 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.spark.rpc
    +
    +/**
    + * For search mode master status control
    + *
    + * @param msg status message
    + */
    +class MasterStatus(var msg: String = "Status") {
    --- End diff --
    
    I think this is not needed


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/4327/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    LGTM


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/5504/



---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r185725067
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -70,7 +71,9 @@ class Master(sparkConf: SparkConf, port: Int) {
     
       /** start service and listen on port passed in constructor */
       def startService(): Unit = {
    +    val isStarted: AtomicBoolean = new AtomicBoolean(false)
    --- End diff --
    
    ok


---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r185718131
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -80,9 +83,32 @@ class Master(sparkConf: SparkConf, port: Int) {
               rpcEnv = new NettyRpcEnvFactory().create(config)
               val registryEndpoint: RpcEndpoint = new Registry(rpcEnv, Master.this)
               rpcEnv.setupEndpoint("registry-service", registryEndpoint)
    +          if (isStarted.compareAndSet(false, false)) {
    +            synchronized {
    +              if (isStarted.compareAndSet(false, false)) {
    +                isStarted.set(true)
    +              }
    +            }
    +          }
               rpcEnv.awaitTermination()
             }
           }).start()
    +      var count = 0
    +      val countThreshold = 5000
    +      while (isStarted.compareAndSet(false, false) && count < countThreshold) {
    +        LOG.info(s"Waiting search mode master to start, retrying $count times")
    +        Thread.sleep(10)
    +        count = count + 1;
    +      }
    +      if (count >= countThreshold) {
    +        LOG.error(s"Search mode try $countThreshold times to start master but failed")
    +        throw new RuntimeException(
    +          s"Search mode try $countThreshold times to start master but failed")
    +      } else {
    +        LOG.info("Search mode master started")
    +      }
    +    } else {
    +      LOG.info("Search mode master has already existed before.")
    --- End diff --
    
    change to `Search mode master has already started`


---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xuchuanyin <gi...@git.apache.org>.
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184842473
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -80,9 +96,18 @@ class Master(sparkConf: SparkConf, port: Int) {
               rpcEnv = new NettyRpcEnvFactory().create(config)
               val registryEndpoint: RpcEndpoint = new Registry(rpcEnv, Master.this)
               rpcEnv.setupEndpoint("registry-service", registryEndpoint)
    +          if (!isStarted) {
    +            synchronized {
    +              if (!isStarted) {
    +                flag = true
    +              }
    +            }
    +          }
               rpcEnv.awaitTermination()
             }
           }).start()
    +    } else {
    +      LOG.info("rpcEnv exists:" + rpcEnv.address)
    --- End diff --
    
    What does it mean?
    Does it mean the search mode master has already started or something wrong?
    Better to optimize the log, for example
    ```
    Search mode master has already been started.
    ``
    OR 
    ```
    Address for search mode master has been used by other applications, start failed.
    ```


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/4434/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4688/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4643/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/4445/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/5547/



---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184850096
  
    --- Diff: integration/spark2/src/main/scala/org/apache/carbondata/store/SparkCarbonStore.scala ---
    @@ -110,8 +110,16 @@ class SparkCarbonStore extends MetaCachedCarbonStore {
       }
     
       def startSearchMode(): Unit = {
    +    LOG.info("Search mode master start service")
         master = new Master(session.sparkContext.getConf)
         master.startService()
    +    var count = 0
    +    while (!master.isStarted()) {
    +      Thread.sleep(10)
    --- End diff --
    
    ok


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/4453/



---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xuchuanyin <gi...@git.apache.org>.
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184842088
  
    --- Diff: integration/spark2/src/main/scala/org/apache/carbondata/store/SparkCarbonStore.scala ---
    @@ -110,8 +110,16 @@ class SparkCarbonStore extends MetaCachedCarbonStore {
       }
     
       def startSearchMode(): Unit = {
    +    LOG.info("Search mode master start service")
         master = new Master(session.sparkContext.getConf)
         master.startService()
    +    var count = 0
    +    while (!master.isStarted()) {
    +      Thread.sleep(10)
    +      count = count + 1;
    +      LOG.info("Sleeping and waiting search mode master finish start " + count)
    --- End diff --
    
    This log should be printed before action (line 118)


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/4468/



---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r185717770
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -80,9 +83,32 @@ class Master(sparkConf: SparkConf, port: Int) {
               rpcEnv = new NettyRpcEnvFactory().create(config)
               val registryEndpoint: RpcEndpoint = new Registry(rpcEnv, Master.this)
               rpcEnv.setupEndpoint("registry-service", registryEndpoint)
    +          if (isStarted.compareAndSet(false, false)) {
    --- End diff --
    
    I think you can set it directly


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/5606/



---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184850235
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -80,9 +96,18 @@ class Master(sparkConf: SparkConf, port: Int) {
               rpcEnv = new NettyRpcEnvFactory().create(config)
               val registryEndpoint: RpcEndpoint = new Registry(rpcEnv, Master.this)
               rpcEnv.setupEndpoint("registry-service", registryEndpoint)
    +          if (!isStarted) {
    --- End diff --
    
    I will try


---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184850026
  
    --- Diff: integration/spark2/src/main/scala/org/apache/carbondata/store/SparkCarbonStore.scala ---
    @@ -110,8 +110,16 @@ class SparkCarbonStore extends MetaCachedCarbonStore {
       }
     
       def startSearchMode(): Unit = {
    --- End diff --
    
    thanks


---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xuchuanyin <gi...@git.apache.org>.
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r184842419
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -80,9 +96,18 @@ class Master(sparkConf: SparkConf, port: Int) {
               rpcEnv = new NettyRpcEnvFactory().create(config)
               val registryEndpoint: RpcEndpoint = new Registry(rpcEnv, Master.this)
               rpcEnv.setupEndpoint("registry-service", registryEndpoint)
    +          if (!isStarted) {
    --- End diff --
    
    Why not use atomicBoolean? It needs less code.


---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r185684520
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -64,13 +65,29 @@ class Master(sparkConf: SparkConf, port: Int) {
     
       private val scheduler: Scheduler = new Scheduler
     
    +  /**
    +   * flag is the master status, default value is false, which means master not start now.
    +   * if it is true, then search mode master finished start.
    +   */
    +  private var flag: AtomicBoolean = new AtomicBoolean(false)
    --- End diff --
    
    ok


---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r185725112
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -80,9 +83,32 @@ class Master(sparkConf: SparkConf, port: Int) {
               rpcEnv = new NettyRpcEnvFactory().create(config)
               val registryEndpoint: RpcEndpoint = new Registry(rpcEnv, Master.this)
               rpcEnv.setupEndpoint("registry-service", registryEndpoint)
    +          if (isStarted.compareAndSet(false, false)) {
    --- End diff --
    
    ok


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/4339/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/4433/



---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4707/



---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r185683438
  
    --- Diff: integration/spark2/src/main/scala/org/apache/carbondata/store/SparkCarbonStore.scala ---
    @@ -110,8 +110,16 @@ class SparkCarbonStore extends MetaCachedCarbonStore {
       }
     
       def startSearchMode(): Unit = {
    +    LOG.info("Starting search mode master")
         master = new Master(session.sparkContext.getConf)
         master.startService()
    +    var count = 0
    +    while (master.isStarted().compareAndSet(false, false) && count < 5000) {
    +      LOG.info(s"Waiting search mode master to start, retrying $count times")
    +      Thread.sleep(10)
    +      count = count + 1;
    +    }
    +    LOG.info("Search mode master started")
    --- End diff --
    
    ok


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/5458/



---

[GitHub] carbondata pull request #2239: [CARBONDATA-2408] Fix search mode master Sasl...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2239#discussion_r185683794
  
    --- Diff: integration/spark2/src/main/scala/org/apache/carbondata/store/SparkCarbonStore.scala ---
    @@ -110,8 +110,16 @@ class SparkCarbonStore extends MetaCachedCarbonStore {
       }
     
       def startSearchMode(): Unit = {
    +    LOG.info("Starting search mode master")
         master = new Master(session.sparkContext.getConf)
         master.startService()
    +    var count = 0
    +    while (master.isStarted().compareAndSet(false, false) && count < 5000) {
    --- End diff --
    
    ok,I will try


---

[GitHub] carbondata issue #2239: [CARBONDATA-2408] Fix search mode master SaslExcepti...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2239
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4689/



---