Skip to content

Conversation

@scruz-denodo
Copy link

@scruz-denodo scruz-denodo commented Jul 5, 2024

Rationale for this change

Give access to information about the requests executing the current action to the authentication processs and middlewares.
Adding for now the client and server socket addresses.

This can be useful for checking client origin at authentication phase and to audit accesses on a custom middleware.

What changes are included in this PR?

Create the class org.apache.arrow.flight.RequestInfo for storing the information.
This RequestInfo is also available through org.apache.arrow.flight.CallInfo. By this way, it is available for the middlewares.
Make also available RequestInfo to the CallHeaderAuthenticator, BearerTokenAuthenticator and ServerCallHeaderAuthMiddleware.

I overloaded the methods:

  • org.apache.arrow.flight.auth2.CallHeaderAuthenticator#authenticate(org.apache.arrow.flight.CallHeaders, org.apache.arrow.flight.RequestInfo)
  • org.apache.arrow.flight.auth2.BearerTokenAuthenticator#validateBearer(java.lang.String, org.apache.arrow.flight.RequestInfo)

with a default implementation for maintaining backward compatibility and not breaking previous implementations.

Are these changes tested?

Tested manually after generated the artifacts with the changes. For now, there are no automated tests until checking if the changes are suitable.

@scruz-denodo scruz-denodo requested a review from lidavidm as a code owner July 5, 2024 11:57
@github-actions
Copy link

github-actions bot commented Jul 5, 2024

⚠️ GitHub issue apache/arrow-java#316 has been automatically assigned in GitHub to PR creator.

@lidavidm
Copy link
Member

lidavidm commented Jul 8, 2024

Does this strictly need to be built into Flight? Since you can always use the gRPC APIs directly

@scruz-denodo
Copy link
Author

Does this strictly need to be built into Flight? Since you can always use the gRPC APIs directly

If I am not wrong, using gRPC APIs is a partial solution. Correct me if I am saying something incorrect.

You can create a io.grpc.ServerInterceptor and attach it to the Flight-SQL server.
Something like:

 final MyServerInterceptor myServerInterceptor = new MyServerInterceptor();
 final Consumer<NettyServerBuilder> consumer = (builder) -> {
     builder.intercept(connectionDataServerInterceptor);
 };
flightServerBuilder.transportHint("grpc.builderConsumer", consumer);

The previous interceptor is executed when a client invokes any method. But it is called after the authentication process is done with org.apache.arrow.flight.auth2.CallHeaderAuthenticator.
So, you are no able to use the client information for deciding about the authentication.

@lidavidm
Copy link
Member

lidavidm commented Jul 8, 2024

But you're able to get the FlightService directly and fully control the server builder.

@scruz-denodo
Copy link
Author

But you're able to get the FlightService directly and fully control the server builder.

Maybe I'm missing something, but I don't see how it is possible to do that.

I only see a way of registering a io.grpc.ServerInterceptor on the Builder which is invoking something like the previous code.

 final MyServerInterceptor myServerInterceptor = new MyServerInterceptor();
 final Consumer<NettyServerBuilder> consumer = (builder) -> {
     builder.intercept(myServerInterceptor );
 };

This Consumer is invoked when the org.apache.arrow.flight.FlightServer.Builder#build is executed. This build method also executes the following code as last step.

      builder.intercept(new ServerInterceptorAdapter(interceptors));
      return new FlightServer(location, builder.build(), grpcExecutor);

With that code, the order of the ServerInterceptors added to gRPC provokes that the ServerInterceptorAdapter is invoked before the MyServerInterceptor instance.
So, the authentication process is executed before executing the custom io.grpc.ServerInterceptor.

I do not see a way of redefining the order of the interceptors passed to the builder to force the MyServerInterceptor being executed as first step.

Maybe a change for adding the possibility to redefine that order would fit better?

@lidavidm
Copy link
Member

lidavidm commented Jul 8, 2024

/**
* Creates a Flight service.
*
* @param allocator Memory allocator
* @param producer Specifies the service api
* @param authHandler Authentication handler
* @param executor Executor service
* @return FlightBindingService
*/
public static BindableService createFlightService(
BufferAllocator allocator,
FlightProducer producer,
ServerAuthHandler authHandler,
ExecutorService executor) {
return new FlightBindingService(allocator, producer, authHandler, executor);
}

@scruz-denodo
Copy link
Author

So, if I understand correctly, you are suggesting to avoid the usage of the builder org.apache.arrow.flight.FlightServer.Builder for creating the FlightServer instance.
Reimplementing a new builder for creating the FlightServer object in a controlled code where you can do whetever you want.

In such case, in my opinion, it is a workaround but not the best way to do that thinking on code maintenance and future updates of the library.
I think that the idea should be instantiating the FlightServer through the official apache-arrow builder org.apache.arrow.flight.FlightServer.Builder. By this way, any change on the builder code in future releases will not be lost for people reimplementing that logic.

@lidavidm
Copy link
Member

lidavidm commented Jul 9, 2024

It is more likely we will get rid of FlightBuilder than the other way around.

@scruz-denodo
Copy link
Author

I will try to test that approach today

@scruz-denodo
Copy link
Author

Hi, I still did not have the chance to test your suggestion.

Nevertheless, I was looking the code. It is not only required to create a new org.apache.arrow.flight.FlightServer.Builder, you have to duplicate the entire org.apache.arrow.flight.FlightServer because the constructor of that class is private.

There are few methods on that class, but I continue to think that duplicating code is not a good practice.

It is clear that instantiating the object by this way will allow to configure the interceptors in the desired order. You can implement one which stores the client information on a ThreadLocal variable, so the subsequent interceptors (the flight one) have the information available.

In my opinion, it would be better to have that information through an API method, or having the option to configure the order of the interceptors. For example, allowing to define the interceptors executed prior to the flight one, and other interceptors for being executed after that.

I think that is the objective of the issue apache/arrow-java#316

@github-actions
Copy link

⚠️ GitHub issue #31924 has no components, please add labels for components.

@dbelozerovx1
Copy link

hey there, whats up with an issue, receiving a standard information like client ip is really scuffed right now, since you often need it at the producer level, not only on interceptor level

@lidavidm
Copy link
Member

The recommendation is still to use gRPC directly. gRPC itself only makes it available at that level, so this is no different, and there's not the maintainer bandwidth to wrap more of gRPC (hence: just use gRPC).

@dbelozerovx1
Copy link

Not quite sure, most libs and langs allow you to retrieve an adress from plain grpc calls, or propagate it from the context, but in flight, especially in Java impl, interceptors called after middlewares, so you cant really do anything with this info outside of interceptor level, maybe i am wrong, i would much appreciate if some examples will be provided

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants