1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.slowlog;
21
22 import java.io.IOException;
23 import java.util.ArrayList;
24 import java.util.List;
25 import java.util.Random;
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.hbase.HConstants;
28 import org.apache.hadoop.hbase.NamespaceDescriptor;
29 import org.apache.hadoop.hbase.TableName;
30 import org.apache.hadoop.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.hbase.client.Connection;
32 import org.apache.hadoop.hbase.client.ConnectionFactory;
33 import org.apache.hadoop.hbase.client.Durability;
34 import org.apache.hadoop.hbase.client.Put;
35 import org.apache.hadoop.hbase.client.Table;
36 import org.apache.hadoop.hbase.protobuf.generated.TooSlowLog;
37 import org.apache.hadoop.hbase.util.Bytes;
38 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
39 import org.slf4j.Logger;
40 import org.slf4j.LoggerFactory;
41
42
43
44
45
46
47 @InterfaceAudience.Private
48 public final class SlowLogTableAccessor {
49
50 private static final Logger LOG = LoggerFactory.getLogger(SlowLogTableAccessor.class);
51
52 private static final Random RANDOM = new Random();
53
54 private static Connection connection;
55
56 private SlowLogTableAccessor() {
57 }
58
59
60
61
62
63 public static final TableName SLOW_LOG_TABLE_NAME =
64 TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "slowlog");
65
66 private static void doPut(final Connection connection, final List<Put> puts)
67 throws IOException {
68 try (Table table = connection.getTable(SLOW_LOG_TABLE_NAME)) {
69 table.put(puts);
70 }
71 }
72
73
74
75
76
77
78 public static void addSlowLogRecords(final List<TooSlowLog.SlowLogPayload> slowLogPayloads,
79 final Configuration configuration) {
80 List<Put> puts = new ArrayList<>(slowLogPayloads.size());
81 for (TooSlowLog.SlowLogPayload slowLogPayload : slowLogPayloads) {
82 final byte[] rowKey = getRowKey(slowLogPayload);
83 final Put put = new Put(rowKey);
84 put.setDurability(Durability.SKIP_WAL);
85 put.setPriority(HConstants.NORMAL_QOS);
86 put.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("call_details"),
87 Bytes.toBytes(slowLogPayload.getCallDetails()))
88 .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("client_address"),
89 Bytes.toBytes(slowLogPayload.getClientAddress()))
90 .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("method_name"),
91 Bytes.toBytes(slowLogPayload.getMethodName()))
92 .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("param"),
93 Bytes.toBytes(slowLogPayload.getParam()))
94 .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("processing_time"),
95 Bytes.toBytes(Integer.toString(slowLogPayload.getProcessingTime())))
96 .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("queue_time"),
97 Bytes.toBytes(Integer.toString(slowLogPayload.getQueueTime())))
98 .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("region_name"),
99 Bytes.toBytes(slowLogPayload.getRegionName()))
100 .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("response_size"),
101 Bytes.toBytes(Long.toString(slowLogPayload.getResponseSize())))
102 .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("server_class"),
103 Bytes.toBytes(slowLogPayload.getServerClass()))
104 .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("start_time"),
105 Bytes.toBytes(Long.toString(slowLogPayload.getStartTime())))
106 .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("type"),
107 Bytes.toBytes(slowLogPayload.getType().name()))
108 .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("username"),
109 Bytes.toBytes(slowLogPayload.getUserName()));
110 puts.add(put);
111 }
112 try {
113 if (connection == null) {
114 createConnection(configuration);
115 }
116 doPut(connection, puts);
117 } catch (Exception e) {
118 LOG.warn("Failed to add slow/large log records to hbase:slowlog table.", e);
119 }
120 }
121
122 private static synchronized void createConnection(Configuration configuration)
123 throws IOException {
124 Configuration conf = new Configuration(configuration);
125
126 conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 20000);
127
128 conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
129 conf.setInt("hbase.client.serverside.retries.multiplier", 1);
130 connection = ConnectionFactory.createConnection(conf);
131 }
132
133
134
135
136
137
138
139
140
141 private static byte[] getRowKey(final TooSlowLog.SlowLogPayload slowLogPayload) {
142 String hashcode = String.valueOf(slowLogPayload.hashCode());
143 String lastFiveDig =
144 hashcode.substring((hashcode.length() > 5) ? (hashcode.length() - 5) : 0);
145 if (lastFiveDig.startsWith("-")) {
146 lastFiveDig = String.valueOf(RANDOM.nextInt(99999));
147 }
148 final long currentTimeMillis = EnvironmentEdgeManager.currentTime();
149 final String timeAndHashcode = currentTimeMillis + lastFiveDig;
150 final long rowKeyLong = Long.parseLong(timeAndHashcode);
151 return Bytes.toBytes(rowKeyLong);
152 }
153
154 }