Jelajahi Sumber

Stats: Added verbose option to segments api, with full ram tree as first
additional element per segment.

This commit adds a verbose flag to the _segments api. Currently the
only additional information returned when set to true is the full
ram tree from lucene for each segment.

Ryan Ernst 10 tahun lalu
induk
melakukan
f7f99b8dbf

+ 39 - 0
docs/reference/indices/segments.asciidoc

@@ -74,3 +74,42 @@ compound::   Whether the segment is stored in a compound file. When true, this
              means that Lucene merged all files from the segment in a single
              one in order to save file descriptors.
 
+=== Verbose mode
+
+To add additional information that can be used for debugging, use the `verbose` flag.
+
+NOTE: The format of additional verbose information is experimental and can change at any time.
+
+[source,js]
+--------------------------------------------------
+curl -XGET 'http://localhost:9200/test/_segments?verbose=true'
+--------------------------------------------------
+
+Response:
+
+[source,js]
+--------------------------------------------------
+{
+    ...
+        "_3": {
+            ...
+            "ram_tree": [
+                {
+                    "description": "postings [PerFieldPostings(format=1)]",
+                    "size_in_bytes": 2696,
+                    "children": [
+                        {
+                            "description": "format 'Lucene50_0' ...",
+                            "size_in_bytes": 2608,
+                            "children" :[ ... ]
+                        },
+                        ...
+                    ]
+                },
+                ...
+                ]
+          
+        }
+    ...
+}
+--------------------------------------------------

+ 29 - 0
src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentResponse.java

@@ -22,17 +22,21 @@ package org.elasticsearch.action.admin.indices.segments;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
+import org.apache.lucene.util.Accountable;
 import org.elasticsearch.action.ShardOperationFailedException;
 import org.elasticsearch.action.support.broadcast.BroadcastOperationResponse;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.unit.ByteSizeValue;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentBuilderString;
 import org.elasticsearch.index.engine.Segment;
 
 import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -139,6 +143,13 @@ public class IndicesSegmentResponse extends BroadcastOperationResponse implement
                         if (segment.getMergeId() != null) {
                             builder.field(Fields.MERGE_ID, segment.getMergeId());
                         }
+                        if (segment.ramTree != null) {
+                            builder.startArray(Fields.RAM_TREE);
+                            for (Accountable child : segment.ramTree.getChildResources()) {
+                                toXContent(builder, child);
+                            }
+                            builder.endArray();
+                        }
                         builder.endObject();
                     }
                     builder.endObject();
@@ -155,6 +166,21 @@ public class IndicesSegmentResponse extends BroadcastOperationResponse implement
         builder.endObject();
         return builder;
     }
+    
+    static void toXContent(XContentBuilder builder, Accountable tree) throws IOException {
+        builder.startObject();
+        builder.field(Fields.DESCRIPTION, tree.toString());
+        builder.byteSizeField(Fields.SIZE_IN_BYTES, Fields.SIZE, new ByteSizeValue(tree.ramBytesUsed()));
+        Collection<Accountable> children = tree.getChildResources();
+        if (children.isEmpty() == false) {
+            builder.startArray(Fields.CHILDREN);
+            for (Accountable child : children) {
+                toXContent(builder, child);
+            }
+            builder.endArray();
+        }
+        builder.endObject();
+    }
 
     static final class Fields {
         static final XContentBuilderString INDICES = new XContentBuilderString("indices");
@@ -180,5 +206,8 @@ public class IndicesSegmentResponse extends BroadcastOperationResponse implement
         static final XContentBuilderString MERGE_ID = new XContentBuilderString("merge_id");
         static final XContentBuilderString MEMORY = new XContentBuilderString("memory");
         static final XContentBuilderString MEMORY_IN_BYTES = new XContentBuilderString("memory_in_bytes");
+        static final XContentBuilderString RAM_TREE = new XContentBuilderString("ram_tree");
+        static final XContentBuilderString DESCRIPTION = new XContentBuilderString("description");
+        static final XContentBuilderString CHILDREN = new XContentBuilderString("children");
     }
 }

+ 36 - 0
src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentsRequest.java

@@ -22,9 +22,15 @@ package org.elasticsearch.action.admin.indices.segments;
 import org.elasticsearch.action.support.IndicesOptions;
 import org.elasticsearch.action.support.broadcast.BroadcastOperationRequest;
 import org.elasticsearch.common.Strings;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+
+import java.io.IOException;
 
 public class IndicesSegmentsRequest extends BroadcastOperationRequest<IndicesSegmentsRequest> {
 
+    protected boolean verbose = false;
+    
     public IndicesSegmentsRequest() {
         this(Strings.EMPTY_ARRAY);
     }
@@ -33,4 +39,34 @@ public class IndicesSegmentsRequest extends BroadcastOperationRequest<IndicesSeg
         super(indices);
         indicesOptions(IndicesOptions.fromOptions(false, false, true, false));
     }
+
+    /**
+     * <code>true</code> if detailed information about each segment should be returned,
+     * <code>false</code> otherwise.
+     */
+    public boolean verbose() {
+        return verbose;
+    }
+
+    /**
+     * Sets the <code>verbose</code> option.
+     * @see #verbose()
+     */
+    public void verbose(boolean v) {
+        verbose = v;
+    }
+
+    @Override
+    public void writeTo(StreamOutput out) throws IOException {
+        super.writeTo(out);
+        out.writeBoolean(verbose);
+        
+    }
+
+    @Override
+    public void readFrom(StreamInput in) throws IOException {
+        super.readFrom(in);
+        verbose = in.readBoolean();
+    }
+    
 }

+ 5 - 0
src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentsRequestBuilder.java

@@ -31,6 +31,11 @@ public class IndicesSegmentsRequestBuilder extends BroadcastOperationRequestBuil
     public IndicesSegmentsRequestBuilder(IndicesAdminClient indicesClient) {
         super(indicesClient, new IndicesSegmentsRequest());
     }
+    
+    public IndicesSegmentsRequestBuilder setVerbose(boolean verbose) {
+        request.verbose = verbose;
+        return this;
+    }
 
     @Override
     protected void doExecute(ActionListener<IndicesSegmentResponse> listener) {

+ 5 - 2
src/main/java/org/elasticsearch/action/admin/indices/segments/TransportIndicesSegmentsAction.java

@@ -131,16 +131,19 @@ public class TransportIndicesSegmentsAction extends TransportBroadcastOperationA
     protected ShardSegments shardOperation(IndexShardSegmentRequest request) throws ElasticsearchException {
         IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
         IndexShard indexShard = indexService.shardSafe(request.shardId().id());
-        return new ShardSegments(indexShard.routingEntry(), indexShard.engine().segments());
+        return new ShardSegments(indexShard.routingEntry(), indexShard.engine().segments(request.verbose));
     }
 
     static class IndexShardSegmentRequest extends BroadcastShardOperationRequest {
-
+        final boolean verbose;
+        
         IndexShardSegmentRequest() {
+            verbose = false;
         }
 
         IndexShardSegmentRequest(ShardId shardId, IndicesSegmentsRequest request) {
             super(shardId, request);
+            verbose = request.verbose();
         }
     }
 }

+ 1 - 1
src/main/java/org/elasticsearch/index/engine/Engine.java

@@ -103,7 +103,7 @@ public interface Engine extends CloseableComponent {
     /**
      * The list of segments in the engine.
      */
-    List<Segment> segments();
+    List<Segment> segments(boolean verbose);
 
     /**
      * Returns <tt>true</tt> if a refresh is really needed.

+ 43 - 0
src/main/java/org/elasticsearch/index/engine/Segment.java

@@ -19,6 +19,9 @@
 
 package org.elasticsearch.index.engine;
 
+import com.google.common.collect.Iterators;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
@@ -27,6 +30,10 @@ import org.elasticsearch.common.lucene.Lucene;
 import org.elasticsearch.common.unit.ByteSizeValue;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
 
 public class Segment implements Streamable {
 
@@ -41,6 +48,7 @@ public class Segment implements Streamable {
     public Boolean compound = null;
     public String mergeId;
     public long memoryInBytes;
+    public Accountable ramTree = null;
 
     Segment() {
     }
@@ -143,6 +151,10 @@ public class Segment implements Streamable {
         compound = in.readOptionalBoolean();
         mergeId = in.readOptionalString();
         memoryInBytes = in.readLong();
+        if (in.readBoolean()) {
+            // verbose mode
+            ramTree = readRamTree(in);
+        }
     }
 
     @Override
@@ -157,6 +169,37 @@ public class Segment implements Streamable {
         out.writeOptionalBoolean(compound);
         out.writeOptionalString(mergeId);
         out.writeLong(memoryInBytes);
+        
+        boolean verbose = ramTree != null;
+        out.writeBoolean(verbose);
+        if (verbose) {
+            writeRamTree(out, ramTree);
+        }
+    }
+
+    Accountable readRamTree(StreamInput in) throws IOException {
+        final String name = in.readString();
+        final long bytes = in.readVLong();
+        int numChildren = in.readVInt();
+        if (numChildren == 0) {
+            return Accountables.namedAccountable(name, bytes);
+        }
+        List<Accountable> children = new ArrayList(numChildren);
+        while (numChildren-- > 0) {
+            children.add(readRamTree(in));
+        }
+        return Accountables.namedAccountable(name, children, bytes);
+    }
+    
+    // the ram tree is written recursively since the depth is fairly low (5 or 6)
+    void writeRamTree(StreamOutput out, Accountable tree) throws IOException {
+        out.writeString(tree.toString());
+        out.writeVLong(tree.ramBytesUsed());
+        Collection<Accountable> children = tree.getChildResources();
+        out.writeVInt(children.size());
+        for (Accountable child : children) {
+            writeRamTree(out, child);
+        }
     }
 
     @Override

+ 5 - 1
src/main/java/org/elasticsearch/index/engine/internal/InternalEngine.java

@@ -27,6 +27,7 @@ import org.apache.lucene.search.*;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.elasticsearch.ElasticsearchException;
@@ -1266,7 +1267,7 @@ public class InternalEngine implements Engine {
     }
 
     @Override
-    public List<Segment> segments() {
+    public List<Segment> segments(boolean verbose) {
         try (InternalLock _ = readLock.acquire()) {
             ensureOpen();
             Map<String, Segment> segments = new HashMap<>();
@@ -1290,6 +1291,9 @@ public class InternalEngine implements Engine {
                     }
                     final SegmentReader segmentReader = segmentReader(reader.reader());
                     segment.memoryInBytes = segmentReader.ramBytesUsed();
+                    if (verbose) {
+                        segment.ramTree = Accountables.namedAccountable("root", segmentReader);
+                    }
                     // TODO: add more fine grained mem stats values to per segment info here
                     segments.put(info.info.name, segment);
                 }

+ 2 - 2
src/main/java/org/elasticsearch/index/engine/internal/InternalEngineHolder.java

@@ -274,8 +274,8 @@ public class InternalEngineHolder extends AbstractIndexShardComponent implements
     }
 
     @Override
-    public List<Segment> segments() {
-        return engineSafe().segments();
+    public List<Segment> segments(boolean verbose) {
+        return engineSafe().segments(verbose);
     }
 
     @Override

+ 1 - 0
src/main/java/org/elasticsearch/rest/action/admin/indices/segments/RestIndicesSegmentsAction.java

@@ -48,6 +48,7 @@ public class RestIndicesSegmentsAction extends BaseRestHandler {
     @Override
     public void handleRequest(final RestRequest request, final RestChannel channel, final Client client) {
         IndicesSegmentsRequest indicesSegmentsRequest = new IndicesSegmentsRequest(Strings.splitStringByCommaToArray(request.param("index")));
+        indicesSegmentsRequest.verbose(request.paramAsBoolean("verbose", false));
         indicesSegmentsRequest.listenerThreaded(false);
         indicesSegmentsRequest.indicesOptions(IndicesOptions.fromRequest(request, indicesSegmentsRequest.indicesOptions()));
         client.admin().indices().segments(indicesSegmentsRequest, new RestBuilderListener<IndicesSegmentResponse>(channel) {

+ 59 - 0
src/test/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentsRequestTests.java

@@ -0,0 +1,59 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch 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.elasticsearch.action.admin.indices.segments;
+
+import org.elasticsearch.common.settings.ImmutableSettings;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.index.engine.Segment;
+import org.elasticsearch.test.ElasticsearchSingleNodeTest;
+import org.junit.Before;
+
+import java.util.List;
+
+public class IndicesSegmentsRequestTests extends ElasticsearchSingleNodeTest {
+    
+    @Before
+    public void setupIndex() {
+        Settings settings = ImmutableSettings.builder()
+            // don't allow any merges so that the num docs is the expected segments
+            .put("index.merge.policy.segments_per_tier", 1000000f)
+            .build();
+        createIndex("test", settings);
+
+        int numDocs = scaledRandomIntBetween(100, 1000);
+        for (int j = 0; j < numDocs; ++j) {
+            String id = Integer.toString(j);
+            client().prepareIndex("test", "type1", id).setSource("text", "sometext").get();
+        }
+        client().admin().indices().prepareFlush("test").get();
+    }
+
+    public void testBasic() {
+        IndicesSegmentResponse rsp = client().admin().indices().prepareSegments("test").get();
+        List<Segment> segments = rsp.getIndices().get("test").iterator().next().getShards()[0].getSegments();
+        assertNull(segments.get(0).ramTree);
+    }
+    
+    public void testVerbose() {
+        IndicesSegmentResponse rsp = client().admin().indices().prepareSegments("test").setVerbose(true).get();
+        List<Segment> segments = rsp.getIndices().get("test").iterator().next().getShards()[0].getSegments();
+        assertNotNull(segments.get(0).ramTree);
+    }
+}

+ 44 - 14
src/test/java/org/elasticsearch/index/engine/internal/InternalEngineTests.java

@@ -241,7 +241,7 @@ public class InternalEngineTests extends ElasticsearchLuceneTestCase {
 
     @Test
     public void testSegments() throws Exception {
-        List<Segment> segments = engine.segments();
+        List<Segment> segments = engine.segments(false);
         assertThat(segments.isEmpty(), equalTo(true));
         assertThat(engine.segmentsStats().getCount(), equalTo(0l));
         assertThat(engine.segmentsStats().getMemoryInBytes(), equalTo(0l));
@@ -255,7 +255,7 @@ public class InternalEngineTests extends ElasticsearchLuceneTestCase {
         engine.create(new Engine.Create(null, newUid("2"), doc2));
         engine.refresh("test", false);
 
-        segments = engine.segments();
+        segments = engine.segments(false);
         assertThat(segments.size(), equalTo(1));
         SegmentsStats stats = engine.segmentsStats();
         assertThat(stats.getCount(), equalTo(1l));
@@ -269,10 +269,11 @@ public class InternalEngineTests extends ElasticsearchLuceneTestCase {
         assertThat(segments.get(0).getNumDocs(), equalTo(2));
         assertThat(segments.get(0).getDeletedDocs(), equalTo(0));
         assertThat(segments.get(0).isCompound(), equalTo(defaultCompound));
+        assertThat(segments.get(0).ramTree, nullValue());
 
         engine.flush(Engine.FlushType.COMMIT_TRANSLOG, false, false);
 
-        segments = engine.segments();
+        segments = engine.segments(false);
         assertThat(segments.size(), equalTo(1));
         assertThat(engine.segmentsStats().getCount(), equalTo(1l));
         assertThat(segments.get(0).isCommitted(), equalTo(true));
@@ -287,7 +288,7 @@ public class InternalEngineTests extends ElasticsearchLuceneTestCase {
         engine.create(new Engine.Create(null, newUid("3"), doc3));
         engine.refresh("test", false);
 
-        segments = engine.segments();
+        segments = engine.segments(false);
         assertThat(segments.size(), equalTo(2));
         assertThat(engine.segmentsStats().getCount(), equalTo(2l));
         assertThat(engine.segmentsStats().getTermsMemoryInBytes(), greaterThan(stats.getTermsMemoryInBytes()));
@@ -313,7 +314,7 @@ public class InternalEngineTests extends ElasticsearchLuceneTestCase {
         engine.delete(new Engine.Delete("test", "1", newUid("1")));
         engine.refresh("test", false);
 
-        segments = engine.segments();
+        segments = engine.segments(false);
         assertThat(segments.size(), equalTo(2));
         assertThat(engine.segmentsStats().getCount(), equalTo(2l));
         assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true));
@@ -334,7 +335,7 @@ public class InternalEngineTests extends ElasticsearchLuceneTestCase {
         engine.create(new Engine.Create(null, newUid("4"), doc4));
         engine.refresh("test", false);
 
-        segments = engine.segments();
+        segments = engine.segments(false);
         assertThat(segments.size(), equalTo(3));
         assertThat(engine.segmentsStats().getCount(), equalTo(3l));
         assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true));
@@ -356,6 +357,33 @@ public class InternalEngineTests extends ElasticsearchLuceneTestCase {
         assertThat(segments.get(2).getDeletedDocs(), equalTo(0));
         assertThat(segments.get(2).isCompound(), equalTo(true));
     }
+    
+    public void testVerboseSegments() throws Exception {
+        List<Segment> segments = engine.segments(true);
+        assertThat(segments.isEmpty(), equalTo(true));
+        
+        ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), Lucene.STANDARD_ANALYZER, B_1, false);
+        engine.create(new Engine.Create(null, newUid("1"), doc));
+        engine.refresh("test", false);
+
+        segments = engine.segments(true);
+        assertThat(segments.size(), equalTo(1));
+        assertThat(segments.get(0).ramTree, notNullValue());
+        
+        ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), Lucene.STANDARD_ANALYZER, B_2, false);
+        engine.create(new Engine.Create(null, newUid("2"), doc2));
+        engine.refresh("test", false);
+        ParsedDocument doc3 = testParsedDocument("3", "3", "test", null, -1, -1, testDocumentWithTextField(), Lucene.STANDARD_ANALYZER, B_3, false);
+        engine.create(new Engine.Create(null, newUid("3"), doc3));
+        engine.refresh("test", false);
+
+        segments = engine.segments(true);
+        assertThat(segments.size(), equalTo(3));
+        assertThat(segments.get(0).ramTree, notNullValue());
+        assertThat(segments.get(1).ramTree, notNullValue());
+        assertThat(segments.get(2).ramTree, notNullValue());
+        
+    }
 
     public void testStartAndAcquireConcurrently() throws IOException {
         ConcurrentMergeSchedulerProvider mergeSchedulerProvider = new ConcurrentMergeSchedulerProvider(shardId, EMPTY_SETTINGS, threadPool, new IndexSettingsService(shardId.index(), EMPTY_SETTINGS));
@@ -419,19 +447,21 @@ public class InternalEngineTests extends ElasticsearchLuceneTestCase {
         Engine.Index index = new Engine.Index(null, newUid("1"), doc);
         engine.index(index);
         engine.flush(Engine.FlushType.COMMIT_TRANSLOG, false, false);
-        assertThat(engine.segments().size(), equalTo(1));
+        assertThat(engine.segments(false).size(), equalTo(1));
         index = new Engine.Index(null, newUid("2"), doc);
         engine.index(index);
         engine.flush(Engine.FlushType.COMMIT_TRANSLOG, false, false);
-        assertThat(engine.segments().size(), equalTo(2));
-        for (Segment segment : engine.segments()) {
+        List<Segment> segments = engine.segments(false);
+        assertThat(segments.size(), equalTo(2));
+        for (Segment segment : segments) {
             assertThat(segment.getMergeId(), nullValue());
         }
         index = new Engine.Index(null, newUid("3"), doc);
         engine.index(index);
         engine.flush(Engine.FlushType.COMMIT_TRANSLOG, false, false);
-        assertThat(engine.segments().size(), equalTo(3));
-        for (Segment segment : engine.segments()) {
+        segments = engine.segments(false);
+        assertThat(segments.size(), equalTo(3));
+        for (Segment segment : segments) {
             assertThat(segment.getMergeId(), nullValue());
         }
 
@@ -440,7 +470,7 @@ public class InternalEngineTests extends ElasticsearchLuceneTestCase {
         engine.forceMerge(false, false);
         waitTillMerge.get().await();
 
-        for (Segment segment : engine.segments()) {
+        for (Segment segment : engine.segments(false)) {
             assertThat(segment.getMergeId(), notNullValue());
         }
 
@@ -453,7 +483,7 @@ public class InternalEngineTests extends ElasticsearchLuceneTestCase {
         // now, optimize and wait for merges, see that we have no merge flag
         engine.forceMerge(true, true);
 
-        for (Segment segment : engine.segments()) {
+        for (Segment segment : engine.segments(false)) {
             assertThat(segment.getMergeId(), nullValue());
         }
         // we could have multiple underlying merges, so the generation may increase more than once
@@ -463,7 +493,7 @@ public class InternalEngineTests extends ElasticsearchLuceneTestCase {
         final long gen2 = store.readLastCommittedSegmentsInfo().getGeneration();
         engine.forceMerge(flush, false);
         waitTillMerge.get().await();
-        for (Segment segment : engine.segments()) {
+        for (Segment segment : engine.segments(false)) {
             assertThat(segment.getMergeId(), nullValue());
         }
         waitForMerge.get().countDown();