private void doFinish() {
      if (finished.compareAndSet(false, true)) {
        Releasables.close(indexShardReference);
        final ShardId shardId = shardIt.shardId();
        final ActionWriteResponse.ShardInfo.Failure[] failuresArray;
        if (!shardReplicaFailures.isEmpty()) {
          int slot = 0;
          failuresArray = new ActionWriteResponse.ShardInfo.Failure[shardReplicaFailures.size()];
          for (Map.Entry<String, Throwable> entry : shardReplicaFailures.entrySet()) {
            RestStatus restStatus = ExceptionsHelper.status(entry.getValue());
            failuresArray[slot++] =
                new ActionWriteResponse.ShardInfo.Failure(
                    shardId.getIndex(),
                    shardId.getId(),
                    entry.getKey(),
                    entry.getValue(),
                    restStatus,
                    false);
          }
        } else {
          failuresArray = ActionWriteResponse.EMPTY;
        }
        finalResponse.setShardInfo(
            new ActionWriteResponse.ShardInfo(totalShards, success.get(), failuresArray));

        listener.onResponse(finalResponse);
      }
    }
 /**
  * Checks whether we can perform a write based on the required active shard count setting. Returns
  * **null* if OK to proceed, or a string describing the reason to stop
  */
 protected String checkActiveShardCount() {
   final ShardId shardId = primary.routingEntry().shardId();
   final String indexName = shardId.getIndexName();
   final ClusterState state = clusterStateSupplier.get();
   assert state != null : "replication operation must have access to the cluster state";
   final ActiveShardCount waitForActiveShards = request.waitForActiveShards();
   if (waitForActiveShards == ActiveShardCount.NONE) {
     return null; // not waiting for any shards
   }
   IndexRoutingTable indexRoutingTable = state.getRoutingTable().index(indexName);
   if (indexRoutingTable == null) {
     logger.trace("[{}] index not found in the routing table", shardId);
     return "Index " + indexName + " not found in the routing table";
   }
   IndexShardRoutingTable shardRoutingTable = indexRoutingTable.shard(shardId.getId());
   if (shardRoutingTable == null) {
     logger.trace("[{}] shard not found in the routing table", shardId);
     return "Shard " + shardId + " not found in the routing table";
   }
   if (waitForActiveShards.enoughShardsActive(shardRoutingTable)) {
     return null;
   } else {
     final String resolvedShards =
         waitForActiveShards == ActiveShardCount.ALL
             ? Integer.toString(shardRoutingTable.shards().size())
             : waitForActiveShards.toString();
     logger.trace(
         "[{}] not enough active copies to meet shard count of [{}] (have {}, needed {}), scheduling a retry. op [{}], "
             + "request [{}]",
         shardId,
         waitForActiveShards,
         shardRoutingTable.activeShards().size(),
         resolvedShards,
         opType,
         request);
     return "Not enough active copies to meet shard count of ["
         + waitForActiveShards
         + "] (have "
         + shardRoutingTable.activeShards().size()
         + ", needed "
         + resolvedShards
         + ").";
   }
 }
 public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
   builder.startObject();
   {
     builder.startObject("shard");
     {
       builder.field("index", shard.getIndexName());
       builder.field("index_uuid", shard.getIndex().getUUID());
       builder.field("id", shard.getId());
       builder.field("primary", primary);
     }
     builder.endObject(); // end shard
     builder.field("assigned", this.assignedNodeId != null);
     // If assigned, show the node id of the node it's assigned to
     if (assignedNodeId != null) {
       builder.field("assigned_node_id", this.assignedNodeId);
     }
     builder.field("shard_state_fetch_pending", this.hasPendingAsyncFetch);
     // If we have unassigned info, show that
     if (unassignedInfo != null) {
       unassignedInfo.toXContent(builder, params);
       builder.timeValueField(
           "allocation_delay_in_millis",
           "allocation_delay",
           TimeValue.timeValueMillis(allocationDelayMillis));
       builder.timeValueField(
           "remaining_delay_in_millis",
           "remaining_delay",
           TimeValue.timeValueMillis(remainingDelayMillis));
     }
     builder.startObject("nodes");
     for (NodeExplanation explanation : nodeExplanations.values()) {
       explanation.toXContent(builder, params);
     }
     builder.endObject(); // end nodes
   }
   builder.endObject(); // end wrapping object
   return builder;
 }
  public synchronized IndexShard createShard(ShardRouting routing) throws IOException {
    final boolean primary = routing.primary();
    /*
     * TODO: we execute this in parallel but it's a synced method. Yet, we might
     * be able to serialize the execution via the cluster state in the future. for now we just
     * keep it synced.
     */
    if (closed.get()) {
      throw new IllegalStateException("Can't create shard " + routing.shardId() + ", closed");
    }
    final Settings indexSettings = this.indexSettings.getSettings();
    final ShardId shardId = routing.shardId();
    boolean success = false;
    Store store = null;
    IndexShard indexShard = null;
    ShardLock lock = null;
    try {
      lock = nodeEnv.shardLock(shardId, TimeUnit.SECONDS.toMillis(5));
      eventListener.beforeIndexShardCreated(shardId, indexSettings);
      ShardPath path;
      try {
        path = ShardPath.loadShardPath(logger, nodeEnv, shardId, this.indexSettings);
      } catch (IllegalStateException ex) {
        logger.warn("{} failed to load shard path, trying to remove leftover", shardId);
        try {
          ShardPath.deleteLeftoverShardDirectory(logger, nodeEnv, lock, this.indexSettings);
          path = ShardPath.loadShardPath(logger, nodeEnv, shardId, this.indexSettings);
        } catch (Exception inner) {
          ex.addSuppressed(inner);
          throw ex;
        }
      }

      if (path == null) {
        // TODO: we should, instead, hold a "bytes reserved" of how large we anticipate this shard
        // will be, e.g. for a shard
        // that's being relocated/replicated we know how large it will become once it's done
        // copying:
        // Count up how many shards are currently on each data path:
        Map<Path, Integer> dataPathToShardCount = new HashMap<>();
        for (IndexShard shard : this) {
          Path dataPath = shard.shardPath().getRootStatePath();
          Integer curCount = dataPathToShardCount.get(dataPath);
          if (curCount == null) {
            curCount = 0;
          }
          dataPathToShardCount.put(dataPath, curCount + 1);
        }
        path =
            ShardPath.selectNewPathForShard(
                nodeEnv,
                shardId,
                this.indexSettings,
                routing.getExpectedShardSize() == ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE
                    ? getAvgShardSizeInBytes()
                    : routing.getExpectedShardSize(),
                dataPathToShardCount);
        logger.debug("{} creating using a new path [{}]", shardId, path);
      } else {
        logger.debug("{} creating using an existing path [{}]", shardId, path);
      }

      if (shards.containsKey(shardId.id())) {
        throw new IndexShardAlreadyExistsException(shardId + " already exists");
      }

      logger.debug("creating shard_id {}", shardId);
      // if we are on a shared FS we only own the shard (ie. we can safely delete it) if we are the
      // primary.
      final boolean canDeleteShardContent =
          IndexMetaData.isOnSharedFilesystem(indexSettings) == false
              || (primary && IndexMetaData.isOnSharedFilesystem(indexSettings));
      final Engine.Warmer engineWarmer =
          (searcher) -> {
            IndexShard shard = getShardOrNull(shardId.getId());
            if (shard != null) {
              warmer.warm(searcher, shard, IndexService.this.indexSettings);
            }
          };
      store =
          new Store(
              shardId,
              this.indexSettings,
              indexStore.newDirectoryService(path),
              lock,
              new StoreCloseListener(
                  shardId, canDeleteShardContent, () -> eventListener.onStoreClosed(shardId)));
      if (useShadowEngine(primary, indexSettings)) {
        indexShard =
            new ShadowIndexShard(
                routing,
                this.indexSettings,
                path,
                store,
                indexCache,
                mapperService,
                similarityService,
                indexFieldData,
                engineFactory,
                eventListener,
                searcherWrapper,
                threadPool,
                bigArrays,
                engineWarmer,
                searchOperationListeners);
        // no indexing listeners - shadow  engines don't index
      } else {
        indexShard =
            new IndexShard(
                routing,
                this.indexSettings,
                path,
                store,
                indexCache,
                mapperService,
                similarityService,
                indexFieldData,
                engineFactory,
                eventListener,
                searcherWrapper,
                threadPool,
                bigArrays,
                engineWarmer,
                searchOperationListeners,
                indexingOperationListeners);
      }
      eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created");
      eventListener.afterIndexShardCreated(indexShard);
      shards = newMapBuilder(shards).put(shardId.id(), indexShard).immutableMap();
      success = true;
      return indexShard;
    } catch (ShardLockObtainFailedException e) {
      throw new IOException("failed to obtain in-memory shard lock", e);
    } finally {
      if (success == false) {
        if (lock != null) {
          IOUtils.closeWhileHandlingException(lock);
        }
        closeShard("initialization failed", shardId, indexShard, store, eventListener);
      }
    }
  }
    ASyncAction(
        MultiPercolateRequest multiPercolateRequest,
        List<Object> percolateRequests,
        ActionListener<MultiPercolateResponse> finalListener,
        ClusterState clusterState) {
      this.finalListener = finalListener;
      this.multiPercolateRequest = multiPercolateRequest;
      this.percolateRequests = percolateRequests;
      responsesByItemAndShard = new AtomicReferenceArray<>(percolateRequests.size());
      expectedOperationsPerItem = new AtomicReferenceArray<>(percolateRequests.size());
      reducedResponses = new AtomicArray<>(percolateRequests.size());

      // Resolving concrete indices and routing and grouping the requests by shard
      requestsByShard = new HashMap<>();
      // Keep track what slots belong to what shard, in case a request to a shard fails on all
      // copies
      shardToSlots = new HashMap<>();
      int expectedResults = 0;
      for (int slot = 0; slot < percolateRequests.size(); slot++) {
        Object element = percolateRequests.get(slot);
        assert element != null;
        if (element instanceof PercolateRequest) {
          PercolateRequest percolateRequest = (PercolateRequest) element;
          String[] concreteIndices;
          try {
            concreteIndices =
                indexNameExpressionResolver.concreteIndices(clusterState, percolateRequest);
          } catch (IndexNotFoundException e) {
            reducedResponses.set(slot, e);
            responsesByItemAndShard.set(slot, new AtomicReferenceArray(0));
            expectedOperationsPerItem.set(slot, new AtomicInteger(0));
            continue;
          }
          Map<String, Set<String>> routing =
              indexNameExpressionResolver.resolveSearchRouting(
                  clusterState, percolateRequest.routing(), percolateRequest.indices());
          // TODO: I only need shardIds, ShardIterator(ShardRouting) is only needed in
          // TransportShardMultiPercolateAction
          GroupShardsIterator shards =
              clusterService
                  .operationRouting()
                  .searchShards(
                      clusterState, concreteIndices, routing, percolateRequest.preference());
          if (shards.size() == 0) {
            reducedResponses.set(slot, new UnavailableShardsException(null, "No shards available"));
            responsesByItemAndShard.set(slot, new AtomicReferenceArray(0));
            expectedOperationsPerItem.set(slot, new AtomicInteger(0));
            continue;
          }

          // The shard id is used as index in the atomic ref array, so we need to find out how many
          // shards there are regardless of routing:
          int numShards =
              clusterService
                  .operationRouting()
                  .searchShardsCount(clusterState, concreteIndices, null);
          responsesByItemAndShard.set(slot, new AtomicReferenceArray(numShards));
          expectedOperationsPerItem.set(slot, new AtomicInteger(shards.size()));
          for (ShardIterator shard : shards) {
            ShardId shardId = shard.shardId();
            TransportShardMultiPercolateAction.Request requests = requestsByShard.get(shardId);
            if (requests == null) {
              requestsByShard.put(
                  shardId,
                  requests =
                      new TransportShardMultiPercolateAction.Request(
                          multiPercolateRequest,
                          shardId.getIndex(),
                          shardId.getId(),
                          percolateRequest.preference()));
            }
            logger.trace("Adding shard[{}] percolate request for item[{}]", shardId, slot);
            requests.add(
                new TransportShardMultiPercolateAction.Request.Item(
                    slot, new PercolateShardRequest(shardId, percolateRequest)));

            IntArrayList items = shardToSlots.get(shardId);
            if (items == null) {
              shardToSlots.put(shardId, items = new IntArrayList());
            }
            items.add(slot);
          }
          expectedResults++;
        } else if (element instanceof Throwable || element instanceof MultiGetResponse.Failure) {
          logger.trace("item[{}] won't be executed, reason: {}", slot, element);
          reducedResponses.set(slot, element);
          responsesByItemAndShard.set(slot, new AtomicReferenceArray(0));
          expectedOperationsPerItem.set(slot, new AtomicInteger(0));
        }
      }
      expectedOperations = new AtomicInteger(expectedResults);
    }