001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.coprocessor;
019
020import java.io.IOException;
021import java.lang.reflect.InvocationTargetException;
022import java.util.List;
023import java.util.Map;
024import org.apache.hadoop.conf.Configuration;
025import org.apache.hadoop.hbase.HRegionLocation;
026import org.apache.hadoop.hbase.ServerName;
027import org.apache.yetus.audience.InterfaceAudience;
028import org.slf4j.Logger;
029import org.slf4j.LoggerFactory;
030
031@InterfaceAudience.Private
032public class ClientMetaCoprocessorHost
033  extends CoprocessorHost<ClientMetaCoprocessor, ClientMetaCoprocessorEnvironment> {
034
035  private static final Logger LOG = LoggerFactory.getLogger(ClientMetaCoprocessorHost.class);
036
037  private static class ClientMetaEnvironment extends BaseEnvironment<ClientMetaCoprocessor>
038    implements ClientMetaCoprocessorEnvironment {
039
040    public ClientMetaEnvironment(ClientMetaCoprocessor impl, int priority, int seq,
041      Configuration conf) {
042      super(impl, priority, seq, conf);
043    }
044  }
045
046  public ClientMetaCoprocessorHost(Configuration conf) {
047    // RPCServer cannot be aborted, so we don't pass Abortable down here.
048    super(null);
049    this.conf = conf;
050    boolean coprocessorsEnabled =
051      conf.getBoolean(COPROCESSORS_ENABLED_CONF_KEY, DEFAULT_COPROCESSORS_ENABLED);
052    LOG.trace("System coprocessor loading is {}", (coprocessorsEnabled ? "enabled" : "disabled"));
053    loadSystemCoprocessors(conf, CLIENT_META_COPROCESSOR_CONF_KEY);
054  }
055
056  @Override
057  public ClientMetaCoprocessorEnvironment createEnvironment(ClientMetaCoprocessor instance,
058    int priority, int sequence, Configuration conf) {
059    return new ClientMetaEnvironment(instance, priority, sequence, conf);
060  }
061
062  @Override
063  public ClientMetaCoprocessor checkAndGetInstance(Class<?> implClass)
064    throws InstantiationException, IllegalAccessException {
065    try {
066      if (ClientMetaCoprocessor.class.isAssignableFrom(implClass)) {
067        return implClass.asSubclass(ClientMetaCoprocessor.class).getDeclaredConstructor()
068          .newInstance();
069      } else {
070        LOG.error("{} is not of type ClientMetaCoprocessor. Check the configuration of {}",
071          implClass.getName(), CLIENT_META_COPROCESSOR_CONF_KEY);
072        return null;
073      }
074    } catch (NoSuchMethodException | InvocationTargetException e) {
075      throw (InstantiationException) new InstantiationException(implClass.getName()).initCause(e);
076    }
077  }
078
079  private final ObserverGetter<ClientMetaCoprocessor, ClientMetaObserver> clientMetaObserverGetter =
080    ClientMetaCoprocessor::getClientMetaObserver;
081
082  public void preGetClusterId() throws IOException {
083    execOperation(coprocEnvironments.isEmpty()
084      ? null
085      : new ObserverOperationWithoutResult<ClientMetaObserver>(clientMetaObserverGetter) {
086        @Override
087        protected void call(ClientMetaObserver observer) throws IOException {
088          observer.preGetClusterId(this);
089        }
090      });
091  }
092
093  public String postGetClusterId(String clusterId) throws IOException {
094    if (coprocEnvironments.isEmpty()) {
095      return clusterId;
096    }
097
098    return execOperationWithResult(new ObserverOperationWithResult<ClientMetaObserver, String>(
099      clientMetaObserverGetter, clusterId) {
100      @Override
101      protected String call(ClientMetaObserver observer) throws IOException {
102        return observer.postGetClusterId(this, getResult());
103      }
104    });
105  }
106
107  public void preGetActiveMaster() throws IOException {
108    execOperation(coprocEnvironments.isEmpty()
109      ? null
110      : new ObserverOperationWithoutResult<ClientMetaObserver>(clientMetaObserverGetter) {
111        @Override
112        protected void call(ClientMetaObserver observer) throws IOException {
113          observer.preGetActiveMaster(this);
114        }
115      });
116  }
117
118  public ServerName postGetActiveMaster(ServerName serverName) throws IOException {
119    if (coprocEnvironments.isEmpty()) {
120      return serverName;
121    }
122
123    return execOperationWithResult(new ObserverOperationWithResult<ClientMetaObserver, ServerName>(
124      clientMetaObserverGetter, serverName) {
125      @Override
126      protected ServerName call(ClientMetaObserver observer) throws IOException {
127        return observer.postGetActiveMaster(this, getResult());
128      }
129    });
130  }
131
132  public void preGetMasters() throws IOException {
133    execOperation(coprocEnvironments.isEmpty()
134      ? null
135      : new ObserverOperationWithoutResult<ClientMetaObserver>(clientMetaObserverGetter) {
136        @Override
137        protected void call(ClientMetaObserver observer) throws IOException {
138          observer.preGetMasters(this);
139        }
140      });
141  }
142
143  public Map<ServerName, Boolean> postGetMasters(Map<ServerName, Boolean> serverNames)
144    throws IOException {
145    if (coprocEnvironments.isEmpty()) {
146      return serverNames;
147    }
148
149    return execOperationWithResult(
150      new ObserverOperationWithResult<ClientMetaObserver, Map<ServerName, Boolean>>(
151        clientMetaObserverGetter, serverNames) {
152        @Override
153        protected Map<ServerName, Boolean> call(ClientMetaObserver observer) throws IOException {
154          return observer.postGetMasters(this, getResult());
155        }
156      });
157  }
158
159  public void preGetBootstrapNodes() throws IOException {
160    execOperation(coprocEnvironments.isEmpty()
161      ? null
162      : new ObserverOperationWithoutResult<ClientMetaObserver>(clientMetaObserverGetter) {
163        @Override
164        protected void call(ClientMetaObserver observer) throws IOException {
165          observer.preGetBootstrapNodes(this);
166        }
167      });
168  }
169
170  public List<ServerName> postGetBootstrapNodes(List<ServerName> bootstrapNodes)
171    throws IOException {
172    if (coprocEnvironments.isEmpty()) {
173      return bootstrapNodes;
174    }
175
176    return execOperationWithResult(
177      new ObserverOperationWithResult<ClientMetaObserver, List<ServerName>>(
178        clientMetaObserverGetter, bootstrapNodes) {
179        @Override
180        protected List<ServerName> call(ClientMetaObserver observer) throws IOException {
181          return observer.postGetBootstrapNodes(this, getResult());
182        }
183      });
184  }
185
186  public void preGetMetaLocations() throws IOException {
187    execOperation(coprocEnvironments.isEmpty()
188      ? null
189      : new ObserverOperationWithoutResult<ClientMetaObserver>(clientMetaObserverGetter) {
190        @Override
191        protected void call(ClientMetaObserver observer) throws IOException {
192          observer.preGetMetaLocations(this);
193        }
194      });
195  }
196
197  public List<HRegionLocation> postGetMetaLocations(List<HRegionLocation> metaLocations)
198    throws IOException {
199    if (coprocEnvironments.isEmpty()) {
200      return metaLocations;
201    }
202
203    return execOperationWithResult(
204      new ObserverOperationWithResult<ClientMetaObserver, List<HRegionLocation>>(
205        clientMetaObserverGetter, metaLocations) {
206        @Override
207        protected List<HRegionLocation> call(ClientMetaObserver observer) throws IOException {
208          return observer.postGetMetaLocations(this, getResult());
209        }
210      });
211  }
212}