Procházet zdrojové kódy

Remove per-type indexing stats (#47203)

With only a single type, the per-type filters for indexing stats are no longer useful.

Relates to #41059
Alan Woodward před 6 roky
rodič
revize
ff99bc1d3f
15 změnil soubory, kde provedl 30 přidání a 278 odebrání
  1. 0 81
      rest-api-spec/src/main/resources/rest-api-spec/test/indices.stats/15_types.yml
  2. 1 1
      server/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStats.java
  3. 7 22
      server/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStatsFlags.java
  4. 0 17
      server/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsRequest.java
  5. 0 9
      server/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsRequestBuilder.java
  6. 1 1
      server/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java
  7. 4 6
      server/src/main/java/org/elasticsearch/index/shard/IndexShard.java
  8. 10 45
      server/src/main/java/org/elasticsearch/index/shard/IndexingStats.java
  9. 4 51
      server/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java
  10. 0 3
      server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestNodesStatsAction.java
  11. 0 5
      server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestIndicesStatsAction.java
  12. 0 34
      server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java
  13. 1 1
      x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/indices/IndexStatsMonitoringDocTests.java
  14. 1 1
      x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/indices/IndicesStatsMonitoringDocTests.java
  15. 1 1
      x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/node/NodeStatsMonitoringDocTests.java

+ 0 - 81
rest-api-spec/src/main/resources/rest-api-spec/test/indices.stats/15_types.yml

@@ -1,81 +0,0 @@
----
-setup:
-
-  - do:
-      index:
-          index: test1
-          type:  bar
-          id:    1
-          body:  { "bar": "bar", "baz": "baz" }
-
-  - do:
-      index:
-          index: test2
-          type:  baz
-          id:    1
-          body:  { "bar": "bar", "baz": "baz" }
-
-
----
-"Types - blank":
-  - do:
-      indices.stats: {}
-
-  - match:    { _all.primaries.indexing.index_total: 2 }
-  - is_false:   _all.primaries.indexing.types
-
----
-"Types - one":
-  - do:
-      indices.stats: { types: bar }
-
-  - match:    { _all.primaries.indexing.types.bar.index_total: 1 }
-  - is_false:   _all.primaries.indexing.types.baz
-
----
-"Types - multi":
-  - do:
-      indices.stats: { types: "bar,baz" }
-
-  - match:    { _all.primaries.indexing.types.bar.index_total: 1 }
-  - match:    { _all.primaries.indexing.types.baz.index_total: 1 }
-
----
-"Types - star":
-  - do:
-      indices.stats: { types: "*" }
-
-  - match:    { _all.primaries.indexing.types.bar.index_total: 1 }
-  - match:    { _all.primaries.indexing.types.baz.index_total: 1 }
-
----
-"Types - pattern":
-  - do:
-      indices.stats: { types: "*r" }
-
-  - match:    { _all.primaries.indexing.types.bar.index_total: 1 }
-  - is_false:   _all.primaries.indexing.types.baz
-
----
-"Types - _all metric":
-  - do:
-      indices.stats: { types: bar, metric: _all }
-
-  - match:    { _all.primaries.indexing.types.bar.index_total: 1 }
-  - is_false:   _all.primaries.indexing.types.baz
-
----
-"Types - indexing metric":
-  - do:
-      indices.stats: { types: bar, metric: indexing }
-
-  - match:    { _all.primaries.indexing.types.bar.index_total: 1 }
-  - is_false:   _all.primaries.indexing.types.baz
-
----
-"Types - multi metric":
-  - do:
-      indices.stats: { types: bar, metric: [ indexing, search ] }
-
-  - match:    { _all.primaries.indexing.types.bar.index_total: 1 }
-  - is_false:   _all.primaries.indexing.types.baz

+ 1 - 1
server/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStats.java

@@ -177,7 +177,7 @@ public class CommonStats implements Writeable, ToXContentFragment {
                         store = indexShard.storeStats();
                         break;
                     case Indexing:
-                        indexing = indexShard.indexingStats(flags.types());
+                        indexing = indexShard.indexingStats();
                         break;
                     case Get:
                         get = indexShard.getStats();

+ 7 - 22
server/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStatsFlags.java

@@ -20,6 +20,7 @@
 package org.elasticsearch.action.admin.indices.stats;
 
 import org.elasticsearch.Version;
+import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
@@ -34,7 +35,6 @@ public class CommonStatsFlags implements Writeable, Cloneable {
     public static final CommonStatsFlags NONE = new CommonStatsFlags().clear();
 
     private EnumSet<Flag> flags = EnumSet.allOf(Flag.class);
-    private String[] types = null;
     private String[] groups = null;
     private String[] fieldDataFields = null;
     private String[] completionDataFields = null;
@@ -59,7 +59,9 @@ public class CommonStatsFlags implements Writeable, Cloneable {
                 flags.add(flag);
             }
         }
-        types = in.readStringArray();
+        if (in.getVersion().before(Version.V_8_0_0)) {
+            in.readStringArray();
+        }
         groups = in.readStringArray();
         fieldDataFields = in.readStringArray();
         completionDataFields = in.readStringArray();
@@ -77,7 +79,9 @@ public class CommonStatsFlags implements Writeable, Cloneable {
         }
         out.writeLong(longFlags);
 
-        out.writeStringArrayNullable(types);
+        if (out.getVersion().before(Version.V_8_0_0)) {
+            out.writeStringArrayNullable(Strings.EMPTY_ARRAY);
+        }
         out.writeStringArrayNullable(groups);
         out.writeStringArrayNullable(fieldDataFields);
         out.writeStringArrayNullable(completionDataFields);
@@ -92,7 +96,6 @@ public class CommonStatsFlags implements Writeable, Cloneable {
      */
     public CommonStatsFlags all() {
         flags = EnumSet.allOf(Flag.class);
-        types = null;
         groups = null;
         fieldDataFields = null;
         completionDataFields = null;
@@ -106,7 +109,6 @@ public class CommonStatsFlags implements Writeable, Cloneable {
      */
     public CommonStatsFlags clear() {
         flags = EnumSet.noneOf(Flag.class);
-        types = null;
         groups = null;
         fieldDataFields = null;
         completionDataFields = null;
@@ -123,23 +125,6 @@ public class CommonStatsFlags implements Writeable, Cloneable {
         return flags.toArray(new Flag[flags.size()]);
     }
 
-    /**
-     * Document types to return stats for. Mainly affects {@link Flag#Indexing} when
-     * enabled, returning specific indexing stats for those types.
-     */
-    public CommonStatsFlags types(String... types) {
-        this.types = types;
-        return this;
-    }
-
-    /**
-     * Document types to return stats for. Mainly affects {@link Flag#Indexing} when
-     * enabled, returning specific indexing stats for those types.
-     */
-    public String[] types() {
-        return this.types;
-    }
-
     /**
      * Sets specific search group stats to retrieve the stats for. Mainly affects search
      * when enabled.

+ 0 - 17
server/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsRequest.java

@@ -78,23 +78,6 @@ public class IndicesStatsRequest extends BroadcastRequest<IndicesStatsRequest> {
         return this;
     }
 
-    /**
-     * Document types to return stats for. Mainly affects {@link #indexing(boolean)} when
-     * enabled, returning specific indexing stats for those types.
-     */
-    public IndicesStatsRequest types(String... types) {
-        flags.types(types);
-        return this;
-    }
-
-    /**
-     * Document types to return stats for. Mainly affects {@link #indexing(boolean)} when
-     * enabled, returning specific indexing stats for those types.
-     */
-    public String[] types() {
-        return this.flags.types();
-    }
-
     /**
      * Sets specific search group stats to retrieve the stats for. Mainly affects search
      * when enabled.

+ 0 - 9
server/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsRequestBuilder.java

@@ -54,15 +54,6 @@ public class IndicesStatsRequestBuilder
         return this;
     }
 
-    /**
-     * Document types to return stats for. Mainly affects {@link #setIndexing(boolean)} when
-     * enabled, returning specific indexing stats for those types.
-     */
-    public IndicesStatsRequestBuilder setTypes(String... types) {
-        request.types(types);
-        return this;
-    }
-
     public IndicesStatsRequestBuilder setGroups(String... groups) {
         request.groups(groups);
         return this;

+ 1 - 1
server/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java

@@ -237,7 +237,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
                 if (indexServiceOrNull !=  null) {
                     IndexShard shard = indexService.getShardOrNull(shardId.getId());
                     if (shard != null) {
-                        shard.noopUpdate(request.type());
+                        shard.noopUpdate();
                     }
                 }
                 listener.onResponse(update);

+ 4 - 6
server/src/main/java/org/elasticsearch/index/shard/IndexShard.java

@@ -985,7 +985,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
         return getEngine().getSeqNoStats(replicationTracker.getGlobalCheckpoint());
     }
 
-    public IndexingStats indexingStats(String... types) {
+    public IndexingStats indexingStats() {
         Engine engine = getEngineOrNull();
         final boolean throttled;
         final long throttleTimeInMillis;
@@ -996,7 +996,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
             throttled = engine.isThrottled();
             throttleTimeInMillis = engine.getIndexThrottleTimeInMillis();
         }
-        return internalIndexingStats.stats(throttled, throttleTimeInMillis, types);
+        return internalIndexingStats.stats(throttled, throttleTimeInMillis);
     }
 
     public SearchStats searchStats(String... groups) {
@@ -2387,11 +2387,9 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
 
     /**
      * Should be called for each no-op update operation to increment relevant statistics.
-     *
-     * @param type the doc type of the update
      */
-    public void noopUpdate(String type) {
-        internalIndexingStats.noopUpdate(type);
+    public void noopUpdate() {
+        internalIndexingStats.noopUpdate();
     }
 
     public void maybeCheckIndex() {

+ 10 - 45
server/src/main/java/org/elasticsearch/index/shard/IndexingStats.java

@@ -19,7 +19,7 @@
 
 package org.elasticsearch.index.shard;
 
-import org.elasticsearch.common.Nullable;
+import org.elasticsearch.Version;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
@@ -27,9 +27,9 @@ import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.ToXContentFragment;
 import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.index.mapper.MapperService;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.Map;
 
 public class IndexingStats implements Writeable, ToXContentFragment {
@@ -182,47 +182,30 @@ public class IndexingStats implements Writeable, ToXContentFragment {
 
     private final Stats totalStats;
 
-    @Nullable
-    private Map<String, Stats> typeStats;
-
     public IndexingStats() {
         totalStats = new Stats();
     }
 
     public IndexingStats(StreamInput in) throws IOException {
         totalStats = new Stats(in);
-        if (in.readBoolean()) {
-            typeStats = in.readMap(StreamInput::readString, Stats::new);
+        if (in.getVersion().before(Version.V_8_0_0)) {
+            if (in.readBoolean()) {
+                Map<String, Stats> typeStats = in.readMap(StreamInput::readString, Stats::new);
+                assert typeStats.size() == 1;
+                assert typeStats.containsKey(MapperService.SINGLE_MAPPING_NAME);
+            }
         }
     }
 
-    public IndexingStats(Stats totalStats, @Nullable Map<String, Stats> typeStats) {
+    public IndexingStats(Stats totalStats) {
         this.totalStats = totalStats;
-        this.typeStats = typeStats;
     }
 
     public void add(IndexingStats indexingStats) {
-        add(indexingStats, true);
-    }
-
-    public void add(IndexingStats indexingStats, boolean includeTypes) {
         if (indexingStats == null) {
             return;
         }
         addTotals(indexingStats);
-        if (includeTypes && indexingStats.typeStats != null && !indexingStats.typeStats.isEmpty()) {
-            if (typeStats == null) {
-                typeStats = new HashMap<>(indexingStats.typeStats.size());
-            }
-            for (Map.Entry<String, Stats> entry : indexingStats.typeStats.entrySet()) {
-                Stats stats = typeStats.get(entry.getKey());
-                if (stats == null) {
-                    typeStats.put(entry.getKey(), entry.getValue());
-                } else {
-                    stats.add(entry.getValue());
-                }
-            }
-        }
     }
 
     public void addTotals(IndexingStats indexingStats) {
@@ -236,31 +219,16 @@ public class IndexingStats implements Writeable, ToXContentFragment {
         return this.totalStats;
     }
 
-    @Nullable
-    public Map<String, Stats> getTypeStats() {
-        return this.typeStats;
-    }
-
     @Override
     public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException {
         builder.startObject(Fields.INDEXING);
         totalStats.toXContent(builder, params);
-        if (typeStats != null && !typeStats.isEmpty()) {
-            builder.startObject(Fields.TYPES);
-            for (Map.Entry<String, Stats> entry : typeStats.entrySet()) {
-                builder.startObject(entry.getKey());
-                entry.getValue().toXContent(builder, params);
-                builder.endObject();
-            }
-            builder.endObject();
-        }
         builder.endObject();
         return builder;
     }
 
     static final class Fields {
         static final String INDEXING = "indexing";
-        static final String TYPES = "types";
         static final String INDEX_TOTAL = "index_total";
         static final String INDEX_TIME = "index_time";
         static final String INDEX_TIME_IN_MILLIS = "index_time_in_millis";
@@ -279,11 +247,8 @@ public class IndexingStats implements Writeable, ToXContentFragment {
     @Override
     public void writeTo(StreamOutput out) throws IOException {
         totalStats.writeTo(out);
-        if (typeStats == null || typeStats.isEmpty()) {
+        if (out.getVersion().before(Version.V_8_0_0)) {
             out.writeBoolean(false);
-        } else {
-            out.writeBoolean(true);
-            out.writeMap(typeStats, StreamOutput::writeString, (stream, stats) -> stats.writeTo(stream));
         }
     }
 }

+ 4 - 51
server/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java

@@ -21,54 +21,32 @@ package org.elasticsearch.index.shard;
 
 import org.elasticsearch.common.metrics.CounterMetric;
 import org.elasticsearch.common.metrics.MeanMetric;
-import org.elasticsearch.common.regex.Regex;
-import org.elasticsearch.common.util.Maps;
 import org.elasticsearch.index.engine.Engine;
 
-import java.util.HashMap;
-import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-import static java.util.Collections.emptyMap;
-
 /**
  * Internal class that maintains relevant indexing statistics / metrics.
  * @see IndexShard
  */
 final class InternalIndexingStats implements IndexingOperationListener {
+
     private final StatsHolder totalStats = new StatsHolder();
-    private volatile Map<String, StatsHolder> typesStats = emptyMap();
 
     /**
      * Returns the stats, including type specific stats. If the types are null/0 length, then nothing
      * is returned for them. If they are set, then only types provided will be returned, or
      * {@code _all} for all types.
      */
-    IndexingStats stats(boolean isThrottled, long currentThrottleInMillis, String... types) {
+    IndexingStats stats(boolean isThrottled, long currentThrottleInMillis) {
         IndexingStats.Stats total = totalStats.stats(isThrottled, currentThrottleInMillis);
-        Map<String, IndexingStats.Stats> typesSt = null;
-        if (types != null && types.length > 0) {
-            typesSt = new HashMap<>(typesStats.size());
-            if (types.length == 1 && types[0].equals("_all")) {
-                for (Map.Entry<String, StatsHolder> entry : typesStats.entrySet()) {
-                    typesSt.put(entry.getKey(), entry.getValue().stats(isThrottled, currentThrottleInMillis));
-                }
-            } else {
-                for (Map.Entry<String, StatsHolder> entry : typesStats.entrySet()) {
-                    if (Regex.simpleMatch(types, entry.getKey())) {
-                        typesSt.put(entry.getKey(), entry.getValue().stats(isThrottled, currentThrottleInMillis));
-                    }
-                }
-            }
-        }
-        return new IndexingStats(total, typesSt);
+        return new IndexingStats(total);
     }
 
     @Override
     public Engine.Index preIndex(ShardId shardId, Engine.Index operation) {
         if (operation.origin().isRecovery() == false) {
             totalStats.indexCurrent.inc();
-            typeStats(operation.type()).indexCurrent.inc();
         }
         return operation;
     }
@@ -81,9 +59,6 @@ final class InternalIndexingStats implements IndexingOperationListener {
                     long took = result.getTook();
                     totalStats.indexMetric.inc(took);
                     totalStats.indexCurrent.dec();
-                    StatsHolder typeStats = typeStats(index.type());
-                    typeStats.indexMetric.inc(took);
-                    typeStats.indexCurrent.dec();
                 }
                 break;
             case FAILURE:
@@ -98,9 +73,7 @@ final class InternalIndexingStats implements IndexingOperationListener {
     public void postIndex(ShardId shardId, Engine.Index index, Exception ex) {
         if (!index.origin().isRecovery()) {
             totalStats.indexCurrent.dec();
-            typeStats(index.type()).indexCurrent.dec();
             totalStats.indexFailed.inc();
-            typeStats(index.type()).indexFailed.inc();
         }
     }
 
@@ -108,7 +81,6 @@ final class InternalIndexingStats implements IndexingOperationListener {
     public Engine.Delete preDelete(ShardId shardId, Engine.Delete delete) {
         if (!delete.origin().isRecovery()) {
             totalStats.deleteCurrent.inc();
-            typeStats(delete.type()).deleteCurrent.inc();
         }
         return delete;
 
@@ -122,9 +94,6 @@ final class InternalIndexingStats implements IndexingOperationListener {
                     long took = result.getTook();
                     totalStats.deleteMetric.inc(took);
                     totalStats.deleteCurrent.dec();
-                    StatsHolder typeStats = typeStats(delete.type());
-                    typeStats.deleteMetric.inc(took);
-                    typeStats.deleteCurrent.dec();
                 }
                 break;
             case FAILURE:
@@ -139,27 +108,11 @@ final class InternalIndexingStats implements IndexingOperationListener {
     public void postDelete(ShardId shardId, Engine.Delete delete, Exception ex) {
         if (!delete.origin().isRecovery()) {
             totalStats.deleteCurrent.dec();
-            typeStats(delete.type()).deleteCurrent.dec();
         }
     }
 
-    public void noopUpdate(String type) {
+    void noopUpdate() {
         totalStats.noopUpdates.inc();
-        typeStats(type).noopUpdates.inc();
-    }
-
-    private StatsHolder typeStats(String type) {
-        StatsHolder stats = typesStats.get(type);
-        if (stats == null) {
-            synchronized (this) {
-                stats = typesStats.get(type);
-                if (stats == null) {
-                    stats = new StatsHolder();
-                    typesStats = Maps.copyMapWithAddedEntry(typesStats, type, stats);
-                }
-            }
-        }
-        return stats;
     }
 
     static class StatsHolder {

+ 0 - 3
server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestNodesStatsAction.java

@@ -174,9 +174,6 @@ public class RestNodesStatsAction extends BaseRestHandler {
         if (nodesStatsRequest.indices().isSet(Flag.Search) && (request.hasParam("groups"))) {
             nodesStatsRequest.indices().groups(request.paramAsStringArray("groups", null));
         }
-        if (nodesStatsRequest.indices().isSet(Flag.Indexing) && (request.hasParam("types"))) {
-            nodesStatsRequest.indices().types(request.paramAsStringArray("types", null));
-        }
         if (nodesStatsRequest.indices().isSet(Flag.Segments)) {
             nodesStatsRequest.indices().includeSegmentFileSizes(request.paramAsBoolean("include_segment_file_sizes", false));
         }

+ 0 - 5
server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestIndicesStatsAction.java

@@ -75,7 +75,6 @@ public class RestIndicesStatsAction extends BaseRestHandler {
             "options changed";
         indicesStatsRequest.indicesOptions(IndicesOptions.fromRequest(request, defaultIndicesOption));
         indicesStatsRequest.indices(Strings.splitStringByCommaToArray(request.param("index")));
-        indicesStatsRequest.types(Strings.splitStringByCommaToArray(request.param("types")));
 
         Set<String> metrics = Strings.tokenizeByCommaToSet(request.param("metric", "_all"));
         // short cut, if no metrics have been specified in URI
@@ -109,10 +108,6 @@ public class RestIndicesStatsAction extends BaseRestHandler {
             indicesStatsRequest.groups(Strings.splitStringByCommaToArray(request.param("groups")));
         }
 
-        if (request.hasParam("types")) {
-            indicesStatsRequest.types(Strings.splitStringByCommaToArray(request.param("types")));
-        }
-
         if (indicesStatsRequest.completion() && (request.hasParam("fields") || request.hasParam("completion_fields"))) {
             indicesStatsRequest.completionFields(
                     request.paramAsStringArray("completion_fields", request.paramAsStringArray("fields", Strings.EMPTY_ARRAY)));

+ 0 - 34
server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java

@@ -853,40 +853,6 @@ public class IndexStatsIT extends ESIntegTestCase {
 
     }
 
-    public void testTypesParam() throws Exception {
-        createIndex("test1");
-        createIndex("test2");
-
-        ensureGreen();
-
-        client().prepareIndex("test1", "bar", Integer.toString(1)).setSource("foo", "bar").execute().actionGet();
-        client().prepareIndex("test2", "baz", Integer.toString(1)).setSource("foo", "bar").execute().actionGet();
-        refresh();
-
-        IndicesStatsRequestBuilder builder = client().admin().indices().prepareStats();
-        IndicesStatsResponse stats = builder.execute().actionGet();
-
-        assertThat(stats.getTotal().indexing.getTotal().getIndexCount(), greaterThan(0L));
-        assertThat(stats.getTotal().indexing.getTypeStats(), is(nullValue()));
-
-        stats = builder.setTypes("bar").execute().actionGet();
-        assertThat(stats.getTotal().indexing.getTypeStats().get("bar").getIndexCount(), greaterThan(0L));
-        assertThat(stats.getTotal().indexing.getTypeStats().containsKey("baz"), is(false));
-
-        stats = builder.setTypes("bar", "baz").execute().actionGet();
-        assertThat(stats.getTotal().indexing.getTypeStats().get("bar").getIndexCount(), greaterThan(0L));
-        assertThat(stats.getTotal().indexing.getTypeStats().get("baz").getIndexCount(), greaterThan(0L));
-
-        stats = builder.setTypes("*").execute().actionGet();
-        assertThat(stats.getTotal().indexing.getTypeStats().get("bar").getIndexCount(), greaterThan(0L));
-        assertThat(stats.getTotal().indexing.getTypeStats().get("baz").getIndexCount(), greaterThan(0L));
-
-        stats = builder.setTypes("*r").execute().actionGet();
-        assertThat(stats.getTotal().indexing.getTypeStats().get("bar").getIndexCount(), greaterThan(0L));
-        assertThat(stats.getTotal().indexing.getTypeStats().containsKey("baz"), is(false));
-
-    }
-
     private static void set(Flag flag, IndicesStatsRequestBuilder builder, boolean set) {
         switch (flag) {
             case Docs:

+ 1 - 1
x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/indices/IndexStatsMonitoringDocTests.java

@@ -329,7 +329,7 @@ public class IndexStatsMonitoringDocTests extends BaseFilteredMonitoringDocTestC
         commonStats.getRefresh().add(new RefreshStats(no, ++iota, no, ++iota, (int) no));
 
         final IndexingStats.Stats indexingStats = new IndexingStats.Stats(++iota, ++iota, no, no, no, no, no, no, false, ++iota);
-        commonStats.getIndexing().add(new IndexingStats(indexingStats, null));
+        commonStats.getIndexing().add(new IndexingStats(indexingStats));
 
         final SearchStats.Stats searchStats = new SearchStats.Stats(++iota, ++iota, no, no, no, no, no, no, no, no, no, no);
         commonStats.getSearch().add(new SearchStats(searchStats, no, null));

+ 1 - 1
x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/indices/IndicesStatsMonitoringDocTests.java

@@ -147,7 +147,7 @@ public class IndicesStatsMonitoringDocTests extends BaseFilteredMonitoringDocTes
         commonStats.getStore().add(new StoreStats(2L));
 
         final IndexingStats.Stats indexingStats = new IndexingStats.Stats(3L, 4L, -1L, -1L, -1L, -1L, -1L, -1L, true, 5L);
-        commonStats.getIndexing().add(new IndexingStats(indexingStats, null));
+        commonStats.getIndexing().add(new IndexingStats(indexingStats));
 
         final SearchStats.Stats searchStats = new SearchStats.Stats(6L, 7L, -1L, -1L, -1L, -1L, -1L, -1L, -1L, -1L, -1L, -1L);
         commonStats.getSearch().add(new SearchStats(searchStats, -1L, null));

+ 1 - 1
x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/node/NodeStatsMonitoringDocTests.java

@@ -294,7 +294,7 @@ public class NodeStatsMonitoringDocTests extends BaseFilteredMonitoringDocTestCa
         indicesCommonStats.getStore().add(new StoreStats(++iota));
 
         final IndexingStats.Stats indexingStats = new IndexingStats.Stats(++iota, ++iota, ++iota, no, no, no, no, no, false, ++iota);
-        indicesCommonStats.getIndexing().add(new IndexingStats(indexingStats, null));
+        indicesCommonStats.getIndexing().add(new IndexingStats(indexingStats));
         indicesCommonStats.getQueryCache().add(new QueryCacheStats(++iota, ++iota, ++iota, ++iota, no));
         indicesCommonStats.getRequestCache().add(new RequestCacheStats(++iota, ++iota, ++iota, ++iota));