99import org .elasticsearch .action .ActionListener ;
1010import org .elasticsearch .action .support .PlainActionFuture ;
1111import org .elasticsearch .client .Client ;
12+ import org .elasticsearch .common .settings .Setting ;
1213import org .elasticsearch .common .unit .TimeValue ;
1314import org .elasticsearch .index .Index ;
1415import org .elasticsearch .index .seqno .RetentionLeaseActions ;
1819
1920import java .util .Locale ;
2021import java .util .Optional ;
21-
22- import static org .elasticsearch .index .seqno .RetentionLeaseActions .RETAIN_ALL ;
22+ import java .util .concurrent .TimeUnit ;
2323
2424public class CcrRetentionLeases {
2525
26+ // this setting is intentionally not registered, it is only used in tests
27+ public static final Setting <TimeValue > RETENTION_LEASE_RENEW_INTERVAL_SETTING =
28+ Setting .timeSetting (
29+ "index.ccr.retention_lease.renew_interval" ,
30+ new TimeValue (5 , TimeUnit .MINUTES ),
31+ new TimeValue (0 , TimeUnit .MILLISECONDS ),
32+ Setting .Property .NodeScope );
33+
2634 /**
2735 * The retention lease ID used by followers.
2836 *
@@ -52,20 +60,22 @@ public static String retentionLeaseId(
5260 * Synchronously requests to add a retention lease with the specified retention lease ID on the specified leader shard using the given
5361 * remote client. Note that this method will block up to the specified timeout.
5462 *
55- * @param leaderShardId the leader shard ID
56- * @param retentionLeaseId the retention lease ID
57- * @param remoteClient the remote client on which to execute this request
58- * @param timeout the timeout
63+ * @param leaderShardId the leader shard ID
64+ * @param retentionLeaseId the retention lease ID
65+ * @param retainingSequenceNumber the retaining sequence number
66+ * @param remoteClient the remote client on which to execute this request
67+ * @param timeout the timeout
5968 * @return an optional exception indicating whether or not the retention lease already exists
6069 */
6170 public static Optional <RetentionLeaseAlreadyExistsException > syncAddRetentionLease (
6271 final ShardId leaderShardId ,
6372 final String retentionLeaseId ,
73+ final long retainingSequenceNumber ,
6474 final Client remoteClient ,
6575 final TimeValue timeout ) {
6676 try {
6777 final PlainActionFuture <RetentionLeaseActions .Response > response = new PlainActionFuture <>();
68- asyncAddRetentionLease (leaderShardId , retentionLeaseId , remoteClient , response );
78+ asyncAddRetentionLease (leaderShardId , retentionLeaseId , retainingSequenceNumber , remoteClient , response );
6979 response .actionGet (timeout );
7080 return Optional .empty ();
7181 } catch (final RetentionLeaseAlreadyExistsException e ) {
@@ -78,39 +88,43 @@ public static Optional<RetentionLeaseAlreadyExistsException> syncAddRetentionLea
7888 * remote client. Note that this method will return immediately, with the specified listener callback invoked to indicate a response
7989 * or failure.
8090 *
81- * @param leaderShardId the leader shard ID
82- * @param retentionLeaseId the retention lease ID
83- * @param remoteClient the remote client on which to execute this request
84- * @param listener the listener
91+ * @param leaderShardId the leader shard ID
92+ * @param retentionLeaseId the retention lease ID
93+ * @param retainingSequenceNumber the retaining sequence number
94+ * @param remoteClient the remote client on which to execute this request
95+ * @param listener the listener
8596 */
8697 public static void asyncAddRetentionLease (
8798 final ShardId leaderShardId ,
8899 final String retentionLeaseId ,
100+ final long retainingSequenceNumber ,
89101 final Client remoteClient ,
90102 final ActionListener <RetentionLeaseActions .Response > listener ) {
91103 final RetentionLeaseActions .AddRequest request =
92- new RetentionLeaseActions .AddRequest (leaderShardId , retentionLeaseId , RETAIN_ALL , "ccr" );
104+ new RetentionLeaseActions .AddRequest (leaderShardId , retentionLeaseId , retainingSequenceNumber , "ccr" );
93105 remoteClient .execute (RetentionLeaseActions .Add .INSTANCE , request , listener );
94106 }
95107
96108 /**
97109 * Synchronously requests to renew a retention lease with the specified retention lease ID on the specified leader shard using the given
98110 * remote client. Note that this method will block up to the specified timeout.
99111 *
100- * @param leaderShardId the leader shard ID
101- * @param retentionLeaseId the retention lease ID
102- * @param remoteClient the remote client on which to execute this request
103- * @param timeout the timeout
112+ * @param leaderShardId the leader shard ID
113+ * @param retentionLeaseId the retention lease ID
114+ * @param retainingSequenceNumber the retaining sequence number
115+ * @param remoteClient the remote client on which to execute this request
116+ * @param timeout the timeout
104117 * @return an optional exception indicating whether or not the retention lease already exists
105118 */
106119 public static Optional <RetentionLeaseNotFoundException > syncRenewRetentionLease (
107120 final ShardId leaderShardId ,
108121 final String retentionLeaseId ,
122+ final long retainingSequenceNumber ,
109123 final Client remoteClient ,
110124 final TimeValue timeout ) {
111125 try {
112126 final PlainActionFuture <RetentionLeaseActions .Response > response = new PlainActionFuture <>();
113- asyncRenewRetentionLease (leaderShardId , retentionLeaseId , remoteClient , response );
127+ asyncRenewRetentionLease (leaderShardId , retentionLeaseId , retainingSequenceNumber , remoteClient , response );
114128 response .actionGet (timeout );
115129 return Optional .empty ();
116130 } catch (final RetentionLeaseNotFoundException e ) {
@@ -123,18 +137,20 @@ public static Optional<RetentionLeaseNotFoundException> syncRenewRetentionLease(
123137 * given remote client. Note that this method will return immediately, with the specified listener callback invoked to indicate a
124138 * response or failure.
125139 *
126- * @param leaderShardId the leader shard ID
127- * @param retentionLeaseId the retention lease ID
128- * @param remoteClient the remote client on which to execute this request
129- * @param listener the listener
140+ * @param leaderShardId the leader shard ID
141+ * @param retentionLeaseId the retention lease ID
142+ * @param retainingSequenceNumber the retaining sequence number
143+ * @param remoteClient the remote client on which to execute this request
144+ * @param listener the listener
130145 */
131146 public static void asyncRenewRetentionLease (
132147 final ShardId leaderShardId ,
133148 final String retentionLeaseId ,
149+ final long retainingSequenceNumber ,
134150 final Client remoteClient ,
135151 final ActionListener <RetentionLeaseActions .Response > listener ) {
136152 final RetentionLeaseActions .RenewRequest request =
137- new RetentionLeaseActions .RenewRequest (leaderShardId , retentionLeaseId , RETAIN_ALL , "ccr" );
153+ new RetentionLeaseActions .RenewRequest (leaderShardId , retentionLeaseId , retainingSequenceNumber , "ccr" );
138154 remoteClient .execute (RetentionLeaseActions .Renew .INSTANCE , request , listener );
139155 }
140156
0 commit comments