Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Added random active shard selector for the getchanges operations #33

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -23,22 +23,18 @@ import org.elasticsearch.common.io.stream.StreamOutput
import org.elasticsearch.index.shard.ShardId
import org.elasticsearch.transport.RemoteClusterAwareRequest

class GetChangesRequest : SingleShardRequest<GetChangesRequest>, RemoteClusterAwareRequest {

val remoteNode: DiscoveryNode
class GetChangesRequest : SingleShardRequest<GetChangesRequest> {
val shardId : ShardId
val fromSeqNo: Long
val toSeqNo: Long

constructor(remoteNode: DiscoveryNode, shardId: ShardId, fromSeqNo: Long, toSeqNo: Long) : super(shardId.indexName) {
this.remoteNode = remoteNode
constructor(shardId: ShardId, fromSeqNo: Long, toSeqNo: Long) : super(shardId.indexName) {
this.shardId = shardId
this.fromSeqNo = fromSeqNo
this.toSeqNo = toSeqNo
}

constructor(input : StreamInput) : super(input) {
this.remoteNode = DiscoveryNode(input)
this.shardId = ShardId(input)
this.fromSeqNo = input.readLong()
this.toSeqNo = input.readVLong()
Expand All @@ -50,13 +46,8 @@ class GetChangesRequest : SingleShardRequest<GetChangesRequest>, RemoteClusterAw

override fun writeTo(out: StreamOutput) {
super.writeTo(out)
remoteNode.writeTo(out)
shardId.writeTo(out)
out.writeLong(fromSeqNo)
out.writeVLong(toSeqNo)
}

override fun getPreferredTargetNode(): DiscoveryNode {
return remoteNode
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ class TransportGetChangesAction @Inject constructor(threadPool: ThreadPool, clus
}

override fun shards(state: ClusterState, request: InternalRequest): ShardsIterator {
// TODO: Investigate using any active shards instead of just primary
return state.routingTable().shardRoutingTable(request.request().shardId).primaryShardIt()
// Random active shards
return state.routingTable().shardRoutingTable(request.request().shardId).activeInitializingShardsRandomIt()
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,6 @@ class ShardReplicationTask(id: Long, type: String, action: String, description:
log.info("Adding retentionlease at follower sequence number: ${indexShard.lastSyncedGlobalCheckpoint}")
retentionLeaseHelper.addRetentionLease(remoteShardId, indexShard.lastSyncedGlobalCheckpoint , followerShardId)

var node = primaryShardNode()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Won't you also need to clean up primaryShardNode method?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

addListenerToInterruptTask()

// Not really used yet as we only have one get changes action at a time.
Expand All @@ -154,7 +153,7 @@ class ShardReplicationTask(id: Long, type: String, action: String, description:
while (scope.isActive) {
rateLimiter.acquire()
try {
val changesResponse = getChanges(node, seqNo)
val changesResponse = getChanges(seqNo)
log.info("Got ${changesResponse.changes.size} changes starting from seqNo: $seqNo")
sequencer.send(changesResponse)
seqNo = changesResponse.changes.lastOrNull()?.seqNo()?.inc() ?: seqNo
Expand All @@ -165,7 +164,6 @@ class ShardReplicationTask(id: Long, type: String, action: String, description:
} catch (e: NodeNotConnectedException) {
log.info("Node not connected. Retrying request using a different node. $e")
delay(backOffForNodeDiscovery)
node = primaryShardNode()
rateLimiter.release()
continue
}
Expand All @@ -184,24 +182,9 @@ class ShardReplicationTask(id: Long, type: String, action: String, description:
sequencer.close()
}

private suspend fun primaryShardNode(): DiscoveryNode {
val clusterStateRequest = remoteClient.admin().cluster().prepareState()
.clear()
.setIndices(remoteShardId.indexName)
.setRoutingTable(true)
.setNodes(true)
.setIndicesOptions(IndicesOptions.strictSingleIndexNoExpandForbidClosed())
.request()
val remoteState = remoteClient.suspending(remoteClient.admin().cluster()::state)(clusterStateRequest).state
val shardRouting = remoteState.routingNodes.activePrimary(remoteShardId)
?: throw ShardNotFoundException(remoteShardId, "cluster: $remoteCluster")
return remoteState.nodes().get(shardRouting.currentNodeId())
?: throw NoSuchNodeException("remote: $remoteCluster:${shardRouting.currentNodeId()}")
}

private suspend fun getChanges(remoteNode: DiscoveryNode, fromSeqNo: Long): GetChangesResponse {
private suspend fun getChanges(fromSeqNo: Long): GetChangesResponse {
val remoteClient = client.getRemoteClusterClient(remoteCluster)
val request = GetChangesRequest(remoteNode, remoteShardId, fromSeqNo, fromSeqNo + batchSize)
val request = GetChangesRequest(remoteShardId, fromSeqNo, fromSeqNo + batchSize)
return remoteClient.suspendExecuteWithRetries(replicationMetadata = replicationMetadata,
action = GetChangesAction.INSTANCE, req = request, log = log)
}
Expand Down