Skip to content

Commit

Permalink
Refactor core APIs for new method naming standards.
Browse files Browse the repository at this point in the history
  • Loading branch information
kuujo committed Jun 23, 2017
1 parent 3870ad4 commit 311ab29
Show file tree
Hide file tree
Showing 39 changed files with 63 additions and 58 deletions.
2 changes: 1 addition & 1 deletion core/src/main/java/io/atomix/cluster/ClusterService.java
Expand Up @@ -30,7 +30,7 @@ public interface ClusterService extends ListenerService<ClusterEvent, ClusterEve
* *
* @return local controller node * @return local controller node
*/ */
Node getLocalNode(); Node localNode();


/** /**
* Returns the set of current cluster members. * Returns the set of current cluster members.
Expand Down
6 changes: 3 additions & 3 deletions core/src/main/java/io/atomix/cluster/Node.java
Expand Up @@ -68,20 +68,20 @@ public boolean isReady() {
* *
* @return instance identifier * @return instance identifier
*/ */
NodeId getNodeId(); NodeId nodeId();


/** /**
* Returns the IP address of the controller instance. * Returns the IP address of the controller instance.
* *
* @return IP address * @return IP address
*/ */
InetAddress getInetAddress(); InetAddress getAddress();


/** /**
* Returns the TCP port on which the node listens for connections. * Returns the TCP port on which the node listens for connections.
* *
* @return TCP port * @return TCP port
*/ */
int getTcpPort(); int getPort();


} }
2 changes: 1 addition & 1 deletion core/src/main/java/io/atomix/cluster/Partition.java
Expand Up @@ -28,7 +28,7 @@ public interface Partition {
* *
* @return partition identifier * @return partition identifier
*/ */
PartitionId getPartitionId(); PartitionId partitionId();


/** /**
* Returns the controller nodes that are members of this partition. * Returns the controller nodes that are members of this partition.
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/java/io/atomix/cluster/PartitionId.java
Expand Up @@ -49,7 +49,7 @@ public static PartitionId from(int id) {
* @return number * @return number
*/ */
public int asInt() { public int asInt() {
return value(); return id();
} }


@Override @Override
Expand Down
23 changes: 14 additions & 9 deletions core/src/main/java/io/atomix/cluster/messaging/MessageSubject.java
Expand Up @@ -28,24 +28,29 @@
*/ */
public final class MessageSubject { public final class MessageSubject {


private final String value; private final String name;


public MessageSubject(String value) { public MessageSubject(String name) {
this.value = Preconditions.checkNotNull(value); this.name = Preconditions.checkNotNull(name);
} }


public String value() { /**
return value; * Returns the subject name.
*
* @return the message subject name
*/
public String name() {
return name;
} }


@Override @Override
public String toString() { public String toString() {
return value; return name;
} }


@Override @Override
public int hashCode() { public int hashCode() {
return value.hashCode(); return name.hashCode();
} }


@Override @Override
Expand All @@ -60,11 +65,11 @@ public boolean equals(Object obj) {
return false; return false;
} }
MessageSubject that = (MessageSubject) obj; MessageSubject that = (MessageSubject) obj;
return Objects.equals(this.value, that.value); return Objects.equals(this.name, that.name);
} }


// for serializer // for serializer
protected MessageSubject() { protected MessageSubject() {
this.value = ""; this.name = "";
} }
} }
6 changes: 3 additions & 3 deletions core/src/main/java/io/atomix/leadership/Leader.java
Expand Up @@ -49,7 +49,7 @@ public Leader(NodeId nodeId, long term, long termStartTime) {
* *
* @return node identifier * @return node identifier
*/ */
public NodeId getNodeId() { public NodeId nodeId() {
return nodeId; return nodeId;
} }


Expand All @@ -58,7 +58,7 @@ public NodeId getNodeId() {
* *
* @return leader term * @return leader term
*/ */
public long getTerm() { public long term() {
return term; return term;
} }


Expand All @@ -67,7 +67,7 @@ public long getTerm() {
* *
* @return current leader term start time * @return current leader term start time
*/ */
public long getTimestamp() { public long timestamp() {
return termStartTime; return termStartTime;
} }


Expand Down
2 changes: 1 addition & 1 deletion core/src/main/java/io/atomix/leadership/Leadership.java
Expand Up @@ -56,7 +56,7 @@ public String topic() {
* @return leader node identifier; will be null if there is no leader * @return leader node identifier; will be null if there is no leader
*/ */
public NodeId leaderNodeId() { public NodeId leaderNodeId() {
return leader == null ? null : leader.getNodeId(); return leader == null ? null : leader.nodeId();
} }


/** /**
Expand Down
Expand Up @@ -138,14 +138,14 @@ enum Status {
* *
* @return name * @return name
*/ */
String getName(); String name();


/** /**
* Returns the type of primitive. * Returns the type of primitive.
* *
* @return primitive type * @return primitive type
*/ */
Type getPrimitiveType(); Type primitiveType();


/** /**
* Purges state associated with this primitive. * Purges state associated with this primitive.
Expand Down
8 changes: 4 additions & 4 deletions core/src/main/java/io/atomix/primitives/Synchronous.java
Expand Up @@ -32,13 +32,13 @@ public Synchronous(T primitive) {
} }


@Override @Override
public String getName() { public String name() {
return primitive.getName(); return primitive.name();
} }


@Override @Override
public Type getPrimitiveType() { public Type primitiveType() {
return primitive.getPrimitiveType(); return primitive.primitiveType();
} }


@Override @Override
Expand Down
Expand Up @@ -26,7 +26,7 @@
public interface AsyncAtomicCounter extends DistributedPrimitive { public interface AsyncAtomicCounter extends DistributedPrimitive {


@Override @Override
default DistributedPrimitive.Type getPrimitiveType() { default DistributedPrimitive.Type primitiveType() {
return DistributedPrimitive.Type.COUNTER; return DistributedPrimitive.Type.COUNTER;
} }


Expand Down
Expand Up @@ -23,7 +23,7 @@
public interface AtomicCounter extends DistributedPrimitive { public interface AtomicCounter extends DistributedPrimitive {


@Override @Override
default DistributedPrimitive.Type getPrimitiveType() { default DistributedPrimitive.Type primitiveType() {
return DistributedPrimitive.Type.COUNTER; return DistributedPrimitive.Type.COUNTER;
} }


Expand Down
Expand Up @@ -47,7 +47,7 @@
public interface AsyncLeaderElector extends DistributedPrimitive { public interface AsyncLeaderElector extends DistributedPrimitive {


@Override @Override
default DistributedPrimitive.Type getPrimitiveType() { default DistributedPrimitive.Type primitiveType() {
return DistributedPrimitive.Type.LEADER_ELECTOR; return DistributedPrimitive.Type.LEADER_ELECTOR;
} }


Expand Down
Expand Up @@ -30,7 +30,7 @@
public interface LeaderElector extends DistributedPrimitive { public interface LeaderElector extends DistributedPrimitive {


@Override @Override
default DistributedPrimitive.Type getPrimitiveType() { default DistributedPrimitive.Type primitiveType() {
return DistributedPrimitive.Type.LEADER_ELECTOR; return DistributedPrimitive.Type.LEADER_ELECTOR;
} }


Expand Down
Expand Up @@ -26,7 +26,7 @@
public interface AsyncAtomicIdGenerator extends DistributedPrimitive { public interface AsyncAtomicIdGenerator extends DistributedPrimitive {


@Override @Override
default Type getPrimitiveType() { default Type primitiveType() {
return Type.ID_GENERATOR; return Type.ID_GENERATOR;
} }


Expand Down
Expand Up @@ -23,7 +23,7 @@
public interface AtomicIdGenerator extends DistributedPrimitive { public interface AtomicIdGenerator extends DistributedPrimitive {


@Override @Override
default Type getPrimitiveType() { default Type primitiveType() {
return Type.ID_GENERATOR; return Type.ID_GENERATOR;
} }


Expand Down
Expand Up @@ -27,7 +27,7 @@
public interface AsyncDistributedLock extends DistributedPrimitive { public interface AsyncDistributedLock extends DistributedPrimitive {


@Override @Override
default Type getPrimitiveType() { default Type primitiveType() {
return Type.LOCK; return Type.LOCK;
} }


Expand Down
Expand Up @@ -25,7 +25,7 @@
public interface DistributedLock extends DistributedPrimitive { public interface DistributedLock extends DistributedPrimitive {


@Override @Override
default Type getPrimitiveType() { default Type primitiveType() {
return Type.LOCK; return Type.LOCK;
} }


Expand Down
Expand Up @@ -26,7 +26,7 @@
public interface AsyncAtomicCounterMap<K> extends DistributedPrimitive { public interface AsyncAtomicCounterMap<K> extends DistributedPrimitive {


@Override @Override
default DistributedPrimitive.Type getPrimitiveType() { default DistributedPrimitive.Type primitiveType() {
return DistributedPrimitive.Type.COUNTER_MAP; return DistributedPrimitive.Type.COUNTER_MAP;
} }


Expand Down
Expand Up @@ -58,7 +58,7 @@
public interface AsyncConsistentMap<K, V> extends DistributedPrimitive { public interface AsyncConsistentMap<K, V> extends DistributedPrimitive {


@Override @Override
default DistributedPrimitive.Type getPrimitiveType() { default DistributedPrimitive.Type primitiveType() {
return DistributedPrimitive.Type.CONSISTENT_MAP; return DistributedPrimitive.Type.CONSISTENT_MAP;
} }


Expand Down
Expand Up @@ -39,7 +39,7 @@
public interface AsyncConsistentMultimap<K, V> extends DistributedPrimitive { public interface AsyncConsistentMultimap<K, V> extends DistributedPrimitive {


@Override @Override
default DistributedPrimitive.Type getPrimitiveType() { default DistributedPrimitive.Type primitiveType() {
return Type.CONSISTENT_MULTIMAP; return Type.CONSISTENT_MULTIMAP;
} }


Expand Down
Expand Up @@ -23,7 +23,7 @@
public interface AtomicCounterMap<K> extends DistributedPrimitive { public interface AtomicCounterMap<K> extends DistributedPrimitive {


@Override @Override
default DistributedPrimitive.Type getPrimitiveType() { default DistributedPrimitive.Type primitiveType() {
return DistributedPrimitive.Type.COUNTER_MAP; return DistributedPrimitive.Type.COUNTER_MAP;
} }


Expand Down
Expand Up @@ -44,7 +44,7 @@
public interface EventuallyConsistentMap<K, V> extends DistributedPrimitive { public interface EventuallyConsistentMap<K, V> extends DistributedPrimitive {


@Override @Override
default DistributedPrimitive.Type getPrimitiveType() { default DistributedPrimitive.Type primitiveType() {
return DistributedPrimitive.Type.EVENTUALLY_CONSISTENT_MAP; return DistributedPrimitive.Type.EVENTUALLY_CONSISTENT_MAP;
} }


Expand Down
Expand Up @@ -124,7 +124,7 @@ private <T> T complete(CompletableFuture<T> future) {
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
throw new ConsistentMapException.Interrupted(); throw new ConsistentMapException.Interrupted();
} catch (TimeoutException e) { } catch (TimeoutException e) {
throw new ConsistentMapException.Timeout(getName()); throw new ConsistentMapException.Timeout(name());
} catch (ExecutionException e) { } catch (ExecutionException e) {
Throwables.propagateIfPossible(e.getCause()); Throwables.propagateIfPossible(e.getCause());
throw new ConsistentMapException(e.getCause()); throw new ConsistentMapException(e.getCause());
Expand Down
Expand Up @@ -228,7 +228,7 @@ private <T> T complete(CompletableFuture<T> future) {
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
throw new ConsistentMapException.Interrupted(); throw new ConsistentMapException.Interrupted();
} catch (TimeoutException e) { } catch (TimeoutException e) {
throw new ConsistentMapException.Timeout(getName()); throw new ConsistentMapException.Timeout(name());
} catch (ExecutionException e) { } catch (ExecutionException e) {
Throwables.propagateIfPossible(e.getCause()); Throwables.propagateIfPossible(e.getCause());
throw new ConsistentMapException(e.getCause()); throw new ConsistentMapException(e.getCause());
Expand Down
Expand Up @@ -43,7 +43,7 @@
public interface WorkQueue<E> extends DistributedPrimitive { public interface WorkQueue<E> extends DistributedPrimitive {


@Override @Override
default DistributedPrimitive.Type getPrimitiveType() { default DistributedPrimitive.Type primitiveType() {
return DistributedPrimitive.Type.WORK_QUEUE; return DistributedPrimitive.Type.WORK_QUEUE;
} }


Expand Down
Expand Up @@ -34,7 +34,7 @@
public interface AsyncDistributedSet<E> extends DistributedPrimitive { public interface AsyncDistributedSet<E> extends DistributedPrimitive {


@Override @Override
default DistributedPrimitive.Type getPrimitiveType() { default DistributedPrimitive.Type primitiveType() {
return DistributedPrimitive.Type.SET; return DistributedPrimitive.Type.SET;
} }


Expand Down
Expand Up @@ -33,7 +33,7 @@
public interface AsyncAtomicValue<V> extends DistributedPrimitive { public interface AsyncAtomicValue<V> extends DistributedPrimitive {


@Override @Override
default DistributedPrimitive.Type getPrimitiveType() { default DistributedPrimitive.Type primitiveType() {
return DistributedPrimitive.Type.VALUE; return DistributedPrimitive.Type.VALUE;
} }


Expand Down
Expand Up @@ -60,7 +60,7 @@ public RaftClientCommunicator(String prefix, Serializer serializer, ClusterCommu
} }


private <T, U> CompletableFuture<U> sendAndReceive(MessageSubject subject, T request, MemberId memberId) { private <T, U> CompletableFuture<U> sendAndReceive(MessageSubject subject, T request, MemberId memberId) {
return clusterCommunicator.sendAndReceive(request, subject, serializer::encode, serializer::decode, NodeId.nodeId(memberId.value())); return clusterCommunicator.sendAndReceive(request, subject, serializer::encode, serializer::decode, NodeId.nodeId(memberId.id()));
} }


@Override @Override
Expand Down
Expand Up @@ -77,7 +77,7 @@ public RaftServerCommunicator(String prefix, Serializer serializer, ClusterCommu
} }


private <T, U> CompletableFuture<U> sendAndReceive(MessageSubject subject, T request, MemberId memberId) { private <T, U> CompletableFuture<U> sendAndReceive(MessageSubject subject, T request, MemberId memberId) {
return clusterCommunicator.sendAndReceive(request, subject, serializer::encode, serializer::decode, NodeId.nodeId(memberId.value())); return clusterCommunicator.sendAndReceive(request, subject, serializer::encode, serializer::decode, NodeId.nodeId(memberId.id()));
} }


@Override @Override
Expand Down Expand Up @@ -152,7 +152,7 @@ public CompletableFuture<AppendResponse> append(MemberId memberId, AppendRequest


@Override @Override
public void publish(MemberId memberId, PublishRequest request) { public void publish(MemberId memberId, PublishRequest request) {
clusterCommunicator.unicast(request, context.publishSubject(request.getSession()), serializer::encode, NodeId.nodeId(memberId.value())); clusterCommunicator.unicast(request, context.publishSubject(request.getSession()), serializer::encode, NodeId.nodeId(memberId.id()));
} }


@Override @Override
Expand Down
Expand Up @@ -40,7 +40,7 @@
public interface TransactionContext extends DistributedPrimitive { public interface TransactionContext extends DistributedPrimitive {


@Override @Override
default DistributedPrimitive.Type getPrimitiveType() { default DistributedPrimitive.Type primitiveType() {
return DistributedPrimitive.Type.TRANSACTION_CONTEXT; return DistributedPrimitive.Type.TRANSACTION_CONTEXT;
} }


Expand All @@ -49,7 +49,7 @@ default DistributedPrimitive.Type getPrimitiveType() {
* *
* @return transaction id * @return transaction id
*/ */
TransactionId getTransactionId(); TransactionId transactionId();


/** /**
* Returns if this transaction context is open. * Returns if this transaction context is open.
Expand Down
4 changes: 2 additions & 2 deletions core/src/main/java/io/atomix/transaction/TransactionLog.java
Expand Up @@ -53,7 +53,7 @@ public TransactionId transactionId() {
* *
* @return the transaction lock version * @return the transaction lock version
*/ */
public long getVersion() { public long version() {
return version; return version;
} }


Expand All @@ -62,7 +62,7 @@ public long getVersion() {
* *
* @return a list of transaction log records * @return a list of transaction log records
*/ */
public List<T> getRecords() { public List<T> records() {
return records; return records;
} }


Expand Down

0 comments on commit 311ab29

Please sign in to comment.