Browse Source

Expose the ClusterInfo object in the allocation explain output

This adds an optional parameter to the cluster allocation explain API
that will return the cluster info object, `include_disk_info`, the
output looks like:

GET /_cluster/allocation/explain?include_disk_info -d'
{"index": "i", "shard": 0, "primary": false}'

{
  ... other info ...

  "cluster_info" : {
    "nodes" : {
      "7Uws-vL7R6WVm3ZwQA1n5A" : {
        "node_name" : "Kraven the Hunter",
        "least_available" : {
          "path" : "/path/to/data1",
          "total_bytes" : 165999570944,
          "used_bytes" : 118180614144,
          "free_bytes" : 47818956800,
          "free_disk_percent" : 28.80667493781158,
          "used_disk_percent" : 71.19332506218842
        },
        "most_available" : {
          "path" : "/path/to/data2",
          "total_bytes" : 165999570944,
          "used_bytes" : 118180614144,
          "free_bytes" : 47818956800,
          "free_disk_percent" : 28.80667493781158,
          "used_disk_percent" : 71.19332506218842
        }
      }
    },
    "shard_sizes" : {
      "[i][2][p]_bytes" : 0,
      "[i][4][p]_bytes" : 130,
      "[i][1][p]_bytes" : 0,
      "[i][3][p]_bytes" : 0,
      "[i][0][p]_bytes" : 130
    },
    "shard_paths" : {
      "[i][3], node[7Uws-vL7R6WVm3ZwQA1n5A], [P], s[STARTED], a[id=LegZLDniTVaw0Y1urv7s3g]" : "/path/to/data1/nodes/0",
      "[i][1], node[7Uws-vL7R6WVm3ZwQA1n5A], [P], s[STARTED], a[id=lAU_4vf_SKmoRdtg0ACnjQ]" : "/path/to/data1/nodes/0",
      "[i][2], node[7Uws-vL7R6WVm3ZwQA1n5A], [P], s[STARTED], a[id=Aurpeuj7SeGeyPDDpCtRgg]" : "/path/to/data1/nodes/0",
      "[i][0], node[7Uws-vL7R6WVm3ZwQA1n5A], [P], s[STARTED], a[id=Vgg8GlQTQ82C2j6HYBq8DQ]" : "/path/to/data1/nodes/0",
      "[i][4], node[7Uws-vL7R6WVm3ZwQA1n5A], [P], s[STARTED], a[id=t8hQlVSxQe-58fSeaXcAqg]" : "/path/to/data1/nodes/0"
    }
  }
}

Resolves #14405
Lee Hinman 9 years ago
parent
commit
58db63b610

+ 11 - 0
core/src/main/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainRequest.java

@@ -52,6 +52,7 @@ public class ClusterAllocationExplainRequest extends MasterNodeRequest<ClusterAl
     private Integer shard;
     private Boolean primary;
     private boolean includeYesDecisions = false;
+    private boolean includeDiskInfo = false;
 
     /** Explain the first unassigned shard */
     public ClusterAllocationExplainRequest() {
@@ -134,6 +135,16 @@ public class ClusterAllocationExplainRequest extends MasterNodeRequest<ClusterAl
         return this.includeYesDecisions;
     }
 
+    /** {@code true} to include information about the gathered disk information of nodes in the cluster */
+    public void includeDiskInfo(boolean includeDiskInfo) {
+        this.includeDiskInfo = includeDiskInfo;
+    }
+
+    /** Returns true if information about disk usage and shard sizes should also be returned */
+    public boolean includeDiskInfo() {
+        return this.includeDiskInfo;
+    }
+
     @Override
     public String toString() {
         StringBuilder sb = new StringBuilder("ClusterAllocationExplainRequest[");

+ 12 - 0
core/src/main/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainRequestBuilder.java

@@ -53,6 +53,18 @@ public class ClusterAllocationExplainRequestBuilder
         return this;
     }
 
+    /** Whether to include "YES" decider decisions in the response instead of only "NO" decisions */
+    public ClusterAllocationExplainRequestBuilder setIncludeYesDecisions(boolean includeYesDecisions) {
+        request.includeYesDecisions(includeYesDecisions);
+        return this;
+    }
+
+    /** Whether to include information about the gathered disk information of nodes in the cluster */
+    public ClusterAllocationExplainRequestBuilder setIncludeDiskInfo(boolean includeDiskInfo) {
+        request.includeDiskInfo(includeDiskInfo);
+        return this;
+    }
+
     /**
      * Signal that the first unassigned shard should be used
      */

+ 31 - 4
core/src/main/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplanation.java

@@ -19,6 +19,7 @@
 
 package org.elasticsearch.action.admin.cluster.allocation;
 
+import org.elasticsearch.cluster.ClusterInfo;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.routing.UnassignedInfo;
 import org.elasticsearch.common.Nullable;
@@ -48,10 +49,11 @@ public final class ClusterAllocationExplanation implements ToXContent, Writeable
     private final long allocationDelayMillis;
     private final long remainingDelayMillis;
     private final Map<DiscoveryNode, NodeExplanation> nodeExplanations;
+    private final ClusterInfo clusterInfo;
 
     public ClusterAllocationExplanation(ShardId shard, boolean primary, @Nullable String assignedNodeId, long allocationDelayMillis,
                                         long remainingDelayMillis, @Nullable UnassignedInfo unassignedInfo, boolean hasPendingAsyncFetch,
-                                        Map<DiscoveryNode, NodeExplanation> nodeExplanations) {
+                                        Map<DiscoveryNode, NodeExplanation> nodeExplanations, @Nullable ClusterInfo clusterInfo) {
         this.shard = shard;
         this.primary = primary;
         this.hasPendingAsyncFetch = hasPendingAsyncFetch;
@@ -60,6 +62,7 @@ public final class ClusterAllocationExplanation implements ToXContent, Writeable
         this.allocationDelayMillis = allocationDelayMillis;
         this.remainingDelayMillis = remainingDelayMillis;
         this.nodeExplanations = nodeExplanations;
+        this.clusterInfo = clusterInfo;
     }
 
     public ClusterAllocationExplanation(StreamInput in) throws IOException {
@@ -78,6 +81,11 @@ public final class ClusterAllocationExplanation implements ToXContent, Writeable
             nodeToExplanation.put(nodeExplanation.getNode(), nodeExplanation);
         }
         this.nodeExplanations = nodeToExplanation;
+        if (in.readBoolean()) {
+            this.clusterInfo = new ClusterInfo(in);
+        } else {
+            this.clusterInfo = null;
+        }
     }
 
     @Override
@@ -94,6 +102,12 @@ public final class ClusterAllocationExplanation implements ToXContent, Writeable
         for (NodeExplanation explanation : this.nodeExplanations.values()) {
             explanation.writeTo(out);
         }
+        if (this.clusterInfo != null) {
+            out.writeBoolean(true);
+            this.clusterInfo.writeTo(out);
+        } else {
+            out.writeBoolean(false);
+        }
     }
 
     /** Return the shard that the explanation is about */
@@ -143,6 +157,12 @@ public final class ClusterAllocationExplanation implements ToXContent, Writeable
         return this.nodeExplanations;
     }
 
+    /** Return the cluster disk info for the cluster or null if none available */
+    @Nullable
+    public ClusterInfo getClusterInfo() {
+        return this.clusterInfo;
+    }
+
     public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
         builder.startObject(); {
             builder.startObject("shard"); {
@@ -164,11 +184,18 @@ public final class ClusterAllocationExplanation implements ToXContent, Writeable
                 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.startObject("nodes"); {
+                for (NodeExplanation explanation : nodeExplanations.values()) {
+                    explanation.toXContent(builder, params);
+                }
             }
             builder.endObject(); // end nodes
+            if (this.clusterInfo != null) {
+                builder.startObject("cluster_info"); {
+                    this.clusterInfo.toXContent(builder, params);
+                }
+                builder.endObject(); // end "cluster_info"
+            }
         }
         builder.endObject(); // end wrapping object
         return builder;

+ 8 - 5
core/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportClusterAllocationExplainAction.java

@@ -28,6 +28,7 @@ import org.elasticsearch.action.admin.indices.shards.IndicesShardStoresResponse;
 import org.elasticsearch.action.admin.indices.shards.TransportIndicesShardStoresAction;
 import org.elasticsearch.action.support.ActionFilters;
 import org.elasticsearch.action.support.master.TransportMasterNodeAction;
+import org.elasticsearch.cluster.ClusterInfo;
 import org.elasticsearch.cluster.ClusterInfoService;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.block.ClusterBlockException;
@@ -219,7 +220,7 @@ public class TransportClusterAllocationExplainAction
     public static ClusterAllocationExplanation explainShard(ShardRouting shard, RoutingAllocation allocation, RoutingNodes routingNodes,
                                                             boolean includeYesDecisions, ShardsAllocator shardAllocator,
                                                             List<IndicesShardStoresResponse.StoreStatus> shardStores,
-                                                            GatewayAllocator gatewayAllocator) {
+                                                            GatewayAllocator gatewayAllocator, ClusterInfo clusterInfo) {
         // don't short circuit deciders, we want a full explanation
         allocation.debugDecision(true);
         // get the existing unassigned info if available
@@ -262,16 +263,17 @@ public class TransportClusterAllocationExplainAction
             explanations.put(node, nodeExplanation);
         }
         return new ClusterAllocationExplanation(shard.shardId(), shard.primary(),
-            shard.currentNodeId(), allocationDelayMillis, remainingDelayMillis, ui,
-            gatewayAllocator.hasFetchPending(shard.shardId(), shard.primary()), explanations);
+                shard.currentNodeId(), allocationDelayMillis, remainingDelayMillis, ui,
+                gatewayAllocator.hasFetchPending(shard.shardId(), shard.primary()), explanations, clusterInfo);
     }
 
     @Override
     protected void masterOperation(final ClusterAllocationExplainRequest request, final ClusterState state,
                                    final ActionListener<ClusterAllocationExplainResponse> listener) {
         final RoutingNodes routingNodes = state.getRoutingNodes();
+        final ClusterInfo clusterInfo = clusterInfoService.getClusterInfo();
         final RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, state,
-                clusterInfoService.getClusterInfo(), System.nanoTime(), false);
+                clusterInfo, System.nanoTime(), false);
 
         ShardRouting foundShard = null;
         if (request.useAnyUnassignedShard()) {
@@ -318,7 +320,8 @@ public class TransportClusterAllocationExplainAction
                         shardStoreResponse.getStoreStatuses().get(shardRouting.getIndexName());
                 List<IndicesShardStoresResponse.StoreStatus> shardStoreStatus = shardStatuses.get(shardRouting.id());
                 ClusterAllocationExplanation cae = explainShard(shardRouting, allocation, routingNodes,
-                        request.includeYesDecisions(), shardAllocator, shardStoreStatus, gatewayAllocator);
+                        request.includeYesDecisions(), shardAllocator, shardStoreStatus, gatewayAllocator,
+                        request.includeDiskInfo() ? clusterInfo : null);
                 listener.onResponse(new ClusterAllocationExplainResponse(cae));
             }
 

+ 110 - 1
core/src/main/java/org/elasticsearch/cluster/ClusterInfo.java

@@ -19,16 +19,26 @@
 
 package org.elasticsearch.cluster;
 
+import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
 import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.common.collect.ImmutableOpenMap;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.xcontent.ToXContent;
+import org.elasticsearch.common.xcontent.ToXContent.Params;
+import org.elasticsearch.common.xcontent.XContentBuilder;
 
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
 /**
  * ClusterInfo is an object representing a map of nodes to {@link DiskUsage}
  * and a map of shard ids to shard sizes, see
  * <code>InternalClusterInfoService.shardIdentifierFromRouting(String)</code>
  * for the key used in the shardSizes map
  */
-public class ClusterInfo {
+public class ClusterInfo implements ToXContent, Writeable {
     private final ImmutableOpenMap<String, DiskUsage> leastAvailableSpaceUsage;
     private final ImmutableOpenMap<String, DiskUsage> mostAvailableSpaceUsage;
     final ImmutableOpenMap<String, Long> shardSizes;
@@ -57,6 +67,105 @@ public class ClusterInfo {
         this.routingToDataPath = routingToDataPath;
     }
 
+    public ClusterInfo(StreamInput in) throws IOException {
+        int size = in.readInt();
+        Map<String, DiskUsage> leastMap = new HashMap<>(size);
+        for (int i = 0; i < size; i++) {
+            leastMap.put(in.readString(), new DiskUsage(in));
+        }
+
+        size = in.readInt();
+        Map<String, DiskUsage> mostMap = new HashMap<>(size);
+        for (int i = 0; i < size; i++) {
+            mostMap.put(in.readString(), new DiskUsage(in));
+        }
+
+        size = in.readInt();
+        Map<String, Long> sizeMap = new HashMap<>(size);
+        for (int i = 0; i < size; i++) {
+            sizeMap.put(in.readString(), in.readLong());
+        }
+
+        size = in.readInt();
+        Map<ShardRouting, String> routingMap = new HashMap<>(size);
+        for (int i = 0; i < size; i++) {
+            routingMap.put(new ShardRouting(in), in.readString());
+        }
+
+        ImmutableOpenMap.Builder<String, DiskUsage> leastBuilder = ImmutableOpenMap.builder();
+        this.leastAvailableSpaceUsage = leastBuilder.putAll(leastMap).build();
+        ImmutableOpenMap.Builder<String, DiskUsage> mostBuilder = ImmutableOpenMap.builder();
+        this.mostAvailableSpaceUsage = mostBuilder.putAll(mostMap).build();
+        ImmutableOpenMap.Builder<String, Long> sizeBuilder = ImmutableOpenMap.builder();
+        this.shardSizes = sizeBuilder.putAll(sizeMap).build();
+        ImmutableOpenMap.Builder<ShardRouting, String> routingBuilder = ImmutableOpenMap.builder();
+        this.routingToDataPath = routingBuilder.putAll(routingMap).build();
+    }
+
+    @Override
+    public void writeTo(StreamOutput out) throws IOException {
+        out.writeVInt(this.leastAvailableSpaceUsage.size());
+        for (ObjectObjectCursor<String, DiskUsage> c : this.leastAvailableSpaceUsage) {
+            out.writeString(c.key);
+            c.value.writeTo(out);
+        }
+        out.writeVInt(this.mostAvailableSpaceUsage.size());
+        for (ObjectObjectCursor<String, DiskUsage> c : this.mostAvailableSpaceUsage) {
+            out.writeString(c.key);
+            c.value.writeTo(out);
+        }
+        out.writeVInt(this.shardSizes.size());
+        for (ObjectObjectCursor<String, Long> c : this.shardSizes) {
+            out.writeString(c.key);
+            if (c.value == null) {
+                out.writeLong(-1);
+            } else {
+                out.writeLong(c.value);
+            }
+        }
+        out.writeVInt(this.routingToDataPath.size());
+        for (ObjectObjectCursor<ShardRouting, String> c : this.routingToDataPath) {
+            c.key.writeTo(out);
+            out.writeString(c.value);
+        }
+    }
+
+    public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
+        builder.startObject("nodes"); {
+            for (ObjectObjectCursor<String, DiskUsage> c : this.leastAvailableSpaceUsage) {
+                builder.startObject(c.key); { // node
+                    builder.field("node_name", c.value.getNodeName());
+                    builder.startObject("least_available"); {
+                        c.value.toShortXContent(builder, params);
+                    }
+                    builder.endObject(); // end "least_available"
+                    builder.startObject("most_available"); {
+                        DiskUsage most = this.mostAvailableSpaceUsage.get(c.key);
+                        if (most != null) {
+                            most.toShortXContent(builder, params);
+                        }
+                    }
+                    builder.endObject(); // end "most_available"
+                }
+                builder.endObject(); // end $nodename
+            }
+        }
+        builder.endObject(); // end "nodes"
+        builder.startObject("shard_sizes"); {
+            for (ObjectObjectCursor<String, Long> c : this.shardSizes) {
+                builder.byteSizeField(c.key + "_bytes", c.key, c.value);
+            }
+        }
+        builder.endObject(); // end "shard_sizes"
+        builder.startObject("shard_paths"); {
+            for (ObjectObjectCursor<ShardRouting, String> c : this.routingToDataPath) {
+                builder.field(c.key.toString(), c.value);
+            }
+        }
+        builder.endObject(); // end "shard_paths"
+        return builder;
+    }
+
     /**
      * Returns a node id to disk usage mapping for the path that has the least available space on the node.
      */

+ 46 - 1
core/src/main/java/org/elasticsearch/cluster/DiskUsage.java

@@ -20,12 +20,19 @@
 package org.elasticsearch.cluster;
 
 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;
 import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.common.xcontent.ToXContent;
+import org.elasticsearch.common.xcontent.ToXContent.Params;
+import org.elasticsearch.common.xcontent.XContentBuilder;
 
+import java.io.IOException;
 /**
  * Encapsulation class used to represent the amount of disk used on a node.
  */
-public class DiskUsage {
+public class DiskUsage implements ToXContent, Writeable {
     final String nodeId;
     final String nodeName;
     final String path;
@@ -44,6 +51,44 @@ public class DiskUsage {
         this.path = path;
     }
 
+    public DiskUsage(StreamInput in) throws IOException {
+        this.nodeId = in.readString();
+        this.nodeName = in.readString();
+        this.path = in.readString();
+        this.totalBytes = in.readVLong();
+        this.freeBytes = in.readVLong();
+    }
+
+    @Override
+    public void writeTo(StreamOutput out) throws IOException {
+        out.writeString(this.nodeId);
+        out.writeString(this.nodeName);
+        out.writeString(this.path);
+        out.writeVLong(this.totalBytes);
+        out.writeVLong(this.freeBytes);
+    }
+
+    private static double truncatePercent(double pct) {
+        return Math.round(pct * 10.0) / 10.0;
+    }
+
+    public XContentBuilder toShortXContent(XContentBuilder builder, Params params) throws IOException {
+        builder.field("path", this.path);
+        builder.byteSizeField("total_bytes", "total", this.totalBytes);
+        builder.byteSizeField("used_bytes", "used", this.getUsedBytes());
+        builder.byteSizeField("free_bytes", "free", this.freeBytes);
+        builder.field("free_disk_percent", truncatePercent(this.getFreeDiskAsPercentage()));
+        builder.field("used_disk_percent", truncatePercent(this.getUsedDiskAsPercentage()));
+        return builder;
+    }
+
+    public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
+        builder.field("node_id", this.nodeId);
+        builder.field("node_name", this.nodeName);
+        builder = toShortXContent(builder, params);
+        return builder;
+    }
+
     public String getNodeId() {
         return nodeId;
     }

+ 1 - 0
core/src/main/java/org/elasticsearch/rest/action/admin/cluster/allocation/RestClusterAllocationExplainAction.java

@@ -76,6 +76,7 @@ public class RestClusterAllocationExplainAction extends BaseRestHandler {
 
         try {
             req.includeYesDecisions(request.paramAsBoolean("include_yes_decisions", false));
+            req.includeDiskInfo(request.paramAsBoolean("include_disk_info", false));
             client.admin().cluster().allocationExplain(req, new RestBuilderListener<ClusterAllocationExplainResponse>(channel) {
                 @Override
                 public RestResponse buildResponse(ClusterAllocationExplainResponse response, XContentBuilder builder) throws Exception {

+ 2 - 2
core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplanationTests.java

@@ -202,7 +202,7 @@ public final class ClusterAllocationExplanationTests extends ESTestCase {
                 yesDecision, nodeWeight, storeStatus, "", activeAllocationIds, false);
         nodeExplanations.put(ne.getNode(), ne);
         ClusterAllocationExplanation cae = new ClusterAllocationExplanation(shard, true,
-                "assignedNode", allocationDelay, remainingDelay, null, false, nodeExplanations);
+                "assignedNode", allocationDelay, remainingDelay, null, false, nodeExplanations, null);
         BytesStreamOutput out = new BytesStreamOutput();
         cae.writeTo(out);
         StreamInput in = StreamInput.wrap(out.bytes());
@@ -240,7 +240,7 @@ public final class ClusterAllocationExplanationTests extends ESTestCase {
         Map<DiscoveryNode, NodeExplanation> nodeExplanations = new HashMap<>(1);
         nodeExplanations.put(ne.getNode(), ne);
         ClusterAllocationExplanation cae = new ClusterAllocationExplanation(shardId, true,
-                "assignedNode", 42, 42, null, false, nodeExplanations);
+                "assignedNode", 42, 42, null, false, nodeExplanations, null);
         XContentBuilder builder = XContentFactory.jsonBuilder();
         cae.toXContent(builder, ToXContent.EMPTY_PARAMS);
         assertEquals("{\"shard\":{\"index\":\"foo\",\"index_uuid\":\"uuid\",\"id\":0,\"primary\":true},\"assigned\":true," +

+ 10 - 1
docs/reference/cluster/allocation-explain.asciidoc

@@ -183,10 +183,19 @@ shard it finds by sending an empty body, such as:
 $ curl -XGET 'http://localhost:9200/_cluster/allocation/explain'
 --------------------------------------------------
 
-And if you would like to include all decisions that were factored into the final
+If you would like to include all decisions that were factored into the final
 decision, the `include_yes_decisions` parameter will return all decisions:
 
 [source,js]
 --------------------------------------------------
 $ curl -XGET 'http://localhost:9200/_cluster/allocation/explain?include_yes_decisions=true'
 --------------------------------------------------
+
+Additionally, you can return information gathered by the cluster info service
+about disk usage and shard sizes by setting the `include_disk_info` parameter to
+`true`:
+
+[source,js]
+--------------------------------------------------
+$ curl -XGET 'http://localhost:9200/_cluster/allocation/explain?include_disk_info=true'
+--------------------------------------------------

+ 4 - 0
rest-api-spec/src/main/resources/rest-api-spec/api/cluster.allocation_explain.json

@@ -10,6 +10,10 @@
         "include_yes_decisions": {
           "type": "boolean",
           "description": "Return 'YES' decisions in explanation (default: false)"
+        },
+        "include_disk_info": {
+          "type": "boolean",
+          "description": "Return information about disk usage and shard sizes (default: false)"
         }
       }
     },

+ 3 - 1
rest-api-spec/src/main/resources/rest-api-spec/test/cluster.allocation_explain/10_basic.yaml

@@ -60,7 +60,8 @@
   - set: {master_node: node_id}
 
   - do:
-      cluster.allocation_explain: {}
+      cluster.allocation_explain:
+        include_disk_info: true
 
   - match: { assigned: false }
   - match: { unassigned_info.reason: "INDEX_CREATED" }
@@ -68,6 +69,7 @@
   - match: { shard.index: "test" }
   - match: { shard.id: 0 }
   - match: { shard.primary: false }
+  - is_true: cluster_info
   # - is_true: nodes.$node_id.node_name
   # - match: { nodes.$node_id.node_attributes.testattr: "test" }
   # - match: { nodes.$node_id.node_attributes.portsfile: "true" }