|
@@ -21,6 +21,7 @@ import java.io.FileNotFoundException;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.net.InetSocketAddress;
|
|
import java.net.InetSocketAddress;
|
|
import java.net.URL;
|
|
import java.net.URL;
|
|
|
|
+import java.security.PrivilegedExceptionAction;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.Collection;
|
|
import java.util.Collection;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
@@ -42,6 +43,7 @@ import org.apache.hadoop.mapreduce.util.ConfigUtil;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.io.Text;
|
|
|
|
+import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.token.Token;
|
|
import org.apache.hadoop.security.token.Token;
|
|
import org.apache.hadoop.security.token.TokenRenewer;
|
|
import org.apache.hadoop.security.token.TokenRenewer;
|
|
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
|
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
|
@@ -421,6 +423,11 @@ public class JobClient extends CLI {
|
|
}
|
|
}
|
|
|
|
|
|
Cluster cluster;
|
|
Cluster cluster;
|
|
|
|
+ /**
|
|
|
|
+ * Ugi of the client. We store this ugi when the client is created and
|
|
|
|
+ * then make sure that the same ugi is used to run the various protocols.
|
|
|
|
+ */
|
|
|
|
+ UserGroupInformation clientUgi;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Create a job client.
|
|
* Create a job client.
|
|
@@ -458,6 +465,7 @@ public class JobClient extends CLI {
|
|
public void init(JobConf conf) throws IOException {
|
|
public void init(JobConf conf) throws IOException {
|
|
setConf(conf);
|
|
setConf(conf);
|
|
cluster = new Cluster(conf);
|
|
cluster = new Cluster(conf);
|
|
|
|
+ clientUgi = UserGroupInformation.getCurrentUser();
|
|
}
|
|
}
|
|
|
|
|
|
@InterfaceAudience.Private
|
|
@InterfaceAudience.Private
|
|
@@ -487,8 +495,7 @@ public class JobClient extends CLI {
|
|
@Override
|
|
@Override
|
|
public boolean isManaged(Token<?> token) throws IOException {
|
|
public boolean isManaged(Token<?> token) throws IOException {
|
|
return true;
|
|
return true;
|
|
- }
|
|
|
|
-
|
|
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -500,6 +507,7 @@ public class JobClient extends CLI {
|
|
public JobClient(InetSocketAddress jobTrackAddr,
|
|
public JobClient(InetSocketAddress jobTrackAddr,
|
|
Configuration conf) throws IOException {
|
|
Configuration conf) throws IOException {
|
|
cluster = new Cluster(jobTrackAddr, conf);
|
|
cluster = new Cluster(jobTrackAddr, conf);
|
|
|
|
+ clientUgi = UserGroupInformation.getCurrentUser();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -562,21 +570,34 @@ public class JobClient extends CLI {
|
|
* @throws FileNotFoundException
|
|
* @throws FileNotFoundException
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- public RunningJob submitJob(JobConf conf) throws FileNotFoundException,
|
|
|
|
|
|
+ public RunningJob submitJob(final JobConf conf) throws FileNotFoundException,
|
|
IOException {
|
|
IOException {
|
|
try {
|
|
try {
|
|
conf.setBooleanIfUnset("mapred.mapper.new-api", false);
|
|
conf.setBooleanIfUnset("mapred.mapper.new-api", false);
|
|
conf.setBooleanIfUnset("mapred.reducer.new-api", false);
|
|
conf.setBooleanIfUnset("mapred.reducer.new-api", false);
|
|
- Job job = Job.getInstance(conf);
|
|
|
|
- job.submit();
|
|
|
|
|
|
+ Job job = clientUgi.doAs(new PrivilegedExceptionAction<Job> () {
|
|
|
|
+ @Override
|
|
|
|
+ public Job run() throws IOException, ClassNotFoundException,
|
|
|
|
+ InterruptedException {
|
|
|
|
+ Job job = Job.getInstance(conf);
|
|
|
|
+ job.submit();
|
|
|
|
+ return job;
|
|
|
|
+ }
|
|
|
|
+ });
|
|
return new NetworkedJob(job);
|
|
return new NetworkedJob(job);
|
|
} catch (InterruptedException ie) {
|
|
} catch (InterruptedException ie) {
|
|
throw new IOException("interrupted", ie);
|
|
throw new IOException("interrupted", ie);
|
|
- } catch (ClassNotFoundException cnfe) {
|
|
|
|
- throw new IOException("class not found", cnfe);
|
|
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private Job getJobUsingCluster(final JobID jobid) throws IOException,
|
|
|
|
+ InterruptedException {
|
|
|
|
+ return clientUgi.doAs(new PrivilegedExceptionAction<Job>() {
|
|
|
|
+ public Job run() throws IOException, InterruptedException {
|
|
|
|
+ return cluster.getJob(jobid);
|
|
|
|
+ }
|
|
|
|
+ });
|
|
|
|
+ }
|
|
/**
|
|
/**
|
|
* Get an {@link RunningJob} object to track an ongoing job. Returns
|
|
* Get an {@link RunningJob} object to track an ongoing job. Returns
|
|
* null if the id does not correspond to any known job.
|
|
* null if the id does not correspond to any known job.
|
|
@@ -586,9 +607,10 @@ public class JobClient extends CLI {
|
|
* <code>jobid</code> doesn't correspond to any known job.
|
|
* <code>jobid</code> doesn't correspond to any known job.
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- public RunningJob getJob(JobID jobid) throws IOException {
|
|
|
|
|
|
+ public RunningJob getJob(final JobID jobid) throws IOException {
|
|
try {
|
|
try {
|
|
- Job job = cluster.getJob(jobid);
|
|
|
|
|
|
+
|
|
|
|
+ Job job = getJobUsingCluster(jobid);
|
|
if (job != null) {
|
|
if (job != null) {
|
|
JobStatus status = JobStatus.downgrade(job.getStatus());
|
|
JobStatus status = JobStatus.downgrade(job.getStatus());
|
|
if (status != null) {
|
|
if (status != null) {
|
|
@@ -621,9 +643,10 @@ public class JobClient extends CLI {
|
|
return getTaskReports(jobId, TaskType.MAP);
|
|
return getTaskReports(jobId, TaskType.MAP);
|
|
}
|
|
}
|
|
|
|
|
|
- private TaskReport[] getTaskReports(JobID jobId, TaskType type) throws IOException {
|
|
|
|
|
|
+ private TaskReport[] getTaskReports(final JobID jobId, TaskType type) throws
|
|
|
|
+ IOException {
|
|
try {
|
|
try {
|
|
- Job j = cluster.getJob(jobId);
|
|
|
|
|
|
+ Job j = getJobUsingCluster(jobId);
|
|
if(j == null) {
|
|
if(j == null) {
|
|
return EMPTY_TASK_REPORTS;
|
|
return EMPTY_TASK_REPORTS;
|
|
}
|
|
}
|
|
@@ -688,10 +711,11 @@ public class JobClient extends CLI {
|
|
* @param state the state of the task
|
|
* @param state the state of the task
|
|
* (pending/running/completed/failed/killed)
|
|
* (pending/running/completed/failed/killed)
|
|
*/
|
|
*/
|
|
- public void displayTasks(JobID jobId, String type, String state)
|
|
|
|
|
|
+ public void displayTasks(final JobID jobId, String type, String state)
|
|
throws IOException {
|
|
throws IOException {
|
|
try {
|
|
try {
|
|
- super.displayTasks(cluster.getJob(jobId), type, state);
|
|
|
|
|
|
+ Job job = getJobUsingCluster(jobId);
|
|
|
|
+ super.displayTasks(job, type, state);
|
|
} catch (InterruptedException ie) {
|
|
} catch (InterruptedException ie) {
|
|
throw new IOException(ie);
|
|
throw new IOException(ie);
|
|
}
|
|
}
|
|
@@ -706,15 +730,20 @@ public class JobClient extends CLI {
|
|
*/
|
|
*/
|
|
public ClusterStatus getClusterStatus() throws IOException {
|
|
public ClusterStatus getClusterStatus() throws IOException {
|
|
try {
|
|
try {
|
|
- ClusterMetrics metrics = cluster.getClusterStatus();
|
|
|
|
- return new ClusterStatus(metrics.getTaskTrackerCount(),
|
|
|
|
- metrics.getBlackListedTaskTrackerCount(), cluster.getTaskTrackerExpiryInterval(),
|
|
|
|
- metrics.getOccupiedMapSlots(),
|
|
|
|
- metrics.getOccupiedReduceSlots(), metrics.getMapSlotCapacity(),
|
|
|
|
- metrics.getReduceSlotCapacity(),
|
|
|
|
- cluster.getJobTrackerStatus(),
|
|
|
|
- metrics.getDecommissionedTaskTrackerCount());
|
|
|
|
- } catch (InterruptedException ie) {
|
|
|
|
|
|
+ return clientUgi.doAs(new PrivilegedExceptionAction<ClusterStatus>() {
|
|
|
|
+ public ClusterStatus run() throws IOException, InterruptedException {
|
|
|
|
+ ClusterMetrics metrics = cluster.getClusterStatus();
|
|
|
|
+ return new ClusterStatus(metrics.getTaskTrackerCount(),
|
|
|
|
+ metrics.getBlackListedTaskTrackerCount(), cluster.getTaskTrackerExpiryInterval(),
|
|
|
|
+ metrics.getOccupiedMapSlots(),
|
|
|
|
+ metrics.getOccupiedReduceSlots(), metrics.getMapSlotCapacity(),
|
|
|
|
+ metrics.getReduceSlotCapacity(),
|
|
|
|
+ cluster.getJobTrackerStatus(),
|
|
|
|
+ metrics.getDecommissionedTaskTrackerCount());
|
|
|
|
+ }
|
|
|
|
+ });
|
|
|
|
+ }
|
|
|
|
+ catch (InterruptedException ie) {
|
|
throw new IOException(ie);
|
|
throw new IOException(ie);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -750,13 +779,17 @@ public class JobClient extends CLI {
|
|
*/
|
|
*/
|
|
public ClusterStatus getClusterStatus(boolean detailed) throws IOException {
|
|
public ClusterStatus getClusterStatus(boolean detailed) throws IOException {
|
|
try {
|
|
try {
|
|
- ClusterMetrics metrics = cluster.getClusterStatus();
|
|
|
|
- return new ClusterStatus(arrayToStringList(cluster.getActiveTaskTrackers()),
|
|
|
|
- arrayToBlackListInfo(cluster.getBlackListedTaskTrackers()),
|
|
|
|
- cluster.getTaskTrackerExpiryInterval(), metrics.getOccupiedMapSlots(),
|
|
|
|
- metrics.getOccupiedReduceSlots(), metrics.getMapSlotCapacity(),
|
|
|
|
- metrics.getReduceSlotCapacity(),
|
|
|
|
- cluster.getJobTrackerStatus());
|
|
|
|
|
|
+ return clientUgi.doAs(new PrivilegedExceptionAction<ClusterStatus>() {
|
|
|
|
+ public ClusterStatus run() throws IOException, InterruptedException {
|
|
|
|
+ ClusterMetrics metrics = cluster.getClusterStatus();
|
|
|
|
+ return new ClusterStatus(arrayToStringList(cluster.getActiveTaskTrackers()),
|
|
|
|
+ arrayToBlackListInfo(cluster.getBlackListedTaskTrackers()),
|
|
|
|
+ cluster.getTaskTrackerExpiryInterval(), metrics.getOccupiedMapSlots(),
|
|
|
|
+ metrics.getOccupiedReduceSlots(), metrics.getMapSlotCapacity(),
|
|
|
|
+ metrics.getReduceSlotCapacity(),
|
|
|
|
+ cluster.getJobTrackerStatus());
|
|
|
|
+ }
|
|
|
|
+ });
|
|
} catch (InterruptedException ie) {
|
|
} catch (InterruptedException ie) {
|
|
throw new IOException(ie);
|
|
throw new IOException(ie);
|
|
}
|
|
}
|
|
@@ -787,7 +820,14 @@ public class JobClient extends CLI {
|
|
*/
|
|
*/
|
|
public JobStatus[] getAllJobs() throws IOException {
|
|
public JobStatus[] getAllJobs() throws IOException {
|
|
try {
|
|
try {
|
|
- org.apache.hadoop.mapreduce.JobStatus[] jobs = cluster.getAllJobStatuses();
|
|
|
|
|
|
+ org.apache.hadoop.mapreduce.JobStatus[] jobs =
|
|
|
|
+ clientUgi.doAs(new PrivilegedExceptionAction<
|
|
|
|
+ org.apache.hadoop.mapreduce.JobStatus[]> () {
|
|
|
|
+ public org.apache.hadoop.mapreduce.JobStatus[] run()
|
|
|
|
+ throws IOException, InterruptedException {
|
|
|
|
+ return cluster.getAllJobStatuses();
|
|
|
|
+ }
|
|
|
|
+ });
|
|
JobStatus[] stats = new JobStatus[jobs.length];
|
|
JobStatus[] stats = new JobStatus[jobs.length];
|
|
for (int i = 0; i < jobs.length; i++) {
|
|
for (int i = 0; i < jobs.length; i++) {
|
|
stats[i] = JobStatus.downgrade(jobs[i]);
|
|
stats[i] = JobStatus.downgrade(jobs[i]);
|
|
@@ -909,7 +949,12 @@ public class JobClient extends CLI {
|
|
*/
|
|
*/
|
|
public int getDefaultMaps() throws IOException {
|
|
public int getDefaultMaps() throws IOException {
|
|
try {
|
|
try {
|
|
- return cluster.getClusterStatus().getMapSlotCapacity();
|
|
|
|
|
|
+ return clientUgi.doAs(new PrivilegedExceptionAction<Integer>() {
|
|
|
|
+ @Override
|
|
|
|
+ public Integer run() throws IOException, InterruptedException {
|
|
|
|
+ return cluster.getClusterStatus().getMapSlotCapacity();
|
|
|
|
+ }
|
|
|
|
+ });
|
|
} catch (InterruptedException ie) {
|
|
} catch (InterruptedException ie) {
|
|
throw new IOException(ie);
|
|
throw new IOException(ie);
|
|
}
|
|
}
|
|
@@ -923,7 +968,12 @@ public class JobClient extends CLI {
|
|
*/
|
|
*/
|
|
public int getDefaultReduces() throws IOException {
|
|
public int getDefaultReduces() throws IOException {
|
|
try {
|
|
try {
|
|
- return cluster.getClusterStatus().getReduceSlotCapacity();
|
|
|
|
|
|
+ return clientUgi.doAs(new PrivilegedExceptionAction<Integer>() {
|
|
|
|
+ @Override
|
|
|
|
+ public Integer run() throws IOException, InterruptedException {
|
|
|
|
+ return cluster.getClusterStatus().getReduceSlotCapacity();
|
|
|
|
+ }
|
|
|
|
+ });
|
|
} catch (InterruptedException ie) {
|
|
} catch (InterruptedException ie) {
|
|
throw new IOException(ie);
|
|
throw new IOException(ie);
|
|
}
|
|
}
|
|
@@ -936,8 +986,13 @@ public class JobClient extends CLI {
|
|
*/
|
|
*/
|
|
public Path getSystemDir() {
|
|
public Path getSystemDir() {
|
|
try {
|
|
try {
|
|
- return cluster.getSystemDir();
|
|
|
|
- } catch (IOException ioe) {
|
|
|
|
|
|
+ return clientUgi.doAs(new PrivilegedExceptionAction<Path>() {
|
|
|
|
+ @Override
|
|
|
|
+ public Path run() throws IOException, InterruptedException {
|
|
|
|
+ return cluster.getSystemDir();
|
|
|
|
+ }
|
|
|
|
+ });
|
|
|
|
+ } catch (IOException ioe) {
|
|
return null;
|
|
return null;
|
|
} catch (InterruptedException ie) {
|
|
} catch (InterruptedException ie) {
|
|
return null;
|
|
return null;
|
|
@@ -962,7 +1017,11 @@ public class JobClient extends CLI {
|
|
*/
|
|
*/
|
|
public JobQueueInfo[] getRootQueues() throws IOException {
|
|
public JobQueueInfo[] getRootQueues() throws IOException {
|
|
try {
|
|
try {
|
|
- return getJobQueueInfoArray(cluster.getRootQueues());
|
|
|
|
|
|
+ return clientUgi.doAs(new PrivilegedExceptionAction<JobQueueInfo[]>() {
|
|
|
|
+ public JobQueueInfo[] run() throws IOException, InterruptedException {
|
|
|
|
+ return getJobQueueInfoArray(cluster.getRootQueues());
|
|
|
|
+ }
|
|
|
|
+ });
|
|
} catch (InterruptedException ie) {
|
|
} catch (InterruptedException ie) {
|
|
throw new IOException(ie);
|
|
throw new IOException(ie);
|
|
}
|
|
}
|
|
@@ -976,9 +1035,13 @@ public class JobClient extends CLI {
|
|
* @return the array of immediate children JobQueueInfo objects
|
|
* @return the array of immediate children JobQueueInfo objects
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- public JobQueueInfo[] getChildQueues(String queueName) throws IOException {
|
|
|
|
|
|
+ public JobQueueInfo[] getChildQueues(final String queueName) throws IOException {
|
|
try {
|
|
try {
|
|
- return getJobQueueInfoArray(cluster.getChildQueues(queueName));
|
|
|
|
|
|
+ return clientUgi.doAs(new PrivilegedExceptionAction<JobQueueInfo[]>() {
|
|
|
|
+ public JobQueueInfo[] run() throws IOException, InterruptedException {
|
|
|
|
+ return getJobQueueInfoArray(cluster.getChildQueues(queueName));
|
|
|
|
+ }
|
|
|
|
+ });
|
|
} catch (InterruptedException ie) {
|
|
} catch (InterruptedException ie) {
|
|
throw new IOException(ie);
|
|
throw new IOException(ie);
|
|
}
|
|
}
|
|
@@ -993,7 +1056,11 @@ public class JobClient extends CLI {
|
|
*/
|
|
*/
|
|
public JobQueueInfo[] getQueues() throws IOException {
|
|
public JobQueueInfo[] getQueues() throws IOException {
|
|
try {
|
|
try {
|
|
- return getJobQueueInfoArray(cluster.getQueues());
|
|
|
|
|
|
+ return clientUgi.doAs(new PrivilegedExceptionAction<JobQueueInfo[]>() {
|
|
|
|
+ public JobQueueInfo[] run() throws IOException, InterruptedException {
|
|
|
|
+ return getJobQueueInfoArray(cluster.getQueues());
|
|
|
|
+ }
|
|
|
|
+ });
|
|
} catch (InterruptedException ie) {
|
|
} catch (InterruptedException ie) {
|
|
throw new IOException(ie);
|
|
throw new IOException(ie);
|
|
}
|
|
}
|
|
@@ -1007,9 +1074,14 @@ public class JobClient extends CLI {
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
|
|
|
|
- public JobStatus[] getJobsFromQueue(String queueName) throws IOException {
|
|
|
|
|
|
+ public JobStatus[] getJobsFromQueue(final String queueName) throws IOException {
|
|
try {
|
|
try {
|
|
- QueueInfo queue = cluster.getQueue(queueName);
|
|
|
|
|
|
+ QueueInfo queue = clientUgi.doAs(new PrivilegedExceptionAction<QueueInfo>() {
|
|
|
|
+ @Override
|
|
|
|
+ public QueueInfo run() throws IOException, InterruptedException {
|
|
|
|
+ return cluster.getQueue(queueName);
|
|
|
|
+ }
|
|
|
|
+ });
|
|
if (queue == null) {
|
|
if (queue == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
@@ -1032,9 +1104,14 @@ public class JobClient extends CLI {
|
|
* @return Queue information associated to particular queue.
|
|
* @return Queue information associated to particular queue.
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- public JobQueueInfo getQueueInfo(String queueName) throws IOException {
|
|
|
|
|
|
+ public JobQueueInfo getQueueInfo(final String queueName) throws IOException {
|
|
try {
|
|
try {
|
|
- QueueInfo queueInfo = cluster.getQueue(queueName);
|
|
|
|
|
|
+ QueueInfo queueInfo = clientUgi.doAs(new
|
|
|
|
+ PrivilegedExceptionAction<QueueInfo>() {
|
|
|
|
+ public QueueInfo run() throws IOException, InterruptedException {
|
|
|
|
+ return cluster.getQueue(queueName);
|
|
|
|
+ }
|
|
|
|
+ });
|
|
if (queueInfo != null) {
|
|
if (queueInfo != null) {
|
|
return new JobQueueInfo(queueInfo);
|
|
return new JobQueueInfo(queueInfo);
|
|
}
|
|
}
|
|
@@ -1052,7 +1129,14 @@ public class JobClient extends CLI {
|
|
public QueueAclsInfo[] getQueueAclsForCurrentUser() throws IOException {
|
|
public QueueAclsInfo[] getQueueAclsForCurrentUser() throws IOException {
|
|
try {
|
|
try {
|
|
org.apache.hadoop.mapreduce.QueueAclsInfo[] acls =
|
|
org.apache.hadoop.mapreduce.QueueAclsInfo[] acls =
|
|
- cluster.getQueueAclsForCurrentUser();
|
|
|
|
|
|
+ clientUgi.doAs(new
|
|
|
|
+ PrivilegedExceptionAction
|
|
|
|
+ <org.apache.hadoop.mapreduce.QueueAclsInfo[]>() {
|
|
|
|
+ public org.apache.hadoop.mapreduce.QueueAclsInfo[] run()
|
|
|
|
+ throws IOException, InterruptedException {
|
|
|
|
+ return cluster.getQueueAclsForCurrentUser();
|
|
|
|
+ }
|
|
|
|
+ });
|
|
QueueAclsInfo[] ret = new QueueAclsInfo[acls.length];
|
|
QueueAclsInfo[] ret = new QueueAclsInfo[acls.length];
|
|
for (int i = 0 ; i < acls.length; i++ ) {
|
|
for (int i = 0 ; i < acls.length; i++ ) {
|
|
ret[i] = QueueAclsInfo.downgrade(acls[i]);
|
|
ret[i] = QueueAclsInfo.downgrade(acls[i]);
|
|
@@ -1070,8 +1154,14 @@ public class JobClient extends CLI {
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
public Token<DelegationTokenIdentifier>
|
|
public Token<DelegationTokenIdentifier>
|
|
- getDelegationToken(Text renewer) throws IOException, InterruptedException {
|
|
|
|
- return cluster.getDelegationToken(renewer);
|
|
|
|
|
|
+ getDelegationToken(final Text renewer) throws IOException, InterruptedException {
|
|
|
|
+ return clientUgi.doAs(new
|
|
|
|
+ PrivilegedExceptionAction<Token<DelegationTokenIdentifier>>() {
|
|
|
|
+ public Token<DelegationTokenIdentifier> run() throws IOException,
|
|
|
|
+ InterruptedException {
|
|
|
|
+ return cluster.getDelegationToken(renewer);
|
|
|
|
+ }
|
|
|
|
+ });
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|