status.cc 5.8 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146
  1. /**
  2. * Licensed to the Apache Software Foundation (ASF) under one
  3. * or more contributor license agreements. See the NOTICE file
  4. * distributed with this work for additional information
  5. * regarding copyright ownership. The ASF licenses this file
  6. * to you under the Apache License, Version 2.0 (the
  7. * "License"); you may not use this file except in compliance
  8. * with the License. You may obtain a copy of the License at
  9. *
  10. * http://www.apache.org/licenses/LICENSE-2.0
  11. *
  12. * Unless required by applicable law or agreed to in writing, software
  13. * distributed under the License is distributed on an "AS IS" BASIS,
  14. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  15. * See the License for the specific language governing permissions and
  16. * limitations under the License.
  17. */
  18. #include "hdfspp/status.h"
  19. #include <cassert>
  20. #include <sstream>
  21. #include <cstring>
  22. #include <map>
  23. namespace hdfs {
  24. // Server side exceptions that we capture from the RpcResponseHeaderProto
  25. const char * kStatusAccessControlException = "org.apache.hadoop.security.AccessControlException";
  26. const char * kPathIsNotDirectoryException = "org.apache.hadoop.fs.PathIsNotDirectoryException";
  27. const char * kSnapshotException = "org.apache.hadoop.hdfs.protocol.SnapshotException";
  28. const char * kStatusStandbyException = "org.apache.hadoop.ipc.StandbyException";
  29. const char * kStatusSaslException = "javax.security.sasl.SaslException";
  30. const char * kPathNotFoundException = "org.apache.hadoop.fs.InvalidPathException";
  31. const char * kPathNotFoundException2 = "java.io.FileNotFoundException";
  32. const char * kFileAlreadyExistsException = "org.apache.hadoop.fs.FileAlreadyExistsException";
  33. const char * kPathIsNotEmptyDirectoryException = "org.apache.hadoop.fs.PathIsNotEmptyDirectoryException";
  34. const static std::map<std::string, int> kKnownServerExceptionClasses = {
  35. {kStatusAccessControlException, Status::kAccessControlException},
  36. {kPathIsNotDirectoryException, Status::kNotADirectory},
  37. {kSnapshotException, Status::kSnapshotProtocolException},
  38. {kStatusStandbyException, Status::kStandbyException},
  39. {kStatusSaslException, Status::kAuthenticationFailed},
  40. {kPathNotFoundException, Status::kPathNotFound},
  41. {kPathNotFoundException2, Status::kPathNotFound},
  42. {kFileAlreadyExistsException, Status::kFileAlreadyExists},
  43. {kPathIsNotEmptyDirectoryException, Status::kPathIsNotEmptyDirectory}
  44. };
  45. Status::Status(int code, const char *msg1)
  46. : code_(code) {
  47. if(msg1) {
  48. msg_ = msg1;
  49. }
  50. }
  51. Status::Status(int code, const char *exception_class_name, const char *exception_details)
  52. : code_(code) {
  53. // If we can assure this never gets nullptr args this can be
  54. // in the initializer list.
  55. if(exception_class_name)
  56. exception_class_ = exception_class_name;
  57. if(exception_details)
  58. msg_ = exception_details;
  59. std::map<std::string, int>::const_iterator it = kKnownServerExceptionClasses.find(exception_class_);
  60. if(it != kKnownServerExceptionClasses.end()) {
  61. code_ = it->second;
  62. }
  63. }
  64. Status Status::OK() {
  65. return Status();
  66. }
  67. Status Status::InvalidArgument(const char *msg) {
  68. return Status(kInvalidArgument, msg);
  69. }
  70. Status Status::PathNotFound(const char *msg){
  71. return Status(kPathNotFound, msg);
  72. }
  73. Status Status::ResourceUnavailable(const char *msg) {
  74. return Status(kResourceUnavailable, msg);
  75. }
  76. Status Status::Unimplemented() {
  77. return Status(kUnimplemented, "");
  78. }
  79. Status Status::Exception(const char *exception_class_name, const char *error_message) {
  80. // Server side exception but can be represented by std::errc codes
  81. if (exception_class_name && (strcmp(exception_class_name, kStatusAccessControlException) == 0) )
  82. return Status(kPermissionDenied, error_message);
  83. else if (exception_class_name && (strcmp(exception_class_name, kStatusSaslException) == 0))
  84. return AuthenticationFailed();
  85. else if (exception_class_name && (strcmp(exception_class_name, kPathNotFoundException) == 0))
  86. return Status(kPathNotFound, error_message);
  87. else if (exception_class_name && (strcmp(exception_class_name, kPathNotFoundException2) == 0))
  88. return Status(kPathNotFound, error_message);
  89. else if (exception_class_name && (strcmp(exception_class_name, kPathIsNotDirectoryException) == 0))
  90. return Status(kNotADirectory, error_message);
  91. else if (exception_class_name && (strcmp(exception_class_name, kSnapshotException) == 0))
  92. return Status(kInvalidArgument, error_message);
  93. else if (exception_class_name && (strcmp(exception_class_name, kFileAlreadyExistsException) == 0))
  94. return Status(kFileAlreadyExists, error_message);
  95. else if (exception_class_name && (strcmp(exception_class_name, kPathIsNotEmptyDirectoryException) == 0))
  96. return Status(kPathIsNotEmptyDirectory, error_message);
  97. else
  98. return Status(kException, exception_class_name, error_message);
  99. }
  100. Status Status::Error(const char *error_message) {
  101. return Status(kAuthenticationFailed, error_message);
  102. }
  103. Status Status::AuthenticationFailed() {
  104. return Status(kAuthenticationFailed, "Authentication failed");
  105. }
  106. Status Status::Canceled() {
  107. return Status(kOperationCanceled, "Operation canceled");
  108. }
  109. Status Status::InvalidOffset(const char *msg){
  110. return Status(kInvalidOffset, msg);
  111. }
  112. std::string Status::ToString() const {
  113. if (code_ == kOk) {
  114. return "OK";
  115. }
  116. std::stringstream ss;
  117. if(!exception_class_.empty()) {
  118. ss << exception_class_ << ":";
  119. }
  120. ss << msg_;
  121. return ss.str();
  122. }
  123. }