Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 910DD200CBD for ; Thu, 6 Jul 2017 22:43:03 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 8FC9C16761C; Thu, 6 Jul 2017 20:43:03 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id E21C116761D for ; Thu, 6 Jul 2017 22:43:00 +0200 (CEST) Received: (qmail 65545 invoked by uid 500); 6 Jul 2017 20:43:00 -0000 Mailing-List: contact commits-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list commits@accumulo.apache.org Received: (qmail 64614 invoked by uid 99); 6 Jul 2017 20:42:58 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 06 Jul 2017 20:42:58 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 7A77DF21AB; Thu, 6 Jul 2017 20:42:56 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: ctubbsii@apache.org To: commits@accumulo.apache.org Date: Thu, 06 Jul 2017 20:43:18 -0000 Message-Id: <56b1ccef17fa429d9615d7f7cf3cbb7c@git.apache.org> In-Reply-To: <3b8d0f56f4e142fcbb35fd820ce25ed9@git.apache.org> References: <3b8d0f56f4e142fcbb35fd820ce25ed9@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [24/54] [abbrv] accumulo git commit: ACCUMULO-2181/3005 Porting Monitor archived-at: Thu, 06 Jul 2017 20:43:03 -0000 http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/master/MasterResource.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/master/MasterResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/master/MasterResource.java new file mode 100644 index 0000000..ce6578e --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/master/MasterResource.java @@ -0,0 +1,247 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.master; + +import java.lang.management.ManagementFactory; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import javax.ws.rs.GET; +import javax.ws.rs.Path; + +import org.apache.accumulo.core.master.thrift.DeadServer; +import org.apache.accumulo.core.master.thrift.MasterMonitorInfo; +import org.apache.accumulo.core.master.thrift.TabletServerStatus; +import org.apache.accumulo.core.util.AddressUtil; +import org.apache.accumulo.monitor.Monitor; +import org.apache.accumulo.monitor.rest.api.BasicResource; +import org.apache.accumulo.monitor.rest.api.log.DeadLoggerInformation; +import org.apache.accumulo.monitor.rest.api.log.DeadLoggerList; +import org.apache.accumulo.monitor.rest.api.tserver.BadTabletServerInformation; +import org.apache.accumulo.monitor.rest.api.tserver.BadTabletServers; +import org.apache.accumulo.monitor.rest.api.tserver.DeadServerInformation; +import org.apache.accumulo.monitor.rest.api.tserver.DeadServerList; +import org.apache.accumulo.monitor.rest.api.tserver.ServerShuttingDownInformation; +import org.apache.accumulo.monitor.rest.api.tserver.ServersShuttingDown; +import org.apache.accumulo.server.master.state.TabletServerState; + +/** + * + * Responsible for generating a new Master information JSON object + * + * @since 2.0.0 + * + */ +public class MasterResource extends BasicResource { + public static final String NO_MASTERS = "No Masters running"; + + /** + * Gets the MasterMonitorInfo, allowing for mocking frameworks for testability + */ + protected MasterMonitorInfo getMmi() { + return Monitor.getMmi(); + } + + /** + * Generates a master information JSON object + * + * @return master JSON object + */ + @GET + public MasterInformation getTables() { + + MasterInformation masterInformation; + + if (Monitor.getMmi() != null) { + String gcStatus = "Waiting"; + String label = ""; + if (Monitor.getGcStatus() != null) { + long start = 0; + if (Monitor.getGcStatus().current.started != 0 || Monitor.getGcStatus().currentLog.started != 0) { + start = Math.max(Monitor.getGcStatus().current.started, Monitor.getGcStatus().currentLog.started); + label = "Running"; + } else if (Monitor.getGcStatus().lastLog.finished != 0) { + start = Monitor.getGcStatus().lastLog.finished; + } + if (start != 0) { + gcStatus = String.valueOf(start); + } + } else { + gcStatus = "Down"; + } + + List tservers = new ArrayList<>(); + for (TabletServerStatus up : Monitor.getMmi().tServerInfo) { + tservers.add(up.name); + } + for (DeadServer down : Monitor.getMmi().deadTabletServers) { + tservers.add(down.server); + } + List masters = Monitor.getContext().getInstance().getMasterLocations(); + + String master = masters.size() == 0 ? "Down" : AddressUtil.parseAddress(masters.get(0), false).getHostText(); + Integer onlineTabletServers = Monitor.getMmi().tServerInfo.size(); + Integer totalTabletServers = tservers.size(); + Integer tablets = Monitor.getTotalTabletCount(); + Integer unassignedTablets = Monitor.getMmi().unassignedTablets; + long entries = Monitor.getTotalEntries(); + double ingest = Monitor.getTotalIngestRate(); + double entriesRead = Monitor.getTotalScanRate(); + double entriesReturned = Monitor.getTotalQueryRate(); + long holdTime = Monitor.getTotalHoldTime(); + double osLoad = ManagementFactory.getOperatingSystemMXBean().getSystemLoadAverage(); + + int tables = Monitor.getTotalTables(); + int deadTabletServers = Monitor.getMmi().deadTabletServers.size(); + long lookups = Monitor.getTotalLookups(); + long uptime = System.currentTimeMillis() - Monitor.getStartTime(); + + masterInformation = new MasterInformation(master, onlineTabletServers, totalTabletServers, gcStatus, tablets, unassignedTablets, entries, ingest, + entriesRead, entriesReturned, holdTime, osLoad, tables, deadTabletServers, lookups, uptime, label, getGoalState(), getState(), getNumBadTservers(), + getServersShuttingDown(), getDeadTservers(), getDeadLoggers()); + } else { + masterInformation = new MasterInformation(); + } + return masterInformation; + } + + /** + * Returns the current state of the master + * + * @return master state + */ + public String getState() { + MasterMonitorInfo mmi = getMmi(); + if (null == mmi) { + return NO_MASTERS; + } + return mmi.state.toString(); + } + + /** + * Returns the goal state of the master + * + * @return master goal state + */ + public String getGoalState() { + MasterMonitorInfo mmi = getMmi(); + if (null == mmi) { + return NO_MASTERS; + } + return mmi.goalState.name(); + } + + /** + * Generates a dead server list as a JSON object + * + * @return dead server list + */ + public DeadServerList getDeadTservers() { + MasterMonitorInfo mmi = getMmi(); + if (null == mmi) { + return new DeadServerList(); + } + + DeadServerList deadServers = new DeadServerList(); + // Add new dead servers to the list + for (DeadServer dead : mmi.deadTabletServers) { + deadServers.addDeadServer(new DeadServerInformation(dead.server, dead.lastStatus, dead.status)); + } + return deadServers; + } + + /** + * Generates a dead logger list as a JSON object + * + * @return dead logger list + */ + public DeadLoggerList getDeadLoggers() { + MasterMonitorInfo mmi = getMmi(); + if (null == mmi) { + return new DeadLoggerList(); + } + + DeadLoggerList deadLoggers = new DeadLoggerList(); + // Add new dead loggers to the list + for (DeadServer dead : mmi.deadTabletServers) { + deadLoggers.addDeadLogger(new DeadLoggerInformation(dead.server, dead.lastStatus, dead.status)); + } + return deadLoggers; + } + + /** + * Generates bad tserver lists as a JSON object + * + * @return bad tserver list + */ + public BadTabletServers getNumBadTservers() { + MasterMonitorInfo mmi = getMmi(); + if (null == mmi) { + return new BadTabletServers(); + } + + Map badServers = mmi.getBadTServers(); + + if (null == badServers || badServers.isEmpty()) { + return new BadTabletServers(); + } + + BadTabletServers readableBadServers = new BadTabletServers(); + // Add new bad tservers to the list + for (Entry badServer : badServers.entrySet()) { + try { + TabletServerState state = TabletServerState.getStateById(badServer.getValue()); + readableBadServers.addBadServer(new BadTabletServerInformation(badServer.getKey(), state.name())); + } catch (IndexOutOfBoundsException e) { + readableBadServers.addBadServer(new BadTabletServerInformation(badServer.getKey(), "Unknown state")); + } + } + return readableBadServers; + } + + /** + * Generates a JSON object of a list of servers shutting down + * + * @return servers shutting down list + */ + public ServersShuttingDown getServersShuttingDown() { + ServersShuttingDown servers = new ServersShuttingDown(); + // Add new servers to the list + for (String server : Monitor.getMmi().serversShuttingDown) { + servers.addServerShuttingDown(new ServerShuttingDownInformation(server)); + } + return servers; + } + + /** + * Generates a list of tservers + * + * @return tserver + */ + @Path("/tserver_info") + @GET + public List getTabletServerInfo() { + MasterMonitorInfo mmi = getMmi(); + if (null == mmi) { + return Collections.emptyList(); + } + return mmi.getTServerInfo(); + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/problem/ProblemDetailInformation.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/problem/ProblemDetailInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/problem/ProblemDetailInformation.java new file mode 100644 index 0000000..a07427f --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/problem/ProblemDetailInformation.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.problem; + +/** + * + * Generates a problem detail as a JSON object + * + * @since 2.0.0 + * + */ +public class ProblemDetailInformation { + + // Variable names become JSON keys + public String tableName, tableID, type, server; + public Long time; + public String resource, exception; + + public ProblemDetailInformation() {} + + /** + * Stores a problem detail + * + * @param tableName + * Table name of the problem + * @param tableID + * Table ID of the problem + * @param type + * Type of problem + * @param server + * Location of the problem + * @param time + * Time of the problem + * @param resource + * Resource with the problem + * @param exception + * Exception of the problem + */ + public ProblemDetailInformation(String tableName, String tableID, String type, String server, Long time, String resource, String exception) { + this.tableName = tableName; + this.tableID = tableID; + this.type = type; + this.server = server; + this.time = time; + this.resource = resource; + this.exception = exception; + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/problem/ProblemSummaryInformation.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/problem/ProblemSummaryInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/problem/ProblemSummaryInformation.java new file mode 100644 index 0000000..2ebfd68 --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/problem/ProblemSummaryInformation.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.problem; + +/** + * + * Generates a problem summary object + * + * @since 2.0.0 + * + */ +public class ProblemSummaryInformation { + + // Variable names become JSON keys + public String tableName, tableID; + public Integer fileRead, fileWrite, tableLoad; + + public ProblemSummaryInformation() {} + + /** + * Stores a single problem summary object + * + * @param tableName + * Name of the table with a problem + * @param tableID + * ID of the table with a problem + * @param fileRead + * Number of files read + * @param fileWrite + * Number of files written + * @param tableLoad + * Number of table loads + */ + public ProblemSummaryInformation(String tableName, String tableID, Integer fileRead, Integer fileWrite, Integer tableLoad) { + this.tableName = tableName; + this.tableID = tableID; + this.fileRead = fileRead; + this.fileWrite = fileWrite; + this.tableLoad = tableLoad; + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/problem/ProblemsResource.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/problem/ProblemsResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/problem/ProblemsResource.java new file mode 100644 index 0000000..23ea737 --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/problem/ProblemsResource.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.problem; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import javax.ws.rs.Consumes; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.MediaType; + +import org.apache.accumulo.core.client.impl.Tables; +import org.apache.accumulo.monitor.Monitor; +import org.apache.accumulo.monitor.rest.api.BasicResource; +import org.apache.accumulo.server.client.HdfsZooInstance; +import org.apache.accumulo.server.problems.ProblemReport; +import org.apache.accumulo.server.problems.ProblemReports; +import org.apache.accumulo.server.problems.ProblemType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + * Generates a problem summary and details as a JSON object + * + * @since 2.0.0 + * + */ +public class ProblemsResource extends BasicResource { + + /** + * Generates a list with the problem summary + * + * @return problem summary list + */ + @GET + @Path("/summary") + public Map> getSummary() { + + Map> jsonObj = new HashMap>(); + + List problems = new ArrayList<>(); + + Map tidToNameMap = Tables.getIdToNameMap(HdfsZooInstance.getInstance()); + + if (Monitor.getProblemException() == null) { + for (Entry> entry : Monitor.getProblemSummary().entrySet()) { + Integer readCount = null, writeCount = null, loadCount = null; + + for (ProblemType pt : ProblemType.values()) { + Integer pcount = entry.getValue().get(pt); + if (pt.equals(ProblemType.FILE_READ)) { + readCount = pcount; + } else if (pt.equals(ProblemType.FILE_WRITE)) { + writeCount = pcount; + } else if (pt.equals(ProblemType.TABLET_LOAD)) { + loadCount = pcount; + } + } + + String tableName = Tables.getPrintableTableNameFromId(tidToNameMap, entry.getKey()); + + problems.add(new ProblemSummaryInformation(tableName, entry.getKey(), readCount, writeCount, loadCount)); + } + } + jsonObj.put("problemSummary", problems); + + return jsonObj; + } + + /** + * REST call to clear problem reports from a table + * + * @param tableID + * Table ID to clear problems + */ + @POST + @Consumes(MediaType.TEXT_PLAIN) + @Path("/summary") + public void clearTableProblems(@QueryParam("s") String tableID) { + Logger log = LoggerFactory.getLogger(Monitor.class); + try { + ProblemReports.getInstance(Monitor.getContext()).deleteProblemReports(tableID); + } catch (Exception e) { + log.error("Failed to delete problem reports for table " + tableID, e); + } + } + + /** + * Generates a list of the problem details as a JSON object + * + * @return problem details list + */ + @GET + @Path("/details") + public Map> getDetails() { + + Map> jsonObj = new HashMap>(); + + List problems = new ArrayList<>(); + + Map tidToNameMap = Tables.getIdToNameMap(HdfsZooInstance.getInstance()); + + if (Monitor.getProblemException() == null) { + for (Entry> entry : Monitor.getProblemSummary().entrySet()) { + ArrayList problemReports = new ArrayList<>(); + Iterator iter = entry.getKey() == null ? ProblemReports.getInstance(Monitor.getContext()).iterator() : ProblemReports.getInstance( + Monitor.getContext()).iterator(entry.getKey()); + while (iter.hasNext()) + problemReports.add(iter.next()); + for (ProblemReport pr : problemReports) { + String tableName = Tables.getPrintableTableNameFromId(tidToNameMap, pr.getTableName()); + + problems.add(new ProblemDetailInformation(tableName, entry.getKey(), pr.getProblemType().name(), pr.getServer(), pr.getTime(), pr.getResource(), pr + .getException())); + } + } + } + jsonObj.put("problemDetails", problems); + + return jsonObj; + } + + /** + * REST call to clear specific problem details + * + * @param tableID + * Table ID to clear + * @param resource + * Resource to clear + * @param ptype + * Problem type to clear + */ + @POST + @Consumes(MediaType.TEXT_PLAIN) + @Path("/details") + public void clearDetailsProblems(@QueryParam("table") String tableID, @QueryParam("resource") String resource, @QueryParam("ptype") String ptype) { + Logger log = LoggerFactory.getLogger(Monitor.class); + try { + ProblemReports.getInstance(Monitor.getContext()).deleteProblemReport(tableID, ProblemType.valueOf(ptype), resource); + } catch (Exception e) { + log.error("Failed to delete problem reports for table " + tableID, e); + } + } + + @GET + @Path("/exception") + public Exception getException() { + return Monitor.getProblemException(); + } + +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/replication/ReplicationInformation.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/replication/ReplicationInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/replication/ReplicationInformation.java new file mode 100644 index 0000000..636ab99 --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/replication/ReplicationInformation.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.replication; + +/** + * + * Generates the replication information as a JSON object + * + * @since 2.0.0 + * + */ +public class ReplicationInformation { + + // Variable names become JSON keys + public String tableName, peerName, remoteIdentifier, replicaSystemType; + public long filesNeedingReplication; + + public ReplicationInformation() {} + + /** + * Stores new replication information + * + * @param tableName + * Name of the table being replicated + * @param peerName + * Name of the peer + * @param remoteIdentifier + * Identifier of the remote + * @param replicaSystemType + * System type replica + * @param filesNeedingReplication + * Number of files needing replication + */ + public ReplicationInformation(String tableName, String peerName, String remoteIdentifier, String replicaSystemType, long filesNeedingReplication) { + this.tableName = tableName; + this.peerName = peerName; + this.remoteIdentifier = remoteIdentifier; + this.replicaSystemType = replicaSystemType; + this.filesNeedingReplication = filesNeedingReplication; + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/replication/ReplicationResource.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/replication/ReplicationResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/replication/ReplicationResource.java new file mode 100644 index 0000000..dbfc55c --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/replication/ReplicationResource.java @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.replication; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import javax.ws.rs.GET; + +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.BatchScanner; +import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.client.TableOfflineException; +import org.apache.accumulo.core.client.admin.TableOperations; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.metadata.MetadataTable; +import org.apache.accumulo.core.metadata.RootTable; +import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection; +import org.apache.accumulo.core.replication.ReplicationTable; +import org.apache.accumulo.core.replication.ReplicationTarget; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.monitor.Monitor; +import org.apache.accumulo.monitor.rest.api.BasicResource; +import org.apache.accumulo.server.replication.ReplicaSystem; +import org.apache.accumulo.server.replication.ReplicaSystemFactory; +import org.apache.hadoop.io.Text; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + * Generates the replication table with information from the Monitor + * + * @since 2.0.0 + * + */ +public class ReplicationResource extends BasicResource { + private static final Logger log = LoggerFactory.getLogger(ReplicationResource.class); + + /** + * Generates the replication table as a JSON object + * + * @return Replication list + */ + @GET + public List getReplicationInformation() throws AccumuloException, AccumuloSecurityException { + final Connector conn = Monitor.getContext().getConnector(); + + final TableOperations tops = conn.tableOperations(); + + final Map properties = conn.instanceOperations().getSystemConfiguration(); + final Map peers = new HashMap<>(); + final String definedPeersPrefix = Property.REPLICATION_PEERS.getKey(); + final ReplicaSystemFactory replicaSystemFactory = new ReplicaSystemFactory(); + + // Get the defined peers and what ReplicaSystem impl they're using + for (Entry property : properties.entrySet()) { + String key = property.getKey(); + // Filter out cruft that we don't want + if (key.startsWith(definedPeersPrefix) && !key.startsWith(Property.REPLICATION_PEER_USER.getKey()) + && !key.startsWith(Property.REPLICATION_PEER_PASSWORD.getKey())) { + String peerName = property.getKey().substring(definedPeersPrefix.length()); + ReplicaSystem replica; + try { + replica = replicaSystemFactory.get(property.getValue()); + } catch (Exception e) { + log.warn("Could not instantiate ReplicaSystem for {} with configuration {}", property.getKey(), property.getValue(), e); + continue; + } + + peers.put(peerName, replica.getClass().getName()); + } + } + + final String targetPrefix = Property.TABLE_REPLICATION_TARGET.getKey(); + + // The total set of configured targets + Set allConfiguredTargets = new HashSet<>(); + + // Number of files per target we have to replicate + Map targetCounts = new HashMap<>(); + + Map tableNameToId = tops.tableIdMap(); + Map tableIdToName = invert(tableNameToId); + + for (String table : tops.list()) { + if (MetadataTable.NAME.equals(table) || RootTable.NAME.equals(table)) { + continue; + } + String localId = tableNameToId.get(table); + if (null == localId) { + log.trace("Could not determine ID for {}", table); + continue; + } + + Iterable> propertiesForTable; + try { + propertiesForTable = tops.getProperties(table); + } catch (TableNotFoundException e) { + log.warn("Could not fetch properties for {}", table, e); + continue; + } + + for (Entry prop : propertiesForTable) { + if (prop.getKey().startsWith(targetPrefix)) { + String peerName = prop.getKey().substring(targetPrefix.length()); + String remoteIdentifier = prop.getValue(); + ReplicationTarget target = new ReplicationTarget(peerName, remoteIdentifier, localId); + + allConfiguredTargets.add(target); + } + } + } + + // Read over the queued work + BatchScanner bs; + try { + bs = conn.createBatchScanner(ReplicationTable.NAME, Authorizations.EMPTY, 4); + } catch (TableOfflineException | TableNotFoundException e) { + log.error("Could not read replication table", e); + return Collections.emptyList(); + } + + bs.setRanges(Collections.singleton(new Range())); + WorkSection.limit(bs); + try { + Text buffer = new Text(); + for (Entry entry : bs) { + Key k = entry.getKey(); + k.getColumnQualifier(buffer); + ReplicationTarget target = ReplicationTarget.from(buffer); + + // TODO ACCUMULO-2835 once explicit lengths are tracked, we can give size-based estimates instead of just file-based + Long count = targetCounts.get(target); + if (null == count) { + targetCounts.put(target, Long.valueOf(1l)); + } else { + targetCounts.put(target, count + 1); + } + } + } finally { + bs.close(); + } + + List replicationInformation = new ArrayList<>(); + for (ReplicationTarget configuredTarget : allConfiguredTargets) { + String tableName = tableIdToName.get(configuredTarget.getSourceTableId()); + if (null == tableName) { + log.trace("Could not determine table name from id {}", configuredTarget.getSourceTableId()); + continue; + } + + String replicaSystemClass = peers.get(configuredTarget.getPeerName()); + if (null == replicaSystemClass) { + log.trace("Could not determine configured ReplicaSystem for {}", configuredTarget.getPeerName()); + continue; + } + + Long numFiles = targetCounts.get(configuredTarget); + + replicationInformation.add(new ReplicationInformation(tableName, configuredTarget.getPeerName(), configuredTarget.getRemoteIdentifier(), + replicaSystemClass, (null == numFiles) ? 0 : numFiles)); + } + + return replicationInformation; + } + + protected Map invert(Map map) { + Map newMap = new HashMap<>(map.size()); + for (Entry entry : map.entrySet()) { + newMap.put(entry.getValue(), entry.getKey()); + } + return newMap; + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/scan/ScanInformation.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/scan/ScanInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/scan/ScanInformation.java new file mode 100644 index 0000000..770bf5d --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/scan/ScanInformation.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.scan; + +import org.apache.accumulo.core.master.thrift.TabletServerStatus; + +/** + * + * Generates a scan JSON object + * + * @since 2.0.0 + * + */ +public class ScanInformation { + + // Variable names become JSON keys + public String server; + + public long scanCount; + public Long oldestScan; + + public ScanInformation() {} + + /** + * Stores new scan information + * + * @param tserverInfo + * status of the tserver + * @param scanCount + * number of scans + * @param oldestScan + * time of oldest scan + */ + public ScanInformation(TabletServerStatus tserverInfo, long scanCount, Long oldestScan) { + this.server = tserverInfo.getName(); + this.scanCount = scanCount; + this.oldestScan = oldestScan; + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/scan/Scans.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/scan/Scans.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/scan/Scans.java new file mode 100644 index 0000000..0709d2a --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/scan/Scans.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.scan; + +import java.util.ArrayList; +import java.util.List; + +/** + * + * Generates a new scan list as a JSON object + * + * @since 2.0.0 + * + */ +public class Scans { + + // Variable names become JSON keys + public List scans; + + /** + * Initializes the array list + */ + public Scans() { + scans = new ArrayList<>(); + } + + /** + * Adds a new scan to the list + * + * @param scan + * Scan object to add + */ + public void addScan(ScanInformation scan) { + scans.add(scan); + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/scan/ScansResource.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/scan/ScansResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/scan/ScansResource.java new file mode 100644 index 0000000..184c54b --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/scan/ScansResource.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.scan; + +import java.util.Map; + +import javax.ws.rs.GET; + +import org.apache.accumulo.core.master.thrift.TabletServerStatus; +import org.apache.accumulo.monitor.Monitor; +import org.apache.accumulo.monitor.Monitor.ScanStats; +import org.apache.accumulo.monitor.rest.api.BasicResource; + +import com.google.common.net.HostAndPort; + +/** + * + * Generate a new Scan list JSON object + * + * @since 2.0.0 + * + */ +public class ScansResource extends BasicResource { + + /** + * Generates a new JSON object with scan information + * + * @return Scan JSON object + */ + @GET + public Scans getTables() { + + Scans scans = new Scans(); + + Map entry = Monitor.getScans(); + + // Adds new scans to the array + for (TabletServerStatus tserverInfo : Monitor.getMmi().getTServerInfo()) { + ScanStats stats = entry.get(HostAndPort.fromString(tserverInfo.name)); + if (stats != null) { + scans.addScan(new ScanInformation(tserverInfo, stats.scanCount, stats.oldestScan)); + } + } + return scans; + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/statistic/StatisticsResource.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/statistic/StatisticsResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/statistic/StatisticsResource.java new file mode 100644 index 0000000..68900fe --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/statistic/StatisticsResource.java @@ -0,0 +1,289 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.statistic; + +import java.util.ArrayList; +import java.util.List; + +import javax.ws.rs.GET; +import javax.ws.rs.Path; + +import org.apache.accumulo.core.gc.thrift.GCStatus; +import org.apache.accumulo.core.util.Pair; +import org.apache.accumulo.monitor.Monitor; +import org.apache.accumulo.monitor.rest.api.BasicResource; + +/** + * + * Generates a list of statistics as a JSON object + * + * @since 2.0.0 + * + */ +public class StatisticsResource extends BasicResource { + + /** + * Generates the total lookup rate + * + * @return Lookup rate + */ + @GET + @Path("lookupRate") + public double getLookupRate() { + return Monitor.getLookupRate(); + } + + /** + * Generates the total number of tables + * + * @return Total number of tables + */ + @GET + @Path("totalTables") + public int getTotalTables() { + return Monitor.getTotalTables(); + } + + /** + * Generates the total tablet count + * + * @return Total tablet count + */ + @GET + @Path("totalTabletCount") + public int getTotalTabletCount() { + return Monitor.getTotalTabletCount(); + } + + /** + * Generates the total entries + * + * @return Total number of entries + */ + @GET + @Path("totalEntries") + public long getTotalEntries() { + return Monitor.getTotalEntries(); + } + + /** + * Generates the total ingest rate + * + * @return Total number of ingest rate + */ + @GET + @Path("totalIngestRate") + public double getTotalIngestRate() { + return Monitor.getTotalIngestRate(); + } + + /** + * Generates the total query rate + * + * @return Total number of query rate + */ + @GET + @Path("totalQueryRate") + public double getTotalQueryRate() { + return Monitor.getTotalQueryRate(); + } + + /** + * Generates the total scan rate + * + * @return Total number of scan rate + */ + @GET + @Path("totalScanRate") + public double getTotalScanRate() { + return Monitor.getTotalScanRate(); + } + + /** + * Generates the total hold time + * + * @return Total hold time + */ + @GET + @Path("totalHoldTime") + public long getTotalHoldTime() { + return Monitor.getTotalHoldTime(); + } + + /** + * Generates the garbage collector status + * + * @return GC status + */ + @GET + @Path("gcStatus") + public GCStatus getGcStatus() { + return Monitor.getGcStatus(); + } + + /** + * Generates the total lookups + * + * @return Total number of lookups + */ + @GET + @Path("totalLookups") + public long getTotalLookups() { + return Monitor.getTotalLookups(); + } + + /** + * Generates a list with the scan rate over time + * + * @return Scan rate over time + */ + @GET + @Path("time/scanRate") + public List> getScanRate() { + return Monitor.getScanRateOverTime(); + } + + /** + * Generates a list with the query rate over time + * + * @return Query rate over time + */ + @GET + @Path("time/queryRate") + public List> getQueryRate() { + return Monitor.getQueryRateOverTime(); + } + + /** + * Generates a list with the scan entries over time + * + * @return Scan entries over time + */ + @GET + @Path("time/scanEntries") + public List>>> getScanEntries() { + + List>>> scanEntries = new ArrayList<>(); + + Pair>> read = new Pair<>("Read", Monitor.getScanRateOverTime()); + Pair>> returned = new Pair<>("Returned", Monitor.getQueryRateOverTime()); + + scanEntries.add(read); + scanEntries.add(returned); + + return scanEntries; + } + + /** + * Generates a list with the query byte rate over time + * + * @return Query byte rate over time + */ + @GET + @Path("time/queryByteRate") + public List> getQueryByteRate() { + return Monitor.getQueryByteRateOverTime(); + } + + /** + * Generates a list with the load over time + * + * @return Load over time + */ + @GET + @Path("time/load") + public List> getLoad() { + return Monitor.getLoadOverTime(); + } + + /** + * Generates a list with the ingest rate over time + * + * @return Ingest rate over time + */ + @GET + @Path("time/ingestRate") + public List> getIngestRate() { + return Monitor.getIngestRateOverTime(); + } + + /** + * Generates a list with the ingest byte rate over time + * + * @return Ingest byte rate over time + */ + @GET + @Path("time/ingestByteRate") + public List> getIngestByteRate() { + return Monitor.getIngestByteRateOverTime(); + } + + /** + * Generates a list with the minor compactions over time + * + * @return Minor compactions over time + */ + @GET + @Path("time/minorCompactions") + public List> getMinorCompactions() { + return Monitor.getMinorCompactionsOverTime(); + } + + /** + * Generates a list with the major compactions over time + * + * @return Major compactions over time + */ + @GET + @Path("time/majorCompactions") + public List> getMajorCompactions() { + return Monitor.getMajorCompactionsOverTime(); + } + + /** + * Generates a list with the lookups over time + * + * @return Lookups over time + */ + @GET + @Path("time/lookups") + public List> getLookups() { + return Monitor.getLookupsOverTime(); + } + + /** + * Generates a list with the index cache hit rate over time + * + * @return Index cache hit rate over time + */ + @GET + @Path("time/indexCacheHitRate") + public List> getIndexCacheHitRate() { + return Monitor.getIndexCacheHitRateOverTime(); + } + + /** + * Generates a list with the data cache hit rate over time + * + * @return Data cache hit rate over time + */ + @GET + @Path("time/dataCacheHitRate") + public List> getDataCacheHitRate() { + return Monitor.getDataCacheHitRateOverTime(); + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/status/StatusInformation.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/status/StatusInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/status/StatusInformation.java new file mode 100644 index 0000000..9ad1e07 --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/status/StatusInformation.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.status; + +/** + * + * Generates status of master, gc, and tservers, as well as log and problem report + * + * @since 2.0.0 + * + */ +public class StatusInformation { + + // Variable names become JSON keys + public String masterStatus, gcStatus, tServerStatus; + + public Integer logNumber; + public boolean logsHaveError; + public Integer problemNumber; + + public StatusInformation() { + this.masterStatus = null; + this.gcStatus = null; + this.tServerStatus = null; + this.logNumber = 0; + this.logsHaveError = false; + this.problemNumber = 0; + } + + /** + * Generate the status report for the services + * + * @param masterStatus + * Status for the master + * @param gcStatus + * Status for the GC + * @param tServerStatus + * Status for the tserver + * @param logNumber + * Number of log reports + * @param logsHaveError + * Check if log reports include errors + * @param problemNumber + * Number of problems per table + */ + public StatusInformation(String masterStatus, String gcStatus, String tServerStatus, Integer logNumber, boolean logsHaveError, Integer problemNumber) { + this.masterStatus = masterStatus; + this.gcStatus = gcStatus; + this.tServerStatus = tServerStatus; + this.logNumber = logNumber; + this.logsHaveError = logsHaveError; + this.problemNumber = problemNumber; + + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/status/StatusResource.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/status/StatusResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/status/StatusResource.java new file mode 100644 index 0000000..2929b44 --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/status/StatusResource.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.status; + +import java.util.ArrayList; +import java.util.List; + +import javax.ws.rs.GET; + +import org.apache.accumulo.core.master.thrift.DeadServer; +import org.apache.accumulo.core.master.thrift.TabletServerStatus; +import org.apache.accumulo.monitor.Monitor; +import org.apache.accumulo.monitor.rest.api.BasicResource; +import org.apache.accumulo.server.monitor.DedupedLogEvent; +import org.apache.accumulo.server.monitor.LogService; +import org.apache.log4j.Level; + +/** + * + * Generates the status for master, gc, and tservers as well as log and problem reports + * + * @since 2.0.0 + * + */ +public class StatusResource extends BasicResource { + + /** + * Generates the JSON object with the status + * + * @return Status report + */ + @GET + public StatusInformation getTables() { + + StatusInformation status; + String masterStatus; + String gcStatus; + String tServerStatus = "ERROR"; + + if (Monitor.getMmi() != null) { + if (Monitor.getGcStatus() != null) { + gcStatus = "OK"; + } else { + gcStatus = "ERROR"; + } + + List tservers = new ArrayList<>(); + for (TabletServerStatus up : Monitor.getMmi().tServerInfo) { + tservers.add(up.name); + } + for (DeadServer down : Monitor.getMmi().deadTabletServers) { + tservers.add(down.server); + } + List masters = Monitor.getContext().getInstance().getMasterLocations(); + + masterStatus = masters.size() == 0 ? "ERROR" : "OK"; + + int tServerUp = Monitor.getMmi().getTServerInfoSize(); + int tServerDown = Monitor.getMmi().getDeadTabletServersSize(); + int tServerBad = Monitor.getMmi().getBadTServersSize(); + + /* + * If there are no dead or bad servers and there are tservers up, status is OK, if there are dead or bad servers and there is at least a tserver up, + * status is WARN, otherwise, the status is an error. + */ + if ((tServerDown > 0 || tServerBad > 0) && tServerUp > 0) { + tServerStatus = "WARN"; + } else if ((tServerDown == 0 || tServerBad == 0) && tServerUp > 0) { + tServerStatus = "OK"; + } else if (tServerUp == 0) { + tServerStatus = "ERROR"; + } + } else { + masterStatus = "ERROR"; + if (null == Monitor.getGcStatus()) { + gcStatus = "ERROR"; + } else { + gcStatus = "OK"; + } + tServerStatus = "ERROR"; + } + + List logs = LogService.getInstance().getEvents(); + boolean logsHaveError = false; + for (DedupedLogEvent dedupedLogEvent : logs) { + if (dedupedLogEvent.getEvent().getLevel().isGreaterOrEqual(Level.ERROR)) { + logsHaveError = true; + break; + } + } + + int numProblems = Monitor.getProblemSummary().entrySet().size(); + + status = new StatusInformation(masterStatus, gcStatus, tServerStatus, logs.size(), logsHaveError, numProblems); + + return status; + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/CompactionsList.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/CompactionsList.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/CompactionsList.java new file mode 100644 index 0000000..504de22 --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/CompactionsList.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.table; + +/** + * + * Generates a list of compactions as a JSON object + * + * @since 2.0.0 + * + */ +public class CompactionsList { + + // Variable names become JSON keys + public Integer running, queued; + + public CompactionsList() { + running = null; + queued = null; + } + + /** + * Generate a compation list + * + * @param running + * Number of running compactions + * @param queued + * Number of queued compactions + */ + public CompactionsList(Integer running, Integer queued) { + this.running = running; + this.queued = queued; + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/CompactionsTypes.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/CompactionsTypes.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/CompactionsTypes.java new file mode 100644 index 0000000..569822d --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/CompactionsTypes.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.table; + +/** + * + * Generates a list of compactions based on type + * + * @since 2.0.0 + * + */ +public class CompactionsTypes { + + // Variable names become JSON objects + public CompactionsList scans, major, minor; + + public CompactionsTypes() { + scans = new CompactionsList(); + major = new CompactionsList(); + minor = new CompactionsList(); + } + + /** + * Create a new compaction list based on types + * + * @param scans + * Scan compaction list + * @param major + * Major compaction list + * @param minor + * Minor compaction list + */ + public CompactionsTypes(CompactionsList scans, CompactionsList major, CompactionsList minor) { + this.scans = scans; + this.major = major; + this.minor = minor; + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TableInformation.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TableInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TableInformation.java new file mode 100644 index 0000000..da26bdd --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TableInformation.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.table; + +import org.apache.accumulo.core.master.thrift.TableInfo; + +/** + * + * Generates table information as a JSON object + * + * @since 2.0.0 + * + */ +public class TableInformation { + + // Variable names become JSON keys + public String tablename, tableId, tableState; + + public int tablets, onlineTablets; + public long recs, recsInMemory; + + public double ingest, ingestByteRate, query, queryByteRate; + + public CompactionsList majorCompactions, minorCompactions, scans; + + private int queuedMajorCompactions, runningMajorCompactions, queuedMinorCompactions, runningMinorCompactions, queuedScans, runningScans; + + public double entriesRead, entriesReturned; + public Double holdTime; + + public int offlineTablets; + + public TableInformation() {} + + /** + * Generate a table with just the state + * + * @param tableName + * Table name to create + * @param tableId + * Table ID to create + * @param tableState + * State of the table + */ + public TableInformation(String tableName, String tableId, String tableState) { + this.tablename = tableName; + this.tableId = tableId; + this.tableState = tableState; + } + + /** + * Generate table based on the thrift table info + * + * @param tableName + * Name of the table to create + * @param tableId + * ID of the table to create + * @param info + * Thift table info + * @param holdTime + * Hold time for the table + * @param tableState + * State of the table + */ + public TableInformation(String tableName, String tableId, TableInfo info, Double holdTime, String tableState) { + this.tablename = tableName; + this.tableId = tableId; + + this.tablets = info.tablets; + this.offlineTablets = info.tablets - info.onlineTablets; + this.onlineTablets = info.onlineTablets; + + this.recs = info.recs; + this.recsInMemory = info.recsInMemory; + + this.ingest = info.getIngestRate(); + this.ingestByteRate = info.getIngestByteRate(); + + this.query = info.getQueryRate(); + this.queryByteRate = info.getQueryByteRate(); + + this.entriesRead = info.scanRate; + this.entriesReturned = info.queryRate; + + this.holdTime = holdTime; + + if (null != info.scans) { + this.queuedScans = info.scans.queued; + this.runningScans = info.scans.running; + } else { + this.queuedScans = 0; + this.runningScans = 0; + } + + if (null != info.minors) { + this.queuedMinorCompactions = info.minors.queued; + this.runningMinorCompactions = info.minors.running; + } else { + this.queuedMinorCompactions = 0; + this.runningMinorCompactions = 0; + } + + if (null != info.majors) { + this.queuedMajorCompactions = info.majors.queued; + this.runningMajorCompactions = info.majors.running; + } else { + this.queuedMajorCompactions = 0; + this.runningMajorCompactions = 0; + } + + this.majorCompactions = new CompactionsList(runningMajorCompactions, queuedMajorCompactions); + this.minorCompactions = new CompactionsList(runningMinorCompactions, queuedMinorCompactions); + this.scans = new CompactionsList(runningScans, queuedScans); + + this.tableState = tableState; + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TableInformationList.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TableInformationList.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TableInformationList.java new file mode 100644 index 0000000..fd1a3ac --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TableInformationList.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.table; + +import java.util.ArrayList; +import java.util.List; + +/** + * + * Generates a list with table information + * + * @since 2.0.0 + * + */ +public class TableInformationList { + + // Variable names become JSON keys + public List table; + + /** + * Initializes the array list + */ + public TableInformationList() { + table = new ArrayList<>(); + } + + /** + * Adds a new table to the list + * + * @param table + * Table information to add + */ + public void addTable(TableInformation table) { + this.table.add(table); + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TableNamespace.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TableNamespace.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TableNamespace.java new file mode 100644 index 0000000..b71094e --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TableNamespace.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.table; + +import java.util.ArrayList; +import java.util.List; + +/** + * + * Generates a list of tables grouped by namespace + * + * @since 2.0.0 + * + */ +public class TableNamespace { + + // Variable names become JSON keys + public String namespace; + public List table; + + public TableNamespace() {} + + /** + * Creates a new namespace group with an empty list + * + * @param name + * Namespace for the group + */ + public TableNamespace(String name) { + namespace = name; + table = new ArrayList<>(); + } + + /** + * Add a new table to the list + * + * @param table + * Table to add + */ + public void addTable(TableInformation table) { + this.table.add(table); + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TablesList.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TablesList.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TablesList.java new file mode 100644 index 0000000..e1af4ef --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TablesList.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.table; + +import java.util.ArrayList; +import java.util.List; + +/** + * + * Generates a list of tables as a JSON object + * + * @since 2.0.0 + * + */ +public class TablesList { + + // Variable names become JSON keys + public List tables; + + /** + * Initializes array list + */ + public TablesList() { + tables = new ArrayList<>(); + } + + /** + * Add a table to the list + * + * @param table + * Table to add + */ + public void addTable(TableNamespace table) { + tables.add(table); + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TablesResource.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TablesResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TablesResource.java new file mode 100644 index 0000000..7959b6d --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/table/TablesResource.java @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.table; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.TreeSet; + +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; + +import org.apache.accumulo.core.client.Instance; +import org.apache.accumulo.core.client.impl.Namespaces; +import org.apache.accumulo.core.client.impl.Tables; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.impl.KeyExtent; +import org.apache.accumulo.core.master.thrift.TableInfo; +import org.apache.accumulo.core.master.thrift.TabletServerStatus; +import org.apache.accumulo.core.metadata.MetadataTable; +import org.apache.accumulo.core.metadata.RootTable; +import org.apache.accumulo.monitor.Monitor; +import org.apache.accumulo.monitor.rest.api.BasicResource; +import org.apache.accumulo.monitor.rest.api.tserver.TabletServer; +import org.apache.accumulo.monitor.rest.api.tserver.TabletServers; +import org.apache.accumulo.server.client.HdfsZooInstance; +import org.apache.accumulo.server.master.state.MetaDataTableScanner; +import org.apache.accumulo.server.master.state.TabletLocationState; +import org.apache.accumulo.server.tables.TableManager; +import org.apache.accumulo.server.util.TableInfoUtil; +import org.apache.hadoop.io.Text; + +/** + * + * Generates a tables list from the Monitor as a JSON object + * + * @since 2.0.0 + * + */ +public class TablesResource extends BasicResource { + + private static final TabletServerStatus NO_STATUS = new TabletServerStatus(); + + /** + * Generates a table list based on the namespace + * + * @param namespace + * Namespace used to filter the tables + * @return Table list + */ + private TablesList generateTables(String namespace) { + Map tidToNameMap = Tables.getIdToNameMap(HdfsZooInstance.getInstance()); + SortedMap tableStats = new TreeMap<>(); + + if (Monitor.getMmi() != null && Monitor.getMmi().tableMap != null) + for (Entry te : Monitor.getMmi().tableMap.entrySet()) + tableStats.put(Tables.getPrintableTableNameFromId(tidToNameMap, te.getKey()), te.getValue()); + + Map compactingByTable = TableInfoUtil.summarizeTableStats(Monitor.getMmi()); + TableManager tableManager = TableManager.getInstance(); + + SortedMap namespaces = Namespaces.getNameToIdMap(Monitor.getContext().getInstance()); + + TablesList tableNamespace = new TablesList(); + List tables = new ArrayList<>(); + + // Add the tables that have the selected namespace + for (String key : namespaces.keySet()) { + if (namespace.equals("*") || namespace.equals(key) || (key.equals("") && namespace.equals("-"))) { + tableNamespace.addTable(new TableNamespace(key)); + } + } + + // Add tables to the list + for (Entry entry : Tables.getNameToIdMap(HdfsZooInstance.getInstance()).entrySet()) { + String tableName = entry.getKey(), tableId = entry.getValue(); + TableInfo tableInfo = tableStats.get(tableName); + if (null != tableInfo) { + Double holdTime = compactingByTable.get(tableId); + if (holdTime == null) + holdTime = Double.valueOf(0.); + + for (TableNamespace name : tableNamespace.tables) { + // Check if table has the default namespace + if (!tableName.contains(".") && name.namespace.equals("")) { + name.addTable(new TableInformation(tableName, tableId, tableInfo, holdTime, tableManager.getTableState(tableId).name())); + } else if (tableName.startsWith(name.namespace + ".")) { + name.addTable(new TableInformation(tableName, tableId, tableInfo, holdTime, tableManager.getTableState(tableId).name())); + } + } + + tables.add(new TableInformation(tableName, tableId, tableInfo, holdTime, tableManager.getTableState(tableId).name())); + } else { + for (TableNamespace name : tableNamespace.tables) { + if (!tableName.contains(".") && name.namespace.equals("")) { + name.addTable(new TableInformation(tableName, tableId, tableManager.getTableState(tableId).name())); + } else if (tableName.startsWith(name.namespace + ".")) { + name.addTable(new TableInformation(tableName, tableId, tableManager.getTableState(tableId).name())); + } + } + tables.add(new TableInformation(tableName, tableId, tableManager.getTableState(tableId).name())); + } + } + + return tableNamespace; + + } + + /** + * Generates a list of all the tables + * + * @return list with all tables + */ + @GET + public TablesList getTables() { + return generateTables("*"); + } + + /** + * Generates a list with the selected namespace + * + * @param namespace + * Namespace to filter tables + * @return list with selected tables + */ + @GET + @Path("/namespace/{namespace}") + public TablesList getTable(@PathParam("namespace") String namespace) { + return generateTables(namespace); + } + + /** + * Generates a list of participating tservers for a table + * + * @param tableId + * Table ID to find participating tservers + * @return List of participating tservers + */ + @Path("/{tableId}") + @GET + public TabletServers getParticipatingTabletServers(@PathParam("tableId") String tableId) throws Exception { + Instance instance = Monitor.getContext().getInstance(); + + TreeSet locs = new TreeSet<>(); + if (RootTable.ID.equals(tableId)) { + locs.add(instance.getRootTabletLocation()); + } else { + String systemTableName = MetadataTable.ID.equals(tableId) ? RootTable.NAME : MetadataTable.NAME; + MetaDataTableScanner scanner = new MetaDataTableScanner(Monitor.getContext(), new Range(KeyExtent.getMetadataEntry(tableId, new Text()), + KeyExtent.getMetadataEntry(tableId, null)), systemTableName); + + while (scanner.hasNext()) { + TabletLocationState state = scanner.next(); + if (state.current != null) { + try { + locs.add(state.current.hostPort()); + } catch (Exception ex) {} + } + } + scanner.close(); + } + + TabletServers tabletServers = new TabletServers(Monitor.getMmi().tServerInfo.size()); + + List tservers = new ArrayList<>(); + if (Monitor.getMmi() != null) { + for (TabletServerStatus tss : Monitor.getMmi().tServerInfo) { + try { + if (tss.name != null && locs.contains(tss.name)) + tservers.add(tss); + } catch (Exception ex) { + + } + } + } + + // Adds tservers to the list + for (TabletServerStatus status : tservers) { + if (status == null) + status = NO_STATUS; + TableInfo summary = TableInfoUtil.summarizeTableStats(status); + if (tableId != null) + summary = status.tableMap.get(tableId); + if (summary == null) + continue; + + TabletServer tabletServerInfo = new TabletServer(); + tabletServerInfo.updateTabletServerInfo(status, summary); + + tabletServers.addTablet(tabletServerInfo); + } + + return tabletServers; + + } + + /** + * Generates the list of existing namespaces + * + * @return list of namespaces as a JSON object + */ + @Path("namespaces") + @GET + public Map> getNamespaces() { + + Map> jsonObj = new HashMap>(); + + List namespaces = new ArrayList<>(); + SortedMap ns = Namespaces.getNameToIdMap(Monitor.getContext().getInstance()); + + // Adds the namespaces to the list + for (String key : ns.keySet()) { + namespaces.add(key); + } + + jsonObj.put("namespaces", namespaces); + + return jsonObj; + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/AddlInformation.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/AddlInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/AddlInformation.java new file mode 100644 index 0000000..5289144 --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/AddlInformation.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.trace; + +import java.util.ArrayList; +import java.util.List; + +/** + * + * Generates addiotional information for the selected trace + * + * @since 2.0.0 + * + */ +public class AddlInformation { + + // Variable names become JSON keys + public List data; + public List annotations; + + /** + * Initializes data and annotation array lists + */ + public AddlInformation() { + data = new ArrayList<>(); + annotations = new ArrayList<>(); + } + + /** + * Add a new data + * + * @param data + * Data to add + */ + public void addData(DataInformation data) { + this.data.add(data); + } + + /** + * Add a new annotation + * + * @param annotations + * Annotation to add + */ + public void addAnnotations(AnnotationInformation annotations) { + this.annotations.add(annotations); + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/AnnotationInformation.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/AnnotationInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/AnnotationInformation.java new file mode 100644 index 0000000..ad0ae3c --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/AnnotationInformation.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.trace; + +/** + * + * Generates annotation information for the trace + * + * @since 2.0.0 + * + */ +public class AnnotationInformation { + + // Variable names become JSON keys + public String annotation; + public long time; + + public AnnotationInformation() {} + + /** + * Stores a trace annotation + * + * @param annotation + * Annotation message + * @param time + * Annotation time + */ + public AnnotationInformation(String annotation, long time) { + this.annotation = annotation; + this.time = time; + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/DataInformation.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/DataInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/DataInformation.java new file mode 100644 index 0000000..886c525 --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/DataInformation.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.trace; + +/** + * + * Generate a new data information for a trace + * + * @since 2.0.0 + * + */ +public class DataInformation { + + // Variable names become JSON keys + public String key, value; + + public DataInformation() {} + + /** + * Creates new data table with key/value + * + * @param key + * Key for the data + * @param value + * Value for the data + */ + public DataInformation(String key, String value) { + this.key = key; + this.value = value; + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/RecentTracesInformation.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/RecentTracesInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/RecentTracesInformation.java new file mode 100644 index 0000000..aab7968 --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/RecentTracesInformation.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.trace; + +import org.apache.accumulo.tracer.thrift.RemoteSpan; + +/** + * + * Generates a recent trace + * + * @since 2.0.0 + * + */ +public class RecentTracesInformation { + + // Variable names become JSON keys + public String type; + public Long avg; + + public int total = 0; + + public long min = Long.MAX_VALUE, max = Long.MIN_VALUE; + private long totalMS = 0l; + public long histogram[] = new long[] {0l, 0l, 0l, 0l, 0l, 0l}; + + public RecentTracesInformation() {} + + /** + * Adds the type of the trace + * + * @param type + * Trace type + */ + public RecentTracesInformation(String type) { + this.type = type; + } + + /** + * Adds a span for the trace + * + * @param span + * Remote span to obtain information + */ + public void addSpan(RemoteSpan span) { + total++; + long ms = span.stop - span.start; + totalMS += ms; + min = Math.min(min, ms); + max = Math.max(max, ms); + int index = 0; + while (ms >= 10 && index < histogram.length) { + ms /= 10; + index++; + } + histogram[index]++; + + avg = total != 0 ? totalMS / total : null; + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/0ca5cd33/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/RecentTracesList.java ---------------------------------------------------------------------- diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/RecentTracesList.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/RecentTracesList.java new file mode 100644 index 0000000..fbb4b45 --- /dev/null +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/api/trace/RecentTracesList.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.accumulo.monitor.rest.api.trace; + +import java.util.ArrayList; +import java.util.List; + +/** + * + * Generates a list of recent traces + * + * @since 2.0.0 + * + */ +public class RecentTracesList { + + // Variable names become JSON keys + public List recentTraces; + + /** + * Initializes recent traces array list + */ + public RecentTracesList() { + recentTraces = new ArrayList<>(); + } + + /** + * Adds a new trace to the list + * + * @param trace + * Trace to add + */ + public void addTrace(RecentTracesInformation trace) { + recentTraces.add(trace); + } +}