You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by ab...@apache.org on 2023/04/14 09:53:48 UTC

[druid] branch master updated: Add start_time column to sys.servers (#13358)

This is an automated email from the ASF dual-hosted git repository.

abhishek pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/druid.git


The following commit(s) were added to refs/heads/master by this push:
     new e3c160f2f2 Add start_time column to sys.servers  (#13358)
e3c160f2f2 is described below

commit e3c160f2f2a6a3a19f571ddaf18f0c998fdf42f5
Author: Atul Mohan <at...@gmail.com>
AuthorDate: Fri Apr 14 02:53:34 2023 -0700

    Add start_time column to sys.servers  (#13358)
    
    Adds a new column start_time to sys.servers that captures the time at which the server was added to the cluster.
---
 docs/querying/sql-metadata-tables.md               |   2 +-
 .../security/AbstractAuthConfigurationTest.java    |   9 +-
 .../results/auth_test_sys_schema_servers.json      |   6 +-
 .../apache/druid/discovery/DiscoveryDruidNode.java |  30 +++++-
 .../druid/discovery/DiscoveryDruidNodeTest.java    |   1 +
 .../java/org/apache/druid/cli/CliOverlord.java     |  12 +--
 .../druid/sql/calcite/schema/SystemSchema.java     |  10 +-
 .../druid/sql/calcite/schema/SystemSchemaTest.java | 103 ++++++++++++++-------
 .../__snapshots__/services-view.spec.tsx.snap      |  11 +++
 .../src/views/services-view/services-view.spec.tsx |   2 +
 .../src/views/services-view/services-view.tsx      |  64 +++++++++----
 website/.spelling                                  |   1 +
 12 files changed, 181 insertions(+), 70 deletions(-)

diff --git a/docs/querying/sql-metadata-tables.md b/docs/querying/sql-metadata-tables.md
index fbf2eb0594..3d60936595 100644
--- a/docs/querying/sql-metadata-tables.md
+++ b/docs/querying/sql-metadata-tables.md
@@ -208,7 +208,7 @@ Servers table lists all discovered servers in the cluster.
 |current_size|BIGINT|Current size of segments in bytes on this server. Only valid for HISTORICAL type, for other types it's 0|
 |max_size|BIGINT|Max size in bytes this server recommends to assign to segments see [druid.server.maxSize](../configuration/index.md#historical-general-configuration). Only valid for HISTORICAL type, for other types it's 0|
 |is_leader|BIGINT|1 if the server is currently the 'leader' (for services which have the concept of leadership), otherwise 0 if the server is not the leader, or the default long value (0 or null depending on `druid.generic.useDefaultValueForNull`) if the server type does not have the concept of leadership|
-
+|start_time|STRING|Timestamp in ISO8601 format when the server was announced in the cluster|
 To retrieve information about all servers, use the query:
 
 ```sql
diff --git a/integration-tests/src/test/java/org/apache/druid/tests/security/AbstractAuthConfigurationTest.java b/integration-tests/src/test/java/org/apache/druid/tests/security/AbstractAuthConfigurationTest.java
index 5c6e027459..1c58679161 100644
--- a/integration-tests/src/test/java/org/apache/druid/tests/security/AbstractAuthConfigurationTest.java
+++ b/integration-tests/src/test/java/org/apache/druid/tests/security/AbstractAuthConfigurationTest.java
@@ -276,7 +276,7 @@ public abstract class AbstractAuthConfigurationTest
     verifySystemSchemaServerQuery(
         adminClient,
         SYS_SCHEMA_SERVERS_QUERY,
-        getServersWithoutCurrentSize(adminServers)
+        getServersWithoutCurrentSizeAndStartTime(adminServers)
     );
 
     LOG.info("Checking sys.server_segments query as admin...");
@@ -767,7 +767,7 @@ public abstract class AbstractAuthConfigurationTest
     String content = responseHolder.getContent();
     List<Map<String, Object>> responseMap = jsonMapper.readValue(content, SYS_SCHEMA_RESULTS_TYPE_REFERENCE);
     if (isServerQuery) {
-      responseMap = getServersWithoutCurrentSize(responseMap);
+      responseMap = getServersWithoutCurrentSizeAndStartTime(responseMap);
     }
     Assert.assertEquals(responseMap, expectedResults);
   }
@@ -914,7 +914,7 @@ public abstract class AbstractAuthConfigurationTest
         SYS_SCHEMA_RESULTS_TYPE_REFERENCE
     );
 
-    adminServers = getServersWithoutCurrentSize(
+    adminServers = getServersWithoutCurrentSizeAndStartTime(
         jsonMapper.readValue(
             fillServersTemplate(
                 config,
@@ -937,13 +937,14 @@ public abstract class AbstractAuthConfigurationTest
    * curr_size on historicals changes because cluster state is not isolated across different
    * integration tests, zero it out for consistent test results
    */
-  protected static List<Map<String, Object>> getServersWithoutCurrentSize(List<Map<String, Object>> servers)
+  protected static List<Map<String, Object>> getServersWithoutCurrentSizeAndStartTime(List<Map<String, Object>> servers)
   {
     return Lists.transform(
         servers,
         (server) -> {
           Map<String, Object> newServer = new HashMap<>(server);
           newServer.put("curr_size", 0);
+          newServer.put("start_time", "0");
           return newServer;
         }
     );
diff --git a/integration-tests/src/test/resources/results/auth_test_sys_schema_servers.json b/integration-tests/src/test/resources/results/auth_test_sys_schema_servers.json
index d27614e2ec..776c2ba2f5 100644
--- a/integration-tests/src/test/resources/results/auth_test_sys_schema_servers.json
+++ b/integration-tests/src/test/resources/results/auth_test_sys_schema_servers.json
@@ -8,7 +8,8 @@
     "tier": "_default_tier",
     "curr_size": 2208932412,
     "max_size": 5000000000,
-    "is_leader": %%NON_LEADER%%
+    "is_leader": %%NON_LEADER%%,
+    "start_time": "0"
   },
   {
     "server": "%%BROKER%%:8282",
@@ -19,6 +20,7 @@
     "tier": "_default_tier",
     "curr_size": 0,
     "max_size": 1000000000,
-    "is_leader": %%NON_LEADER%%
+    "is_leader": %%NON_LEADER%%,
+    "start_time": "0"
   }
 ]
diff --git a/server/src/main/java/org/apache/druid/discovery/DiscoveryDruidNode.java b/server/src/main/java/org/apache/druid/discovery/DiscoveryDruidNode.java
index 5ecdf5e043..461e5fea0d 100644
--- a/server/src/main/java/org/apache/druid/discovery/DiscoveryDruidNode.java
+++ b/server/src/main/java/org/apache/druid/discovery/DiscoveryDruidNode.java
@@ -26,10 +26,12 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.Maps;
 import org.apache.druid.jackson.StringObjectPairList;
+import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.NonnullPair;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.server.DruidNode;
+import org.joda.time.DateTime;
 
 import javax.annotation.Nullable;
 import java.util.HashMap;
@@ -50,6 +52,7 @@ public class DiscoveryDruidNode
 
   private final DruidNode druidNode;
   private final NodeRole nodeRole;
+  private final DateTime startTime;
 
   /**
    * Map of service name -> DruidServices.
@@ -66,6 +69,16 @@ public class DiscoveryDruidNode
       NodeRole nodeRole,
       Map<String, DruidService> services
   )
+  {
+    this(druidNode, nodeRole, services, DateTimes.nowUtc());
+  }
+
+  public DiscoveryDruidNode(
+      DruidNode druidNode,
+      NodeRole nodeRole,
+      Map<String, DruidService> services,
+      DateTime startTime
+  )
   {
     this.druidNode = druidNode;
     this.nodeRole = nodeRole;
@@ -73,6 +86,7 @@ public class DiscoveryDruidNode
     if (services != null && !services.isEmpty()) {
       this.services.putAll(services);
     }
+    this.startTime = startTime;
   }
 
   @JsonCreator
@@ -80,6 +94,7 @@ public class DiscoveryDruidNode
       @JsonProperty("druidNode") DruidNode druidNode,
       @JsonProperty("nodeType") NodeRole nodeRole,
       @JsonProperty("services") Map<String, StringObjectPairList> rawServices,
+      @JsonProperty("startTime") DateTime startTime,
       @JacksonInject ObjectMapper jsonMapper
   )
   {
@@ -95,7 +110,7 @@ public class DiscoveryDruidNode
         }
       }
     }
-    return new DiscoveryDruidNode(druidNode, nodeRole, services);
+    return new DiscoveryDruidNode(druidNode, nodeRole, services, startTime);
   }
 
   /**
@@ -106,10 +121,10 @@ public class DiscoveryDruidNode
    * This is definitely a bug of DataNodeService, but, since renaming one of those duplicate keys will
    * break compatibility, DataNodeService still has the deprecated "type" property.
    * See the Javadoc of DataNodeService for more details.
-   *
+   * <p>
    * This function catches such duplicate keys and rewrites the deprecated "type" to "serverType",
    * so that we don't lose any properties.
-   *
+   * <p>
    * This method can be removed together when we entirely remove the deprecated "type" property from DataNodeService.
    */
   @Deprecated
@@ -166,6 +181,12 @@ public class DiscoveryDruidNode
     return druidNode;
   }
 
+  @JsonProperty
+  public DateTime getStartTime()
+  {
+    return startTime;
+  }
+
   @Nullable
   @JsonIgnore
   public <T extends DruidService> T getService(String key, Class<T> clazz)
@@ -205,7 +226,8 @@ public class DiscoveryDruidNode
     return "DiscoveryDruidNode{" +
            "druidNode=" + druidNode +
            ", nodeRole='" + nodeRole + '\'' +
-           ", services=" + services +
+           ", services=" + services + '\'' +
+           ", startTime=" + startTime +
            '}';
   }
 }
diff --git a/server/src/test/java/org/apache/druid/discovery/DiscoveryDruidNodeTest.java b/server/src/test/java/org/apache/druid/discovery/DiscoveryDruidNodeTest.java
index 64976bd822..0e55e5b68e 100644
--- a/server/src/test/java/org/apache/druid/discovery/DiscoveryDruidNodeTest.java
+++ b/server/src/test/java/org/apache/druid/discovery/DiscoveryDruidNodeTest.java
@@ -59,6 +59,7 @@ public class DiscoveryDruidNodeTest
   {
     EqualsVerifier.forClass(DiscoveryDruidNode.class)
                   .withNonnullFields("druidNode", "nodeRole", "services")
+                  .withIgnoredFields("startTime")
                   .usingGetClass()
                   .verify();
   }
diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java
index e170afd52e..7451855b21 100644
--- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java
+++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java
@@ -269,13 +269,13 @@ public class CliOverlord extends ServerRunnable
 
             if (standalone) {
               LifecycleModule.register(binder, Server.class);
-            }
 
-            bindAnnouncer(
-                binder,
-                IndexingService.class,
-                DiscoverySideEffectsProvider.create()
-            );
+              bindAnnouncer(
+                  binder,
+                  IndexingService.class,
+                  DiscoverySideEffectsProvider.create()
+              );
+            }
 
             Jerseys.addResource(binder, SelfDiscoveryResource.class);
             LifecycleModule.registerKey(binder, Key.get(SelfDiscoveryResource.class));
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java
index 10090e7258..3e6cb3e58f 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java
@@ -163,6 +163,7 @@ public class SystemSchema extends AbstractSchema
       .add("curr_size", ColumnType.LONG)
       .add("max_size", ColumnType.LONG)
       .add("is_leader", ColumnType.LONG)
+      .add("start_time", ColumnType.STRING)
       .build();
 
   static final RowSignature SERVER_SEGMENTS_SIGNATURE = RowSignature
@@ -595,7 +596,8 @@ public class SystemSchema extends AbstractSchema
           null,
           UNKNOWN_SIZE,
           UNKNOWN_SIZE,
-          NullHandling.defaultLongValue()
+          NullHandling.defaultLongValue(),
+          toStringOrNull(discoveryDruidNode.getStartTime())
       };
     }
 
@@ -614,7 +616,8 @@ public class SystemSchema extends AbstractSchema
           null,
           UNKNOWN_SIZE,
           UNKNOWN_SIZE,
-          isLeader ? 1L : 0L
+          isLeader ? 1L : 0L,
+          toStringOrNull(discoveryDruidNode.getStartTime())
       };
     }
 
@@ -648,7 +651,8 @@ public class SystemSchema extends AbstractSchema
           druidServerToUse.getTier(),
           currentSize,
           druidServerToUse.getMaxSize(),
-          NullHandling.defaultLongValue()
+          NullHandling.defaultLongValue(),
+          toStringOrNull(discoveryDruidNode.getStartTime())
       };
     }
 
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java
index 1acc5aef1c..50d0f09a85 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java
@@ -49,6 +49,7 @@ import org.apache.druid.discovery.DruidNodeDiscovery;
 import org.apache.druid.discovery.DruidNodeDiscoveryProvider;
 import org.apache.druid.discovery.NodeRole;
 import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.Intervals;
@@ -106,6 +107,7 @@ import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.jboss.netty.handler.codec.http.HttpResponse;
 import org.jboss.netty.handler.codec.http.HttpResponseStatus;
 import org.jboss.netty.handler.codec.http.HttpVersion;
+import org.joda.time.DateTime;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
@@ -380,40 +382,48 @@ public class SystemSchemaTest extends CalciteTestBase
 
   final List<DataSegment> realtimeSegments = ImmutableList.of(segment2, segment4, segment5);
 
+  private final DateTime startTime = DateTimes.nowUtc();
+
   private final DiscoveryDruidNode coordinator = new DiscoveryDruidNode(
       new DruidNode("s1", "localhost", false, 8081, null, true, false),
       NodeRole.COORDINATOR,
-      ImmutableMap.of()
+      ImmutableMap.of(),
+      startTime
   );
 
   private final DiscoveryDruidNode coordinator2 = new DiscoveryDruidNode(
       new DruidNode("s1", "localhost", false, 8181, null, true, false),
       NodeRole.COORDINATOR,
-      ImmutableMap.of()
+      ImmutableMap.of(),
+      startTime
   );
 
   private final DiscoveryDruidNode overlord = new DiscoveryDruidNode(
       new DruidNode("s2", "localhost", false, 8090, null, true, false),
       NodeRole.OVERLORD,
-      ImmutableMap.of()
+      ImmutableMap.of(),
+      startTime
   );
 
   private final DiscoveryDruidNode overlord2 = new DiscoveryDruidNode(
       new DruidNode("s2", "localhost", false, 8190, null, true, false),
       NodeRole.OVERLORD,
-      ImmutableMap.of()
+      ImmutableMap.of(),
+      startTime
   );
 
   private final DiscoveryDruidNode broker1 = new DiscoveryDruidNode(
       new DruidNode("s3", "localhost", false, 8082, null, true, false),
       NodeRole.BROKER,
-      ImmutableMap.of()
+      ImmutableMap.of(),
+      startTime
   );
 
   private final DiscoveryDruidNode broker2 = new DiscoveryDruidNode(
       new DruidNode("s3", "brokerHost", false, 8082, null, true, false),
       NodeRole.BROKER,
-      ImmutableMap.of()
+      ImmutableMap.of(),
+      startTime
   );
 
   private final DiscoveryDruidNode brokerWithBroadcastSegments = new DiscoveryDruidNode(
@@ -421,13 +431,15 @@ public class SystemSchemaTest extends CalciteTestBase
       NodeRole.BROKER,
       ImmutableMap.of(
           DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.BROKER, 0)
-      )
+      ),
+      startTime
   );
 
   private final DiscoveryDruidNode router = new DiscoveryDruidNode(
       new DruidNode("s4", "localhost", false, 8888, null, true, false),
       NodeRole.ROUTER,
-      ImmutableMap.of()
+      ImmutableMap.of(),
+      startTime
   );
 
   private final DiscoveryDruidNode historical1 = new DiscoveryDruidNode(
@@ -435,7 +447,8 @@ public class SystemSchemaTest extends CalciteTestBase
       NodeRole.HISTORICAL,
       ImmutableMap.of(
           DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0)
-      )
+      ),
+      startTime
   );
 
   private final DiscoveryDruidNode historical2 = new DiscoveryDruidNode(
@@ -443,7 +456,8 @@ public class SystemSchemaTest extends CalciteTestBase
       NodeRole.HISTORICAL,
       ImmutableMap.of(
           DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0)
-      )
+      ),
+      startTime
   );
 
   private final DiscoveryDruidNode lameHistorical = new DiscoveryDruidNode(
@@ -451,13 +465,15 @@ public class SystemSchemaTest extends CalciteTestBase
       NodeRole.HISTORICAL,
       ImmutableMap.of(
           DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0)
-      )
+      ),
+      startTime
   );
 
   private final DiscoveryDruidNode middleManager = new DiscoveryDruidNode(
       new DruidNode("s6", "mmHost", false, 8091, null, true, false),
       NodeRole.MIDDLE_MANAGER,
-      ImmutableMap.of()
+      ImmutableMap.of(),
+      startTime
   );
 
   private final DiscoveryDruidNode peon1 = new DiscoveryDruidNode(
@@ -465,7 +481,8 @@ public class SystemSchemaTest extends CalciteTestBase
       NodeRole.PEON,
       ImmutableMap.of(
           DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0)
-      )
+      ),
+      startTime
   );
 
   private final DiscoveryDruidNode peon2 = new DiscoveryDruidNode(
@@ -473,7 +490,8 @@ public class SystemSchemaTest extends CalciteTestBase
       NodeRole.PEON,
       ImmutableMap.of(
           DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0)
-      )
+      ),
+      startTime
   );
 
   private final DiscoveryDruidNode indexer = new DiscoveryDruidNode(
@@ -481,7 +499,8 @@ public class SystemSchemaTest extends CalciteTestBase
       NodeRole.INDEXER,
       ImmutableMap.of(
           DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0)
-      )
+      ),
+      startTime
   );
 
   private final ImmutableDruidServer druidServer1 = new ImmutableDruidServer(
@@ -536,7 +555,7 @@ public class SystemSchemaTest extends CalciteTestBase
     final SystemSchema.ServersTable serversTable = (SystemSchema.ServersTable) schema.getTableMap().get("servers");
     final RelDataType serverRowType = serversTable.getRowType(new JavaTypeFactoryImpl());
     final List<RelDataTypeField> serverFields = serverRowType.getFieldList();
-    Assert.assertEquals(9, serverFields.size());
+    Assert.assertEquals(10, serverFields.size());
     Assert.assertEquals("server", serverFields.get(0).getName());
     Assert.assertEquals(SqlTypeName.VARCHAR, serverFields.get(0).getType().getSqlTypeName());
   }
@@ -814,6 +833,7 @@ public class SystemSchemaTest extends CalciteTestBase
 
     final List<Object[]> expectedRows = new ArrayList<>();
     final Long nonLeader = NullHandling.defaultLongValue();
+    final String startTimeStr = startTime.toString();
     expectedRows.add(
         createExpectedRow(
             "brokerHost:8082",
@@ -824,7 +844,8 @@ public class SystemSchemaTest extends CalciteTestBase
             null,
             0L,
             0L,
-            nonLeader
+            nonLeader,
+            startTimeStr
         )
     );
     expectedRows.add(
@@ -837,7 +858,8 @@ public class SystemSchemaTest extends CalciteTestBase
             "tier",
             0L,
             1000L,
-            nonLeader
+            nonLeader,
+            startTimeStr
         )
     );
     expectedRows.add(
@@ -850,7 +872,8 @@ public class SystemSchemaTest extends CalciteTestBase
             "tier",
             400L,
             1000L,
-            nonLeader
+            nonLeader,
+            startTimeStr
         )
     );
     expectedRows.add(
@@ -863,7 +886,8 @@ public class SystemSchemaTest extends CalciteTestBase
             "tier",
             0L,
             1000L,
-            nonLeader
+            nonLeader,
+            startTimeStr
         )
     );
     expectedRows.add(
@@ -876,7 +900,8 @@ public class SystemSchemaTest extends CalciteTestBase
             "tier",
             0L,
             1000L,
-            nonLeader
+            nonLeader,
+            startTimeStr
         )
     );
     expectedRows.add(createExpectedRow(
@@ -888,7 +913,8 @@ public class SystemSchemaTest extends CalciteTestBase
         "tier",
         0L,
         1000L,
-        nonLeader
+        nonLeader,
+        startTimeStr
     ));
     expectedRows.add(
         createExpectedRow(
@@ -900,7 +926,8 @@ public class SystemSchemaTest extends CalciteTestBase
             null,
             0L,
             0L,
-            1L
+            1L,
+            startTimeStr
         )
     );
     expectedRows.add(
@@ -913,7 +940,8 @@ public class SystemSchemaTest extends CalciteTestBase
             null,
             0L,
             0L,
-            nonLeader
+            nonLeader,
+            startTimeStr
         )
     );
     expectedRows.add(
@@ -926,7 +954,8 @@ public class SystemSchemaTest extends CalciteTestBase
             "tier",
             200L,
             1000L,
-            nonLeader
+            nonLeader,
+            startTimeStr
         )
     );
     expectedRows.add(
@@ -939,7 +968,8 @@ public class SystemSchemaTest extends CalciteTestBase
             null,
             0L,
             0L,
-            1L
+            1L,
+            startTimeStr
         )
     );
     expectedRows.add(
@@ -952,7 +982,8 @@ public class SystemSchemaTest extends CalciteTestBase
             null,
             0L,
             0L,
-            0L
+            0L,
+            startTimeStr
         )
     );
     expectedRows.add(
@@ -965,7 +996,8 @@ public class SystemSchemaTest extends CalciteTestBase
             null,
             0L,
             0L,
-            0L
+            0L,
+            startTimeStr
         )
     );
     expectedRows.add(
@@ -978,7 +1010,8 @@ public class SystemSchemaTest extends CalciteTestBase
             null,
             0L,
             0L,
-            nonLeader
+            nonLeader,
+            startTimeStr
         )
     );
     expectedRows.add(
@@ -991,7 +1024,8 @@ public class SystemSchemaTest extends CalciteTestBase
             null,
             0L,
             0L,
-            nonLeader
+            nonLeader,
+            startTimeStr
         )
     );
     expectedRows.add(createExpectedRow(
@@ -1003,7 +1037,8 @@ public class SystemSchemaTest extends CalciteTestBase
         "tier",
         0L,
         1000L,
-        nonLeader
+        nonLeader,
+        startTimeStr
     ));
     Assert.assertEquals(expectedRows.size(), rows.size());
     for (int i = 0; i < rows.size(); i++) {
@@ -1035,7 +1070,8 @@ public class SystemSchemaTest extends CalciteTestBase
       @Nullable String tier,
       @Nullable Long currSize,
       @Nullable Long maxSize,
-      @Nullable Long isLeader
+      @Nullable Long isLeader,
+      String startTime
   )
   {
     return new Object[]{
@@ -1047,7 +1083,8 @@ public class SystemSchemaTest extends CalciteTestBase
         tier,
         currSize,
         maxSize,
-        isLeader
+        isLeader,
+        startTime
     };
   }
 
diff --git a/web-console/src/views/services-view/__snapshots__/services-view.spec.tsx.snap b/web-console/src/views/services-view/__snapshots__/services-view.spec.tsx.snap
index bfec768131..7679fe55a9 100644
--- a/web-console/src/views/services-view/__snapshots__/services-view.spec.tsx.snap
+++ b/web-console/src/views/services-view/__snapshots__/services-view.spec.tsx.snap
@@ -58,6 +58,7 @@ exports[`ServicesView renders data 1`] = `
           "Current size",
           "Max size",
           "Usage",
+          "Start time",
           "Detail",
           "Actions",
         ]
@@ -196,6 +197,14 @@ exports[`ServicesView renders data 1`] = `
           "show": true,
           "width": 140,
         },
+        Object {
+          "Aggregated": [Function],
+          "Cell": [Function],
+          "Header": "Start time",
+          "accessor": "start_time",
+          "show": true,
+          "width": 200,
+        },
         Object {
           "Aggregated": [Function],
           "Cell": [Function],
@@ -230,6 +239,7 @@ exports[`ServicesView renders data 1`] = `
             "plaintext_port": 8082,
             "service": "localhost:8082",
             "service_type": "broker",
+            "start_time": 0,
             "tier": null,
             "tls_port": -1,
           },
@@ -245,6 +255,7 @@ exports[`ServicesView renders data 1`] = `
             "segmentsToLoadSize": 0,
             "service": "localhost:8083",
             "service_type": "historical",
+            "start_time": 0,
             "tier": "_default_tier",
             "tls_port": -1,
           },
diff --git a/web-console/src/views/services-view/services-view.spec.tsx b/web-console/src/views/services-view/services-view.spec.tsx
index 212ed5b12a..681ba7695d 100644
--- a/web-console/src/views/services-view/services-view.spec.tsx
+++ b/web-console/src/views/services-view/services-view.spec.tsx
@@ -49,6 +49,7 @@ jest.mock('../../utils', () => {
                 curr_size: 0,
                 max_size: 0,
                 is_leader: 0,
+                start_time: 0,
               },
               {
                 service: 'localhost:8083',
@@ -64,6 +65,7 @@ jest.mock('../../utils', () => {
                 segmentsToDrop: 0,
                 segmentsToLoadSize: 0,
                 segmentsToDropSize: 0,
+                start_time: 0,
               },
             ],
           ],
diff --git a/web-console/src/views/services-view/services-view.tsx b/web-console/src/views/services-view/services-view.tsx
index 636232cf7d..05c563c6b6 100644
--- a/web-console/src/views/services-view/services-view.tsx
+++ b/web-console/src/views/services-view/services-view.tsx
@@ -59,23 +59,43 @@ import type { BasicAction } from '../../utils/basic-action';
 
 import './services-view.scss';
 
-const allColumns: string[] = [
-  'Service',
-  'Type',
-  'Tier',
-  'Host',
-  'Port',
-  'Current size',
-  'Max size',
-  'Usage',
-  'Detail',
-  ACTION_COLUMN_LABEL,
-];
-
 const tableColumns: Record<CapabilitiesMode, string[]> = {
-  'full': allColumns,
-  'no-sql': allColumns,
-  'no-proxy': ['Service', 'Type', 'Tier', 'Host', 'Port', 'Current size', 'Max size', 'Usage'],
+  'full': [
+    'Service',
+    'Type',
+    'Tier',
+    'Host',
+    'Port',
+    'Current size',
+    'Max size',
+    'Usage',
+    'Start time',
+    'Detail',
+    ACTION_COLUMN_LABEL,
+  ],
+  'no-sql': [
+    'Service',
+    'Type',
+    'Tier',
+    'Host',
+    'Port',
+    'Current size',
+    'Max size',
+    'Usage',
+    'Detail',
+    ACTION_COLUMN_LABEL,
+  ],
+  'no-proxy': [
+    'Service',
+    'Type',
+    'Tier',
+    'Host',
+    'Port',
+    'Current size',
+    'Max size',
+    'Usage',
+    'Start time',
+  ],
 };
 
 function formatQueues(
@@ -128,6 +148,7 @@ interface ServiceResultRow {
   readonly max_size: NumberLike;
   readonly plaintext_port: number;
   readonly tls_port: number;
+  readonly start_time: string;
   loadQueueInfo?: LoadQueueInfo;
   workerInfo?: WorkerInfo;
 }
@@ -178,7 +199,8 @@ export class ServicesView extends React.PureComponent<ServicesViewProps, Service
   "tls_port",
   "curr_size",
   "max_size",
-  "is_leader"
+  "is_leader",
+  "start_time"
 FROM sys.servers
 ORDER BY
   (
@@ -515,6 +537,14 @@ ORDER BY
               }
             },
           },
+          {
+            Header: 'Start time',
+            show: visibleColumns.shown('Start time'),
+            accessor: 'start_time',
+            width: 200,
+            Cell: this.renderFilterableCell('start_time'),
+            Aggregated: () => '',
+          },
           {
             Header: 'Detail',
             show: visibleColumns.shown('Detail'),
diff --git a/website/.spelling b/website/.spelling
index bf846ba207..a437f559b2 100644
--- a/website/.spelling
+++ b/website/.spelling
@@ -475,6 +475,7 @@ smooshed
 splittable
 ssl
 sslmode
+start_time
 stdout
 storages
 stringDictionaryEncoding


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org