/**
     * checks for any cluster state blocks. Returns true if operation is OK to proceeded. if false
     * is return, no further action is needed. The method takes care of any continuation, by either
     * responding to the listener or scheduling a retry
     */
    protected boolean checkBlocks() {
      ClusterBlockException blockException = checkGlobalBlock(observer.observedState());
      if (blockException != null) {
        if (blockException.retryable()) {
          logger.trace("cluster is blocked ({}), scheduling a retry", blockException.getMessage());
          retry(blockException);
        } else {
          finishAsFailed(blockException);
        }
        return false;
      }
      if (resolveIndex()) {
        internalRequest.concreteIndex(
            indexNameExpressionResolver.concreteSingleIndex(
                observer.observedState(), internalRequest.request()));
      } else {
        internalRequest.concreteIndex(internalRequest.request().index());
      }

      resolveRequest(observer.observedState(), internalRequest, listener);

      blockException = checkRequestBlock(observer.observedState(), internalRequest);
      if (blockException != null) {
        if (blockException.retryable()) {
          logger.trace("cluster is blocked ({}), scheduling a retry", blockException.getMessage());
          retry(blockException);
        } else {
          finishAsFailed(blockException);
        }
        return false;
      }
      return true;
    }
 void retryBecauseUnavailable(ShardId shardId, String message) {
   retry(
       new UnavailableShardsException(
           shardId,
           message
               + " Timeout: ["
               + internalRequest.request().timeout()
               + "], request: "
               + internalRequest.request().toString()));
 }
    /**
     * checks whether we can perform a write based on the write consistency setting returns **null*
     * if OK to proceed, or a string describing the reason to stop
     */
    String checkWriteConsistency(ShardRouting shard) {
      if (checkWriteConsistency == false) {
        return null;
      }

      final WriteConsistencyLevel consistencyLevel;
      if (internalRequest.request().consistencyLevel() != WriteConsistencyLevel.DEFAULT) {
        consistencyLevel = internalRequest.request().consistencyLevel();
      } else {
        consistencyLevel = defaultWriteConsistencyLevel;
      }
      final int sizeActive;
      final int requiredNumber;
      IndexRoutingTable indexRoutingTable =
          observer.observedState().getRoutingTable().index(shard.index());
      if (indexRoutingTable != null) {
        IndexShardRoutingTable shardRoutingTable = indexRoutingTable.shard(shard.getId());
        if (shardRoutingTable != null) {
          sizeActive = shardRoutingTable.activeShards().size();
          if (consistencyLevel == WriteConsistencyLevel.QUORUM && shardRoutingTable.getSize() > 2) {
            // only for more than 2 in the number of shardIt it makes sense, otherwise its 1 shard
            // with 1 replica, quorum is 1 (which is what it is initialized to)
            requiredNumber = (shardRoutingTable.getSize() / 2) + 1;
          } else if (consistencyLevel == WriteConsistencyLevel.ALL) {
            requiredNumber = shardRoutingTable.getSize();
          } else {
            requiredNumber = 1;
          }
        } else {
          sizeActive = 0;
          requiredNumber = 1;
        }
      } else {
        sizeActive = 0;
        requiredNumber = 1;
      }

      if (sizeActive < requiredNumber) {
        logger.trace(
            "not enough active copies of shard [{}] to meet write consistency of [{}] (have {}, needed {}), scheduling a retry.",
            shard.shardId(),
            consistencyLevel,
            sizeActive,
            requiredNumber);
        return "Not enough active copies to meet write consistency of ["
            + consistencyLevel
            + "] (have "
            + sizeActive
            + ", needed "
            + requiredNumber
            + ").";
      } else {
        return null;
      }
    }
    void retry(final @Nullable Throwable failure) {
      if (observer.isTimedOut()) {
        // we running as a last attempt after a timeout has happened. don't retry
        return;
      }

      // make it threaded operation so we fork on the discovery listener thread
      internalRequest.request().beforeLocalFork();
      observer.waitForNextChange(
          new ClusterStateObserver.Listener() {
            @Override
            public void onNewClusterState(ClusterState state) {
              doStart();
            }

            @Override
            public void onClusterServiceClose() {
              listener.onFailure(new NodeClosedException(nodes.localNode()));
            }

            @Override
            public void onTimeout(TimeValue timeout) {
              // just to be on the safe side, see if we can start it now?
              if (!doStart()) {
                Throwable listenFailure = failure;
                if (listenFailure == null) {
                  if (shardIt == null) {
                    listenFailure =
                        new UnavailableShardsException(
                            new ShardId(internalRequest.concreteIndex(), -1),
                            "Timeout waiting for ["
                                + timeout
                                + "], request: "
                                + internalRequest.request().toString());
                  } else {
                    listenFailure =
                        new UnavailableShardsException(
                            shardIt.shardId(),
                            "["
                                + shardIt.size()
                                + "] shardIt, ["
                                + shardIt.sizeActive()
                                + "] active : Timeout waiting for ["
                                + timeout
                                + "], request: "
                                + internalRequest.request().toString());
                  }
                }
                listener.onFailure(listenFailure);
              }
            }
          },
          internalRequest.request().timeout());
    }
 private void onFailure(ShardRouting shardRouting, Throwable e) {
   if (logger.isTraceEnabled() && e != null) {
     logger.trace(
         shardRouting.shortSummary() + ": Failed to execute [" + internalRequest.request() + "]",
         e);
   }
   perform(e);
 }
    protected boolean doStart() {
      nodes = observer.observedState().nodes();
      try {
        ClusterBlockException blockException = checkGlobalBlock(observer.observedState());
        if (blockException != null) {
          if (blockException.retryable()) {
            retry(blockException);
            return false;
          } else {
            throw blockException;
          }
        }
        internalRequest.concreteIndex(
            observer
                .observedState()
                .metaData()
                .concreteSingleIndex(
                    internalRequest.request().index(), internalRequest.request().indicesOptions()));
        // check if we need to execute, and if not, return
        if (!resolveRequest(observer.observedState(), internalRequest, listener)) {
          return true;
        }
        blockException = checkRequestBlock(observer.observedState(), internalRequest);
        if (blockException != null) {
          if (blockException.retryable()) {
            retry(blockException);
            return false;
          } else {
            throw blockException;
          }
        }
        shardIt = shards(observer.observedState(), internalRequest);
      } catch (Throwable e) {
        listener.onFailure(e);
        return true;
      }

      // no shardIt, might be in the case between index gateway recovery and shardIt initialization
      if (shardIt.size() == 0) {
        retry(null);
        return false;
      }

      // this transport only make sense with an iterator that returns a single shard routing (like
      // primary)
      assert shardIt.size() == 1;

      ShardRouting shard = shardIt.nextOrNull();
      assert shard != null;

      if (!shard.active()) {
        retry(null);
        return false;
      }

      if (!operationStarted.compareAndSet(false, true)) {
        return true;
      }

      internalRequest.request().shardId = shardIt.shardId().id();
      if (shard.currentNodeId().equals(nodes.localNodeId())) {
        internalRequest.request().beforeLocalFork();
        try {
          threadPool
              .executor(executor)
              .execute(
                  new Runnable() {
                    @Override
                    public void run() {
                      try {
                        shardOperation(internalRequest, listener);
                      } catch (Throwable e) {
                        if (retryOnFailure(e)) {
                          operationStarted.set(false);
                          // we already marked it as started when we executed it (removed the
                          // listener) so pass false
                          // to re-add to the cluster listener
                          retry(null);
                        } else {
                          listener.onFailure(e);
                        }
                      }
                    }
                  });
        } catch (Throwable e) {
          if (retryOnFailure(e)) {
            retry(null);
          } else {
            listener.onFailure(e);
          }
        }
      } else {
        DiscoveryNode node = nodes.get(shard.currentNodeId());
        transportService.sendRequest(
            node,
            actionName,
            internalRequest.request(),
            transportOptions(),
            new BaseTransportResponseHandler<Response>() {

              @Override
              public Response newInstance() {
                return newResponse();
              }

              @Override
              public String executor() {
                return ThreadPool.Names.SAME;
              }

              @Override
              public void handleResponse(Response response) {
                listener.onResponse(response);
              }

              @Override
              public void handleException(TransportException exp) {
                // if we got disconnected from the node, or the node / shard is not in the right
                // state (being closed)
                if (exp.unwrapCause() instanceof ConnectTransportException
                    || exp.unwrapCause() instanceof NodeClosedException
                    || retryOnFailure(exp)) {
                  operationStarted.set(false);
                  // we already marked it as started when we executed it (removed the listener) so
                  // pass false
                  // to re-add to the cluster listener
                  retry(null);
                } else {
                  listener.onFailure(exp);
                }
              }
            });
      }
      return true;
    }
 public void start() {
   this.observer =
       new ClusterStateObserver(clusterService, internalRequest.request().timeout(), logger);
   doStart();
 }
    private void perform(final Throwable lastException) {
      final ShardRouting shard = shardsIt == null ? null : shardsIt.nextOrNull();
      if (shard == null) {
        Throwable failure = lastException;
        if (failure == null) {
          failure =
              new NoShardAvailableActionException(
                  null, "No shard available for [" + internalRequest.request() + "]");
        } else {
          if (logger.isDebugEnabled()) {
            logger.debug("failed to execute [" + internalRequest.request() + "]", failure);
          }
        }
        listener.onFailure(failure);
      } else {
        if (shard.currentNodeId().equals(nodes.localNodeId())) {
          // we don't prefer local shard, so try and do it here
          if (!internalRequest.request().preferLocalShard()) {
            try {
              if (internalRequest.request().operationThreaded()) {
                internalRequest.request().beforeLocalFork();
                threadPool
                    .executor(executor)
                    .execute(
                        new Runnable() {
                          @Override
                          public void run() {
                            try {
                              Response response =
                                  shardOperation(internalRequest.request(), shard.shardId());
                              listener.onResponse(response);
                            } catch (Throwable e) {
                              onFailure(shard, e);
                            }
                          }
                        });
              } else {
                final Response response =
                    shardOperation(internalRequest.request(), shard.shardId());
                listener.onResponse(response);
              }
            } catch (Throwable e) {
              onFailure(shard, e);
            }
          } else {
            perform(lastException);
          }
        } else {
          DiscoveryNode node = nodes.get(shard.currentNodeId());
          transportService.sendRequest(
              node,
              transportShardAction,
              new ShardSingleOperationRequest(internalRequest.request(), shard.shardId()),
              new BaseTransportResponseHandler<Response>() {
                @Override
                public Response newInstance() {
                  return newResponse();
                }

                @Override
                public String executor() {
                  return ThreadPool.Names.SAME;
                }

                @Override
                public void handleResponse(final Response response) {
                  listener.onResponse(response);
                }

                @Override
                public void handleException(TransportException exp) {
                  onFailure(shard, exp);
                }
              });
        }
      }
    }
    /** First get should try and use a shard that exists on a local node for better performance */
    private void performFirst() {
      if (shardsIt == null) {
        // just execute it on the local node
        if (internalRequest.request().operationThreaded()) {
          internalRequest.request().beforeLocalFork();
          threadPool
              .executor(executor())
              .execute(
                  new Runnable() {
                    @Override
                    public void run() {
                      try {
                        Response response = shardOperation(internalRequest.request(), null);
                        listener.onResponse(response);
                      } catch (Throwable e) {
                        onFailure(null, e);
                      }
                    }
                  });
          return;
        } else {
          try {
            final Response response = shardOperation(internalRequest.request(), null);
            listener.onResponse(response);
            return;
          } catch (Throwable e) {
            onFailure(null, e);
          }
        }
        return;
      }

      if (internalRequest.request().preferLocalShard()) {
        boolean foundLocal = false;
        ShardRouting shardX;
        while ((shardX = shardsIt.nextOrNull()) != null) {
          final ShardRouting shard = shardX;
          if (shard.currentNodeId().equals(nodes.localNodeId())) {
            foundLocal = true;
            if (internalRequest.request().operationThreaded()) {
              internalRequest.request().beforeLocalFork();
              threadPool
                  .executor(executor())
                  .execute(
                      new Runnable() {
                        @Override
                        public void run() {
                          try {
                            Response response =
                                shardOperation(internalRequest.request(), shard.shardId());
                            listener.onResponse(response);
                          } catch (Throwable e) {
                            shardsIt.reset();
                            onFailure(shard, e);
                          }
                        }
                      });
              return;
            } else {
              try {
                final Response response =
                    shardOperation(internalRequest.request(), shard.shardId());
                listener.onResponse(response);
                return;
              } catch (Throwable e) {
                shardsIt.reset();
                onFailure(shard, e);
              }
            }
          }
        }
        if (!foundLocal) {
          // no local node get, go remote
          shardsIt.reset();
          perform(null);
        }
      } else {
        perform(null);
      }
    }
 /** perform the operation on the node holding the primary */
 void performOnPrimary(final ShardRouting primary, final ShardIterator shardsIt) {
   final String writeConsistencyFailure = checkWriteConsistency(primary);
   if (writeConsistencyFailure != null) {
     retryBecauseUnavailable(primary.shardId(), writeConsistencyFailure);
     return;
   }
   final ReplicationPhase replicationPhase;
   try {
     indexShardReference = getIndexShardOperationsCounter(primary.shardId());
     PrimaryOperationRequest por =
         new PrimaryOperationRequest(
             primary.id(), internalRequest.concreteIndex(), internalRequest.request());
     Tuple<Response, ReplicaRequest> primaryResponse =
         shardOperationOnPrimary(observer.observedState(), por);
     logger.trace("operation completed on primary [{}]", primary);
     replicationPhase =
         new ReplicationPhase(
             shardsIt,
             primaryResponse.v2(),
             primaryResponse.v1(),
             observer,
             primary,
             internalRequest,
             listener,
             indexShardReference,
             shardFailedTimeout);
   } catch (Throwable e) {
     // shard has not been allocated yet, retry it here
     if (retryPrimaryException(e)) {
       logger.trace(
           "had an error while performing operation on primary ({}), scheduling a retry.",
           e.getMessage());
       // We have to close here because when we retry we will increment get a new reference on
       // index shard again and we do not want to
       // increment twice.
       Releasables.close(indexShardReference);
       // We have to reset to null here because whe we retry it might be that we never get to the
       // point where we assign a new reference
       // (for example, in case the operation was rejected because queue is full). In this case
       // we would release again once one of the finish methods is called.
       indexShardReference = null;
       retry(e);
       return;
     }
     if (ExceptionsHelper.status(e) == RestStatus.CONFLICT) {
       if (logger.isTraceEnabled()) {
         logger.trace(
             primary.shortSummary() + ": Failed to execute [" + internalRequest.request() + "]",
             e);
       }
     } else {
       if (logger.isDebugEnabled()) {
         logger.debug(
             primary.shortSummary() + ": Failed to execute [" + internalRequest.request() + "]",
             e);
       }
     }
     finishAsFailed(e);
     return;
   }
   finishAndMoveToReplication(replicationPhase);
 }
    /** send the request to the node holding the primary or execute if local */
    protected void routeRequestOrPerformLocally(
        final ShardRouting primary, final ShardIterator shardsIt) {
      if (primary.currentNodeId().equals(observer.observedState().nodes().localNodeId())) {
        try {
          threadPool
              .executor(executor)
              .execute(
                  new AbstractRunnable() {
                    @Override
                    public void onFailure(Throwable t) {
                      finishAsFailed(t);
                    }

                    @Override
                    protected void doRun() throws Exception {
                      performOnPrimary(primary, shardsIt);
                    }
                  });
        } catch (Throwable t) {
          finishAsFailed(t);
        }
      } else {
        DiscoveryNode node = observer.observedState().nodes().get(primary.currentNodeId());
        transportService.sendRequest(
            node,
            actionName,
            internalRequest.request(),
            transportOptions,
            new BaseTransportResponseHandler<Response>() {

              @Override
              public Response newInstance() {
                return newResponseInstance();
              }

              @Override
              public String executor() {
                return ThreadPool.Names.SAME;
              }

              @Override
              public void handleResponse(Response response) {
                finishOnRemoteSuccess(response);
              }

              @Override
              public void handleException(TransportException exp) {
                try {
                  // if we got disconnected from the node, or the node / shard is not in the right
                  // state (being closed)
                  if (exp.unwrapCause() instanceof ConnectTransportException
                      || exp.unwrapCause() instanceof NodeClosedException
                      || retryPrimaryException(exp)) {
                    // we already marked it as started when we executed it (removed the listener) so
                    // pass false
                    // to re-add to the cluster listener
                    logger.trace(
                        "received an error from node the primary was assigned to ({}), scheduling a retry",
                        exp.getMessage());
                    retry(exp);
                  } else {
                    finishAsFailed(exp);
                  }
                } catch (Throwable t) {
                  finishWithUnexpectedFailure(t);
                }
              }
            });
      }
    }
 PrimaryPhase(Request request, ActionListener<Response> listener) {
   this.internalRequest = new InternalRequest(request);
   this.listener = listener;
   this.observer =
       new ClusterStateObserver(clusterService, internalRequest.request().timeout(), logger);
 }