abhishekagarwal87 commented on code in PR #16974:
URL: https://github.com/apache/druid/pull/16974#discussion_r1753416622
##########
processing/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java:
##########
@@ -64,13 +66,15 @@ public class HttpInputSource
private final PasswordProvider httpAuthenticationPasswordProvider;
private final SystemFields systemFields;
private final HttpInputSourceConfig config;
+ private final Map<String, String> headersMap;
Review Comment:
```suggestion
private final Map<String, String> requestHeaders;
```
##########
processing/src/test/java/org/apache/druid/data/input/impl/HttpEntityTest.java:
##########
Review Comment:
can you add one test with non-empty headers map?
##########
processing/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java:
##########
@@ -100,6 +98,29 @@ public static void
throwIfInvalidProtocols(HttpInputSourceConfig config, List<UR
}
}
+ public static void throwIfForbiddenHeaders(HttpInputSourceConfig config,
Map<String, String> headersMap)
+ {
+ if (!config.getAllowedHeaders().isEmpty() && headersMap.size() > 0) {
+ Set<String> forbiddenHeaderSet = headersMap.keySet()
+ .stream()
+ .map(StringUtils::toLowerCase)
+ .filter(h ->
!config.getAllowedHeaders().contains(h))
+ .collect(Collectors.toSet());
+ if (!forbiddenHeaderSet.isEmpty()) {
+ throw new IAE("Got forbidden headers %s, allowed headers are only %s ",
+ forbiddenHeaderSet, config.getAllowedHeaders());
+ }
+ }
Review Comment:
Also please use InvalidInput.exception.
##########
processing/src/main/java/org/apache/druid/data/input/impl/HttpEntity.java:
##########
@@ -80,10 +85,15 @@ public Predicate<Throwable> getRetryCondition()
return t -> t instanceof IOException;
}
- public static InputStream openInputStream(URI object, String userName,
PasswordProvider passwordProvider, long offset)
+ public static InputStream openInputStream(URI object, String userName,
PasswordProvider passwordProvider, long offset, final Map<String, String>
requestHeaders)
throws IOException
{
final URLConnection urlConnection = object.toURL().openConnection();
+ if (requestHeaders.size() > 0) {
Review Comment:
also need to check that requestHeaders is not null.
##########
processing/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java:
##########
@@ -100,6 +98,29 @@ public static void
throwIfInvalidProtocols(HttpInputSourceConfig config, List<UR
}
}
+ public static void throwIfForbiddenHeaders(HttpInputSourceConfig config,
Map<String, String> headersMap)
+ {
+ if (!config.getAllowedHeaders().isEmpty() && headersMap.size() > 0) {
+ Set<String> forbiddenHeaderSet = headersMap.keySet()
+ .stream()
+ .map(StringUtils::toLowerCase)
+ .filter(h ->
!config.getAllowedHeaders().contains(h))
+ .collect(Collectors.toSet());
+ if (!forbiddenHeaderSet.isEmpty()) {
+ throw new IAE("Got forbidden headers %s, allowed headers are only %s ",
+ forbiddenHeaderSet, config.getAllowedHeaders());
+ }
+ }
Review Comment:
this could be simplified to
```
for key in headersMap
if (!config.allowedHeaders.contains(key))
throw new IAE(" Header [%s] is not allowed to be set. Only headers are
allowed are [%s]. You can allow the headers by changing property <insert
property name> ",
key, config.getAllowedHeaders());
```
##########
processing/src/main/java/org/apache/druid/data/input/impl/HttpEntity.java:
##########
@@ -80,10 +85,15 @@ public Predicate<Throwable> getRetryCondition()
return t -> t instanceof IOException;
}
- public static InputStream openInputStream(URI object, String userName,
PasswordProvider passwordProvider, long offset)
+ public static InputStream openInputStream(URI object, String userName,
PasswordProvider passwordProvider, long offset, final Map<String, String>
requestHeaders)
throws IOException
{
final URLConnection urlConnection = object.toURL().openConnection();
+ if (requestHeaders.size() > 0) {
Review Comment:
if not, then requestHeaders is not nullable.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]