44
44
import org .apache .kafka .common .network .ChannelState ;
45
45
import org .apache .kafka .common .network .ConnectionMode ;
46
46
import org .apache .kafka .common .network .ListenerName ;
47
+ import org .apache .kafka .common .network .NetworkReceive ;
47
48
import org .apache .kafka .common .network .NetworkSend ;
48
49
import org .apache .kafka .common .network .NetworkTestUtils ;
49
50
import org .apache .kafka .common .network .NioEchoServer ;
119
120
import java .util .Map ;
120
121
import java .util .Random ;
121
122
import java .util .Set ;
123
+ import java .util .concurrent .Semaphore ;
122
124
import java .util .concurrent .atomic .AtomicInteger ;
123
125
import java .util .function .Function ;
124
126
import java .util .stream .Collectors ;
@@ -1856,6 +1858,69 @@ public void testSslClientAuthRequiredOverriddenForSaslSslListener() throws Excep
1856
1858
verifySslClientAuthForSaslSslListener (false , SslClientAuth .REQUIRED );
1857
1859
}
1858
1860
1861
+ @ Test
1862
+ public void testServerSidePendingSendDuringReauthentication () throws Exception {
1863
+ SecurityProtocol securityProtocol = SecurityProtocol .SASL_PLAINTEXT ;
1864
+ TestJaasConfig jaasConfig = configureMechanisms ("PLAIN" , Collections .singletonList ("PLAIN" ));
1865
+ jaasConfig .createOrUpdateEntry (TestJaasConfig .LOGIN_CONTEXT_SERVER , PlainLoginModule .class .getName (), new HashMap <>());
1866
+ jaasConfig .setClientOptions ("PLAIN" , TestServerCallbackHandler .USERNAME , TestServerCallbackHandler .PASSWORD );
1867
+ String callbackPrefix = ListenerName .forSecurityProtocol (securityProtocol ).saslMechanismConfigPrefix ("PLAIN" );
1868
+ saslServerConfigs .put (callbackPrefix + BrokerSecurityConfigs .SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG ,
1869
+ TestServerCallbackHandler .class .getName ());
1870
+ server = createEchoServer (securityProtocol );
1871
+
1872
+ String node = "node1" ;
1873
+ try {
1874
+ createClientConnection (securityProtocol , node );
1875
+ NetworkTestUtils .waitForChannelReady (selector , node );
1876
+ server .verifyAuthenticationMetrics (1 , 0 );
1877
+
1878
+ /*
1879
+ * Now start the reauthentication on the connection. First, we have to sleep long enough so
1880
+ * that the next write will cause re-authentication
1881
+ */
1882
+ delay ((long ) (CONNECTIONS_MAX_REAUTH_MS_VALUE * 1.1 ));
1883
+ server .verifyReauthenticationMetrics (0 , 0 );
1884
+
1885
+ // block reauthentication to complete
1886
+ TestServerCallbackHandler .sem .acquire ();
1887
+
1888
+ String prefix = TestUtils .randomString (100 );
1889
+ // send a client request to start a reauthentication.
1890
+ selector .send (new NetworkSend (node , ByteBufferSend .sizePrefixed (ByteBuffer .wrap ((prefix + "-0" ).getBytes (StandardCharsets .UTF_8 )))));
1891
+ // wait till reauthentication is blocked
1892
+ TestUtils .waitForCondition (() -> {
1893
+ selector .poll (10L );
1894
+ return TestServerCallbackHandler .sem .hasQueuedThreads ();
1895
+ }, 5000 , "Reauthentication is not blocked" );
1896
+
1897
+ // Set the client's channel `send` to null to allow setting a new send on the server's selector.
1898
+ // Without this, NioEchoServer will throw an error while processing the client request,
1899
+ // since we're manually setting a server side send to simulate the issue.
1900
+ TestUtils .setFieldValue (selector .channel (node ), "send" , null );
1901
+
1902
+ // extract the channel id from the server's selector and directly set a send on it.
1903
+ String channelId = server .selector ().channels ().get (0 ).id ();
1904
+ String payload = prefix + "-1" ;
1905
+ server .selector ().send (new NetworkSend (channelId , ByteBufferSend .sizePrefixed (ByteBuffer .wrap (payload .getBytes (StandardCharsets .UTF_8 )))));
1906
+ // allow reauthentication to complete
1907
+ TestServerCallbackHandler .sem .release ();
1908
+
1909
+ TestUtils .waitForCondition (() -> {
1910
+ selector .poll (10L );
1911
+ for (NetworkReceive receive : selector .completedReceives ()) {
1912
+ assertEquals (payload , new String (Utils .toArray (receive .payload ()), StandardCharsets .UTF_8 ));
1913
+ return true ;
1914
+ }
1915
+ return false ;
1916
+ }, 5000 , "Failed Receive the server send after reauthentication" );
1917
+
1918
+ server .verifyReauthenticationMetrics (1 , 0 );
1919
+ } finally {
1920
+ closeClientConnectionIfNecessary ();
1921
+ }
1922
+ }
1923
+
1859
1924
private void verifySslClientAuthForSaslSslListener (boolean useListenerPrefix ,
1860
1925
SslClientAuth configuredClientAuth ) throws Exception {
1861
1926
@@ -2312,6 +2377,7 @@ public static class TestServerCallbackHandler extends PlainServerCallbackHandler
2312
2377
static final String USERNAME = "TestServerCallbackHandler-user" ;
2313
2378
static final String PASSWORD = "TestServerCallbackHandler-password" ;
2314
2379
private volatile boolean configured ;
2380
+ public static Semaphore sem = new Semaphore (1 );
2315
2381
2316
2382
@ Override
2317
2383
public void configure (Map <String , ?> configs , String mechanism , List <AppConfigurationEntry > jaasConfigEntries ) {
@@ -2325,7 +2391,14 @@ public void configure(Map<String, ?> configs, String mechanism, List<AppConfigur
2325
2391
protected boolean authenticate (String username , char [] password ) {
2326
2392
if (!configured )
2327
2393
throw new IllegalStateException ("Server callback handler not configured" );
2328
- return USERNAME .equals (username ) && new String (password ).equals (PASSWORD );
2394
+ try {
2395
+ sem .acquire ();
2396
+ return USERNAME .equals (username ) && new String (password ).equals (PASSWORD );
2397
+ } catch (InterruptedException e ) {
2398
+ throw new RuntimeException (e );
2399
+ } finally {
2400
+ sem .release ();
2401
+ }
2329
2402
}
2330
2403
}
2331
2404
0 commit comments