hdfs_ext_test.cc 12 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342
  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_mini_dfs.h"
  19. #include "hdfspp/hdfs_ext.h"
  20. #include <chrono>
  21. namespace hdfs {
  22. class HdfsExtTest: public ::testing::Test {
  23. public:
  24. MiniCluster cluster;
  25. };
  26. // Make sure we can set up a mini-cluster and connect to it
  27. TEST_F(HdfsExtTest, TestGetBlockLocations) {
  28. HdfsHandle connection = cluster.connect_c();
  29. EXPECT_NE(nullptr, connection.handle());
  30. hdfsBlockLocations * blocks = nullptr;
  31. // Free a null pointer
  32. int result = hdfsFreeBlockLocations(blocks);
  33. EXPECT_EQ(0, result);
  34. // Test non-extant files
  35. result = hdfsGetBlockLocations(connection, "non_extant_file", &blocks);
  36. EXPECT_NE(0, result); // Should be an error
  37. // Test an extant file
  38. std::string filename = connection.newFile(1024);
  39. result = hdfsGetBlockLocations(connection, filename.c_str(), &blocks);
  40. EXPECT_EQ(0, result);
  41. EXPECT_EQ(1024, blocks->fileLength);
  42. EXPECT_EQ(1, blocks->num_blocks);
  43. EXPECT_EQ(0, blocks->isUnderConstruction);
  44. EXPECT_NE(0, blocks->isLastBlockComplete);
  45. EXPECT_EQ(1024, blocks->blocks->num_bytes);
  46. EXPECT_EQ(0, blocks->blocks->start_offset);
  47. EXPECT_EQ(1, blocks->blocks->num_locations);
  48. EXPECT_NE(nullptr, blocks->blocks->locations->hostname);
  49. EXPECT_NE(nullptr, blocks->blocks->locations->ip_address);
  50. EXPECT_NE(0, blocks->blocks->locations->xfer_port);
  51. result = hdfsFreeBlockLocations(blocks);
  52. EXPECT_EQ(0, result);
  53. }
  54. // Writing a file to the filesystem and checking the used space
  55. TEST_F(HdfsExtTest, TestGetUsed) {
  56. using namespace std::chrono;
  57. HdfsHandle connection = cluster.connect_c();
  58. hdfsFS fs = connection.handle();
  59. EXPECT_NE(nullptr, fs);
  60. // File system's used space before writing
  61. tOffset used_before_write;
  62. EXPECT_GE(used_before_write = hdfsGetUsed(fs), 0);
  63. // Write to a file
  64. tOffset fileSize = 1024;
  65. std::string filename = connection.newFile(fileSize);
  66. //Need to run hdfsGetUsed() in a loop until the refreshInterval
  67. //is passed on the filesystem and the used space is updated
  68. //Time-out is 3 minutes
  69. tOffset used_after_write;
  70. tOffset difference;
  71. minutes beginTime = duration_cast<minutes>(
  72. system_clock::now().time_since_epoch());
  73. minutes currentTime;
  74. do{
  75. EXPECT_GE(used_after_write = hdfsGetUsed(fs), 0);
  76. difference = used_after_write - used_before_write;
  77. currentTime = duration_cast<minutes>(
  78. system_clock::now().time_since_epoch());
  79. } while (difference == 0 && currentTime.count() - beginTime.count() < 3);
  80. //There should be at least fileSize bytes added to the used space
  81. EXPECT_GT(difference, fileSize);
  82. //There could be additional metadata added to the used space,
  83. //but no more than double the fileSize
  84. EXPECT_LT(difference, fileSize * 2);
  85. }
  86. //Testing allow, disallow, create, and delete snapshot
  87. TEST_F(HdfsExtTest, TestSnapshotOperations) {
  88. HdfsHandle connection = cluster.connect_c();
  89. hdfsFS fs = connection.handle();
  90. EXPECT_NE(nullptr, fs);
  91. //argument 'path' is NULL
  92. EXPECT_EQ(-1, hdfsAllowSnapshot(fs, nullptr));
  93. EXPECT_EQ((int) std::errc::invalid_argument, errno);
  94. EXPECT_EQ(-1, hdfsCreateSnapshot(fs, nullptr, "Bad"));
  95. EXPECT_EQ((int) std::errc::invalid_argument, errno);
  96. EXPECT_EQ(-1, hdfsDeleteSnapshot(fs, nullptr, "Bad"));
  97. EXPECT_EQ((int) std::errc::invalid_argument, errno);
  98. EXPECT_EQ(-1, hdfsDisallowSnapshot(fs, nullptr));
  99. EXPECT_EQ((int) std::errc::invalid_argument, errno);
  100. //argument 'name' is NULL for deletion
  101. EXPECT_EQ(-1, hdfsDeleteSnapshot(fs, "/dir/", nullptr));
  102. EXPECT_EQ((int) std::errc::invalid_argument, errno);
  103. //Path not found
  104. std::string path = "/wrong/dir/";
  105. EXPECT_EQ(-1, hdfsAllowSnapshot(fs, path.c_str()));
  106. EXPECT_EQ((int) std::errc::no_such_file_or_directory, errno);
  107. EXPECT_EQ(-1, hdfsCreateSnapshot(fs, path.c_str(), "Bad"));
  108. EXPECT_EQ((int) std::errc::no_such_file_or_directory, errno);
  109. EXPECT_EQ(-1, hdfsDeleteSnapshot(fs, path.c_str(), "Bad"));
  110. EXPECT_EQ((int) std::errc::no_such_file_or_directory, errno);
  111. EXPECT_EQ(-1, hdfsDisallowSnapshot(fs, path.c_str()));
  112. EXPECT_EQ((int) std::errc::no_such_file_or_directory, errno);
  113. //Not a directory
  114. path = connection.newFile(1024); //1024 byte file
  115. EXPECT_EQ(-1, hdfsAllowSnapshot(fs, path.c_str()));
  116. EXPECT_EQ((int) std::errc::not_a_directory, errno);
  117. EXPECT_EQ(-1, hdfsCreateSnapshot(fs, path.c_str(), "Bad"));
  118. EXPECT_EQ((int) std::errc::not_a_directory, errno);
  119. EXPECT_EQ(-1, hdfsDeleteSnapshot(fs, path.c_str(), "Bad"));
  120. EXPECT_EQ((int) std::errc::not_a_directory, errno);
  121. EXPECT_EQ(-1, hdfsDisallowSnapshot(fs, path.c_str()));
  122. EXPECT_EQ((int) std::errc::not_a_directory, errno);
  123. //Not snapshottable directory
  124. std::string dirName = connection.newDir();
  125. EXPECT_EQ(0, hdfsDisallowSnapshot(fs, dirName.c_str()));
  126. EXPECT_EQ(-1, hdfsCreateSnapshot(fs, dirName.c_str(), "Bad"));
  127. EXPECT_EQ((int) std::errc::invalid_argument, errno);
  128. //Verify snapshot created
  129. EXPECT_EQ(0, hdfsAllowSnapshot(fs, dirName.c_str()));
  130. EXPECT_EQ(0, hdfsCreateSnapshot(fs, dirName.c_str(), "Good"));
  131. std::string snapDir = dirName + ".snapshot/";
  132. int size;
  133. hdfsFileInfo *file_infos;
  134. EXPECT_NE(nullptr, file_infos = hdfsListDirectory(fs, snapDir.c_str(), &size));
  135. EXPECT_EQ(1, size);
  136. EXPECT_STREQ("Good", file_infos[0].mName);
  137. hdfsFreeFileInfo(file_infos, 1);
  138. //Verify snapshot deleted
  139. EXPECT_EQ(0, hdfsDeleteSnapshot(fs, dirName.c_str(), "Good"));
  140. EXPECT_EQ(nullptr, file_infos = hdfsListDirectory(fs, snapDir.c_str(), &size));
  141. EXPECT_EQ(0, size);
  142. hdfsFreeFileInfo(file_infos, 0);
  143. }
  144. //Testing creating directories
  145. TEST_F(HdfsExtTest, TestMkdirs) {
  146. HdfsHandle connection = cluster.connect_c();
  147. hdfsFS fs = connection.handle();
  148. EXPECT_NE(nullptr, fs);
  149. //Correct operation
  150. EXPECT_EQ(0, hdfsCreateDirectory(fs, "/myDir123"));
  151. //TODO Should return error if directory already exists?
  152. //EXPECT_EQ(-1, hdfsCreateDirectory(fs, "/myDir123"));
  153. //EXPECT_EQ((int) std::errc::file_exists, errno);
  154. //Creating directory on a path of the existing file
  155. std::string path = connection.newFile(1024); //1024 byte file
  156. EXPECT_EQ(-1, hdfsCreateDirectory(fs, path.c_str()));
  157. EXPECT_EQ((int) std::errc::file_exists, errno);
  158. }
  159. //Testing deleting files and directories
  160. TEST_F(HdfsExtTest, TestDelete) {
  161. HdfsHandle connection = cluster.connect_c();
  162. hdfsFS fs = connection.handle();
  163. EXPECT_NE(nullptr, fs);
  164. //Path not found
  165. EXPECT_EQ(-1, hdfsDelete(fs, "/wrong_path", 1));
  166. EXPECT_EQ((int) std::errc::no_such_file_or_directory, errno);
  167. EXPECT_EQ(0, hdfsCreateDirectory(fs, "/myDir"));
  168. std::string path = connection.newFile("/myDir", 1024); //1024 byte file
  169. //Non-recursive delete should fail on a non-empty directory
  170. //error ENOTEMPTY(39) for libhdfspp or 255 for libhdfs
  171. EXPECT_EQ(-1, hdfsDelete(fs, "/myDir", 0));
  172. EXPECT_EQ((int) std::errc::directory_not_empty, errno);
  173. //Correct operation
  174. EXPECT_EQ(0, hdfsDelete(fs, "/myDir", 1));
  175. }
  176. //Testing renaming files and directories
  177. TEST_F(HdfsExtTest, TestRename) {
  178. HdfsHandle connection = cluster.connect_c();
  179. hdfsFS fs = connection.handle();
  180. EXPECT_NE(nullptr, fs);
  181. //Creating directory with two files
  182. EXPECT_EQ(0, hdfsCreateDirectory(fs, "/myDir"));
  183. std::string file1 = connection.newFile("/myDir", 1024); //1024 byte file
  184. std::string file2 = connection.newFile("/myDir", 1024); //1024 byte file
  185. std::string file3 = connection.newFile(1024); //1024 byte file
  186. //Path not found
  187. EXPECT_EQ(-1, hdfsRename(fs, "/wrong_path", "/new_name"));
  188. EXPECT_EQ((int) std::errc::invalid_argument, errno);
  189. //No parent directory in new path
  190. EXPECT_EQ(-1, hdfsRename(fs, file1.c_str(), "/wrong_parent/new_name"));
  191. EXPECT_EQ((int ) std::errc::invalid_argument, errno);
  192. //New name already exists in the folder
  193. EXPECT_EQ(-1, hdfsRename(fs, file1.c_str(), file2.c_str()));
  194. EXPECT_EQ((int ) std::errc::invalid_argument, errno);
  195. //Correct operation
  196. EXPECT_EQ(0, hdfsRename(fs, file1.c_str(), "/myDir/new_awesome_name"));
  197. EXPECT_EQ(0, hdfsRename(fs, file3.c_str(), "/myDir/another_file"));
  198. EXPECT_EQ(0, hdfsRename(fs, "/myDir", "/new_awesome_dir"));
  199. //Verification
  200. int numEntries;
  201. hdfsFileInfo * dirList = hdfsListDirectory(fs, "/new_awesome_dir", &numEntries);
  202. EXPECT_NE(nullptr, dirList);
  203. EXPECT_EQ(3, numEntries);
  204. hdfsFreeFileInfo(dirList, 3);
  205. }
  206. //Testing Chmod and Chown
  207. TEST_F(HdfsExtTest, TestChmodChown) {
  208. HdfsHandle connection = cluster.connect_c();
  209. hdfsFS fs = connection.handle();
  210. EXPECT_NE(nullptr, fs);
  211. //Path not found
  212. std::string path = "/wrong/dir/";
  213. EXPECT_EQ(-1, hdfsChmod(fs, path.c_str(), 0777));
  214. EXPECT_EQ((int ) std::errc::no_such_file_or_directory, errno);
  215. EXPECT_EQ(-1, hdfsChown(fs, path.c_str(), "foo", "bar"));
  216. EXPECT_EQ((int ) std::errc::no_such_file_or_directory, errno);
  217. //Wrong arguments
  218. path = connection.newFile(1024); //1024 byte file
  219. EXPECT_EQ(-1, hdfsChmod(fs, nullptr, 0777));
  220. EXPECT_EQ((int ) std::errc::invalid_argument, errno);
  221. EXPECT_EQ(-1, hdfsChmod(fs, path.c_str(), 07777));
  222. EXPECT_EQ((int ) std::errc::invalid_argument, errno);
  223. EXPECT_EQ(-1, hdfsChmod(fs, path.c_str(), -1));
  224. EXPECT_EQ((int ) std::errc::invalid_argument, errno);
  225. EXPECT_EQ(-1, hdfsChown(fs, nullptr, "foo", "bar"));
  226. EXPECT_EQ((int ) std::errc::invalid_argument, errno);
  227. //Permission denied
  228. HdfsHandle connection2 = cluster.connect_c("OtherGuy");
  229. hdfsFS fs2 = connection2.handle();
  230. EXPECT_EQ(-1, hdfsChmod(fs2, path.c_str(), 0123));
  231. EXPECT_EQ((int ) std::errc::permission_denied, errno);
  232. EXPECT_EQ(-1, hdfsChown(fs2, path.c_str(), "cool", "nice"));
  233. EXPECT_EQ((int ) std::errc::permission_denied, errno);
  234. //Verify Chmod and Chown worked
  235. EXPECT_EQ(0, hdfsChmod(fs, path.c_str(), 0123));
  236. EXPECT_EQ(0, hdfsChown(fs, path.c_str(), "cool", "nice"));
  237. hdfsFileInfo *file_info;
  238. EXPECT_NE(nullptr, file_info = hdfsGetPathInfo(fs, path.c_str()));
  239. EXPECT_EQ(0123, file_info->mPermissions);
  240. EXPECT_STREQ("cool", file_info->mOwner);
  241. EXPECT_STREQ("nice", file_info->mGroup);
  242. hdfsFreeFileInfo(file_info, 1);
  243. }
  244. //Testing EOF
  245. TEST_F(HdfsExtTest, TestEOF) {
  246. HdfsHandle connection = cluster.connect_c();
  247. hdfsFS fs = connection.handle();
  248. EXPECT_NE(nullptr, fs);
  249. //Write to a file
  250. errno = 0;
  251. int size = 256;
  252. std::string path = "/eofTest";
  253. hdfsFile file = hdfsOpenFile(fs, path.c_str(), O_WRONLY, 0, 0, 0);
  254. EXPECT_NE(nullptr, file);
  255. void * buf = malloc(size);
  256. memset(buf, ' ', size);
  257. EXPECT_EQ(size, hdfsWrite(fs, file, buf, size));
  258. free(buf);
  259. EXPECT_EQ(0, hdfsCloseFile(fs, file));
  260. EXPECT_EQ(0, errno);
  261. //Test normal reading (no EOF)
  262. char buffer[300];
  263. EXPECT_EQ(0, errno);
  264. file = hdfsOpenFile(fs, path.c_str(), O_RDONLY, 0, 0, 0);
  265. EXPECT_EQ(size, hdfsPread(fs, file, 0, buffer, sizeof(buffer)));
  266. //Read executes correctly, but causes a warning (captured in HDFS-10595)
  267. //and sets errno to EINPROGRESS 115 : Operation now in progress
  268. errno = 0;
  269. //Test reading at offset past the EOF
  270. EXPECT_EQ(-1, hdfsPread(fs, file, sizeof(buffer), buffer, sizeof(buffer)));
  271. EXPECT_EQ(Status::kInvalidOffset, errno);
  272. EXPECT_EQ(0, hdfsCloseFile(fs, file));
  273. EXPECT_EQ(0, errno);
  274. }
  275. }
  276. int main(int argc, char *argv[]) {
  277. // The following line must be executed to initialize Google Mock
  278. // (and Google Test) before running the tests.
  279. ::testing::InitGoogleMock(&argc, argv);
  280. int exit_code = RUN_ALL_TESTS();
  281. google::protobuf::ShutdownProtobufLibrary();
  282. return exit_code;
  283. }