otterc commented on code in PR #2231:
URL: 
https://github.com/apache/incubator-celeborn/pull/2231#discussion_r1456385840


##########
common/src/main/java/org/apache/celeborn/common/network/sasl/registration/RegistrationClientBootstrap.java:
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.celeborn.common.network.sasl.registration;
+
+import static org.apache.celeborn.common.network.sasl.SaslUtils.*;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.google.protobuf.ByteString;
+import io.netty.channel.Channel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.celeborn.common.exception.CelebornException;
+import org.apache.celeborn.common.network.client.TransportClient;
+import org.apache.celeborn.common.network.client.TransportClientBootstrap;
+import org.apache.celeborn.common.network.protocol.TransportMessage;
+import org.apache.celeborn.common.network.sasl.CelebornSaslClient;
+import org.apache.celeborn.common.network.sasl.SaslClientBootstrap;
+import org.apache.celeborn.common.network.sasl.SaslCredentials;
+import org.apache.celeborn.common.network.sasl.SaslTimeoutException;
+import org.apache.celeborn.common.network.util.TransportConf;
+import org.apache.celeborn.common.protocol.MessageType;
+import org.apache.celeborn.common.protocol.PbAuthType;
+import org.apache.celeborn.common.protocol.PbAuthenticationInitiationRequest;
+import org.apache.celeborn.common.protocol.PbAuthenticationInitiationResponse;
+import org.apache.celeborn.common.protocol.PbRegisterApplicationRequest;
+import org.apache.celeborn.common.protocol.PbRegisterApplicationResponse;
+import org.apache.celeborn.common.protocol.PbSaslMechanism;
+import org.apache.celeborn.common.protocol.PbSaslRequest;
+import org.apache.celeborn.common.util.JavaUtils;
+
+/**
+ * Bootstraps a {@link TransportClient} by registering application (if the 
application is not
+ * registered). If the application is already registered, it will bootstrap 
the client by performing
+ * SASL authentication.
+ */
+public class RegistrationClientBootstrap implements TransportClientBootstrap {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(RegistrationClientBootstrap.class);
+
+  private static final String VERSION = "1.0";
+
+  /**
+   * TODO: This should be made configurable. For now, we only support 
ANONYMOUS for client-auth and
+   * DIGEST-MD5 for connect-auth.
+   */
+  private static final List<PbSaslMechanism> SASL_MECHANISMS =
+      Lists.newArrayList(
+          PbSaslMechanism.newBuilder()
+              .setMechanism(ANONYMOUS)
+              .addAuthTypes(PbAuthType.CLIENT_AUTH)
+              .build(),
+          PbSaslMechanism.newBuilder()
+              .setMechanism(DIGEST_MD5)
+              .addAuthTypes(PbAuthType.CONNECTION_AUTH)
+              .build());
+
+  private final TransportConf conf;
+  private final String appId;
+  private final SaslCredentials saslCredentials;
+
+  private final RegistrationInfo registrationInfo;
+
+  public RegistrationClientBootstrap(
+      TransportConf conf,
+      String appId,
+      SaslCredentials saslCredentials,
+      RegistrationInfo registrationInfo) {
+    this.conf = Preconditions.checkNotNull(conf, "conf");
+    this.appId = Preconditions.checkNotNull(appId, "appId");
+    this.saslCredentials = Preconditions.checkNotNull(saslCredentials, 
"saslCredentials");
+    this.registrationInfo = Preconditions.checkNotNull(registrationInfo, 
"registrationInfo");
+  }
+
+  @Override
+  public void doBootstrap(TransportClient client, Channel channel) throws 
RuntimeException {
+    if (registrationInfo.getRegistrationState() == 
RegistrationInfo.RegistrationState.REGISTERED) {
+      LOG.info("client has already registered, skip register.");
+      doSaslBootstrap(client, channel);
+      return;
+    }
+    try {
+      LOG.info("authentication initiation started for {}", appId);
+      doAuthInitiation(client);
+      LOG.info("authentication initiation successful for {}", appId);
+      doClientAuthentication(client);
+      LOG.info("client authenticated for {}", appId);
+      register(client);
+      LOG.info("Registration for {}", appId);
+      
registrationInfo.setRegistrationState(RegistrationInfo.RegistrationState.REGISTERED);
+    } catch (Throwable e) {
+      LOG.error("Registration failed for {}", appId, e);
+      
registrationInfo.setRegistrationState(RegistrationInfo.RegistrationState.FAILED);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private void doAuthInitiation(TransportClient client) throws IOException, 
CelebornException {
+    PbAuthenticationInitiationRequest authInitRequest =
+        PbAuthenticationInitiationRequest.newBuilder()
+            .setVersion(VERSION)
+            .setAuthEnabled(true)
+            .addAllSaslMechanisms(SASL_MECHANISMS)
+            .build();
+    TransportMessage msg =
+        new TransportMessage(
+            MessageType.AUTHENTICATION_INITIATION_REQUEST, 
authInitRequest.toByteArray());
+    ByteBuffer authInitResponseBuffer;
+    try {
+      authInitResponseBuffer = client.sendRpcSync(msg.toByteBuffer(), 
conf.saslTimeoutMs());
+    } catch (RuntimeException ex) {
+      // TODO: Auth initiation timed out. Will just throw SaslTimeoutException 
for now

Review Comment:
   Yes, I will remove this. I probably was thinking if we need to add a 
different exception when authInit or registration requests timeouts. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to