summaryrefslogtreecommitdiff
path: root/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java
blob: 19722f04a52907a2948e157c6a718594d03feff0 (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
/**
 * 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
 * <p>
 * http://www.apache.org/licenses/LICENSE-2.0
 * <p>
 * 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.hadoop.ozone.container.common.statemachine;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ozone.protocolPB
    .StorageContainerDatanodeProtocolClientSideTranslatorPB;
import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolPB;
import org.apache.hadoop.security.UserGroupInformation;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.management.ObjectName;
import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;

import static org.apache.hadoop.hdds.scm.HddsServerUtil
    .getScmRpcTimeOutInMilliseconds;

/**
 * SCMConnectionManager - Acts as a class that manages the membership
 * information of the SCMs that we are working with.
 */
public class SCMConnectionManager
    implements Closeable, SCMConnectionManagerMXBean {
  private static final Logger LOG =
      LoggerFactory.getLogger(SCMConnectionManager.class);

  private final ReadWriteLock mapLock;
  private final Map<InetSocketAddress, EndpointStateMachine> scmMachines;

  private final int rpcTimeout;
  private final Configuration conf;
  private final ObjectName jmxBean;

  public SCMConnectionManager(Configuration conf) {
    this.mapLock = new ReentrantReadWriteLock();
    Long timeOut = getScmRpcTimeOutInMilliseconds(conf);
    this.rpcTimeout = timeOut.intValue();
    this.scmMachines = new HashMap<>();
    this.conf = conf;
    jmxBean = MBeans.register("OzoneDataNode",
        "SCMConnectionManager",
        this);
  }


  /**
   * Returns Config.
   *
   * @return ozoneConfig.
   */
  public Configuration getConf() {
    return conf;
  }

  /**
   * Get RpcTimeout.
   *
   * @return - Return RPC timeout.
   */
  public int getRpcTimeout() {
    return rpcTimeout;
  }


  /**
   * Takes a read lock.
   */
  public void readLock() {
    this.mapLock.readLock().lock();
  }

  /**
   * Releases the read lock.
   */
  public void readUnlock() {
    this.mapLock.readLock().unlock();
  }

  /**
   * Takes the write lock.
   */
  public void writeLock() {
    this.mapLock.writeLock().lock();
  }

  /**
   * Releases the write lock.
   */
  public void writeUnlock() {
    this.mapLock.writeLock().unlock();
  }

  /**
   * adds a new SCM machine to the target set.
   *
   * @param address - Address of the SCM machine to send heatbeat to.
   * @throws IOException
   */
  public void addSCMServer(InetSocketAddress address) throws IOException {
    writeLock();
    try {
      if (scmMachines.containsKey(address)) {
        LOG.warn("Trying to add an existing SCM Machine to Machines group. " +
            "Ignoring the request.");
        return;
      }
      RPC.setProtocolEngine(conf, StorageContainerDatanodeProtocolPB.class,
          ProtobufRpcEngine.class);
      long version =
          RPC.getProtocolVersion(StorageContainerDatanodeProtocolPB.class);

      StorageContainerDatanodeProtocolPB rpcProxy = RPC.getProxy(
          StorageContainerDatanodeProtocolPB.class, version,
          address, UserGroupInformation.getCurrentUser(), conf,
          NetUtils.getDefaultSocketFactory(conf), getRpcTimeout());

      StorageContainerDatanodeProtocolClientSideTranslatorPB rpcClient =
          new StorageContainerDatanodeProtocolClientSideTranslatorPB(rpcProxy);

      EndpointStateMachine endPoint =
          new EndpointStateMachine(address, rpcClient, conf);
      scmMachines.put(address, endPoint);
    } finally {
      writeUnlock();
    }
  }

  /**
   * Removes a  SCM machine for the target set.
   *
   * @param address - Address of the SCM machine to send heatbeat to.
   * @throws IOException
   */
  public void removeSCMServer(InetSocketAddress address) throws IOException {
    writeLock();
    try {
      if (!scmMachines.containsKey(address)) {
        LOG.warn("Trying to remove a non-existent SCM machine. " +
            "Ignoring the request.");
        return;
      }

      EndpointStateMachine endPoint = scmMachines.get(address);
      endPoint.close();
      scmMachines.remove(address);
    } finally {
      writeUnlock();
    }
  }

  /**
   * Returns all known RPCEndpoints.
   *
   * @return - List of RPC Endpoints.
   */
  public Collection<EndpointStateMachine> getValues() {
    return scmMachines.values();
  }

  @Override
  public void close() throws IOException {
    getValues().forEach(endpointStateMachine
        -> IOUtils.cleanupWithLogger(LOG, endpointStateMachine));
    MBeans.unregister(jmxBean);
  }

  @Override
  public List<EndpointStateMachineMBean> getSCMServers() {
    readLock();
    try {
      return Collections
          .unmodifiableList(new ArrayList<>(scmMachines.values()));

    } finally {
      readUnlock();
    }
  }
}