[ 
https://issues.apache.org/jira/browse/RATIS-334?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16662751#comment-16662751
 ] 

Josh Elser commented on RATIS-334:
----------------------------------

{code:java}
+public class LogInfo {
{code}
{{equals(Object)}}, {{toString()}}, and {{hashCode()}}?
{code:java}
+    System.out.println("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!");
{code}
;) should we just have an exception thrown here to prevent the "unhandled 
message" case?
{code:java}
-        case CREATELOG:
-          return processCreateLogRequest(logServiceRequestProto);
-        case ARCHIVELOG:
-          return processArchiveLog(logServiceRequestProto);
+//        case CREATELOG:
+//          return processCreateLogRequest(logServiceRequestProto);
+//        case ARCHIVELOG:
+//          return processArchiveLog(logServiceRequestProto);
{code}
Just remove these from LogStateMachine instead of commenting out? Same in 
{{LogServiceImpl}}.
{code:java}
+        return 
LogServiceFactory.getInstance().createLogService(getRaftClient(info), null);
{code}
I know you're planning on work around this, but consolidate calls to 
{{LogServiceFactory.getInstance()}} in the constructor of {{LogServiceClient}}? 
Ignore if you're already re-working it.
{code:java}
public class NoEnoughWorkersException{code}
s/No/Not/ ? Also in the error message.
{code:java}
+    try {
+      reply = raftClient.sendAsync(Message.valueOf(LogServiceProtoUtil
+          .toAppendBBEntryLogRequestProto(parent.getName(), list)
+          .toByteString())).get();
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    } catch (ExecutionException e) {
+      e.printStackTrace();
+    }{code}
Re-set the interrupted exception and propagate for {{InterruptedException}}. 
Unwrap and re-throw the {{ExecutionException}}?
{code:java}
+    @Parameter(names = "-port", description = "Port number")
+    private int port = 9999;
{code}
Doing some poking around here to make sure there's nothing obvious in the 
"ecosystem" that might default to using 9999.

I'm also wondering – ideally, we can just set {{0}} and let the service pick an 
ephemeral port. I guess that would require the service to register itself 
somewhere for us to find it? You have an idea about how that should work?
{code:java}
+        if (host == null) {
+            host = LogServiceUtils.getHostName();
+        }
{code}
Probably a good idea to let this be configurable too (multiple NICs with 
public/private network setups).
{code:java}
+        try {
+            req = 
MetaSMRequestProto.parseFrom(x.getStateMachineLogEntry().getLogData());
+        } catch (InvalidProtocolBufferException e) {
+            e.printStackTrace();
+        }{code}
Propagate. Here, and elsewhere in {{MetaStateMachine.}}
{code:java}
+            case UNREGISTERREQUEST:
+                LogServiceUnregisterLogRequestProto unregReq = 
req.getUnregisterRequest();
+                logname = LogServiceProtoUtil.toLogName(unregReq.getLogname());
+                map.remove(logname);
+                break;
+            case PINGREQUEST:
+                LogServicePingRequestProto pingRequest = req.getPingRequest();
+                RaftPeer peer = 
MetaServiceProtoUtil.toRaftPeer(pingRequest.getPeer());
+                if (peers.contains(peer)) {
+                    //Do Nothing, that's just heartbeat
+                } else {
+                    peers.add(peer);
+                    avail.add(new PeerGroups(peer));
+                }
+                break;{code}
DEBUG or TRACE logging for these? Will assuredly need it later :)
{code:java}
+        if (currentGroup == null) {
+            try {
+                List<RaftGroup> x = 
StreamSupport.stream(raftServer.getGroups().spliterator(), false).filter(group 
-> group.getGroupId().equals(metaGroupID)).collect(Collectors.toList());
+                if (x.size() == 1) {
+                    currentGroup = x.get(0);
+                }
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        }{code}
This isn't threadsafe. What about making the global {{currentGroup}} some 
{{AtomicReference}} and then copy it into a local variable when you need it? 
Admittedly, I don't quite grok what this is doing yet
{code:java}
+    public static String getHostName() {
+        try (DatagramSocket socket = new DatagramSocket()) {
+            socket.connect(InetAddress.getByName("8.8.8.8"), 10002);
+            return socket.getLocalAddress().getHostName();
+        } catch (Exception e) {
+            return "localhost";
+        }{code}
Heh :), switch to {{InetAddress.getLocalHost().getHostName()}} for now. I think 
we'll have to provide the ability later on to specify hostnames for quorum 
members (same reason as above).
{code:java}
+        RaftServerConfigKeys.setStorageDirs(properties,
+                Collections.singletonList(new File("/tmp/logservice/" + 
UUID.randomUUID().toString())));
{code}
Can we push down the log dir into {{LogServiceWorker}}? Will be necessary for 
production, but also nice to have all of the test artifacts in {{target/}} for 
the short-term.

I think you have this configuration for {{MasterServer}} already. Does it make 
sense to have some common configuration across the DML and DDL services?

All-in-all, this looks phenomenal. I'm inclined to merge this as-is, and spin 
out follow-on stuff to clean up the rest of this all. Thoughts, [~vrodionov], 
[~sergey.soldatov]?

> Implement server membership for LogService Metadata Service
> -----------------------------------------------------------
>
>                 Key: RATIS-334
>                 URL: https://issues.apache.org/jira/browse/RATIS-334
>             Project: Ratis
>          Issue Type: Sub-task
>          Components: LogService
>            Reporter: Josh Elser
>            Assignee: Sergey Soldatov
>            Priority: Major
>         Attachments: RATIS-334-v1.patch, RATIS-334-v3.patch, 
> RATIS-334-wip.patch
>
>
> Half of the Metadata Service for managing LogStreams.
> RATIS-279 is handling the "DDL" operations, and Rajesh suggested that we spin 
> out management of the servers which are available to back these LogStreams in 
> a second task. This is it.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to