Skip to content

Commit

Permalink
[#319] Support Hadoop 3.3.0
Browse files Browse the repository at this point in the history
Support hadoop330 and compatible with other versions.
  • Loading branch information
singer-bin authored Jul 30, 2024
1 parent 88f031e commit d9bc43d
Show file tree
Hide file tree
Showing 3 changed files with 316 additions and 0 deletions.
1 change: 1 addition & 0 deletions build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -151,6 +151,7 @@ configurations {
}

dependencies {
compile 'commons-lang:commons-lang:2.6'
outboundDep files('lib/airconcurrentmap.jar')
outboundDep group: 'org.pac4j', name: 'spark-pac4j', version: '1.2.3'
outboundDep group: 'org.pac4j', name: 'pac4j-ldap', version: '1.9.4'
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,214 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.apache.hadoop.hdfs.server.namenode;

import java.io.IOException;
import java.io.PrintWriter;
import java.util.Arrays;
import java.util.Date;
import java.util.Map;
import java.util.TreeMap;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import javax.servlet.http.HttpServletResponse;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
import org.apache.hadoop.hdfs.server.namenode.queries.Histograms;
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
import org.apache.hadoop.hdfs.util.Canceler;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.security.AccessControlException;

public class VersionContext implements VersionInterface {

private FSNamesystem namesystem;

@Override // VersionInterface
public void setNamesystem(FSNamesystem namesystem) {
this.namesystem = namesystem;
}

@Override // VersionInterface
public void dumpINodeInDetail(String path, HttpServletResponse resp) throws IOException {
PrintWriter writer = resp.getWriter();
try {
if (namesystem == null) {
writer.write("Namesystem is not fully initialized.\n");
writer.flush();
return;
}
Map<String, Object> nodeDetails = new TreeMap<>();
INode node = namesystem.getFSDirectory().getINode(path);
nodeDetails.put("path", node.getFullPathName());
nodeDetails.put("permisssions", node.getPermissionStatus().toString());
nodeDetails.put("accessTime", new Date(node.getAccessTime()));
nodeDetails.put("modTime", new Date(node.getModificationTime()));
nodeDetails.put("nodeId", node.getId());
nodeDetails.put(
"storagePolicy",
BlockStoragePolicySuite.createDefaultSuite().getPolicy(node.getStoragePolicyID()));
nodeDetails.put("nsQuota", node.getQuotaCounts().getNameSpace());
nodeDetails.put("dsQuota", node.getQuotaCounts().getStorageSpace());
XAttrFeature xattrs = node.getXAttrFeature();
nodeDetails.put("xAttrs", ((xattrs == null) ? "NONE" : xattrs.getXAttrs()));
AclFeature aclFeature = node.getAclFeature();
nodeDetails.put("aclsCount", ((aclFeature == null) ? "NONE" : aclFeature.getEntriesSize()));
if (node.isFile()) {
nodeDetails.put("type", "file");
INodeFile file = node.asFile();
nodeDetails.put("underConstruction", file.isUnderConstruction());
nodeDetails.put("isWithSnapshot", file.isWithSnapshot());
nodeDetails.put("fileSize", file.computeFileSize());
nodeDetails.put("replicationFactor", file.getFileReplication());
nodeDetails.put("numBlocks", file.getBlocks().length);
nodeDetails.put(
"blocks",
Arrays.stream(file.getBlocks())
.map(k -> k.getBlockName() + "_" + k.getGenerationStamp() + " " + k.getNumBytes())
.collect(Collectors.toList()));
} else {
nodeDetails.put("type", "directory");
INodeDirectory dir = node.asDirectory();
nodeDetails.put("snapshottable", dir.isSnapshottable());
nodeDetails.put("isWithSnapshot", dir.isWithSnapshot());
nodeDetails.put(
"children",
StreamSupport.stream(
dir.getChildrenList(Snapshot.CURRENT_STATE_ID).spliterator(), false)
.map(INode::getLocalName)
.collect(Collectors.toList()));
}
String json = Histograms.toJson(nodeDetails);
writer.write(json);
writer.flush();
} finally {
IOUtils.closeStream(writer);
}
}

@Override // VersionInterface
public Function<INode, Long> getFilterFunctionToLongForINode(String filter) {
switch (filter) {
case "dirNumChildren":
return x -> ((long) x.asDirectory().getChildrenList(Snapshot.CURRENT_STATE_ID).size());
case "dirSubTreeSize":
return x -> {
FSDirectory fsd = namesystem.dir;
ContentSummaryComputationContext cscc =
new ContentSummaryComputationContext(
fsd,
fsd.getFSNamesystem(),
fsd.getContentCountLimit(),
fsd.getContentSleepMicroSec());
try {
return x.computeContentSummary(Snapshot.CURRENT_STATE_ID, cscc)
.getCounts()
.getStoragespace();
} catch (AccessControlException e) {
throw new RuntimeException(e);
}
};
case "dirSubTreeNumFiles":
return x -> {
FSDirectory fsd = namesystem.dir;
ContentSummaryComputationContext cscc =
new ContentSummaryComputationContext(
fsd,
fsd.getFSNamesystem(),
fsd.getContentCountLimit(),
fsd.getContentSleepMicroSec());
try {
return x.computeContentSummary(Snapshot.CURRENT_STATE_ID, cscc)
.getCounts()
.getFileCount();
} catch (AccessControlException e) {
throw new RuntimeException(e);
}
};
case "dirSubTreeNumDirs":
return x -> {
FSDirectory fsd = namesystem.dir;
ContentSummaryComputationContext cscc =
new ContentSummaryComputationContext(
fsd,
fsd.getFSNamesystem(),
fsd.getContentCountLimit(),
fsd.getContentSleepMicroSec());
try {
return x.computeContentSummary(Snapshot.CURRENT_STATE_ID, cscc)
.getCounts()
.getDirectoryCount();
} catch (AccessControlException e) {
throw new RuntimeException(e);
}
};
case "storageType":
return x -> ((long) x.getStoragePolicyID());
default:
return null;
}
}

@Override // VersionInterface
public Function<INode, Boolean> getFilterFunctionToBooleanForINode(String filter) {
switch (filter) {
case "hasQuota":
return node -> node.asDirectory().isWithQuota();
default:
return null;
}
}

@Override // VersionInterface
public void saveNamespace() throws IOException {
namesystem.saveNamespace(0, 0);
}

@Override // VersionInterface
public void saveLegacyOivImage(String dir) throws IOException {
namesystem.getFSImage().saveLegacyOIVImage(namesystem, dir, new Canceler());
}

@Override // VersionInterface
public Long getNsQuota(INode node) {
return node.getQuotaCounts().getNameSpace();
}

@Override // VersionInterface
public Long getNsQuotaUsed(INode node) {
return node.computeQuotaUsage(BlockStoragePolicySuite.createDefaultSuite()).getNameSpace();
}

@Override // VersionInterface
public Long getDsQuota(INode node) {
return node.getQuotaCounts().getStorageSpace();
}

@Override // VersionInterface
public Long getDsQuotaUsed(INode node) {
return node.computeQuotaUsage(BlockStoragePolicySuite.createDefaultSuite()).getStorageSpace();
}

@Override // VersionInterface
public void startStandbyServices(Configuration conf) throws IOException {
namesystem.startStandbyServices(conf, false);
}
}
101 changes: 101 additions & 0 deletions src/main/3.3.0/org/apache/hadoop/util/GSetSeperatorWrapper.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,101 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.apache.hadoop.util;

import com.google.common.collect.Iterators;
import java.util.Collection;
import java.util.Iterator;
import java.util.Map;
import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INodeWithAdditionalFields;
import org.jetbrains.annotations.NotNull;

public class GSetSeperatorWrapper implements GSet<INode, INodeWithAdditionalFields> {

private final Map<INode, INodeWithAdditionalFields> fileSet;
private final Map<INode, INodeWithAdditionalFields> dirSet;

/**
* Constructor.
*
* @param files mapping of inode files to maintain
* @param dirs mapping of inode dirs to maintain
*/
public GSetSeperatorWrapper(
Map<INode, INodeWithAdditionalFields> files, Map<INode, INodeWithAdditionalFields> dirs) {
this.fileSet = files;
this.dirSet = dirs;
}

@Override
public int size() {
return fileSet.size() + dirSet.size();
}

@Override
public boolean contains(INode key) {
return fileSet.containsKey(key) || dirSet.containsKey(key);
}

@Override
public INodeWithAdditionalFields get(INode key) {
INodeWithAdditionalFields val;
val = fileSet.get(key);
if (val != null) {
return val;
}
return dirSet.get(key);
}

@Override
public INodeWithAdditionalFields put(INodeWithAdditionalFields element) {
if (element.isFile()) {
return fileSet.put(element, element);
}
return dirSet.put(element, element);
}

@Override
public INodeWithAdditionalFields remove(INode key) {
INodeWithAdditionalFields removed;
removed = fileSet.remove(key);
if (removed != null) {
return removed;
}
return dirSet.remove(key);
}

@Override
public void clear() {
fileSet.clear();
dirSet.clear();
}

@Override
public Collection<INodeWithAdditionalFields> values() {
return CollectionsView.combine(fileSet.values(), dirSet.values());
}

@NotNull
@Override
public Iterator<INodeWithAdditionalFields> iterator() {
return Iterators.concat(fileSet.values().iterator(), dirSet.values().iterator());
}
}

0 comments on commit d9bc43d

Please sign in to comment.