Uploaded image for project: 'Solr'
  1. Solr
  2. SOLR-9200

Add Delegation Token Support to Solr

    Details

    • Type: New Feature
    • Status: Closed
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 6.2, 7.0
    • Component/s: security
    • Labels:
      None

      Description

      SOLR-7468 added support for kerberos authentication via the hadoop authentication filter. Hadoop also has support for an authentication filter that supports delegation tokens, which allow authenticated users the ability to grab/renew/delete a token that can be used to bypass the normal authentication path for a time. This is useful in a variety of use cases:
      1) distributed clients (e.g. MapReduce) where each client may not have access to the user's kerberos credentials. Instead, the job runner can grab a delegation token and use that during task execution.
      2) If the load on the kerberos server is too high, delegation tokens can avoid hitting the kerberos server after the first request
      3) If requests/permissions need to be delegated to another user: the more privileged user can request a delegation token that can be passed to the less privileged user.

      Note to self:
      In https://issues.apache.org/jira/browse/SOLR-7468?focusedCommentId=14579636&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14579636 I made the following comment which I need to investigate further, since I don't know if anything changed in this area:

      3) I'm a little concerned with the "NoContext" code in KerberosPlugin moving forward (I understand this is more a generic auth question than kerberos specific). For example, in the latest version of the filter we are using at Cloudera, we play around with the ServletContext in order to pass information around (https://github.com/cloudera/lucene-solr/blob/cdh5-4.10.3_5.4.2/solr/core/src/java/org/apache/solr/servlet/SolrHadoopAuthenticationFilter.java#L106). Is there any way we can get the actual ServletContext in a plugin?

      1. SOLR-9200_branch_6x.patch
        156 kB
        Gregory Chanan
      2. SOLR-9200_branch_6x.patch
        156 kB
        Gregory Chanan
      3. SOLR-9200_branch_6x.patch
        156 kB
        Gregory Chanan
      4. SOLR-9200.patch
        156 kB
        Gregory Chanan
      5. SOLR-9200.patch
        155 kB
        Gregory Chanan
      6. SOLR-9200.patch
        30 kB
        Gregory Chanan
      7. SOLR-9200.patch
        100 kB
        Gregory Chanan
      8. SOLR-9200.patch
        100 kB
        Gregory Chanan

        Issue Links

          Activity

          Hide
          gchanan Gregory Chanan added a comment - - edited

          I started working on this. One issue I immediately hit was HADOOP-12767 – it appears upgrading the httpclient version causes a null check to need to be inserted on the path of delegation token checking.

          Also note that HADOOP-12767 was fixed in hadoop 2.8 but the latest stable release is 2.7.2.

          Show
          gchanan Gregory Chanan added a comment - - edited I started working on this. One issue I immediately hit was HADOOP-12767 – it appears upgrading the httpclient version causes a null check to need to be inserted on the path of delegation token checking. Also note that HADOOP-12767 was fixed in hadoop 2.8 but the latest stable release is 2.7.2.
          Hide
          erickerickson Erick Erickson added a comment -

          Greg:

          What's the status of this one? Are you actively working on it or is it on the back burner?

          Show
          erickerickson Erick Erickson added a comment - Greg: What's the status of this one? Are you actively working on it or is it on the back burner?
          Hide
          gchanan Gregory Chanan added a comment -

          actively working on it – are you interested in the feature?

          Show
          gchanan Gregory Chanan added a comment - actively working on it – are you interested in the feature?
          Hide
          erickerickson Erick Erickson added a comment -

          It's something I had a question from a client about....

          Show
          erickerickson Erick Erickson added a comment - It's something I had a question from a client about....
          Hide
          gchanan Gregory Chanan added a comment -

          Did some more work on this. Currently blocked because we need HADOOP-11492, which is only available in hadoop 2.7.0+. Upgrading to hadoop 2.7.2 is currently being tracked in SOLR-9076.

          Show
          gchanan Gregory Chanan added a comment - Did some more work on this. Currently blocked because we need HADOOP-11492 , which is only available in hadoop 2.7.0+. Upgrading to hadoop 2.7.2 is currently being tracked in SOLR-9076 .
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Hi Gregory Chanan, please let me know if I can help in any way with this.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Hi Gregory Chanan , please let me know if I can help in any way with this.
          Hide
          gchanan Gregory Chanan added a comment -

          Ishan Chattopadhyaya thanks so much! I attached my current patch; I would very much appreciate a review. My first couple of runs through the tests failed, although I haven't determined if those are just existing flaky tests or not.

          I'll write up some notes shortly.

          Show
          gchanan Gregory Chanan added a comment - Ishan Chattopadhyaya thanks so much! I attached my current patch; I would very much appreciate a review. My first couple of runs through the tests failed, although I haven't determined if those are just existing flaky tests or not. I'll write up some notes shortly.
          Hide
          gchanan Gregory Chanan added a comment -

          Functionality:

          1) This patch allows configuration of delegation tokens for the KerberosPlugin. Existing
          configurations will not change and will not support delegation tokens.

          The configuration parameters are as follows:

          Key Type Default Description
          solr.kerberos.delegation.token.enabled boolean false Set to true to enable delegation tokens
          solr.kerberos.delegation.token.kind String solr-dt Type name of delegation tokens, most likely doesn't need to change
          solr.kerberos.delegation.token.validity integer 36000 Lifetime in seconds for which delegation tokens are valid
          solr.kerberos.delegation.token.signer.secret.provider String zookeeper Where delegation token information is stored internally; if not "zookeeper" delegation tokens won't work across solr servers
          solr.kerberos.delegation.token.signer.secret.provider.zookeper.path String (chrooted path) + /token Zookeeper location where secret provider information is stored
          solr.kerberos.delegation.token.secret.manager.znode.working.path String (chrooted path) + /zkdtsm Zookeeper location where token information is stored

          2) Includes solrj support for delegation tokens as follows:
          a) DelegationTokenRequest/DelegationTokenResponse can be used to get/cancel/renew delegation tokens
          b) HttpSolrClient.Builder now includes a "withDelegationToken" function for creating an HttpSolrClient
          that uses a delegation token to authenticate
          Note that hadoop's delegation token responses are in json map format, so there is a new ResponseParser
          for that in DelegationTokenResponse; I couldn't find an existing response parser that worked

          Issues / Workarounds:

          3) AuthenticationPlugin has an incompatible change that means this should only go into the next major version.
          Basically:

          void doAuthenticate

          changed to:

          boolean doAuthenticate

          This is to support cases where authentication succeeds, but solr itself shouldn't process the request; e.g.
          in the delegation token management operations (get, cancel, renew). The boolean, if false, indicates a short
          circuit out of the rest of the solr dispatch logic.

          4) DelegationTokenKerberosFilter includes a workaround for null query strings. The versions of
          hadoop / httpclient that we use don't work with null HttpServletRequest query strings, which the jetty
          version we use can provide. This can be solved by using HTTPCLIENT-1746 (not released yet) or HADOOP-12767
          (not released in a stable version).

          5) hadoop's delegation token code writes to a closed PrintWriter; this doesn't seem to be a problem for the
          version of jetty that hadoop uses, but it causes an issue with our version. I filed HADOOP-13346 to fix that,
          until then, I wrote a PrintWriterWrapper that ignores closes.

          6) The hadoop zookeeper delegation token code uses Curator rather than SolrZkClient; the conversion
          from SolrZkClient is messy in a few places:
          a) We use the ZkController.ZkClient's ACL Provider for the delegation tokens in ZK, but it's not obvious this
          is what you'd actually want to use. For example, it may be reasonable to have most solr znodes be readable
          (because clients read e.g. clusterstate.json), but you probably don't want the delegation token information
          to be readable by anyone outside "solr". I haven't checked recently, but I don't think we provide any
          built in ACLProviders that would do something reasonable here in the general case. Basically, it's easy to
          get this wrong and leak security information.
          b) Getting credentials information to curator also isn't great. Again, we use ZkController.ZkClient's credentials
          (at AuthenticationPlugin.init) time, but given the "setZkCredentialsToAddAutomatically" function, these could
          in theory change. I haven't looked into changing that into a builder so it's guaranteed not to change.
          c) Retrying logic is handled completely differently. In theory, you'd like the curator logic to follow ZkController.ZkClient's
          ZkClientConnectionStrategy logic, but it's not clear how to implement this. Instead, we just use curator's ExponentialBackoffRetry.

          Show
          gchanan Gregory Chanan added a comment - Functionality: 1) This patch allows configuration of delegation tokens for the KerberosPlugin. Existing configurations will not change and will not support delegation tokens. The configuration parameters are as follows: Key Type Default Description solr.kerberos.delegation.token.enabled boolean false Set to true to enable delegation tokens solr.kerberos.delegation.token.kind String solr-dt Type name of delegation tokens, most likely doesn't need to change solr.kerberos.delegation.token.validity integer 36000 Lifetime in seconds for which delegation tokens are valid solr.kerberos.delegation.token.signer.secret.provider String zookeeper Where delegation token information is stored internally; if not "zookeeper" delegation tokens won't work across solr servers solr.kerberos.delegation.token.signer.secret.provider.zookeper.path String (chrooted path) + /token Zookeeper location where secret provider information is stored solr.kerberos.delegation.token.secret.manager.znode.working.path String (chrooted path) + /zkdtsm Zookeeper location where token information is stored 2) Includes solrj support for delegation tokens as follows: a) DelegationTokenRequest/DelegationTokenResponse can be used to get/cancel/renew delegation tokens b) HttpSolrClient.Builder now includes a "withDelegationToken" function for creating an HttpSolrClient that uses a delegation token to authenticate Note that hadoop's delegation token responses are in json map format, so there is a new ResponseParser for that in DelegationTokenResponse; I couldn't find an existing response parser that worked Issues / Workarounds: 3) AuthenticationPlugin has an incompatible change that means this should only go into the next major version. Basically: void doAuthenticate changed to: boolean doAuthenticate This is to support cases where authentication succeeds, but solr itself shouldn't process the request; e.g. in the delegation token management operations (get, cancel, renew). The boolean, if false, indicates a short circuit out of the rest of the solr dispatch logic. 4) DelegationTokenKerberosFilter includes a workaround for null query strings. The versions of hadoop / httpclient that we use don't work with null HttpServletRequest query strings, which the jetty version we use can provide. This can be solved by using HTTPCLIENT-1746 (not released yet) or HADOOP-12767 (not released in a stable version). 5) hadoop's delegation token code writes to a closed PrintWriter; this doesn't seem to be a problem for the version of jetty that hadoop uses, but it causes an issue with our version. I filed HADOOP-13346 to fix that, until then, I wrote a PrintWriterWrapper that ignores closes. 6) The hadoop zookeeper delegation token code uses Curator rather than SolrZkClient; the conversion from SolrZkClient is messy in a few places: a) We use the ZkController.ZkClient's ACL Provider for the delegation tokens in ZK, but it's not obvious this is what you'd actually want to use. For example, it may be reasonable to have most solr znodes be readable (because clients read e.g. clusterstate.json), but you probably don't want the delegation token information to be readable by anyone outside "solr". I haven't checked recently, but I don't think we provide any built in ACLProviders that would do something reasonable here in the general case. Basically, it's easy to get this wrong and leak security information. b) Getting credentials information to curator also isn't great. Again, we use ZkController.ZkClient's credentials (at AuthenticationPlugin.init) time, but given the "setZkCredentialsToAddAutomatically" function, these could in theory change. I haven't looked into changing that into a builder so it's guaranteed not to change. c) Retrying logic is handled completely differently. In theory, you'd like the curator logic to follow ZkController.ZkClient's ZkClientConnectionStrategy logic, but it's not clear how to implement this. Instead, we just use curator's ExponentialBackoffRetry.
          Hide
          gchanan Gregory Chanan added a comment -

          CC Anshum Gupta, who worked on SOLR-7274.

          Show
          gchanan Gregory Chanan added a comment - CC Anshum Gupta , who worked on SOLR-7274 .
          Hide
          gchanan Gregory Chanan added a comment -

          New patch that passes the forbidden api checks. I had to add one suppression as follows:

          HttpServletResponse rspCloseShield = new HttpServletResponseWrapper(frsp) {
                @SuppressForbidden(reason = "Hadoop DelegationTokenAuthenticationFilter uses response writer, this" +
                    "is providing a CloseShield on top of that")
                @Override
                public PrintWriter getWriter() throws IOException {
                  final PrintWriter pw = new PrintWriterWrapper(frsp.getWriter()) {
                    @Override
                    public void close() {};
                  };
                  return pw;
                }
              };
          

          I'm not 100% sure if the getWriter problem affects the hadoop usage, which is here:
          https://github.com/apache/hadoop/blob/9d46a49c746b9e1ef552dbb10d1e22f87db68c76/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java#L280-L282

          I can certainly submit a patch to hadoop (or add it to HADOOP-13346) to use OutputStream instead – Uwe Schindler?

          Show
          gchanan Gregory Chanan added a comment - New patch that passes the forbidden api checks. I had to add one suppression as follows: HttpServletResponse rspCloseShield = new HttpServletResponseWrapper(frsp) { @SuppressForbidden(reason = "Hadoop DelegationTokenAuthenticationFilter uses response writer, this " + "is providing a CloseShield on top of that" ) @Override public PrintWriter getWriter() throws IOException { final PrintWriter pw = new PrintWriterWrapper(frsp.getWriter()) { @Override public void close() {}; }; return pw; } }; I'm not 100% sure if the getWriter problem affects the hadoop usage, which is here: https://github.com/apache/hadoop/blob/9d46a49c746b9e1ef552dbb10d1e22f87db68c76/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java#L280-L282 I can certainly submit a patch to hadoop (or add it to HADOOP-13346 ) to use OutputStream instead – Uwe Schindler ?
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Hi Greg. I'm reviewing the patch and shall post my comments soon. However, I just thought I'll share my thoughts on the following:

          3) AuthenticationPlugin has an incompatible change that means this should only go into the next major version.

          Since AuthenticationPlugin is marked @lucene.experimental, I think we should be good to go with the change within 6x. Afair, we've done such incompatible changes in the past within 5x (when we were working on the BasicAuth functionality. Sorry, I don't have the exact JIRAs handy at the moment). Any 3rd party authentication plugins would suffer, but it should be immediately obvious to the developers how to fix it.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Hi Greg. I'm reviewing the patch and shall post my comments soon. However, I just thought I'll share my thoughts on the following: 3) AuthenticationPlugin has an incompatible change that means this should only go into the next major version. Since AuthenticationPlugin is marked @lucene.experimental, I think we should be good to go with the change within 6x. Afair, we've done such incompatible changes in the past within 5x (when we were working on the BasicAuth functionality. Sorry, I don't have the exact JIRAs handy at the moment). Any 3rd party authentication plugins would suffer, but it should be immediately obvious to the developers how to fix it.
          Hide
          anshumg Anshum Gupta added a comment -

          Don't think I can get to this one today or tomorrow but I'll try and take a look whenever I get time.

          Show
          anshumg Anshum Gupta added a comment - Don't think I can get to this one today or tomorrow but I'll try and take a look whenever I get time.
          Hide
          gchanan Gregory Chanan added a comment -

          Ishan Chattopadhyaya your argument sounds reasonable to me.

          Anshum Gupta and Ishan, thanks for taking a look.

          Show
          gchanan Gregory Chanan added a comment - Ishan Chattopadhyaya your argument sounds reasonable to me. Anshum Gupta and Ishan, thanks for taking a look.
          Hide
          gchanan Gregory Chanan added a comment -

          Looks like there is some adverse interaction between TestSolrCloudWithDelegationTokens and TestSolrCloudWithKerberosAlt when they run in the same jvm; in the same jvm TestSolrCloudWithKerberosAlt fails when run second. I beasted the test individually and it didn't fail in 100 runs.

          Show
          gchanan Gregory Chanan added a comment - Looks like there is some adverse interaction between TestSolrCloudWithDelegationTokens and TestSolrCloudWithKerberosAlt when they run in the same jvm; in the same jvm TestSolrCloudWithKerberosAlt fails when run second. I beasted the test individually and it didn't fail in 100 runs.
          Hide
          gchanan Gregory Chanan added a comment -

          Looks like there is some adverse interaction between TestSolrCloudWithDelegationTokens and TestSolrCloudWithKerberosAlt when they run in the same jvm; in the same jvm TestSolrCloudWithKerberosAlt fails when run second. I beasted the test individually and it didn't fail in 100 runs.

          Figured this out – in Krb5HttpClientBuilder the JaasConfiguration is set up statically with baseConfig whatever is the current config, but the tests like TestSolrCloudWithKerberosAlt require that the baseConfig is what they set it to at the beginning of test;

          Show
          gchanan Gregory Chanan added a comment - Looks like there is some adverse interaction between TestSolrCloudWithDelegationTokens and TestSolrCloudWithKerberosAlt when they run in the same jvm; in the same jvm TestSolrCloudWithKerberosAlt fails when run second. I beasted the test individually and it didn't fail in 100 runs. Figured this out – in Krb5HttpClientBuilder the JaasConfiguration is set up statically with baseConfig whatever is the current config, but the tests like TestSolrCloudWithKerberosAlt require that the baseConfig is what they set it to at the beginning of test;
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Apologies for delay in the review; I have caught on a viral infection and have still not fully recovered. Here are some comments from before I fell sick.

          1. Can we eliminate the NoContext and replace with AttributeOnlyServletContext?
          2. I think it makes sense to write a test for internode communication to ensure that delegation tokens based authentication is working end to end. The TestSolrCloudWithDelegationTokens seems to use a 2 node mini solr cloud cluster, but it wasn't apparent to me if there are any calls made to the cluster from the test that require internode communication (maybe there is and I've missed it).
          3. How can we ensure that no HttpSolrClients are created from elsewhere in Solr while delegation tokens based kerberos authentication is enabled (so that DTs are always passed on)? Do you think we should ban the creation of HSCs, and provide for a factory method?
          4. I see there are various places in Solr codebase where HSCs are created, e.g. SnitchContext, LeaderInitiatedRecoveryThread, OverseerAutoReplicaFailoverThread, OverseerCollectionMessageHandler, RecoveryStrategy, SyncStrategy, IndexFetcher, CdcrRequestHandler, CdcrUpdateLogSynchronizer, SolrConfigHandler, ManagedSchema, SolrCmdDistributor. These are for internode communication, but these clients do not get the delegation tokens passed on while making the request. Shouldn't they be using delegation tokens too, to make it consistent?
          5. Minor: Unused imports in AuthenticationPlugin
          6. Minor: Unnecessary change to SolrZkClient
          7. TestSolrCloudWithDelegationTokens:
            System.setProperty(KerberosPlugin.DELEGATION_TOKEN_PARAM, "true"); 
            

            line has been repeated in the test

          8.  solr.kerberos.delegation.token.signer.secret.provider.zookeper.path 	String 	(chrooted path) + /token 	Zookeeper location where secret provider information is stored
            solr.kerberos.delegation.token.secret.manager.znode.working.path 	String 	(chrooted path) + /zkdtsm 	Zookeeper location where token information is stored 

            Does it make more sense if we move all znodes related to security under a nested znode like /security/*, i.e. modify the defaults here to /security/zkdtsm and /security/token? It might be easier for sysadmins to know what purpose these znodes are present for.

          9. I liked the idea of Solr protecting these znodes right away through ZK ACLs; it would prevent against vulnerabilities for setups where the sysadmin has forgotten to turn it on for the right znodes. However, should we make that behaviour configurable and make it possible for the sysadmin to not ACL protect those znodes, and allow him to do so manually? The usecase I have in mind, and I'm not sure if this is a real world usecase or not, is where a sysadmin is dealing with a failed ZK setup, and is cloning all the znodes into a fresh ZK cluster: I imagine that such a clone would not work for those two znodes if Solr has ACL protected them.
          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Apologies for delay in the review; I have caught on a viral infection and have still not fully recovered. Here are some comments from before I fell sick. Can we eliminate the NoContext and replace with AttributeOnlyServletContext? I think it makes sense to write a test for internode communication to ensure that delegation tokens based authentication is working end to end. The TestSolrCloudWithDelegationTokens seems to use a 2 node mini solr cloud cluster, but it wasn't apparent to me if there are any calls made to the cluster from the test that require internode communication (maybe there is and I've missed it). How can we ensure that no HttpSolrClients are created from elsewhere in Solr while delegation tokens based kerberos authentication is enabled (so that DTs are always passed on)? Do you think we should ban the creation of HSCs, and provide for a factory method? I see there are various places in Solr codebase where HSCs are created, e.g. SnitchContext, LeaderInitiatedRecoveryThread, OverseerAutoReplicaFailoverThread, OverseerCollectionMessageHandler, RecoveryStrategy, SyncStrategy, IndexFetcher, CdcrRequestHandler, CdcrUpdateLogSynchronizer, SolrConfigHandler, ManagedSchema, SolrCmdDistributor. These are for internode communication, but these clients do not get the delegation tokens passed on while making the request. Shouldn't they be using delegation tokens too, to make it consistent? Minor: Unused imports in AuthenticationPlugin Minor: Unnecessary change to SolrZkClient TestSolrCloudWithDelegationTokens: System .setProperty(KerberosPlugin.DELEGATION_TOKEN_PARAM, " true " ); line has been repeated in the test solr.kerberos.delegation.token.signer.secret.provider.zookeper.path String (chrooted path) + /token Zookeeper location where secret provider information is stored solr.kerberos.delegation.token.secret.manager.znode.working.path String (chrooted path) + /zkdtsm Zookeeper location where token information is stored Does it make more sense if we move all znodes related to security under a nested znode like /security/*, i.e. modify the defaults here to /security/zkdtsm and /security/token? It might be easier for sysadmins to know what purpose these znodes are present for. I liked the idea of Solr protecting these znodes right away through ZK ACLs; it would prevent against vulnerabilities for setups where the sysadmin has forgotten to turn it on for the right znodes. However, should we make that behaviour configurable and make it possible for the sysadmin to not ACL protect those znodes, and allow him to do so manually? The usecase I have in mind, and I'm not sure if this is a real world usecase or not, is where a sysadmin is dealing with a failed ZK setup, and is cloning all the znodes into a fresh ZK cluster: I imagine that such a clone would not work for those two znodes if Solr has ACL protected them.
          Hide
          gchanan Gregory Chanan added a comment -

          Here's a new version of the patch, taking into account Ishan Chattopadhyaya's comments. In particular:

          1. All security related znodes have been moved under /security
          2. The existing ZkACLProviders now derived from SecurityAwareZkACLProvider, which by default does not allow read only access to znodes under /security for non-solr users.
          3. Moved all kerberos-related test setup from KerberosTestUtil to KerberosTestServices. This object does "all in one" setup, i.e. dealing with non-supported locales, kdc setup, Configuration management. It also handles resetting the Configuration at the end of the test, so other tests in the same jvm can run successfully.
          4. Got rid of extra imports.

          Show
          gchanan Gregory Chanan added a comment - Here's a new version of the patch, taking into account Ishan Chattopadhyaya 's comments. In particular: 1. All security related znodes have been moved under /security 2. The existing ZkACLProviders now derived from SecurityAwareZkACLProvider, which by default does not allow read only access to znodes under /security for non-solr users. 3. Moved all kerberos-related test setup from KerberosTestUtil to KerberosTestServices. This object does "all in one" setup, i.e. dealing with non-supported locales, kdc setup, Configuration management. It also handles resetting the Configuration at the end of the test, so other tests in the same jvm can run successfully. 4. Got rid of extra imports.
          Hide
          gchanan Gregory Chanan added a comment -

          sorry attached the wrong patch, should be correct now.

          Show
          gchanan Gregory Chanan added a comment - sorry attached the wrong patch, should be correct now.
          Hide
          gchanan Gregory Chanan added a comment -

          removed some unused imports.

          Show
          gchanan Gregory Chanan added a comment - removed some unused imports.
          Hide
          gchanan Gregory Chanan added a comment -

          I am planning to commit this soon if there are no objections.

          Show
          gchanan Gregory Chanan added a comment - I am planning to commit this soon if there are no objections.
          Hide
          jira-bot ASF subversion and git services added a comment -

          Commit 7bf019a9c2779368133850f141174febede6ad96 in lucene-solr's branch refs/heads/master from Gregory Chanan
          [ https://git-wip-us.apache.org/repos/asf?p=lucene-solr.git;h=7bf019a ]

          SOLR-9200: Add Delegation Token Support to Solr

          Show
          jira-bot ASF subversion and git services added a comment - Commit 7bf019a9c2779368133850f141174febede6ad96 in lucene-solr's branch refs/heads/master from Gregory Chanan [ https://git-wip-us.apache.org/repos/asf?p=lucene-solr.git;h=7bf019a ] SOLR-9200 : Add Delegation Token Support to Solr
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          +1 to all the changes, they all look great!

          I'm still wondering if we need the internode calls within the Solr cluster to use tokens. It seems to me that they do not, as of this patch, even if the user calls use delegation tokens.

          Also, it is my belief that end to end requests work fine, even when internode requests are involved. However there are no tests for this, afaict; neither for kerberos plugin with delegation tokens, nor with delegation tokens. The former situation couldn't be tested at the time of writing the kerberos plugin due to lack of ticket cache of minikdc, but maybe that limitation doesn't stop us from writing tests for the latter. So, even if we don't write such tests here in this issue, I think we should write some as a follow up issue, so that we are alerted when such support breaks.

          Overall, I am okay with us committing the current patch. However, I think I'd be more comfortable if internode calls used tokens as well (or even PKI, instead of tokens). I would ideally do that here (unless there are some reasons for not doing it that I'm missing completely), however we can as well do that as a follow up issue, if you think that is a better approach.

          Many thanks for your excellent work on this issue, we needed this for Solr for long!

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - +1 to all the changes, they all look great! I'm still wondering if we need the internode calls within the Solr cluster to use tokens. It seems to me that they do not, as of this patch, even if the user calls use delegation tokens. Also, it is my belief that end to end requests work fine, even when internode requests are involved. However there are no tests for this, afaict; neither for kerberos plugin with delegation tokens, nor with delegation tokens. The former situation couldn't be tested at the time of writing the kerberos plugin due to lack of ticket cache of minikdc, but maybe that limitation doesn't stop us from writing tests for the latter. So, even if we don't write such tests here in this issue, I think we should write some as a follow up issue, so that we are alerted when such support breaks. Overall, I am okay with us committing the current patch. However, I think I'd be more comfortable if internode calls used tokens as well (or even PKI, instead of tokens). I would ideally do that here (unless there are some reasons for not doing it that I'm missing completely), however we can as well do that as a follow up issue, if you think that is a better approach. Many thanks for your excellent work on this issue, we needed this for Solr for long!
          Hide
          gchanan Gregory Chanan added a comment - - edited

          Thanks for taking a look, Ishan Chattopadhyaya

          I'm still wondering if we need the internode calls within the Solr cluster to use tokens. It seems to me that they do not, as of this patch, even if the user calls use delegation tokens.

          I don't think they do, this is just for client authentication

          Also, it is my belief that end to end requests work fine, even when internode requests are involved. However there are no tests for this, afaict; neither for kerberos plugin with delegation tokens, nor with delegation tokens. The former situation couldn't be tested at the time of writing the kerberos plugin due to lack of ticket cache of minikdc, but maybe that limitation doesn't stop us from writing tests for the latter. So, even if we don't write such tests here in this issue, I think we should write some as a follow up issue, so that we are alerted when such support breaks.

          Sure, SOLR-9324 contains such a test. Although it doesn't really solve the minikdc problem, that just uses a different authentication mechanism.

          Overall, I am okay with us committing the current patch. However, I think I'd be more comfortable if internode calls used tokens as well (or even PKI, instead of tokens). I would ideally do that here (unless there are some reasons for not doing it that I'm missing completely), however we can as well do that as a follow up issue, if you think that is a better approach.

          Maybe I'm misunderstanding something, but don't the internode calls already use PKI – that seems to always be used for internode calls with SolrCloud. I don't see what's different with this patch than before it.

          Show
          gchanan Gregory Chanan added a comment - - edited Thanks for taking a look, Ishan Chattopadhyaya I'm still wondering if we need the internode calls within the Solr cluster to use tokens. It seems to me that they do not, as of this patch, even if the user calls use delegation tokens. I don't think they do, this is just for client authentication Also, it is my belief that end to end requests work fine, even when internode requests are involved. However there are no tests for this, afaict; neither for kerberos plugin with delegation tokens, nor with delegation tokens. The former situation couldn't be tested at the time of writing the kerberos plugin due to lack of ticket cache of minikdc, but maybe that limitation doesn't stop us from writing tests for the latter. So, even if we don't write such tests here in this issue, I think we should write some as a follow up issue, so that we are alerted when such support breaks. Sure, SOLR-9324 contains such a test. Although it doesn't really solve the minikdc problem, that just uses a different authentication mechanism. Overall, I am okay with us committing the current patch. However, I think I'd be more comfortable if internode calls used tokens as well (or even PKI, instead of tokens). I would ideally do that here (unless there are some reasons for not doing it that I'm missing completely), however we can as well do that as a follow up issue, if you think that is a better approach. Maybe I'm misunderstanding something, but don't the internode calls already use PKI – that seems to always be used for internode calls with SolrCloud. I don't see what's different with this patch than before it.
          Hide
          steve_rowe Steve Rowe added a comment -

          The commit on this issue is causing the Maven build to fail (from https://builds.apache.org/job/Lucene-Solr-Maven-master/1814/):

           [mvn] [ERROR] COMPILATION ERROR : 
                [mvn] [INFO] -------------------------------------------------------------
                [mvn] [ERROR] /x1/jenkins/jenkins-slave/workspace/Lucene-Solr-Maven-master/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java:[29,77] package com.carrotsearch.ant.tasks.junit4.dependencies.com.google.common.base does not exist
                [mvn] [ERROR] /x1/jenkins/jenkins-slave/workspace/Lucene-Solr-Maven-master/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java:[195,7] cannot find symbol
                [mvn]   symbol:   variable Preconditions
                [mvn]   location: class org.apache.solr.cloud.KerberosTestServices.Builder
                [mvn] [ERROR] /x1/jenkins/jenkins-slave/workspace/Lucene-Solr-Maven-master/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java:[196,7] cannot find symbol
                [mvn]   symbol:   variable Preconditions
                [mvn]   location: class org.apache.solr.cloud.KerberosTestServices.Builder
                [mvn] [ERROR] /x1/jenkins/jenkins-slave/workspace/Lucene-Solr-Maven-master/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java:[206,7] cannot find symbol
                [mvn]   symbol:   variable Preconditions
                [mvn]   location: class org.apache.solr.cloud.KerberosTestServices.Builder
                [mvn] [ERROR] /x1/jenkins/jenkins-slave/workspace/Lucene-Solr-Maven-master/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java:[207,7] cannot find symbol
                [mvn]   symbol:   variable Preconditions
                [mvn]   location: class org.apache.solr.cloud.KerberosTestServices.Builder
                [mvn] [INFO] 5 errors 
          

          This import line in KerberosTestServices is the source of the problems:

          import com.carrotsearch.ant.tasks.junit4.dependencies.com.google.common.base.Preconditions;
          

          I'm guessing the problem is that the junit4 ant tasks aren't on Maven's test classpath.

          But why is this Guava class being loaded via this shaded route, when Guava is a direct Solr dependency? AFAICT all other uses of Preconditions in Solr use this import instead - I'll test now to see if that makes the Maven build happy:

          import com.google.common.base.Preconditions;
          
          Show
          steve_rowe Steve Rowe added a comment - The commit on this issue is causing the Maven build to fail (from https://builds.apache.org/job/Lucene-Solr-Maven-master/1814/ ): [mvn] [ERROR] COMPILATION ERROR : [mvn] [INFO] ------------------------------------------------------------- [mvn] [ERROR] /x1/jenkins/jenkins-slave/workspace/Lucene-Solr-Maven-master/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java:[29,77] package com.carrotsearch.ant.tasks.junit4.dependencies.com.google.common.base does not exist [mvn] [ERROR] /x1/jenkins/jenkins-slave/workspace/Lucene-Solr-Maven-master/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java:[195,7] cannot find symbol [mvn] symbol: variable Preconditions [mvn] location: class org.apache.solr.cloud.KerberosTestServices.Builder [mvn] [ERROR] /x1/jenkins/jenkins-slave/workspace/Lucene-Solr-Maven-master/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java:[196,7] cannot find symbol [mvn] symbol: variable Preconditions [mvn] location: class org.apache.solr.cloud.KerberosTestServices.Builder [mvn] [ERROR] /x1/jenkins/jenkins-slave/workspace/Lucene-Solr-Maven-master/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java:[206,7] cannot find symbol [mvn] symbol: variable Preconditions [mvn] location: class org.apache.solr.cloud.KerberosTestServices.Builder [mvn] [ERROR] /x1/jenkins/jenkins-slave/workspace/Lucene-Solr-Maven-master/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java:[207,7] cannot find symbol [mvn] symbol: variable Preconditions [mvn] location: class org.apache.solr.cloud.KerberosTestServices.Builder [mvn] [INFO] 5 errors This import line in KerberosTestServices is the source of the problems: import com.carrotsearch.ant.tasks.junit4.dependencies.com.google.common.base.Preconditions; I'm guessing the problem is that the junit4 ant tasks aren't on Maven's test classpath. But why is this Guava class being loaded via this shaded route, when Guava is a direct Solr dependency? AFAICT all other uses of Preconditions in Solr use this import instead - I'll test now to see if that makes the Maven build happy: import com.google.common.base.Preconditions;
          Hide
          steve_rowe Steve Rowe added a comment -

          mvn test-compile succeeds when I change the import statement. I'll go commit the change now. Gregory Chanan, feel free to revert if you intentional did it that way for some reason; if so, I can work on addressing the Maven build problems in a different way.

          Show
          steve_rowe Steve Rowe added a comment - mvn test-compile succeeds when I change the import statement. I'll go commit the change now. Gregory Chanan , feel free to revert if you intentional did it that way for some reason; if so, I can work on addressing the Maven build problems in a different way.
          Hide
          jira-bot ASF subversion and git services added a comment -

          Commit cead204fb6fecd576e8e1444e0de879d24d42f1b in lucene-solr's branch refs/heads/master from Steve Rowe
          [ https://git-wip-us.apache.org/repos/asf?p=lucene-solr.git;h=cead204 ]

          SOLR-9200: Use direct package for Guava's Preconditions class

          Show
          jira-bot ASF subversion and git services added a comment - Commit cead204fb6fecd576e8e1444e0de879d24d42f1b in lucene-solr's branch refs/heads/master from Steve Rowe [ https://git-wip-us.apache.org/repos/asf?p=lucene-solr.git;h=cead204 ] SOLR-9200 : Use direct package for Guava's Preconditions class
          Hide
          gchanan Gregory Chanan added a comment -

          I think your change is correct, looks like some auto import issue. Not sure why it didn't cause problems locally.

          Show
          gchanan Gregory Chanan added a comment - I think your change is correct, looks like some auto import issue. Not sure why it didn't cause problems locally.
          Hide
          erickerickson Erick Erickson added a comment -

          What are the plans for merging this into 6x? I don't see a commit yet or are letting it bake for a while on trunk?

          Show
          erickerickson Erick Erickson added a comment - What are the plans for merging this into 6x? I don't see a commit yet or are letting it bake for a while on trunk?
          Hide
          gchanan Gregory Chanan added a comment -

          Just letting it bake for a bit. Will merge to 6x today or early next week.

          Show
          gchanan Gregory Chanan added a comment - Just letting it bake for a bit. Will merge to 6x today or early next week.
          Hide
          gchanan Gregory Chanan added a comment -

          Here's a branch 6 patch, will commit soon assuming tests pass.

          Show
          gchanan Gregory Chanan added a comment - Here's a branch 6 patch, will commit soon assuming tests pass.
          Hide
          jira-bot ASF subversion and git services added a comment -

          Commit cead204fb6fecd576e8e1444e0de879d24d42f1b in lucene-solr's branch refs/heads/apiv2 from Steve Rowe
          [ https://git-wip-us.apache.org/repos/asf?p=lucene-solr.git;h=cead204 ]

          SOLR-9200: Use direct package for Guava's Preconditions class

          Show
          jira-bot ASF subversion and git services added a comment - Commit cead204fb6fecd576e8e1444e0de879d24d42f1b in lucene-solr's branch refs/heads/apiv2 from Steve Rowe [ https://git-wip-us.apache.org/repos/asf?p=lucene-solr.git;h=cead204 ] SOLR-9200 : Use direct package for Guava's Preconditions class
          Hide
          gchanan Gregory Chanan added a comment -

          Interestingly, had to add SuppressSSL to DelegationToken test for 6x, because of some PKIAuthentication errors. The PKI tests have that annotation as well, so the interesting part is why that isn't required on master.

          Show
          gchanan Gregory Chanan added a comment - Interestingly, had to add SuppressSSL to DelegationToken test for 6x, because of some PKIAuthentication errors. The PKI tests have that annotation as well, so the interesting part is why that isn't required on master.
          Hide
          jira-bot ASF subversion and git services added a comment -

          Commit c60cd2529b9c9d3e57e23e67e7c55a75269a23f9 in lucene-solr's branch refs/heads/master from Gregory Chanan
          [ https://git-wip-us.apache.org/repos/asf?p=lucene-solr.git;h=c60cd25 ]

          SOLR-9200: Prepare Locale before starting MiniKdc

          Show
          jira-bot ASF subversion and git services added a comment - Commit c60cd2529b9c9d3e57e23e67e7c55a75269a23f9 in lucene-solr's branch refs/heads/master from Gregory Chanan [ https://git-wip-us.apache.org/repos/asf?p=lucene-solr.git;h=c60cd25 ] SOLR-9200 : Prepare Locale before starting MiniKdc
          Hide
          jira-bot ASF subversion and git services added a comment -

          Commit 6cfec0bd7a56f7e70046093c603e6f5982c83c69 in lucene-solr's branch refs/heads/branch_6x from Gregory Chanan
          [ https://git-wip-us.apache.org/repos/asf?p=lucene-solr.git;h=6cfec0b ]

          SOLR-9200: Add Delegation Token Support to Solr

          Show
          jira-bot ASF subversion and git services added a comment - Commit 6cfec0bd7a56f7e70046093c603e6f5982c83c69 in lucene-solr's branch refs/heads/branch_6x from Gregory Chanan [ https://git-wip-us.apache.org/repos/asf?p=lucene-solr.git;h=6cfec0b ] SOLR-9200 : Add Delegation Token Support to Solr
          Hide
          mikemccand Michael McCandless added a comment -

          Bulk close resolved issues after 6.2.0 release.

          Show
          mikemccand Michael McCandless added a comment - Bulk close resolved issues after 6.2.0 release.
          Hide
          ctargett Cassandra Targett added a comment -

          Gregory Chanan or Ishan Chattopadhyaya - is the functionality described in this earlier comment https://issues.apache.org/jira/browse/SOLR-9200?focusedCommentId=15366913&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15366913 still accurate? This has not yet been added to the Solr Ref Guide, and since I think there is some interest for it, we should try to get it in while we're waiting for the issue with publishing 6.2 to be resolved.

          It belongs with the Kerberos documentation at https://cwiki.apache.org/confluence/display/solr/Kerberos+Authentication+Plugin, correct?

          Show
          ctargett Cassandra Targett added a comment - Gregory Chanan or Ishan Chattopadhyaya - is the functionality described in this earlier comment https://issues.apache.org/jira/browse/SOLR-9200?focusedCommentId=15366913&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15366913 still accurate? This has not yet been added to the Solr Ref Guide, and since I think there is some interest for it, we should try to get it in while we're waiting for the issue with publishing 6.2 to be resolved. It belongs with the Kerberos documentation at https://cwiki.apache.org/confluence/display/solr/Kerberos+Authentication+Plugin , correct?
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Cassandra Targett, I just had a look and it seems the parameters described in the comment are reflective of the changes committed, and can be imported as is into the Ref Guide. Gregory Chanan, please pitch in if that is not accurate.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Cassandra Targett , I just had a look and it seems the parameters described in the comment are reflective of the changes committed, and can be imported as is into the Ref Guide. Gregory Chanan , please pitch in if that is not accurate.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          I think a section on delegation tokens can be added to the ref guide page for Kerberos Plugin. Here's a proposed wording (based on the original description in this issue):

          Delegation tokens
          -----------------

          The Kerberos plugin can, optionally, be configured to use delegation tokens. The plugin allows authenticated users the ability to grab, renew or delete a token that can be used to bypass the normal authentication path for a time. This is useful in a variety of use cases:
          1) distributed clients (e.g. MapReduce) where each client may not have access to the user's Kerberos credentials. Instead, the job runner can grab a delegation token and use that during task execution.
          2) If the load on the Kerberos server is too high, delegation tokens can avoid hitting the Kerberos server after the first request.
          3) If requests/permissions need to be delegated to another user: the more privileged user can request a delegation token that can be passed to the less privileged user.

          Enabling delegation tokens can be done by setting the solr.kerberos.delegation.token.enabled parameter to true.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - I think a section on delegation tokens can be added to the ref guide page for Kerberos Plugin. Here's a proposed wording (based on the original description in this issue): Delegation tokens ----------------- The Kerberos plugin can, optionally, be configured to use delegation tokens. The plugin allows authenticated users the ability to grab, renew or delete a token that can be used to bypass the normal authentication path for a time. This is useful in a variety of use cases: 1) distributed clients (e.g. MapReduce) where each client may not have access to the user's Kerberos credentials. Instead, the job runner can grab a delegation token and use that during task execution. 2) If the load on the Kerberos server is too high, delegation tokens can avoid hitting the Kerberos server after the first request. 3) If requests/permissions need to be delegated to another user: the more privileged user can request a delegation token that can be passed to the less privileged user. Enabling delegation tokens can be done by setting the solr.kerberos.delegation.token.enabled parameter to true.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          In my previous comment about the parameters, I missed out the fact that the last two parameters are now (chroot)/security/token or (chroot)/security/zkdtsm.

          solr.kerberos.delegation.token.signer.secret.provider.zookeper.path String (chrooted path) + /security/token Zookeeper location where secret provider information is stored
          solr.kerberos.delegation.token.secret.manager.znode.working.path String (chrooted path) + /security/zkdtsm Zookeeper location where token information is stored

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - In my previous comment about the parameters, I missed out the fact that the last two parameters are now (chroot)/security/token or (chroot)/security/zkdtsm. solr.kerberos.delegation.token.signer.secret.provider.zookeper.path String (chrooted path) + /security/token Zookeeper location where secret provider information is stored solr.kerberos.delegation.token.secret.manager.znode.working.path String (chrooted path) + /security/zkdtsm Zookeeper location where token information is stored
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Maybe I'm misunderstanding something, but don't the internode calls already use PKI – that seems to always be used for internode calls with SolrCloud. I don't see what's different with this patch than before it.

          Internode calls use PKI authentication for any plugin that does not implement HttpClientInterceptorPlugin. Kerberos plugin uses that "interceptor" and hence deals with its own internode communication (each node has a client principal associated with it, specified in a jaas config file, for making internode calls). I think the committed patch here for delegation tokens does not have the internode communications using the delegation tokens. If that is the case, we can open another issue to add a test and fix.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Maybe I'm misunderstanding something, but don't the internode calls already use PKI – that seems to always be used for internode calls with SolrCloud. I don't see what's different with this patch than before it. Internode calls use PKI authentication for any plugin that does not implement HttpClientInterceptorPlugin . Kerberos plugin uses that "interceptor" and hence deals with its own internode communication (each node has a client principal associated with it, specified in a jaas config file, for making internode calls). I think the committed patch here for delegation tokens does not have the internode communications using the delegation tokens. If that is the case, we can open another issue to add a test and fix.
          Hide
          timothy.frey Timothy Frey added a comment -

          Hi there,

          Not sure if this is the appropriate place to report this issue. We're building a Solr 6.2 cluster and we want to use BasicAuth. We notice that if we just set a request header of "SolrAuth" with any value lets us completely bypass the BasicAuth layer. I noticed a change to this function:

          https://github.com/apache/lucene-solr/blob/6cfec0bd7a56f7e70046093c603e6f5982c83c69/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java#L92

          Looks like every failure start of that method returns "true" which indicates that authentication was successful. Am I reading this correctly?

          Show
          timothy.frey Timothy Frey added a comment - Hi there, Not sure if this is the appropriate place to report this issue. We're building a Solr 6.2 cluster and we want to use BasicAuth. We notice that if we just set a request header of "SolrAuth" with any value lets us completely bypass the BasicAuth layer. I noticed a change to this function: https://github.com/apache/lucene-solr/blob/6cfec0bd7a56f7e70046093c603e6f5982c83c69/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java#L92 Looks like every failure start of that method returns "true" which indicates that authentication was successful. Am I reading this correctly?
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Thanks for reporting it, Timothy. Can you please create another issue with the appropriate title and add these details and any other information necessary to reproduce this bug? It sounds like a serious bug to me, but I haven't looked much into it.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Thanks for reporting it, Timothy. Can you please create another issue with the appropriate title and add these details and any other information necessary to reproduce this bug? It sounds like a serious bug to me, but I haven't looked much into it.
          Hide
          mkhludnev Mikhail Khludnev added a comment -

          https://jenkins.thetaphi.de/job/Lucene-Solr-master-Windows/6175/

             [junit4]   2> 1072871 ERROR (jetty-launcher-1462-thread-2) [n:127.0.0.1:64463_solr    ] o.a.h.u.Shell Failed to locate the winutils binary in the hadoop binary path
             [junit4]   2> java.io.IOException: Could not locate executable null\bin\winutils.exe in the Hadoop binaries.
             [junit4]   2>        at org.apache.hadoop.util.Shell.getQualifiedBinPath(Shell.java:356)
             [junit4]   2>        at org.apache.hadoop.util.Shell.getWinUtilsPath(Shell.java:371)
             [junit4]   2>        at org.apache.hadoop.util.Shell.<clinit>(Shell.java:364)
             [junit4]   2>        at org.apache.hadoop.util.StringUtils.<clinit>(StringUtils.java:80)
             [junit4]   2>        at org.apache.hadoop.conf.Configuration.getBoolean(Configuration.java:1437)
             [junit4]   2>        at org.apache.hadoop.security.token.delegation.web.DelegationTokenManager.<init>(DelegationTokenManager.java:115)
          

          Uwe Schindler, is it possible to provide -Dhadoop.home.dir=C:\hadoop where bin\winutils.exe is located. Or just ignore it from windows run?

          Show
          mkhludnev Mikhail Khludnev added a comment - https://jenkins.thetaphi.de/job/Lucene-Solr-master-Windows/6175/ [junit4] 2> 1072871 ERROR (jetty-launcher-1462-thread-2) [n:127.0.0.1:64463_solr ] o.a.h.u.Shell Failed to locate the winutils binary in the hadoop binary path [junit4] 2> java.io.IOException: Could not locate executable null \bin\winutils.exe in the Hadoop binaries. [junit4] 2> at org.apache.hadoop.util.Shell.getQualifiedBinPath(Shell.java:356) [junit4] 2> at org.apache.hadoop.util.Shell.getWinUtilsPath(Shell.java:371) [junit4] 2> at org.apache.hadoop.util.Shell.<clinit>(Shell.java:364) [junit4] 2> at org.apache.hadoop.util.StringUtils.<clinit>(StringUtils.java:80) [junit4] 2> at org.apache.hadoop.conf.Configuration.getBoolean(Configuration.java:1437) [junit4] 2> at org.apache.hadoop.security.token.delegation.web.DelegationTokenManager.<init>(DelegationTokenManager.java:115) Uwe Schindler , is it possible to provide -Dhadoop.home.dir=C:\hadoop where bin\winutils.exe is located. Or just ignore it from windows run?
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          How about we re-close this, since this has already been released, and use another issue to track this failure? I've created SOLR-9630 for this.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - How about we re-close this, since this has already been released, and use another issue to track this failure? I've created SOLR-9630 for this.
          Hide
          thetaphi Uwe Schindler added a comment -

          Please add a assumeFalse(Constants.WINDOWS) for all hadoop tests. We do this on all other tests already.

          Show
          thetaphi Uwe Schindler added a comment - Please add a assumeFalse(Constants.WINDOWS) for all hadoop tests. We do this on all other tests already.

            People

            • Assignee:
              gchanan Gregory Chanan
              Reporter:
              gchanan Gregory Chanan
            • Votes:
              0 Vote for this issue
              Watchers:
              12 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development