You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by yu...@apache.org on 2017/09/19 07:23:45 UTC

[1/8] incubator-rocketmq git commit: initialize RocketMQ5

Repository: incubator-rocketmq
Updated Branches:
  refs/heads/rocketmq5 [created] 0b88e66fa


http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcProxyCommon.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcProxyCommon.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcProxyCommon.java
new file mode 100644
index 0000000..c5d9a3c
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcProxyCommon.java
@@ -0,0 +1,329 @@
+/*
+ * 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.rocketmq.rpc.impl.service;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import org.apache.rocketmq.remoting.api.AsyncHandler;
+import org.apache.rocketmq.remoting.api.RemotingService;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.remoting.api.serializable.Serializer;
+import org.apache.rocketmq.remoting.api.serializable.SerializerFactory;
+import org.apache.rocketmq.remoting.external.ThreadUtils;
+import org.apache.rocketmq.remoting.impl.protocol.serializer.SerializerFactoryImpl;
+import org.apache.rocketmq.rpc.annotation.MethodType;
+import org.apache.rocketmq.rpc.annotation.RemoteMethod;
+import org.apache.rocketmq.rpc.annotation.RemoteService;
+import org.apache.rocketmq.rpc.annotation.VisibleForInternal;
+import org.apache.rocketmq.rpc.api.Promise;
+import org.apache.rocketmq.rpc.api.PromiseListener;
+import org.apache.rocketmq.rpc.impl.command.ResponseCode;
+import org.apache.rocketmq.rpc.impl.command.RpcRequestCode;
+import org.apache.rocketmq.rpc.impl.config.RpcCommonConfig;
+import org.apache.rocketmq.rpc.impl.context.RpcCallerContext;
+import org.apache.rocketmq.rpc.impl.exception.ServiceExceptionHandlerManager;
+import org.apache.rocketmq.rpc.impl.exception.ServiceExceptionManager;
+import org.apache.rocketmq.rpc.impl.metrics.ServiceStats;
+import org.apache.rocketmq.rpc.impl.promise.DefaultPromise;
+import org.apache.rocketmq.rpc.internal.ExceptionMessageUtil;
+import org.apache.rocketmq.rpc.internal.RpcErrorMapper;
+import org.apache.rocketmq.rpc.internal.ServiceUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.rocketmq.remoting.internal.ExceptionUtils.getStackTrace;
+
+public abstract class RpcProxyCommon {
+    protected final static SerializerFactory SERIALIZER_FACTORY = new SerializerFactoryImpl();
+    protected static final Logger log = LoggerFactory.getLogger(RpcProxyCommon.class);
+    protected final ThreadLocal<RpcCallerContext> threadLocalCallerContext = new ThreadLocal<>();
+    protected RpcCommonConfig rpcCommonConfig;
+    protected ExecutorService callServiceThreadPool;
+    protected ExecutorService promiseExecutorService;
+    protected ServiceStats serviceStats;
+
+    public RpcProxyCommon(RpcCommonConfig rpcCommonConfig) {
+        this.rpcCommonConfig = rpcCommonConfig;
+        this.serviceStats = new ServiceStats();
+        this.promiseExecutorService = ThreadUtils.newThreadPoolExecutor(rpcCommonConfig.getClientAsyncCallbackExecutorThreads(),
+            rpcCommonConfig.getClientAsyncCallbackExecutorThreads(), rpcCommonConfig.getServiceThreadKeepAliveTime(),
+            TimeUnit.MILLISECONDS, new ArrayBlockingQueue<Runnable>(rpcCommonConfig.getServiceThreadBlockQueueSize()), "promiseExecutorService", true);
+        this.callServiceThreadPool = ThreadUtils.newThreadPoolExecutor(rpcCommonConfig.getClientAsyncCallbackExecutorThreads(),
+            rpcCommonConfig.getClientAsyncCallbackExecutorThreads(), rpcCommonConfig.getServiceThreadKeepAliveTime(),
+            TimeUnit.MILLISECONDS, new ArrayBlockingQueue<Runnable>(rpcCommonConfig.getServiceThreadBlockQueueSize()), "callServiceThread", true);
+    }
+
+    private RemotingCommand createRemoteRequest(RemoteService serviceExport, Method method, Object[] args,
+        RemoteMethod asynchronous) {
+        RemotingCommand request;
+        VisibleForInternal visible = method.getAnnotation(VisibleForInternal.class);
+        if (visible != null) {
+            request = remotingService().commandFactory().createRequest();
+            request.opCode(asynchronous.name());
+            if (args != null && args.length != 0) {
+                request.parameter(args[0]);
+            }
+        } else {
+            String requestCode = ServiceUtil.toRequestCode(serviceExport, method);
+            request = createRemoteRequest(requestCode, asynchronous.version(), args);
+        }
+        return request;
+    }
+
+    public RemotingCommand createRemoteRequest(String requestCode, String version, Object[] args) {
+        RemotingCommand request = remotingService().commandFactory().createRequest();
+        request.opCode(RpcRequestCode.CALL_SERVICE);
+        RpcServiceCallBody serviceCallBody = new RpcServiceCallBody();
+        serviceCallBody.setServiceId(requestCode);
+
+        if (args != null) {
+            Serializer serializer = ServiceUtil.selectSerializer(args, SERIALIZER_FACTORY.type(rpcCommonConfig.getSerializerName()));
+            assert serializer != null;
+            request.serializerType(serializer.type());
+            for (Object arg : args) {
+                ByteBuffer byteBuffer = serializer.encode(arg);
+                byte[] array = new byte[byteBuffer.remaining()];
+                byteBuffer.get(array);
+                serviceCallBody.addParameter(array);
+            }
+        }
+        serviceCallBody.setServiceVersion(version);
+        request.parameter(serviceCallBody);
+        return request;
+    }
+
+    @SuppressWarnings("unchecked")
+    Object invoke0(Object proxy, RpcJdkProxy rpcJdkProxy, Class<?> service, final Method method,
+        Object[] args) throws Throwable {
+        RemoteMethod asynchronous = getRemoteMethod(method);
+        final String requestCode = createRequestCode(method, service);
+        RemoteService serviceExport = getRemoteService(service);
+        RemotingCommand request = createRemoteRequest(serviceExport, method, args, asynchronous);
+        return invokeRemoteMethod(rpcJdkProxy, requestCode, request, method.getGenericReturnType(), asynchronous.type());
+    }
+
+    public Object invokeRemoteMethod(final RpcJdkProxy rpcJdkProxy, final String requestCode,
+        RemotingCommand request, final Type returnType, MethodType type) {
+        try {
+            final long startTime = System.currentTimeMillis();
+            InvokeRunnable invokeRunnable = new InvokeRunnable(rpcJdkProxy, requestCode, request, type);
+            final Future future = callServiceThreadPool.submit(invokeRunnable);
+            /*
+            final Promise result = callService(method, args);
+            */
+            final DefaultPromise resPromise = new DefaultPromise();
+            Object result = future.get(rpcCommonConfig.getClientInvokeServiceTimeout(), TimeUnit.MILLISECONDS);
+            if (result == null || !(result instanceof Promise)) {
+                throw ServiceExceptionManager.TranslateException(RpcErrorMapper.RpcErrorLatitude.LOCAL.getCode(),
+                    new Exception("illegal response type is return"));
+            } else if (result instanceof Exception) {
+                throw ServiceExceptionManager.TranslateException(RpcErrorMapper.RpcErrorLatitude.LOCAL.getCode(),
+                    new Exception((Throwable) result));
+            }
+
+            final DefaultPromise promise = (DefaultPromise) result;
+            switch (type) {
+                case SYNC:
+                    final Object response = promise.get(rpcCommonConfig.getClientInvokeServiceTimeout());
+                    if (response == null) {
+                        throw new Exception("get result is null");
+                    }
+                    if (response instanceof Exception)
+                        throw (Exception) response;
+                    return processResponse(returnType, (RemotingCommand) response,
+                        requestCode, startTime);
+                case ASYNC:
+                    promiseExecutorService.submit(new Runnable() {
+                        @Override
+                        public void run() {
+                            promise.addListener(new PromiseListener() {
+                                @Override
+                                public void operationCompleted(Promise promise) {
+                                    Object response = promise.get();
+                                    if (response == null) {
+                                        resPromise.setFailure(new Exception("get result is null"));
+                                        return;
+                                    }
+                                    if (response instanceof Exception) {
+                                        resPromise.setFailure((Exception) response);
+                                        return;
+                                    }
+                                    Type restyle = ((ParameterizedType) returnType).getActualTypeArguments()[0];
+                                    try {
+                                        resPromise.set(processResponse(restyle, (RemotingCommand) response, requestCode, startTime));
+                                    } catch (Exception e) {
+                                        resPromise.setFailure(ServiceExceptionManager.TranslateException(RpcErrorMapper.RpcErrorLatitude.LOCAL.getCode(), e));
+                                    }
+                                }
+
+                                @Override
+                                public void operationFailed(Promise promise) {
+                                    resPromise.setFailure(ServiceExceptionManager.TranslateException(RpcErrorMapper.RpcErrorLatitude.LOCAL.getCode(), promise.getThrowable()));
+                                }
+                            });
+                        }
+                    });
+                    return resPromise;
+            }
+            throw ServiceExceptionManager.TranslateException(RpcErrorMapper.RpcErrorLatitude.LOCAL.getCode(), new Exception("illegal response type is return"));
+        } catch (ExecutionException e) {
+            throw ServiceExceptionManager.TranslateException(RpcErrorMapper.RpcErrorLatitude.LOCAL.getCode(), e.getCause());
+        } catch (Exception e) {
+            throw ServiceExceptionManager.TranslateException(RpcErrorMapper.RpcErrorLatitude.LOCAL.getCode(), e);
+        }
+    }
+
+    private Object processResponse(Type returnType, RemotingCommand remotingCommand,
+        String requestCode, long startTime) throws Exception {
+        RpcCallerContext rpcCallerContext = this.threadLocalCallerContext.get();
+        if (rpcCallerContext != null) {
+            rpcCallerContext.setRemotingResponse(remotingCommand);
+        }
+
+        String code = remotingCommand.opCode();
+        if (code.equals(ResponseCode.SUCCESS)) {
+            serviceStats.addCallerOKQPSValue(requestCode, 1, 1);
+            serviceStats.addCallerRTValue(requestCode, (int) (System.currentTimeMillis() - startTime), 1);
+            try {
+                if (!returnType.equals(Void.class)) {
+                    Serializer serialization = remotingService().serializerFactory().get(remotingCommand.serializerType());
+                    if (remotingCommand.parameterBytes() == null)
+                        return null;
+                    return serialization.decode(remotingCommand.parameterBytes(), returnType);
+                } else {
+                    return Void.class.newInstance();
+                }
+            } catch (Exception e) {
+                throw ServiceExceptionManager.TranslateException(RpcErrorMapper.RpcErrorLatitude.LOCAL.getCode(), e);
+            }
+        } else {
+            log.error("process response is failed, errorCode:{}, remark:{}", code, remotingCommand.remark());
+            serviceStats.addCallerFailedQPSValue(requestCode, 1, 1);
+            serviceStats.addCallerRTValue(requestCode, (int) (System.currentTimeMillis() - startTime), 1);
+            ServiceExceptionHandlerManager.exceptionHandler(code, remotingCommand, remotingService().serializerFactory());
+            return null;
+        }
+    }
+
+    private String createRequestCode(Method method, Class<?> service) {
+        return ServiceUtil.toRequestCode(getRemoteService(service), method);
+    }
+
+    private RemoteService getRemoteService(Class<?> service) {
+        RemoteService serviceExport = service.getAnnotation(RemoteService.class);
+        if (serviceExport == null) {
+            throw new IllegalArgumentException("ServiceExport:" + service.getName() + " do not remark annotation");
+        }
+        return serviceExport;
+    }
+
+    private RemoteMethod getRemoteMethod(Method method) {
+        RemoteMethod asynchronous = method.getAnnotation(RemoteMethod.class);
+        if (asynchronous == null) {
+            throw new IllegalArgumentException("ServiceMethod:" + method.getName() + " do not remark annotation");
+        }
+        return asynchronous;
+    }
+
+    private Promise invokeAsyncWrapper(final RpcJdkProxy rpcJdkProxy, RemotingCommand request,
+        final String requestCode) {
+        final RpcCallerContext rpcCallerContext = this.threadLocalCallerContext.get();
+        final DefaultPromise promise = new DefaultPromise();
+        final long startTime = System.currentTimeMillis();
+        rpcJdkProxy.invokeAsync(request, new AsyncHandler() {
+            @Override
+            public void onFailure(RemotingCommand request) {
+                serviceStats.addCallerFailedQPSValue(requestCode, 1, 1);
+                serviceStats.addCallerRTValue(requestCode, (int) (System.currentTimeMillis() - startTime), 1);
+                promise.setFailure(new Exception(ExceptionMessageUtil.buildExceptionMessage("request:" + request.toString() + " onFailure")));
+            }
+
+            @Override
+            public void onTimeout(long costTimeMillis, long timeoutMillis) {
+                serviceStats.addCallerFailedQPSValue(requestCode, 1, 1);
+                serviceStats.addCallerRTValue(requestCode, (int) (System.currentTimeMillis() - startTime), 1);
+                promise.setFailure(new Exception(ExceptionMessageUtil.buildExceptionMessage("service timeout")));
+            }
+
+            @Override
+            public void onSuccess(RemotingCommand response) {
+                if (null != rpcCallerContext) {
+                    rpcCallerContext.setRemotingResponse(response);
+                }
+                try {
+                    promise.set(response);
+                } catch (Throwable e) {
+                    promise.setFailure(new Exception(ExceptionMessageUtil.buildExceptionMessage(getStackTrace(e))));
+                }
+            }
+
+        });
+        return promise;
+    }
+
+    public abstract RemotingService remotingService();
+
+    public RpcCommonConfig getRpcCommonConfig() {
+        return rpcCommonConfig;
+    }
+
+    public void setRpcCommonConfig(final RpcCommonConfig rpcCommonConfig) {
+        this.rpcCommonConfig = rpcCommonConfig;
+    }
+
+    private class InvokeRunnable implements Callable {
+        private RpcJdkProxy rpcJdkProxy;
+        private String requestCode;
+        private RemotingCommand request;
+        private MethodType type;
+
+        InvokeRunnable(RpcJdkProxy rpcJdkProxy, String requestCode, RemotingCommand request,
+            MethodType type) {
+            this.rpcJdkProxy = rpcJdkProxy;
+            this.requestCode = requestCode;
+            this.request = request;
+            this.type = type;
+        }
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public Object call() {
+            Promise promise = new DefaultPromise();
+
+            try {
+                if (type == MethodType.ONEWAY) {
+                    rpcJdkProxy.invokeOneWay(request);
+                } else {
+                    return RpcProxyCommon.this.invokeAsyncWrapper(rpcJdkProxy, request, requestCode);
+                }
+            } catch (Exception e) {
+                promise.set(ServiceExceptionManager.TranslateException(RpcErrorMapper.RpcErrorLatitude.LOCAL.getCode(), e));
+            }
+            return promise;
+
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcProxyFactory.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcProxyFactory.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcProxyFactory.java
new file mode 100644
index 0000000..17abcd8
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcProxyFactory.java
@@ -0,0 +1,41 @@
+/*
+ * 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.rocketmq.rpc.impl.service;
+
+import org.apache.rocketmq.remoting.api.RemotingClient;
+import org.apache.rocketmq.remoting.api.RemotingServer;
+import org.apache.rocketmq.remoting.api.channel.RemotingChannel;
+import org.apache.rocketmq.rpc.impl.config.RpcCommonConfig;
+
+public class RpcProxyFactory {
+    public static RpcJdkProxy createServiceProxy(Class<?> service,
+        RpcProxyCommon rpcProxyCommon,
+        RemotingClient remotingClient,
+        RpcCommonConfig rpcCommonConfig,
+        String remotingAddress) {
+        return new RpcJdkProxyClient(service, rpcProxyCommon, remotingClient, rpcCommonConfig, remotingAddress);
+    }
+
+    public static RpcJdkProxy createServiceProxy(Class<?> service,
+        RpcProxyCommon rpcProxyCommon,
+        RemotingServer remotingServer,
+        RpcCommonConfig rpcCommonConfig,
+        RemotingChannel remotingChannel) {
+        return new RpcJdkProxyServer(service, rpcProxyCommon, remotingServer, rpcCommonConfig, remotingChannel);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcServiceCallBody.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcServiceCallBody.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcServiceCallBody.java
new file mode 100644
index 0000000..ce78587
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcServiceCallBody.java
@@ -0,0 +1,81 @@
+/*
+ * 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.rocketmq.rpc.impl.service;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.msgpack.annotation.Message;
+
+@Message
+public class RpcServiceCallBody {
+    private String serviceId;
+    private String serviceVersion;
+    private String token;
+    private List<byte[]> parameters;
+
+    public RpcServiceCallBody() {
+        parameters = new ArrayList<>();
+    }
+
+    public String getServiceId() {
+        return serviceId;
+    }
+
+    public void setServiceId(final String serviceId) {
+        this.serviceId = serviceId;
+    }
+
+    public String getToken() {
+        return token;
+    }
+
+    public void setToken(final String token) {
+        this.token = token;
+    }
+
+    public byte[] getParameter(int index) {
+        return parameters.get(index);
+    }
+
+    public List<byte[]> getParameters() {
+        return parameters;
+    }
+
+    public void setParameters(final List<byte[]> parameters) {
+        this.parameters = parameters;
+    }
+
+    public void addParameter(final byte[] parameter) {
+        parameters.add(parameter);
+    }
+
+    public String getServiceVersion() {
+        return serviceVersion;
+    }
+
+    public void setServiceVersion(final String serviceVersion) {
+        this.serviceVersion = serviceVersion;
+    }
+
+    @Override
+    public String toString() {
+        return ToStringBuilder.reflectionToString(this);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/ExceptionMessageUtil.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/ExceptionMessageUtil.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/ExceptionMessageUtil.java
new file mode 100644
index 0000000..de869f5
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/ExceptionMessageUtil.java
@@ -0,0 +1,24 @@
+/*
+ * 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.rocketmq.rpc.internal;
+
+public class ExceptionMessageUtil {
+    public static String buildExceptionMessage(String msg) {
+        return msg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/HttpTinyClient.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/HttpTinyClient.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/HttpTinyClient.java
new file mode 100644
index 0000000..47c31c9
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/HttpTinyClient.java
@@ -0,0 +1,130 @@
+/*
+ * 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.rocketmq.rpc.internal;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.util.Iterator;
+import java.util.List;
+
+public class HttpTinyClient {
+
+    static public HttpResult httpGet(String url, List<String> headers, List<String> paramValues,
+        String encoding, long readTimeoutMs) throws IOException {
+        String encodedContent = encodingParams(paramValues, encoding);
+        url += (null == encodedContent) ? "" : ("?" + encodedContent);
+
+        HttpURLConnection conn = null;
+        try {
+            conn = (HttpURLConnection) new URL(url).openConnection();
+            conn.setRequestMethod("GET");
+            conn.setConnectTimeout((int) readTimeoutMs);
+            conn.setReadTimeout((int) readTimeoutMs);
+            setHeaders(conn, headers, encoding);
+
+            conn.connect();
+            int respCode = conn.getResponseCode();
+            String resp = null;
+
+            if (HttpURLConnection.HTTP_OK == respCode) {
+                resp = IOTinyUtils.toString(conn.getInputStream(), encoding);
+            } else {
+                resp = IOTinyUtils.toString(conn.getErrorStream(), encoding);
+            }
+            return new HttpResult(respCode, resp);
+        } finally {
+            if (conn != null) {
+                conn.disconnect();
+            }
+        }
+    }
+
+    static private String encodingParams(List<String> paramValues, String encoding)
+        throws UnsupportedEncodingException {
+        StringBuilder sb = new StringBuilder();
+        if (null == paramValues) {
+            return null;
+        }
+
+        for (Iterator<String> iter = paramValues.iterator(); iter.hasNext(); ) {
+            sb.append(iter.next()).append("=");
+            sb.append(URLEncoder.encode(iter.next(), encoding));
+            if (iter.hasNext()) {
+                sb.append("&");
+            }
+        }
+        return sb.toString();
+    }
+
+    static private void setHeaders(HttpURLConnection conn, List<String> headers, String encoding) {
+        if (null != headers) {
+            for (Iterator<String> iter = headers.iterator(); iter.hasNext(); ) {
+                conn.addRequestProperty(iter.next(), iter.next());
+            }
+        }
+        conn.addRequestProperty("Client-Version", "V1.0.0");
+        conn.addRequestProperty("Content-org.apache.rocketmq.annotation.MethodType", "application/x-www-form-urlencoded;charset=" + encoding);
+        String ts = String.valueOf(System.currentTimeMillis());
+        conn.addRequestProperty("Metaq-Client-RequestTS", ts);
+    }
+
+    static public HttpResult httpPost(String url, List<String> headers, List<String> paramValues,
+        String encoding, long readTimeoutMs) throws IOException {
+        String encodedContent = encodingParams(paramValues, encoding);
+
+        HttpURLConnection conn = null;
+        try {
+            conn = (HttpURLConnection) new URL(url).openConnection();
+            conn.setRequestMethod("POST");
+            conn.setConnectTimeout(3000);
+            conn.setReadTimeout((int) readTimeoutMs);
+            conn.setDoOutput(true);
+            conn.setDoInput(true);
+            setHeaders(conn, headers, encoding);
+
+            conn.getOutputStream().write(encodedContent.getBytes("UTF-8"));
+
+            int respCode = conn.getResponseCode();
+            String resp = null;
+
+            if (HttpURLConnection.HTTP_OK == respCode) {
+                resp = IOTinyUtils.toString(conn.getInputStream(), encoding);
+            } else {
+                resp = IOTinyUtils.toString(conn.getErrorStream(), encoding);
+            }
+            return new HttpResult(respCode, resp);
+        } finally {
+            if (null != conn) {
+                conn.disconnect();
+            }
+        }
+    }
+
+    static public class HttpResult {
+        final public int code;
+        final public String content;
+
+        public HttpResult(int code, String content) {
+            this.code = code;
+            this.content = content;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/IOTinyUtils.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/IOTinyUtils.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/IOTinyUtils.java
new file mode 100644
index 0000000..fddab17
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/IOTinyUtils.java
@@ -0,0 +1,157 @@
+/*
+ * 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.rocketmq.rpc.internal;
+
+import java.io.BufferedReader;
+import java.io.CharArrayWriter;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.Reader;
+import java.io.Writer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayList;
+import java.util.List;
+
+public class IOTinyUtils {
+
+    static public String toString(InputStream input, String encoding) throws IOException {
+        return (null == encoding) ? toString(new InputStreamReader(input, "UTF-8")) : toString(new InputStreamReader(
+            input, encoding));
+    }
+
+    static public String toString(Reader reader) throws IOException {
+        CharArrayWriter sw = new CharArrayWriter();
+        copy(reader, sw);
+        return sw.toString();
+    }
+
+    static public long copy(Reader input, Writer output) throws IOException {
+        char[] buffer = new char[1 << 12];
+        long count = 0;
+        for (int n = 0; (n = input.read(buffer)) >= 0; ) {
+            output.write(buffer, 0, n);
+            count += n;
+        }
+        return count;
+    }
+
+    static public List<String> readLines(Reader input) throws IOException {
+        BufferedReader reader = toBufferedReader(input);
+        List<String> list = new ArrayList<String>();
+        String line = null;
+        for (; ; ) {
+            line = reader.readLine();
+            if (null != line) {
+                list.add(line);
+            } else {
+                break;
+            }
+        }
+        return list;
+    }
+
+    static private BufferedReader toBufferedReader(Reader reader) {
+        return reader instanceof BufferedReader ? (BufferedReader) reader : new BufferedReader(reader);
+    }
+
+    static public void copyFile(String source, String target) throws IOException {
+        File sf = new File(source);
+        if (!sf.exists()) {
+            throw new IllegalArgumentException("source file does not exist.");
+        }
+        File tf = new File(target);
+        tf.getParentFile().mkdirs();
+        if (!tf.exists() && !tf.createNewFile()) {
+            throw new RuntimeException("failed to create target file.");
+        }
+
+        FileChannel sc = null;
+        FileChannel tc = null;
+        try {
+            tc = new FileOutputStream(tf).getChannel();
+            sc = new FileInputStream(sf).getChannel();
+            sc.transferTo(0, sc.size(), tc);
+        } finally {
+            if (null != sc) {
+                sc.close();
+            }
+            if (null != tc) {
+                tc.close();
+            }
+        }
+    }
+
+    public static void delete(File fileOrDir) throws IOException {
+        if (fileOrDir == null) {
+            return;
+        }
+
+        if (fileOrDir.isDirectory()) {
+            cleanDirectory(fileOrDir);
+        }
+
+        fileOrDir.delete();
+    }
+
+    public static void cleanDirectory(File directory) throws IOException {
+        if (!directory.exists()) {
+            String message = directory + " does not exist";
+            throw new IllegalArgumentException(message);
+        }
+
+        if (!directory.isDirectory()) {
+            String message = directory + " is not a directory";
+            throw new IllegalArgumentException(message);
+        }
+
+        File[] files = directory.listFiles();
+        if (files == null) { // null if security restricted
+            throw new IOException("Failed to list contents of " + directory);
+        }
+
+        IOException exception = null;
+        for (File file : files) {
+            try {
+                delete(file);
+            } catch (IOException ioe) {
+                exception = ioe;
+            }
+        }
+
+        if (null != exception) {
+            throw exception;
+        }
+    }
+
+    public static void writeStringToFile(File file, String data, String encoding) throws IOException {
+        OutputStream os = null;
+        try {
+            os = new FileOutputStream(file);
+            os.write(data.getBytes(encoding));
+        } finally {
+            if (null != os) {
+                os.close();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/RpcErrorMapper.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/RpcErrorMapper.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/RpcErrorMapper.java
new file mode 100644
index 0000000..9def61d
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/RpcErrorMapper.java
@@ -0,0 +1,67 @@
+/*
+ * 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.rocketmq.rpc.internal;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+
+/**
+ * Every error consists of error-latitude and error-longitude
+ */
+public class RpcErrorMapper {
+    private RpcErrorLatitude errorLatitude;
+
+    @Override
+    public String toString() {
+        return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE);
+    }
+
+    public enum RpcErrorLatitude {
+        LOCAL("01"),
+        REMOTE("02");
+
+        private static Map<String, RpcErrorLatitude> stringToEnum = new HashMap<String, RpcErrorLatitude>();
+
+        static {
+            for (RpcErrorLatitude el : values()) {
+                stringToEnum.put(el.getCode(), el);
+            }
+        }
+
+        private String code;
+
+        RpcErrorLatitude(String code) {
+            this.code = code;
+        }
+
+        public static RpcErrorLatitude fromString(String cdoe) {
+            return stringToEnum.get(cdoe);
+        }
+
+        public String getCode() {
+            return code;
+        }
+
+        public void setCode(String code) {
+            this.code = code;
+        }
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/ServiceUtil.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/ServiceUtil.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/ServiceUtil.java
new file mode 100644
index 0000000..87cefc3
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/internal/ServiceUtil.java
@@ -0,0 +1,218 @@
+/*
+ * 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.rocketmq.rpc.internal;
+
+import java.io.Serializable;
+import java.lang.reflect.Method;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.util.Set;
+import java.util.TreeSet;
+import org.apache.rocketmq.remoting.api.serializable.Serializer;
+import org.apache.rocketmq.remoting.impl.protocol.serializer.SerializerFactoryImpl;
+import org.apache.rocketmq.rpc.annotation.RemoteMethod;
+import org.apache.rocketmq.rpc.annotation.RemoteService;
+import org.apache.rocketmq.rpc.impl.service.RpcEntry;
+
+public class ServiceUtil {
+    private static int methodExceptionLimit = 7;
+    private static int methodEmbeddedGenericParameterLimit = 2;
+    private static int methodSuperInterfaceLimit = 128;
+    private static int methodParameterLimit = 7;
+    private static SerializerFactoryImpl serializerFactory = new SerializerFactoryImpl();
+
+    public static String toRequestCode(final RemoteService sc, Method method) {
+        RemoteMethod asynchronous = method.getAnnotation(RemoteMethod.class);
+        if (asynchronous != null) {
+            return String.format("%s_%s/%s_%s", sc.name(), sc.version(),
+                asynchronous.name().isEmpty() ? sharedToString(method) : asynchronous.name(),
+                asynchronous.version());
+        } else
+            throw new IllegalArgumentException("RemoteMethod annotation not exist in method " + method.getName());
+    }
+
+    public static Set<String> getRequestCode(final Object obj) {
+        Set<String> requestCodeSet = new TreeSet<>();
+        Class<?>[] interfaces = obj.getClass().getInterfaces();
+        for (Class<?> itf : interfaces) {
+            RemoteService serviceExport = itf.getAnnotation(RemoteService.class);
+            if (null == serviceExport) {
+                throw new IllegalArgumentException("The Interface:" + itf.getName() + " do not remark RemoteService annotationType");
+            }
+
+            Method[] methods = itf.getMethods();
+            for (Method method : methods) {
+                if (method.getAnnotation(RemoteMethod.class) != null) {
+                    if (ServiceUtil.testServiceExportMethod(method)) {
+                        String requestCode = ServiceUtil.toRequestCode(serviceExport, method);
+                        RpcEntry se = new RpcEntry();
+                        se.setServiceExport(serviceExport);
+                        se.setObject(obj);
+                        se.setMethod(method);
+                        requestCodeSet.add(requestCode);
+                    }
+                } else {
+                    throw new IllegalArgumentException("RemoteMethod annotation not exist in method " + method.getName());
+                }
+            }
+        }
+
+        if (requestCodeSet.isEmpty()) {
+            throw new IllegalArgumentException("interface no method or interface not set annotation");
+        }
+        return requestCodeSet;
+    }
+
+    private static boolean isGenericParameterDepthMuchLimit(Type type, int count) {
+        if (count > methodEmbeddedGenericParameterLimit)
+            return true;
+        if (type instanceof ParameterizedType) {
+            ParameterizedType parameterizedType = (ParameterizedType) type;
+            Type[] types = parameterizedType.getActualTypeArguments();
+            boolean isGenericParameter;
+            for (final Type typ : types) {
+                isGenericParameter = isGenericParameterDepthMuchLimit(typ, count + 1);
+                if (isGenericParameter)
+                    return isGenericParameter;
+            }
+        }
+        return false;
+    }
+
+    private static boolean getGenericParameterTypes(Type[] types) {
+        boolean genericFlag = false;
+        for (Type type : types) {
+            genericFlag |= isGenericParameterDepthMuchLimit(type, 0);
+        }
+        return genericFlag;
+    }
+
+    private static boolean isInterfaceInHeritDepthMuchLimit(Class<?> obj, int count) {
+        if (count > methodSuperInterfaceLimit)
+            return true;
+        final RemoteService serviceExport = obj.getAnnotation(RemoteService.class);
+        if (serviceExport == null)
+            return false;
+        Class<?>[] types = obj.getInterfaces();
+        boolean isGenericParameter = false;
+        for (Class<?> type : types) {
+            isGenericParameter |= isInterfaceInHeritDepthMuchLimit(type, count + 1);
+        }
+        return isGenericParameter;
+    }
+
+    private static boolean isSeriliaze(Class<?> type) {
+        return Serializable.class.isAssignableFrom(type);
+    }
+
+    public static boolean testServiceExportMethod(Method method) {
+        Class<?>[] exceptions = method.getExceptionTypes();
+        Type[] types = method.getGenericParameterTypes();
+        if (method.getParameterTypes() != null
+            && method.getParameterTypes().length > methodParameterLimit)
+            throw new IllegalArgumentException("method: " + method.getName() + " argument more than max limit : "
+                + methodParameterLimit);
+        if (exceptions != null && exceptions.length > methodExceptionLimit)
+            throw new IllegalArgumentException(" method: " + method.getName() + " exception more than max limit : "
+                + methodExceptionLimit);
+
+        if (getGenericParameterTypes(types))
+            throw new IllegalArgumentException(" method: " + method.getName() + " generic argument depth more than max limit : "
+                + methodEmbeddedGenericParameterLimit);
+
+        Class<?>[] classObjs = method.getClass().getInterfaces();
+        boolean isInterfaceHeritDepthMuch = false;
+        for (Class classObj : classObjs) {
+            isInterfaceHeritDepthMuch |= isInterfaceInHeritDepthMuchLimit(classObj, 0);
+        }
+        if (isInterfaceHeritDepthMuch) {
+            throw new IllegalArgumentException(" method: " + method.getName() + " interface inherit depth more than max limit : "
+                + methodSuperInterfaceLimit);
+        }
+        /*
+        Class<?>[] tys = method.getParameterTypes();
+        for (Class type : tys) {
+            if (!isSeriliaze(type)) {
+                throw new IllegalArgumentException("method parameter type:" + type.getName() + " can not Seriliaze !");
+            }
+        }
+        */
+        return true;
+    }
+
+    private static String sharedToString(Method method) {
+        try {
+            StringBuilder sb = new StringBuilder();
+
+            String methodName = method.getName();
+            if (methodName.contains("Async")) {
+                int index = methodName.indexOf("Async");
+                methodName = methodName.substring(0, index);
+            }
+            if (methodName.contains("Oneway")) {
+                int index = methodName.indexOf("Oneway");
+                methodName = methodName.substring(0, index);
+            }
+            sb.append(methodName);
+            sb.append('<');
+
+            Class<?>[] types = method.getParameterTypes();
+            for (int j = 0; j < types.length; j++) {
+                sb.append(types[j].getCanonicalName());
+                if (j < (types.length - 1))
+                    sb.append(",");
+            }
+            sb.append('>');
+            return sb.toString();
+        } catch (Exception e) {
+            return "<" + e + ">";
+        }
+    }
+
+    private static boolean isSerializer(Object[] args, Serializer serializer) {
+        boolean success;
+        if (serializer == null)
+            return false;
+        try {
+            for (Object arg : args) {
+                serializer.encode(arg);
+            }
+            success = true;
+        } catch (Exception e) {
+            success = false;
+        }
+        return success;
+    }
+
+    private static Serializer selectSerialization(Object[] args, Serializer defaultSerializer) {
+        if (!isSerializer(args, defaultSerializer)) {
+            Serializer[] serializers = serializerFactory.getTables();
+            for (Serializer serializer : serializers) {
+                if (isSerializer(args, serializer))
+                    return serializer;
+            }
+        } else
+            return defaultSerializer;
+        return null;
+    }
+
+    public static Serializer selectSerializer(Object[] args, byte type) {
+        Serializer defaultSerializer = serializerFactory.get(type);
+        return ServiceUtil.selectSerialization(args, defaultSerializer);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/style/checkstyle.xml
----------------------------------------------------------------------
diff --git a/style/checkstyle.xml b/style/checkstyle.xml
new file mode 100644
index 0000000..aca6ed1
--- /dev/null
+++ b/style/checkstyle.xml
@@ -0,0 +1,140 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+
+<!DOCTYPE module PUBLIC
+    "-//Puppy Crawl//DTD Check Configuration 1.3//EN"
+    "http://www.puppycrawl.com/dtds/configuration_1_3.dtd">
+<!--Refer http://checkstyle.sourceforge.net/reports/google-java-style.html#s2.2-file-encoding -->
+<module name="Checker">
+
+    <property name="localeLanguage" value="en"/>
+
+    <!--To configure the check to report on the first instance in each file-->
+    <module name="FileTabCharacter"/>
+
+    <!-- header -->
+    <module name="RegexpHeader">
+        <property name="header" value="/\*\nLicensed to the Apache Software Foundation*"/>
+        <property name="fileExtensions" value="java"/>
+    </module>
+
+    <module name="RegexpHeader">
+        <property name="header" value="#[\s]*Licensed to the Apache Software Foundation*"/>
+        <property name="fileExtensions" value="properties"/>
+    </module>
+
+    <module name="RegexpSingleline">
+        <!--<property name="format" value="System\.out\.println"/>-->
+        <property name="message" value="Prohibit invoking System.out.println in source code !"/>
+    </module>
+
+    <!--<module name="RegexpSingleline">-->
+    <!--<property name="format" value="//FIXME"/>-->
+    <!--<property name="message" value="Recommended fix FIXME task !"/>-->
+    <!--</module>-->
+
+    <module name="RegexpSingleline">
+        <property name="format" value="//TODO"/>
+        <property name="message" value="Recommended fix TODO task !"/>
+    </module>
+
+    <module name="RegexpSingleline">
+        <property name="format" value="@alibaba"/>
+        <property name="message" value="Recommended remove @alibaba keyword!"/>
+    </module>
+    <module name="RegexpSingleline">
+        <property name="format" value="@taobao"/>
+        <property name="message" value="Recommended remove @taobao keyword!"/>
+    </module>
+    <module name="RegexpSingleline">
+        <property name="format" value="@author"/>
+        <property name="message" value="Recommended remove @author tag in javadoc!"/>
+    </module>
+
+    <module name="RegexpSingleline">
+        <property name="format"
+                  value=".*[\u3400-\u4DB5\u4E00-\u9FA5\u9FA6-\u9FBB\uF900-\uFA2D\uFA30-\uFA6A\uFA70-\uFAD9\uFF00-\uFFEF\u2E80-\u2EFF\u3000-\u303F\u31C0-\u31EF]+.*"/>
+        <property name="message" value="Not allow chinese character !"/>
+    </module>
+
+    <module name="FileLength">
+        <property name="max" value="3000"/>
+    </module>
+
+    <module name="TreeWalker">
+
+        <module name="UnusedImports">
+            <property name="processJavadoc" value="true"/>
+        </module>
+        <module name="RedundantImport"/>
+
+        <!--<module name="IllegalImport" />-->
+
+        <!--Checks that classes that override equals() also override hashCode()-->
+        <module name="EqualsHashCode"/>
+        <!--Checks for over-complicated boolean expressions. Currently finds code like if (topic == true), topic || true, !false, etc.-->
+        <module name="SimplifyBooleanExpression"/>
+        <module name="OneStatementPerLine"/>
+        <module name="UnnecessaryParentheses"/>
+        <!--Checks for over-complicated boolean return statements. For example the following code-->
+        <module name="SimplifyBooleanReturn"/>
+
+        <!--Check that the default is after all the cases in producerGroup switch statement-->
+        <module name="DefaultComesLast"/>
+        <!--Detects empty statements (standalone ";" semicolon)-->
+        <module name="EmptyStatement"/>
+        <!--Checks that long constants are defined with an upper ell-->
+        <module name="UpperEll"/>
+        <module name="ConstantName">
+            <property name="format" value="(^[A-Z][A-Z0-9]*(_[A-Z0-9]+)*$)|(^log$)"/>
+        </module>
+        <!--Checks that local, non-final variable names conform to producerGroup format specified by the format property-->
+        <module name="LocalVariableName"/>
+        <!--Validates identifiers for local, final variables, including catch parameters-->
+        <module name="LocalFinalVariableName"/>
+        <!--Validates identifiers for non-static fields-->
+        <module name="MemberName"/>
+        <!--Validates identifiers for class type parameters-->
+        <!--<module name="ClassTypeParameterName">-->
+        <!--<property name="format" value="^[A-Z0-9]*$"/>-->
+        <!--</module>-->
+        <!--Validates identifiers for method type parameters-->
+        <module name="MethodTypeParameterName">
+            <property name="format" value="^[A-Z0-9]*$"/>
+        </module>
+        <module name="PackageName"/>
+        <module name="ParameterName"/>
+        <module name="StaticVariableName"/>
+        <module name="TypeName"/>
+        <!--Checks that there are no import statements that use the * notation-->
+        <module name="AvoidStarImport"/>
+
+        <!--whitespace-->
+        <module name="GenericWhitespace"/>
+        <!--<module name="NoWhitespaceBefore"/>-->
+        <module name="NoWhitespaceAfter"/>
+        <module name="WhitespaceAround">
+            <property name="allowEmptyConstructors" value="true"/>
+            <property name="allowEmptyMethods" value="true"/>
+        </module>
+        <!--<module name="Indentation"/>-->
+        <module name="MethodParamPad"/>
+        <module name="ParenPad"/>
+        <module name="TypecastParenPad"/>
+    </module>
+</module>

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/style/codeStyle.xml
----------------------------------------------------------------------
diff --git a/style/codeStyle.xml b/style/codeStyle.xml
new file mode 100644
index 0000000..7c7ce54
--- /dev/null
+++ b/style/codeStyle.xml
@@ -0,0 +1,143 @@
+<!--
+  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.
+  -->
+
+<code_scheme name="rocketmq">
+    <option name="USE_SAME_INDENTS" value="true"/>
+    <option name="IGNORE_SAME_INDENTS_FOR_LANGUAGES" value="true"/>
+    <option name="OTHER_INDENT_OPTIONS">
+        <value>
+            <option name="INDENT_SIZE" value="4"/>
+            <option name="CONTINUATION_INDENT_SIZE" value="4"/>
+            <option name="TAB_SIZE" value="4"/>
+            <option name="USE_TAB_CHARACTER" value="false"/>
+            <option name="SMART_TABS" value="false"/>
+            <option name="LABEL_INDENT_SIZE" value="0"/>
+            <option name="LABEL_INDENT_ABSOLUTE" value="false"/>
+            <option name="USE_RELATIVE_INDENTS" value="false"/>
+        </value>
+    </option>
+    <option name="PREFER_LONGER_NAMES" value="false"/>
+    <option name="CLASS_COUNT_TO_USE_IMPORT_ON_DEMAND" value="1000"/>
+    <option name="NAMES_COUNT_TO_USE_IMPORT_ON_DEMAND" value="1000"/>
+    <option name="PACKAGES_TO_USE_IMPORT_ON_DEMAND">
+        <value/>
+    </option>
+    <option name="IMPORT_LAYOUT_TABLE">
+        <value>
+            <package name="" withSubpackages="true" static="false"/>
+            <emptyLine/>
+            <package name="" withSubpackages="true" static="true"/>
+        </value>
+    </option>
+    <option name="JD_ALIGN_PARAM_COMMENTS" value="false"/>
+    <option name="JD_ALIGN_EXCEPTION_COMMENTS" value="false"/>
+    <option name="JD_P_AT_EMPTY_LINES" value="false"/>
+    <option name="JD_KEEP_INVALID_TAGS" value="false"/>
+    <option name="JD_DO_NOT_WRAP_ONE_LINE_COMMENTS" value="true"/>
+    <option name="KEEP_CONTROL_STATEMENT_IN_ONE_LINE" value="false"/>
+    <option name="KEEP_BLANK_LINES_IN_DECLARATIONS" value="1"/>
+    <option name="KEEP_BLANK_LINES_IN_CODE" value="1"/>
+    <option name="KEEP_BLANK_LINES_BEFORE_RBRACE" value="1"/>
+    <option name="WHILE_ON_NEW_LINE" value="true"/>
+    <option name="ALIGN_MULTILINE_PARAMETERS" value="false"/>
+    <option name="ALIGN_MULTILINE_FOR" value="false"/>
+    <option name="SPACE_AFTER_TYPE_CAST" value="true"/>
+    <option name="SPACE_BEFORE_ARRAY_INITIALIZER_LBRACE" value="true"/>
+    <option name="METHOD_PARAMETERS_WRAP" value="1"/>
+    <option name="ARRAY_INITIALIZER_LBRACE_ON_NEXT_LINE" value="true"/>
+    <option name="LABELED_STATEMENT_WRAP" value="1"/>
+    <option name="WRAP_COMMENTS" value="true"/>
+    <option name="METHOD_ANNOTATION_WRAP" value="1"/>
+    <option name="CLASS_ANNOTATION_WRAP" value="1"/>
+    <option name="FIELD_ANNOTATION_WRAP" value="1"/>
+    <JavaCodeStyleSettings>
+        <option name="CLASS_NAMES_IN_JAVADOC" value="3"/>
+    </JavaCodeStyleSettings>
+    <XML>
+        <option name="XML_LEGACY_SETTINGS_IMPORTED" value="true"/>
+    </XML>
+    <ADDITIONAL_INDENT_OPTIONS fileType="haml">
+        <option name="INDENT_SIZE" value="2"/>
+    </ADDITIONAL_INDENT_OPTIONS>
+    <codeStyleSettings language="Groovy">
+        <option name="KEEP_CONTROL_STATEMENT_IN_ONE_LINE" value="false"/>
+        <option name="KEEP_BLANK_LINES_IN_DECLARATIONS" value="1"/>
+        <option name="KEEP_BLANK_LINES_IN_CODE" value="1"/>
+        <option name="KEEP_BLANK_LINES_BEFORE_RBRACE" value="1"/>
+        <option name="ALIGN_MULTILINE_PARAMETERS" value="false"/>
+        <option name="ALIGN_MULTILINE_FOR" value="false"/>
+        <option name="METHOD_PARAMETERS_WRAP" value="1"/>
+        <option name="METHOD_ANNOTATION_WRAP" value="1"/>
+        <option name="CLASS_ANNOTATION_WRAP" value="1"/>
+        <option name="FIELD_ANNOTATION_WRAP" value="1"/>
+        <option name="PARENT_SETTINGS_INSTALLED" value="true"/>
+        <indentOptions>
+            <option name="CONTINUATION_INDENT_SIZE" value="4"/>
+        </indentOptions>
+    </codeStyleSettings>
+    <codeStyleSettings language="HOCON">
+        <option name="KEEP_BLANK_LINES_BEFORE_RBRACE" value="1"/>
+        <option name="PARENT_SETTINGS_INSTALLED" value="true"/>
+    </codeStyleSettings>
+    <codeStyleSettings language="JAVA">
+        <option name="KEEP_CONTROL_STATEMENT_IN_ONE_LINE" value="false"/>
+        <option name="KEEP_BLANK_LINES_IN_DECLARATIONS" value="1"/>
+        <option name="KEEP_BLANK_LINES_IN_CODE" value="1"/>
+        <option name="KEEP_BLANK_LINES_BEFORE_RBRACE" value="1"/>
+        <option name="WHILE_ON_NEW_LINE" value="true"/>
+        <option name="ALIGN_MULTILINE_PARAMETERS" value="false"/>
+        <option name="ALIGN_MULTILINE_FOR" value="false"/>
+        <option name="SPACE_BEFORE_ARRAY_INITIALIZER_LBRACE" value="true"/>
+        <option name="METHOD_PARAMETERS_WRAP" value="1"/>
+        <option name="ARRAY_INITIALIZER_LBRACE_ON_NEXT_LINE" value="true"/>
+        <option name="LABELED_STATEMENT_WRAP" value="1"/>
+        <option name="METHOD_ANNOTATION_WRAP" value="1"/>
+        <option name="CLASS_ANNOTATION_WRAP" value="1"/>
+        <option name="FIELD_ANNOTATION_WRAP" value="1"/>
+        <option name="PARENT_SETTINGS_INSTALLED" value="true"/>
+        <indentOptions>
+            <option name="CONTINUATION_INDENT_SIZE" value="4"/>
+        </indentOptions>
+    </codeStyleSettings>
+    <codeStyleSettings language="JSON">
+        <option name="KEEP_BLANK_LINES_IN_CODE" value="1"/>
+        <option name="PARENT_SETTINGS_INSTALLED" value="true"/>
+    </codeStyleSettings>
+    <codeStyleSettings language="Scala">
+        <option name="KEEP_BLANK_LINES_IN_DECLARATIONS" value="1"/>
+        <option name="KEEP_BLANK_LINES_IN_CODE" value="1"/>
+        <option name="KEEP_BLANK_LINES_BEFORE_RBRACE" value="1"/>
+        <option name="WHILE_ON_NEW_LINE" value="true"/>
+        <option name="ALIGN_MULTILINE_PARAMETERS" value="false"/>
+        <option name="ALIGN_MULTILINE_FOR" value="false"/>
+        <option name="METHOD_PARAMETERS_WRAP" value="1"/>
+        <option name="METHOD_ANNOTATION_WRAP" value="1"/>
+        <option name="CLASS_ANNOTATION_WRAP" value="1"/>
+        <option name="FIELD_ANNOTATION_WRAP" value="1"/>
+        <option name="PARENT_SETTINGS_INSTALLED" value="true"/>
+        <indentOptions>
+            <option name="INDENT_SIZE" value="4"/>
+            <option name="CONTINUATION_INDENT_SIZE" value="4"/>
+            <option name="TAB_SIZE" value="4"/>
+        </indentOptions>
+    </codeStyleSettings>
+    <codeStyleSettings language="XML">
+        <indentOptions>
+            <option name="CONTINUATION_INDENT_SIZE" value="4"/>
+        </indentOptions>
+    </codeStyleSettings>
+</code_scheme>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/style/copyright/Apache.xml
----------------------------------------------------------------------
diff --git a/style/copyright/Apache.xml b/style/copyright/Apache.xml
new file mode 100644
index 0000000..7ca71b5
--- /dev/null
+++ b/style/copyright/Apache.xml
@@ -0,0 +1,24 @@
+<!--
+  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.
+  -->
+
+<component name="CopyrightManager">
+    <copyright>
+        <option name="myName" value="Apache"/>
+        <option name="notice"
+                value="Licensed to the Apache Software Foundation (ASF) under one or more&#10;contributor license agreements.  See the NOTICE file distributed with&#10;this work for additional information regarding copyright ownership.&#10;The ASF licenses this file to You under the Apache License, Version 2.0&#10;(the &quot;License&quot;); you may not use this file except in compliance with&#10;the License.  You may obtain a copy of the License at&#10;&#10;    http://www.apache.org/licenses/LICENSE-2.0&#10;&#10;Unless required by applicable law or agreed to in writing, software&#10;distributed under the License is distributed on an &quot;AS IS&quot; BASIS,&#10;WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.&#10;See the License for the specific language governing permissions and&#10;limitations under the License."/>
+    </copyright>
+</component>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/style/copyright/profiles_settings.xml
----------------------------------------------------------------------
diff --git a/style/copyright/profiles_settings.xml b/style/copyright/profiles_settings.xml
new file mode 100644
index 0000000..d326b8c
--- /dev/null
+++ b/style/copyright/profiles_settings.xml
@@ -0,0 +1,64 @@
+<!--
+  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.
+  -->
+
+<component name="CopyrightManager">
+    <settings default="Apache">
+        <module2copyright>
+            <element module="All" copyright="Apache"/>
+        </module2copyright>
+        <LanguageOptions name="GSP">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="prefixLines" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="HTML">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="prefixLines" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="JAVA">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="addBlankAfter" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="JSP">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="prefixLines" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="JSPX">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="prefixLines" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="MXML">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="prefixLines" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="Properties">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="block" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="SPI">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="block" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="XML">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="prefixLines" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="__TEMPLATE__">
+            <option name="separateBefore" value="true"/>
+            <option name="lenBefore" value="1"/>
+        </LanguageOptions>
+    </settings>
+</component>
\ No newline at end of file


[7/8] incubator-rocketmq git commit: initialize RocketMQ5

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/NestedRuntimeException.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/NestedRuntimeException.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/NestedRuntimeException.java
new file mode 100644
index 0000000..7ef01db
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/NestedRuntimeException.java
@@ -0,0 +1,97 @@
+/*
+ * 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.rocketmq.remoting.api.exception;
+
+/**
+ * Handy class for wrapping runtime {@code Exceptions} with a root cause.
+ *
+ * <p>This class is {@code abstract} to force the programmer to extend
+ * the class. {@code getMessage} will include nested exception
+ * information; {@code getRootCause} will include the innermost cause of
+ * this exception, if any; {@code printStackTrace} and other like methods will
+ * delegate to the wrapped exception, if any.
+ *
+ * @since 1.0.0
+ */
+public abstract class NestedRuntimeException extends RuntimeException {
+    private static final long serialVersionUID = -8371779880133933367L;
+
+    /**
+     * Construct a {@code NestedRuntimeException} with the specified detail message.
+     *
+     * @param msg the detail message
+     */
+    public NestedRuntimeException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * Construct a {@code NestedRuntimeException} with the specified detail message
+     * and nested exception.
+     *
+     * @param msg the detail message
+     * @param cause the nested exception
+     */
+    public NestedRuntimeException(String msg, Throwable cause) {
+        super(msg, cause);
+    }
+
+    /**
+     * Build a message for the given base message and root cause.
+     *
+     * @param message the base message
+     * @param cause the root cause
+     * @return the full exception message
+     */
+    private static String getMessageWithCause(String message, Throwable cause) {
+        if (cause != null) {
+            StringBuilder sb = new StringBuilder();
+            if (message != null) {
+                sb.append(message).append("; ");
+            }
+            sb.append("nested exception is ").append(cause);
+            return sb.toString();
+        } else {
+            return message;
+        }
+    }
+
+    /**
+     * Return the detail message, including the message from the nested exception
+     * if there is one.
+     */
+    @Override
+    public String getMessage() {
+        return getMessageWithCause(super.getMessage(), getCause());
+    }
+
+    /**
+     * Retrieve the innermost cause of this exception, if any.
+     *
+     * @return the innermost exception, or {@code null} if none
+     */
+    public Throwable getRootCause() {
+        Throwable rootCause = null;
+        Throwable cause = getCause();
+        while (cause != null && cause != rootCause) {
+            rootCause = cause;
+            cause = cause.getCause();
+        }
+        return rootCause;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/RemoteAccessException.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/RemoteAccessException.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/RemoteAccessException.java
new file mode 100644
index 0000000..6ce6dd4
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/RemoteAccessException.java
@@ -0,0 +1,57 @@
+/*
+ * 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.rocketmq.remoting.api.exception;
+
+/**
+ * Generic remote access exception. A service proxy for any remoting
+ * protocol should throw this exception or subclasses of it, in order
+ * to transparently expose a plain Java business interface.
+ *
+ * <p>A client may catch RemoteAccessException if it wants to, but as
+ * remote access errors are typically unrecoverable, it will probably let
+ * such exceptions propagate to a higher level that handles them generically.
+ * In this case, the client opCode doesn't show any signs of being involved in
+ * remote access, as there aren't any remoting-specific dependencies.
+ *
+ * @since 1.0.0
+ */
+public class RemoteAccessException extends NestedRuntimeException {
+    private static final long serialVersionUID = 6280428909532427263L;
+
+    /**
+     * Constructor for RemoteAccessException with the specified detail message.
+     *
+     * @param msg the detail message
+     */
+    public RemoteAccessException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * Constructor for RemoteAccessException with the specified detail message
+     * and nested exception.
+     *
+     * @param msg the detail message
+     * @param cause the root cause (usually from using an underlying
+     * remoting API such as RMI)
+     */
+    public RemoteAccessException(String msg, Throwable cause) {
+        super(msg, cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/RemoteCodecException.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/RemoteCodecException.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/RemoteCodecException.java
new file mode 100644
index 0000000..a8b9e4e
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/RemoteCodecException.java
@@ -0,0 +1,33 @@
+/*
+ * 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.rocketmq.remoting.api.exception;
+
+/**
+ * @since 1.0.0
+ */
+public class RemoteCodecException extends RemoteAccessException {
+    private static final long serialVersionUID = -7597014042746200543L;
+
+    public RemoteCodecException(String msg) {
+        super(msg);
+    }
+
+    public RemoteCodecException(String msg, Throwable cause) {
+        super(msg, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/RemoteConnectFailureException.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/RemoteConnectFailureException.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/RemoteConnectFailureException.java
new file mode 100644
index 0000000..af0a6e9
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/RemoteConnectFailureException.java
@@ -0,0 +1,49 @@
+/*
+ * 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.rocketmq.remoting.api.exception;
+
+/**
+ * RemoteConnectFailureException will be thrown when connection
+ * could not be established with a remote service.
+ *
+ * @since 1.0.0
+ */
+public class RemoteConnectFailureException extends RemoteAccessException {
+    private static final long serialVersionUID = -5565366231695911316L;
+
+    /**
+     * Constructor for RemoteConnectFailureException with the specified detail message
+     * and nested exception.
+     *
+     * @param msg the detail message
+     * @param cause the root cause from the remoting API in use
+     */
+    public RemoteConnectFailureException(String msg, Throwable cause) {
+        super(msg, cause);
+    }
+
+    /**
+     * Constructor for RemoteConnectFailureException with the specified detail message.
+     *
+     * @param msg the detail message
+     */
+    public RemoteConnectFailureException(String msg) {
+        super(msg);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/RemoteTimeoutException.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/RemoteTimeoutException.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/RemoteTimeoutException.java
new file mode 100644
index 0000000..adfcc8d
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/exception/RemoteTimeoutException.java
@@ -0,0 +1,70 @@
+/*
+ * 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.rocketmq.remoting.api.exception;
+
+/**
+ * RemoteTimeoutException will be thrown when the execution
+ * of the target method did not complete before a configurable
+ * timeout, for example when a reply message was not received.
+ *
+ * @since 1.0.0
+ */
+public class RemoteTimeoutException extends RemoteAccessException {
+    private static final long serialVersionUID = 8710772392914461626L;
+
+    /**
+     * Constructor for RemoteTimeoutException with the specified detail message,configurable timeout.
+     *
+     * @param msg the detail message
+     * @param timeoutMillis configurable timeout
+     */
+    public RemoteTimeoutException(String msg, long timeoutMillis) {
+        this(msg, timeoutMillis, null);
+    }
+
+    /**
+     * Constructor for RemoteTimeoutException with the specified detail message,configurable timeout
+     * and nested exception..
+     *
+     * @param msg the detail message
+     * @param timeoutMillis configurable timeout
+     * @param cause Exception cause
+     */
+    public RemoteTimeoutException(String msg, long timeoutMillis, Throwable cause) {
+        super(String.format("%s, waiting for %s ms", msg, timeoutMillis), cause);
+    }
+
+    /**
+     * Constructor for RemoteTimeoutException with the specified detail message.
+     *
+     * @param msg the detail message
+     */
+    public RemoteTimeoutException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * Constructor for RemoteTimeoutException with the specified detail message
+     * and nested exception.
+     *
+     * @param msg the detail message
+     * @param cause the root cause from the remoting API in use
+     */
+    public RemoteTimeoutException(String msg, Throwable cause) {
+        super(msg, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/ExceptionContext.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/ExceptionContext.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/ExceptionContext.java
new file mode 100644
index 0000000..2452309
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/ExceptionContext.java
@@ -0,0 +1,76 @@
+/*
+ * 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.rocketmq.remoting.api.interceptor;
+
+import org.apache.rocketmq.remoting.api.RemotingEndPoint;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+
+public class ExceptionContext extends RequestContext {
+    private Throwable exception;
+    private String remark;
+
+    public ExceptionContext(RemotingEndPoint remotingEndPoint, String remoteAddr, RemotingCommand request,
+        Throwable exception, String remark) {
+        super(remotingEndPoint, remoteAddr, request);
+        this.remotingEndPoint = remotingEndPoint;
+        this.remoteAddr = remoteAddr;
+        this.request = request;
+        this.exception = exception;
+        this.remark = remark;
+    }
+
+    public RemotingEndPoint getRemotingEndPoint() {
+        return remotingEndPoint;
+    }
+
+    public void setRemotingEndPoint(RemotingEndPoint remotingEndPoint) {
+        this.remotingEndPoint = remotingEndPoint;
+    }
+
+    public String getRemoteAddr() {
+        return remoteAddr;
+    }
+
+    public void setRemoteAddr(String remoteAddr) {
+        this.remoteAddr = remoteAddr;
+    }
+
+    public RemotingCommand getRequest() {
+        return request;
+    }
+
+    public void setRequest(RemotingCommand request) {
+        this.request = request;
+    }
+
+    public Throwable getException() {
+        return exception;
+    }
+
+    public void setException(Throwable exception) {
+        this.exception = exception;
+    }
+
+    public String getRemark() {
+        return remark;
+    }
+
+    public void setRemark(String remark) {
+        this.remark = remark;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/Interceptor.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/Interceptor.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/Interceptor.java
new file mode 100644
index 0000000..62257ef
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/Interceptor.java
@@ -0,0 +1,26 @@
+/*
+ * 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.rocketmq.remoting.api.interceptor;
+
+public interface Interceptor {
+    void beforeRequest(final RequestContext context);
+
+    void afterResponseReceived(final ResponseContext context);
+
+    void onException(final ExceptionContext context);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/InterceptorGroup.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/InterceptorGroup.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/InterceptorGroup.java
new file mode 100644
index 0000000..9ffc696
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/InterceptorGroup.java
@@ -0,0 +1,49 @@
+/*
+ * 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.rocketmq.remoting.api.interceptor;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class InterceptorGroup {
+    private final List<Interceptor> interceptors = new ArrayList<Interceptor>();
+
+    public void registerInterceptor(final Interceptor interceptor) {
+        if (interceptor != null) {
+            this.interceptors.add(interceptor);
+        }
+    }
+
+    public void beforeRequest(final RequestContext context) {
+        for (Interceptor interceptor : interceptors) {
+            interceptor.beforeRequest(context);
+        }
+    }
+
+    public void afterResponseReceived(final ResponseContext context) {
+        for (Interceptor interceptor : interceptors) {
+            interceptor.afterResponseReceived(context);
+        }
+    }
+
+    public void onException(final ExceptionContext context) {
+        for (Interceptor interceptor : interceptors) {
+            interceptor.onException(context);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/RequestContext.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/RequestContext.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/RequestContext.java
new file mode 100644
index 0000000..d961556
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/RequestContext.java
@@ -0,0 +1,65 @@
+/*
+ * 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.rocketmq.remoting.api.interceptor;
+
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.rocketmq.remoting.api.RemotingEndPoint;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+
+public class RequestContext {
+    protected RemotingEndPoint remotingEndPoint;
+    protected String remoteAddr;
+    protected RemotingCommand request;
+
+    public RequestContext(RemotingEndPoint remotingEndPoint, String remoteAddr, RemotingCommand request) {
+        super();
+        this.remotingEndPoint = remotingEndPoint;
+        this.remoteAddr = remoteAddr;
+        this.request = request;
+    }
+
+    public RemotingEndPoint getRemotingEndPoint() {
+        return remotingEndPoint;
+    }
+
+    public void setRemotingEndPoint(RemotingEndPoint remotingEndPoint) {
+        this.remotingEndPoint = remotingEndPoint;
+    }
+
+    public String getRemoteAddr() {
+        return remoteAddr;
+    }
+
+    public void setRemoteAddr(String remoteAddr) {
+        this.remoteAddr = remoteAddr;
+    }
+
+    public RemotingCommand getRequest() {
+        return request;
+    }
+
+    public void setRequest(RemotingCommand request) {
+        this.request = request;
+    }
+
+    @Override
+    public String toString() {
+        return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/ResponseContext.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/ResponseContext.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/ResponseContext.java
new file mode 100644
index 0000000..97ec2e6
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/interceptor/ResponseContext.java
@@ -0,0 +1,73 @@
+/*
+ * 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.rocketmq.remoting.api.interceptor;
+
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.rocketmq.remoting.api.RemotingEndPoint;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+
+public class ResponseContext extends RequestContext {
+    private RemotingCommand response;
+
+    public ResponseContext(RemotingEndPoint remotingEndPoint, String remoteAddr, RemotingCommand request,
+        RemotingCommand response) {
+        super(remotingEndPoint, remoteAddr, request);
+        this.remotingEndPoint = remotingEndPoint;
+        this.remoteAddr = remoteAddr;
+        this.request = request;
+        this.response = response;
+    }
+
+    public RemotingEndPoint getRemotingEndPoint() {
+        return remotingEndPoint;
+    }
+
+    public void setRemotingEndPoint(RemotingEndPoint remotingEndPoint) {
+        this.remotingEndPoint = remotingEndPoint;
+    }
+
+    public String getRemoteAddr() {
+        return remoteAddr;
+    }
+
+    public void setRemoteAddr(String remoteAddr) {
+        this.remoteAddr = remoteAddr;
+    }
+
+    public RemotingCommand getRequest() {
+        return request;
+    }
+
+    public void setRequest(RemotingCommand request) {
+        this.request = request;
+    }
+
+    public RemotingCommand getResponse() {
+        return response;
+    }
+
+    public void setResponse(RemotingCommand response) {
+        this.response = response;
+    }
+
+    @Override
+    public String toString() {
+        return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/protocol/Protocol.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/protocol/Protocol.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/protocol/Protocol.java
new file mode 100644
index 0000000..5caf167
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/protocol/Protocol.java
@@ -0,0 +1,39 @@
+/*
+ * 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.rocketmq.remoting.api.protocol;
+
+import org.apache.rocketmq.remoting.api.channel.ChannelHandlerContextWrapper;
+
+public interface Protocol {
+    /**
+     * Minimum Viable Protocol
+     */
+    String MVP = "mvp";
+    String HTTP2 = "http2";
+    String WEBSOCKET = "websocket";
+
+    byte MVP_MAGIC = 0x14;
+    byte WEBSOCKET_MAGIC = 0x15;
+    byte HTTP_2_MAGIC = 0x16;
+
+    String name();
+
+    byte type();
+
+    void assembleHandler(ChannelHandlerContextWrapper ctx);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/protocol/ProtocolFactory.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/protocol/ProtocolFactory.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/protocol/ProtocolFactory.java
new file mode 100644
index 0000000..cf016f9
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/protocol/ProtocolFactory.java
@@ -0,0 +1,30 @@
+/*
+ * 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.rocketmq.remoting.api.protocol;
+
+public interface ProtocolFactory {
+    void register(Protocol protocol);
+
+    void resetAll(Protocol protocol);
+
+    byte type(String protocolName);
+
+    Protocol get(byte type);
+
+    void clearAll();
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/serializable/Serializer.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/serializable/Serializer.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/serializable/Serializer.java
new file mode 100644
index 0000000..8ef8dcd
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/serializable/Serializer.java
@@ -0,0 +1,36 @@
+/*
+ * 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.rocketmq.remoting.api.serializable;
+
+import java.lang.reflect.Type;
+import java.nio.ByteBuffer;
+import org.apache.rocketmq.remoting.common.TypePresentation;
+
+public interface Serializer {
+    String name();
+
+    byte type();
+
+    <T> T decode(final byte[] content, final Class<T> c);
+
+    <T> T decode(final byte[] content, final TypePresentation<T> typePresentation);
+
+    <T> T decode(final byte[] content, final Type type);
+
+    ByteBuffer encode(final Object object);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/serializable/SerializerFactory.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/serializable/SerializerFactory.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/serializable/SerializerFactory.java
new file mode 100644
index 0000000..b47bf99
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/serializable/SerializerFactory.java
@@ -0,0 +1,28 @@
+/*
+ * 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.rocketmq.remoting.api.serializable;
+
+public interface SerializerFactory {
+    void register(Serializer serialization);
+
+    byte type(String serializationName);
+
+    Serializer get(byte type);
+
+    void clearAll();
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/common/Pair.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/common/Pair.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/common/Pair.java
new file mode 100644
index 0000000..505e104
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/common/Pair.java
@@ -0,0 +1,44 @@
+/*
+ * 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.rocketmq.remoting.common;
+
+public class Pair<L, R> {
+    private L left;
+    private R right;
+
+    public Pair(L left, R right) {
+        this.left = left;
+        this.right = right;
+    }
+
+    public L getLeft() {
+        return left;
+    }
+
+    public void setLeft(L left) {
+        this.left = left;
+    }
+
+    public R getRight() {
+        return right;
+    }
+
+    public void setRight(R right) {
+        this.right = right;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/common/TypePresentation.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/common/TypePresentation.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/common/TypePresentation.java
new file mode 100644
index 0000000..ef3d5f8
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/common/TypePresentation.java
@@ -0,0 +1,73 @@
+/*
+ * 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.rocketmq.remoting.common;
+
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Represents a generic type {@code T}. Java doesn't yet provide a way to
+ * represent generic types, so this class does. Forces clients to create a
+ * subclass of this class which enables retrieval the type information even at
+ * runtime.
+ *
+ * <p>For example, to create a type literal for {@code List<String>}, you can
+ * create an empty anonymous inner class:
+ *
+ * <pre>
+ * TypePresentation&lt;List&lt;String&gt;&gt; list = new TypePresentation&lt;List&lt;String&gt;&gt;() {};
+ * </pre>
+ *
+ * To create a type literal for {@code Map<String, Integer>}:
+ *
+ * <pre>
+ * TypePresentation&lt;Map&lt;String, Integer&gt;&gt; map = new TypePresentation&lt;Map&lt;String, Integer&gt;&gt;() {};
+ * </pre>
+ *
+ * This syntax cannot be used to create type literals that have wildcard
+ * parameters, such as {@code Class<?>} or {@code List<? extends CharSequence>}.
+ *
+ * @since 1.0.0
+ */
+public class TypePresentation<T> {
+    static ConcurrentMap<Class<?>, ConcurrentMap<Type, ConcurrentMap<Type, Type>>> classTypeCache
+        = new ConcurrentHashMap<Class<?>, ConcurrentMap<Type, ConcurrentMap<Type, Type>>>(16, 0.75f, 1);
+    protected final Type type;
+
+    /**
+     * Constructs a new type literal. Derives represented class from type
+     * parameter.
+     *
+     * <p>Clients create an empty anonymous subclass. Doing so embeds the type
+     * parameter in the anonymous class's type hierarchy so we can reconstitute it
+     * at runtime despite erasure.
+     */
+    protected TypePresentation() {
+        Type superClass = getClass().getGenericSuperclass();
+        type = ((ParameterizedType) superClass).getActualTypeArguments()[0];
+    }
+
+    /**
+     * @return underlying {@code Type} instance.
+     */
+    public Type getType() {
+        return type;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/pom.xml
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/pom.xml b/remoting-core/remoting-impl/pom.xml
new file mode 100644
index 0000000..53d1854
--- /dev/null
+++ b/remoting-core/remoting-impl/pom.xml
@@ -0,0 +1,71 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.rocketmq</groupId>
+        <artifactId>remoting-core</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>remoting-impl</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-all</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.alibaba</groupId>
+            <artifactId>fastjson</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.msgpack</groupId>
+            <artifactId>msgpack</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.esotericsoftware</groupId>
+            <artifactId>kryo</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-tcnative-boringssl-static</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>remoting-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>clirr-maven-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/ChannelEventListenerGroup.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/ChannelEventListenerGroup.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/ChannelEventListenerGroup.java
new file mode 100644
index 0000000..8af61f7
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/ChannelEventListenerGroup.java
@@ -0,0 +1,61 @@
+/*
+ * 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.rocketmq.remoting.common;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.rocketmq.remoting.api.channel.ChannelEventListener;
+import org.apache.rocketmq.remoting.api.channel.RemotingChannel;
+
+public class ChannelEventListenerGroup {
+    private final List<ChannelEventListener> listenerList = new ArrayList<ChannelEventListener>();
+
+    public int size() {
+        return this.listenerList.size();
+    }
+
+    public void registerChannelEventListener(final ChannelEventListener listener) {
+        if (listener != null) {
+            this.listenerList.add(listener);
+        }
+    }
+
+    public void onChannelConnect(final RemotingChannel channel) {
+        for (ChannelEventListener listener : listenerList) {
+            listener.onChannelConnect(channel);
+        }
+    }
+
+    public void onChannelClose(final RemotingChannel channel) {
+        for (ChannelEventListener listener : listenerList) {
+            listener.onChannelClose(channel);
+        }
+    }
+
+    public void onChannelException(final RemotingChannel channel) {
+        for (ChannelEventListener listener : listenerList) {
+            listener.onChannelException(channel);
+        }
+    }
+
+    public void onChannelIdle(final RemotingChannel channel) {
+        for (ChannelEventListener listener : listenerList) {
+            listener.onChannelIdle(channel);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/RemotingCommandFactoryMeta.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/RemotingCommandFactoryMeta.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/RemotingCommandFactoryMeta.java
new file mode 100644
index 0000000..d5c0aaa
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/RemotingCommandFactoryMeta.java
@@ -0,0 +1,49 @@
+/*
+ * 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.rocketmq.remoting.common;
+
+import org.apache.rocketmq.remoting.api.protocol.ProtocolFactory;
+import org.apache.rocketmq.remoting.api.serializable.SerializerFactory;
+import org.apache.rocketmq.remoting.impl.protocol.Httpv2Protocol;
+import org.apache.rocketmq.remoting.impl.protocol.ProtocolFactoryImpl;
+import org.apache.rocketmq.remoting.impl.protocol.serializer.MsgPackSerializer;
+import org.apache.rocketmq.remoting.impl.protocol.serializer.SerializerFactoryImpl;
+
+public class RemotingCommandFactoryMeta {
+    private final ProtocolFactory protocolFactory = new ProtocolFactoryImpl();
+    private final SerializerFactory serializerFactory = new SerializerFactoryImpl();
+    private byte protocolType = Httpv2Protocol.MVP_MAGIC;
+    private byte serializeType = MsgPackSerializer.SERIALIZER_TYPE;
+
+    public RemotingCommandFactoryMeta() {
+    }
+
+    public RemotingCommandFactoryMeta(String protocolName, String serializeName) {
+        this.protocolType = protocolFactory.type(protocolName);
+        this.serializeType = serializerFactory.type(serializeName);
+    }
+
+    public byte getSerializeType() {
+        return serializeType;
+    }
+
+    public byte getProtocolType() {
+        return protocolType;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/ResponseResult.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/ResponseResult.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/ResponseResult.java
new file mode 100644
index 0000000..2557cdf
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/ResponseResult.java
@@ -0,0 +1,190 @@
+/*
+ * 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.rocketmq.remoting.common;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.rocketmq.remoting.api.AsyncHandler;
+import org.apache.rocketmq.remoting.api.RemotingEndPoint;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.remoting.api.interceptor.ExceptionContext;
+import org.apache.rocketmq.remoting.api.interceptor.InterceptorGroup;
+import org.apache.rocketmq.remoting.api.interceptor.ResponseContext;
+
+public class ResponseResult {
+    private final long beginTimestamp = System.currentTimeMillis();
+    private final CountDownLatch countDownLatch = new CountDownLatch(1);
+    private final AtomicBoolean interceptorExecuted = new AtomicBoolean(false);
+
+    private int requestId;
+    private long timeoutMillis;
+    private AsyncHandler asyncHandler;
+
+    private volatile RemotingCommand responseCommand;
+    private volatile boolean sendRequestOK = true;
+    private volatile Throwable cause;
+    private SemaphoreReleaseOnlyOnce once;
+
+    private RemotingCommand requestCommand;
+    private InterceptorGroup interceptorGroup;
+    private String remoteAddr;
+
+    public ResponseResult(int requestId, long timeoutMillis, AsyncHandler asyncHandler, SemaphoreReleaseOnlyOnce once) {
+        this.requestId = requestId;
+        this.timeoutMillis = timeoutMillis;
+        this.asyncHandler = asyncHandler;
+        this.once = once;
+    }
+
+    public ResponseResult(int requestId, long timeoutMillis) {
+        this.requestId = requestId;
+        this.timeoutMillis = timeoutMillis;
+    }
+
+    public void executeRequestSendFailed() {
+        if (this.interceptorExecuted.compareAndSet(false, true)) {
+            try {
+                interceptorGroup.onException(new ExceptionContext(RemotingEndPoint.REQUEST, this.remoteAddr, this.requestCommand,
+                    cause, "REQUEST_SEND_FAILED"));
+            } catch (Throwable e) {
+            }
+            //Sync call
+            if (null != asyncHandler) {
+                asyncHandler.onFailure(requestCommand);
+            }
+        }
+    }
+
+    public void executeCallbackArrived(final RemotingCommand response) {
+        if (this.interceptorExecuted.compareAndSet(false, true)) {
+            try {
+                interceptorGroup.afterResponseReceived(new ResponseContext(RemotingEndPoint.REQUEST, this.remoteAddr,
+                    this.requestCommand, response));
+            } catch (Throwable e) {
+            }
+            if (null != asyncHandler) {
+                asyncHandler.onSuccess(response);
+            }
+        }
+    }
+
+    public void onTimeout(long costTimeMillis, long timoutMillis) {
+        if (this.interceptorExecuted.compareAndSet(false, true)) {
+            try {
+                interceptorGroup.onException(new ExceptionContext(RemotingEndPoint.REQUEST, this.remoteAddr, this.requestCommand,
+                    null, "CALLBACK_TIMEOUT"));
+            } catch (Throwable e) {
+            }
+            if (null != asyncHandler) {
+                asyncHandler.onTimeout(costTimeMillis, timoutMillis);
+            }
+        }
+    }
+
+    public void release() {
+        if (this.once != null) {
+            this.once.release();
+        }
+    }
+
+    public RemotingCommand waitResponse(final long timeoutMillis) {
+        try {
+            this.countDownLatch.await(timeoutMillis, TimeUnit.MILLISECONDS);
+        } catch (InterruptedException e) {
+            e.printStackTrace();
+        }
+        return this.responseCommand;
+    }
+
+    public void putResponse(final RemotingCommand responseCommand) {
+        this.responseCommand = responseCommand;
+        this.countDownLatch.countDown();
+    }
+
+    public long getBeginTimestamp() {
+        return beginTimestamp;
+    }
+
+    public boolean isSendRequestOK() {
+        return sendRequestOK;
+    }
+
+    public void setSendRequestOK(boolean sendRequestOK) {
+        this.sendRequestOK = sendRequestOK;
+    }
+
+    public long getTimeoutMillis() {
+        return timeoutMillis;
+    }
+
+    public AsyncHandler getAsyncHandler() {
+        return asyncHandler;
+    }
+
+    public Throwable getCause() {
+        return cause;
+    }
+
+    public void setCause(Throwable cause) {
+        this.cause = cause;
+    }
+
+    public RemotingCommand getResponseCommand() {
+        return responseCommand;
+    }
+
+    public void setResponseCommand(RemotingCommand responseCommand) {
+        this.responseCommand = responseCommand;
+    }
+
+    public int getRequestId() {
+        return requestId;
+    }
+
+    public RemotingCommand getRequestCommand() {
+        return requestCommand;
+    }
+
+    public void setRequestCommand(RemotingCommand requestCommand) {
+        this.requestCommand = requestCommand;
+    }
+
+    public InterceptorGroup getInterceptorGroup() {
+        return interceptorGroup;
+    }
+
+    public void setInterceptorGroup(InterceptorGroup interceptorGroup) {
+        this.interceptorGroup = interceptorGroup;
+    }
+
+    public String getRemoteAddr() {
+        return remoteAddr;
+    }
+
+    public void setRemoteAddr(String remoteAddr) {
+        this.remoteAddr = remoteAddr;
+    }
+
+    @Override
+    public String toString() {
+        return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/SemaphoreReleaseOnlyOnce.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/SemaphoreReleaseOnlyOnce.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/SemaphoreReleaseOnlyOnce.java
new file mode 100644
index 0000000..1c5849b
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/SemaphoreReleaseOnlyOnce.java
@@ -0,0 +1,40 @@
+/*
+ * 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.rocketmq.remoting.common;
+
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class SemaphoreReleaseOnlyOnce {
+    private final AtomicBoolean released = new AtomicBoolean(false);
+    private final Semaphore semaphore;
+
+    public SemaphoreReleaseOnlyOnce(Semaphore semaphore) {
+        this.semaphore = semaphore;
+    }
+
+    public void release() {
+        if (this.released.compareAndSet(false, true)) {
+            this.semaphore.release();
+        }
+    }
+
+    public Semaphore getSemaphore() {
+        return semaphore;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/metrics/ChannelMetrics.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/metrics/ChannelMetrics.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/metrics/ChannelMetrics.java
new file mode 100755
index 0000000..db959b7
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/common/metrics/ChannelMetrics.java
@@ -0,0 +1,27 @@
+/*
+ * 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.rocketmq.remoting.common.metrics;
+
+import io.netty.channel.group.ChannelGroup;
+
+public interface ChannelMetrics {
+
+    Integer getChannelCount();
+
+    ChannelGroup getChannels();
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/config/RemotingConfig.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/config/RemotingConfig.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/config/RemotingConfig.java
new file mode 100644
index 0000000..b330041
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/config/RemotingConfig.java
@@ -0,0 +1,375 @@
+/*
+ * 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.rocketmq.remoting.config;
+
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.rocketmq.remoting.api.protocol.Protocol;
+import org.apache.rocketmq.remoting.impl.protocol.compression.GZipCompressor;
+import org.apache.rocketmq.remoting.impl.protocol.serializer.MsgPackSerializer;
+
+public class RemotingConfig extends TcpSocketConfig {
+    private int connectionMaxRetries = 3;
+    private int connectionChannelReaderIdleSeconds = 0;
+    private int connectionChannelWriterIdleSeconds = 0;
+    /**
+     * IdleStateEvent will be triggered when neither read nor write was
+     * performed for the specified period of this time. Specify {@code 0} to
+     * disable
+     */
+    private int connectionChannelIdleSeconds = 120;
+    private int writeBufLowWaterMark = 32 * 10240;
+    private int writeBufHighWaterMark = 64 * 10240;
+    private int threadTaskLowWaterMark = 30000;
+    private int threadTaskHighWaterMark = 50000;
+    private int connectionRetryBackoffMillis = 3000;
+    private String protocolName = Protocol.MVP;
+    private String serializerName = MsgPackSerializer.SERIALIZER_NAME;
+    private String compressorName = GZipCompressor.COMPRESSOR_NAME;
+    private int serviceThreadBlockQueueSize = 50000;
+    private boolean clientNativeEpollEnable = false;
+    private int clientWorkerThreads = 16 + Runtime.getRuntime().availableProcessors() * 2;
+    private int clientConnectionFutureAwaitTimeoutMillis = 30000;
+    private int clientAsyncCallbackExecutorThreads = 16 + Runtime.getRuntime().availableProcessors() * 2;
+    private int clientOnewayInvokeSemaphore = 20480;
+
+    //=============Server configuration==================
+    private int clientAsyncInvokeSemaphore = 20480;
+    private boolean clientPooledBytebufAllocatorEnable = false;
+    private boolean clientCloseSocketIfTimeout = true;
+    private boolean clientShortConnectionEnable = false;
+    private long clientPublishServiceTimeout = 10000;
+    private long clientConsumerServiceTimeout = 10000;
+    private long clientInvokeServiceTimeout = 10000;
+    private int clientMaxRetryCount = 10;
+    private int clientSleepBeforeRetry = 100;
+    private int serverListenPort = 8888;
+    /**
+     * If server only listened 1 port,recommend to set the value to 1
+     */
+    private int serverAcceptorThreads = 1;
+    private int serverIoThreads = 16 + Runtime.getRuntime().availableProcessors() * 2;
+    private int serverWorkerThreads = 16 + Runtime.getRuntime().availableProcessors() * 2;
+    private int serverOnewayInvokeSemaphore = 256;
+    private int serverAsyncInvokeSemaphore = 6400;
+    private boolean serverNativeEpollEnable = false;
+    private int serverAsyncCallbackExecutorThreads = Runtime.getRuntime().availableProcessors() * 2;
+    private boolean serverPooledBytebufAllocatorEnable = true;
+    private boolean serverAuthOpenEnable = true;
+
+    @Override
+    public String toString() {
+        return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE);
+    }
+
+    public int getConnectionMaxRetries() {
+        return connectionMaxRetries;
+    }
+
+    public void setConnectionMaxRetries(final int connectionMaxRetries) {
+        this.connectionMaxRetries = connectionMaxRetries;
+    }
+
+    public int getConnectionChannelReaderIdleSeconds() {
+        return connectionChannelReaderIdleSeconds;
+    }
+
+    public void setConnectionChannelReaderIdleSeconds(final int connectionChannelReaderIdleSeconds) {
+        this.connectionChannelReaderIdleSeconds = connectionChannelReaderIdleSeconds;
+    }
+
+    public int getConnectionChannelWriterIdleSeconds() {
+        return connectionChannelWriterIdleSeconds;
+    }
+
+    public void setConnectionChannelWriterIdleSeconds(final int connectionChannelWriterIdleSeconds) {
+        this.connectionChannelWriterIdleSeconds = connectionChannelWriterIdleSeconds;
+    }
+
+    public int getConnectionChannelIdleSeconds() {
+        return connectionChannelIdleSeconds;
+    }
+
+    public void setConnectionChannelIdleSeconds(final int connectionChannelIdleSeconds) {
+        this.connectionChannelIdleSeconds = connectionChannelIdleSeconds;
+    }
+
+    public int getWriteBufLowWaterMark() {
+        return writeBufLowWaterMark;
+    }
+
+    public void setWriteBufLowWaterMark(final int writeBufLowWaterMark) {
+        this.writeBufLowWaterMark = writeBufLowWaterMark;
+    }
+
+    public int getWriteBufHighWaterMark() {
+        return writeBufHighWaterMark;
+    }
+
+    public void setWriteBufHighWaterMark(final int writeBufHighWaterMark) {
+        this.writeBufHighWaterMark = writeBufHighWaterMark;
+    }
+
+    public int getThreadTaskLowWaterMark() {
+        return threadTaskLowWaterMark;
+    }
+
+    public void setThreadTaskLowWaterMark(final int threadTaskLowWaterMark) {
+        this.threadTaskLowWaterMark = threadTaskLowWaterMark;
+    }
+
+    public int getThreadTaskHighWaterMark() {
+        return threadTaskHighWaterMark;
+    }
+
+    public void setThreadTaskHighWaterMark(final int threadTaskHighWaterMark) {
+        this.threadTaskHighWaterMark = threadTaskHighWaterMark;
+    }
+
+    public int getConnectionRetryBackoffMillis() {
+        return connectionRetryBackoffMillis;
+    }
+
+    public void setConnectionRetryBackoffMillis(final int connectionRetryBackoffMillis) {
+        this.connectionRetryBackoffMillis = connectionRetryBackoffMillis;
+    }
+
+    public String getProtocolName() {
+        return protocolName;
+    }
+
+    public void setProtocolName(final String protocolName) {
+        this.protocolName = protocolName;
+    }
+
+    public String getSerializerName() {
+        return serializerName;
+    }
+
+    public void setSerializerName(final String serializerName) {
+        this.serializerName = serializerName;
+    }
+
+    public String getCompressorName() {
+        return compressorName;
+    }
+
+    public void setCompressorName(final String compressorName) {
+        this.compressorName = compressorName;
+    }
+
+    public int getServiceThreadBlockQueueSize() {
+        return serviceThreadBlockQueueSize;
+    }
+
+    public void setServiceThreadBlockQueueSize(final int serviceThreadBlockQueueSize) {
+        this.serviceThreadBlockQueueSize = serviceThreadBlockQueueSize;
+    }
+
+    public boolean isClientNativeEpollEnable() {
+        return clientNativeEpollEnable;
+    }
+
+    public void setClientNativeEpollEnable(final boolean clientNativeEpollEnable) {
+        this.clientNativeEpollEnable = clientNativeEpollEnable;
+    }
+
+    public int getClientWorkerThreads() {
+        return clientWorkerThreads;
+    }
+
+    public void setClientWorkerThreads(final int clientWorkerThreads) {
+        this.clientWorkerThreads = clientWorkerThreads;
+    }
+
+    public int getClientConnectionFutureAwaitTimeoutMillis() {
+        return clientConnectionFutureAwaitTimeoutMillis;
+    }
+
+    public void setClientConnectionFutureAwaitTimeoutMillis(final int clientConnectionFutureAwaitTimeoutMillis) {
+        this.clientConnectionFutureAwaitTimeoutMillis = clientConnectionFutureAwaitTimeoutMillis;
+    }
+
+    public int getClientAsyncCallbackExecutorThreads() {
+        return clientAsyncCallbackExecutorThreads;
+    }
+
+    public void setClientAsyncCallbackExecutorThreads(final int clientAsyncCallbackExecutorThreads) {
+        this.clientAsyncCallbackExecutorThreads = clientAsyncCallbackExecutorThreads;
+    }
+
+    public int getClientOnewayInvokeSemaphore() {
+        return clientOnewayInvokeSemaphore;
+    }
+
+    public void setClientOnewayInvokeSemaphore(final int clientOnewayInvokeSemaphore) {
+        this.clientOnewayInvokeSemaphore = clientOnewayInvokeSemaphore;
+    }
+
+    public int getClientAsyncInvokeSemaphore() {
+        return clientAsyncInvokeSemaphore;
+    }
+
+    public void setClientAsyncInvokeSemaphore(final int clientAsyncInvokeSemaphore) {
+        this.clientAsyncInvokeSemaphore = clientAsyncInvokeSemaphore;
+    }
+
+    public boolean isClientPooledBytebufAllocatorEnable() {
+        return clientPooledBytebufAllocatorEnable;
+    }
+
+    public void setClientPooledBytebufAllocatorEnable(final boolean clientPooledBytebufAllocatorEnable) {
+        this.clientPooledBytebufAllocatorEnable = clientPooledBytebufAllocatorEnable;
+    }
+
+    public boolean isClientCloseSocketIfTimeout() {
+        return clientCloseSocketIfTimeout;
+    }
+
+    public void setClientCloseSocketIfTimeout(final boolean clientCloseSocketIfTimeout) {
+        this.clientCloseSocketIfTimeout = clientCloseSocketIfTimeout;
+    }
+
+    public boolean isClientShortConnectionEnable() {
+        return clientShortConnectionEnable;
+    }
+
+    public void setClientShortConnectionEnable(final boolean clientShortConnectionEnable) {
+        this.clientShortConnectionEnable = clientShortConnectionEnable;
+    }
+
+    public long getClientPublishServiceTimeout() {
+        return clientPublishServiceTimeout;
+    }
+
+    public void setClientPublishServiceTimeout(final long clientPublishServiceTimeout) {
+        this.clientPublishServiceTimeout = clientPublishServiceTimeout;
+    }
+
+    public long getClientConsumerServiceTimeout() {
+        return clientConsumerServiceTimeout;
+    }
+
+    public void setClientConsumerServiceTimeout(final long clientConsumerServiceTimeout) {
+        this.clientConsumerServiceTimeout = clientConsumerServiceTimeout;
+    }
+
+    public long getClientInvokeServiceTimeout() {
+        return clientInvokeServiceTimeout;
+    }
+
+    public void setClientInvokeServiceTimeout(final long clientInvokeServiceTimeout) {
+        this.clientInvokeServiceTimeout = clientInvokeServiceTimeout;
+    }
+
+    public int getClientMaxRetryCount() {
+        return clientMaxRetryCount;
+    }
+
+    public void setClientMaxRetryCount(final int clientMaxRetryCount) {
+        this.clientMaxRetryCount = clientMaxRetryCount;
+    }
+
+    public int getClientSleepBeforeRetry() {
+        return clientSleepBeforeRetry;
+    }
+
+    public void setClientSleepBeforeRetry(final int clientSleepBeforeRetry) {
+        this.clientSleepBeforeRetry = clientSleepBeforeRetry;
+    }
+
+    public int getServerListenPort() {
+        return serverListenPort;
+    }
+
+    public void setServerListenPort(final int serverListenPort) {
+        this.serverListenPort = serverListenPort;
+    }
+
+    public int getServerAcceptorThreads() {
+        return serverAcceptorThreads;
+    }
+
+    public void setServerAcceptorThreads(final int serverAcceptorThreads) {
+        this.serverAcceptorThreads = serverAcceptorThreads;
+    }
+
+    public int getServerIoThreads() {
+        return serverIoThreads;
+    }
+
+    public void setServerIoThreads(final int serverIoThreads) {
+        this.serverIoThreads = serverIoThreads;
+    }
+
+    public int getServerWorkerThreads() {
+        return serverWorkerThreads;
+    }
+
+    public void setServerWorkerThreads(final int serverWorkerThreads) {
+        this.serverWorkerThreads = serverWorkerThreads;
+    }
+
+    public int getServerOnewayInvokeSemaphore() {
+        return serverOnewayInvokeSemaphore;
+    }
+
+    public void setServerOnewayInvokeSemaphore(final int serverOnewayInvokeSemaphore) {
+        this.serverOnewayInvokeSemaphore = serverOnewayInvokeSemaphore;
+    }
+
+    public int getServerAsyncInvokeSemaphore() {
+        return serverAsyncInvokeSemaphore;
+    }
+
+    public void setServerAsyncInvokeSemaphore(final int serverAsyncInvokeSemaphore) {
+        this.serverAsyncInvokeSemaphore = serverAsyncInvokeSemaphore;
+    }
+
+    public boolean isServerNativeEpollEnable() {
+        return serverNativeEpollEnable;
+    }
+
+    public void setServerNativeEpollEnable(final boolean serverNativeEpollEnable) {
+        this.serverNativeEpollEnable = serverNativeEpollEnable;
+    }
+
+    public int getServerAsyncCallbackExecutorThreads() {
+        return serverAsyncCallbackExecutorThreads;
+    }
+
+    public void setServerAsyncCallbackExecutorThreads(final int serverAsyncCallbackExecutorThreads) {
+        this.serverAsyncCallbackExecutorThreads = serverAsyncCallbackExecutorThreads;
+    }
+
+    public boolean isServerPooledBytebufAllocatorEnable() {
+        return serverPooledBytebufAllocatorEnable;
+    }
+
+    public void setServerPooledBytebufAllocatorEnable(final boolean serverPooledBytebufAllocatorEnable) {
+        this.serverPooledBytebufAllocatorEnable = serverPooledBytebufAllocatorEnable;
+    }
+
+    public boolean isServerAuthOpenEnable() {
+        return serverAuthOpenEnable;
+    }
+
+    public void setServerAuthOpenEnable(final boolean serverAuthOpenEnable) {
+        this.serverAuthOpenEnable = serverAuthOpenEnable;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/config/TcpSocketConfig.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/config/TcpSocketConfig.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/config/TcpSocketConfig.java
new file mode 100755
index 0000000..4dfcde7
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/config/TcpSocketConfig.java
@@ -0,0 +1,98 @@
+/*
+ * 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.rocketmq.remoting.config;
+
+/**
+ * TCP socket configuration
+ *
+ * @see java.net.SocketOptions
+ */
+public class TcpSocketConfig {
+    private boolean tcpSoReuseAddress;
+    private boolean tcpSoKeepAlive;
+    private boolean tcpSoNoDelay;
+    private int tcpSoSndBufSize;  // see /proc/sys/net/ipv4/tcp_rmem
+    private int tcpSoRcvBufSize;  // see /proc/sys/net/ipv4/tcp_wmem
+    private int tcpSoBacklogSize;
+    private int tcpSoLinger;
+    private int tcpSoTimeout;
+
+    public boolean isTcpSoReuseAddress() {
+        return tcpSoReuseAddress;
+    }
+
+    public void setTcpSoReuseAddress(final boolean tcpSoReuseAddress) {
+        this.tcpSoReuseAddress = tcpSoReuseAddress;
+    }
+
+    public boolean isTcpSoKeepAlive() {
+        return tcpSoKeepAlive;
+    }
+
+    public void setTcpSoKeepAlive(final boolean tcpSoKeepAlive) {
+        this.tcpSoKeepAlive = tcpSoKeepAlive;
+    }
+
+    public boolean isTcpSoNoDelay() {
+        return tcpSoNoDelay;
+    }
+
+    public void setTcpSoNoDelay(final boolean tcpSoNoDelay) {
+        this.tcpSoNoDelay = tcpSoNoDelay;
+    }
+
+    public int getTcpSoSndBufSize() {
+        return tcpSoSndBufSize;
+    }
+
+    public void setTcpSoSndBufSize(final int tcpSoSndBufSize) {
+        this.tcpSoSndBufSize = tcpSoSndBufSize;
+    }
+
+    public int getTcpSoRcvBufSize() {
+        return tcpSoRcvBufSize;
+    }
+
+    public void setTcpSoRcvBufSize(final int tcpSoRcvBufSize) {
+        this.tcpSoRcvBufSize = tcpSoRcvBufSize;
+    }
+
+    public int getTcpSoBacklogSize() {
+        return tcpSoBacklogSize;
+    }
+
+    public void setTcpSoBacklogSize(final int tcpSoBacklogSize) {
+        this.tcpSoBacklogSize = tcpSoBacklogSize;
+    }
+
+    public int getTcpSoLinger() {
+        return tcpSoLinger;
+    }
+
+    public void setTcpSoLinger(final int tcpSoLinger) {
+        this.tcpSoLinger = tcpSoLinger;
+    }
+
+    public int getTcpSoTimeout() {
+        return tcpSoTimeout;
+    }
+
+    public void setTcpSoTimeout(final int tcpSoTimeout) {
+        this.tcpSoTimeout = tcpSoTimeout;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/external/ThreadUtils.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/external/ThreadUtils.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/external/ThreadUtils.java
new file mode 100644
index 0000000..1a80d20
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/external/ThreadUtils.java
@@ -0,0 +1,177 @@
+/*
+ * 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.rocketmq.remoting.external;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public final class ThreadUtils {
+    private static final Logger LOG = LoggerFactory.getLogger(ThreadUtils.class);
+
+    /**
+     * A constructor to stop this class being constructed.
+     */
+    private ThreadUtils() {
+        // Unused
+
+    }
+
+    public static ExecutorService newThreadPoolExecutor(int corePoolSize,
+        int maximumPoolSize,
+        long keepAliveTime,
+        TimeUnit unit,
+        BlockingQueue<Runnable> workQueue, String processName, boolean isDaemon) {
+        return new ThreadPoolExecutor(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, newThreadFactory(processName, isDaemon));
+    }
+
+    public static ExecutorService newSingleThreadExecutor(String processName, boolean isDaemon) {
+        return Executors.newSingleThreadExecutor(newThreadFactory(processName, isDaemon));
+    }
+
+    public static ScheduledExecutorService newSingleThreadScheduledExecutor(String processName, boolean isDaemon) {
+        return Executors.newSingleThreadScheduledExecutor(newThreadFactory(processName, isDaemon));
+    }
+
+    public static ScheduledExecutorService newFixedThreadScheduledPool(int nThreads, String processName,
+        boolean isDaemon) {
+        return Executors.newScheduledThreadPool(nThreads, newThreadFactory(processName, isDaemon));
+    }
+
+    public static ThreadFactory newThreadFactory(String processName, boolean isDaemon) {
+        return newGenericThreadFactory("Remoting-" + processName, isDaemon);
+    }
+
+    public static ThreadFactory newGenericThreadFactory(String processName) {
+        return newGenericThreadFactory(processName, false);
+    }
+
+    public static ThreadFactory newGenericThreadFactory(String processName, int threads) {
+        return newGenericThreadFactory(processName, threads, false);
+    }
+
+    public static ThreadFactory newGenericThreadFactory(final String processName, final boolean isDaemon) {
+        return new ThreadFactory() {
+            private AtomicInteger threadIndex = new AtomicInteger(0);
+
+            @Override
+            public Thread newThread(Runnable r) {
+                Thread thread = new Thread(r, String.format("%s_%d", processName, this.threadIndex.incrementAndGet()));
+                thread.setDaemon(isDaemon);
+                return thread;
+            }
+        };
+    }
+
+    public static ThreadFactory newGenericThreadFactory(final String processName, final int threads,
+        final boolean isDaemon) {
+        return new ThreadFactory() {
+            private AtomicInteger threadIndex = new AtomicInteger(0);
+
+            @Override
+            public Thread newThread(Runnable r) {
+                Thread thread = new Thread(r, String.format("%s_%d_%d", processName, threads, this.threadIndex.incrementAndGet()));
+                thread.setDaemon(isDaemon);
+                return thread;
+            }
+        };
+    }
+
+    /**
+     * Create a new thread
+     *
+     * @param name The name of the thread
+     * @param runnable The work for the thread to do
+     * @param daemon Should the thread block JVM stop?
+     * @return The unstarted thread
+     */
+    public static Thread newThread(String name, Runnable runnable, boolean daemon) {
+        Thread thread = new Thread(runnable, name);
+        thread.setDaemon(daemon);
+        thread.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
+            public void uncaughtException(Thread t, Throwable e) {
+                LOG.error("Uncaught exception in thread '" + t.getName() + "':", e);
+            }
+        });
+        return thread;
+    }
+
+    /**
+     * Shutdown passed thread using isAlive and join.
+     *
+     * @param t Thread to stop
+     */
+    public static void shutdownGracefully(final Thread t) {
+        shutdownGracefully(t, 0);
+    }
+
+    /**
+     * Shutdown passed thread using isAlive and join.
+     *
+     * @param millis Pass 0 if we're to wait forever.
+     * @param t Thread to stop
+     */
+    public static void shutdownGracefully(final Thread t, final long millis) {
+        if (t == null)
+            return;
+        while (t.isAlive()) {
+            try {
+                t.interrupt();
+                t.join(millis);
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            }
+        }
+    }
+
+    /**
+     * An implementation of the graceful stop sequence recommended by
+     * {@link ExecutorService}.
+     *
+     * @param executor executor
+     * @param timeout timeout
+     * @param timeUnit timeUnit
+     */
+    public static void shutdownGracefully(ExecutorService executor, long timeout, TimeUnit timeUnit) {
+        // Disable new tasks from being submitted.
+        executor.shutdown();
+        try {
+            // Wait a while for existing tasks to terminate.
+            if (!executor
+                .awaitTermination(timeout, timeUnit)) {
+                executor.shutdownNow();
+                // Wait a while for tasks to respond to being cancelled.
+                if (!executor.awaitTermination(timeout, timeUnit)) {
+                    LOG.warn(String.format("%s didn't terminate!", executor));
+                }
+            }
+        } catch (InterruptedException ie) {
+            // (Re-)Cancel if current thread also interrupted.
+            executor.shutdownNow();
+            // Preserve interrupt status.
+            Thread.currentThread().interrupt();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/buffer/NettyByteBufferWrapper.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/buffer/NettyByteBufferWrapper.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/buffer/NettyByteBufferWrapper.java
new file mode 100644
index 0000000..e17bcfd
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/buffer/NettyByteBufferWrapper.java
@@ -0,0 +1,114 @@
+/*
+ * 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.rocketmq.remoting.impl.buffer;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.Channel;
+import java.nio.ByteBuffer;
+import org.apache.rocketmq.remoting.api.buffer.ByteBufferWrapper;
+
+public class NettyByteBufferWrapper implements ByteBufferWrapper {
+    private final ByteBuf buffer;
+    private final Channel channel;
+
+    public NettyByteBufferWrapper(ByteBuf buffer) {
+        this(buffer, null);
+    }
+
+    public NettyByteBufferWrapper(ByteBuf buffer, Channel channel) {
+        this.channel = channel;
+        this.buffer = buffer;
+    }
+
+    public void writeByte(int index, byte data) {
+        buffer.writeByte(data);
+    }
+
+    public void writeByte(byte data) {
+        buffer.writeByte(data);
+    }
+
+    public byte readByte() {
+        return buffer.readByte();
+    }
+
+    public void writeInt(int data) {
+        buffer.writeInt(data);
+    }
+
+    public void writeBytes(byte[] data) {
+        buffer.writeBytes(data);
+    }
+
+    @Override
+    public void writeBytes(final ByteBuffer data) {
+        buffer.writeBytes(data);
+    }
+
+    public int readableBytes() {
+        return buffer.readableBytes();
+    }
+
+    public int readInt() {
+        return buffer.readInt();
+    }
+
+    public void readBytes(byte[] dst) {
+        buffer.readBytes(dst);
+    }
+
+    @Override
+    public void readBytes(final ByteBuffer dst) {
+        buffer.readBytes(dst);
+    }
+
+    public int readerIndex() {
+        return buffer.readerIndex();
+    }
+
+    public void setReaderIndex(int index) {
+        buffer.setIndex(index, buffer.writerIndex());
+    }
+
+    @Override
+    public void writeLong(long value) {
+        buffer.writeLong(value);
+    }
+
+    @Override
+    public long readLong() {
+        return buffer.readLong();
+    }
+
+    @Override
+    public void ensureCapacity(int capacity) {
+        buffer.capacity(capacity);
+    }
+
+    @Override
+    public short readShort() {
+        return buffer.readShort();
+    }
+
+    @Override
+    public void writeShort(final short value) {
+        buffer.writeShort(value);
+    }
+}
+
+



[8/8] incubator-rocketmq git commit: initialize RocketMQ5

Posted by yu...@apache.org.
initialize RocketMQ5


Project: http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/commit/0b88e66f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/tree/0b88e66f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/diff/0b88e66f

Branch: refs/heads/rocketmq5
Commit: 0b88e66fa8c8121d35a133e277d9e40093e91b06
Parents: 
Author: yukon <yu...@apache.org>
Authored: Tue Sep 19 15:22:16 2017 +0800
Committer: yukon <yu...@apache.org>
Committed: Tue Sep 19 15:22:16 2017 +0800

----------------------------------------------------------------------
 .gitignore                                      |  10 +
 CONTRIBUTING.md                                 |  15 +
 README.md                                       |  51 ++
 example/pom.xml                                 |  39 ++
 .../rpc/example/model/TradeRequest.java         |  47 ++
 .../rpc/example/model/TradeResponse.java        |  38 ++
 .../rpc/example/model/TradeServiceAPI.java      |  47 ++
 .../rpc/example/model/TradeServiceAPIGen.java   |  60 ++
 .../rpc/example/model/TradeServiceAPIImpl.java  |  68 ++
 .../rocketmq/rpc/example/quickstart/Client.java |  35 +
 .../rocketmq/rpc/example/quickstart/Server.java |  30 +
 pom.xml                                         | 166 +++++
 remoting-core/pom.xml                           | 146 +++++
 remoting-core/remoting-api/pom.xml              |  41 ++
 .../rocketmq/remoting/api/AsyncHandler.java     |  34 +
 .../remoting/api/ConnectionService.java         |  24 +
 .../rocketmq/remoting/api/ObjectLifecycle.java  |  29 +
 .../rocketmq/remoting/api/RemotingClient.java   |  28 +
 .../rocketmq/remoting/api/RemotingEndPoint.java |  23 +
 .../remoting/api/RemotingMarshaller.java        |  27 +
 .../rocketmq/remoting/api/RemotingServer.java   |  32 +
 .../rocketmq/remoting/api/RemotingService.java  |  40 ++
 .../rocketmq/remoting/api/RequestProcessor.java |  25 +
 .../remoting/api/buffer/ByteBufferWrapper.java  |  56 ++
 .../api/channel/ChannelEventListener.java       |  28 +
 .../channel/ChannelHandlerContextWrapper.java   |  21 +
 .../remoting/api/channel/ChunkRegion.java       |  54 ++
 .../remoting/api/channel/RemotingChannel.java   |  78 +++
 .../remoting/api/command/RemotingCommand.java   |  90 +++
 .../api/command/RemotingCommandFactory.java     |  24 +
 .../remoting/api/command/TrafficType.java       |  40 ++
 .../remoting/api/compressable/Compressor.java   |  28 +
 .../api/compressable/CompressorFactory.java     |  28 +
 .../api/exception/NestedRuntimeException.java   |  97 +++
 .../api/exception/RemoteAccessException.java    |  57 ++
 .../api/exception/RemoteCodecException.java     |  33 +
 .../RemoteConnectFailureException.java          |  49 ++
 .../api/exception/RemoteTimeoutException.java   |  70 ++
 .../api/interceptor/ExceptionContext.java       |  76 +++
 .../remoting/api/interceptor/Interceptor.java   |  26 +
 .../api/interceptor/InterceptorGroup.java       |  49 ++
 .../api/interceptor/RequestContext.java         |  65 ++
 .../api/interceptor/ResponseContext.java        |  73 +++
 .../remoting/api/protocol/Protocol.java         |  39 ++
 .../remoting/api/protocol/ProtocolFactory.java  |  30 +
 .../remoting/api/serializable/Serializer.java   |  36 ++
 .../api/serializable/SerializerFactory.java     |  28 +
 .../apache/rocketmq/remoting/common/Pair.java   |  44 ++
 .../remoting/common/TypePresentation.java       |  73 +++
 remoting-core/remoting-impl/pom.xml             |  71 ++
 .../common/ChannelEventListenerGroup.java       |  61 ++
 .../common/RemotingCommandFactoryMeta.java      |  49 ++
 .../remoting/common/ResponseResult.java         | 190 ++++++
 .../common/SemaphoreReleaseOnlyOnce.java        |  40 ++
 .../remoting/common/metrics/ChannelMetrics.java |  27 +
 .../remoting/config/RemotingConfig.java         | 375 +++++++++++
 .../remoting/config/TcpSocketConfig.java        |  98 +++
 .../rocketmq/remoting/external/ThreadUtils.java | 177 +++++
 .../impl/buffer/NettyByteBufferWrapper.java     | 114 ++++
 .../ChannelHandlerContextWrapperImpl.java       |  33 +
 .../remoting/impl/channel/FileRegionImpl.java   |  85 +++
 .../remoting/impl/channel/NettyChannelImpl.java |  94 +++
 .../remoting/impl/command/CodecHelper.java      | 180 ++++++
 .../command/RemotingCommandFactoryImpl.java     |  53 ++
 .../impl/command/RemotingCommandImpl.java       | 210 ++++++
 .../impl/command/RequestIdGenerator.java        |  34 +
 .../remoting/impl/netty/NettyChannelEvent.java  |  55 ++
 .../impl/netty/NettyChannelEventType.java       |  25 +
 .../impl/netty/NettyRemotingAbstract.java       | 641 +++++++++++++++++++
 .../impl/netty/NettyRemotingClient.java         | 499 +++++++++++++++
 .../impl/netty/NettyRemotingServer.java         | 286 +++++++++
 .../impl/netty/RemotingBootstrapFactory.java    |  60 ++
 .../impl/netty/handler/ChannelStatistics.java   |  61 ++
 .../remoting/impl/netty/handler/Decoder.java    | 107 ++++
 .../remoting/impl/netty/handler/Encoder.java    |  89 +++
 .../impl/netty/handler/ExceptionHandler.java    |  37 ++
 .../impl/netty/handler/Http2Handler.java        | 139 ++++
 .../impl/netty/handler/ProtocolSelector.java    |  65 ++
 .../remoting/impl/protocol/Httpv2Protocol.java  |  53 ++
 .../impl/protocol/ProtocolFactoryImpl.java      |  83 +++
 .../impl/protocol/RemotingCoreProtocol.java     |  46 ++
 .../impl/protocol/WebSocketProtocol.java        |  38 ++
 .../compression/CompressorFactoryImpl.java      |  68 ++
 .../protocol/compression/GZipCompressor.java    | 100 +++
 .../protocol/serializer/JsonSerializer.java     |  88 +++
 .../protocol/serializer/Kryo3Serializer.java    |  90 +++
 .../protocol/serializer/MsgPackSerializer.java  |  78 +++
 .../serializer/SerializerFactoryImpl.java       |  69 ++
 .../protocol/serializer/ThreadSafeKryo.java     |  99 +++
 .../rocketmq/remoting/internal/BeanUtils.java   | 210 ++++++
 .../rocketmq/remoting/internal/ByteUtils.java   | 379 +++++++++++
 .../remoting/internal/ExceptionUtils.java       |  80 +++
 .../rocketmq/remoting/internal/JvmUtils.java    |  94 +++
 .../remoting/internal/NetworkUtils.java         |  81 +++
 .../remoting/internal/PropertyUtils.java        |  60 ++
 .../remoting/internal/UIDGenerator.java         |  97 +++
 .../apache/rocketmq/remoting/package-info.java  |  49 ++
 .../impl/netty/handler/DecoderTest.java         | 163 +++++
 .../protocol/serializer/SerializersTest.java    | 263 ++++++++
 .../remoting/internal/BeanUtilsTest.java        | 134 ++++
 .../remoting/internal/ExceptionUtilsTest.java   |  43 ++
 .../remoting/internal/JvmUtilsTest.java         |  32 +
 remoting-core/rpc-api/pom.xml                   |  38 ++
 .../rocketmq/rpc/annotation/MethodType.java     |  35 +
 .../rocketmq/rpc/annotation/RemoteMethod.java   |  53 ++
 .../rocketmq/rpc/annotation/RemoteService.java  |  48 ++
 .../rpc/annotation/VisibleForInternal.java      |  32 +
 .../apache/rocketmq/rpc/api/AdvancedClient.java |  36 ++
 .../apache/rocketmq/rpc/api/AdvancedServer.java |  38 ++
 .../org/apache/rocketmq/rpc/api/Promise.java    | 114 ++++
 .../rocketmq/rpc/api/PromiseListener.java       |  41 ++
 .../apache/rocketmq/rpc/api/SimpleClient.java   |  32 +
 .../apache/rocketmq/rpc/api/SimpleServer.java   |  33 +
 remoting-core/rpc-impl/pom.xml                  |  45 ++
 .../rocketmq/rpc/RpcBootstrapFactory.java       |  36 ++
 .../rpc/impl/client/AdvancedClientImpl.java     |  57 ++
 .../rpc/impl/client/SimpleClientImpl.java       | 132 ++++
 .../rocketmq/rpc/impl/command/ResponseCode.java | 118 ++++
 .../rpc/impl/command/RpcRequestCode.java        |  22 +
 .../rpc/impl/config/RpcCommonConfig.java        |  63 ++
 .../rpc/impl/context/RpcCallerContext.java      |  48 ++
 .../rpc/impl/context/RpcProviderContext.java    |  67 ++
 .../impl/exception/ServiceAuthException.java    |  68 ++
 .../exception/ServiceExceptionHandlerCode.java  |  56 ++
 .../ServiceExceptionHandlerManager.java         | 133 ++++
 .../ServiceExceptionInvokeMessage.java          |  50 ++
 .../impl/exception/ServiceExceptionManager.java |  32 +
 .../impl/exception/ServiceRuntimeException.java |  87 +++
 .../exception/ServiceSignatureException.java    |  47 ++
 .../rpc/impl/metrics/DefaultServiceAPI.java     |  42 ++
 .../rpc/impl/metrics/DefaultServiceAPIImpl.java |  59 ++
 .../rpc/impl/metrics/DownloadFileRequest.java   |  56 ++
 .../rpc/impl/metrics/ExecuteResult.java         |  57 ++
 .../rocketmq/rpc/impl/metrics/MethodStats.java  |  92 +++
 .../rocketmq/rpc/impl/metrics/ServiceStats.java | 125 ++++
 .../rpc/impl/metrics/StackTracesAll.java        |  38 ++
 .../rocketmq/rpc/impl/metrics/StatsAll.java     |  60 ++
 .../rocketmq/rpc/impl/metrics/StatsItem.java    | 442 +++++++++++++
 .../rocketmq/rpc/impl/metrics/StatsItemSet.java | 193 ++++++
 .../rocketmq/rpc/impl/metrics/ThreadStats.java  |  55 ++
 .../rocketmq/rpc/impl/metrics/Threading.java    |  87 +++
 .../rpc/impl/metrics/TimestampRegion.java       |  65 ++
 .../rocketmq/rpc/impl/metrics/UtilAll.java      |  84 +++
 .../rpc/impl/processor/RpcRequestProcessor.java | 213 ++++++
 .../rpc/impl/promise/DefaultPromise.java        | 228 +++++++
 .../rocketmq/rpc/impl/promise/FutureState.java  |  51 ++
 .../rpc/impl/server/AdvancedServerImpl.java     |  57 ++
 .../rpc/impl/server/SimpleServerImpl.java       | 104 +++
 .../rpc/impl/service/RpcConnectionListener.java |  49 ++
 .../rocketmq/rpc/impl/service/RpcEntry.java     |  51 ++
 .../rpc/impl/service/RpcInstanceAbstract.java   |  88 +++
 .../rocketmq/rpc/impl/service/RpcJdkProxy.java  |  54 ++
 .../rpc/impl/service/RpcJdkProxyClient.java     |  50 ++
 .../rpc/impl/service/RpcJdkProxyServer.java     |  51 ++
 .../rpc/impl/service/RpcProxyCommon.java        | 329 ++++++++++
 .../rpc/impl/service/RpcProxyFactory.java       |  41 ++
 .../rpc/impl/service/RpcServiceCallBody.java    |  81 +++
 .../rpc/internal/ExceptionMessageUtil.java      |  24 +
 .../rocketmq/rpc/internal/HttpTinyClient.java   | 130 ++++
 .../rocketmq/rpc/internal/IOTinyUtils.java      | 157 +++++
 .../rocketmq/rpc/internal/RpcErrorMapper.java   |  67 ++
 .../rocketmq/rpc/internal/ServiceUtil.java      | 218 +++++++
 style/checkstyle.xml                            | 140 ++++
 style/codeStyle.xml                             | 143 +++++
 style/copyright/Apache.xml                      |  24 +
 style/copyright/profiles_settings.xml           |  64 ++
 166 files changed, 14229 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
new file mode 100644
index 0000000..f12ac58
--- /dev/null
+++ b/.gitignore
@@ -0,0 +1,10 @@
+# Created by .ignore support plugin (hsz.mobi)
+.classpath
+.project
+.settings/
+target/
+*.log*
+*.iml
+.idea/
+*.versionsBackup
+*.DS_Store

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/CONTRIBUTING.md
----------------------------------------------------------------------
diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md
new file mode 100644
index 0000000..3a0a87f
--- /dev/null
+++ b/CONTRIBUTING.md
@@ -0,0 +1,15 @@
+## How To Contribute
+
+We are always very happy to have contributions, whether for trivial cleanups or big new features. We want to have high quality, well documented codes for each programming language. Nor is code the only way to contribute to the project. We strongly value documentation and gladly accept improvements to the documentation.
+Contributing code
+
+To submit a change for inclusion, please do the following:
+If the change is non-trivial please include some unit tests that cover the new functionality.
+
+If you are introducing a completely new feature or API it is a good idea to start a wiki and get consensus on the basic design first.
+
+It is our job to follow up on patches in a timely fashion. Nag us if we aren't doing our job (sometimes we drop things).
+
+## Becoming a Committer
+
+We are always interested in adding new contributors. What we look for are series of contributions, good taste and ongoing interest in the project. If you are interested in becoming a committer, please let one of the existing committers know and they can help you walk through the process.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
new file mode 100644
index 0000000..d356b6b
--- /dev/null
+++ b/README.md
@@ -0,0 +1,51 @@
+## Angelia
+Angelia provides a single API for most network related service that uses pluggable transports and codecs. The Angelia API provides the ability for making synchronous, asynchronous, oneway remote calls, push and pull callbacks. The intention is to allow for the use of different transports to fit different needs, yet still maintain the same API for making the remote invocations and only requiring configuration changes, not code changes.
+
+Angelia is a standalone project, separate from the Alibaba RocketMQ and Jukola project, but will be the framework used for many of the projects and components when making remote calls. Angelia is included in the recent releases of the Alibaba Jukola and can be run as a service within the container as well.
+
+### Features
+The features available with Angelia are:
+
+#### 1. Pluggable transports – can use different protocol transports the same remoting API.
+Provided transports:
+
+##### **MVP, a custom-build Minimum Viable Protocol**
+##### **HTTP2**
+     
+#### 2. Pluggable codecs – can use different codecs to convert the invocation payloads into desired data format for wire transfer.
+
+#### 3. Pluggable serialization - can use different serialization implementations for data streams.
+
+Provided serialization implementations:
+
+##### **MessagePack**
+##### **Kryo**
+##### **Fastjson**
+
+#### 4. Data Compression - can use compression codec for compresssion of large payloads.
+
+All the features within Angelia were created with ease of use and extensibility in mind. If you have a suggestion for a new feature or an improvement to a current feature, please let me know.
+
+
+
+
+
+
+
+
+## Documentation
+### [Quick Start]()
+### [User Guide]()
+### [Developer Guide]()
+### [Release Note]()
+
+
+
+----------
+
+## Contributing
+We are always very happy to have contributions, whether for trivial cleanups,big new features or other material rewards. more details see [here](CONTRIBUTING.md) 
+ 
+----------
+## License
+[Apache License, Version 2.0](http://www.apache.org/licenses/LICENSE-2.0.html) Copyright (C) 2010-2016 Alibaba Group Holding Limited
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/example/pom.xml
----------------------------------------------------------------------
diff --git a/example/pom.xml b/example/pom.xml
new file mode 100644
index 0000000..0bf832e
--- /dev/null
+++ b/example/pom.xml
@@ -0,0 +1,39 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.rocketmq</groupId>
+        <artifactId>rpc-all</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>example</artifactId>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rpc-impl</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
+
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeRequest.java
----------------------------------------------------------------------
diff --git a/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeRequest.java b/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeRequest.java
new file mode 100644
index 0000000..f84afcd
--- /dev/null
+++ b/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeRequest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.rocketmq.rpc.example.model;
+
+import org.msgpack.annotation.Message;
+
+@Message
+public class TradeRequest {
+    private long orderId;
+    private String orderDesc;
+
+    public long getOrderId() {
+        return orderId;
+    }
+
+    public void setOrderId(long orderId) {
+        this.orderId = orderId;
+    }
+
+    public String getOrderDesc() {
+        return orderDesc;
+    }
+
+    public void setOrderDesc(String orderDesc) {
+        this.orderDesc = orderDesc;
+    }
+
+    @Override
+    public String toString() {
+        return "TradeRequest [orderId=" + orderId + ", orderDesc=" + orderDesc + "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeResponse.java
----------------------------------------------------------------------
diff --git a/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeResponse.java b/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeResponse.java
new file mode 100644
index 0000000..1c9358c
--- /dev/null
+++ b/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeResponse.java
@@ -0,0 +1,38 @@
+/*
+ * 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.rocketmq.rpc.example.model;
+
+import org.msgpack.annotation.Message;
+
+@Message
+public class TradeResponse {
+    private long storeTimestamp;
+
+    public long getStoreTimestamp() {
+        return storeTimestamp;
+    }
+
+    public void setStoreTimestamp(long storeTimestamp) {
+        this.storeTimestamp = storeTimestamp;
+    }
+
+    @Override
+    public String toString() {
+        return "TradeResponse [storeTimestamp=" + storeTimestamp + "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeServiceAPI.java
----------------------------------------------------------------------
diff --git a/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeServiceAPI.java b/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeServiceAPI.java
new file mode 100644
index 0000000..6014558
--- /dev/null
+++ b/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeServiceAPI.java
@@ -0,0 +1,47 @@
+/*
+ * 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.rocketmq.rpc.example.model;
+
+import java.util.ArrayList;
+import org.apache.rocketmq.rpc.annotation.RemoteMethod;
+import org.apache.rocketmq.rpc.annotation.RemoteService;
+
+@RemoteService(name = "TradeServiceAPI")
+public interface TradeServiceAPI {
+    @RemoteMethod(name = "commitOrder")
+    TradeResponse commitOrder(TradeRequest request) throws InterruptedException;
+
+    @RemoteMethod(name = "deleteOrder")
+    void deleteOrder(TradeRequest request) throws InterruptedException;
+
+    @RemoteMethod(name = "throwUserException")
+    void throwUserException(TradeRequest request) throws Exception;
+
+    @RemoteMethod(name = "throwRuntimeException")
+    void throwRuntimeException(TradeRequest request);
+
+    @RemoteMethod(name = "getOrderList")
+    ArrayList<TradeResponse> getOrderList(TradeRequest request, int count);
+
+    @RemoteMethod(name = "getOrderListSize")
+    int getOrderListSize(ArrayList<TradeRequest> request, String obj);
+
+    @RemoteMethod(name = "setOrder")
+    String setOrder();
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeServiceAPIGen.java
----------------------------------------------------------------------
diff --git a/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeServiceAPIGen.java b/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeServiceAPIGen.java
new file mode 100644
index 0000000..b7aa788
--- /dev/null
+++ b/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeServiceAPIGen.java
@@ -0,0 +1,60 @@
+/*
+ * 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.rocketmq.rpc.example.model;
+
+import java.util.ArrayList;
+import org.apache.rocketmq.rpc.annotation.MethodType;
+import org.apache.rocketmq.rpc.annotation.RemoteMethod;
+import org.apache.rocketmq.rpc.annotation.RemoteService;
+import org.apache.rocketmq.rpc.api.Promise;
+
+@RemoteService(name = "TradeServiceAPI")
+public interface TradeServiceAPIGen extends TradeServiceAPI {
+    @RemoteMethod(name = "commitOrder", type = MethodType.ASYNC)
+    Promise<TradeResponse> commitOrderAsync(TradeRequest request);
+
+    @RemoteMethod(name = "commitOrder", type = MethodType.ONEWAY)
+    void commitOrderOneway(final TradeRequest request);
+
+    @RemoteMethod(name = "deleteOrder", type = MethodType.ASYNC)
+    Promise<TradeResponse> deleteOrderAsync(TradeRequest request);
+
+    @RemoteMethod(name = "deleteOrder", type = MethodType.ONEWAY)
+    void deleteOrderOneway(TradeRequest request);
+
+    @RemoteMethod(name = "throwUserException", type = MethodType.ASYNC)
+    void throwUserExceptionAsync(TradeRequest request);
+
+    @RemoteMethod(name = "throwUserException", type = MethodType.ONEWAY)
+    void throwUserExceptionOneway(TradeRequest request);
+
+    @RemoteMethod(name = "throwRuntimeException", type = MethodType.ASYNC)
+    void throwRuntimeExceptionAsync(TradeRequest request);
+
+    @RemoteMethod(name = "throwRuntimeException", type = MethodType.ONEWAY)
+    void throwRuntimeExceptionOneway(TradeRequest request);
+
+    @RemoteMethod(name = "getOrderList", type = MethodType.ASYNC)
+    Promise<ArrayList<TradeResponse>> getOrderListAsync(TradeRequest request, int count);
+
+    @RemoteMethod(name = "getOrderListSize", type = MethodType.ASYNC)
+    Promise<Integer> getOrderListSizeAsync(ArrayList<TradeRequest> request, String obj);
+
+    @RemoteMethod(name = "setOrder", type = MethodType.ASYNC)
+    Promise<String> setOrderAsync();
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeServiceAPIImpl.java
----------------------------------------------------------------------
diff --git a/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeServiceAPIImpl.java b/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeServiceAPIImpl.java
new file mode 100644
index 0000000..cd196f4
--- /dev/null
+++ b/example/src/main/java/org/apache/rocketmq/rpc/example/model/TradeServiceAPIImpl.java
@@ -0,0 +1,68 @@
+/*
+ * 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.rocketmq.rpc.example.model;
+
+import java.util.ArrayList;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class TradeServiceAPIImpl implements TradeServiceAPI {
+
+    private final AtomicLong count = new AtomicLong();
+
+    public TradeServiceAPIImpl() {
+        super();
+
+    }
+
+    @Override
+    public TradeResponse commitOrder(TradeRequest request) throws InterruptedException {
+        TradeResponse response = new TradeResponse();
+        response.setStoreTimestamp(System.currentTimeMillis());
+        return response;
+    }
+
+    @Override
+    public void deleteOrder(TradeRequest request) throws InterruptedException {
+        System.out.println("deleteOrder: " + request);
+    }
+
+    @Override
+    public void throwUserException(TradeRequest request) throws Exception {
+        throw new Exception("User Exception detail message");
+    }
+
+    @Override
+    public void throwRuntimeException(TradeRequest request) {
+        throw new RuntimeException("runtime");
+    }
+
+    @Override
+    public ArrayList<TradeResponse> getOrderList(final TradeRequest request, final int count) {
+        return null;
+    }
+
+    @Override
+    public int getOrderListSize(final ArrayList<TradeRequest> request, final String obj) {
+        return 0;
+    }
+
+    @Override
+    public String setOrder() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/example/src/main/java/org/apache/rocketmq/rpc/example/quickstart/Client.java
----------------------------------------------------------------------
diff --git a/example/src/main/java/org/apache/rocketmq/rpc/example/quickstart/Client.java b/example/src/main/java/org/apache/rocketmq/rpc/example/quickstart/Client.java
new file mode 100644
index 0000000..a8c0745
--- /dev/null
+++ b/example/src/main/java/org/apache/rocketmq/rpc/example/quickstart/Client.java
@@ -0,0 +1,35 @@
+/*
+ * 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.rocketmq.rpc.example.quickstart;
+
+import java.util.Properties;
+import org.apache.rocketmq.rpc.RpcBootstrapFactory;
+import org.apache.rocketmq.rpc.api.SimpleClient;
+import org.apache.rocketmq.rpc.example.model.TradeRequest;
+import org.apache.rocketmq.rpc.example.model.TradeResponse;
+import org.apache.rocketmq.rpc.example.model.TradeServiceAPI;
+
+public class Client {
+    public static void main(String[] args) throws InterruptedException {
+        SimpleClient client = RpcBootstrapFactory.createClientBootstrap(new Properties());
+        TradeServiceAPI tradeService = client.bind(TradeServiceAPI.class, "127.0.0.1:8888", new Properties());
+        client.start();
+
+        TradeResponse tradeResponse = tradeService.commitOrder(new TradeRequest());
+        System.out.println(tradeResponse);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/example/src/main/java/org/apache/rocketmq/rpc/example/quickstart/Server.java
----------------------------------------------------------------------
diff --git a/example/src/main/java/org/apache/rocketmq/rpc/example/quickstart/Server.java b/example/src/main/java/org/apache/rocketmq/rpc/example/quickstart/Server.java
new file mode 100644
index 0000000..3a1aad2
--- /dev/null
+++ b/example/src/main/java/org/apache/rocketmq/rpc/example/quickstart/Server.java
@@ -0,0 +1,30 @@
+/*
+ * 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.rocketmq.rpc.example.quickstart;
+
+import java.util.Properties;
+import org.apache.rocketmq.rpc.RpcBootstrapFactory;
+import org.apache.rocketmq.rpc.api.SimpleServer;
+import org.apache.rocketmq.rpc.example.model.TradeServiceAPIImpl;
+
+public class Server {
+    public static void main(String[] args) {
+        SimpleServer server = RpcBootstrapFactory.createServerBootstrap(new Properties());
+        server.publish(new TradeServiceAPIImpl());
+        server.start();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
new file mode 100644
index 0000000..9a24623
--- /dev/null
+++ b/pom.xml
@@ -0,0 +1,166 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+    <inceptionYear>2012</inceptionYear>
+    <groupId>org.apache.rocketmq</groupId>
+    <artifactId>rpc-all</artifactId>
+    <version>2.0.0-SNAPSHOT</version>
+    <packaging>pom</packaging>
+
+    <modules>
+        <module>remoting-core</module>
+        <module>example</module>
+    </modules>
+
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <maven.test.skip>false</maven.test.skip>
+        <maven.javadoc.skip>true</maven.javadoc.skip>
+        <!-- compiler settings properties -->
+        <maven.compiler.source>1.7</maven.compiler.source>
+        <maven.compiler.target>1.7</maven.compiler.target>
+        <metrics.version>3.1.2</metrics.version>
+    </properties>
+
+    <build>
+
+        <plugins>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>versions-maven-plugin</artifactId>
+                <version>2.2</version>
+            </plugin>
+            <plugin>
+                <artifactId>maven-enforcer-plugin</artifactId>
+                <version>1.4.1</version>
+                <executions>
+                    <execution>
+                        <id>enforce-ban-circular-dependencies</id>
+                        <goals>
+                            <goal>enforce</goal>
+                        </goals>
+                    </execution>
+                </executions>
+                <configuration>
+                    <rules>
+                        <banCircularDependencies/>
+                    </rules>
+                    <fail>true</fail>
+                </configuration>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>extra-enforcer-rules</artifactId>
+                        <version>1.0-beta-4</version>
+                    </dependency>
+                </dependencies>
+            </plugin>
+            <plugin>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <version>3.5.1</version>
+                <configuration>
+                    <source>${maven.compiler.source}</source>
+                    <target>${maven.compiler.target}</target>
+                    <compilerVersion>${maven.compiler.source}</compilerVersion>
+                    <showDeprecation>true</showDeprecation>
+                    <showWarnings>true</showWarnings>
+                </configuration>
+            </plugin>
+            <plugin>
+                <artifactId>maven-javadoc-plugin</artifactId>
+                <version>2.10.4</version>
+                <executions>
+                    <execution>
+                        <id>attach-javadocs</id>
+                        <goals>
+                            <goal>jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <artifactId>maven-source-plugin</artifactId>
+                <version>3.0.1</version>
+                <executions>
+                    <execution>
+                        <id>attach-sources</id>
+                        <goals>
+                            <goal>jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <artifactId>maven-checkstyle-plugin</artifactId>
+                <version>2.17</version>
+                <executions>
+                    <execution>
+                        <id>verify</id>
+                        <phase>verify</phase>
+                        <configuration>
+                            <configLocation>style/checkstyle.xml</configLocation>
+                            <encoding>${project.build.sourceEncoding}</encoding>
+                            <consoleOutput>true</consoleOutput>
+                            <failsOnError>true</failsOnError>
+                        </configuration>
+                        <goals>
+                            <goal>check</goal>
+                        </goals>
+                    </execution>
+                </executions>
+                <configuration>
+                    <sourceDirectory>${project.build.sourceDirectory}</sourceDirectory>
+                </configuration>
+            </plugin>
+        </plugins>
+
+        <pluginManagement>
+            <plugins>
+                <plugin>
+                    <groupId>org.codehaus.mojo</groupId>
+                    <artifactId>clirr-maven-plugin</artifactId>
+                    <version>2.7</version>
+                </plugin>
+            </plugins>
+        </pluginManagement>
+
+        <resources>
+            <resource>
+                <directory>src/main/resources</directory>
+                <filtering>false</filtering>
+            </resource>
+        </resources>
+    </build>
+
+    <distributionManagement>
+        <repository>
+            <id>releases</id>
+            <url>http://mvnrepo.alibaba-inc.com/mvn/releases</url>
+        </repository>
+        <snapshotRepository>
+            <id>snapshots</id>
+            <url>http://mvnrepo.alibaba-inc.com/mvn/snapshots</url>
+        </snapshotRepository>
+    </distributionManagement>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/pom.xml
----------------------------------------------------------------------
diff --git a/remoting-core/pom.xml b/remoting-core/pom.xml
new file mode 100644
index 0000000..702b826
--- /dev/null
+++ b/remoting-core/pom.xml
@@ -0,0 +1,146 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.rocketmq</groupId>
+        <artifactId>rpc-all</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>remoting-core</artifactId>
+    <packaging>pom</packaging>
+
+    <modules>
+        <module>remoting-api</module>
+        <module>remoting-impl</module>
+        <module>rpc-api</module>
+        <module>rpc-impl</module>
+    </modules>
+
+    <dependencies>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>4.11</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.assertj</groupId>
+            <artifactId>assertj-core</artifactId>
+            <version>2.6.0</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <version>2.6.3</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+    <dependencyManagement>
+        <dependencies>
+            <dependency>
+                <groupId>org.slf4j</groupId>
+                <artifactId>slf4j-api</artifactId>
+                <version>1.7.6</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.commons</groupId>
+                <artifactId>commons-lang3</artifactId>
+                <version>3.4</version>
+            </dependency>
+            <dependency>
+                <groupId>org.jetbrains</groupId>
+                <artifactId>annotations</artifactId>
+                <version>15.0</version>
+            </dependency>
+            <dependency>
+                <groupId>io.netty</groupId>
+                <artifactId>netty-all</artifactId>
+                <version>4.1.6.Final</version>
+            </dependency>
+            <dependency>
+                <groupId>com.alibaba</groupId>
+                <artifactId>fastjson</artifactId>
+                <version>1.2.29</version>
+            </dependency>
+            <dependency>
+                <groupId>org.msgpack</groupId>
+                <artifactId>msgpack</artifactId>
+                <version>0.6.12</version>
+            </dependency>
+            <dependency>
+                <groupId>com.esotericsoftware</groupId>
+                <artifactId>kryo</artifactId>
+                <version>3.0.3</version>
+            </dependency>
+            <dependency>
+                <groupId>org.javassist</groupId>
+                <artifactId>javassist</artifactId>
+                <version>3.20.0-GA</version>
+            </dependency>
+            <!-- http://netty.io/wiki/requirements-for-4.x.html#wiki-h3-8 -->
+            <dependency>
+                <groupId>io.netty</groupId>
+                <artifactId>netty-tcnative-boringssl-static</artifactId>
+                <version>1.1.33.Fork22</version>
+            </dependency>
+            <dependency>
+                <groupId>io.dropwizard.metrics</groupId>
+                <artifactId>metrics-core</artifactId>
+                <version>${metrics.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>io.dropwizard.metrics</groupId>
+                <artifactId>metrics-healthchecks</artifactId>
+                <version>${metrics.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>com.google.guava</groupId>
+                <artifactId>guava</artifactId>
+                <version>19.0</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.rocketmq</groupId>
+                <artifactId>remoting-api</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.rocketmq</groupId>
+                <artifactId>rpc-api</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.rocketmq</groupId>
+                <artifactId>remoting-impl</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.rocketmq</groupId>
+                <artifactId>rpc-impl</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+        </dependencies>
+    </dependencyManagement>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/pom.xml
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/pom.xml b/remoting-core/remoting-api/pom.xml
new file mode 100644
index 0000000..c0b87da
--- /dev/null
+++ b/remoting-core/remoting-api/pom.xml
@@ -0,0 +1,41 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.rocketmq</groupId>
+        <artifactId>remoting-core</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>remoting-api</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.jetbrains</groupId>
+            <artifactId>annotations</artifactId>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/AsyncHandler.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/AsyncHandler.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/AsyncHandler.java
new file mode 100644
index 0000000..106431b
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/AsyncHandler.java
@@ -0,0 +1,34 @@
+/*
+ * 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.rocketmq.remoting.api;
+
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+
+/**
+ * The AsyncHandler interface is implemented when wishing to receive callback notification of the completion of
+ * service invoked asynchronously.
+ *
+ * @since 1.0.0
+ */
+public interface AsyncHandler {
+    void onFailure(RemotingCommand command);
+
+    void onSuccess(RemotingCommand command);
+
+    void onTimeout(long costTimeMillis, long timeoutMillis);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/ConnectionService.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/ConnectionService.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/ConnectionService.java
new file mode 100644
index 0000000..c42498f
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/ConnectionService.java
@@ -0,0 +1,24 @@
+/*
+ * 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.rocketmq.remoting.api;
+
+import org.apache.rocketmq.remoting.api.channel.ChannelEventListener;
+
+public interface ConnectionService {
+    void registerChannelEventListener(ChannelEventListener listener);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/ObjectLifecycle.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/ObjectLifecycle.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/ObjectLifecycle.java
new file mode 100644
index 0000000..c4a75c8
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/ObjectLifecycle.java
@@ -0,0 +1,29 @@
+/*
+ * 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.rocketmq.remoting.api;
+
+import javax.annotation.PostConstruct;
+import javax.annotation.PreDestroy;
+
+public interface ObjectLifecycle {
+    @PostConstruct
+    void start();
+
+    @PreDestroy
+    void stop();
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingClient.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingClient.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingClient.java
new file mode 100644
index 0000000..1603af4
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingClient.java
@@ -0,0 +1,28 @@
+/*
+ * 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.rocketmq.remoting.api;
+
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+
+public interface RemotingClient extends RemotingService {
+    RemotingCommand invoke(String address, RemotingCommand request, long timeoutMillis);
+
+    void invokeAsync(String address, RemotingCommand request, AsyncHandler asyncHandler, long timeoutMillis);
+
+    void invokeOneWay(String address, RemotingCommand request, long timeoutMillis);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingEndPoint.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingEndPoint.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingEndPoint.java
new file mode 100644
index 0000000..2bc3edf
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingEndPoint.java
@@ -0,0 +1,23 @@
+/*
+ * 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.rocketmq.remoting.api;
+
+public enum RemotingEndPoint {
+    REQUEST,
+    RESPONSE
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingMarshaller.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingMarshaller.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingMarshaller.java
new file mode 100644
index 0000000..0386a03
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingMarshaller.java
@@ -0,0 +1,27 @@
+/*
+ * 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.rocketmq.remoting.api;
+
+import org.apache.rocketmq.remoting.api.protocol.ProtocolFactory;
+import org.apache.rocketmq.remoting.api.serializable.SerializerFactory;
+
+public interface RemotingMarshaller {
+    ProtocolFactory protocolFactory();
+
+    SerializerFactory serializerFactory();
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingServer.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingServer.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingServer.java
new file mode 100644
index 0000000..f36c83c
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingServer.java
@@ -0,0 +1,32 @@
+/*
+ * 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.rocketmq.remoting.api;
+
+import org.apache.rocketmq.remoting.api.channel.RemotingChannel;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+
+public interface RemotingServer extends RemotingService {
+    int localListenPort();
+
+    RemotingCommand invoke(RemotingChannel remotingChannel, RemotingCommand request, long timeoutMillis);
+
+    void invokeAsync(RemotingChannel remotingChannel, RemotingCommand request, AsyncHandler asyncHandler,
+        long timeoutMillis);
+
+    void invokeOneWay(RemotingChannel remotingChannel, RemotingCommand request, long timeoutMillis);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingService.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingService.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingService.java
new file mode 100644
index 0000000..2f6343c
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RemotingService.java
@@ -0,0 +1,40 @@
+/*
+ * 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.rocketmq.remoting.api;
+
+import java.util.concurrent.ExecutorService;
+import org.apache.rocketmq.remoting.api.command.RemotingCommandFactory;
+import org.apache.rocketmq.remoting.api.interceptor.Interceptor;
+import org.apache.rocketmq.remoting.common.Pair;
+
+public interface RemotingService extends RemotingMarshaller, ConnectionService, ObjectLifecycle {
+    void registerInterceptor(Interceptor interceptor);
+
+    void registerRequestProcessor(final String requestCode, final RequestProcessor processor,
+        final ExecutorService executor);
+
+    void registerRequestProcessor(final String requestCode, final RequestProcessor processor);
+
+    void unregisterRequestProcessor(final String requestCode);
+
+    Pair<RequestProcessor, ExecutorService> processor(final String requestCode);
+
+    String remotingInstanceId();
+
+    RemotingCommandFactory commandFactory();
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RequestProcessor.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RequestProcessor.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RequestProcessor.java
new file mode 100644
index 0000000..cd201c2
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/RequestProcessor.java
@@ -0,0 +1,25 @@
+/*
+ * 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.rocketmq.remoting.api;
+
+import org.apache.rocketmq.remoting.api.channel.RemotingChannel;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+
+public interface RequestProcessor {
+    RemotingCommand processRequest(RemotingChannel channel, RemotingCommand request);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/buffer/ByteBufferWrapper.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/buffer/ByteBufferWrapper.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/buffer/ByteBufferWrapper.java
new file mode 100644
index 0000000..7cae3ac
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/buffer/ByteBufferWrapper.java
@@ -0,0 +1,56 @@
+/*
+ * 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.rocketmq.remoting.api.buffer;
+
+import java.nio.ByteBuffer;
+
+public interface ByteBufferWrapper {
+    void writeByte(int index, byte data);
+
+    void writeByte(byte data);
+
+    byte readByte();
+
+    void writeInt(int data);
+
+    void writeBytes(byte[] data);
+
+    void writeBytes(ByteBuffer data);
+
+    int readableBytes();
+
+    int readInt();
+
+    void readBytes(byte[] dst);
+
+    void readBytes(ByteBuffer dst);
+
+    int readerIndex();
+
+    void setReaderIndex(int readerIndex);
+
+    void writeLong(long id);
+
+    long readLong();
+
+    void ensureCapacity(int capacity);
+
+    short readShort();
+
+    void writeShort(short value);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/channel/ChannelEventListener.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/channel/ChannelEventListener.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/channel/ChannelEventListener.java
new file mode 100644
index 0000000..0c0afcf
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/channel/ChannelEventListener.java
@@ -0,0 +1,28 @@
+/*
+ * 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.rocketmq.remoting.api.channel;
+
+public interface ChannelEventListener {
+    void onChannelConnect(final RemotingChannel channel);
+
+    void onChannelClose(final RemotingChannel channel);
+
+    void onChannelException(final RemotingChannel channel);
+
+    void onChannelIdle(final RemotingChannel channel);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/channel/ChannelHandlerContextWrapper.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/channel/ChannelHandlerContextWrapper.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/channel/ChannelHandlerContextWrapper.java
new file mode 100644
index 0000000..05c3b18
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/channel/ChannelHandlerContextWrapper.java
@@ -0,0 +1,21 @@
+/*
+ * 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.rocketmq.remoting.api.channel;
+
+public interface ChannelHandlerContextWrapper<T> {
+    T getContext();
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/channel/ChunkRegion.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/channel/ChunkRegion.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/channel/ChunkRegion.java
new file mode 100644
index 0000000..8266ff0
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/channel/ChunkRegion.java
@@ -0,0 +1,54 @@
+/*
+ * 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.rocketmq.remoting.api.channel;
+
+import java.io.IOException;
+import java.nio.channels.WritableByteChannel;
+
+public interface ChunkRegion {
+    void release();
+
+    /**
+     * @return Returns the offset in the file where the transfer began.
+     */
+    long position();
+
+    /**
+     * @return Return the bytes which was transferred already
+     */
+    long transferred();
+
+    /**
+     * @return Returns the number of bytes to transfer.
+     */
+    long count();
+
+    /**
+     * Transfers the content of this file region to the specified channel.
+     *
+     * @param target the destination of the transfer
+     * @param position the relative offset of the file where the transfer begins
+     * from. For example, <tt>0</tt> will make the transfer start
+     * from {@link #position()}th byte and
+     * <tt>{@link #count()} - 1</tt> will make the last byte of the
+     * region transferred.
+     * @return the length of the transferred file region
+     * @throws IOException IOException
+     */
+    long transferTo(WritableByteChannel target, long position) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/channel/RemotingChannel.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/channel/RemotingChannel.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/channel/RemotingChannel.java
new file mode 100644
index 0000000..28cfc53
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/channel/RemotingChannel.java
@@ -0,0 +1,78 @@
+/*
+ * 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.rocketmq.remoting.api.channel;
+
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+
+public interface RemotingChannel {
+    /**
+     * Returns the local address where this {@code RemotingChannel} is bound to.  The returned
+     * {@link SocketAddress} is supposed to be down-cast into more concrete
+     * type such as {@link InetSocketAddress} to retrieve the detailed
+     * information.
+     *
+     * @return the local address of this channel.
+     * {@code null} if this channel is not bound.
+     */
+    SocketAddress localAddress();
+
+    /**
+     * Returns the remote address where this {@code RemotingChannel} is connected to.  The
+     * returned {@link SocketAddress} is supposed to be down-cast into more
+     * concrete type such as {@link InetSocketAddress} to retrieve the detailed
+     * information.
+     *
+     * @return the remote address of this channel.
+     * {@code null} if this channel is not connected.
+     */
+    SocketAddress remoteAddress();
+
+    /**
+     * Returns {@code true} if and only if the I/O thread will perform the
+     * requested write operation immediately.  Any write requests made when
+     * this method returns {@code false} are queued until the I/O thread is
+     * ready to process the queued write requests.
+     */
+    boolean isWritable();
+
+    /**
+     * Returns {@code true} if the {@code RemotingChannel} is active and so connected.
+     */
+    boolean isActive();
+
+    /**
+     * Requests to close the {@code RemotingChannel} immediately.
+     */
+    void close();
+
+    /**
+     * Writes a response {@code RemotingCommand} to remote.
+     *
+     * @param command the response command
+     */
+    void reply(RemotingCommand command);
+
+    /**
+     * Writes a response {@code ChunkRegion} to remote.
+     *
+     * @param fileRegion the response chunk file region
+     */
+    void reply(ChunkRegion fileRegion);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/command/RemotingCommand.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/command/RemotingCommand.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/command/RemotingCommand.java
new file mode 100644
index 0000000..f21a45d
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/command/RemotingCommand.java
@@ -0,0 +1,90 @@
+/*
+ * 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.rocketmq.remoting.api.command;
+
+import java.lang.reflect.Type;
+import java.util.Map;
+import org.apache.rocketmq.remoting.api.serializable.SerializerFactory;
+import org.apache.rocketmq.remoting.common.TypePresentation;
+
+public interface RemotingCommand {
+    byte protocolType();
+
+    void protocolType(byte value);
+
+    int requestID();
+
+    void requestID(int value);
+
+    byte serializerType();
+
+    void serializerType(byte value);
+
+    TrafficType trafficType();
+
+    void trafficType(TrafficType value);
+
+    String opCode();
+
+    void opCode(String value);
+
+    String remark();
+
+    void remark(String value);
+
+    Map<String, String> properties();
+
+    void properties(Map<String, String> value);
+
+    String property(String key);
+
+    void property(String key, String value);
+
+    Object parameter();
+
+    void parameter(Object value);
+
+    byte[] parameterBytes();
+
+    void parameterBytes(byte[] value);
+
+    byte[] extraPayload();
+
+    void extraPayload(byte[] value);
+
+    <T> T parameter(final SerializerFactory serializerFactory, Class<T> c);
+
+    <T> T parameter(final SerializerFactory serializerFactory, final TypePresentation<T> typePresentation);
+
+    <T> T parameter(final SerializerFactory serializerFactory, final Type type);
+
+    enum CommandFlag {
+        SUCCESS("0"),
+        ERROR("-1");
+
+        private String flag;
+
+        CommandFlag(final String flag) {
+            this.flag = flag;
+        }
+
+        public String flag() {
+            return flag;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/command/RemotingCommandFactory.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/command/RemotingCommandFactory.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/command/RemotingCommandFactory.java
new file mode 100644
index 0000000..82a5c6a
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/command/RemotingCommandFactory.java
@@ -0,0 +1,24 @@
+/*
+ * 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.rocketmq.remoting.api.command;
+
+public interface RemotingCommandFactory {
+    RemotingCommand createRequest();
+
+    RemotingCommand createResponse(RemotingCommand command);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/command/TrafficType.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/command/TrafficType.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/command/TrafficType.java
new file mode 100644
index 0000000..efebfe7
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/command/TrafficType.java
@@ -0,0 +1,40 @@
+/*
+ * 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.rocketmq.remoting.api.command;
+
+public enum TrafficType {
+    REQUEST_SYNC,
+    REQUEST_ASYNC,
+    REQUEST_ONEWAY,
+    RESPONSE;
+
+    public static TrafficType parse(int index) {
+        switch (index) {
+            case 0:
+                return REQUEST_SYNC;
+            case 1:
+                return REQUEST_ASYNC;
+            case 2:
+                return REQUEST_ONEWAY;
+            case 3:
+                return RESPONSE;
+            default:
+                throw new IllegalArgumentException("Not supported " + index);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/compressable/Compressor.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/compressable/Compressor.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/compressable/Compressor.java
new file mode 100644
index 0000000..4688c45
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/compressable/Compressor.java
@@ -0,0 +1,28 @@
+/*
+ * 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.rocketmq.remoting.api.compressable;
+
+public interface Compressor {
+    String name();
+
+    byte type();
+
+    byte[] compress(final byte[] content) throws Exception;
+
+    byte[] deCompress(final byte[] content) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/compressable/CompressorFactory.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/compressable/CompressorFactory.java b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/compressable/CompressorFactory.java
new file mode 100644
index 0000000..2494c78
--- /dev/null
+++ b/remoting-core/remoting-api/src/main/java/org/apache/rocketmq/remoting/api/compressable/CompressorFactory.java
@@ -0,0 +1,28 @@
+/*
+ * 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.rocketmq.remoting.api.compressable;
+
+public interface CompressorFactory {
+    void register(Compressor compressor);
+
+    byte type(String compressionName);
+
+    Compressor get(byte type);
+
+    void clearAll();
+}


[4/8] incubator-rocketmq git commit: initialize RocketMQ5

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/ByteUtils.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/ByteUtils.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/ByteUtils.java
new file mode 100644
index 0000000..c298ce7
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/ByteUtils.java
@@ -0,0 +1,379 @@
+/*
+ * 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.rocketmq.remoting.internal;
+
+/**
+ * Copy from Bouncy Castle Crypto APIs
+ *
+ * This class is a utility class for manipulating byte arrays.
+ */
+public final class ByteUtils {
+
+    private static final char[] HEX_CHARS = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'};
+
+    /**
+     * Default constructor (private)
+     */
+    private ByteUtils() {
+        // empty
+    }
+
+    /**
+     * Compare two byte arrays (perform null checks beforehand).
+     *
+     * @param left the first byte array
+     * @param right the second byte array
+     * @return the result of the comparison
+     */
+    public static boolean equals(byte[] left, byte[] right) {
+        if (left == null) {
+            return right == null;
+        }
+        if (right == null) {
+            return false;
+        }
+
+        if (left.length != right.length) {
+            return false;
+        }
+        boolean result = true;
+        for (int i = left.length - 1; i >= 0; i--) {
+            result &= left[i] == right[i];
+        }
+        return result;
+    }
+
+    /**
+     * Compare two two-dimensional byte arrays. No null checks are performed.
+     *
+     * @param left the first byte array
+     * @param right the second byte array
+     * @return the result of the comparison
+     */
+    public static boolean equals(byte[][] left, byte[][] right) {
+        if (left.length != right.length) {
+            return false;
+        }
+
+        boolean result = true;
+        for (int i = left.length - 1; i >= 0; i--) {
+            result &= ByteUtils.equals(left[i], right[i]);
+        }
+
+        return result;
+    }
+
+    /**
+     * Compare two three-dimensional byte arrays. No null checks are performed.
+     *
+     * @param left the first byte array
+     * @param right the second byte array
+     * @return the result of the comparison
+     */
+    public static boolean equals(byte[][][] left, byte[][][] right) {
+        if (left.length != right.length) {
+            return false;
+        }
+
+        boolean result = true;
+        for (int i = left.length - 1; i >= 0; i--) {
+            if (left[i].length != right[i].length) {
+                return false;
+            }
+            for (int j = left[i].length - 1; j >= 0; j--) {
+                result &= ByteUtils.equals(left[i][j], right[i][j]);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Computes a hashcode based on the contents of a one-dimensional byte array
+     * rather than its identity.
+     *
+     * @param array the array to compute the hashcode of
+     * @return the hashcode
+     */
+    public static int deepHashCode(byte[] array) {
+        int result = 1;
+        for (int i = 0; i < array.length; i++) {
+            result = 31 * result + array[i];
+        }
+        return result;
+    }
+
+    /**
+     * Computes a hashcode based on the contents of a two-dimensional byte array
+     * rather than its identity.
+     *
+     * @param array the array to compute the hashcode of
+     * @return the hashcode
+     */
+    public static int deepHashCode(byte[][] array) {
+        int result = 1;
+        for (int i = 0; i < array.length; i++) {
+            result = 31 * result + deepHashCode(array[i]);
+        }
+        return result;
+    }
+
+    /**
+     * Computes a hashcode based on the contents of a three-dimensional byte
+     * array rather than its identity.
+     *
+     * @param array the array to compute the hashcode of
+     * @return the hashcode
+     */
+    public static int deepHashCode(byte[][][] array) {
+        int result = 1;
+        for (int i = 0; i < array.length; i++) {
+            result = 31 * result + deepHashCode(array[i]);
+        }
+        return result;
+    }
+
+    /**
+     * Return a clone of the given byte array (performs null check beforehand).
+     *
+     * @param array the array to clone
+     * @return the clone of the given array, or <tt>null</tt> if the array is
+     * <tt>null</tt>
+     */
+    public static byte[] clone(byte[] array) {
+        if (array == null) {
+            return null;
+        }
+        byte[] result = new byte[array.length];
+        System.arraycopy(array, 0, result, 0, array.length);
+        return result;
+    }
+
+    /**
+     * Convert a string containing hexadecimal characters to a byte-array.
+     *
+     * @param s a hex string
+     * @return a byte array with the corresponding value
+     */
+    public static byte[] fromHexString(String s) {
+        char[] rawChars = s.toUpperCase().toCharArray();
+
+        int hexChars = 0;
+        for (int i = 0; i < rawChars.length; i++) {
+            if ((rawChars[i] >= '0' && rawChars[i] <= '9')
+                || (rawChars[i] >= 'A' && rawChars[i] <= 'F')) {
+                hexChars++;
+            }
+        }
+
+        byte[] byteString = new byte[(hexChars + 1) >> 1];
+
+        int pos = hexChars & 1;
+
+        for (int i = 0; i < rawChars.length; i++) {
+            if (rawChars[i] >= '0' && rawChars[i] <= '9') {
+                byteString[pos >> 1] <<= 4;
+                byteString[pos >> 1] |= rawChars[i] - '0';
+            } else if (rawChars[i] >= 'A' && rawChars[i] <= 'F') {
+                byteString[pos >> 1] <<= 4;
+                byteString[pos >> 1] |= rawChars[i] - 'A' + 10;
+            } else {
+                continue;
+            }
+            pos++;
+        }
+
+        return byteString;
+    }
+
+    /**
+     * Convert a byte array to the corresponding hexstring.
+     *
+     * @param input the byte array to be converted
+     * @return the corresponding hexstring
+     */
+    public static String toHexString(byte[] input) {
+        String result = "";
+        for (int i = 0; i < input.length; i++) {
+            result += HEX_CHARS[(input[i] >>> 4) & 0x0f];
+            result += HEX_CHARS[(input[i]) & 0x0f];
+        }
+        return result;
+    }
+
+    /**
+     * Convert a byte array to the corresponding hex string.
+     *
+     * @param input the byte array to be converted
+     * @param prefix the prefix to put at the beginning of the hex string
+     * @param seperator a separator string
+     * @return the corresponding hex string
+     */
+    public static String toHexString(byte[] input, String prefix,
+        String seperator) {
+        String result = new String(prefix);
+        for (int i = 0; i < input.length; i++) {
+            result += HEX_CHARS[(input[i] >>> 4) & 0x0f];
+            result += HEX_CHARS[(input[i]) & 0x0f];
+            if (i < input.length - 1) {
+                result += seperator;
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Convert a byte array to the corresponding bit string.
+     *
+     * @param input the byte array to be converted
+     * @return the corresponding bit string
+     */
+    public static String toBinaryString(byte[] input) {
+        String result = "";
+        int i;
+        for (i = 0; i < input.length; i++) {
+            int e = input[i];
+            for (int ii = 0; ii < 8; ii++) {
+                int b = (e >>> ii) & 1;
+                result += b;
+            }
+            if (i != input.length - 1) {
+                result += " ";
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Compute the bitwise XOR of two arrays of bytes. The arrays have to be of
+     * same length. No length checking is performed.
+     *
+     * @param x1 the first array
+     * @param x2 the second array
+     * @return x1 XOR x2
+     */
+    public static byte[] xor(byte[] x1, byte[] x2) {
+        byte[] out = new byte[x1.length];
+
+        for (int i = x1.length - 1; i >= 0; i--) {
+            out[i] = (byte) (x1[i] ^ x2[i]);
+        }
+        return out;
+    }
+
+    /**
+     * Concatenate two byte arrays. No null checks are performed.
+     *
+     * @param x1 the first array
+     * @param x2 the second array
+     * @return (x2||x1) (little-endian order, i.e. x1 is at lower memory
+     * addresses)
+     */
+    public static byte[] concatenate(byte[] x1, byte[] x2) {
+        byte[] result = new byte[x1.length + x2.length];
+
+        System.arraycopy(x1, 0, result, 0, x1.length);
+        System.arraycopy(x2, 0, result, x1.length, x2.length);
+
+        return result;
+    }
+
+    /**
+     * Convert a 2-dimensional byte array into a 1-dimensional byte array by
+     * concatenating all entries.
+     *
+     * @param array a 2-dimensional byte array
+     * @return the concatenated input array
+     */
+    public static byte[] concatenate(byte[][] array) {
+        int rowLength = array[0].length;
+        byte[] result = new byte[array.length * rowLength];
+        int index = 0;
+        for (int i = 0; i < array.length; i++) {
+            System.arraycopy(array[i], 0, result, index, rowLength);
+            index += rowLength;
+        }
+        return result;
+    }
+
+    /**
+     * Split a byte array <tt>input</tt> into two arrays at <tt>index</tt>,
+     * i.e. the first array will have the lower <tt>index</tt> bytes, the
+     * second one the higher <tt>input.length - index</tt> bytes.
+     *
+     * @param input the byte array to be split
+     * @param index the index where the byte array is split
+     * @return the splitted input array as an array of two byte arrays
+     * @throws ArrayIndexOutOfBoundsException if <tt>index</tt> is out of bounds
+     */
+    public static byte[][] split(byte[] input, int index)
+        throws ArrayIndexOutOfBoundsException {
+        if (index > input.length) {
+            throw new ArrayIndexOutOfBoundsException();
+        }
+        byte[][] result = new byte[2][];
+        result[0] = new byte[index];
+        result[1] = new byte[input.length - index];
+        System.arraycopy(input, 0, result[0], 0, index);
+        System.arraycopy(input, index, result[1], 0, input.length - index);
+        return result;
+    }
+
+    /**
+     * Generate a subarray of a given byte array.
+     *
+     * @param input the input byte array
+     * @param start the start index
+     * @param end the end index
+     * @return a subarray of <tt>input</tt>, ranging from <tt>start</tt>
+     * (inclusively) to <tt>end</tt> (exclusively)
+     */
+    public static byte[] subArray(byte[] input, int start, int end) {
+        byte[] result = new byte[end - start];
+        System.arraycopy(input, start, result, 0, end - start);
+        return result;
+    }
+
+    /**
+     * Generate a subarray of a given byte array.
+     *
+     * @param input the input byte array
+     * @param start the start index
+     * @return a subarray of <tt>input</tt>, ranging from <tt>start</tt> to
+     * the end of the array
+     */
+    public static byte[] subArray(byte[] input, int start) {
+        return subArray(input, start, input.length);
+    }
+
+    /**
+     * Rewrite a byte array as a char array
+     *
+     * @param input -
+     * the byte array
+     * @return char array
+     */
+    public static char[] toCharArray(byte[] input) {
+        char[] result = new char[input.length];
+        for (int i = 0; i < input.length; i++) {
+            result[i] = (char) input[i];
+        }
+        return result;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/ExceptionUtils.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/ExceptionUtils.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/ExceptionUtils.java
new file mode 100644
index 0000000..6386ca0
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/ExceptionUtils.java
@@ -0,0 +1,80 @@
+/*
+ * 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.rocketmq.remoting.internal;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.StringTokenizer;
+
+public class ExceptionUtils {
+
+    private static final String LINE_SEPARATOR = System.getProperty("line.separator");
+
+    /**
+     * <p>Gets the stack trace from a Throwable as a String.</p>
+     *
+     * <p>The result of this method vary by JDK version as this method
+     * uses {@link Throwable#printStackTrace(java.io.PrintWriter)}.
+     * On JDK1.3 and earlier, the cause exception will not be shown
+     * unless the specified throwable alters printStackTrace.</p>
+     *
+     * @param throwable the <code>Throwable</code> to be examined
+     * @return the stack trace as generated by the exception's
+     * <code>printStackTrace(PrintWriter)</code> method
+     */
+    public static String getStackTrace(final Throwable throwable) {
+        final StringWriter sw = new StringWriter();
+        final PrintWriter pw = new PrintWriter(sw, true);
+        throwable.printStackTrace(pw);
+        return sw.getBuffer().toString();
+    }
+
+    /**
+     * <p>Produces a <code>List</code> of stack frames - the message
+     * is not included. Only the trace of the specified exception is
+     * returned, any caused by trace is stripped.</p>
+     *
+     * <p>This works in most cases - it will only fail if the exception
+     * message contains a line that starts with:
+     * <code>&quot;&nbsp;&nbsp;&nbsp;at&quot;.</code></p>
+     *
+     * @param t is any throwable
+     * @return List of stack frames
+     */
+    static List<String> getStackFrameList(final Throwable t) {
+        final String stackTrace = getStackTrace(t);
+        final String linebreak = LINE_SEPARATOR;
+        final StringTokenizer frames = new StringTokenizer(stackTrace, linebreak);
+        final List<String> list = new ArrayList<String>();
+        boolean traceStarted = false;
+        while (frames.hasMoreTokens()) {
+            final String token = frames.nextToken();
+            // Determine if the line starts with <whitespace>at
+            final int at = token.indexOf("at");
+            if (at != -1 && token.substring(0, at).trim().isEmpty()) {
+                traceStarted = true;
+                list.add(token);
+            } else if (traceStarted) {
+                break;
+            }
+        }
+        return list;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/JvmUtils.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/JvmUtils.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/JvmUtils.java
new file mode 100644
index 0000000..fb97a14
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/JvmUtils.java
@@ -0,0 +1,94 @@
+/*
+ * 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.rocketmq.remoting.internal;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.util.Random;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public final class JvmUtils {
+    public static final String OS_NAME = System.getProperty("os.name").toLowerCase();
+    private final static Logger LOG = LoggerFactory.getLogger(JvmUtils.class);
+    //public static final String OS_VERSION = System.getProperty("os.version").toLowerCase();
+
+    /**
+     * A constructor to stop this class being constructed.
+     */
+    private JvmUtils() {
+        // Unused
+    }
+
+    public static boolean isWindows() {
+        return OS_NAME.startsWith("win");
+    }
+
+    public static boolean isWindows10() {
+        return OS_NAME.startsWith("win") && OS_NAME.endsWith("10");
+    }
+
+    public static boolean isMacOSX() {
+        return OS_NAME.contains("mac");
+    }
+
+    public static boolean isLinux() {
+        return OS_NAME.startsWith("linux");
+    }
+
+    public static boolean isUnix() {
+        return OS_NAME.contains("nix") ||
+            OS_NAME.contains("nux") ||
+            OS_NAME.contains("aix") ||
+            OS_NAME.contains("bsd") ||
+            OS_NAME.contains("sun") ||
+            OS_NAME.contains("hpux");
+    }
+
+    public static boolean isSolaris() {
+        return OS_NAME.startsWith("sun");
+    }
+
+    public static int getProcessId() {
+        String pid = null;
+        final File self = new File("/proc/self");
+        try {
+            if (self.exists()) {
+                pid = self.getCanonicalFile().getName();
+            }
+        } catch (IOException ignored) {
+            //Ignore it
+        }
+
+        if (pid == null) {
+            pid = ManagementFactory.getRuntimeMXBean().getName().split("@", 0)[0];
+        }
+
+        if (pid == null) {
+            int rpid = new Random().nextInt(1 << 16);
+            LOG.warn("Unable to determine PID, picked a random number {}", rpid);
+
+            return rpid;
+        } else {
+            return Integer.parseInt(pid);
+        }
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/NetworkUtils.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/NetworkUtils.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/NetworkUtils.java
new file mode 100644
index 0000000..0e8ae21
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/NetworkUtils.java
@@ -0,0 +1,81 @@
+/*
+ * 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.rocketmq.remoting.internal;
+
+import java.net.Inet4Address;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.net.SocketException;
+import java.net.UnknownHostException;
+import java.util.Enumeration;
+import org.jetbrains.annotations.NotNull;
+
+public final class NetworkUtils {
+
+    public static final String DEFAULT_LOCAL_ADDRESS = "127.0.0.1";
+    public static final String DEFAULT_LOCAL_HOSTNAME = "localhost";
+
+    /**
+     * A constructor to stop this class being constructed.
+     */
+    private NetworkUtils() {
+        // Unused
+    }
+
+    public static InetAddress getLoopbackAddress() {
+        try {
+            return InetAddress.getByName(null);
+        } catch (UnknownHostException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static boolean isLocalhost(@NotNull String host) {
+        return host.equalsIgnoreCase(DEFAULT_LOCAL_HOSTNAME) || host.equals(DEFAULT_LOCAL_ADDRESS);
+    }
+
+    public static String getLocalHostIp() {
+        try {
+            for (Enumeration<NetworkInterface> ifaces = NetworkInterface.getNetworkInterfaces(); ifaces.hasMoreElements(); ) {
+                NetworkInterface iface = ifaces.nextElement();
+                // Workaround for docker0 bridge
+                if ("docker0".equals(iface.getName()) || !iface.isUp()) {
+                    continue;
+                }
+                InetAddress ia;
+                for (Enumeration<InetAddress> ips = iface.getInetAddresses(); ips.hasMoreElements(); ) {
+                    ia = ips.nextElement();
+                    if (ia instanceof Inet4Address) {
+                        // Check if the address is any local or loop back(127.0.0.1 or ::1)
+                        if (!ia.isLoopbackAddress() && ia.getHostAddress().indexOf(':') == -1) {
+                            if (ia.isSiteLocalAddress()) {
+                                return ia.getHostAddress();
+                            } else if (!ia.isLinkLocalAddress() && !ia.isAnyLocalAddress()
+                                && !ia.isMulticastAddress()) {
+                                return ia.getHostAddress();
+                            }
+                        }
+                    }
+                }
+            }
+        } catch (SocketException e) {
+            throw new RuntimeException("Could not get local host ip", e);
+        }
+        return DEFAULT_LOCAL_ADDRESS;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/PropertyUtils.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/PropertyUtils.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/PropertyUtils.java
new file mode 100644
index 0000000..7e76a7e
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/PropertyUtils.java
@@ -0,0 +1,60 @@
+/*
+ * 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.rocketmq.remoting.internal;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.Properties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public final class PropertyUtils {
+    private static final Logger LOG = LoggerFactory.getLogger(PropertyUtils.class);
+
+    public static String getPropertyIgnoreCase(final Properties properties, final String key) {
+        String value = null;
+        if (properties != null) {
+            for (Map.Entry<Object, Object> next : properties.entrySet()) {
+                if (next.getKey().toString().equalsIgnoreCase(key)) {
+                    return next.getValue().toString();
+                }
+            }
+        }
+        return value;
+    }
+
+    /**
+     * Read a properties file from the given path
+     *
+     * @param filename The path of the file to read
+     * @return Property file instance
+     */
+    public static Properties loadProps(String filename) {
+        Properties props = new Properties();
+        try (InputStream propStream = new FileInputStream(filename)) {
+            props.load(propStream);
+        } catch (IOException e) {
+            LOG.error(String.format("Loading properties from file %s error !", filename), e);
+            System.exit(1);
+        }
+        return props;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/UIDGenerator.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/UIDGenerator.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/UIDGenerator.java
new file mode 100644
index 0000000..a4b1293
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/UIDGenerator.java
@@ -0,0 +1,97 @@
+/*
+ * 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.rocketmq.remoting.internal;
+
+import java.nio.ByteBuffer;
+import java.util.Calendar;
+
+public class UIDGenerator {
+    private static ThreadLocal<UIDGenerator> generatorLocal = new ThreadLocal<UIDGenerator>() {
+        @Override
+        protected UIDGenerator initialValue() {
+            return new UIDGenerator();
+        }
+    };
+    private short counter;
+    private int basePos = 0;
+    private long startTime;
+    private long nextStartTime;
+    private StringBuilder sb = null;
+    private ByteBuffer buffer = ByteBuffer.allocate(6);
+
+    private UIDGenerator() {
+        int len = 4 + 2 + 4 + 4 + 2;
+
+        sb = new StringBuilder(len * 2);
+        ByteBuffer tempBuffer = ByteBuffer.allocate(len - buffer.limit());
+        tempBuffer.position(2);
+        tempBuffer.putInt(JvmUtils.getProcessId());
+        tempBuffer.position(0);
+        try {
+            tempBuffer.put((byte) 1);
+        } catch (Exception e) {
+            tempBuffer.put(createFakeIP());
+        }
+        tempBuffer.position(6);
+        tempBuffer.putInt(UIDGenerator.class.getClassLoader().hashCode());
+        sb.append(ByteUtils.toHexString(tempBuffer.array()));
+        basePos = sb.length();
+        setStartTime(System.currentTimeMillis());
+        counter = 0;
+    }
+
+    public static UIDGenerator instance() {
+        return generatorLocal.get();
+    }
+
+    public String createUID() {
+        long current = System.currentTimeMillis();
+        if (current >= nextStartTime) {
+            setStartTime(current);
+        }
+        buffer.position(0);
+        sb.setLength(basePos);
+        buffer.putInt((int) (System.currentTimeMillis() - startTime));
+        buffer.putShort(counter++);
+        sb.append(ByteUtils.toHexString(buffer.array()));
+        return sb.toString();
+    }
+
+    private void setStartTime(long millis) {
+        Calendar cal = Calendar.getInstance();
+        cal.setTimeInMillis(millis);
+        cal.set(Calendar.DAY_OF_MONTH, 1);
+        cal.set(Calendar.HOUR, 0);
+        cal.set(Calendar.MINUTE, 0);
+        cal.set(Calendar.SECOND, 0);
+        cal.set(Calendar.MILLISECOND, 0);
+        startTime = cal.getTimeInMillis();
+        cal.add(Calendar.MONTH, 1);
+        nextStartTime = cal.getTimeInMillis();
+    }
+
+    public byte[] createFakeIP() {
+        ByteBuffer bb = ByteBuffer.allocate(8);
+        bb.putLong(System.currentTimeMillis());
+        bb.position(4);
+        byte[] fakeIP = new byte[4];
+        bb.get(fakeIP);
+        return fakeIP;
+    }
+}
+    

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/package-info.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/package-info.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/package-info.java
new file mode 100644
index 0000000..e64f66b
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/package-info.java
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package contains all the transport classes that can be reused any times.
+ *
+ * Remoting wire-format protocol description:
+ *
+ * <pre>
+ * 2015-04-29 16:07:14 v1.0
+ * 2016-04-23 16:18:05 v2.0
+ * 2016-05-31 09:33:11 v3.0
+ * 2016-11-10 09:33:11 v3.1 remove deprecated tag field
+ *
+ *
+ * 1.Protocol Type                            1 byte
+ * 2.Total Length                             4 byte,exclude protocol type size
+ * 3.RequestID                                4 byte,used for repeatable requests,connection reuse.an requestID string
+ * representing a client-generated, globally unique for some time unit, identifier for the request
+ * 4.Serializer Type                          1 byte
+ * 5.Traffic Type                             1 byte,0-sync;1-async;2-oneway;3-response
+ * 6.OpCode Length                            2 byte
+ * 7.OpCode                                   variant length,utf8 string
+ * 8.Remark Length                            2 byte
+ * 9.Remark                                   variant length,utf8 string
+ * 10.Properties Size                         2 byte
+ * Property Length                            2 byte
+ * Property Body                              variant length,utf8,Key\nValue
+ * 11.Inbound or OutBound payload length      4 byte
+ * 12.Inbound or OutBound payload             variant length, max size limitation is 16M
+ * 13.Extra payload                           variant length
+ *
+ * </pre>
+ */
+package org.apache.rocketmq.remoting;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/impl/netty/handler/DecoderTest.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/impl/netty/handler/DecoderTest.java b/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/impl/netty/handler/DecoderTest.java
new file mode 100644
index 0000000..4fb664f
--- /dev/null
+++ b/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/impl/netty/handler/DecoderTest.java
@@ -0,0 +1,163 @@
+/*
+ * 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.rocketmq.remoting.impl.netty.handler;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.SimpleChannelInboundHandler;
+import io.netty.channel.embedded.EmbeddedChannel;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.rocketmq.remoting.api.protocol.Protocol;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ *
+ */
+public class DecoderTest {
+
+    private EmbeddedChannel channel;
+    private AtomicInteger messagesReceived;
+    private AtomicInteger exceptionsCaught;
+
+    @Before
+    public void setUp() {
+        exceptionsCaught = new AtomicInteger(0);
+        messagesReceived = new AtomicInteger(0);
+        Decoder decoder = new Decoder();
+
+        channel = new EmbeddedChannel(decoder, new SimpleChannelInboundHandler<Object>() {
+
+            @Override
+            protected void channelRead0(ChannelHandlerContext ctx, Object msg) throws Exception {
+                messagesReceived.incrementAndGet();
+            }
+
+            @Override
+            public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
+                throws Exception {
+                exceptionsCaught.incrementAndGet();
+            }
+        });
+    }
+
+    @After
+    public void tearDown() {
+        channel.close();
+    }
+
+    @Test
+    public void decodeEmptyBufferTest() throws Exception {
+        // Send an empty buffer and make sure nothing breaks
+        channel.writeInbound(Unpooled.EMPTY_BUFFER);
+        channel.finish();
+
+        Assert.assertEquals(exceptionsCaught.get(), 0);
+        Assert.assertEquals(messagesReceived.get(), 0);
+    }
+
+    @Test
+    public void decodeHalfMessageTest() throws Exception {
+        //Case 1
+        ByteBuf buf = Unpooled.buffer().writeBytes(new byte[] {'a', 'b', 'c'});
+
+        channel.writeInbound(buf.duplicate());
+        channel.finish();
+
+        Assert.assertEquals(exceptionsCaught.get(), 0);
+        Assert.assertEquals(messagesReceived.get(), 0);
+
+        //Case 2
+        buf = Unpooled.buffer();
+        setUp();
+
+        buf.writeByte(Protocol.HTTP_2_MAGIC);
+        buf.writeInt(22);
+
+        channel.writeInbound(buf.duplicate());
+        TimeUnit.MILLISECONDS.sleep(100);
+
+        buf = Unpooled.buffer();
+        buf.writeInt(12);
+        buf.writeByte(1);
+        buf.writeByte(1);
+        buf.writeShort(2);
+        buf.writeBytes(new byte[] {'c', 'd'});
+        buf.writeShort(2);
+        buf.writeBytes(new byte[] {'e', 'f'});
+        buf.writeShort(0);
+        buf.writeInt(2);
+        buf.writeBytes(new byte[] {'g', 'h'});
+
+        channel.writeInbound(buf.duplicate());
+        channel.finish();
+
+        Assert.assertEquals(exceptionsCaught.get(), 0);
+        Assert.assertEquals(messagesReceived.get(), 1);
+    }
+
+    @Test
+    public void decodeIllegalLengthTest() throws Exception {
+        ByteBuf buf = Unpooled.buffer();
+
+        buf.writeByte(Protocol.HTTP_2_MAGIC);
+        buf.writeInt(0);
+
+        buf.writeInt(12);
+        buf.writeByte(1);
+        buf.writeByte(1);
+        buf.writeBytes(new byte[] {'a', 'b'});
+        buf.writeBytes(new byte[] {'c', 'd'});
+        buf.writeBytes(new byte[] {'e', 'f'});
+        buf.writeInt(15);
+
+        channel.writeInbound(buf.duplicate());
+        channel.finish();
+
+        Assert.assertEquals(exceptionsCaught.get(), 1);
+    }
+
+    @Test
+    public void decodeTest() throws Exception {
+        ByteBuf buf = Unpooled.buffer();
+
+        buf.writeByte(Protocol.HTTP_2_MAGIC);
+        buf.writeInt(22);
+
+        buf.writeInt(12);
+        buf.writeByte(1);
+        buf.writeByte(1);
+        buf.writeShort(2);
+        buf.writeBytes(new byte[] {'c', 'd'});
+        buf.writeShort(2);
+        buf.writeBytes(new byte[] {'e', 'f'});
+        buf.writeShort(0);
+        buf.writeInt(2);
+        buf.writeBytes(new byte[] {'g', 'h'});
+
+        channel.writeInbound(buf.duplicate());
+        channel.finish();
+
+        Assert.assertEquals(exceptionsCaught.get(), 0);
+        Assert.assertEquals(messagesReceived.get(), 1);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/impl/protocol/serializer/SerializersTest.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/impl/protocol/serializer/SerializersTest.java b/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/impl/protocol/serializer/SerializersTest.java
new file mode 100644
index 0000000..2aea401
--- /dev/null
+++ b/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/impl/protocol/serializer/SerializersTest.java
@@ -0,0 +1,263 @@
+/*
+ * 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.rocketmq.remoting.impl.protocol.serializer;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import org.apache.rocketmq.remoting.api.serializable.Serializer;
+import org.apache.rocketmq.remoting.common.TypePresentation;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.msgpack.annotation.Message;
+
+/**
+ *
+ */
+public class SerializersTest {
+    private static MsgPackSerializer msgPackSerializer;
+    private static Kryo3Serializer kryo3Serializer;
+    private static JsonSerializer jsonSerializer;
+
+    @BeforeClass
+    public static void init() {
+        msgPackSerializer = new MsgPackSerializer();
+        kryo3Serializer = new Kryo3Serializer();
+        jsonSerializer = new JsonSerializer();
+    }
+
+    @Test
+    public void msgPackSerializerTest() {
+        runOneByOne(msgPackSerializer);
+    }
+
+    @Test
+    public void kyroSerializerTest() {
+        runOneByOne(kryo3Serializer);
+    }
+
+    @Test
+    public void fastJsonSerializerTest() {
+        runOneByOne(jsonSerializer);
+    }
+
+    private void runOneByOne(Serializer serializer) {
+        listStringTest(serializer);
+        listModelTest(serializer);
+        modelTest(serializer);
+        mapTest(serializer);
+        listAndMapTest(serializer);
+        arrayTest(serializer);
+    }
+
+    private void listStringTest(Serializer serializer) {
+        // Create serialize objects.
+        List<String> src = new ArrayList<>();
+        src.add("msgpack");
+        src.add("kumofs");
+        src.add("viver");
+
+        ByteBuffer srcBuf = serializer.encode(src);
+        List<String> dst = serializer.decode(srcBuf.array(), new TypePresentation<List<String>>() {
+        });
+        Assert.assertEquals(dst, src);
+
+        List<List<String>> ll = new ArrayList<>();
+        ll.add(src);
+
+        srcBuf = serializer.encode(ll);
+        List<List<String>> llDst = serializer.decode(srcBuf.array(), new TypePresentation<List<List<String>>>() {
+        });
+        Assert.assertEquals(llDst, ll);
+
+    }
+
+    private void listModelTest(Serializer serializer) {
+        // Create serialize objects.
+        List<Model> src = new ArrayList<>();
+        src.add(new Model(10));
+        src.add(new Model(12));
+        src.add(new Model(14));
+
+        ByteBuffer srcBuf = serializer.encode(src);
+        List<Model> dst = serializer.decode(srcBuf.array(), new TypePresentation<List<Model>>() {
+        });
+        Assert.assertEquals(dst, src);
+    }
+
+    private void modelTest(Serializer serializer) {
+        Model src = new Model(123);
+
+        ByteBuffer srcBuf = serializer.encode(src);
+
+        Model dst = serializer.decode(srcBuf.array(), Model.class);
+        Assert.assertEquals(dst, src);
+    }
+
+    private void mapTest(Serializer serializer) {
+        Map<String, Model> src = new HashMap<>();
+
+        src.put("foo", new Model(123));
+        src.put("bar", new Model(234));
+
+        ByteBuffer srcBuf = serializer.encode(src);
+
+        Map<String, Model> dst = serializer.decode(srcBuf.array(), new TypePresentation<Map<String, Model>>() {
+        });
+        Assert.assertEquals(dst, src);
+    }
+
+    private void listAndMapTest(Serializer serializer) {
+        Map<String, List<Model>> src = new HashMap<>();
+
+        List<Model> list = new ArrayList<>();
+        list.add(new Model(123));
+        list.add(new Model(456));
+        src.put("foo", list);
+
+        ByteBuffer srcBuf = serializer.encode(src);
+
+        Map<String, List<Model>> dst = serializer.decode(srcBuf.array(), new TypePresentation<Map<String, List<Model>>>() {
+        });
+        Assert.assertEquals(dst, src);
+    }
+
+    private void arrayTest(Serializer serializer) {
+        Model[] models = new Model[3];
+        models[0] = new Model(1);
+        models[1] = new Model(2);
+        models[2] = new Model(3);
+
+        ByteBuffer srcBuf = serializer.encode(models);
+
+        Model[] models1 = serializer.decode(srcBuf.array(), Model[].class);
+        Assert.assertArrayEquals(models1, models);
+
+        List<Model[]> arrayInList = new LinkedList<>();
+        arrayInList.add(models);
+        srcBuf = serializer.encode(arrayInList);
+        List<Model[]> arrayInList1 = serializer.decode(srcBuf.array(), new TypePresentation<List<Model[]>>() {
+        });
+        Assert.assertArrayEquals(arrayInList.get(0), arrayInList1.get(0));
+    }
+}
+
+@Message
+class Model {
+    private int a;
+
+    private boolean isCCP = Boolean.TRUE;
+    private byte multilingualer = Byte.MAX_VALUE;
+    private short age = Short.MAX_VALUE;
+    private char education = Character.MAX_VALUE;
+    private int phoneNumber = Integer.MAX_VALUE;
+    private long anniversary = Long.MAX_VALUE;
+    private float cet4Score = Float.MAX_VALUE;
+    private double cet6Score = Double.MAX_VALUE;
+
+    private Map<String, Long> map = new HashMap<>();
+
+    //protected Date birthday = Calendar.getInstance().getTime();
+    private BigDecimal salary = BigDecimal.valueOf(11000.13);
+
+    // protected TimeZone location = Calendar.getInstance().getTimeZone();
+    //protected Timestamp location_time = new java.sql.Timestamp(Calendar.getInstance().getTimeInMillis());
+
+    //protected Locale locale = Locale.getDefault();
+    //protected EnumSet certificates = EnumSet.allOf(Certificates.class);
+    //protected BitSet qRCode = BitSet.valueOf(new long[]{123, 345});
+
+    public Model() {
+        init();
+    }
+
+    Model(final int a) {
+        this.a = a;
+        init();
+    }
+
+    private void init() {
+        map.put("Hehe", 123L);
+        map.put("grsgfg", 656L);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        Model model = (Model) o;
+
+        if (a != model.a)
+            return false;
+        if (isCCP != model.isCCP)
+            return false;
+        if (multilingualer != model.multilingualer)
+            return false;
+        if (age != model.age)
+            return false;
+        if (education != model.education)
+            return false;
+        if (phoneNumber != model.phoneNumber)
+            return false;
+        if (anniversary != model.anniversary)
+            return false;
+        if (Float.compare(model.cet4Score, cet4Score) != 0)
+            return false;
+        if (Double.compare(model.cet6Score, cet6Score) != 0)
+            return false;
+        if (map != null ? !map.equals(model.map) : model.map != null)
+            return false;
+        return salary != null ? salary.equals(model.salary) : model.salary == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result;
+        long temp;
+        result = a;
+        result = 31 * result + (isCCP ? 1 : 0);
+        result = 31 * result + (int) multilingualer;
+        result = 31 * result + (int) age;
+        result = 31 * result + (int) education;
+        result = 31 * result + phoneNumber;
+        result = 31 * result + (int) (anniversary ^ (anniversary >>> 32));
+        result = 31 * result + (cet4Score != +0.0f ? Float.floatToIntBits(cet4Score) : 0);
+        temp = Double.doubleToLongBits(cet6Score);
+        result = 31 * result + (int) (temp ^ (temp >>> 32));
+        result = 31 * result + (map != null ? map.hashCode() : 0);
+        result = 31 * result + (salary != null ? salary.hashCode() : 0);
+        return result;
+    }
+
+    public int getA() {
+        return a;
+    }
+
+    public void setA(final int a) {
+        this.a = a;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/internal/BeanUtilsTest.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/internal/BeanUtilsTest.java b/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/internal/BeanUtilsTest.java
new file mode 100644
index 0000000..c86bc4c
--- /dev/null
+++ b/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/internal/BeanUtilsTest.java
@@ -0,0 +1,134 @@
+/*
+ * 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.rocketmq.remoting.internal;
+
+import java.util.Properties;
+import org.apache.rocketmq.remoting.config.RemotingConfig;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class BeanUtilsTest {
+    private Properties properties = new Properties();
+    private final static String CLIENT_CONFIG_PREFIX = "client.";
+    private final static String SERVER_CONFIG_PREFIX = "server.";
+    private final static String TCP_CONFIG_PREFIX = "tcp.";
+    private final static String CONNECTION_CHANNEL_IDLE_SECONDS = "connection.channel.idle.seconds";
+    private final static String CLIENT_ASYNC_CALLBACK_EXECUTOR_THREADS = CLIENT_CONFIG_PREFIX + "async.callback.executor.threads";
+    private final static String CLIENT_POOLEDBYTEBUF_ALLOCATOR_ENABLE = CLIENT_CONFIG_PREFIX + "pooled.bytebuf.allocator.enable";
+    private final static String SERVER_LISTEN_PORT = SERVER_CONFIG_PREFIX + "listen.port";
+    private final static String TCP_SO_REUSE_ADDRESS = TCP_CONFIG_PREFIX + "so.reuse.address";
+    private final static String TCP_SO_KEEPALIVE = TCP_CONFIG_PREFIX + "so.keep.alive";
+    private final static String TCP_SO_NODELAY = TCP_CONFIG_PREFIX + "so.no.delay";
+    private final static String TCP_SO_SNDBUF_SIZE = TCP_CONFIG_PREFIX + "so.snd.buf.size";
+    private final static String TCP_SO_RCVBUF_SIZE = TCP_CONFIG_PREFIX + "so.rcv.buf.size";
+    private final static String TCP_SO_BACKLOG_SIZE = TCP_CONFIG_PREFIX + "so.backlog.size";
+    private final static String TCP_SO_TIMEOUT = TCP_CONFIG_PREFIX + "so.timeout";
+    private final static String TCP_SO_LINGER = TCP_CONFIG_PREFIX + "so.linger";
+
+    @Before
+    public void init() {
+        properties.put(CONNECTION_CHANNEL_IDLE_SECONDS, 3);
+
+        properties.put(CLIENT_ASYNC_CALLBACK_EXECUTOR_THREADS, 10);
+        properties.put(CLIENT_POOLEDBYTEBUF_ALLOCATOR_ENABLE, true);
+
+        properties.put(SERVER_LISTEN_PORT, 900);
+
+        properties.put(TCP_SO_REUSE_ADDRESS, false);
+        properties.put(TCP_SO_KEEPALIVE, false);
+        properties.put(TCP_SO_NODELAY, false);
+        properties.put(TCP_SO_SNDBUF_SIZE, 100);
+        properties.put(TCP_SO_RCVBUF_SIZE, 100);
+        properties.put(TCP_SO_BACKLOG_SIZE, 100);
+        properties.put(TCP_SO_TIMEOUT, 5000);
+        properties.put(TCP_SO_LINGER, 100);
+
+        properties.put(ClientConfig.STRING_TEST, "kaka");
+    }
+
+    @Test
+    public void populateTest() {
+        ClientConfig config = BeanUtils.populate(properties, ClientConfig.class);
+
+        //RemotingConfig config = BeanUtils.populate(properties, RemotingConfig.class);
+        Assert.assertEquals(config.getConnectionChannelIdleSeconds(), 3);
+
+        Assert.assertEquals(config.getClientAsyncCallbackExecutorThreads(), 10);
+        Assert.assertTrue(config.isClientPooledBytebufAllocatorEnable());
+
+        Assert.assertEquals(config.getServerListenPort(), 900);
+
+        Assert.assertFalse(config.isTcpSoReuseAddress());
+
+        Assert.assertFalse(config.isTcpSoKeepAlive());
+        Assert.assertFalse(config.isTcpSoNoDelay());
+        Assert.assertEquals(config.getTcpSoSndBufSize(), 100);
+        Assert.assertEquals(config.getTcpSoRcvBufSize(), 100);
+        Assert.assertEquals(config.getTcpSoBacklogSize(), 100);
+        Assert.assertEquals(config.getTcpSoTimeout(), 5000);
+        Assert.assertEquals(config.getTcpSoLinger(), 100);
+
+        Assert.assertEquals(config.getStringTest(), "kaka");
+    }
+
+    @Test
+    public void populateExistObj() {
+        ClientConfig config = new ClientConfig();
+        config.setServerListenPort(8118);
+
+        Assert.assertEquals(config.getServerListenPort(), 8118);
+
+        config = BeanUtils.populate(properties, config);
+
+        Assert.assertEquals(config.getConnectionChannelIdleSeconds(), 3);
+
+        Assert.assertEquals(config.getClientAsyncCallbackExecutorThreads(), 10);
+        Assert.assertTrue(config.isClientPooledBytebufAllocatorEnable());
+
+        Assert.assertEquals(config.getServerListenPort(), 900);
+
+        Assert.assertFalse(config.isTcpSoReuseAddress());
+        Assert.assertFalse(config.isTcpSoKeepAlive());
+        Assert.assertFalse(config.isTcpSoNoDelay());
+        Assert.assertEquals(config.getTcpSoSndBufSize(), 100);
+        Assert.assertEquals(config.getTcpSoRcvBufSize(), 100);
+        Assert.assertEquals(config.getTcpSoBacklogSize(), 100);
+        Assert.assertEquals(config.getTcpSoTimeout(), 5000);
+        Assert.assertEquals(config.getTcpSoLinger(), 100);
+
+        Assert.assertEquals(config.getStringTest(), "kaka");
+    }
+
+    public static class ClientConfig extends RemotingConfig {
+        public final static String STRING_TEST = "string.test";
+        String stringTest = "foobar";
+
+        public ClientConfig() {
+        }
+
+        public String getStringTest() {
+            return stringTest;
+        }
+
+        public void setStringTest(String stringTest) {
+            this.stringTest = stringTest;
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/internal/ExceptionUtilsTest.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/internal/ExceptionUtilsTest.java b/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/internal/ExceptionUtilsTest.java
new file mode 100644
index 0000000..bb46558
--- /dev/null
+++ b/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/internal/ExceptionUtilsTest.java
@@ -0,0 +1,43 @@
+/*
+ * 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.rocketmq.remoting.internal;
+
+import org.junit.Test;
+
+/**
+ *
+ */
+public class ExceptionUtilsTest {
+
+    @Test(expected = Error.class)
+    public void getStackTraceTest() {
+        try {
+            try {
+                throw new IllegalArgumentException("Faked exception 1");
+            } catch (IllegalArgumentException e) {
+                System.out.println(ExceptionUtils.getStackTrace(e));
+                throw new RuntimeException(e);
+            }
+        } catch (RuntimeException e) {
+            System.out.println(ExceptionUtils.getStackTrace(e));
+            throw new Error(e);
+        }
+
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/internal/JvmUtilsTest.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/internal/JvmUtilsTest.java b/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/internal/JvmUtilsTest.java
new file mode 100644
index 0000000..475bf64
--- /dev/null
+++ b/remoting-core/remoting-impl/src/test/java/org/apache/rocketmq/remoting/internal/JvmUtilsTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.rocketmq.remoting.internal;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ *
+ */
+public class JvmUtilsTest {
+
+    @Test
+    public void getProcessIdTest() {
+        Assert.assertTrue(JvmUtils.getProcessId() > 0);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-api/pom.xml
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-api/pom.xml b/remoting-core/rpc-api/pom.xml
new file mode 100644
index 0000000..ad1ab62
--- /dev/null
+++ b/remoting-core/rpc-api/pom.xml
@@ -0,0 +1,38 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.rocketmq</groupId>
+        <artifactId>remoting-core</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>rpc-api</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>remoting-api</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/annotation/MethodType.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/annotation/MethodType.java b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/annotation/MethodType.java
new file mode 100644
index 0000000..d67c018
--- /dev/null
+++ b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/annotation/MethodType.java
@@ -0,0 +1,35 @@
+/*
+ * 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.rocketmq.rpc.annotation;
+
+public enum MethodType {
+    /**
+     * Mark a RemoteMethod as a synchronous method.
+     */
+    SYNC,
+
+    /**
+     * Mark a RemoteMethod as a asynchronous method.
+     */
+    ASYNC,
+
+    /**
+     * Mark a RemoteMethod as a oneway method.
+     */
+    ONEWAY
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/annotation/RemoteMethod.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/annotation/RemoteMethod.java b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/annotation/RemoteMethod.java
new file mode 100644
index 0000000..a545342
--- /dev/null
+++ b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/annotation/RemoteMethod.java
@@ -0,0 +1,53 @@
+/*
+ * 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.rocketmq.rpc.annotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Target(ElementType.METHOD)
+@Retention(RetentionPolicy.RUNTIME)
+public @interface RemoteMethod {
+    String name() default "";
+
+    /**
+     * The API version (NOT the product version), composed as a dot delimited
+     * string with major, minor, and patch level components.
+     * <pre>
+     * - Major: Incremented for backward incompatible changes. An example would
+     * be changes to the number or disposition of method arguments.
+     * - Minor: Incremented for backward compatible changes. An example would
+     * be the addition of a new (optional) method.
+     * - Patch: Incremented for bug fixes. The patch level should be increased
+     * for every edit that doesn't result in a change to major/minor.
+     * </pre>
+     * See the Semantic Versioning Specification (SemVer) http://semver.org.
+     *
+     * @return the string version presentation
+     */
+    String version() default "1.0.0";
+
+    String description() default "";
+
+    MethodType type() default MethodType.SYNC;
+}
+
+
+

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/annotation/RemoteService.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/annotation/RemoteService.java b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/annotation/RemoteService.java
new file mode 100644
index 0000000..461d92e
--- /dev/null
+++ b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/annotation/RemoteService.java
@@ -0,0 +1,48 @@
+/*
+ * 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.rocketmq.rpc.annotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Target(ElementType.TYPE)
+@Retention(RetentionPolicy.RUNTIME)
+public @interface RemoteService {
+    String name();
+
+    /**
+     * The API version (NOT the product version), composed as a dot delimited
+     * string with major, minor, and patch level components.
+     * <pre>
+     * - Major: Incremented for backward incompatible changes. An example would
+     * be changes to the number or disposition of method arguments.
+     * - Minor: Incremented for backward compatible changes. An example would
+     * be the addition of a new (optional) method.
+     * - Patch: Incremented for bug fixes. The patch level should be increased
+     * for every edit that doesn't result in a change to major/minor.
+     * </pre>
+     * See the Semantic Versioning Specification (SemVer) http://semver.org.
+     *
+     * @return the string version presentation
+     */
+    String version() default "1.0.0";
+
+    String description() default "";
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/annotation/VisibleForInternal.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/annotation/VisibleForInternal.java b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/annotation/VisibleForInternal.java
new file mode 100644
index 0000000..fe9986d
--- /dev/null
+++ b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/annotation/VisibleForInternal.java
@@ -0,0 +1,32 @@
+/*
+ * 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.rocketmq.rpc.annotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Annotates a program element that exists, or is more widely visible than otherwise necessary, only
+ * for internal communication.
+ */
+@Target(ElementType.METHOD)
+@Retention(RetentionPolicy.RUNTIME)
+public @interface VisibleForInternal {
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/AdvancedClient.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/AdvancedClient.java b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/AdvancedClient.java
new file mode 100644
index 0000000..881dc68
--- /dev/null
+++ b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/AdvancedClient.java
@@ -0,0 +1,36 @@
+/*
+ * 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.rocketmq.rpc.api;
+
+public interface AdvancedClient {
+    <T> T callSync(String address,
+        final String serviceCode,
+        final String version,
+        final Object[] parameter,
+        final Class<T> responseType) throws Exception;
+
+    <T> Promise<T> callAsync(String address,
+        final String serviceCode,
+        final String version,
+        final Object[] parameter,
+        final Class<T> responseType) throws Exception;
+
+    void callOneway(String address,
+        final String serviceCode,
+        final String version,
+        final Object[] parameter) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/AdvancedServer.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/AdvancedServer.java b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/AdvancedServer.java
new file mode 100644
index 0000000..3dd4188
--- /dev/null
+++ b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/AdvancedServer.java
@@ -0,0 +1,38 @@
+/*
+ * 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.rocketmq.rpc.api;
+
+import org.apache.rocketmq.remoting.api.channel.RemotingChannel;
+
+public interface AdvancedServer {
+    <T> T callSync(final RemotingChannel channel,
+        final String serviceCode,
+        final String version,
+        final Object[] parameter,
+        final Class<T> responseType) throws Exception;
+
+    <T> Promise<T> callAsync(final RemotingChannel channel,
+        final String serviceCode,
+        final String version,
+        final Object[] parameter,
+        final Class<T> responseType) throws Exception;
+
+    void callOneway(final RemotingChannel channel,
+        final String serviceCode,
+        final String version,
+        final Object[] parameter) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/Promise.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/Promise.java b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/Promise.java
new file mode 100644
index 0000000..ec94be3
--- /dev/null
+++ b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/Promise.java
@@ -0,0 +1,114 @@
+/*
+ * 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.rocketmq.rpc.api;
+
+/**
+ * A {@code Promise} represents the result of an asynchronous computation.  Methods are provided to check if the
+ * computation is complete, to wait for its completion, and to retrieve the result of the computation.  The result can
+ * only be retrieved using method {@code get} when the computation has completed, blocking if necessary until it is
+ * ready.  Cancellation is performed by the {@code cancel} method.  Additional methods are provided to determine if the
+ * task completed normally or was cancelled. Once a computation has completed, the computation cannot be cancelled. If
+ * you would like to use a {@code Promise} for the sake of cancellability but not provide a usable result, you can
+ * declare types of the form {@code Promise<?>} and return {@code null} as a result of the underlying task.
+ *
+ * @since 1.0.0
+ */
+public interface Promise<V> {
+
+    /**
+     * Attempts to cancel execution of this task.  This attempt will fail if the task has already completed, has already
+     * been cancelled, or could not be cancelled for some other reason. If successful, and this task has not started
+     * when {@code cancel} is called, this task should never run.  If the task has already started, then the {@code
+     * mayInterruptIfRunning} parameter determines whether the thread executing this task should be interrupted in an
+     * attempt to stop the task.
+     * <p>
+     * After this method returns, subsequent calls to {@link #isDone} will always return {@code true}.  Subsequent calls
+     * to {@link #isCancelled} will always return {@code true} if this method returned {@code true}.
+     *
+     * @param mayInterruptIfRunning {@code true} if the thread executing this task should be interrupted; otherwise,
+     * in-progress tasks are allowed to complete
+     * @return {@code false} if the task could not be cancelled, typically because it has already completed normally;
+     * {@code true} otherwise
+     */
+    boolean cancel(boolean mayInterruptIfRunning);
+
+    /**
+     * Returns {@code true} if this task was cancelled before it completed
+     * normally.
+     *
+     * @return {@code true} if this task was cancelled before it completed
+     */
+    boolean isCancelled();
+
+    /**
+     * Returns {@code true} if this task completed.
+     * <p>
+     * Completion may be due to normal termination, an exception, or
+     * cancellation -- in all of these cases, this method will return
+     * {@code true}.
+     *
+     * @return {@code true} if this task completed
+     */
+    boolean isDone();
+
+    /**
+     * Waits if necessary for the computation to complete, and then
+     * retrieves its result.
+     *
+     * @return the computed result
+     */
+    V get();
+
+    /**
+     * Waits if necessary for at most the given time for the computation
+     * to complete, and then retrieves its result, if available.
+     *
+     * @param timeout the maximum time to wait
+     * @return the computed result <p> if the computation was cancelled
+     */
+    V get(long timeout);
+
+    /**
+     * Set the value to this promise and mark it completed if set successfully.
+     *
+     * @param value Value
+     * @return Whether set is success
+     */
+    boolean set(V value);
+
+    /**
+     * Marks this promise as a failure and notifies all listeners.
+     *
+     * @param cause the cause
+     * @return Whether set is success
+     */
+    boolean setFailure(Throwable cause);
+
+    /**
+     * Adds the specified listener to this promise. The specified listener is notified when this promise is done. If
+     * this promise is already completed, the specified listener will be notified immediately.
+     *
+     * @param listener PromiseListener
+     */
+    void addListener(PromiseListener<V> listener);
+
+    /**
+     * @return a throwable caught by the promise
+     */
+    Throwable getThrowable();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/PromiseListener.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/PromiseListener.java b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/PromiseListener.java
new file mode 100644
index 0000000..88dd731
--- /dev/null
+++ b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/PromiseListener.java
@@ -0,0 +1,41 @@
+/*
+ * 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.rocketmq.rpc.api;
+
+/**
+ * A listener that is called back when a Promise is done.
+ * {@code PromiseListener} instances are attached to {@link Promise} by passing
+ * them in to {@link Promise#addListener(PromiseListener)}.
+ *
+ * @since 1.0.0
+ */
+public interface PromiseListener<V> {
+    /**
+     * Invoked when the operation associated with the {@code Promise} has been completed successfully.
+     *
+     * @param promise the source {@code Promise} which called this callback
+     */
+    void operationCompleted(Promise<V> promise);
+
+    /**
+     * Invoked when the operation associated with the {@code Promise} has been completed unsuccessfully.
+     *
+     * @param promise the source {@code Promise} which called this callback
+     */
+    void operationFailed(Promise<V> promise);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/SimpleClient.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/SimpleClient.java b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/SimpleClient.java
new file mode 100644
index 0000000..ece1f60
--- /dev/null
+++ b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/SimpleClient.java
@@ -0,0 +1,32 @@
+/*
+ * 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.rocketmq.rpc.api;
+
+import java.util.Properties;
+import java.util.concurrent.ExecutorService;
+import org.apache.rocketmq.remoting.api.ObjectLifecycle;
+
+public interface SimpleClient extends ObjectLifecycle {
+    <T> T bind(Class<T> service, String address, Properties properties); //keyValue
+
+    void publish(Object service);
+
+    void publish(Object service, ExecutorService executorService);
+
+    AdvancedClient advancedClient();
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/SimpleServer.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/SimpleServer.java b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/SimpleServer.java
new file mode 100644
index 0000000..4bf5372
--- /dev/null
+++ b/remoting-core/rpc-api/src/main/java/org/apache/rocketmq/rpc/api/SimpleServer.java
@@ -0,0 +1,33 @@
+/*
+ * 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.rocketmq.rpc.api;
+
+import java.util.Properties;
+import java.util.concurrent.ExecutorService;
+import org.apache.rocketmq.remoting.api.ObjectLifecycle;
+import org.apache.rocketmq.remoting.api.channel.RemotingChannel;
+
+public interface SimpleServer extends ObjectLifecycle {
+    <T> T bind(final Class<T> service, final RemotingChannel channel, final Properties properties);
+
+    AdvancedServer advancedServer();
+
+    void publish(final Object service);
+
+    void publish(Object service, ExecutorService executorService);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/pom.xml
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/pom.xml b/remoting-core/rpc-impl/pom.xml
new file mode 100644
index 0000000..162d0b8
--- /dev/null
+++ b/remoting-core/rpc-impl/pom.xml
@@ -0,0 +1,45 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.rocketmq</groupId>
+        <artifactId>remoting-core</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>rpc-impl</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rpc-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>remoting-impl</artifactId>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file



[3/8] incubator-rocketmq git commit: initialize RocketMQ5

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/RpcBootstrapFactory.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/RpcBootstrapFactory.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/RpcBootstrapFactory.java
new file mode 100644
index 0000000..13ce176
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/RpcBootstrapFactory.java
@@ -0,0 +1,36 @@
+/*
+ * 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.rocketmq.rpc;
+
+import java.util.Properties;
+import org.apache.rocketmq.remoting.internal.BeanUtils;
+import org.apache.rocketmq.rpc.api.SimpleClient;
+import org.apache.rocketmq.rpc.api.SimpleServer;
+import org.apache.rocketmq.rpc.impl.client.SimpleClientImpl;
+import org.apache.rocketmq.rpc.impl.config.RpcCommonConfig;
+import org.apache.rocketmq.rpc.impl.server.SimpleServerImpl;
+
+public class RpcBootstrapFactory {
+    public static SimpleServer createServerBootstrap(Properties properties) {
+        return new SimpleServerImpl(BeanUtils.populate(properties, RpcCommonConfig.class));
+    }
+
+    public static SimpleClient createClientBootstrap(Properties properties) {
+        return new SimpleClientImpl(BeanUtils.populate(properties, RpcCommonConfig.class));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/client/AdvancedClientImpl.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/client/AdvancedClientImpl.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/client/AdvancedClientImpl.java
new file mode 100644
index 0000000..e94f589
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/client/AdvancedClientImpl.java
@@ -0,0 +1,57 @@
+/*
+ * 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.rocketmq.rpc.impl.client;
+
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.rpc.annotation.MethodType;
+import org.apache.rocketmq.rpc.api.AdvancedClient;
+import org.apache.rocketmq.rpc.api.Promise;
+import org.apache.rocketmq.rpc.impl.service.RpcJdkProxy;
+import org.apache.rocketmq.rpc.impl.service.RpcProxyFactory;
+
+public class AdvancedClientImpl implements AdvancedClient {
+    private final SimpleClientImpl simpleClient;
+
+    public AdvancedClientImpl(final SimpleClientImpl simpleClient) {
+        this.simpleClient = simpleClient;
+    }
+
+    @Override
+    public <T> T callSync(final String address, final String serviceCode, final String version,
+        final Object[] parameter,
+        final Class<T> responseType) throws Exception {
+        RemotingCommand request = simpleClient.createRemoteRequest(serviceCode, version, parameter);
+        RpcJdkProxy rpcJdkProxy = RpcProxyFactory.createServiceProxy(null, simpleClient, simpleClient.getRemotingClient(), simpleClient.getRpcCommonConfig(), address);
+        return (T) simpleClient.invokeRemoteMethod(rpcJdkProxy, serviceCode, request, responseType, MethodType.SYNC);
+    }
+
+    @Override
+    public <T> Promise<T> callAsync(final String address, final String serviceCode, final String version,
+        final Object[] parameter, final Class<T> responseType) throws Exception {
+        RemotingCommand request = simpleClient.createRemoteRequest(serviceCode, version, parameter);
+        RpcJdkProxy rpcJdkProxy = RpcProxyFactory.createServiceProxy(null, simpleClient, simpleClient.getRemotingClient(), simpleClient.getRpcCommonConfig(), address);
+        return (Promise<T>) simpleClient.invokeRemoteMethod(rpcJdkProxy, serviceCode, request, responseType, MethodType.ASYNC);
+    }
+
+    @Override
+    public void callOneway(final String address, final String serviceCode, final String version,
+        final Object[] parameter) throws Exception {
+        RemotingCommand request = simpleClient.createRemoteRequest(serviceCode, version, parameter);
+        RpcJdkProxy rpcJdkProxy = RpcProxyFactory.createServiceProxy(null, simpleClient, simpleClient.getRemotingClient(), simpleClient.getRpcCommonConfig(), address);
+        simpleClient.invokeRemoteMethod(rpcJdkProxy, serviceCode, request, Void.TYPE, MethodType.ONEWAY);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/client/SimpleClientImpl.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/client/SimpleClientImpl.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/client/SimpleClientImpl.java
new file mode 100644
index 0000000..787e8c1
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/client/SimpleClientImpl.java
@@ -0,0 +1,132 @@
+/*
+ * 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.rocketmq.rpc.impl.client;
+
+import java.util.Properties;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.rocketmq.remoting.api.RemotingClient;
+import org.apache.rocketmq.remoting.api.RemotingService;
+import org.apache.rocketmq.remoting.api.interceptor.Interceptor;
+import org.apache.rocketmq.remoting.external.ThreadUtils;
+import org.apache.rocketmq.remoting.impl.netty.RemotingBootstrapFactory;
+import org.apache.rocketmq.rpc.api.AdvancedClient;
+import org.apache.rocketmq.rpc.api.SimpleClient;
+import org.apache.rocketmq.rpc.impl.config.RpcCommonConfig;
+import org.apache.rocketmq.rpc.impl.exception.ServiceExceptionManager;
+import org.apache.rocketmq.rpc.impl.service.RpcConnectionListener;
+import org.apache.rocketmq.rpc.impl.service.RpcInstanceAbstract;
+import org.apache.rocketmq.rpc.impl.service.RpcProxyFactory;
+import org.apache.rocketmq.rpc.internal.RpcErrorMapper;
+
+public class SimpleClientImpl extends RpcInstanceAbstract implements SimpleClient {
+    private RpcCommonConfig rpcCommonConfig;
+    private RemotingClient remotingClient;
+    private ExecutorService callServiceThreadPool;
+
+    public SimpleClientImpl(RpcCommonConfig rpcCommonConfig) {
+        this(rpcCommonConfig, RemotingBootstrapFactory.createRemotingClient(rpcCommonConfig));
+    }
+
+    private SimpleClientImpl(RpcCommonConfig rpcCommonConfig, RemotingClient remotingClient) {
+        super(rpcCommonConfig);
+        this.remotingClient = remotingClient;
+        this.rpcCommonConfig = rpcCommonConfig;
+        this.callServiceThreadPool = ThreadUtils.newThreadPoolExecutor(rpcCommonConfig.getClientAsyncCallbackExecutorThreads(),
+            rpcCommonConfig.getClientAsyncCallbackExecutorThreads(), rpcCommonConfig.getServiceThreadKeepAliveTime(),
+            TimeUnit.MILLISECONDS, new ArrayBlockingQueue<Runnable>(rpcCommonConfig.getServiceThreadBlockQueueSize()), "clientCallServiceThread", true);
+    }
+
+    public void initialize() {
+        this.remotingClient.registerChannelEventListener(new RpcConnectionListener(this));
+    }
+
+    @Override
+    public void start() {
+        try {
+            initialize();
+            super.start();
+            this.remotingClient.start();
+        } catch (Exception e) {
+            throw ServiceExceptionManager.TranslateException(RpcErrorMapper.RpcErrorLatitude.LOCAL.getCode(), e);
+        }
+    }
+
+    @Override
+    public void stop() {
+        try {
+            super.stop();
+            ThreadUtils.shutdownGracefully(this.callServiceThreadPool, 3000, TimeUnit.MILLISECONDS);
+            this.remotingClient.stop();
+        } catch (Exception e) {
+            throw ServiceExceptionManager.TranslateException(RpcErrorMapper.RpcErrorLatitude.LOCAL.getCode(), e);
+        }
+    }
+
+    @Override
+    public <T> T bind(final Class<T> service, final String address, final Properties properties) {
+        return this.narrow0(service, RpcProxyFactory.createServiceProxy(service, this, remotingClient, rpcCommonConfig, address));
+    }
+
+    @Override
+    public void publish(final Object service) {
+        this.publishService0(service);
+    }
+
+    @Override
+    public void publish(final Object service, final ExecutorService executorService) {
+        this.publishService0(service, executorService);
+    }
+
+    @Override
+    public AdvancedClient advancedClient() {
+        return new AdvancedClientImpl(this);
+    }
+
+    @Override
+    public RemotingService remotingService() {
+        return this.remotingClient;
+    }
+
+    @Override
+    public void registerServiceListener() {
+
+    }
+
+    public ExecutorService getCallServiceThreadPool() {
+        return callServiceThreadPool;
+    }
+
+    public void setCallServiceThreadPool(final ExecutorService callServiceThreadPool) {
+        this.callServiceThreadPool = callServiceThreadPool;
+    }
+
+    public void registerInterceptor(Interceptor interceptor) {
+        if (interceptor != null)
+            this.remotingClient.registerInterceptor(interceptor);
+    }
+
+    public RemotingClient getRemotingClient() {
+        return remotingClient;
+    }
+
+    public void setRemotingClient(final RemotingClient remotingClient) {
+        this.remotingClient = remotingClient;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/command/ResponseCode.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/command/ResponseCode.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/command/ResponseCode.java
new file mode 100644
index 0000000..270adaa
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/command/ResponseCode.java
@@ -0,0 +1,118 @@
+/*
+ * 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.rocketmq.rpc.impl.command;
+
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+
+/**
+ * 1xx: SDK exception
+ * <p>
+ * 4xx: User exception
+ * <p>
+ * 5xx: Server exception
+ * <p>
+ * 6xx: Common exception
+ */
+public class ResponseCode {
+
+    public static final String ILLEGAL_ACCESS = "100";
+    public static final String NULL_POINTER = "102";
+    public static final String FAIL_INVOKE = "103";
+    public static final String INSTANTIATED_FAIL = "104";
+    public static final String SUCCESS = "0";
+
+    public static final String USER_SERVICE_EXCEPTION = "400";
+    public static final String USER_EXCEPTION_CLASS_NOT_FOUND = "401";
+    public static final String USER_EXCEPTION_METHOD_NOT_FOUND = "402";
+
+    public static final String PROVIDER_OFF_LINE = "501";
+    public static final String CONSUMER_OFF_LINE = "502";
+    public static final String PROVIDER_NOT_EXIST = "503";
+    public static final String CONSUMER_NOT_EXIST = "504";
+    public static final String NAMESPACE_NOT_EXIST = "505";
+    public static final String CLIENT_NOT_REGISTERED = "506";
+    public static final String NO_SERVICE_ON_LINE = "507";
+    public static final String NO_SERVICE_ON_THIS_SERVER = "508";
+    public static final String SERVICE_ALREADY_ONLINE = "509";
+    public static final String SERVER_NOT_EXIST = "510";
+    public static final String SERVER_ALREADY_ONLINE = "511";
+    public static final String NO_SERVER_ON_LINE = "512";
+    public static final String ACCESS_DENIED = "513";
+
+    public static final String SYSTEM_ERROR = "600";
+    public static final String SYSTEM_BUSY = "601";
+    public static final String BAD_REQUEST = "602";
+    public static final String PARAMETER_ERROR = "603";
+    public static final String SEND_REQUEST_FAILED = "604";
+    public static final String REQUEST_TIMEOUT = "605";
+    public static final String UNKNOWN_EXCEPTION = "606";
+
+    @Override
+    public String toString() {
+        return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE);
+    }
+
+    public enum ResponseStatus {
+        R_100(100, "Illegal access"),
+        R_101(101, "Illegal argument"),
+        R_102(102, "Null pointer"),
+        R_103(103, "Invoke failed"),
+        R_104(104, "Failed initialization"),
+        R_200(200, "Success"),
+        R_400(400, "User service happened"),
+        R_401(401, "Class not found"),
+        R_402(402, "Method not found"),
+        R_501(501, "Service provider offline "),
+        R_502(502, "Service consumer offline"),
+        R_503(503, "Service provider not exist"),
+        R_504(504, "Service consumer not exist"),
+        R_505(505, "Namespace not exist"),
+        R_506(506, "Client not registered"),
+        R_507(507, "No service online"),
+        R_508(508, "No service on this server"),
+        R_509(509, "Server already online"),
+        R_510(510, "Server not exist"),
+        R_511(511, "Server already online"),
+        R_512(512, "No server online"),
+        R_513(513, "Access denied"),
+        R_600(600, "System error"),
+        R_601(601, "System busy"),
+        R_602(602, "Bad request"),
+        R_603(603, "Parameter error"),
+        R_604(604, "Send request failed"),
+        R_605(605, "Request timeout"),
+        R_606(606, "unknown exception");
+
+        private int responseCode = 0;
+        private String responseSimpleMessage = "";
+
+        ResponseStatus(int responseCode, String responseSimpleMessage) {
+            this.responseCode = responseCode;
+            this.responseSimpleMessage = responseSimpleMessage;
+        }
+
+        public int getResponseCode() {
+            return responseCode;
+        }
+
+        public String getResponseSimpleMessage() {
+            return responseSimpleMessage;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/command/RpcRequestCode.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/command/RpcRequestCode.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/command/RpcRequestCode.java
new file mode 100644
index 0000000..3507754
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/command/RpcRequestCode.java
@@ -0,0 +1,22 @@
+/*
+ * 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.rocketmq.rpc.impl.command;
+
+public interface RpcRequestCode {
+    String CALL_SERVICE = "service/call";
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/config/RpcCommonConfig.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/config/RpcCommonConfig.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/config/RpcCommonConfig.java
new file mode 100644
index 0000000..7c00a09
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/config/RpcCommonConfig.java
@@ -0,0 +1,63 @@
+/*
+ * 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.rocketmq.rpc.impl.config;
+
+import org.apache.rocketmq.remoting.config.RemotingConfig;
+
+public class RpcCommonConfig extends RemotingConfig {
+    public final static String SERVICE_INVOKE_TIMEOUT = "service.invoke.timeout";
+    public final static String SERVICE_THREAD_KEEP_ALIVE_TIME = "service.thread.keep.alive.time";
+    private final static String SERVICE_ADDRESS_CACHE_TIME = "service.address.cache.time";
+    private final static String SERVICE_CACHE_MAX_COUNT = "service.cache.max.count";
+    private long serviceInvokeTimeout = 10000;
+    private long serviceThreadKeepAliveTime = 60000;
+    private long serviceAddressCacheTime = 30;
+    private long serviceCacheMaxCount = 20000;
+
+    public long getServiceInvokeTimeout() {
+        return serviceInvokeTimeout;
+    }
+
+    public void setServiceInvokeTimeout(final long serviceInvokeTimeout) {
+        this.serviceInvokeTimeout = serviceInvokeTimeout;
+    }
+
+    public long getServiceThreadKeepAliveTime() {
+        return serviceThreadKeepAliveTime;
+    }
+
+    public void setServiceThreadKeepAliveTime(final long serviceThreadKeepAliveTime) {
+        this.serviceThreadKeepAliveTime = serviceThreadKeepAliveTime;
+    }
+
+    public long getServiceAddressCacheTime() {
+        return serviceAddressCacheTime;
+    }
+
+    public void setServiceAddressCacheTime(long serviceAddressCacheTime) {
+        this.serviceAddressCacheTime = serviceAddressCacheTime;
+    }
+
+    public long getServiceCacheMaxCount() {
+        return serviceCacheMaxCount;
+    }
+
+    public void setServiceCacheMaxCount(long serviceCacheMaxCount) {
+        this.serviceCacheMaxCount = serviceCacheMaxCount;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/context/RpcCallerContext.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/context/RpcCallerContext.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/context/RpcCallerContext.java
new file mode 100644
index 0000000..bb8433c
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/context/RpcCallerContext.java
@@ -0,0 +1,48 @@
+/*
+ * 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.rocketmq.rpc.impl.context;
+
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+
+public class RpcCallerContext {
+    private RemotingCommand remotingRequest;
+    private RemotingCommand remotingResponse;
+
+    public RemotingCommand getRemotingRequest() {
+        return remotingRequest;
+    }
+
+    public void setRemotingRequest(RemotingCommand remotingRequest) {
+        this.remotingRequest = remotingRequest;
+    }
+
+    public RemotingCommand getRemotingResponse() {
+        return remotingResponse;
+    }
+
+    public void setRemotingResponse(RemotingCommand remotingResponse) {
+        this.remotingResponse = remotingResponse;
+    }
+
+    @Override
+    public String toString() {
+        return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/context/RpcProviderContext.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/context/RpcProviderContext.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/context/RpcProviderContext.java
new file mode 100644
index 0000000..547a0ec
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/context/RpcProviderContext.java
@@ -0,0 +1,67 @@
+/*
+ * 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.rocketmq.rpc.impl.context;
+
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.rocketmq.remoting.api.channel.RemotingChannel;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+
+public class RpcProviderContext {
+    private RemotingChannel remotingChannel;
+    private RemotingCommand remotingRequest;
+    private RemotingCommand remotingResponse;
+    private boolean returnResponse = true;
+
+    public RemotingChannel getRemotingChannel() {
+        return remotingChannel;
+    }
+
+    public void setRemotingChannel(RemotingChannel remotingChannel) {
+        this.remotingChannel = remotingChannel;
+    }
+
+    public RemotingCommand getRemotingRequest() {
+        return remotingRequest;
+    }
+
+    public void setRemotingRequest(RemotingCommand remotingRequest) {
+        this.remotingRequest = remotingRequest;
+    }
+
+    public RemotingCommand getRemotingResponse() {
+        return remotingResponse;
+    }
+
+    public void setRemotingResponse(RemotingCommand remotingResponse) {
+        this.remotingResponse = remotingResponse;
+    }
+
+    public boolean isReturnResponse() {
+        return returnResponse;
+    }
+
+    public void setReturnResponse(boolean returnResponse) {
+        this.returnResponse = returnResponse;
+    }
+
+    @Override
+    public String toString() {
+        return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceAuthException.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceAuthException.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceAuthException.java
new file mode 100644
index 0000000..ab53a54
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceAuthException.java
@@ -0,0 +1,68 @@
+/*
+ * 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.rocketmq.rpc.impl.exception;
+
+import org.apache.rocketmq.remoting.api.exception.RemoteAccessException;
+
+public class ServiceAuthException extends RemoteAccessException {
+    private static final long serialVersionUID = 1L;
+
+    private String status;
+    private int code;
+
+    public ServiceAuthException(String status, int code) {
+        super("");
+        this.status = status;
+        this.code = code;
+    }
+
+    public ServiceAuthException(String status, int code, String message) {
+        super(message);
+        this.status = status;
+        this.code = code;
+    }
+
+    public ServiceAuthException(String status, int code, Throwable throwable) {
+        super(throwable.getMessage());
+        this.status = status;
+        this.code = code;
+    }
+
+    public ServiceAuthException(String status, int code, String message, Throwable throwable) {
+        super(message, throwable);
+        this.status = status;
+        this.code = code;
+    }
+
+    public String getStatus() {
+        return status;
+    }
+
+    public void setStatus(String status) {
+        this.status = status;
+    }
+
+    public int getCode() {
+        return code;
+    }
+
+    public void setCode(int code) {
+        this.code = code;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceExceptionHandlerCode.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceExceptionHandlerCode.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceExceptionHandlerCode.java
new file mode 100644
index 0000000..900e82a
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceExceptionHandlerCode.java
@@ -0,0 +1,56 @@
+/*
+ * 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.rocketmq.rpc.impl.exception;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.rocketmq.remoting.api.exception.RemoteConnectFailureException;
+import org.apache.rocketmq.remoting.api.exception.RemoteTimeoutException;
+import org.apache.rocketmq.rpc.impl.command.ResponseCode;
+
+public class ServiceExceptionHandlerCode extends ResponseCode {
+
+    private static Map<Class, Integer> exceptionCodeMap = new HashMap<>();
+
+    static {
+        exceptionCodeMap.put(IllegalAccessException.class, 100);
+        exceptionCodeMap.put(IllegalArgumentException.class, 101);
+        exceptionCodeMap.put(NullPointerException.class, 102);
+        exceptionCodeMap.put(InstantiationException.class, 104);
+        exceptionCodeMap.put(NumberFormatException.class, 105);
+        exceptionCodeMap.put(RemoteTimeoutException.class, 106);
+        exceptionCodeMap.put(RemoteConnectFailureException.class, 107);
+        exceptionCodeMap.put(ClassNotFoundException.class, 402);
+    }
+
+    public static int searchExceptionCode(Class type) {
+        Integer code = exceptionCodeMap.get(type);
+        if (code == null)
+            //Default exception code
+            return 100;
+        return code;
+    }
+
+    public static ResponseStatus searchResponseStatus(int code) {
+        for (ResponseStatus responseStatus : ResponseStatus.values()) {
+            if (code == responseStatus.getResponseCode())
+                return responseStatus;
+        }
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceExceptionHandlerManager.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceExceptionHandlerManager.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceExceptionHandlerManager.java
new file mode 100644
index 0000000..c5bc599
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceExceptionHandlerManager.java
@@ -0,0 +1,133 @@
+/*
+ * 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.rocketmq.rpc.impl.exception;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.remoting.api.serializable.Serializer;
+import org.apache.rocketmq.remoting.api.serializable.SerializerFactory;
+import org.apache.rocketmq.rpc.impl.command.ResponseCode;
+import org.apache.rocketmq.rpc.internal.RpcErrorMapper;
+
+import static org.apache.rocketmq.remoting.internal.ExceptionUtils.getStackTrace;
+
+public class ServiceExceptionHandlerManager {
+    private static boolean isIllegalCode(int code) {
+        return code > 0;
+    }
+
+    private static ServiceExceptionInvokeMessage getInvokeExceptionMessage(RemotingCommand remotingCommand,
+        SerializerFactory serializerFactory) {
+        Serializer serialization = serializerFactory.get(remotingCommand.serializerType());
+        if (remotingCommand.parameterBytes() == null)
+            return null;
+        return serialization.decode(remotingCommand.parameterBytes(), ServiceExceptionInvokeMessage.class);
+    }
+
+    private static Exception getResponseException(String className, RemotingCommand remotingCommand,
+        SerializerFactory serializerFactory)
+        throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException,
+        InstantiationException, InvocationTargetException {
+        return parseException(serializerFactory, remotingCommand, className);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static Exception parseException(SerializerFactory serializerFactory, RemotingCommand remotingCommand,
+        String className)
+        throws ClassNotFoundException, NoSuchMethodException,
+        IllegalAccessException, InvocationTargetException, InstantiationException {
+        Class exceptionClass = Class.forName(className);
+        assert exceptionClass != null;
+        Class[] oneParamTypes = {String.class};
+        Class[] twoParamTypes = {String.class, Throwable.class};
+        ServiceExceptionInvokeMessage serviceExceptionInvokeMessage = getInvokeExceptionMessage(remotingCommand, serializerFactory);
+        if (serviceExceptionInvokeMessage == null)
+            return new Exception();
+
+        if (serviceExceptionInvokeMessage.getThrowable() == null) {
+            Object[] onsParams = {serviceExceptionInvokeMessage.getErrorMessage()};
+            Constructor constructor = exceptionClass.getConstructor(oneParamTypes);
+            assert constructor != null;
+            return (Exception) constructor.newInstance(onsParams);
+        } else {
+            Object[] twoParams = {serviceExceptionInvokeMessage.getErrorMessage(), serviceExceptionInvokeMessage.getThrowable()};
+            Constructor constructor = exceptionClass.getConstructor(twoParamTypes);
+            assert constructor != null;
+            return (Exception) constructor.newInstance(twoParams);
+        }
+    }
+
+    public static void exceptionHandler(String code, RemotingCommand remotingCommand,
+        SerializerFactory serializerFactory) throws Exception {
+        int intCode;
+        try {
+            intCode = Integer.parseInt(code);
+        } catch (NumberFormatException e) {
+            throw ServiceExceptionManager.TranslateException(RpcErrorMapper.RpcErrorLatitude.REMOTE.getCode(), e);
+        }
+
+        if (isIllegalCode(intCode)) {
+            ServiceExceptionInvokeMessage returnResult = getInvokeExceptionMessage(remotingCommand, serializerFactory);
+            if (returnResult == null) {
+                ResponseCode.ResponseStatus responseStatus = ServiceExceptionHandlerCode.searchResponseStatus(intCode);
+                if (responseStatus != null) {
+                    throw new ServiceRuntimeException(RpcErrorMapper.RpcErrorLatitude.REMOTE.getCode(),
+                        String.valueOf(responseStatus.getResponseCode()));
+                } else {
+                    throw new ServiceRuntimeException(RpcErrorMapper.RpcErrorLatitude.REMOTE.getCode(),
+                        ResponseCode.UNKNOWN_EXCEPTION);
+                }
+            }
+
+            try {
+                Exception exception = getResponseException(returnResult.getClassFullName(), remotingCommand, serializerFactory);
+                if (intCode == Integer.valueOf(ResponseCode.USER_SERVICE_EXCEPTION)) {
+                    throw exception;
+                } else {
+                    throw new ServiceRuntimeException(RpcErrorMapper.RpcErrorLatitude.REMOTE.getCode(),
+                        String.valueOf(intCode),
+                        getStackTrace(exception));
+                }
+            } catch (ClassNotFoundException e) {
+                throw new ServiceRuntimeException(RpcErrorMapper.RpcErrorLatitude.LOCAL.getCode(),
+                    String.valueOf(ResponseCode.USER_EXCEPTION_CLASS_NOT_FOUND),
+                    e.getMessage());
+            } catch (NoSuchMethodException e) {
+                throw new ServiceRuntimeException(RpcErrorMapper.RpcErrorLatitude.REMOTE.getCode(),
+                    String.valueOf(ResponseCode.USER_EXCEPTION_METHOD_NOT_FOUND),
+                    e.getMessage());
+            } catch (IllegalAccessException e) {
+                throw new ServiceRuntimeException(RpcErrorMapper.RpcErrorLatitude.REMOTE.getCode(),
+                    String.valueOf(ResponseCode.ILLEGAL_ACCESS),
+                    e.getMessage());
+            } catch (InstantiationException e) {
+                throw new ServiceRuntimeException(RpcErrorMapper.RpcErrorLatitude.REMOTE.getCode(),
+                    String.valueOf(ResponseCode.INSTANTIATED_FAIL),
+                    e.getMessage());
+            } catch (InvocationTargetException e) {
+                throw new ServiceRuntimeException(RpcErrorMapper.RpcErrorLatitude.REMOTE.getCode(),
+                    String.valueOf(ResponseCode.FAIL_INVOKE),
+                    e.getMessage());
+            }
+        } else {
+            throw new ServiceRuntimeException(RpcErrorMapper.RpcErrorLatitude.REMOTE.getCode(),
+                ResponseCode.UNKNOWN_EXCEPTION);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceExceptionInvokeMessage.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceExceptionInvokeMessage.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceExceptionInvokeMessage.java
new file mode 100644
index 0000000..6f77ca9
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceExceptionInvokeMessage.java
@@ -0,0 +1,50 @@
+/*
+ * 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.rocketmq.rpc.impl.exception;
+
+import java.io.Serializable;
+
+public class ServiceExceptionInvokeMessage implements Serializable {
+    private String classFullName;
+    private String errorMessage;
+    private Throwable throwable;
+
+    public String getClassFullName() {
+        return classFullName;
+    }
+
+    public void setClassFullName(final String classFullName) {
+        this.classFullName = classFullName;
+    }
+
+    public String getErrorMessage() {
+        return errorMessage;
+    }
+
+    public void setErrorMessage(final String errorMessage) {
+        this.errorMessage = errorMessage;
+    }
+
+    public Throwable getThrowable() {
+        return throwable;
+    }
+
+    public void setThrowable(Throwable throwable) {
+        this.throwable = throwable;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceExceptionManager.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceExceptionManager.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceExceptionManager.java
new file mode 100644
index 0000000..721dd2a
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceExceptionManager.java
@@ -0,0 +1,32 @@
+/*
+ * 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.rocketmq.rpc.impl.exception;
+
+public class ServiceExceptionManager {
+    public static ServiceRuntimeException TranslateException(String code, Throwable e) {
+        if (e instanceof ServiceRuntimeException)
+            return (ServiceRuntimeException) e;
+        int errorCode = ServiceExceptionHandlerCode.searchExceptionCode(e.getClass());
+        /*
+        int errorCode = ServiceExceptionHandlerCode.searchExceptionCode(e.getClass());
+        if (errorCode == 100)
+            return new ServiceRuntimeException(code, String.valueOf(errorCode), e.getMessage());
+        */
+        return new ServiceRuntimeException(code, String.valueOf(errorCode), e);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceRuntimeException.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceRuntimeException.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceRuntimeException.java
new file mode 100644
index 0000000..8716f5c
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceRuntimeException.java
@@ -0,0 +1,87 @@
+/*
+ * 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.rocketmq.rpc.impl.exception;
+
+import org.apache.rocketmq.remoting.api.exception.RemoteAccessException;
+import org.apache.rocketmq.rpc.impl.command.ResponseCode;
+
+public class ServiceRuntimeException extends RemoteAccessException {
+    private static final String URL = "http://jukola.alibaba.com";
+    private String code;
+    private String helpUrl;
+    private String exMsg;
+
+    public ServiceRuntimeException(String latitude, String code) {
+        this(latitude, code, "");
+    }
+
+    public ServiceRuntimeException(String latitude, String code, Throwable e) {
+        this(latitude, code, e.getMessage());
+        if (e.getCause() != null)
+            this.setStackTrace(e.getCause().getStackTrace());
+        else
+            this.setStackTrace(e.getStackTrace());
+    }
+
+    public ServiceRuntimeException(String latitude, String code, String msg) {
+        super(buildExceptionMsg(latitude, code, msg));
+        this.code = code;
+        this.exMsg = msg;
+        this.helpUrl = buildHelpUrl(latitude, code);
+    }
+
+    private static String buildExceptionMsg(String latitude, String code, String msg) {
+        StringBuilder sb = new StringBuilder();
+        ResponseCode.ResponseStatus responseStatus = ServiceExceptionHandlerCode.searchResponseStatus(Integer.valueOf(code));
+        String helpUrl = buildHelpUrl(latitude, code);
+        if (responseStatus != null)
+            if (msg != null && !msg.isEmpty())
+                sb.append(msg).append(", see for more ").append(helpUrl);
+            else
+                sb.append(responseStatus.getResponseSimpleMessage()).append(", see for more ").append(helpUrl);
+        return sb.toString();
+    }
+
+    private static String buildHelpUrl(String latitude, String code) {
+        return URL + "/" + latitude + "/" + code;
+    }
+
+    public String getCode() {
+        return code;
+    }
+
+    public void setCode(String code) {
+        this.code = code;
+    }
+
+    public String getHelpUrl() {
+        return helpUrl;
+    }
+
+    public void setHelpUrl(String helpUrl) {
+        this.helpUrl = helpUrl;
+    }
+
+    public String getExMsg() {
+        return exMsg;
+    }
+
+    public void setExMsg(String exMsg) {
+        this.exMsg = exMsg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceSignatureException.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceSignatureException.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceSignatureException.java
new file mode 100644
index 0000000..ce4870f
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/exception/ServiceSignatureException.java
@@ -0,0 +1,47 @@
+/*
+ * 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.rocketmq.rpc.impl.exception;
+
+import org.apache.rocketmq.remoting.api.exception.RemoteAccessException;
+
+public class ServiceSignatureException extends RemoteAccessException {
+
+    private static final long serialVersionUID = -3662598055526208602L;
+    private final int code;
+
+    public ServiceSignatureException(int code) {
+        super("");
+        this.code = code;
+    }
+
+    public ServiceSignatureException(int code, String message, Throwable cause) {
+        super(message, cause);
+        this.code = code;
+    }
+
+    public ServiceSignatureException(int code, String message) {
+        super(message);
+        this.code = code;
+    }
+
+    public ServiceSignatureException(int code, Throwable cause) {
+        super(cause.getMessage());
+        this.code = code;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/DefaultServiceAPI.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/DefaultServiceAPI.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/DefaultServiceAPI.java
new file mode 100644
index 0000000..25db808
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/DefaultServiceAPI.java
@@ -0,0 +1,42 @@
+/*
+ * 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.rocketmq.rpc.impl.metrics;
+
+import org.apache.rocketmq.rpc.annotation.RemoteMethod;
+import org.apache.rocketmq.rpc.annotation.RemoteService;
+
+@RemoteService(name = "DefaultServiceAPI", version = "1.0.0")
+public interface DefaultServiceAPI {
+    @RemoteMethod(name = "collectStackAll", version = "1.0.0")
+    StackTracesAll collectStackAll();
+
+    @RemoteMethod(name = "collectStatsAll", version = "1.0.0")
+    StatsAll collectStatsAll();
+
+    @RemoteMethod(name = "list", version = "1.0.0")
+    void list();
+
+    @RemoteMethod(name = "trace", version = "1.0.0")
+    void trace();
+
+    @RemoteMethod(name = "ping", version = "1.0.0")
+    void ping();
+
+    @RemoteMethod(name = "metrics", version = "1.0.0")
+    void metrics();
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/DefaultServiceAPIImpl.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/DefaultServiceAPIImpl.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/DefaultServiceAPIImpl.java
new file mode 100644
index 0000000..eb45d60
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/DefaultServiceAPIImpl.java
@@ -0,0 +1,59 @@
+/*
+ * 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.rocketmq.rpc.impl.metrics;
+
+import java.util.TreeMap;
+
+public class DefaultServiceAPIImpl implements DefaultServiceAPI {
+    private final ServiceStats serviceStats;
+    private final ThreadStats threadStats;
+
+    public DefaultServiceAPIImpl(ServiceStats serviceStats, ThreadStats threadStats) {
+        this.serviceStats = serviceStats;
+        this.threadStats = threadStats;
+    }
+
+    public StackTracesAll collectStackAll() {
+        StackTracesAll sta = new StackTracesAll();
+        sta.setStackTraces(UtilAll.jstack());
+        return sta;
+    }
+
+    public StatsAll collectStatsAll() {
+        StatsAll statsAll = serviceStats.stats();
+        TreeMap<Threading, TimestampRegion> tm = this.threadStats.cloneStatsTable();
+        statsAll.setStatsThreading(tm);
+        return statsAll;
+    }
+
+    public void list() {
+
+    }
+
+    public void trace() {
+
+    }
+
+    public void ping() {
+
+    }
+
+    public void metrics() {
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/DownloadFileRequest.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/DownloadFileRequest.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/DownloadFileRequest.java
new file mode 100644
index 0000000..dca7d5e
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/DownloadFileRequest.java
@@ -0,0 +1,56 @@
+/*
+ * 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.rocketmq.rpc.impl.metrics;
+
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+
+public class DownloadFileRequest {
+    private String fileFullName;
+    private int offset;
+    private int size;
+
+    public String getFileFullName() {
+        return fileFullName;
+    }
+
+    public void setFileFullName(String fileFullName) {
+        this.fileFullName = fileFullName;
+    }
+
+    public int getOffset() {
+        return offset;
+    }
+
+    public void setOffset(int offset) {
+        this.offset = offset;
+    }
+
+    public int getSize() {
+        return size;
+    }
+
+    public void setSize(int size) {
+        this.size = size;
+    }
+
+    @Override
+    public String toString() {
+        return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/ExecuteResult.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/ExecuteResult.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/ExecuteResult.java
new file mode 100644
index 0000000..1eb795f
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/ExecuteResult.java
@@ -0,0 +1,57 @@
+/*
+ * 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.rocketmq.rpc.impl.metrics;
+
+public class ExecuteResult {
+    private int ret;
+    private String stdout;
+    private String stderr;
+
+    public int getRet() {
+        return ret;
+    }
+
+    public void setRet(int ret) {
+        this.ret = ret;
+    }
+
+    public String getStdout() {
+        return stdout;
+    }
+
+    public void setStdout(String stdout) {
+        this.stdout = stdout;
+    }
+
+    public String getStderr() {
+        return stderr;
+    }
+
+    public void setStderr(String stderr) {
+        this.stderr = stderr;
+    }
+
+    @Override
+    public String toString() {
+        return "ExecuteResult{" +
+            "ret=" + ret +
+            ", stdout='" + stdout + '\'' +
+            ", stderr='" + stderr + '\'' +
+            '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/MethodStats.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/MethodStats.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/MethodStats.java
new file mode 100644
index 0000000..175d5a7
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/MethodStats.java
@@ -0,0 +1,92 @@
+/*
+ * 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.rocketmq.rpc.impl.metrics;
+
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+
+public class MethodStats {
+    private double qpsOK;
+    private long failedTimesInMinutes;
+    private double rtAvgInMinutes;
+    private long rtMaxInMinutes;
+    private long rtMaxIn10Minutes;
+    private long rtMaxInHour;
+    private long[] rtRegion;
+
+    public double getQpsOK() {
+        return qpsOK;
+    }
+
+    public void setQpsOK(double qpsOK) {
+        this.qpsOK = qpsOK;
+    }
+
+    public long getFailedTimesInMinutes() {
+        return failedTimesInMinutes;
+    }
+
+    public void setFailedTimesInMinutes(long failedTimesInMinutes) {
+        this.failedTimesInMinutes = failedTimesInMinutes;
+    }
+
+    public double getRtAvgInMinutes() {
+        return rtAvgInMinutes;
+    }
+
+    public void setRtAvgInMinutes(double rtAvgInMinutes) {
+        this.rtAvgInMinutes = rtAvgInMinutes;
+    }
+
+    public long getRtMaxInMinutes() {
+        return rtMaxInMinutes;
+    }
+
+    public void setRtMaxInMinutes(long rtMaxInMinutes) {
+        this.rtMaxInMinutes = rtMaxInMinutes;
+    }
+
+    public long getRtMaxIn10Minutes() {
+        return rtMaxIn10Minutes;
+    }
+
+    public void setRtMaxIn10Minutes(long rtMaxIn10Minutes) {
+        this.rtMaxIn10Minutes = rtMaxIn10Minutes;
+    }
+
+    public long getRtMaxInHour() {
+        return rtMaxInHour;
+    }
+
+    public void setRtMaxInHour(long rtMaxInHour) {
+        this.rtMaxInHour = rtMaxInHour;
+    }
+
+    public long[] getRtRegion() {
+        return rtRegion;
+    }
+
+    public void setRtRegion(long[] rtRegion) {
+        this.rtRegion = rtRegion;
+    }
+
+    @Override
+    public String toString() {
+        return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/ServiceStats.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/ServiceStats.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/ServiceStats.java
new file mode 100644
index 0000000..20026a9
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/ServiceStats.java
@@ -0,0 +1,125 @@
+/*
+ * 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.rocketmq.rpc.impl.metrics;
+
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.rocketmq.remoting.external.ThreadUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ServiceStats {
+    private static final Logger log = LoggerFactory.getLogger("ServiceStats");
+    private final StatsItemSet callerOKQPS;
+    private final StatsItemSet callerFailedQPS;
+    private final StatsItemSet callerRT;
+    private final StatsItemSet providerOKQPS;
+    private final StatsItemSet providerFailedQPS;
+    private final StatsItemSet providerRT;
+    private final ScheduledExecutorService scheduledExecutorService = ThreadUtils.newSingleThreadScheduledExecutor("ServiceStatsScheduledThread", true);
+
+    public ServiceStats() {
+        this.callerOKQPS = new StatsItemSet("SERVICE_QPS_CALLER_OK", this.scheduledExecutorService, log);
+        this.callerFailedQPS = new StatsItemSet("SERVICE_QPS_CALLER_FAILED", this.scheduledExecutorService, log);
+        this.callerRT = new StatsItemSet("SERVICE_RT_CALLER", this.scheduledExecutorService, log);
+        this.providerOKQPS = new StatsItemSet("SERVICE_QPS_PROVIDER_OK", this.scheduledExecutorService, log);
+        this.providerFailedQPS = new StatsItemSet("SERVICE_QPS_PROVIDER_FAILED", this.scheduledExecutorService, log);
+        this.providerRT = new StatsItemSet("SERVICE_RT_PROVIDER", this.scheduledExecutorService, log);
+    }
+
+    private static MethodStats methodStats(StatsItem itemOK,
+        StatsItem itemFailed,
+        StatsItem itemRT) {
+        MethodStats methodStats = new MethodStats();
+        if (itemOK != null) {
+            methodStats.setQpsOK(itemOK.getStatsDataInMinute().getTps());
+        }
+        if (itemFailed != null) {
+            methodStats.setFailedTimesInMinutes(itemFailed.getStatsDataInMinute().getSum());
+        }
+        if (itemRT != null) {
+            methodStats.setRtAvgInMinutes(itemRT.getStatsDataInMinute().getAvgpt());
+            methodStats.setRtMaxInMinutes(itemRT.getValueMaxInMinutes().get());
+            methodStats.setRtMaxIn10Minutes(itemRT.getValueMaxIn10Minutes().get());
+            methodStats.setRtMaxInHour(itemRT.getValueMaxInHour().get());
+            methodStats.setRtRegion(itemRT.valueRegion());
+        }
+        return methodStats;
+    }
+
+    public void addCallerOKQPSValue(final String statsKey, final int incValue, final int incTimes) {
+        this.callerOKQPS.addValue(statsKey, incValue, incTimes);
+    }
+
+    public void addCallerFailedQPSValue(final String statsKey, final int incValue, final int incTimes) {
+        this.callerFailedQPS.addValue(statsKey, incValue, incTimes);
+    }
+
+    public void addCallerRTValue(final String statsKey, final int incValue, final int incTimes) {
+        this.callerRT.addValue(statsKey, incValue, incTimes);
+    }
+
+    public void addProviderOKQPSValue(final String statsKey, final int incValue, final int incTimes) {
+        this.providerOKQPS.addValue(statsKey, incValue, incTimes);
+    }
+
+    public void addProviderFailedQPSValue(final String statsKey, final int incValue, final int incTimes) {
+        this.providerFailedQPS.addValue(statsKey, incValue, incTimes);
+    }
+
+    public void addProviderRTValue(final String statsKey, final int incValue, final int incTimes) {
+        this.providerRT.addValue(statsKey, incValue, incTimes);
+    }
+
+    public void start() {
+        this.callerOKQPS.init();
+        this.callerFailedQPS.init();
+        this.callerRT.init();
+        this.providerOKQPS.init();
+        this.providerFailedQPS.init();
+        this.providerRT.init();
+    }
+
+    public void stop() {
+        ThreadUtils.shutdownGracefully(this.scheduledExecutorService, 3000, TimeUnit.MILLISECONDS);
+    }
+
+    StatsAll stats() {
+        StatsAll sa = new StatsAll();
+        Set<String> keySetCaller = this.callerOKQPS.getStatsItemTable().keySet();
+        Set<String> keySetProvider = this.providerOKQPS.getStatsItemTable().keySet();
+
+        for (String statsKey : keySetCaller) {
+            MethodStats caller = methodStats(this.callerOKQPS.getStatsItemTable().get(statsKey),
+                this.callerFailedQPS.getStatsItemTable().get(statsKey),
+                this.callerRT.getStatsItemTable().get(statsKey)
+            );
+            sa.getStatsCaller().put(statsKey, caller);
+        }
+
+        for (String statsKey : keySetProvider) {
+            MethodStats provider = methodStats(this.providerOKQPS.getStatsItemTable().get(statsKey),
+                this.providerFailedQPS.getStatsItemTable().get(statsKey),
+                this.providerRT.getStatsItemTable().get(statsKey)
+            );
+            sa.getStatsProvider().put(statsKey, provider);
+        }
+        return sa;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/StackTracesAll.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/StackTracesAll.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/StackTracesAll.java
new file mode 100644
index 0000000..1057187
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/StackTracesAll.java
@@ -0,0 +1,38 @@
+/*
+ * 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.rocketmq.rpc.impl.metrics;
+
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+
+public class StackTracesAll {
+    private String stackTraces;
+
+    public String getStackTraces() {
+        return stackTraces;
+    }
+
+    public void setStackTraces(String stackTraces) {
+        this.stackTraces = stackTraces;
+    }
+
+    @Override
+    public String toString() {
+        return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/StatsAll.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/StatsAll.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/StatsAll.java
new file mode 100644
index 0000000..7a40285
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/StatsAll.java
@@ -0,0 +1,60 @@
+/*
+ * 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.rocketmq.rpc.impl.metrics;
+
+import java.util.TreeMap;
+
+public class StatsAll {
+    private TreeMap<String/* request code */, MethodStats> statsCaller = new TreeMap<String, MethodStats>();
+    private TreeMap<String/* request code */, MethodStats> statsProvider = new TreeMap<String, MethodStats>();
+    private TreeMap<Threading, TimestampRegion> statsThreading = new TreeMap<Threading, TimestampRegion>();
+
+    public TreeMap<String, MethodStats> getStatsCaller() {
+        return statsCaller;
+    }
+
+    public void setStatsCaller(TreeMap<String, MethodStats> statsCaller) {
+        this.statsCaller = statsCaller;
+    }
+
+    public TreeMap<String, MethodStats> getStatsProvider() {
+        return statsProvider;
+    }
+
+    public void setStatsProvider(TreeMap<String, MethodStats> statsProvider) {
+        this.statsProvider = statsProvider;
+    }
+
+    public TreeMap<Threading, TimestampRegion> getStatsThreading() {
+        return statsThreading;
+    }
+
+    public void setStatsThreading(TreeMap<Threading, TimestampRegion> statsThreading) {
+        this.statsThreading = statsThreading;
+    }
+
+    @Override
+    public String toString() {
+        return "StatsAll{" +
+            "statsCaller=" + statsCaller +
+            ", statsProvider=" + statsProvider +
+            ", statsThreading=" + statsThreading +
+            '}';
+    }
+
+}


[6/8] incubator-rocketmq git commit: initialize RocketMQ5

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/channel/ChannelHandlerContextWrapperImpl.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/channel/ChannelHandlerContextWrapperImpl.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/channel/ChannelHandlerContextWrapperImpl.java
new file mode 100644
index 0000000..bbd33ea
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/channel/ChannelHandlerContextWrapperImpl.java
@@ -0,0 +1,33 @@
+/*
+ * 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.rocketmq.remoting.impl.channel;
+
+import org.apache.rocketmq.remoting.api.channel.ChannelHandlerContextWrapper;
+
+public class ChannelHandlerContextWrapperImpl<ChannelHandlerContext> implements ChannelHandlerContextWrapper {
+
+    private io.netty.channel.ChannelHandlerContext context;
+
+    public ChannelHandlerContextWrapperImpl(io.netty.channel.ChannelHandlerContext context) {
+        this.context = context;
+    }
+
+    public io.netty.channel.ChannelHandlerContext getContext() {
+        return context;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/channel/FileRegionImpl.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/channel/FileRegionImpl.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/channel/FileRegionImpl.java
new file mode 100644
index 0000000..b90afc1
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/channel/FileRegionImpl.java
@@ -0,0 +1,85 @@
+/*
+ * 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.rocketmq.remoting.impl.channel;
+
+import io.netty.channel.FileRegion;
+import io.netty.util.AbstractReferenceCounted;
+import java.io.IOException;
+import java.nio.channels.WritableByteChannel;
+import org.apache.rocketmq.remoting.api.channel.ChunkRegion;
+
+public class FileRegionImpl extends AbstractReferenceCounted implements FileRegion {
+    private final ChunkRegion chunkRegion;
+
+    public FileRegionImpl(ChunkRegion chunkRegion) {
+        this.chunkRegion = chunkRegion;
+    }
+
+    @Override
+    public long position() {
+        return chunkRegion.position();
+    }
+
+    @Override
+    public long transfered() {
+        return chunkRegion.transferred();
+    }
+
+    @Override
+    public long transferred() {
+        return chunkRegion.transferred();
+    }
+
+    @Override
+    public long count() {
+        return chunkRegion.count();
+    }
+
+    @Override
+    public long transferTo(WritableByteChannel target, long position) throws IOException {
+        return chunkRegion.transferTo(target, position);
+    }
+
+    @Override
+    protected void deallocate() {
+        chunkRegion.release();
+    }
+
+    @Override
+    public FileRegion retain() {
+        super.retain();
+        return this;
+    }
+
+    @Override
+    public FileRegion retain(int increment) {
+        super.retain(increment);
+        return this;
+    }
+
+    @Override
+    public FileRegion touch() {
+        return this;
+    }
+
+    @Override
+    public FileRegion touch(Object hint) {
+        return this;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/channel/NettyChannelImpl.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/channel/NettyChannelImpl.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/channel/NettyChannelImpl.java
new file mode 100644
index 0000000..ba4a969
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/channel/NettyChannelImpl.java
@@ -0,0 +1,94 @@
+/*
+ * 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.rocketmq.remoting.impl.channel;
+
+import io.netty.channel.Channel;
+import java.net.SocketAddress;
+import org.apache.rocketmq.remoting.api.channel.ChunkRegion;
+import org.apache.rocketmq.remoting.api.channel.RemotingChannel;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+
+public class NettyChannelImpl implements RemotingChannel {
+    private final io.netty.channel.Channel channel;
+
+    public NettyChannelImpl(Channel channel) {
+        this.channel = channel;
+    }
+
+    @Override
+    public SocketAddress localAddress() {
+        return channel.localAddress();
+    }
+
+    @Override
+    public SocketAddress remoteAddress() {
+        return channel.remoteAddress();
+    }
+
+    @Override
+    public boolean isWritable() {
+        return channel.isWritable();
+    }
+
+    @Override
+    public boolean isActive() {
+        return channel.isActive();
+    }
+
+    @Override
+    public void close() {
+        channel.close();
+    }
+
+    @Override
+    public void reply(final RemotingCommand command) {
+        channel.writeAndFlush(command);
+    }
+
+    @Override
+    public void reply(final ChunkRegion fileRegion) {
+        channel.writeAndFlush(fileRegion);
+    }
+
+    public io.netty.channel.Channel getChannel() {
+        return channel;
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        final NettyChannelImpl that = (NettyChannelImpl) o;
+
+        return channel != null ? channel.equals(that.channel) : that.channel == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        return channel != null ? channel.hashCode() : 0;
+    }
+
+    @Override
+    public String toString() {
+        return "NettyChannelImpl [channel=" + channel + "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/command/CodecHelper.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/command/CodecHelper.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/command/CodecHelper.java
new file mode 100644
index 0000000..44d4fd9
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/command/CodecHelper.java
@@ -0,0 +1,180 @@
+/*
+ * 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.rocketmq.remoting.impl.command;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.Map.Entry;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.remoting.api.command.TrafficType;
+import org.apache.rocketmq.remoting.api.exception.RemoteCodecException;
+
+public class CodecHelper {
+    //ProtocolType + TotalLength + RequestId + SerializeType + TrafficType + CodeLength + RemarkLength + PropertiesSize + ParameterLength
+    public final static int MIN_PROTOCOL_LEN = 1 + 4 + 4 + 1 + 1 + 2 + 2 + 2 + 4;
+    public final static char PROPERTY_SEPARATOR = '\n';
+    public final static Charset REMOTING_CHARSET = Charset.forName("UTF-8");
+
+    public final static int CODE_MAX_LEN = 512;
+    public final static int PARAMETER_MAX_LEN = 33554432;
+    public final static int BODY_MAX_LEN = 33554432;
+    public final static int PACKET_MAX_LEN = 33554432;
+
+    public static ByteBuffer encodeHeader(final RemotingCommand command, final int parameterLength,
+        final int extraPayload) {
+        byte[] code = command.opCode().getBytes(REMOTING_CHARSET);
+        int codeLength = code.length;
+
+        byte[] remark = command.remark().getBytes(REMOTING_CHARSET);
+        int remarkLength = remark.length;
+
+        byte[][] props = null;
+        int propsLength = 0;
+        StringBuilder sb = new StringBuilder();
+        if (!command.properties().isEmpty()) {
+            props = new byte[command.properties().size()][];
+            int i = 0;
+            for (Entry<String, String> next : command.properties().entrySet()) {
+                sb.setLength(0);
+                sb.append(next.getKey());
+                sb.append(PROPERTY_SEPARATOR);
+                sb.append(next.getValue());
+
+                props[i] = sb.toString().getBytes(REMOTING_CHARSET);
+
+                propsLength += 2;
+                propsLength += props[i].length;
+                i++;
+            }
+        }
+
+        int totalLength = MIN_PROTOCOL_LEN - 1 - 4
+            + codeLength
+            + remarkLength
+            + propsLength
+            + parameterLength
+            + extraPayload;
+
+        int headerLength = 1 + 4 + totalLength - parameterLength - extraPayload;
+
+        ByteBuffer buf = ByteBuffer.allocate(headerLength);
+        buf.put(command.protocolType());
+        buf.putInt(totalLength);
+        buf.putInt(command.requestID());
+        buf.put(command.serializerType());
+        buf.put((byte) command.trafficType().ordinal());
+
+        buf.putShort((short) codeLength);
+        if (codeLength > 0) {
+            buf.put(code);
+        }
+        buf.putShort((short) remarkLength);
+        if (remarkLength > 0) {
+            buf.put(remark);
+        }
+        if (props != null) {
+            buf.putShort((short) props.length);
+            for (byte[] prop : props) {
+                buf.putShort((short) prop.length);
+                buf.put(prop);
+            }
+        } else {
+            buf.putShort((short) 0);
+        }
+
+        buf.putInt(parameterLength);
+
+        buf.flip();
+
+        return buf;
+    }
+
+    public static RemotingCommand decode(final ByteBuffer byteBuffer) {
+        RemotingCommandImpl cmd = new RemotingCommandImpl();
+        int totalLength = byteBuffer.limit();
+        cmd.requestID(byteBuffer.getInt());
+        cmd.serializerType(byteBuffer.get());
+        cmd.trafficType(TrafficType.parse(byteBuffer.get()));
+
+        {
+            short size = byteBuffer.getShort();
+            if (size > 0 && size <= CODE_MAX_LEN) {
+                byte[] bytes = new byte[size];
+                byteBuffer.get(bytes);
+                String str = new String(bytes, REMOTING_CHARSET);
+                cmd.opCode(str);
+            } else {
+                throw new RemoteCodecException(String.format("Code length: %d over max limit: %d", size, CODE_MAX_LEN));
+            }
+        }
+
+        {
+            short size = byteBuffer.getShort();
+            if (size > 0) {
+                byte[] bytes = new byte[size];
+                byteBuffer.get(bytes);
+                String str = new String(bytes, REMOTING_CHARSET);
+                cmd.remark(str);
+            }
+        }
+
+        {
+            short size = byteBuffer.getShort();
+            if (size > 0) {
+                for (int i = 0; i < size; i++) {
+                    short length = byteBuffer.getShort();
+                    if (length > 0) {
+                        byte[] bytes = new byte[length];
+                        byteBuffer.get(bytes);
+                        String str = new String(bytes, REMOTING_CHARSET);
+                        int index = str.indexOf(PROPERTY_SEPARATOR);
+                        if (index > 0) {
+                            String key = str.substring(0, index);
+                            String value = str.substring(index + 1);
+                            cmd.property(key, value);
+                        }
+                    }
+                }
+            }
+        }
+
+        {
+            int size = byteBuffer.getInt();
+            if (size > 0 && size <= PARAMETER_MAX_LEN) {
+                byte[] bytes = new byte[size];
+                byteBuffer.get(bytes);
+                cmd.parameterBytes(bytes);
+            } else if (size != 0) {
+                throw new RemoteCodecException(String.format("Parameter size: %d over max limit: %d", size, PARAMETER_MAX_LEN));
+            }
+        }
+
+        {
+            int size = totalLength - byteBuffer.position();
+            if (size > 0 && size <= BODY_MAX_LEN) {
+                byte[] bytes = new byte[size];
+                byteBuffer.get(bytes);
+                cmd.extraPayload(bytes);
+            } else if (size != 0) {
+                throw new RemoteCodecException(String.format("Body size: %d over max limit: %d", size, BODY_MAX_LEN));
+            }
+        }
+
+        return cmd;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/command/RemotingCommandFactoryImpl.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/command/RemotingCommandFactoryImpl.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/command/RemotingCommandFactoryImpl.java
new file mode 100644
index 0000000..f5d2126
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/command/RemotingCommandFactoryImpl.java
@@ -0,0 +1,53 @@
+/*
+ * 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.rocketmq.remoting.impl.command;
+
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.remoting.api.command.RemotingCommandFactory;
+import org.apache.rocketmq.remoting.api.command.TrafficType;
+import org.apache.rocketmq.remoting.common.RemotingCommandFactoryMeta;
+
+public class RemotingCommandFactoryImpl implements RemotingCommandFactory {
+    private RemotingCommandFactoryMeta remotingCommandFactoryMeta;
+
+    public RemotingCommandFactoryImpl() {
+        this(new RemotingCommandFactoryMeta());
+    }
+
+    public RemotingCommandFactoryImpl(final RemotingCommandFactoryMeta remotingCommandFactoryMeta) {
+        this.remotingCommandFactoryMeta = remotingCommandFactoryMeta;
+    }
+
+    @Override
+    public RemotingCommand createRequest() {
+        RemotingCommand request = new RemotingCommandImpl();
+        request.protocolType(this.remotingCommandFactoryMeta.getProtocolType());
+        request.serializerType(this.remotingCommandFactoryMeta.getSerializeType());
+        return request;
+    }
+
+    @Override
+    public RemotingCommand createResponse(final RemotingCommand command) {
+        RemotingCommand response = new RemotingCommandImpl();
+        response.requestID(command.requestID());
+        response.protocolType(command.protocolType());
+        response.serializerType(command.serializerType());
+        response.trafficType(TrafficType.RESPONSE);
+        return response;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/command/RemotingCommandImpl.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/command/RemotingCommandImpl.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/command/RemotingCommandImpl.java
new file mode 100644
index 0000000..bcf2338
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/command/RemotingCommandImpl.java
@@ -0,0 +1,210 @@
+/*
+ * 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.rocketmq.remoting.impl.command;
+
+import java.lang.reflect.Type;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.remoting.api.command.TrafficType;
+import org.apache.rocketmq.remoting.api.serializable.SerializerFactory;
+import org.apache.rocketmq.remoting.common.TypePresentation;
+
+public class RemotingCommandImpl implements RemotingCommand {
+    public final static RequestIdGenerator REQUEST_ID_GENERATOR = RequestIdGenerator.inst;
+
+    private byte protocolType;
+    private byte serializeType;
+
+    private volatile int requestId = REQUEST_ID_GENERATOR.incrementAndGet();
+    private TrafficType trafficType = TrafficType.REQUEST_SYNC;
+    private String code = CommandFlag.SUCCESS.flag();
+    private String remark = "";
+    private Map<String, String> properties = new HashMap<String, String>();
+    private Object parameter;
+    private byte[] extraPayload;
+
+    private byte[] parameterByte;
+
+    protected RemotingCommandImpl() {
+    }
+
+    @Override
+    public byte protocolType() {
+        return this.protocolType;
+    }
+
+    @Override
+    public void protocolType(byte value) {
+        this.protocolType = value;
+    }
+
+    @Override
+    public int requestID() {
+        return requestId;
+    }
+
+    @Override
+    public void requestID(int value) {
+        this.requestId = value;
+    }
+
+    @Override
+    public byte serializerType() {
+        return this.serializeType;
+    }
+
+    @Override
+    public void serializerType(byte value) {
+        this.serializeType = value;
+    }
+
+    @Override
+    public TrafficType trafficType() {
+        return this.trafficType;
+    }
+
+    @Override
+    public void trafficType(TrafficType value) {
+        this.trafficType = value;
+    }
+
+    @Override
+    public String opCode() {
+        return this.code;
+    }
+
+    @Override
+    public void opCode(String value) {
+        this.code = value;
+    }
+
+    @Override
+    public String remark() {
+        return this.remark;
+    }
+
+    @Override
+    public void remark(String value) {
+        this.remark = value;
+    }
+
+    @Override
+    public Map<String, String> properties() {
+        return this.properties;
+    }
+
+    @Override
+    public void properties(Map<String, String> value) {
+        this.properties = value;
+    }
+
+    @Override
+    public String property(String key) {
+        return this.properties.get(key);
+    }
+
+    @Override
+    public void property(String key, String value) {
+        this.properties.put(key, value);
+    }
+
+    @Override
+    public Object parameter() {
+        return this.parameter;
+    }
+
+    @Override
+    public void parameter(Object value) {
+        this.parameter = value;
+    }
+
+    @Override
+    public byte[] parameterBytes() {
+        return this.getParameterByte();
+    }
+
+    public byte[] getParameterByte() {
+        return parameterByte;
+    }
+
+    public void setParameterByte(byte[] parameterByte) {
+        this.parameterByte = parameterByte;
+    }
+
+    @Override
+    public void parameterBytes(byte[] value) {
+        this.setParameterByte(value);
+    }
+
+    @Override
+    public byte[] extraPayload() {
+        return this.extraPayload;
+    }
+
+    @Override
+    public void extraPayload(byte[] value) {
+        this.extraPayload = value;
+    }
+
+    @Override
+    public <T> T parameter(SerializerFactory serializerFactory, Class<T> c) {
+        if (this.parameter() != null)
+            return (T) this.parameter();
+        final T decode = serializerFactory.get(this.serializerType()).decode(this.parameterBytes(), c);
+        this.parameter(decode);
+        return decode;
+    }
+
+    @Override
+    public <T> T parameter(SerializerFactory serializerFactory, TypePresentation<T> typePresentation) {
+        if (this.parameter() != null)
+            return (T) this.parameter();
+        final T decode = serializerFactory.get(this.serializerType()).decode(this.parameterBytes(), typePresentation);
+        this.parameter(decode);
+        return decode;
+    }
+
+    @Override
+    public <T> T parameter(SerializerFactory serializerFactory, Type type) {
+        if (this.parameter() != null)
+            return (T) this.parameter();
+        final T decode = serializerFactory.get(this.serializerType()).decode(this.parameterBytes(), type);
+        this.parameter(decode);
+        return decode;
+    }
+
+    @Override
+    public int hashCode() {
+        return HashCodeBuilder.reflectionHashCode(this);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        return EqualsBuilder.reflectionEquals(this, o);
+    }
+
+    @Override
+    public String toString() {
+        return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/command/RequestIdGenerator.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/command/RequestIdGenerator.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/command/RequestIdGenerator.java
new file mode 100644
index 0000000..9b85c95
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/command/RequestIdGenerator.java
@@ -0,0 +1,34 @@
+/*
+ * 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.rocketmq.remoting.impl.command;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class RequestIdGenerator {
+    public static RequestIdGenerator inst = new RequestIdGenerator();
+
+    private AtomicInteger generator = new AtomicInteger(0);
+
+    private RequestIdGenerator() {
+
+    }
+
+    public int incrementAndGet() {
+        return generator.incrementAndGet();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyChannelEvent.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyChannelEvent.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyChannelEvent.java
new file mode 100644
index 0000000..ec9cece
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyChannelEvent.java
@@ -0,0 +1,55 @@
+/*
+ * 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.rocketmq.remoting.impl.netty;
+
+import io.netty.channel.Channel;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+
+public class NettyChannelEvent {
+    private final Channel channel;
+    private final NettyChannelEventType type;
+    private final Throwable cause;
+
+    public NettyChannelEvent(NettyChannelEventType type, Channel channel) {
+        this(type, channel, null);
+    }
+
+    public NettyChannelEvent(NettyChannelEventType type, Channel channel, Throwable cause) {
+        this.type = type;
+        this.channel = channel;
+        this.cause = cause;
+    }
+
+    public NettyChannelEventType getType() {
+        return type;
+    }
+
+    public Channel getChannel() {
+        return channel;
+    }
+
+    public Throwable getCause() {
+        return cause;
+    }
+
+    @Override
+    public String toString() {
+        return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyChannelEventType.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyChannelEventType.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyChannelEventType.java
new file mode 100644
index 0000000..1bf2277
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyChannelEventType.java
@@ -0,0 +1,25 @@
+/*
+ * 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.rocketmq.remoting.impl.netty;
+
+public enum NettyChannelEventType {
+    ACTIVE,
+    INACTIVE,
+    IDLE,
+    EXCEPTION
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyRemotingAbstract.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyRemotingAbstract.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyRemotingAbstract.java
new file mode 100644
index 0000000..1af62cb
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyRemotingAbstract.java
@@ -0,0 +1,641 @@
+/*
+ * 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.rocketmq.remoting.impl.netty;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.SimpleChannelInboundHandler;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.rocketmq.remoting.api.AsyncHandler;
+import org.apache.rocketmq.remoting.api.RemotingEndPoint;
+import org.apache.rocketmq.remoting.api.RemotingService;
+import org.apache.rocketmq.remoting.api.RequestProcessor;
+import org.apache.rocketmq.remoting.api.channel.ChannelEventListener;
+import org.apache.rocketmq.remoting.api.channel.RemotingChannel;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.remoting.api.command.RemotingCommandFactory;
+import org.apache.rocketmq.remoting.api.command.TrafficType;
+import org.apache.rocketmq.remoting.api.exception.RemoteTimeoutException;
+import org.apache.rocketmq.remoting.api.interceptor.ExceptionContext;
+import org.apache.rocketmq.remoting.api.interceptor.Interceptor;
+import org.apache.rocketmq.remoting.api.interceptor.InterceptorGroup;
+import org.apache.rocketmq.remoting.api.interceptor.RequestContext;
+import org.apache.rocketmq.remoting.api.interceptor.ResponseContext;
+import org.apache.rocketmq.remoting.api.protocol.ProtocolFactory;
+import org.apache.rocketmq.remoting.api.serializable.Serializer;
+import org.apache.rocketmq.remoting.api.serializable.SerializerFactory;
+import org.apache.rocketmq.remoting.common.ChannelEventListenerGroup;
+import org.apache.rocketmq.remoting.common.Pair;
+import org.apache.rocketmq.remoting.common.RemotingCommandFactoryMeta;
+import org.apache.rocketmq.remoting.common.ResponseResult;
+import org.apache.rocketmq.remoting.common.SemaphoreReleaseOnlyOnce;
+import org.apache.rocketmq.remoting.config.RemotingConfig;
+import org.apache.rocketmq.remoting.external.ThreadUtils;
+import org.apache.rocketmq.remoting.impl.channel.NettyChannelImpl;
+import org.apache.rocketmq.remoting.impl.command.RemotingCommandFactoryImpl;
+import org.apache.rocketmq.remoting.impl.protocol.ProtocolFactoryImpl;
+import org.apache.rocketmq.remoting.impl.protocol.serializer.SerializerFactoryImpl;
+import org.apache.rocketmq.remoting.internal.UIDGenerator;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class NettyRemotingAbstract implements RemotingService {
+    protected static final Logger LOG = LoggerFactory.getLogger(NettyRemotingAbstract.class);
+    protected final ProtocolFactory protocolFactory = new ProtocolFactoryImpl();
+    protected final SerializerFactory serializerFactory = new SerializerFactoryImpl();
+    protected final ChannelEventExecutor channelEventExecutor = new ChannelEventExecutor("ChannelEventExecutor");
+    private final Semaphore semaphoreOneway;
+    private final Semaphore semaphoreAsync;
+    private final Map<Integer, ResponseResult> ackTables = new ConcurrentHashMap<Integer, ResponseResult>(256);
+    private final Map<String, Pair<RequestProcessor, ExecutorService>> processorTables = new ConcurrentHashMap<String, Pair<RequestProcessor, ExecutorService>>();
+    private final AtomicLong count = new AtomicLong(0);
+    private final RemotingCommandFactory remotingCommandFactory;
+    private final String remotingInstanceId = UIDGenerator.instance().createUID();
+
+    private final ExecutorService publicExecutor;
+    protected ScheduledExecutorService houseKeepingService = ThreadUtils.newSingleThreadScheduledExecutor("HouseKeepingService", true);
+    private InterceptorGroup interceptorGroup = new InterceptorGroup();
+    private ChannelEventListenerGroup channelEventListenerGroup = new ChannelEventListenerGroup();
+
+    NettyRemotingAbstract(RemotingConfig clientConfig) {
+        this(clientConfig, new RemotingCommandFactoryMeta());
+    }
+
+    NettyRemotingAbstract(RemotingConfig clientConfig, RemotingCommandFactoryMeta remotingCommandFactoryMeta) {
+        this.semaphoreOneway = new Semaphore(clientConfig.getClientOnewayInvokeSemaphore(), true);
+        this.semaphoreAsync = new Semaphore(clientConfig.getClientAsyncInvokeSemaphore(), true);
+        this.publicExecutor = ThreadUtils.newThreadPoolExecutor(clientConfig.getClientAsyncCallbackExecutorThreads(), clientConfig.getClientAsyncCallbackExecutorThreads(), 60,
+            TimeUnit.SECONDS, new ArrayBlockingQueue<Runnable>(10000), "PublicExecutor", true);
+        this.remotingCommandFactory = new RemotingCommandFactoryImpl(remotingCommandFactoryMeta);
+    }
+
+    public SerializerFactory getSerializerFactory() {
+        return serializerFactory;
+    }
+
+    protected void putNettyEvent(final NettyChannelEvent event) {
+        this.channelEventExecutor.putNettyEvent(event);
+    }
+
+    protected void startUpHouseKeepingService() {
+        this.houseKeepingService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                scanResponseTable();
+            }
+        }, 3000, 1000, TimeUnit.MICROSECONDS);
+    }
+
+    @Override
+    public void start() {
+        if (this.channelEventListenerGroup.size() > 0) {
+            this.channelEventExecutor.start();
+        }
+    }
+
+    @Override
+    public void stop() {
+        ThreadUtils.shutdownGracefully(publicExecutor, 2000, TimeUnit.MILLISECONDS);
+        ThreadUtils.shutdownGracefully(channelEventExecutor);
+    }
+
+    protected void processMessageReceived(ChannelHandlerContext ctx, RemotingCommand command) throws Exception {
+        if (command != null) {
+            switch (command.trafficType()) {
+                case REQUEST_ONEWAY:
+                case REQUEST_ASYNC:
+                case REQUEST_SYNC:
+                    processRequestCommand(ctx, command);
+                    break;
+                case RESPONSE:
+                    processResponseCommand(ctx, command);
+                    break;
+                default:
+                    LOG.warn("Not supported The traffic type {} !", command.trafficType());
+                    break;
+            }
+        }
+    }
+
+    public void processRequestCommand(final ChannelHandlerContext ctx, final RemotingCommand cmd) {
+        Pair<RequestProcessor, ExecutorService> processorExecutorPair = this.processorTables.get(cmd.opCode());
+
+        RemotingChannel channel = new NettyChannelImpl(ctx.channel());
+
+        Runnable run = buildProcessorTask(ctx, cmd, processorExecutorPair, channel);
+
+        try {
+            processorExecutorPair.getRight().submit(run);
+        } catch (RejectedExecutionException e) {
+            if ((System.currentTimeMillis() % 10000) == 0) {
+                LOG.warn(String.format("Request %s from %s Rejected by server executor %s !", cmd,
+                    extractRemoteAddress(ctx.channel()), processorExecutorPair.getRight().toString()));
+            }
+
+            if (cmd.trafficType() != TrafficType.REQUEST_ONEWAY) {
+                interceptorGroup.onException(new ExceptionContext(RemotingEndPoint.RESPONSE,
+                    extractRemoteAddress(ctx.channel()), cmd, e, "FLOW_CONTROL"));
+
+                RemotingCommand response = remotingCommandFactory.createResponse(cmd);
+                response.opCode(RemotingCommand.CommandFlag.ERROR.flag());
+                response.remark("SYSTEM_BUSY");
+                writeAndFlush(ctx.channel(), response);
+            }
+        }
+    }
+
+    @NotNull
+    private Runnable buildProcessorTask(final ChannelHandlerContext ctx, final RemotingCommand cmd,
+        final Pair<RequestProcessor, ExecutorService> processorExecutorPair, final RemotingChannel channel) {
+        return new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    interceptorGroup.beforeRequest(new RequestContext(RemotingEndPoint.RESPONSE,
+                        extractRemoteAddress(ctx.channel()), cmd));
+
+                    RemotingCommand response = processorExecutorPair.getLeft().processRequest(channel, cmd);
+
+                    interceptorGroup.afterResponseReceived(new ResponseContext(RemotingEndPoint.RESPONSE,
+                        extractRemoteAddress(ctx.channel()), cmd, response));
+
+                    handleResponse(response, cmd, ctx);
+                } catch (Throwable e) {
+                    LOG.error(String.format("Process request %s error !", cmd.toString()), e);
+
+                    handleException(e, cmd, ctx);
+                }
+            }
+        };
+    }
+
+    private void handleException(Throwable e, RemotingCommand cmd, ChannelHandlerContext ctx) {
+        if (cmd.trafficType() != TrafficType.REQUEST_ONEWAY) {
+            //FiXME Exception interceptor can not throw exception
+            interceptorGroup.onException(new ExceptionContext(RemotingEndPoint.RESPONSE, extractRemoteAddress(ctx.channel()), cmd, e, ""));
+            RemotingCommand response = remotingCommandFactory.createResponse(cmd);
+            response.opCode(RemotingCommand.CommandFlag.ERROR.flag());
+            response.remark(serializeException(cmd.serializerType(), e));
+            response.property("Exception", e.getClass().getName());
+            ctx.writeAndFlush(response);
+        }
+    }
+
+    private String serializeException(byte serializeType, Throwable exception) {
+        final Serializer serialization = getSerializerFactory().get(serializeType);
+        return serialization.encode(exception).toString();
+    }
+
+    private void handleResponse(RemotingCommand response, RemotingCommand cmd, ChannelHandlerContext ctx) {
+        if (cmd.trafficType() != TrafficType.REQUEST_ONEWAY) {
+            if (response != null) {
+                try {
+                    writeAndFlush(ctx.channel(), response);
+                } catch (Throwable e) {
+                    LOG.error(String.format("Process request %s success, but transfer response %s failed !",
+                        cmd.toString(), response.toString()), e);
+                }
+            }
+        }
+
+    }
+
+    private void processResponseCommand(ChannelHandlerContext ctx, RemotingCommand cmd) {
+        final ResponseResult responseResult = ackTables.get(cmd.requestID());
+        if (responseResult != null) {
+            responseResult.setResponseCommand(cmd);
+            responseResult.release();
+
+            long time = System.currentTimeMillis();
+            ackTables.remove(cmd.requestID());
+            if (count.incrementAndGet() % 5000 == 0)
+                LOG.warn("REQUEST ID:{}, cost time:{}, ackTables.size:{}", cmd.requestID(), time - responseResult.getBeginTimestamp(),
+                    ackTables.size());
+            if (responseResult.getAsyncHandler() != null) {
+                boolean sameThread = false;
+                ExecutorService executor = this.getCallbackExecutor();
+                if (executor != null) {
+                    try {
+                        executor.submit(new Runnable() {
+                            @Override
+                            public void run() {
+                                try {
+                                    responseResult.executeCallbackArrived(responseResult.getResponseCommand());
+                                } catch (Throwable e) {
+                                    LOG.warn("Execute callback error !", e);
+                                }
+                            }
+                        });
+                    } catch (RejectedExecutionException e) {
+                        sameThread = true;
+                        LOG.warn("Execute submit error !", e);
+                    }
+                } else {
+                    sameThread = true;
+                }
+
+                if (sameThread) {
+                    try {
+                        responseResult.executeCallbackArrived(responseResult.getResponseCommand());
+                    } catch (Throwable e) {
+                        LOG.warn("Execute callback in response thread error !", e);
+                    }
+                }
+            } else {
+                responseResult.putResponse(cmd);
+            }
+        } else {
+            LOG.warn("request {} from {} has not matched response !", cmd, extractRemoteAddress(ctx.channel()));
+        }
+    }
+
+    private void writeAndFlush(final Channel channel, final Object msg, final ChannelFutureListener listener) {
+        channel.writeAndFlush(msg).addListener(listener);
+    }
+
+    private void writeAndFlush(final Channel channel, final Object msg) {
+        channel.writeAndFlush(msg);
+    }
+
+    public ExecutorService getCallbackExecutor() {
+        return this.publicExecutor;
+    }
+
+    void scanResponseTable() {
+        /*
+        Iterator<Map.Entry<Integer, ResponseResult>> iterator = this.ackTables.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Integer, ResponseResult> next = iterator.next();
+            ResponseResult result = next.getValue();
+
+            if ((result.getBeginTimestamp() + result.getTimeoutMillis()) <= System.currentTimeMillis()) {
+                iterator.remove();
+                try {
+                    long timeoutMillis = result.getTimeoutMillis();
+                    long costTimeMillis = System.currentTimeMillis() - result.getBeginTimestamp();
+                    result.onTimeout(timeoutMillis, costTimeMillis);
+                    LOG.error("scan response table command {} failed", result.getRequestId());
+                } catch (Throwable e) {
+                    LOG.warn("Error occurred when execute timeout callback !", e);
+                } finally {
+                    result.release();
+                    LOG.warn("Removed timeout request {} ", result);
+                }
+            }
+        }
+        */
+    }
+
+    public RemotingCommand invokeWithInterceptor(final Channel channel, final RemotingCommand request,
+        long timeoutMillis) {
+        request.trafficType(TrafficType.REQUEST_SYNC);
+
+        final String remoteAddr = extractRemoteAddress(channel);
+
+        //FIXME try catch here
+        this.interceptorGroup.beforeRequest(new RequestContext(RemotingEndPoint.REQUEST, remoteAddr, request));
+
+        RemotingCommand responseCommand = this.invoke0(remoteAddr, channel, request, timeoutMillis);
+
+        this.interceptorGroup.afterResponseReceived(new ResponseContext(RemotingEndPoint.REQUEST,
+            extractRemoteAddress(channel), request, responseCommand));
+
+        return responseCommand;
+    }
+
+    private RemotingCommand invoke0(final String remoteAddr, final Channel channel, final RemotingCommand request,
+        final long timeoutMillis) {
+        try {
+            final int opaque = request.requestID();
+            final ResponseResult responseResult = new ResponseResult(opaque, timeoutMillis);
+            responseResult.setRequestCommand(request);
+            //FIXME one interceptor for all case ?
+            responseResult.setInterceptorGroup(this.interceptorGroup);
+            responseResult.setRemoteAddr(remoteAddr);
+
+            this.ackTables.put(opaque, responseResult);
+
+            ChannelFutureListener listener = new ChannelFutureListener() {
+                @Override
+                public void operationComplete(ChannelFuture f) throws Exception {
+                    if (f.isSuccess()) {
+                        responseResult.setSendRequestOK(true);
+                        return;
+                    } else {
+                        responseResult.setSendRequestOK(false);
+
+                        ackTables.remove(opaque);
+                        responseResult.setCause(f.cause());
+                        responseResult.putResponse(null);
+
+                        LOG.warn("Send request command to {} failed !", remoteAddr);
+                    }
+                }
+            };
+
+            this.writeAndFlush(channel, request, listener);
+
+            RemotingCommand responseCommand = responseResult.waitResponse(timeoutMillis);
+
+            if (null == responseCommand) {
+                if (responseResult.isSendRequestOK()) {
+                    throw new RemoteTimeoutException(extractRemoteAddress(channel), timeoutMillis, responseResult.getCause());
+                }
+                /*
+                else {
+                    throw new RemoteAccessException(extractRemoteAddress(channel), responseResult.getCause());
+                }*/
+            }
+
+            return responseCommand;
+        } finally {
+            this.ackTables.remove(request.requestID());
+        }
+    }
+
+    public void invokeAsyncWithInterceptor(final Channel channel, final RemotingCommand request,
+        final AsyncHandler invokeCallback, long timeoutMillis) {
+        request.trafficType(TrafficType.REQUEST_ASYNC);
+
+        final String remoteAddr = extractRemoteAddress(channel);
+
+        this.interceptorGroup.beforeRequest(new RequestContext(RemotingEndPoint.REQUEST, remoteAddr, request));
+
+        Exception exception = null;
+
+        try {
+            this.invokeAsync0(remoteAddr, channel, request, timeoutMillis, invokeCallback);
+        } catch (InterruptedException e) {
+            exception = e;
+        } finally {
+            if (null != exception) {
+                try {
+                    this.interceptorGroup.onException(new ExceptionContext(RemotingEndPoint.REQUEST, extractRemoteAddress(channel), request, exception, "REMOTING_EXCEPTION"));
+                } catch (Throwable e) {
+                    LOG.warn("onException ", e);
+                }
+            }
+        }
+    }
+
+    private void invokeAsync0(final String remoteAddr, final Channel channel, final RemotingCommand request,
+        final long timeoutMillis, final AsyncHandler invokeCallback) throws InterruptedException {
+        boolean acquired = this.semaphoreAsync.tryAcquire(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (acquired) {
+            final int requestID = request.requestID();
+
+            SemaphoreReleaseOnlyOnce once = new SemaphoreReleaseOnlyOnce(this.semaphoreAsync);
+
+            final ResponseResult responseResult = new ResponseResult(request.requestID(), timeoutMillis, invokeCallback, once);
+            responseResult.setRequestCommand(request);
+            responseResult.setInterceptorGroup(this.interceptorGroup);
+            responseResult.setRemoteAddr(remoteAddr);
+
+            this.ackTables.put(request.requestID(), responseResult);
+            try {
+                ChannelFutureListener listener = new ChannelFutureListener() {
+                    @Override
+                    public void operationComplete(ChannelFuture f) throws Exception {
+                        responseResult.setSendRequestOK(f.isSuccess());
+                        if (f.isSuccess()) {
+                            return;
+                        }
+
+                        responseResult.putResponse(null);
+                        ackTables.remove(requestID);
+                        try {
+                            responseResult.executeRequestSendFailed();
+                        } catch (Throwable e) {
+                            LOG.warn("Execute callback error !", e);
+                        } finally {
+                            responseResult.release();
+                        }
+
+                        LOG.warn("Send request command to channel  failed.", remoteAddr);
+                    }
+                };
+
+                this.writeAndFlush(channel, request, listener);
+            } catch (Exception e) {
+                responseResult.release();
+                LOG.error("Send request command to channel " + channel + " error !", e);
+            }
+        } else {
+            String info = String.format("Semaphore tryAcquire %d ms timeout for request %s ,waiting thread nums: %d,availablePermits: %d",
+                timeoutMillis, request.toString(), semaphoreAsync.getQueueLength(), this.semaphoreAsync.availablePermits());
+            LOG.error(info);
+            throw new RemoteTimeoutException(info);
+        }
+    }
+
+    public void invokeOnewayWithInterceptor(final Channel channel, final RemotingCommand request, long timeoutMillis) {
+        request.trafficType(TrafficType.REQUEST_ONEWAY);
+
+        this.interceptorGroup.beforeRequest(new RequestContext(RemotingEndPoint.REQUEST, extractRemoteAddress(channel), request));
+
+        Exception exception = null;
+
+        try {
+            this.invokeOneway0(channel, request, timeoutMillis);
+        } catch (InterruptedException e) {
+            exception = e;
+        } finally {
+            if (null != exception) {
+                try {
+                    this.interceptorGroup.onException(new ExceptionContext(RemotingEndPoint.REQUEST, extractRemoteAddress(channel), request, exception, "REMOTING_EXCEPTION"));
+                } catch (Throwable e) {
+                    LOG.warn("onException ", e);
+                }
+            }
+        }
+    }
+
+    private void invokeOneway0(final Channel channel, final RemotingCommand request,
+        final long timeoutMillis) throws InterruptedException {
+        boolean acquired = this.semaphoreOneway.tryAcquire(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (acquired) {
+            final SemaphoreReleaseOnlyOnce once = new SemaphoreReleaseOnlyOnce(this.semaphoreOneway);
+            try {
+                final SocketAddress socketAddress = channel.remoteAddress();
+
+                ChannelFutureListener listener = new ChannelFutureListener() {
+                    @Override
+                    public void operationComplete(ChannelFuture f) throws Exception {
+                        once.release();
+                        if (!f.isSuccess()) {
+                            LOG.warn("Send request command to channel {} failed !", socketAddress);
+                        }
+                    }
+                };
+
+                this.writeAndFlush(channel, request, listener);
+            } catch (Exception e) {
+                once.release();
+                LOG.error("Send request command to channel " + channel + " error !", e);
+            }
+        } else {
+            String info = String.format("Semaphore tryAcquire %d ms timeout for request %s ,waiting thread nums: %d,availablePermits: %d",
+                timeoutMillis, request.toString(), semaphoreAsync.getQueueLength(), this.semaphoreAsync.availablePermits());
+            LOG.error(info);
+            throw new RemoteTimeoutException(info);
+        }
+    }
+
+    public String getRemotingInstanceId() {
+        return remotingInstanceId;
+    }
+
+    @Override
+    public ProtocolFactory protocolFactory() {
+        return this.protocolFactory;
+    }
+
+    @Override
+    public SerializerFactory serializerFactory() {
+        return this.serializerFactory;
+    }
+
+    @Override
+    public RemotingCommandFactory commandFactory() {
+        return this.remotingCommandFactory;
+    }
+
+    @Override
+    public void registerRequestProcessor(String requestCode, RequestProcessor processor, ExecutorService executor) {
+        Pair<RequestProcessor, ExecutorService> pair = new Pair<RequestProcessor, ExecutorService>(processor, executor);
+        if (!this.processorTables.containsKey(requestCode)) {
+            this.processorTables.put(requestCode, pair);
+        }
+    }
+
+    @Override
+    public void registerRequestProcessor(String requestCode, RequestProcessor processor) {
+        this.registerRequestProcessor(requestCode, processor, publicExecutor);
+    }
+
+    @Override
+    public void unregisterRequestProcessor(String requestCode) {
+        this.processorTables.remove(requestCode);
+    }
+
+    @Override
+    public String remotingInstanceId() {
+        return this.getRemotingInstanceId();
+    }
+
+    @Override
+    public void registerInterceptor(Interceptor interceptor) {
+        this.interceptorGroup.registerInterceptor(interceptor);
+    }
+
+    @Override
+    public void registerChannelEventListener(ChannelEventListener listener) {
+        this.channelEventListenerGroup.registerChannelEventListener(listener);
+    }
+
+    @Override
+    public Pair<RequestProcessor, ExecutorService> processor(String requestCode) {
+        return processorTables.get(requestCode);
+    }
+
+    protected String extractRemoteAddress(Channel channel) {
+        return ((InetSocketAddress) channel.remoteAddress()).getAddress().getHostAddress();
+    }
+
+    class ChannelEventExecutor extends Thread {
+        private final static int MAX_SIZE = 10000;
+        private final LinkedBlockingQueue<NettyChannelEvent> eventQueue = new LinkedBlockingQueue<NettyChannelEvent>();
+        private String name;
+
+        public ChannelEventExecutor(String nettyEventExector) {
+            super(nettyEventExector);
+            this.name = nettyEventExector;
+        }
+        //private final AtomicBoolean isStopped = new AtomicBoolean(true);
+
+        public void putNettyEvent(final NettyChannelEvent event) {
+            if (this.eventQueue.size() <= MAX_SIZE) {
+                this.eventQueue.add(event);
+            } else {
+                LOG.warn("event queue size[{}] enough, so drop this event {}", this.eventQueue.size(), event.toString());
+            }
+        }
+
+        @Override
+        public void run() {
+            LOG.info(this.name + " service started");
+
+            ChannelEventListenerGroup listener = NettyRemotingAbstract.this.channelEventListenerGroup;
+
+            while (true) {
+                try {
+                    NettyChannelEvent event = this.eventQueue.poll(3000, TimeUnit.MILLISECONDS);
+                    if (event != null && listener != null) {
+                        RemotingChannel channel = new NettyChannelImpl(event.getChannel());
+
+                        LOG.warn("Channel Event, {}", event);
+
+                        switch (event.getType()) {
+                            case IDLE:
+                                listener.onChannelIdle(channel);
+                                break;
+                            case INACTIVE:
+                                listener.onChannelClose(channel);
+                                break;
+                            case ACTIVE:
+                                listener.onChannelConnect(channel);
+                                break;
+                            case EXCEPTION:
+                                listener.onChannelException(channel);
+                                break;
+                            default:
+                                break;
+                        }
+                    }
+                } catch (Exception e) {
+                    LOG.error("error", e);
+                    break;
+                }
+            }
+        }
+
+    }
+
+    protected class EventDispatcher extends SimpleChannelInboundHandler<RemotingCommand> {
+
+        @Override
+        protected void channelRead0(ChannelHandlerContext ctx, RemotingCommand msg) throws Exception {
+            processMessageReceived(ctx, msg);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyRemotingClient.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyRemotingClient.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyRemotingClient.java
new file mode 100644
index 0000000..7481574
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyRemotingClient.java
@@ -0,0 +1,499 @@
+/*
+ * 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.rocketmq.remoting.impl.netty;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelDuplexHandler;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.WriteBufferWaterMark;
+import io.netty.channel.epoll.EpollEventLoopGroup;
+import io.netty.channel.epoll.EpollSocketChannel;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.handler.codec.http2.Http2SecurityUtil;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+import io.netty.handler.ssl.SupportedCipherSuiteFilter;
+import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
+import io.netty.handler.timeout.IdleState;
+import io.netty.handler.timeout.IdleStateEvent;
+import io.netty.handler.timeout.IdleStateHandler;
+import io.netty.util.concurrent.DefaultEventExecutorGroup;
+import io.netty.util.concurrent.EventExecutorGroup;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import javax.net.ssl.SSLException;
+import org.apache.rocketmq.remoting.api.AsyncHandler;
+import org.apache.rocketmq.remoting.api.RemotingClient;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.remoting.api.command.TrafficType;
+import org.apache.rocketmq.remoting.api.exception.RemoteConnectFailureException;
+import org.apache.rocketmq.remoting.api.exception.RemoteTimeoutException;
+import org.apache.rocketmq.remoting.api.protocol.Protocol;
+import org.apache.rocketmq.remoting.common.RemotingCommandFactoryMeta;
+import org.apache.rocketmq.remoting.config.RemotingConfig;
+import org.apache.rocketmq.remoting.external.ThreadUtils;
+import org.apache.rocketmq.remoting.impl.netty.handler.Decoder;
+import org.apache.rocketmq.remoting.impl.netty.handler.Encoder;
+import org.apache.rocketmq.remoting.impl.netty.handler.ExceptionHandler;
+import org.apache.rocketmq.remoting.impl.netty.handler.Http2Handler;
+import org.apache.rocketmq.remoting.internal.JvmUtils;
+
+public class NettyRemotingClient extends NettyRemotingAbstract implements RemotingClient {
+    private static final long LOCK_TIMEOUT_MILLIS = 3000;
+    private final Bootstrap clientBootstrap = new Bootstrap();
+    private final EventLoopGroup ioGroup;
+    private final Class<? extends SocketChannel> socketChannelClass;
+
+    private final RemotingConfig clientConfig;
+
+    private final ConcurrentHashMap<String, ChannelWrapper> channelTables = new ConcurrentHashMap<String, ChannelWrapper>();
+    private final Lock lockChannelTables = new ReentrantLock();
+    private EventExecutorGroup workerGroup;
+    private SslContext sslContext;
+
+    NettyRemotingClient(final RemotingConfig clientConfig) {
+        super(clientConfig, new RemotingCommandFactoryMeta(clientConfig.getProtocolName(), clientConfig.getSerializerName()));
+        this.clientConfig = clientConfig;
+
+        if (JvmUtils.isLinux() && this.clientConfig.isClientNativeEpollEnable()) {
+            this.ioGroup = new EpollEventLoopGroup(clientConfig.getClientWorkerThreads(), ThreadUtils.newGenericThreadFactory("NettyClientEpollIoThreads",
+                clientConfig.getClientWorkerThreads()));
+            socketChannelClass = EpollSocketChannel.class;
+        } else {
+            this.ioGroup = new NioEventLoopGroup(clientConfig.getClientWorkerThreads(), ThreadUtils.newGenericThreadFactory("NettyClientNioIoThreads",
+                clientConfig.getClientWorkerThreads()));
+            socketChannelClass = NioSocketChannel.class;
+        }
+
+        this.workerGroup = new DefaultEventExecutorGroup(clientConfig.getClientWorkerThreads(),
+            ThreadUtils.newGenericThreadFactory("NettyClientWorkerThreads", clientConfig.getClientWorkerThreads()));
+
+        if (Protocol.HTTP2.equals(clientConfig.getProtocolName())) {
+            buildSslContext();
+        }
+    }
+
+    private void applyOptions(Bootstrap bootstrap) {
+        if (null != clientConfig) {
+            if (clientConfig.getTcpSoLinger() > 0) {
+                bootstrap.option(ChannelOption.SO_LINGER, clientConfig.getTcpSoLinger());
+            }
+
+            if (clientConfig.getTcpSoSndBufSize() > 0) {
+                bootstrap.option(ChannelOption.SO_SNDBUF, clientConfig.getTcpSoSndBufSize());
+            }
+            if (clientConfig.getTcpSoRcvBufSize() > 0) {
+                bootstrap.option(ChannelOption.SO_RCVBUF, clientConfig.getTcpSoRcvBufSize());
+            }
+
+            bootstrap.option(ChannelOption.SO_REUSEADDR, clientConfig.isTcpSoReuseAddress()).
+                option(ChannelOption.SO_KEEPALIVE, clientConfig.isTcpSoKeepAlive()).
+                option(ChannelOption.TCP_NODELAY, clientConfig.isTcpSoNoDelay()).
+                option(ChannelOption.CONNECT_TIMEOUT_MILLIS, clientConfig.getTcpSoTimeout()).
+                option(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(clientConfig.getWriteBufLowWaterMark(),
+                    clientConfig.getWriteBufHighWaterMark()));
+        }
+    }
+
+    @Override
+    public void start() {
+        super.start();
+
+        this.clientBootstrap.group(this.ioGroup).channel(socketChannelClass)
+            .handler(new ChannelInitializer<SocketChannel>() {
+                @Override
+                public void initChannel(SocketChannel ch) throws Exception {
+                    if (Protocol.HTTP2.equals(clientConfig.getProtocolName())) {
+                        ch.pipeline().addFirst(sslContext.newHandler(ch.alloc()), Http2Handler.newHandler(false));
+                    }
+                    ch.pipeline().addLast(workerGroup, new Decoder(), new Encoder(), new IdleStateHandler(clientConfig.getConnectionChannelReaderIdleSeconds(),
+                            clientConfig.getConnectionChannelWriterIdleSeconds(), clientConfig.getConnectionChannelIdleSeconds()),
+                        new ClientConnectionHandler(), new EventDispatcher(), new ExceptionHandler());
+                }
+            });
+
+        applyOptions(clientBootstrap);
+
+        startUpHouseKeepingService();
+    }
+
+    private void buildSslContext() {
+        SslProvider provider = OpenSsl.isAvailable() ? SslProvider.OPENSSL : SslProvider.JDK;
+        try {
+            sslContext = SslContextBuilder.forClient()
+                .sslProvider(provider)
+                    /* NOTE: the cipher filter may not include all ciphers required by the HTTP/2 specification.
+                     * Please refer to the HTTP/2 specification for cipher requirements. */
+                .ciphers(Http2SecurityUtil.CIPHERS, SupportedCipherSuiteFilter.INSTANCE)
+                .trustManager(InsecureTrustManagerFactory.INSTANCE)
+                .build();
+        } catch (SSLException e) {
+            e.printStackTrace();
+        }
+    }
+
+    @Override
+    public void stop() {
+        // try {
+        ThreadUtils.shutdownGracefully(houseKeepingService, 3000, TimeUnit.MILLISECONDS);
+
+        for (ChannelWrapper cw : this.channelTables.values()) {
+            this.closeChannel(null, cw.getChannel());
+        }
+
+        this.channelTables.clear();
+
+        this.ioGroup.shutdownGracefully();
+
+        ThreadUtils.shutdownGracefully(channelEventExecutor);
+
+        this.workerGroup.shutdownGracefully();
+        /*
+        } catch (Exception e) {
+            LOG.error("RemotingClient stopped error !", e);
+        }
+        */
+
+        super.stop();
+    }
+
+    private void closeChannel(final String addr, final Channel channel) {
+        if (null == channel)
+            return;
+
+        final String addrRemote = null == addr ? extractRemoteAddress(channel) : addr;
+        try {
+            if (this.lockChannelTables.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) {
+                try {
+                    boolean removeItemFromTable = true;
+                    ChannelWrapper prevCW = this.channelTables.get(addrRemote);
+                    //Workaround for null
+                    if (null == prevCW) {
+                        return;
+                    }
+
+                    LOG.info("Begin to close the remote address {} channel {}", addrRemote, prevCW);
+
+                    if (prevCW.getChannel() != channel) {
+                        LOG.info("Channel {} has been closed,this is a new channel.", prevCW.getChannel(), channel);
+                        removeItemFromTable = false;
+                    }
+
+                    if (removeItemFromTable) {
+                        this.channelTables.remove(addrRemote);
+                        LOG.info("Channel {} has been removed !", addrRemote);
+                    }
+
+                    channel.close().addListener(new ChannelFutureListener() {
+                        @Override
+                        public void operationComplete(ChannelFuture future) throws Exception {
+                            LOG.warn("Close channel {} {}", channel, future.isSuccess());
+                        }
+                    });
+                } catch (Exception e) {
+                    LOG.error("Close channel error !", e);
+                } finally {
+                    this.lockChannelTables.unlock();
+                }
+            } else {
+                LOG.warn("Can not lock channel table in {} ms", LOCK_TIMEOUT_MILLIS);
+            }
+        } catch (InterruptedException e) {
+            LOG.error("Close channel error !", e);
+        }
+    }
+
+    private Channel createIfAbsent(final String addr) {
+        ChannelWrapper cw = this.channelTables.get(addr);
+        if (cw != null && cw.isActive()) {
+            return cw.getChannel();
+        }
+        return this.createChannel(addr);
+    }
+
+    //FIXME need test to verify
+    private Channel createChannel(final String addr) {
+        ChannelWrapper cw = null;
+        try {
+            if (this.lockChannelTables.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) {
+                try {
+                    boolean createNewConnection;
+                    cw = this.channelTables.get(addr);
+                    if (cw != null) {
+                        if (cw.isActive()) {
+                            return cw.getChannel();
+                        } else if (!cw.getChannelFuture().isDone()) {
+                            createNewConnection = false;
+                        } else {
+                            this.channelTables.remove(addr);
+                            createNewConnection = true;
+                        }
+                    } else {
+                        createNewConnection = true;
+                    }
+
+                    if (createNewConnection) {
+                        String[] s = addr.split(":");
+                        SocketAddress socketAddress = new InetSocketAddress(s[0], Integer.valueOf(s[1]));
+                        ChannelFuture channelFuture = this.clientBootstrap.connect(socketAddress);
+                        LOG.info("createChannel: begin to connect remote host[{}] asynchronously", addr);
+                        cw = new ChannelWrapper(channelFuture);
+                        this.channelTables.put(addr, cw);
+                    }
+                } catch (Exception e) {
+                    LOG.error("createChannel: create channel exception", e);
+                } finally {
+                    this.lockChannelTables.unlock();
+                }
+            } else {
+                LOG.warn("createChannel: try to lock channel table, but timeout, {}ms", LOCK_TIMEOUT_MILLIS);
+            }
+        } catch (InterruptedException e) {
+            e.printStackTrace();
+        }
+
+        if (cw != null) {
+            ChannelFuture channelFuture = cw.getChannelFuture();
+            if (channelFuture.awaitUninterruptibly(this.clientConfig.getClientConnectionFutureAwaitTimeoutMillis())) {
+                if (cw.isActive()) {
+                    LOG.info("createChannel: connect remote host[{}] success, {}", addr, channelFuture.toString());
+                    return cw.getChannel();
+                } else {
+                    LOG.warn("createChannel: connect remote host[" + addr + "] failed, and destroy the channel" + channelFuture.toString(), channelFuture.cause());
+                    this.closeChannel(addr, cw.getChannel());
+                }
+            } else {
+                LOG.warn("createChannel: connect remote host[{}] timeout {}ms, {}, and destroy the channel", addr, this.clientConfig.getClientConnectionFutureAwaitTimeoutMillis(),
+                    channelFuture.toString());
+                this.closeChannel(addr, cw.getChannel());
+            }
+        }
+        return null;
+    }
+
+    private void closeChannel(final Channel channel) {
+        if (null == channel)
+            return;
+
+        try {
+            if (this.lockChannelTables.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) {
+                try {
+                    boolean removeItemFromTable = true;
+                    ChannelWrapper prevCW = null;
+                    String addrRemote = null;
+
+                    for (Map.Entry<String, ChannelWrapper> entry : channelTables.entrySet()) {
+                        ChannelWrapper prev = entry.getValue();
+                        if (prev.getChannel() != null) {
+                            if (prev.getChannel() == channel) {
+                                prevCW = prev;
+                                addrRemote = entry.getKey();
+                                break;
+                            }
+                        }
+                    }
+
+                    if (null == prevCW) {
+                        LOG.info("eventCloseChannel: the channel[{}] has been removed from the channel table before", addrRemote);
+                        removeItemFromTable = false;
+                    }
+
+                    if (removeItemFromTable) {
+                        this.channelTables.remove(addrRemote);
+                        LOG.info("closeChannel: the channel[{}] was removed from channel table", addrRemote);
+                        //RemotingHelper.closeChannel(channel);
+                    }
+                } catch (Exception e) {
+                    LOG.error("closeChannel: close the channel exception", e);
+                } finally {
+                    this.lockChannelTables.unlock();
+                }
+            } else {
+                LOG.warn("closeChannel: try to lock channel table, but timeout, {}ms", LOCK_TIMEOUT_MILLIS);
+            }
+        } catch (InterruptedException e) {
+            LOG.error("closeChannel exception", e);
+        }
+    }
+
+    @Override
+    public RemotingCommand invoke(final String address, final RemotingCommand request, final long timeoutMillis) {
+        request.trafficType(TrafficType.REQUEST_SYNC);
+
+        Channel channel = this.createIfAbsent(address);
+        if (channel != null && channel.isActive()) {
+            try {
+                return this.invokeWithInterceptor(channel, request, timeoutMillis);
+
+            } catch (RemoteTimeoutException e) {
+                if (this.clientConfig.isClientCloseSocketIfTimeout()) {
+                    LOG.warn("invoke: timeout, so close the socket {} ms, {}", timeoutMillis, address);
+                    this.closeChannel(address, channel);
+                }
+
+                LOG.warn("invoke: wait response timeout<{}ms> exception, so close the channel[{}]", timeoutMillis, address);
+                throw e;
+            } finally {
+                /*
+                if (this.clientConfig.isClientShortConnectionEnable()) {
+                    this.closeChannel(addr, channel);
+                }
+                */
+            }
+        } else {
+            this.closeChannel(address, channel);
+            throw new RemoteConnectFailureException(address);
+        }
+
+    }
+
+    @Override
+    public void invokeAsync(final String address, final RemotingCommand request, final AsyncHandler asyncHandler,
+        final long timeoutMillis) {
+
+        final Channel channel = this.createIfAbsent(address);
+        if (channel != null && channel.isActive()) {
+            // We support Netty's channel-level backpressure thereby respecting slow receivers on the other side.
+            if (!channel.isWritable()) {
+                // Note: It's up to the layer above a transport to decide whether or not to requeue a canceled write.
+                LOG.warn("Channel statistics - bytesBeforeUnwritable:{},bytesBeforeWritable:{}", channel.bytesBeforeUnwritable(), channel.bytesBeforeWritable());
+            }
+            this.invokeAsyncWithInterceptor(channel, request, asyncHandler, timeoutMillis);
+        } else {
+            this.closeChannel(address, channel);
+        }
+    }
+
+    @Override
+    public void invokeOneWay(final String address, final RemotingCommand request, final long timeoutMillis) {
+        final Channel channel = this.createIfAbsent(address);
+        if (channel != null && channel.isActive()) {
+            if (!channel.isWritable()) {
+                //if (this.clientConfig.isSocketFlowControl()) {
+                LOG.warn("Channel statistics - bytesBeforeUnwritable:{},bytesBeforeWritable:{}", channel.bytesBeforeUnwritable(), channel.bytesBeforeWritable());
+                //throw new ServiceInvocationFailureException(String.format("Channel[%s] is not writable now", channel.toString()));
+            }
+            this.invokeOnewayWithInterceptor(channel, request, timeoutMillis);
+        } else {
+            this.closeChannel(address, channel);
+        }
+    }
+
+    private class ChannelWrapper {
+        private final ChannelFuture channelFuture;
+
+        ChannelWrapper(ChannelFuture channelFuture) {
+            this.channelFuture = channelFuture;
+        }
+
+        boolean isActive() {
+            return this.channelFuture.channel() != null && this.channelFuture.channel().isActive();
+        }
+
+        boolean isWriteable() {
+            return this.channelFuture.channel().isWritable();
+        }
+
+        private Channel getChannel() {
+            return this.channelFuture.channel();
+        }
+
+        ChannelFuture getChannelFuture() {
+            return channelFuture;
+        }
+    }
+
+    private class ClientConnectionHandler extends ChannelDuplexHandler {
+
+        @Override
+        public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception {
+            LOG.warn("Channel {} channelWritabilityChanged event triggered - bytesBeforeUnwritable:{},bytesBeforeWritable:{}", ctx.channel(),
+                ctx.channel().bytesBeforeUnwritable(), ctx.channel().bytesBeforeWritable());
+        }
+
+        @Override
+        public void connect(ChannelHandlerContext ctx, SocketAddress remoteAddress, SocketAddress localAddress,
+            ChannelPromise promise)
+            throws Exception {
+            LOG.info("Connected from {} to {}.", localAddress, remoteAddress);
+            super.connect(ctx, remoteAddress, localAddress, promise);
+
+            putNettyEvent(new NettyChannelEvent(NettyChannelEventType.ACTIVE, ctx.channel()));
+        }
+
+        @Override
+        public void disconnect(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception {
+            LOG.info("Remote address {} disconnect channel {}.", ctx.channel().remoteAddress(), ctx.channel());
+
+            closeChannel(ctx.channel());
+
+            super.disconnect(ctx, promise);
+
+            putNettyEvent(new NettyChannelEvent(NettyChannelEventType.INACTIVE, ctx.channel()));
+        }
+
+        @Override
+        public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception {
+            LOG.info("Remote address {} close channel {}.", ctx.channel().remoteAddress(), ctx.channel());
+
+            closeChannel(ctx.channel());
+
+            super.close(ctx, promise);
+
+            putNettyEvent(new NettyChannelEvent(NettyChannelEventType.INACTIVE, ctx.channel()));
+        }
+
+        @Override
+        public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {
+            if (evt instanceof IdleStateEvent) {
+                IdleStateEvent event = (IdleStateEvent) evt;
+                if (event.state().equals(IdleState.ALL_IDLE)) {
+                    LOG.info("Close channel {} because of idle event {} ", ctx.channel(), event);
+                    closeChannel(ctx.channel());
+                    putNettyEvent(new NettyChannelEvent(NettyChannelEventType.IDLE, ctx.channel()));
+                }
+            }
+
+            ctx.fireUserEventTriggered(evt);
+        }
+
+        @Override
+        public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
+            LOG.info("Close channel {} because of error {} ", ctx.channel(), cause);
+
+            closeChannel(ctx.channel());
+            putNettyEvent(new NettyChannelEvent(NettyChannelEventType.EXCEPTION, ctx.channel()));
+        }
+    }
+}


[5/8] incubator-rocketmq git commit: initialize RocketMQ5

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyRemotingServer.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyRemotingServer.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyRemotingServer.java
new file mode 100644
index 0000000..d875f95
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/NettyRemotingServer.java
@@ -0,0 +1,286 @@
+/*
+ * 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.rocketmq.remoting.impl.netty;
+
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.channel.ChannelDuplexHandler;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.epoll.EpollEventLoopGroup;
+import io.netty.channel.epoll.EpollServerSocketChannel;
+import io.netty.channel.group.ChannelGroup;
+import io.netty.channel.group.DefaultChannelGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.ServerSocketChannel;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.handler.codec.http2.Http2SecurityUtil;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+import io.netty.handler.ssl.SupportedCipherSuiteFilter;
+import io.netty.handler.ssl.util.SelfSignedCertificate;
+import io.netty.handler.timeout.IdleState;
+import io.netty.handler.timeout.IdleStateEvent;
+import io.netty.handler.timeout.IdleStateHandler;
+import io.netty.util.concurrent.DefaultEventExecutorGroup;
+import io.netty.util.concurrent.EventExecutorGroup;
+import io.netty.util.concurrent.GlobalEventExecutor;
+import java.net.InetSocketAddress;
+import java.util.concurrent.TimeUnit;
+import org.apache.rocketmq.remoting.api.AsyncHandler;
+import org.apache.rocketmq.remoting.api.RemotingServer;
+import org.apache.rocketmq.remoting.api.channel.RemotingChannel;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.remoting.config.RemotingConfig;
+import org.apache.rocketmq.remoting.external.ThreadUtils;
+import org.apache.rocketmq.remoting.impl.channel.NettyChannelImpl;
+import org.apache.rocketmq.remoting.impl.netty.handler.ChannelStatistics;
+import org.apache.rocketmq.remoting.impl.netty.handler.ProtocolSelector;
+import org.apache.rocketmq.remoting.internal.JvmUtils;
+
+public class NettyRemotingServer extends NettyRemotingAbstract implements RemotingServer {
+    private final RemotingConfig serverConfig;
+
+    private final ServerBootstrap serverBootstrap;
+    private final EventLoopGroup bossGroup;
+    private final EventLoopGroup ioGroup;
+    private EventExecutorGroup workerGroup;
+    private Class<? extends ServerSocketChannel> socketChannelClass;
+
+    private int port;
+    private SslContext sslContext;
+
+    NettyRemotingServer(final RemotingConfig serverConfig) {
+        super(serverConfig);
+
+        this.serverBootstrap = new ServerBootstrap();
+        this.serverConfig = serverConfig;
+
+        if (JvmUtils.isLinux() && this.serverConfig.isServerNativeEpollEnable()) {
+            this.ioGroup = new EpollEventLoopGroup(serverConfig.getServerIoThreads(), ThreadUtils.newGenericThreadFactory("NettyEpollIoThreads",
+                serverConfig.getServerIoThreads()));
+
+            this.bossGroup = new EpollEventLoopGroup(serverConfig.getServerAcceptorThreads(), ThreadUtils.newGenericThreadFactory("NettyBossThreads",
+                serverConfig.getServerAcceptorThreads()));
+
+            this.socketChannelClass = EpollServerSocketChannel.class;
+        } else {
+            this.bossGroup = new NioEventLoopGroup(serverConfig.getServerAcceptorThreads(), ThreadUtils.newGenericThreadFactory("NettyBossThreads",
+                serverConfig.getServerAcceptorThreads()));
+
+            this.ioGroup = new NioEventLoopGroup(serverConfig.getServerIoThreads(), ThreadUtils.newGenericThreadFactory("NettyNioIoThreads",
+                serverConfig.getServerIoThreads()));
+
+            this.socketChannelClass = NioServerSocketChannel.class;
+        }
+
+        this.workerGroup = new DefaultEventExecutorGroup(serverConfig.getServerWorkerThreads(),
+            ThreadUtils.newGenericThreadFactory("NettyWorkerThreads", serverConfig.getServerWorkerThreads()));
+
+        buildHttp2SslContext();
+    }
+
+    private void buildHttp2SslContext() {
+        try {
+            SslProvider provider = OpenSsl.isAvailable() ? SslProvider.OPENSSL : SslProvider.JDK;
+            SelfSignedCertificate ssc;
+            //NOTE: the cipher filter may not include all ciphers required by the HTTP/2 specification.
+            //Please refer to the HTTP/2 specification for cipher requirements.
+            ssc = new SelfSignedCertificate();
+            sslContext = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey())
+                .sslProvider(provider)
+                .ciphers(Http2SecurityUtil.CIPHERS, SupportedCipherSuiteFilter.INSTANCE).build();
+        } catch (Exception e) {
+            LOG.error("Can not build SSL context !", e);
+        }
+    }
+
+    private void applyOptions(ServerBootstrap bootstrap) {
+        //option() is for the NioServerSocketChannel that accepts incoming connections.
+        //childOption() is for the Channels accepted by the parent ServerChannel, which is NioServerSocketChannel in this case
+        if (null != serverConfig) {
+            if (serverConfig.getTcpSoBacklogSize() > 0) {
+                bootstrap.option(ChannelOption.SO_BACKLOG, serverConfig.getTcpSoBacklogSize());
+            }
+
+            if (serverConfig.getTcpSoLinger() > 0) {
+                bootstrap.option(ChannelOption.SO_LINGER, serverConfig.getTcpSoLinger());
+            }
+
+            if (serverConfig.getTcpSoSndBufSize() > 0) {
+                bootstrap.childOption(ChannelOption.SO_SNDBUF, serverConfig.getTcpSoSndBufSize());
+            }
+            if (serverConfig.getTcpSoRcvBufSize() > 0) {
+                bootstrap.childOption(ChannelOption.SO_RCVBUF, serverConfig.getTcpSoRcvBufSize());
+            }
+
+            bootstrap.option(ChannelOption.SO_REUSEADDR, serverConfig.isTcpSoReuseAddress()).
+                childOption(ChannelOption.SO_KEEPALIVE, serverConfig.isTcpSoKeepAlive()).
+                childOption(ChannelOption.TCP_NODELAY, serverConfig.isTcpSoNoDelay()).
+                option(ChannelOption.CONNECT_TIMEOUT_MILLIS, serverConfig.getTcpSoTimeout());
+        }
+
+        if (serverConfig.isServerPooledBytebufAllocatorEnable()) {
+            bootstrap.childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT);
+        }
+    }
+
+    @Override
+    public void start() {
+        super.start();
+
+        final ChannelGroup channels = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE);
+        this.serverBootstrap.group(this.bossGroup, this.ioGroup).
+            channel(socketChannelClass).childHandler(new ChannelInitializer<SocketChannel>() {
+            @Override
+            public void initChannel(SocketChannel ch) throws Exception {
+                channels.add(ch);
+
+                ChannelPipeline cp = ch.pipeline();
+
+                cp.addLast(ChannelStatistics.NAME, new ChannelStatistics(channels));
+
+                cp.addFirst(ProtocolSelector.NAME, new ProtocolSelector(sslContext));
+                cp.addLast(workerGroup, new IdleStateHandler(serverConfig.getConnectionChannelReaderIdleSeconds(),
+                        serverConfig.getConnectionChannelWriterIdleSeconds(),
+                        serverConfig.getConnectionChannelIdleSeconds()),
+                    new ServerConnectionHandler(),
+                    new EventDispatcher());
+            }
+        });
+
+        applyOptions(serverBootstrap);
+
+        ChannelFuture channelFuture = this.serverBootstrap.bind(this.serverConfig.getServerListenPort()).syncUninterruptibly();
+        this.port = ((InetSocketAddress) channelFuture.channel().localAddress()).getPort();
+
+        startUpHouseKeepingService();
+    }
+
+    @Override
+    public void stop() {
+        try {
+
+            ThreadUtils.shutdownGracefully(houseKeepingService, 3000, TimeUnit.MILLISECONDS);
+
+            ThreadUtils.shutdownGracefully(channelEventExecutor);
+
+            this.bossGroup.shutdownGracefully().syncUninterruptibly();
+
+            this.ioGroup.shutdownGracefully().syncUninterruptibly();
+
+            this.workerGroup.shutdownGracefully().syncUninterruptibly();
+        } catch (Exception e) {
+            LOG.error("RemotingServer stopped error !", e);
+        }
+
+        super.stop();
+    }
+
+    @Override
+    public int localListenPort() {
+        return this.port;
+    }
+
+    @Override
+    public RemotingCommand invoke(final RemotingChannel remotingChannel, final RemotingCommand request,
+        final long timeoutMillis) {
+        return invokeWithInterceptor(((NettyChannelImpl) remotingChannel).getChannel(), request, timeoutMillis);
+    }
+
+    @Override
+    public void invokeAsync(final RemotingChannel remotingChannel, final RemotingCommand request,
+        final AsyncHandler asyncHandler,
+        final long timeoutMillis) {
+        invokeAsyncWithInterceptor(((NettyChannelImpl) remotingChannel).getChannel(), request, asyncHandler, timeoutMillis);
+    }
+
+    @Override
+    public void invokeOneWay(final RemotingChannel remotingChannel, final RemotingCommand request,
+        final long timeoutMillis) {
+        invokeOnewayWithInterceptor(((NettyChannelImpl) remotingChannel).getChannel(), request, timeoutMillis);
+    }
+
+    private class ServerConnectionHandler extends ChannelDuplexHandler {
+        @Override
+        public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception {
+            LOG.warn("Channel {} channelWritabilityChanged event triggered - bytesBeforeUnwritable:{},bytesBeforeWritable:{}", ctx.channel(),
+                ctx.channel().bytesBeforeUnwritable(), ctx.channel().bytesBeforeWritable());
+        }
+
+        @Override
+        public void channelRegistered(ChannelHandlerContext ctx) throws Exception {
+            super.channelRegistered(ctx);
+        }
+
+        @Override
+        public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
+            super.channelUnregistered(ctx);
+        }
+
+        @Override
+        public void channelActive(ChannelHandlerContext ctx) throws Exception {
+            super.channelActive(ctx);
+            putNettyEvent(new NettyChannelEvent(NettyChannelEventType.ACTIVE, ctx.channel()));
+        }
+
+        @Override
+        public void channelInactive(ChannelHandlerContext ctx) throws Exception {
+            super.channelInactive(ctx);
+            putNettyEvent(new NettyChannelEvent(NettyChannelEventType.INACTIVE, ctx.channel()));
+        }
+
+        @Override
+        public void userEventTriggered(final ChannelHandlerContext ctx, Object evt) throws Exception {
+            if (evt instanceof IdleStateEvent) {
+                final IdleStateEvent event = (IdleStateEvent) evt;
+                if (event.state().equals(IdleState.ALL_IDLE)) {
+                    ctx.channel().close().addListener(new ChannelFutureListener() {
+                        @Override
+                        public void operationComplete(ChannelFuture future) throws Exception {
+                            LOG.warn("Close channel {} because of event {},result is {}", ctx.channel(), event, future.isSuccess());
+                        }
+                    });
+
+                    putNettyEvent(new NettyChannelEvent(NettyChannelEventType.IDLE, ctx.channel()));
+                }
+            }
+            ctx.fireUserEventTriggered(evt);
+        }
+
+        @Override
+        public void exceptionCaught(final ChannelHandlerContext ctx, final Throwable cause) throws Exception {
+            putNettyEvent(new NettyChannelEvent(NettyChannelEventType.EXCEPTION, ctx.channel(), cause));
+
+            ctx.channel().close().addListener(new ChannelFutureListener() {
+                @Override
+                public void operationComplete(ChannelFuture future) throws Exception {
+                    LOG.warn("Close channel {} because of error {},result is {}", ctx.channel(), cause, future.isSuccess());
+                }
+            });
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/RemotingBootstrapFactory.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/RemotingBootstrapFactory.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/RemotingBootstrapFactory.java
new file mode 100644
index 0000000..4dd502c
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/RemotingBootstrapFactory.java
@@ -0,0 +1,60 @@
+/*
+ * 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.rocketmq.remoting.impl.netty;
+
+import java.util.Properties;
+import org.apache.rocketmq.remoting.api.RemotingClient;
+import org.apache.rocketmq.remoting.config.RemotingConfig;
+import org.apache.rocketmq.remoting.internal.BeanUtils;
+import org.apache.rocketmq.remoting.internal.PropertyUtils;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Remoting Bootstrap entrance.
+ */
+public final class RemotingBootstrapFactory {
+    public static RemotingClient createRemotingClient(@NotNull final String fileName) {
+        Properties prop = PropertyUtils.loadProps(fileName);
+        RemotingConfig config = BeanUtils.populate(prop, RemotingConfig.class);
+        return new NettyRemotingClient(config);
+    }
+
+    public static RemotingClient createRemotingClient(@NotNull final RemotingConfig config) {
+        return new NettyRemotingClient(config);
+    }
+
+    public static RemotingClient createRemotingClient(@NotNull final Properties properties) {
+        RemotingConfig config = BeanUtils.populate(properties, RemotingConfig.class);
+        return new NettyRemotingClient(config);
+    }
+
+    public static NettyRemotingServer createRemotingServer(@NotNull final String fileName) {
+        Properties prop = PropertyUtils.loadProps(fileName);
+        RemotingConfig config = BeanUtils.populate(prop, RemotingConfig.class);
+        return new NettyRemotingServer(config);
+    }
+
+    public static NettyRemotingServer createRemotingServer(@NotNull final Properties properties) {
+        RemotingConfig config = BeanUtils.populate(properties, RemotingConfig.class);
+        return new NettyRemotingServer(config);
+    }
+
+    public static NettyRemotingServer createRemotingServer(@NotNull final RemotingConfig config) {
+        return new NettyRemotingServer(config);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/ChannelStatistics.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/ChannelStatistics.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/ChannelStatistics.java
new file mode 100755
index 0000000..ff0f9c9
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/ChannelStatistics.java
@@ -0,0 +1,61 @@
+/*
+ * 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.rocketmq.remoting.impl.netty.handler;
+
+import io.netty.channel.ChannelDuplexHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.group.ChannelGroup;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.rocketmq.remoting.common.metrics.ChannelMetrics;
+
+public class ChannelStatistics extends ChannelDuplexHandler implements ChannelMetrics {
+    public static final String NAME = ChannelStatistics.class.getSimpleName();
+    private final AtomicInteger channelCount = new AtomicInteger(0);
+    private final ChannelGroup allChannels;
+
+    public ChannelStatistics(ChannelGroup allChannels) {
+        this.allChannels = allChannels;
+    }
+
+    @Override
+    public void channelActive(ChannelHandlerContext ctx) throws Exception {
+        // connect
+        channelCount.incrementAndGet();
+        allChannels.add(ctx.channel());
+        super.channelActive(ctx);
+    }
+
+    @Override
+    public void channelInactive(ChannelHandlerContext ctx) throws Exception {
+        // disconnect
+        channelCount.decrementAndGet();
+        allChannels.remove(ctx.channel());
+        super.channelInactive(ctx);
+    }
+
+    @Override
+    public Integer getChannelCount() {
+        return channelCount.get();
+    }
+
+    @Override
+    public ChannelGroup getChannels() {
+        return allChannels;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/Decoder.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/Decoder.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/Decoder.java
new file mode 100644
index 0000000..87a0912
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/Decoder.java
@@ -0,0 +1,107 @@
+/*
+ * 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.rocketmq.remoting.impl.netty.handler;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.ByteToMessageDecoder;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.rocketmq.remoting.api.buffer.ByteBufferWrapper;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.remoting.api.exception.RemoteCodecException;
+import org.apache.rocketmq.remoting.impl.buffer.NettyByteBufferWrapper;
+import org.apache.rocketmq.remoting.impl.command.CodecHelper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Decoder extends ByteToMessageDecoder {
+    private static final Logger LOG = LoggerFactory.getLogger(Decoder.class);
+
+    public Decoder() {
+    }
+
+    @Override
+    protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) throws Exception {
+        if (!in.isReadable()) {
+            return;
+        }
+
+        NettyByteBufferWrapper wrapper = new NettyByteBufferWrapper(in, ctx.channel());
+
+        Object msg = this.decode(ctx, wrapper);
+        if (msg != null) {
+            out.add(msg);
+        }
+    }
+
+    private Object decode(final ChannelHandlerContext ctx, ByteBufferWrapper wrapper) throws Exception {
+        int originReaderIndex = wrapper.readerIndex();
+
+        byte type = wrapper.readByte();
+        try {
+            RemotingCommand cmd = decode(wrapper, originReaderIndex);
+            if (cmd != null) {
+                cmd.protocolType(type);
+            }
+            return cmd;
+        } catch (final RemoteCodecException e) {
+            LOG.warn("Decode error {}, close the channel {}", e.getMessage(), ctx.channel());
+            ctx.channel().close().addListener(new ChannelFutureListener() {
+                @Override
+                public void operationComplete(ChannelFuture future) throws Exception {
+                    LOG.warn("Close channel {} because of error {},result is {}", ctx.channel(), e, future.isSuccess());
+                }
+            });
+        }
+        return null;
+    }
+
+    public RemotingCommand decode(final ByteBufferWrapper wrapper, final int originReaderIndex) {
+        // Full message isn't available yet, return nothing for now
+        if (wrapper.readableBytes() < CodecHelper.MIN_PROTOCOL_LEN - 1) {
+            wrapper.setReaderIndex(originReaderIndex);
+            return null;
+        }
+
+        int totalLength = wrapper.readInt();
+
+        if (totalLength <= 0) {
+            throw new IllegalArgumentException("Illegal total length " + totalLength);
+        }
+
+        if (totalLength > CodecHelper.PACKET_MAX_LEN) {
+            throw new IllegalArgumentException(String.format("Total length %d is more than limit %d", totalLength, CodecHelper.PACKET_MAX_LEN));
+        }
+
+        if (wrapper.readableBytes() < totalLength) {
+            wrapper.setReaderIndex(originReaderIndex);
+            return null;
+        }
+
+        ByteBuffer totalBuffer = ByteBuffer.allocate(totalLength);
+
+        wrapper.readBytes(totalBuffer);
+
+        totalBuffer.flip();
+
+        return CodecHelper.decode(totalBuffer);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/Encoder.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/Encoder.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/Encoder.java
new file mode 100644
index 0000000..10aa504
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/Encoder.java
@@ -0,0 +1,89 @@
+/*
+ * 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.rocketmq.remoting.impl.netty.handler;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.MessageToByteEncoder;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import org.apache.rocketmq.remoting.api.buffer.ByteBufferWrapper;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.remoting.api.serializable.Serializer;
+import org.apache.rocketmq.remoting.api.serializable.SerializerFactory;
+import org.apache.rocketmq.remoting.impl.buffer.NettyByteBufferWrapper;
+import org.apache.rocketmq.remoting.impl.command.CodecHelper;
+import org.apache.rocketmq.remoting.impl.protocol.serializer.SerializerFactoryImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Encoder extends MessageToByteEncoder<RemotingCommand> {
+    private static final Logger LOG = LoggerFactory.getLogger(Encoder.class);
+
+    private final SerializerFactory serializerFactory = new SerializerFactoryImpl();
+
+    public Encoder() {
+    }
+
+    @Override
+    public void encode(final ChannelHandlerContext ctx, RemotingCommand remotingCommand, ByteBuf out) throws Exception {
+        try {
+            ByteBufferWrapper wrapper = new NettyByteBufferWrapper(out);
+
+            encode(serializerFactory, remotingCommand, wrapper);
+        } catch (final Exception e) {
+            LOG.error("Error occurred when encoding response for channel " + ((InetSocketAddress) ctx.channel().remoteAddress()).getAddress().getHostAddress(), e);
+            if (remotingCommand != null) {
+                LOG.error(remotingCommand.toString());
+            }
+            ctx.channel().close().addListener(new ChannelFutureListener() {
+                @Override
+                public void operationComplete(ChannelFuture future) throws Exception {
+                    LOG.warn("Close channel {} because of error {},result is {}", ctx.channel(), e, future.isSuccess());
+                }
+            });
+        }
+    }
+
+    public void encode(final SerializerFactory serializerFactory, final RemotingCommand remotingCommand,
+        final ByteBufferWrapper out) {
+        ByteBuffer encodeParameter = null;
+        if (remotingCommand.parameterBytes() != null) {
+            encodeParameter = ByteBuffer.wrap(remotingCommand.parameterBytes());
+        } else if (remotingCommand.parameter() != null) {
+            final Serializer serialization = serializerFactory.get(remotingCommand.serializerType());
+            encodeParameter = serialization.encode(remotingCommand.parameter());
+        }
+
+        int parameterLength = encodeParameter != null ? encodeParameter.limit() : 0;
+        int extBodyLength = remotingCommand.extraPayload() != null ? remotingCommand.extraPayload().length : 0;
+
+        ByteBuffer header = CodecHelper.encodeHeader(remotingCommand, parameterLength, extBodyLength);
+        out.writeBytes(header);
+
+        if (encodeParameter != null) {
+            out.writeBytes(encodeParameter);
+        }
+
+        if (remotingCommand.extraPayload() != null) {
+            out.writeBytes(remotingCommand.extraPayload());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/ExceptionHandler.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/ExceptionHandler.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/ExceptionHandler.java
new file mode 100644
index 0000000..52563f4
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/ExceptionHandler.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rocketmq.remoting.impl.netty.handler;
+
+import io.netty.channel.ChannelDuplexHandler;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@ChannelHandler.Sharable
+public class ExceptionHandler extends ChannelDuplexHandler {
+    private final static Logger LOG = LoggerFactory.getLogger(ExceptionHandler.class);
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
+        // Uncaught exceptions from inbound handlers will propagate up to this handler
+        LOG.error(String.format("channel exception %s occurred ! ", ctx.channel()), cause);
+        ctx.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/Http2Handler.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/Http2Handler.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/Http2Handler.java
new file mode 100644
index 0000000..7cdb976
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/Http2Handler.java
@@ -0,0 +1,139 @@
+/*
+ * 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.rocketmq.remoting.impl.netty.handler;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelPromise;
+import io.netty.handler.codec.http2.DefaultHttp2Connection;
+import io.netty.handler.codec.http2.DefaultHttp2ConnectionDecoder;
+import io.netty.handler.codec.http2.DefaultHttp2ConnectionEncoder;
+import io.netty.handler.codec.http2.DefaultHttp2FrameReader;
+import io.netty.handler.codec.http2.DefaultHttp2FrameWriter;
+import io.netty.handler.codec.http2.DefaultHttp2Headers;
+import io.netty.handler.codec.http2.DefaultHttp2HeadersDecoder;
+import io.netty.handler.codec.http2.DefaultHttp2LocalFlowController;
+import io.netty.handler.codec.http2.Http2Connection;
+import io.netty.handler.codec.http2.Http2ConnectionDecoder;
+import io.netty.handler.codec.http2.Http2ConnectionEncoder;
+import io.netty.handler.codec.http2.Http2ConnectionHandler;
+import io.netty.handler.codec.http2.Http2Exception;
+import io.netty.handler.codec.http2.Http2FrameAdapter;
+import io.netty.handler.codec.http2.Http2FrameReader;
+import io.netty.handler.codec.http2.Http2FrameWriter;
+import io.netty.handler.codec.http2.Http2Headers;
+import io.netty.handler.codec.http2.Http2HeadersDecoder;
+import io.netty.handler.codec.http2.Http2Settings;
+import io.netty.handler.codec.http2.StreamBufferingEncoder;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.OK;
+import static io.netty.handler.codec.http2.DefaultHttp2LocalFlowController.DEFAULT_WINDOW_UPDATE_RATIO;
+
+public class Http2Handler extends Http2ConnectionHandler {
+
+    private boolean isServer;
+    private int lastStreamId;
+
+    private Http2Handler(final Http2ConnectionDecoder decoder, final Http2ConnectionEncoder encoder,
+        final Http2Settings initialSettings, final boolean isServer) {
+        super(decoder, encoder, initialSettings);
+        decoder.frameListener(new FrameListener());
+        this.isServer = isServer;
+    }
+
+    public static Http2Handler newHandler(final boolean isServer) {
+
+        Http2HeadersDecoder headersDecoder = new DefaultHttp2HeadersDecoder(true);
+        Http2FrameReader frameReader = new DefaultHttp2FrameReader(headersDecoder);
+        Http2FrameWriter frameWriter = new DefaultHttp2FrameWriter();
+
+        Http2Connection connection = new DefaultHttp2Connection(isServer);
+
+        Http2ConnectionEncoder encoder = new StreamBufferingEncoder(
+            new DefaultHttp2ConnectionEncoder(connection, frameWriter));
+
+        connection.local().flowController(new DefaultHttp2LocalFlowController(connection,
+            DEFAULT_WINDOW_UPDATE_RATIO, true));
+
+        Http2ConnectionDecoder decoder = new DefaultHttp2ConnectionDecoder(connection, encoder,
+            frameReader);
+
+        Http2Settings settings = new Http2Settings();
+
+        if (!isServer)
+            settings.pushEnabled(true);
+
+        settings.initialWindowSize(1048576 * 10); //10MiB
+        settings.maxConcurrentStreams(Integer.MAX_VALUE);
+
+        return newHandler(decoder, encoder, settings, isServer);
+    }
+
+    private static Http2Handler newHandler(final Http2ConnectionDecoder decoder, final Http2ConnectionEncoder encoder,
+        final Http2Settings settings, boolean isServer) {
+        return new Http2Handler(decoder, encoder, settings, isServer);
+    }
+
+    @Override
+    public void write(final ChannelHandlerContext ctx, final Object msg,
+        final ChannelPromise promise) throws Exception {
+        if (isServer) {
+            assert msg instanceof ByteBuf;
+            sendAPushPromise(ctx, lastStreamId, lastStreamId + 1, (ByteBuf) msg);
+        } else {
+
+            final Http2Headers headers = new DefaultHttp2Headers();
+
+            try {
+                long threadId = Thread.currentThread().getId();
+                long streamId = (threadId % 2 == 0) ? threadId + 1 : threadId + 2;
+                encoder().writeHeaders(ctx, (int) streamId, headers, 0, false, promise);
+                encoder().writeData(ctx, (int) streamId, (ByteBuf) msg, 0, false, ctx.newPromise());
+                ctx.flush();
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+
+        }
+    }
+
+    private void sendAPushPromise(ChannelHandlerContext ctx, int streamId, int pushPromiseStreamId,
+        ByteBuf payload) throws Http2Exception {
+
+        encoder().writePushPromise(ctx, streamId, pushPromiseStreamId,
+            new DefaultHttp2Headers().status(OK.codeAsText()), 0, ctx.newPromise());
+
+        //Http2Stream stream = connection.local().reservePushStream(pushPromiseStreamId, connection.connectionStream());
+        Http2Headers headers = new DefaultHttp2Headers();
+        headers.status(OK.codeAsText());
+        encoder().writeHeaders(ctx, pushPromiseStreamId, headers, 0, false, ctx.newPromise());
+        encoder().writeData(ctx, pushPromiseStreamId, payload, 0, false, ctx.newPromise());
+    }
+
+    private class FrameListener extends Http2FrameAdapter {
+        @Override
+        public int onDataRead(ChannelHandlerContext ctx, int streamId, ByteBuf data, int padding,
+            boolean endOfStream) throws Http2Exception {
+            //Http2Handler.this.onDataRead(ctx, streamId, data, endOfStream);
+            data.retain();
+            Http2Handler.this.lastStreamId = streamId;
+            ctx.fireChannelRead(data);
+            return data.readableBytes() + padding;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/ProtocolSelector.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/ProtocolSelector.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/ProtocolSelector.java
new file mode 100644
index 0000000..e00a213
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/netty/handler/ProtocolSelector.java
@@ -0,0 +1,65 @@
+/*
+ * 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.rocketmq.remoting.impl.netty.handler;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.SimpleChannelInboundHandler;
+import io.netty.handler.ssl.SslContext;
+import org.apache.rocketmq.remoting.api.channel.ChannelHandlerContextWrapper;
+import org.apache.rocketmq.remoting.api.protocol.Protocol;
+import org.apache.rocketmq.remoting.api.protocol.ProtocolFactory;
+import org.apache.rocketmq.remoting.impl.channel.ChannelHandlerContextWrapperImpl;
+import org.apache.rocketmq.remoting.impl.protocol.ProtocolFactoryImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ProtocolSelector extends SimpleChannelInboundHandler<ByteBuf> {
+    public static final String NAME = ProtocolSelector.class.getSimpleName();
+    private static final Logger LOG = LoggerFactory.getLogger(ProtocolSelector.class);
+    private ProtocolFactory protocolFactory;
+
+    public ProtocolSelector(final SslContext sslContext) {
+        this.protocolFactory = new ProtocolFactoryImpl(sslContext);
+    }
+
+    @Override
+    protected void channelRead0(final ChannelHandlerContext ctx, final ByteBuf msg) throws Exception {
+        if (msg.readableBytes() < 1) {
+            return;
+        }
+        msg.markReaderIndex();
+        Protocol protocol = protocolFactory.get(msg.readByte());
+        if (protocol == null) {
+            ctx.channel().close().addListener(new ChannelFutureListener() {
+                @Override
+                public void operationComplete(ChannelFuture future) throws Exception {
+                    LOG.warn("Close channel {},result is {}", ctx.channel(), future.isSuccess());
+                }
+            });
+            return;
+        }
+        ChannelHandlerContextWrapper chcw = new ChannelHandlerContextWrapperImpl(ctx);
+        protocol.assembleHandler(chcw);
+        msg.resetReaderIndex();
+        ctx.pipeline().remove(this);
+        ctx.fireChannelRead(msg.retain());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/Httpv2Protocol.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/Httpv2Protocol.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/Httpv2Protocol.java
new file mode 100644
index 0000000..0740cbb
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/Httpv2Protocol.java
@@ -0,0 +1,53 @@
+/*
+ * 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.rocketmq.remoting.impl.protocol;
+
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.ssl.SslContext;
+import org.apache.rocketmq.remoting.api.channel.ChannelHandlerContextWrapper;
+import org.apache.rocketmq.remoting.api.protocol.Protocol;
+import org.apache.rocketmq.remoting.impl.netty.handler.Http2Handler;
+import org.apache.rocketmq.remoting.impl.netty.handler.ProtocolSelector;
+
+public class Httpv2Protocol extends RemotingCoreProtocol {
+    private SslContext sslContext;
+
+    public Httpv2Protocol(final SslContext sslContext) {
+        this.sslContext = sslContext;
+    }
+
+    @Override
+    public String name() {
+        return Protocol.HTTP2;
+    }
+
+    @Override
+    public byte type() {
+        return Protocol.HTTP_2_MAGIC;
+    }
+
+    @Override
+    public void assembleHandler(final ChannelHandlerContextWrapper ctx) {
+        super.assembleHandler(ctx);
+        ChannelHandlerContext chx = (ChannelHandlerContext) ctx.getContext();
+
+        chx.pipeline().addAfter(ProtocolSelector.NAME, "sslHandler", sslContext.newHandler(chx.alloc()));
+        chx.pipeline().addAfter("sslHandler", "http2Handler", Http2Handler.newHandler(true));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/ProtocolFactoryImpl.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/ProtocolFactoryImpl.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/ProtocolFactoryImpl.java
new file mode 100644
index 0000000..15322be
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/ProtocolFactoryImpl.java
@@ -0,0 +1,83 @@
+/*
+ * 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.rocketmq.remoting.impl.protocol;
+
+import io.netty.handler.ssl.SslContext;
+import org.apache.rocketmq.remoting.api.protocol.Protocol;
+import org.apache.rocketmq.remoting.api.protocol.ProtocolFactory;
+
+public class ProtocolFactoryImpl implements ProtocolFactory {
+    private static final int MAX_COUNT = 0x0FF;
+    private final Protocol[] tables = new Protocol[MAX_COUNT];
+
+    private SslContext sslContext;
+
+    public ProtocolFactoryImpl(final SslContext sslContext) {
+        this.sslContext = sslContext;
+        this.register(new RemotingCoreProtocol());
+        this.register(new Httpv2Protocol(sslContext));
+        this.register(new WebSocketProtocol());
+    }
+
+    public ProtocolFactoryImpl() {
+        this.register(new RemotingCoreProtocol());
+        this.register(new Httpv2Protocol(sslContext));
+        this.register(new WebSocketProtocol());
+    }
+
+    @Override
+    public void register(Protocol protocol) {
+        if (tables[protocol.type() & MAX_COUNT] != null) {
+            throw new RuntimeException("protocol header's sign is overlapped");
+        }
+        tables[protocol.type() & MAX_COUNT] = protocol;
+    }
+
+    @Override
+    public void resetAll(final Protocol protocol) {
+        for (int i = 0; i < MAX_COUNT; i++) {
+            tables[i] = protocol;
+        }
+    }
+
+    @Override
+    public byte type(final String protocolName) {
+
+        for (int i = 0; i < this.tables.length; i++) {
+            if (this.tables[i] != null) {
+                if (this.tables[i].name().equalsIgnoreCase(protocolName)) {
+                    return this.tables[i].type();
+                }
+            }
+        }
+
+        throw new IllegalArgumentException(String.format("the protocol: %s not exist", protocolName));
+    }
+
+    @Override
+    public Protocol get(byte type) {
+        return tables[type & MAX_COUNT];
+    }
+
+    @Override
+    public void clearAll() {
+        for (int i = 0; i < this.tables.length; i++) {
+            this.tables[i] = null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/RemotingCoreProtocol.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/RemotingCoreProtocol.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/RemotingCoreProtocol.java
new file mode 100644
index 0000000..317b24f
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/RemotingCoreProtocol.java
@@ -0,0 +1,46 @@
+/*
+ * 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.rocketmq.remoting.impl.protocol;
+
+import io.netty.channel.ChannelHandlerContext;
+import org.apache.rocketmq.remoting.api.channel.ChannelHandlerContextWrapper;
+import org.apache.rocketmq.remoting.api.protocol.Protocol;
+import org.apache.rocketmq.remoting.impl.netty.handler.Decoder;
+import org.apache.rocketmq.remoting.impl.netty.handler.Encoder;
+import org.apache.rocketmq.remoting.impl.netty.handler.ProtocolSelector;
+
+public class RemotingCoreProtocol implements Protocol {
+    @Override
+    public String name() {
+        return MVP;
+    }
+
+    @Override
+    public byte type() {
+        return MVP_MAGIC;
+    }
+
+    @Override
+    public void assembleHandler(final ChannelHandlerContextWrapper ctx) {
+
+        ChannelHandlerContext chx = (ChannelHandlerContext) ctx.getContext();
+
+        chx.pipeline().addAfter(ProtocolSelector.NAME, "decoder", new Decoder());
+        chx.pipeline().addAfter("decoder", "encoder", new Encoder());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/WebSocketProtocol.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/WebSocketProtocol.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/WebSocketProtocol.java
new file mode 100644
index 0000000..18a3a11
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/WebSocketProtocol.java
@@ -0,0 +1,38 @@
+/*
+ * 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.rocketmq.remoting.impl.protocol;
+
+import org.apache.rocketmq.remoting.api.channel.ChannelHandlerContextWrapper;
+import org.apache.rocketmq.remoting.api.protocol.Protocol;
+
+public class WebSocketProtocol implements Protocol {
+    @Override
+    public String name() {
+        return WEBSOCKET;
+    }
+
+    @Override
+    public byte type() {
+        return WEBSOCKET_MAGIC;
+    }
+
+    @Override
+    public void assembleHandler(final ChannelHandlerContextWrapper ctx) {
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/compression/CompressorFactoryImpl.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/compression/CompressorFactoryImpl.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/compression/CompressorFactoryImpl.java
new file mode 100644
index 0000000..10e97ba
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/compression/CompressorFactoryImpl.java
@@ -0,0 +1,68 @@
+/*
+ * 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.rocketmq.remoting.impl.protocol.compression;
+
+import org.apache.rocketmq.remoting.api.compressable.Compressor;
+import org.apache.rocketmq.remoting.api.compressable.CompressorFactory;
+
+public class CompressorFactoryImpl implements CompressorFactory {
+    private static final int MAX_COUNT = 0x0FF;
+    private final Compressor[] tables = new Compressor[MAX_COUNT];
+
+    public CompressorFactoryImpl() {
+        this.register(new GZipCompressor());
+    }
+
+    @Override
+    public void register(Compressor compressor) {
+        if (tables[compressor.type() & MAX_COUNT] != null) {
+            throw new RuntimeException("compressor header's sign is overlapped");
+        }
+        tables[compressor.type() & MAX_COUNT] = compressor;
+    }
+
+    @Override
+    public byte type(String compressionName) {
+        for (Compressor table : this.tables) {
+            if (table != null) {
+                if (table.name().equalsIgnoreCase(compressionName)) {
+                    return table.type();
+                }
+            }
+        }
+
+        throw new IllegalArgumentException(String.format("the compressor: %s not exist", compressionName));
+
+    }
+
+    @Override
+    public Compressor get(byte type) {
+        return tables[type & MAX_COUNT];
+    }
+
+    @Override
+    public void clearAll() {
+        for (int i = 0; i < this.tables.length; i++) {
+            this.tables[i] = null;
+        }
+    }
+
+    public Compressor[] getTables() {
+        return tables;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/compression/GZipCompressor.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/compression/GZipCompressor.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/compression/GZipCompressor.java
new file mode 100644
index 0000000..fc33f4c
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/compression/GZipCompressor.java
@@ -0,0 +1,100 @@
+/*
+ * 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.rocketmq.remoting.impl.protocol.compression;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.zip.GZIPInputStream;
+import java.util.zip.GZIPOutputStream;
+import org.apache.rocketmq.remoting.api.compressable.Compressor;
+
+public class GZipCompressor implements Compressor {
+    public static final int BUFFER = 1024;
+    public static final String COMPRESSOR_NAME = GZipCompressor.class.getSimpleName();
+    public static final byte COMPRESSOR_TYPE = 'G';
+
+    @Override
+    public String name() {
+        return COMPRESSOR_NAME;
+    }
+
+    @Override
+    public byte type() {
+        return COMPRESSOR_TYPE;
+    }
+
+    @Override
+    public byte[] compress(byte[] content) throws Exception {
+        if (content == null)
+            return new byte[0];
+
+        ByteArrayInputStream bais = new ByteArrayInputStream(content);
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+        compress(bais, baos);
+        byte[] output = baos.toByteArray();
+        baos.flush();
+        baos.close();
+        bais.close();
+        return output;
+
+    }
+
+    private void compress(InputStream is, OutputStream os) throws Exception {
+        GZIPOutputStream gos = new GZIPOutputStream(os);
+
+        int count;
+        byte data[] = new byte[BUFFER];
+        while ((count = is.read(data, 0, BUFFER)) != -1) {
+            gos.write(data, 0, count);
+        }
+        gos.finish();
+        gos.flush();
+        gos.close();
+    }
+
+    @Override
+    public byte[] deCompress(byte[] content) throws Exception {
+        if (content == null)
+            return new byte[0];
+
+        ByteArrayInputStream bais = new ByteArrayInputStream(content);
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+        decompress(bais, baos);
+        content = baos.toByteArray();
+        baos.flush();
+        baos.close();
+        bais.close();
+        return content;
+    }
+
+    private void decompress(InputStream is, OutputStream os) throws Exception {
+        GZIPInputStream gis = new GZIPInputStream(is);
+
+        int count;
+        byte data[] = new byte[BUFFER];
+        while ((count = gis.read(data, 0, BUFFER)) != -1) {
+            os.write(data, 0, count);
+        }
+        gis.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/JsonSerializer.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/JsonSerializer.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/JsonSerializer.java
new file mode 100644
index 0000000..c85d44b
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/JsonSerializer.java
@@ -0,0 +1,88 @@
+/*
+ * 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.rocketmq.remoting.impl.protocol.serializer;
+
+import com.alibaba.fastjson.JSON;
+import java.lang.reflect.Type;
+import java.nio.ByteBuffer;
+import org.apache.rocketmq.remoting.api.serializable.Serializer;
+import org.apache.rocketmq.remoting.common.TypePresentation;
+import org.apache.rocketmq.remoting.impl.command.CodecHelper;
+
+public class JsonSerializer implements Serializer {
+    public static final String SERIALIZER_NAME = JsonSerializer.class.getSimpleName();
+    public static final byte SERIALIZER_TYPE = 'J';
+
+    public JsonSerializer() {
+    }
+
+    @Override
+    public String name() {
+        return SERIALIZER_NAME;
+    }
+
+    @Override
+    public byte type() {
+        return SERIALIZER_TYPE;
+    }
+
+    @Override
+    public <T> T decode(final byte[] content, final Class<T> c) {
+        if (content != null) {
+            try {
+                final String jsonString = new String(content, CodecHelper.REMOTING_CHARSET);
+                return JSON.parseObject(jsonString, c);
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public <T> T decode(final byte[] content, final TypePresentation<T> typePresentation) {
+        return decode(content, typePresentation.getType());
+    }
+
+    @Override
+    public <T> T decode(byte[] content, Type type) {
+        if (content != null) {
+            try {
+                final String jsonString = new String(content, CodecHelper.REMOTING_CHARSET);
+                return JSON.parseObject(jsonString, type);
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public ByteBuffer encode(final Object object) {
+        if (object != null) {
+            String jsonString = JSON.toJSONString(object);
+            byte[] bytes = jsonString.getBytes(CodecHelper.REMOTING_CHARSET);
+            try {
+                return ByteBuffer.wrap(bytes, 0, bytes.length);
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/Kryo3Serializer.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/Kryo3Serializer.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/Kryo3Serializer.java
new file mode 100644
index 0000000..06ea217
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/Kryo3Serializer.java
@@ -0,0 +1,90 @@
+/*
+ * 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.rocketmq.remoting.impl.protocol.serializer;
+
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.nio.ByteBuffer;
+import org.apache.rocketmq.remoting.api.serializable.Serializer;
+import org.apache.rocketmq.remoting.common.TypePresentation;
+
+public class Kryo3Serializer implements Serializer {
+    public static final String SERIALIZER_NAME = Kryo3Serializer.class.getSimpleName();
+    public static final byte SERIALIZER_TYPE = 'K';
+
+    public Kryo3Serializer() {
+    }
+
+    @Override
+    public String name() {
+        return SERIALIZER_NAME;
+    }
+
+    @Override
+    public byte type() {
+        return SERIALIZER_TYPE;
+    }
+
+    @Override
+    public <T> T decode(final byte[] content, final Class<T> c) {
+        if (content != null) {
+            Input input = null;
+            try {
+                input = new Input(content);
+                return (T) ThreadSafeKryo.getKryoInstance().readClassAndObject(input);
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            } finally {
+                input.close();
+            }
+        }
+
+        return null;
+    }
+
+    @Override
+    public <T> T decode(final byte[] content, final TypePresentation<T> typePresentation) {
+        return decode(content, typePresentation.getType());
+    }
+
+    @Override
+    public <T> T decode(byte[] content, Type type) {
+        if (type instanceof ParameterizedType) {
+            return decode(content, (Class<? extends T>) ((ParameterizedType) type).getRawType());
+        } else if (type instanceof Class) {
+            return decode(content, (Class<? extends T>) type);
+        }
+        return null;
+    }
+
+    @Override
+    public ByteBuffer encode(final Object object) {
+        if (object != null) {
+            try (Output output = new Output(1024, 1024 * 1024 * 6)) {
+                ThreadSafeKryo.getKryoInstance().writeClassAndObject(output, object);
+                return ByteBuffer.wrap(output.getBuffer(), 0, output.position());
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/MsgPackSerializer.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/MsgPackSerializer.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/MsgPackSerializer.java
new file mode 100644
index 0000000..1097f8f
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/MsgPackSerializer.java
@@ -0,0 +1,78 @@
+/*
+ * 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.rocketmq.remoting.impl.protocol.serializer;
+
+import java.lang.reflect.Type;
+import java.nio.ByteBuffer;
+import org.apache.rocketmq.remoting.api.serializable.Serializer;
+import org.apache.rocketmq.remoting.common.TypePresentation;
+import org.msgpack.MessagePack;
+import org.msgpack.template.Template;
+
+public class MsgPackSerializer implements Serializer {
+    public static final String SERIALIZER_NAME = MsgPackSerializer.class.getSimpleName();
+    public static final byte SERIALIZER_TYPE = 'M';
+    private final MessagePack messagePack = new MessagePack();
+
+    public MsgPackSerializer() {
+    }
+
+    @Override
+    public String name() {
+        return SERIALIZER_NAME;
+    }
+
+    @Override
+    public byte type() {
+        return SERIALIZER_TYPE;
+    }
+
+    @Override
+    public <T> T decode(final byte[] content, final Class<T> c) {
+        try {
+            return messagePack.read(content, c);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public <T> T decode(final byte[] content, final TypePresentation<T> typePresentation) {
+        return decode(content, typePresentation.getType());
+    }
+
+    @Override
+    public <T> T decode(byte[] content, Type type) {
+        Template<T> template = (Template<T>) messagePack.lookup(type);
+        try {
+            return messagePack.read(content, template);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public ByteBuffer encode(final Object object) {
+        try {
+            byte[] data = messagePack.write(object);
+            return ByteBuffer.wrap(data);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/SerializerFactoryImpl.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/SerializerFactoryImpl.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/SerializerFactoryImpl.java
new file mode 100644
index 0000000..632b61f
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/SerializerFactoryImpl.java
@@ -0,0 +1,69 @@
+/*
+ * 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.rocketmq.remoting.impl.protocol.serializer;
+
+import org.apache.rocketmq.remoting.api.serializable.Serializer;
+import org.apache.rocketmq.remoting.api.serializable.SerializerFactory;
+
+public class SerializerFactoryImpl implements SerializerFactory {
+    private static final int MAX_COUNT = 0x0FF;
+    private final Serializer[] tables = new Serializer[MAX_COUNT];
+
+    public SerializerFactoryImpl() {
+        this.register(new JsonSerializer());
+        this.register(new Kryo3Serializer());
+        this.register(new MsgPackSerializer());
+    }
+
+    @Override
+    public void register(Serializer serialization) {
+        if (tables[serialization.type() & MAX_COUNT] != null) {
+            throw new RuntimeException("serialization header's sign is overlapped");
+        }
+        tables[serialization.type() & MAX_COUNT] = serialization;
+    }
+
+    @Override
+    public byte type(final String serializationName) {
+        for (Serializer table : this.tables) {
+            if (table != null) {
+                if (table.name().equalsIgnoreCase(serializationName)) {
+                    return table.type();
+                }
+            }
+        }
+
+        throw new IllegalArgumentException(String.format("the serialization: %s not exist", serializationName));
+    }
+
+    @Override
+    public Serializer get(byte type) {
+        return tables[type & MAX_COUNT];
+    }
+
+    @Override
+    public void clearAll() {
+        for (int i = 0; i < this.tables.length; i++) {
+            this.tables[i] = null;
+        }
+    }
+
+    public Serializer[] getTables() {
+        return tables;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/ThreadSafeKryo.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/ThreadSafeKryo.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/ThreadSafeKryo.java
new file mode 100644
index 0000000..cadfc27
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/impl/protocol/serializer/ThreadSafeKryo.java
@@ -0,0 +1,99 @@
+/*
+ * 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.rocketmq.remoting.impl.protocol.serializer;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.KryoSerializable;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Calendar;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Currency;
+import java.util.Date;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import org.objenesis.strategy.StdInstantiatorStrategy;
+
+public class ThreadSafeKryo {
+    private static final ThreadLocal<Kryo> KRYOS = new ThreadLocal<Kryo>() {
+        protected Kryo initialValue() {
+            Kryo kryo = new Kryo();
+
+            kryo.register(byte[].class);
+            kryo.register(char[].class);
+            kryo.register(short[].class);
+            kryo.register(int[].class);
+            kryo.register(long[].class);
+            kryo.register(float[].class);
+            kryo.register(double[].class);
+            kryo.register(boolean[].class);
+            kryo.register(String[].class);
+            kryo.register(Object[].class);
+            kryo.register(KryoSerializable.class);
+            kryo.register(BigInteger.class);
+            kryo.register(BigDecimal.class);
+            kryo.register(Class.class);
+            kryo.register(Date.class);
+            // kryo.register(Enum.class);
+            kryo.register(EnumSet.class);
+            kryo.register(Currency.class);
+            kryo.register(StringBuffer.class);
+            kryo.register(StringBuilder.class);
+            kryo.register(Collections.EMPTY_LIST.getClass());
+            kryo.register(Collections.EMPTY_MAP.getClass());
+            kryo.register(Collections.EMPTY_SET.getClass());
+            kryo.register(Collections.singletonList(null).getClass());
+            kryo.register(Collections.singletonMap(null, null).getClass());
+            kryo.register(Collections.singleton(null).getClass());
+            kryo.register(TreeSet.class);
+            kryo.register(Collection.class);
+            kryo.register(TreeMap.class);
+            kryo.register(Map.class);
+            try {
+                kryo.register(Class.forName("sun.util.calendar.ZoneInfo"));
+            } catch (ClassNotFoundException e) {
+                // Noop
+            }
+            kryo.register(Calendar.class);
+            kryo.register(Locale.class);
+
+            kryo.register(BitSet.class);
+            kryo.register(HashMap.class);
+            kryo.register(Timestamp.class);
+            kryo.register(ArrayList.class);
+
+            // kryo.setRegistrationRequired(true);
+            kryo.setReferences(false);
+            kryo.setInstantiatorStrategy(new Kryo.DefaultInstantiatorStrategy(new StdInstantiatorStrategy()));
+
+            return kryo;
+        }
+    };
+
+    public static Kryo getKryoInstance() {
+        return KRYOS.get();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/BeanUtils.java
----------------------------------------------------------------------
diff --git a/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/BeanUtils.java b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/BeanUtils.java
new file mode 100644
index 0000000..0177990
--- /dev/null
+++ b/remoting-core/remoting-impl/src/main/java/org/apache/rocketmq/remoting/internal/BeanUtils.java
@@ -0,0 +1,210 @@
+/*
+ * 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.rocketmq.remoting.internal;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public final class BeanUtils {
+    private final static Logger LOG = LoggerFactory.getLogger(BeanUtils.class);
+
+    /**
+     * <p>Populate the JavaBeans properties of the specified bean, based on
+     * the specified name/value pairs.  This method uses Java reflection APIs
+     * to identify corresponding "property setter" method names, and deals
+     * with setter arguments of type <Code>String</Code>, <Code>boolean</Code>,
+     * <Code>int</Code>, <Code>long</Code>, <Code>float</Code>, and
+     * <Code>double</Code>.</p>
+     *
+     * <p>The particular setter method to be called for each property is
+     * determined using the usual JavaBeans introspection mechanisms.  Thus,
+     * you may identify custom setter methods using a BeanInfo class that is
+     * associated with the class of the bean itself.  If no such BeanInfo
+     * class is available, the standard method name conversion ("set" plus
+     * the capitalized name of the property in question) is used.</p>
+     *
+     * <p><strong>NOTE</strong>:  It is contrary to the JavaBeans Specification
+     * to have more than one setter method (with different argument
+     * signatures) for the same property.</p>
+     *
+     * @param clazz JavaBean class whose properties are being populated
+     * @param properties Map keyed by property name, with the corresponding (String or String[]) value(s) to be set
+     * @param <T> Class type
+     * @return Class instance
+     */
+    public static <T> T populate(final Properties properties, final Class<T> clazz) {
+        T obj = null;
+        try {
+            obj = clazz.newInstance();
+            return populate(properties, obj);
+        } catch (Throwable e) {
+            LOG.warn("Error occurs !", e);
+        }
+        return obj;
+    }
+
+    private static <T> void setField(final Field field, final Properties properties, final T obj) throws Exception {
+        Type fieldType = field.getType();
+        String fieldName = field.getName();
+
+        String value = null;
+        String configName = convertToConfigName(fieldName);
+        String envName = convertToEnvName(fieldName);
+
+        if (properties.containsKey(envName)) {
+            value = properties.getProperty(envName);
+        }
+
+        if (properties.containsKey(configName)) {
+            value = properties.getProperty(configName);
+        }
+
+        if (value == null) {
+            return;
+        }
+
+        if (fieldType == Boolean.TYPE) {
+            field.set(obj, Boolean.valueOf(value));
+        } else if (fieldType == Integer.TYPE) {
+            field.set(obj, Integer.valueOf(value));
+        } else if (fieldType == Double.TYPE) {
+            field.set(obj, Double.valueOf(value));
+        } else if (fieldType == Float.TYPE) {
+            field.set(obj, Float.valueOf(value));
+        } else if (fieldType == Long.TYPE) {
+            field.set(obj, Long.valueOf(value));
+        } else
+            field.set(obj, value);
+    }
+
+    private static String convertToConfigName(String variableName) {
+        StringBuilder sb = new StringBuilder();
+        for (char c : variableName.toCharArray()) {
+            if (Character.isUpperCase(c)) {
+                sb.append('.');
+            }
+            sb.append(Character.toLowerCase(c));
+        }
+        return sb.toString();
+    }
+
+    private static String convertToEnvName(String variableName) {
+        StringBuilder sb = new StringBuilder();
+        for (char c : variableName.toCharArray()) {
+            if (Character.isUpperCase(c)) {
+                sb.append('_');
+            }
+            sb.append(Character.toUpperCase(c));
+        }
+        return sb.toString();
+    }
+
+    public static <T> T populate(final Properties properties, final T obj) {
+        Class<?> clazz = obj.getClass();
+        List<Field> allFields = new ArrayList<>();
+        allFields = getAllFields(allFields, clazz);
+        Properties fullProp = extractProperties(properties);
+
+        try {
+            for (Field field : allFields) {
+                if (!Modifier.isStatic(field.getModifiers())) {
+                    field.setAccessible(true);
+                    setField(field, fullProp, obj);
+                }
+            }
+        } catch (Exception e) {
+            LOG.warn("Error occurs !", e);
+        }
+        return obj;
+    }
+
+    public static String configObjectToString(final Object object) {
+        List<Field> allFields = new ArrayList<>();
+        getAllFields(allFields, object.getClass());
+        StringBuilder sb = new StringBuilder();
+        for (Field field : allFields) {
+            if (!Modifier.isStatic(field.getModifiers())) {
+                String name = field.getName();
+                if (!name.startsWith("this")) {
+                    Object value = null;
+                    try {
+                        field.setAccessible(true);
+                        value = field.get(object);
+                        if (null == value) {
+                            value = "";
+                        }
+                    } catch (IllegalAccessException ignored) {
+                    }
+                    sb.append(name).append("=").append(value).append("%n");
+                }
+            }
+        }
+        return sb.toString();
+    }
+
+    private static List<Field> getAllFields(List<Field> fields, Class<?> type) {
+        fields.addAll(Arrays.asList(type.getDeclaredFields()));
+        if (type.getSuperclass() != null) {
+            getAllFields(fields, type.getSuperclass());
+        }
+        return fields;
+    }
+
+    private static Properties extractProperties(final Properties properties) {
+        Properties newPro = new Properties();
+
+        Map<String, String> envMap = System.getenv();
+        for (final Map.Entry<String, String> entry : envMap.entrySet()) {
+            newPro.setProperty(entry.getKey().toUpperCase(), entry.getValue());
+            newPro.setProperty(entry.getKey().toLowerCase(), entry.getValue()); //EnvProp supports A_B_C and a.b.c
+        }
+
+        Properties systemProp = System.getProperties(); //SystemProp supports a.b.c
+        for (final Map.Entry<Object, Object> entry : systemProp.entrySet()) {
+            newPro.setProperty(String.valueOf(entry.getKey()).toLowerCase(), String.valueOf(entry.getValue()));
+        }
+
+        Properties inner = null;
+        try {
+            Field field = Properties.class.getDeclaredField("defaults");
+            field.setAccessible(true);
+            inner = (Properties) field.get(properties);
+        } catch (Exception ignore) {
+        }
+
+        if (inner != null) {
+            for (final Map.Entry<Object, Object> entry : inner.entrySet()) {
+                newPro.setProperty(String.valueOf(entry.getKey()).toLowerCase(), String.valueOf(entry.getValue()));
+            }
+        }
+
+        for (final Map.Entry<Object, Object> entry : properties.entrySet()) {
+            newPro.setProperty(String.valueOf(entry.getKey()).toLowerCase(), String.valueOf(entry.getValue()));
+        }
+
+        return newPro;
+    }
+}


[2/8] incubator-rocketmq git commit: initialize RocketMQ5

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/StatsItem.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/StatsItem.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/StatsItem.java
new file mode 100644
index 0000000..219ee6e
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/StatsItem.java
@@ -0,0 +1,442 @@
+/*
+ * 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.rocketmq.rpc.impl.metrics;
+
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.LinkedList;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import org.slf4j.Logger;
+
+public class StatsItem {
+    private final AtomicLong value = new AtomicLong(0);
+    private final AtomicLong times = new AtomicLong(0);
+    private final AtomicLong[] valueIncDistributeRegion = new AtomicLong[10];
+    private final AtomicLong valueMaxInMinutes = new AtomicLong(0);
+    private final AtomicLong valueMaxIn10Minutes = new AtomicLong(0);
+    private final AtomicLong valueMaxInHour = new AtomicLong(0);
+
+    private final LinkedList<CallSnapshot> csListMinute = new LinkedList<CallSnapshot>();
+    private final LinkedList<CallSnapshot> csListHour = new LinkedList<CallSnapshot>();
+    private final LinkedList<CallSnapshot> csListDay = new LinkedList<CallSnapshot>();
+
+    private final ScheduledExecutorService scheduledExecutorService;
+    private final String statsName;
+    private final String statsKey;
+    private final Logger log;
+
+    public StatsItem(String statsName, String statsKey, ScheduledExecutorService scheduledExecutorService, Logger log) {
+        this.statsName = statsName;
+        this.statsKey = statsKey;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.log = log;
+
+        for (int i = 0; i < this.valueIncDistributeRegion.length; i++) {
+            valueIncDistributeRegion[i] = new AtomicLong(0);
+        }
+    }
+
+    public static boolean compareAndIncreaseOnly(final AtomicLong target, final long value) {
+        long prev = target.get();
+        while (value > prev) {
+            boolean updated = target.compareAndSet(prev, value);
+            if (updated)
+                return true;
+
+            prev = target.get();
+        }
+
+        return false;
+    }
+
+    private static StatsSnapshot computeStatsData(final LinkedList<CallSnapshot> csList) {
+        StatsSnapshot statsSnapshot = new StatsSnapshot();
+        synchronized (csList) {
+            double tps = 0;
+            double avgpt = 0;
+            long sum = 0;
+            if (!csList.isEmpty()) {
+                CallSnapshot first = csList.getFirst();
+                CallSnapshot last = csList.getLast();
+                sum = last.getValue() - first.getValue();
+                tps = (sum * 1000.0d) / (last.getTimestamp() - first.getTimestamp());
+
+                long timesDiff = last.getTimes() - first.getTimes();
+                if (timesDiff > 0) {
+                    avgpt = (sum * 1.0d) / timesDiff;
+                }
+
+            }
+
+            statsSnapshot.setSum(sum);
+            statsSnapshot.setTps(tps);
+            statsSnapshot.setAvgpt(avgpt);
+        }
+
+        return statsSnapshot;
+    }
+
+    public static long computNextMinutesTimeMillis() {
+        Calendar cal = Calendar.getInstance();
+        cal.setTimeInMillis(System.currentTimeMillis());
+        cal.add(Calendar.DAY_OF_MONTH, 0);
+        cal.add(Calendar.HOUR_OF_DAY, 0);
+        cal.add(Calendar.MINUTE, 1);
+        cal.set(Calendar.SECOND, 0);
+        cal.set(Calendar.MILLISECOND, 0);
+
+        return cal.getTimeInMillis();
+    }
+
+    public static long computNextHourTimeMillis() {
+        Calendar cal = Calendar.getInstance();
+        cal.setTimeInMillis(System.currentTimeMillis());
+        cal.add(Calendar.DAY_OF_MONTH, 0);
+        cal.add(Calendar.HOUR_OF_DAY, 1);
+        cal.set(Calendar.MINUTE, 0);
+        cal.set(Calendar.SECOND, 0);
+        cal.set(Calendar.MILLISECOND, 0);
+
+        return cal.getTimeInMillis();
+    }
+
+    public static long computNextMorningTimeMillis() {
+        Calendar cal = Calendar.getInstance();
+        cal.setTimeInMillis(System.currentTimeMillis());
+        cal.add(Calendar.DAY_OF_MONTH, 1);
+        cal.set(Calendar.HOUR_OF_DAY, 0);
+        cal.set(Calendar.MINUTE, 0);
+        cal.set(Calendar.SECOND, 0);
+        cal.set(Calendar.MILLISECOND, 0);
+
+        return cal.getTimeInMillis();
+    }
+
+    public void addValue(final int incValue, final int incTimes) {
+        this.value.addAndGet(incValue);
+        this.times.addAndGet(incTimes);
+        this.setValueIncDistributeRegion(incValue);
+        StatsItem.compareAndIncreaseOnly(this.valueMaxInMinutes, incValue);
+        StatsItem.compareAndIncreaseOnly(this.valueMaxIn10Minutes, incValue);
+        StatsItem.compareAndIncreaseOnly(this.valueMaxInHour, incValue);
+    }
+
+    public StatsSnapshot getStatsDataInMinute() {
+        return computeStatsData(this.csListMinute);
+    }
+
+    public StatsSnapshot getStatsDataInHour() {
+        return computeStatsData(this.csListHour);
+    }
+
+    public StatsSnapshot getStatsDataInDay() {
+        return computeStatsData(this.csListDay);
+    }
+
+    public void init() {
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    samplingInSeconds();
+                } catch (Throwable ignored) {
+                }
+            }
+        }, 0, 10, TimeUnit.SECONDS);
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+                                                              @Override
+                                                              public void run() {
+                                                                  try {
+                                                                      samplingInMinutes();
+                                                                  } catch (Throwable ignored) {
+                                                                  }
+
+                                                              }
+                                                          }, Math.abs(StatsItem.computNextMinutesTimeMillis() - System.currentTimeMillis()), //
+            1000 * 60 * 10, TimeUnit.MILLISECONDS);
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    samplingInHour();
+                } catch (Throwable ignored) {
+                }
+            }
+        }, 0, 1, TimeUnit.HOURS);
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+                                                              @Override
+                                                              public void run() {
+                                                                  try {
+                                                                      printAtMinutes();
+                                                                  } catch (Throwable ignored) {
+                                                                  }
+                                                              }
+                                                          }, Math.abs(StatsItem.computNextMinutesTimeMillis() - System.currentTimeMillis()), //
+            1000 * 60, TimeUnit.MILLISECONDS);
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+                                                              @Override
+                                                              public void run() {
+                                                                  try {
+                                                                      printAtHour();
+                                                                  } catch (Throwable ignored) {
+                                                                  }
+
+                                                              }
+                                                          }, Math.abs(StatsItem.computNextHourTimeMillis() - System.currentTimeMillis()), //
+            1000 * 60 * 60, TimeUnit.MILLISECONDS);
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+                                                              @Override
+                                                              public void run() {
+                                                                  try {
+                                                                      printAtDay();
+                                                                  } catch (Throwable ignored) {
+                                                                  }
+                                                              }
+                                                          }, Math.abs(StatsItem.computNextMorningTimeMillis() - System.currentTimeMillis()) - 2000, //
+            1000 * 60 * 60 * 24, TimeUnit.MILLISECONDS);
+    }
+
+    public void samplingInSeconds() {
+        synchronized (this.csListMinute) {
+            this.csListMinute.add(new CallSnapshot(System.currentTimeMillis(), this.times.get(), this.value.get()));
+            if (this.csListMinute.size() > 7) {
+                this.csListMinute.removeFirst();
+            }
+        }
+    }
+
+    public void samplingInMinutes() {
+        synchronized (this.csListHour) {
+            this.csListHour.add(new CallSnapshot(System.currentTimeMillis(), this.times.get(), this.value.get()));
+            if (this.csListHour.size() > 7) {
+                this.csListHour.removeFirst();
+            }
+        }
+
+        valueMaxIn10Minutes.set(0);
+    }
+
+    public void samplingInHour() {
+        synchronized (this.csListDay) {
+            this.csListDay.add(new CallSnapshot(System.currentTimeMillis(), this.times.get(), this.value.get()));
+            if (this.csListDay.size() > 25) {
+                this.csListDay.removeFirst();
+            }
+        }
+    }
+
+    public void printAtMinutes() {
+        StatsSnapshot ss = computeStatsData(this.csListMinute);
+        log.info(String
+            .format(
+                "[%s] [%s] Stats In One Minute, SUM: %d TPS: %.2f AVGPT: %.4f valueMaxInMinutes: %d valueMaxIn10Minutes: %d valueMaxInHour: %d valueIncDistributeRegion: %s",
+                this.statsName,
+                this.statsKey,
+                ss.getSum(),
+                ss.getTps(),
+                ss.getAvgpt(),
+                this.valueMaxInMinutes.get(),
+                this.valueMaxIn10Minutes.get(),
+                this.valueMaxInHour.get(),
+                Arrays.toString(valueRegion())
+            ));
+
+        valueMaxInMinutes.set(0);
+    }
+
+    public void printAtHour() {
+        StatsSnapshot ss = computeStatsData(this.csListHour);
+        log.info(String
+            .format(
+                "[%s] [%s] Stats In One Hour, SUM: %d TPS: %.2f AVGPT: %.4f valueMaxInMinutes: %d valueMaxIn10Minutes: %d valueMaxInHour: %d valueIncDistributeRegion: %s",
+                this.statsName,
+                this.statsKey,
+                ss.getSum(),
+                ss.getTps(),
+                ss.getAvgpt(),
+                this.valueMaxInMinutes.get(),
+                this.valueMaxIn10Minutes.get(),
+                this.valueMaxInHour.get(),
+                Arrays.toString(valueRegion())
+            ));
+
+        valueMaxInHour.set(0);
+    }
+
+    public void printAtDay() {
+        StatsSnapshot ss = computeStatsData(this.csListDay);
+        log.info(String.format(
+            "[%s] [%s] Stats In One Day, SUM: %d TPS: %.2f AVGPT: %.4f valueMaxInMinutes: %d valueMaxIn10Minutes: %d valueMaxInHour: %d valueIncDistributeRegion: %s",
+            this.statsName,
+            this.statsKey,
+            ss.getSum(),
+            ss.getTps(),
+            ss.getAvgpt(),
+            this.valueMaxInMinutes.get(),
+            this.valueMaxIn10Minutes.get(),
+            this.valueMaxInHour.get(),
+            Arrays.toString(valueRegion())
+        ));
+    }
+
+    long[] valueRegion() {
+        long[] vrs = new long[this.valueIncDistributeRegion.length];
+        for (int i = 0; i < this.valueIncDistributeRegion.length; i++) {
+            vrs[i] = this.valueIncDistributeRegion[i].get();
+        }
+        return vrs;
+    }
+
+    public AtomicLong getValue() {
+        return value;
+    }
+
+    public String getStatsName() {
+        return statsName;
+    }
+
+    public AtomicLong getTimes() {
+        return times;
+    }
+
+    public AtomicLong[] getValueDistributeRegion() {
+        return valueIncDistributeRegion;
+    }
+
+    public AtomicLong[] getValueIncDistributeRegion() {
+        return valueIncDistributeRegion;
+    }
+
+    private void setValueIncDistributeRegion(long value) {
+        // < 1ms
+        if (value <= 0) {
+            this.valueIncDistributeRegion[0].incrementAndGet();
+        }
+        // 1ms ~ 10ms
+        else if (value < 10) {
+            this.valueIncDistributeRegion[1].incrementAndGet();
+        }
+        // 10ms ~ 100ms
+        else if (value < 100) {
+            this.valueIncDistributeRegion[2].incrementAndGet();
+        }
+        // 100ms ~ 500ms
+        else if (value < 500) {
+            this.valueIncDistributeRegion[3].incrementAndGet();
+        }
+        // 500ms ~ 1s
+        else if (value < 1000) {
+            this.valueIncDistributeRegion[4].incrementAndGet();
+        }
+        // 1s ~ 3s
+        else if (value < 3000) {
+            this.valueIncDistributeRegion[5].incrementAndGet();
+        }
+        // 3s ~ 5s
+        else if (value < 5000) {
+            this.valueIncDistributeRegion[6].incrementAndGet();
+        }
+        // 5s ~ 10s
+        else if (value < 10000) {
+            this.valueIncDistributeRegion[7].incrementAndGet();
+        }
+        // 10s ~ 30s
+        else if (value < 30000) {
+            this.valueIncDistributeRegion[8].incrementAndGet();
+        }
+        // >= 30s
+        else {
+            this.valueIncDistributeRegion[9].incrementAndGet();
+        }
+    }
+
+    public AtomicLong getValueMaxInHour() {
+        return valueMaxInHour;
+    }
+
+    public AtomicLong getValueMaxInMinutes() {
+        return valueMaxInMinutes;
+    }
+
+    public AtomicLong getValueMaxIn10Minutes() {
+        return valueMaxIn10Minutes;
+    }
+
+    public static class StatsSnapshot {
+        private long sum;
+        private double tps;
+        private double avgpt;
+
+        public long getSum() {
+            return sum;
+        }
+
+        public void setSum(long sum) {
+            this.sum = sum;
+        }
+
+        public double getTps() {
+            return tps;
+        }
+
+        public void setTps(double tps) {
+            this.tps = tps;
+        }
+
+        public double getAvgpt() {
+            return avgpt;
+        }
+
+        public void setAvgpt(double avgpt) {
+            this.avgpt = avgpt;
+        }
+    }
+
+    class CallSnapshot {
+        private final long timestamp;
+        private final long times;
+        private final long value;
+
+        public CallSnapshot(long timestamp, long times, long value) {
+            super();
+            this.timestamp = timestamp;
+            this.times = times;
+            this.value = value;
+        }
+
+        public long getTimestamp() {
+            return timestamp;
+        }
+
+        public long getTimes() {
+            return times;
+        }
+
+        public long getValue() {
+            return value;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/StatsItemSet.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/StatsItemSet.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/StatsItemSet.java
new file mode 100644
index 0000000..79c1520
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/StatsItemSet.java
@@ -0,0 +1,193 @@
+/*
+ * 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.rocketmq.rpc.impl.metrics;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.slf4j.Logger;
+
+public class StatsItemSet {
+    private final ConcurrentHashMap<String/* key */, StatsItem> statsItemTable = new ConcurrentHashMap<String, StatsItem>(128);
+    private final String statsName;
+    private final ScheduledExecutorService scheduledExecutorService;
+    private final Logger log;
+
+    public StatsItemSet(String statsName, ScheduledExecutorService scheduledExecutorService, Logger log) {
+        this.statsName = statsName;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.log = log;
+    }
+
+    public void init() {
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    samplingInSeconds();
+                } catch (Throwable ignored) {
+                }
+            }
+        }, 0, 10, TimeUnit.SECONDS);
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    samplingInMinutes();
+                } catch (Throwable ignored) {
+                }
+            }
+        }, 0, 10, TimeUnit.MINUTES);
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    samplingInHour();
+                } catch (Throwable ignored) {
+                }
+            }
+        }, 0, 1, TimeUnit.HOURS);
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+                                                              @Override
+                                                              public void run() {
+                                                                  try {
+                                                                      printAtMinutes();
+                                                                  } catch (Throwable ignored) {
+                                                                  }
+                                                              }
+                                                          }, Math.abs(StatsItem.computNextMinutesTimeMillis() - System.currentTimeMillis()), //
+            1000 * 60, TimeUnit.MILLISECONDS);
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+                                                              @Override
+                                                              public void run() {
+                                                                  try {
+                                                                      printAtHour();
+                                                                  } catch (Throwable ignored) {
+                                                                  }
+                                                              }
+                                                          }, Math.abs(StatsItem.computNextHourTimeMillis() - System.currentTimeMillis()), //
+            1000 * 60 * 60, TimeUnit.MILLISECONDS);
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+                                                              @Override
+                                                              public void run() {
+                                                                  try {
+                                                                      printAtDay();
+                                                                  } catch (Throwable ignored) {
+                                                                  }
+                                                              }
+                                                          }, Math.abs(StatsItem.computNextMorningTimeMillis() - System.currentTimeMillis()), //
+            1000 * 60 * 60 * 24, TimeUnit.MILLISECONDS);
+    }
+
+    private void samplingInSeconds() {
+        for (final Map.Entry<String, StatsItem> next : this.statsItemTable.entrySet()) {
+            next.getValue().samplingInSeconds();
+        }
+    }
+
+    private void samplingInMinutes() {
+        for (final Map.Entry<String, StatsItem> next : this.statsItemTable.entrySet()) {
+            next.getValue().samplingInMinutes();
+        }
+    }
+
+    private void samplingInHour() {
+        for (final Map.Entry<String, StatsItem> next : this.statsItemTable.entrySet()) {
+            next.getValue().samplingInHour();
+        }
+    }
+
+    private void printAtMinutes() {
+        for (final Map.Entry<String, StatsItem> next : this.statsItemTable.entrySet()) {
+            next.getValue().printAtMinutes();
+        }
+    }
+
+    private void printAtHour() {
+        for (final Map.Entry<String, StatsItem> next : this.statsItemTable.entrySet()) {
+            next.getValue().printAtHour();
+        }
+    }
+
+    private void printAtDay() {
+        for (final Map.Entry<String, StatsItem> next : this.statsItemTable.entrySet()) {
+            next.getValue().printAtDay();
+        }
+    }
+
+    void addValue(final String statsKey, final int incValue, final int incTimes) {
+        this.getAndCreateStatsItem(statsKey).addValue(incValue, incTimes);
+    }
+
+    private StatsItem getAndCreateStatsItem(final String statsKey) {
+        StatsItem statsItem = this.statsItemTable.get(statsKey);
+        if (null == statsItem) {
+            statsItem = new StatsItem(this.statsName, statsKey, this.scheduledExecutorService, this.log);
+            StatsItem prev = this.statsItemTable.put(statsKey, statsItem);
+            if (null == prev) {
+                // statsItem.init();
+            }
+        }
+
+        return statsItem;
+    }
+
+    public StatsItem.StatsSnapshot getStatsDataInMinute(final String statsKey) {
+        StatsItem statsItem = this.statsItemTable.get(statsKey);
+        if (null != statsItem) {
+            return statsItem.getStatsDataInMinute();
+        }
+        return new StatsItem.StatsSnapshot();
+    }
+
+    public StatsItem.StatsSnapshot getStatsDataInHour(final String statsKey) {
+        StatsItem statsItem = this.statsItemTable.get(statsKey);
+        if (null != statsItem) {
+            return statsItem.getStatsDataInHour();
+        }
+
+        return new StatsItem.StatsSnapshot();
+    }
+
+    public StatsItem.StatsSnapshot getStatsDataInDay(final String statsKey) {
+        StatsItem statsItem = this.statsItemTable.get(statsKey);
+        if (null != statsItem) {
+            return statsItem.getStatsDataInDay();
+        }
+        return new StatsItem.StatsSnapshot();
+    }
+
+    public StatsItem getStatsItem(final String statsKey) {
+        return this.statsItemTable.get(statsKey);
+    }
+
+    ConcurrentHashMap<String, StatsItem> getStatsItemTable() {
+        return statsItemTable;
+    }
+
+    public String getStatsName() {
+        return statsName;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/ThreadStats.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/ThreadStats.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/ThreadStats.java
new file mode 100644
index 0000000..540779e
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/ThreadStats.java
@@ -0,0 +1,55 @@
+/*
+ * 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.rocketmq.rpc.impl.metrics;
+
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class ThreadStats {
+    private final ConcurrentHashMap<Threading, TimestampRegion> statsTable = new ConcurrentHashMap<Threading, TimestampRegion>(64);
+
+    public void beginInvoke(final long beginTimestamp) {
+        Threading th = new Threading(Thread.currentThread().getName(), Thread.currentThread().getId());
+
+        TimestampRegion tr = this.statsTable.get(th);
+        if (null == tr) {
+            tr = new TimestampRegion();
+            this.statsTable.put(th, tr);
+        }
+
+        tr.setBeginTimestamp(beginTimestamp);
+        tr.setEndTimestamp(-1);
+    }
+
+    public void endInvoke(final long endTimestamp) {
+        Threading th = new Threading(Thread.currentThread().getName(), Thread.currentThread().getId());
+        TimestampRegion tr = this.statsTable.get(th);
+        tr.setEndTimestamp(endTimestamp);
+    }
+
+    public TreeMap<Threading, TimestampRegion> cloneStatsTable() {
+        TreeMap<Threading, TimestampRegion> result = new TreeMap<Threading, TimestampRegion>();
+
+        for (final Map.Entry<Threading, TimestampRegion> next : this.statsTable.entrySet()) {
+            result.put(next.getKey(), next.getValue());
+        }
+        return result;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/Threading.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/Threading.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/Threading.java
new file mode 100644
index 0000000..e2c19b5
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/Threading.java
@@ -0,0 +1,87 @@
+/*
+ * 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.rocketmq.rpc.impl.metrics;
+
+public class Threading implements Comparable {
+    private String name;
+    private long id;
+
+    public Threading() {
+    }
+
+    public Threading(String name, long id) {
+        this.name = name;
+        this.id = id;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public long getId() {
+        return id;
+    }
+
+    public void setId(long id) {
+        this.id = id;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = name != null ? name.hashCode() : 0;
+        result = 31 * result + (int) (id ^ (id >>> 32));
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        Threading threading = (Threading) o;
+
+        return id == threading.id && !(name != null ? !name.equals(threading.name) : threading.name != null);
+
+    }
+
+    @Override
+    public String toString() {
+        return "Threading{" +
+            "name='" + name + '\'' +
+            ", id=" + id +
+            '}';
+    }
+
+    @Override
+    public int compareTo(Object o) {
+        Threading t = (Threading) o;
+        int ret = t.name.compareTo(this.name);
+        if (ret == 0) {
+            return Long.valueOf(t.id).compareTo(this.id);
+        }
+
+        return ret;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/TimestampRegion.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/TimestampRegion.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/TimestampRegion.java
new file mode 100644
index 0000000..cf8042b
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/TimestampRegion.java
@@ -0,0 +1,65 @@
+/*
+ * 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.rocketmq.rpc.impl.metrics;
+
+public class TimestampRegion {
+    private volatile long beginTimestamp = -1;
+    private volatile long endTimestamp = -1;
+
+    public long getBeginTimestamp() {
+        return beginTimestamp;
+    }
+
+    public void setBeginTimestamp(long beginTimestamp) {
+        this.beginTimestamp = beginTimestamp;
+    }
+
+    public long getEndTimestamp() {
+        return endTimestamp;
+    }
+
+    public void setEndTimestamp(long endTimestamp) {
+        this.endTimestamp = endTimestamp;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = (int) (beginTimestamp ^ (beginTimestamp >>> 32));
+        result = 31 * result + (int) (endTimestamp ^ (endTimestamp >>> 32));
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        TimestampRegion that = (TimestampRegion) o;
+        return beginTimestamp == that.beginTimestamp && endTimestamp == that.endTimestamp;
+    }
+
+    @Override
+    public String toString() {
+        return "TimestampRegion{" +
+            "beginTimestamp=" + beginTimestamp +
+            ", endTimestamp=" + endTimestamp +
+            '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/UtilAll.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/UtilAll.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/UtilAll.java
new file mode 100644
index 0000000..376ccda
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/metrics/UtilAll.java
@@ -0,0 +1,84 @@
+/*
+ * 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.rocketmq.rpc.impl.metrics;
+
+import java.util.Map;
+
+public class UtilAll {
+
+    public static String jstack() {
+        return jstack(Thread.getAllStackTraces());
+    }
+
+    private static String jstack(Map<Thread, StackTraceElement[]> map) {
+        StringBuilder result = new StringBuilder();
+        try {
+            for (final Map.Entry<Thread, StackTraceElement[]> entry : map.entrySet()) {
+                StackTraceElement[] elements = entry.getValue();
+                Thread thread = entry.getKey();
+                if (elements != null && elements.length > 0) {
+                    String threadName = entry.getKey().getName();
+                    result.append(String.format("%-40s TID: %d STATE: %s\n", threadName, thread.getId(), thread.getState()));
+                    for (StackTraceElement el : elements) {
+                        result.append(String.format("%-40s %s\n", threadName, el.toString()));
+                    }
+                    result.append("\n");
+                }
+            }
+        } catch (Throwable ignored) {
+        }
+
+        return result.toString();
+    }
+
+    public static String jstack(final String threadName, final long threadId) {
+        Map<Thread, StackTraceElement[]> map = Thread.getAllStackTraces();
+        StringBuilder result = new StringBuilder();
+        try {
+            for (final Map.Entry<Thread, StackTraceElement[]> entry : map.entrySet()) {
+                StackTraceElement[] elements = entry.getValue();
+                Thread thread = entry.getKey();
+                if (elements != null && elements.length > 0) {
+                    if (threadName.equals(entry.getKey().getName()) && threadId == entry.getKey().getId()) {
+                        result.append(String.format("%-40s TID: %d STATE: %s\n", threadName, thread.getId(), thread.getState()));
+                        for (StackTraceElement el : elements) {
+                            result.append(String.format("%-40s %s\n", threadName, el.toString()));
+                        }
+                    }
+                }
+            }
+        } catch (Throwable ignored) {
+        }
+        return result.toString();
+    }
+
+    public static ExecuteResult callShellCommand(final String shellString) {
+        Process process = null;
+        try {
+            String[] cmdArray = shellString.split(" ");
+            process = Runtime.getRuntime().exec(cmdArray);
+            process.waitFor();
+        } catch (Throwable ignored) {
+        } finally {
+            if (null != process)
+                process.destroy();
+        }
+
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/processor/RpcRequestProcessor.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/processor/RpcRequestProcessor.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/processor/RpcRequestProcessor.java
new file mode 100644
index 0000000..e7330b8
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/processor/RpcRequestProcessor.java
@@ -0,0 +1,213 @@
+/*
+ * 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.rocketmq.rpc.impl.processor;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Type;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import org.apache.rocketmq.remoting.api.RequestProcessor;
+import org.apache.rocketmq.remoting.api.channel.RemotingChannel;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.remoting.api.serializable.Serializer;
+import org.apache.rocketmq.rpc.annotation.RemoteService;
+import org.apache.rocketmq.rpc.impl.command.ResponseCode;
+import org.apache.rocketmq.rpc.impl.context.RpcProviderContext;
+import org.apache.rocketmq.rpc.impl.exception.ServiceExceptionInvokeMessage;
+import org.apache.rocketmq.rpc.impl.metrics.ServiceStats;
+import org.apache.rocketmq.rpc.impl.service.RpcEntry;
+import org.apache.rocketmq.rpc.impl.service.RpcInstanceAbstract;
+import org.apache.rocketmq.rpc.impl.service.RpcServiceCallBody;
+import org.apache.rocketmq.rpc.internal.ServiceUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RpcRequestProcessor implements RequestProcessor {
+    private static final Logger log = LoggerFactory.getLogger(RpcRequestProcessor.class);
+    private Map<String, RpcEntry> serviceTable = new ConcurrentHashMap<>(64);
+    private Map<String, ExecutorService> executorTable = new ConcurrentHashMap<>(64);
+    private ThreadLocal<RpcProviderContext> threadLocalProviderContext;
+    private RpcInstanceAbstract rpcInstanceAbstract;
+    private ServiceStats serviceStats;
+
+    public RpcRequestProcessor(ThreadLocal<RpcProviderContext> threadLocalProviderContext,
+        final RpcInstanceAbstract rpcInstanceAbstract, ServiceStats stats) {
+        this.threadLocalProviderContext = threadLocalProviderContext;
+        this.rpcInstanceAbstract = rpcInstanceAbstract;
+        this.serviceStats = stats;
+    }
+
+    public Set<String> putNewService(final Object obj) {
+        return putNewService(obj, null);
+    }
+
+    public Set<String> putNewService(final Object obj, final ExecutorService executorService) {
+        Class<?>[] interfaces = obj.getClass().getInterfaces();
+        for (Class<?> itf : interfaces) {
+            RemoteService serviceExport = itf.getAnnotation(RemoteService.class);
+            if (null == serviceExport) {
+                log.warn("Service:{} is not remark annotation", itf.getName());
+                continue;
+            }
+
+            Method[] methods = itf.getMethods();
+            for (Method method : methods) {
+                if (!ServiceUtil.testServiceExportMethod(method)) {
+                    log.error("The method: [{}] not matched RPC standard", method.toGenericString());
+                    continue;
+                }
+
+                String requestCode = ServiceUtil.toRequestCode(serviceExport, method);
+                RpcEntry se = new RpcEntry();
+                se.setServiceExport(serviceExport);
+                se.setObject(obj);
+                se.setMethod(method);
+                this.serviceTable.put(requestCode, se);
+                if (executorService != null) {
+                    this.executorTable.put(requestCode, executorService);
+                }
+            }
+        }
+        return this.serviceTable.keySet();
+    }
+
+    @Override
+    public RemotingCommand processRequest(RemotingChannel channel, RemotingCommand request) {
+        RpcProviderContext rpcProviderContext = new RpcProviderContext();
+        rpcProviderContext.setRemotingChannel(channel);
+        rpcProviderContext.setRemotingRequest(request);
+        rpcProviderContext.setRemotingResponse(rpcInstanceAbstract.remotingService().commandFactory().createResponse(request));
+        rpcProviderContext.setReturnResponse(true);
+        threadLocalProviderContext.set(rpcProviderContext);
+        this.processRequest0(rpcProviderContext, request);
+        threadLocalProviderContext.remove();
+
+        if (rpcProviderContext.isReturnResponse()) {
+            return rpcProviderContext.getRemotingResponse();
+        }
+        return null;
+    }
+
+    private Object[] buildParameter(final RpcProviderContext context, final RemotingCommand request,
+        RpcServiceCallBody serviceCallBody, Type[] parameterTypes) {
+        Object[] parameters = new Object[parameterTypes.length];
+        try {
+            int index = 0;
+            Serializer serialization = this.rpcInstanceAbstract.remotingService()
+                .serializerFactory().get(request.serializerType());
+            for (Type parameterType : parameterTypes) {
+                parameters[index] = serialization.decode(serviceCallBody.getParameter(index), parameterType);
+                index++;
+            }
+        } catch (Exception e) {
+            ServiceExceptionInvokeMessage serviceExceptionInvokeMessage = new ServiceExceptionInvokeMessage();
+            serviceExceptionInvokeMessage.setClassFullName(e.getClass().getName());
+            serviceExceptionInvokeMessage.setErrorMessage(serializeException(request.serializerType(), e));
+            context.getRemotingResponse().opCode(ResponseCode.PARAMETER_ERROR);
+            context.getRemotingResponse().parameter(serviceExceptionInvokeMessage);
+        }
+        return parameters;
+    }
+
+    private String serializeException(byte serializeType, Exception exception) {
+        Serializer serialization = rpcInstanceAbstract.remotingService().serializerFactory().get(serializeType);
+        return serialization.encode(exception).toString();
+    }
+
+    private void dealWithException(final RpcProviderContext context, RemotingCommand request, Exception exception) {
+        if (exception instanceof InvocationTargetException) {
+            context.getRemotingResponse().opCode(ResponseCode.USER_SERVICE_EXCEPTION);
+        } else if (exception instanceof IllegalArgumentException) {
+            context.getRemotingResponse().opCode(ResponseCode.PARAMETER_ERROR);
+        } else if (exception instanceof IllegalAccessException) {
+            context.getRemotingResponse().opCode(ResponseCode.ILLEGAL_ACCESS);
+        } else if (exception instanceof NullPointerException) {
+            context.getRemotingResponse().opCode(ResponseCode.NULL_POINTER);
+        } else {
+            context.getRemotingResponse().opCode(ResponseCode.USER_SERVICE_EXCEPTION);
+        }
+        String remarkMsg, exceptionMsg, exceptionClassname;
+        ServiceExceptionInvokeMessage serviceExceptionInvokeMessage = new ServiceExceptionInvokeMessage();
+        if (exception instanceof InvocationTargetException) {
+            exceptionMsg = ((InvocationTargetException) exception).getTargetException().getMessage();
+            remarkMsg = exceptionMsg == null ? "" : exceptionMsg;
+            exceptionClassname = ((InvocationTargetException) exception).getTargetException().getClass().getName();
+        } else {
+            exceptionMsg = exception.getMessage();
+            remarkMsg = exceptionMsg == null ? "" : exceptionMsg;
+            exceptionClassname = exception.getClass().getName();
+        }
+        serviceExceptionInvokeMessage.setClassFullName(exceptionClassname);
+        serviceExceptionInvokeMessage.setErrorMessage(remarkMsg);
+        if (exception.getCause() != null)
+            serviceExceptionInvokeMessage.setThrowable(exception.getCause());
+        Object[] args = new Object[] {serviceExceptionInvokeMessage};
+        Serializer serializer = ServiceUtil.selectSerializer(args, request.serializerType());
+        if (serializer != null)
+            context.getRemotingResponse().serializerType(serializer.type());
+        context.getRemotingResponse().parameter(serviceExceptionInvokeMessage);
+    }
+
+    private void processRequest0(final RpcProviderContext context, final RemotingCommand request) {
+        final RpcServiceCallBody serviceCallBody =
+            request.parameter(this.rpcInstanceAbstract.remotingService().serializerFactory(), RpcServiceCallBody.class);
+        final RpcEntry entry = this.serviceTable.get(serviceCallBody.getServiceId());
+        if (entry != null) {
+            Runnable runnable = new Runnable() {
+                @Override
+                public void run() {
+                    Type[] parameterTypes = entry.getMethod().getGenericParameterTypes();
+                    Object result = null;
+                    Exception exception = null;
+                    long startTime = System.currentTimeMillis();
+                    try {
+                        if (parameterTypes.length == 0) {
+                            result = entry.getMethod().invoke(entry.getObject());
+                        } else {
+                            Object[] parameters = buildParameter(context, request, serviceCallBody, parameterTypes);
+                            result = entry.getMethod().invoke(entry.getObject(), parameters);
+                        }
+                        serviceStats.addProviderOKQPSValue(serviceCallBody.getServiceId(), 1, 1);
+                        serviceStats.addProviderRTValue(serviceCallBody.getServiceId(), (int) (System.currentTimeMillis() - startTime), 1);
+                    } catch (Exception e) {
+                        exception = e;
+                    }
+
+                    if (exception != null)
+                        dealWithException(context, request, exception);
+                    else if (!entry.getMethod().getReturnType().equals(Void.class)) {
+                        Object[] args = new Object[] {result};
+                        Serializer serializer = ServiceUtil.selectSerializer(args, request.serializerType());
+                        if (serializer != null)
+                            context.getRemotingResponse().serializerType(serializer.type());
+                        context.getRemotingResponse().parameter(result);
+                    }
+                }
+            };
+            ExecutorService executorService = this.executorTable.get(serviceCallBody.getServiceId());
+            if (executorService != null) {
+                executorService.submit(runnable);
+            } else {
+                runnable.run();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/promise/DefaultPromise.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/promise/DefaultPromise.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/promise/DefaultPromise.java
new file mode 100644
index 0000000..0be1014
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/promise/DefaultPromise.java
@@ -0,0 +1,228 @@
+/*
+ * 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.rocketmq.rpc.impl.promise;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.rocketmq.rpc.api.Promise;
+import org.apache.rocketmq.rpc.api.PromiseListener;
+import org.apache.rocketmq.rpc.impl.exception.ServiceExceptionManager;
+import org.apache.rocketmq.rpc.internal.RpcErrorMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DefaultPromise<V> implements Promise<V> {
+    private static final Logger LOG = LoggerFactory.getLogger(DefaultPromise.class);
+    private final Object lock = new Object();
+    private volatile FutureState state = FutureState.DOING;
+    private V result = null;
+    private long timeout;
+    private long createTime;
+    private Throwable exception = null;
+    private List<PromiseListener<V>> promiseListenerList;
+
+    public DefaultPromise() {
+        createTime = System.currentTimeMillis();
+        promiseListenerList = new ArrayList<>();
+        timeout = 5000;
+    }
+
+    @Override
+    public boolean cancel(final boolean mayInterruptIfRunning) {
+        return false;
+    }
+
+    @Override
+    public boolean isCancelled() {
+        return state.isCancelledState();
+    }
+
+    @Override
+    public boolean isDone() {
+        return state.isDoneState();
+    }
+
+    @Override
+    public V get() {
+        return result;
+    }
+
+    @Override
+    public V get(final long timeout) {
+        synchronized (lock) {
+            if (!isDoing()) {
+                return getValueOrThrowable();
+            }
+
+            if (timeout <= 0) {
+                try {
+                    lock.wait();
+                } catch (Exception e) {
+                    cancel(e);
+                }
+                return getValueOrThrowable();
+            } else {
+                long waitTime = timeout - (System.currentTimeMillis() - createTime);
+                if (waitTime > 0) {
+                    for (; ; ) {
+                        try {
+                            lock.wait(waitTime);
+                        } catch (InterruptedException e) {
+                            LOG.error("promise get value interrupted,excepiton:{}", e.getMessage());
+                        }
+
+                        if (!isDoing()) {
+                            break;
+                        } else {
+                            waitTime = timeout - (System.currentTimeMillis() - createTime);
+                            if (waitTime <= 0) {
+                                break;
+                            }
+                        }
+                    }
+                }
+
+                if (isDoing()) {
+                    timeoutSoCancel();
+                }
+            }
+            return getValueOrThrowable();
+        }
+    }
+
+    @Override
+    public boolean set(final V value) {
+        if (value == null)
+            return false;
+        this.result = value;
+        return done();
+    }
+
+    @Override
+    public boolean setFailure(final Throwable cause) {
+        if (cause == null)
+            return false;
+        this.exception = cause;
+        return done();
+    }
+
+    @Override
+    public void addListener(final PromiseListener<V> listener) {
+        if (listener == null) {
+            throw new NullPointerException("FutureListener is null");
+        }
+
+        boolean notifyNow = false;
+        synchronized (lock) {
+            if (!isDoing()) {
+                notifyNow = true;
+            } else {
+                if (promiseListenerList == null) {
+                    promiseListenerList = new ArrayList<>();
+                }
+                promiseListenerList.add(listener);
+            }
+        }
+
+        if (notifyNow) {
+            notifyListener(listener);
+        }
+    }
+
+    @Override
+    public Throwable getThrowable() {
+        return exception;
+    }
+
+    private void notifyListeners() {
+        if (promiseListenerList != null) {
+            for (PromiseListener<V> listener : promiseListenerList) {
+                notifyListener(listener);
+            }
+        }
+    }
+
+    private boolean isSuccess() {
+        return isDone() && (exception == null);
+    }
+
+    private void timeoutSoCancel() {
+        synchronized (lock) {
+            if (!isDoing()) {
+                return;
+            }
+            state = FutureState.CANCELLED;
+            exception = new RuntimeException("Get request result is timeout or interrupted");
+            lock.notifyAll();
+        }
+        notifyListeners();
+    }
+
+    private V getValueOrThrowable() {
+        if (exception != null) {
+            Throwable e = exception.getCause() != null ? exception.getCause() : exception;
+            throw ServiceExceptionManager.TranslateException(RpcErrorMapper.RpcErrorLatitude.LOCAL.getCode(), e);
+        }
+        notifyListeners();
+        return result;
+    }
+
+    private boolean isDoing() {
+        return state.isDoingState();
+    }
+
+    private boolean done() {
+        synchronized (lock) {
+            if (!isDoing()) {
+                return false;
+            }
+
+            state = FutureState.DONE;
+            lock.notifyAll();
+        }
+
+        notifyListeners();
+        return true;
+    }
+
+    private void notifyListener(final PromiseListener<V> listener) {
+        try {
+            if (exception != null)
+                listener.operationFailed(this);
+            else
+                listener.operationCompleted(this);
+        } catch (Throwable t) {
+            LOG.error("notifyListener {} Error:{}", listener.getClass().getSimpleName(), t);
+        }
+    }
+
+    private boolean cancel(Exception e) {
+        synchronized (lock) {
+            if (!isDoing()) {
+                return false;
+            }
+
+            state = FutureState.CANCELLED;
+            exception = e;
+            lock.notifyAll();
+        }
+
+        notifyListeners();
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/promise/FutureState.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/promise/FutureState.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/promise/FutureState.java
new file mode 100644
index 0000000..aebc5e3
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/promise/FutureState.java
@@ -0,0 +1,51 @@
+/*
+ * 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.rocketmq.rpc.impl.promise;
+
+public enum FutureState {
+    /**
+     * the task is doing
+     **/
+    DOING(0),
+    /**
+     * the task is done
+     **/
+    DONE(1),
+    /**
+     * ths task is cancelled
+     **/
+    CANCELLED(2);
+
+    public final int value;
+
+    private FutureState(int value) {
+        this.value = value;
+    }
+
+    public boolean isCancelledState() {
+        return this == CANCELLED;
+    }
+
+    public boolean isDoneState() {
+        return this == DONE;
+    }
+
+    public boolean isDoingState() {
+        return this == DOING;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/server/AdvancedServerImpl.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/server/AdvancedServerImpl.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/server/AdvancedServerImpl.java
new file mode 100644
index 0000000..ba0180c
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/server/AdvancedServerImpl.java
@@ -0,0 +1,57 @@
+/*
+ * 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.rocketmq.rpc.impl.server;
+
+import org.apache.rocketmq.remoting.api.channel.RemotingChannel;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.rpc.annotation.MethodType;
+import org.apache.rocketmq.rpc.api.AdvancedServer;
+import org.apache.rocketmq.rpc.api.Promise;
+import org.apache.rocketmq.rpc.impl.service.RpcJdkProxy;
+import org.apache.rocketmq.rpc.impl.service.RpcProxyFactory;
+
+public class AdvancedServerImpl implements AdvancedServer {
+    private final SimpleServerImpl simpleServer;
+
+    public AdvancedServerImpl(final SimpleServerImpl simpleServer) {
+        this.simpleServer = simpleServer;
+    }
+
+    @Override
+    public <T> T callSync(final RemotingChannel channel, final String serviceCode, final String version,
+        final Object[] parameter, final Class<T> responseType) throws Exception {
+        RemotingCommand request = simpleServer.createRemoteRequest(serviceCode, version, parameter);
+        RpcJdkProxy rpcJdkProxy = RpcProxyFactory.createServiceProxy(null, simpleServer, simpleServer.getRemotingServer(), simpleServer.getRpcCommonConfig(), channel);
+        return (T) simpleServer.invokeRemoteMethod(rpcJdkProxy, serviceCode, request, responseType, MethodType.SYNC);
+    }
+
+    @Override
+    public <T> Promise<T> callAsync(final RemotingChannel channel, final String serviceCode, final String version,
+        final Object[] parameter, final Class<T> responseType) throws Exception {
+        RemotingCommand request = simpleServer.createRemoteRequest(serviceCode, version, parameter);
+        RpcJdkProxy rpcJdkProxy = RpcProxyFactory.createServiceProxy(null, simpleServer, simpleServer.getRemotingServer(), simpleServer.getRpcCommonConfig(), channel);
+        return (Promise<T>) simpleServer.invokeRemoteMethod(rpcJdkProxy, serviceCode, request, responseType, MethodType.ASYNC);
+    }
+
+    @Override
+    public void callOneway(final RemotingChannel channel, final String serviceCode, final String version,
+        final Object[] parameter) throws Exception {
+        RemotingCommand request = simpleServer.createRemoteRequest(serviceCode, version, parameter);
+        RpcJdkProxy rpcJdkProxy = RpcProxyFactory.createServiceProxy(null, simpleServer, simpleServer.getRemotingServer(), simpleServer.getRpcCommonConfig(), channel);
+        simpleServer.invokeRemoteMethod(rpcJdkProxy, serviceCode, request, Void.TYPE, MethodType.ASYNC);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/server/SimpleServerImpl.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/server/SimpleServerImpl.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/server/SimpleServerImpl.java
new file mode 100644
index 0000000..e076cbe
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/server/SimpleServerImpl.java
@@ -0,0 +1,104 @@
+/*
+ * 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.rocketmq.rpc.impl.server;
+
+import java.util.Properties;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.rocketmq.remoting.api.RemotingServer;
+import org.apache.rocketmq.remoting.api.RemotingService;
+import org.apache.rocketmq.remoting.api.channel.RemotingChannel;
+import org.apache.rocketmq.remoting.external.ThreadUtils;
+import org.apache.rocketmq.remoting.impl.netty.RemotingBootstrapFactory;
+import org.apache.rocketmq.rpc.api.AdvancedServer;
+import org.apache.rocketmq.rpc.api.SimpleServer;
+import org.apache.rocketmq.rpc.impl.config.RpcCommonConfig;
+import org.apache.rocketmq.rpc.impl.service.RpcInstanceAbstract;
+import org.apache.rocketmq.rpc.impl.service.RpcProxyFactory;
+
+public class SimpleServerImpl extends RpcInstanceAbstract implements SimpleServer {
+    private RemotingServer remotingServer;
+    private ExecutorService callServiceThreadPool;
+    private RpcCommonConfig rpcCommonConfig;
+
+    public SimpleServerImpl(final RpcCommonConfig remotingConfig) {
+        this(remotingConfig, RemotingBootstrapFactory.createRemotingServer(remotingConfig));
+        this.rpcCommonConfig = remotingConfig;
+        this.callServiceThreadPool = ThreadUtils.newThreadPoolExecutor(rpcCommonConfig.getClientAsyncCallbackExecutorThreads(),
+            rpcCommonConfig.getClientAsyncCallbackExecutorThreads(), rpcCommonConfig.getServiceThreadKeepAliveTime(),
+            TimeUnit.MILLISECONDS, new ArrayBlockingQueue<Runnable>(remotingConfig.getServiceThreadBlockQueueSize()),
+            "serverCallServiceThread", true);
+    }
+
+    public SimpleServerImpl(final RpcCommonConfig remotingConfig, final RemotingServer remotingServer) {
+        super(remotingConfig);
+        this.remotingServer = remotingServer;
+    }
+
+    @Override
+    public RemotingService remotingService() {
+        return this.remotingServer;
+    }
+
+    @Override
+    public void registerServiceListener() {
+
+    }
+
+    @Override
+    public <T> T bind(final Class<T> service, final RemotingChannel channel, final Properties properties) {
+        return this.narrow0(service, RpcProxyFactory.createServiceProxy(service, this, remotingServer, rpcCommonConfig, channel));
+    }
+
+    @Override
+    public AdvancedServer advancedServer() {
+        return new AdvancedServerImpl(this);
+    }
+
+    @Override
+    public void publish(final Object service) {
+        this.publishService0(service);
+    }
+
+    @Override
+    public void publish(final Object service, final ExecutorService executorService) {
+        this.publishService0(service, executorService);
+    }
+
+    @Override
+    public void start() {
+        super.start();
+        this.remotingServer.start();
+    }
+
+    @Override
+    public void stop() {
+        super.stop();
+        ThreadUtils.shutdownGracefully(this.callServiceThreadPool, 3000, TimeUnit.MILLISECONDS);
+        this.remotingServer.stop();
+    }
+
+    public RemotingServer getRemotingServer() {
+        return remotingServer;
+    }
+
+    public void setRemotingServer(final RemotingServer remotingServer) {
+        this.remotingServer = remotingServer;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcConnectionListener.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcConnectionListener.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcConnectionListener.java
new file mode 100644
index 0000000..c0a0e8c
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcConnectionListener.java
@@ -0,0 +1,49 @@
+/*
+ * 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.rocketmq.rpc.impl.service;
+
+import org.apache.rocketmq.remoting.api.channel.ChannelEventListener;
+import org.apache.rocketmq.remoting.api.channel.RemotingChannel;
+
+public class RpcConnectionListener implements ChannelEventListener {
+    private RpcInstanceAbstract rpcInstanceAbstract;
+
+    public RpcConnectionListener(RpcInstanceAbstract rpcInstanceAbstract) {
+        this.rpcInstanceAbstract = rpcInstanceAbstract;
+    }
+
+    @Override
+    public void onChannelConnect(final RemotingChannel remotingChannel) {
+        this.rpcInstanceAbstract.registerServiceListener();
+    }
+
+    @Override
+    public void onChannelClose(final RemotingChannel remotingChannel) {
+
+    }
+
+    @Override
+    public void onChannelException(final RemotingChannel remotingChannel) {
+
+    }
+
+    @Override
+    public void onChannelIdle(final RemotingChannel remotingChannel) {
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcEntry.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcEntry.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcEntry.java
new file mode 100644
index 0000000..9bfd53a
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcEntry.java
@@ -0,0 +1,51 @@
+/*
+ * 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.rocketmq.rpc.impl.service;
+
+import java.lang.reflect.Method;
+import org.apache.rocketmq.rpc.annotation.RemoteService;
+
+public class RpcEntry {
+    private RemoteService serviceExport;
+    private Method method;
+    private Object object;
+
+    public Method getMethod() {
+        return method;
+    }
+
+    public void setMethod(Method method) {
+        this.method = method;
+    }
+
+    public Object getObject() {
+        return object;
+    }
+
+    public void setObject(Object object) {
+        this.object = object;
+    }
+
+    public RemoteService getServiceExport() {
+        return serviceExport;
+    }
+
+    public void setServiceExport(RemoteService serviceExport) {
+        this.serviceExport = serviceExport;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcInstanceAbstract.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcInstanceAbstract.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcInstanceAbstract.java
new file mode 100644
index 0000000..2b1288c
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcInstanceAbstract.java
@@ -0,0 +1,88 @@
+/*
+ * 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.rocketmq.rpc.impl.service;
+
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.apache.rocketmq.remoting.external.ThreadUtils;
+import org.apache.rocketmq.rpc.impl.command.RpcRequestCode;
+import org.apache.rocketmq.rpc.impl.config.RpcCommonConfig;
+import org.apache.rocketmq.rpc.impl.context.RpcProviderContext;
+import org.apache.rocketmq.rpc.impl.metrics.DefaultServiceAPIImpl;
+import org.apache.rocketmq.rpc.impl.metrics.ThreadStats;
+import org.apache.rocketmq.rpc.impl.processor.RpcRequestProcessor;
+
+import static org.apache.rocketmq.remoting.external.ThreadUtils.newThreadFactory;
+
+public abstract class RpcInstanceAbstract extends RpcProxyCommon {
+    protected final RpcRequestProcessor rpcRequestProcessor;
+    protected final ThreadLocal<RpcProviderContext> threadLocalProviderContext = new ThreadLocal<RpcProviderContext>();
+    protected final RpcCommonConfig rpcCommonConfig;
+    protected ThreadStats threadStats;
+    private DefaultServiceAPIImpl defaultServiceAPI;
+    private ThreadPoolExecutor invokeServiceThreadPool;
+
+    public RpcInstanceAbstract(RpcCommonConfig rpcCommonConfig) {
+        super(rpcCommonConfig);
+        this.threadStats = new ThreadStats();
+        this.rpcCommonConfig = rpcCommonConfig;
+        this.rpcRequestProcessor = new RpcRequestProcessor(this.threadLocalProviderContext, this, serviceStats);
+
+        this.invokeServiceThreadPool = new ThreadPoolExecutor(rpcCommonConfig.getClientAsyncCallbackExecutorThreads(),
+            rpcCommonConfig.getClientAsyncCallbackExecutorThreads(), 60, TimeUnit.SECONDS,
+            new ArrayBlockingQueue<Runnable>(rpcCommonConfig.getServiceThreadBlockQueueSize()), newThreadFactory("rpcInvokeServiceThread", true));
+
+    }
+
+    public void start() {
+        this.defaultServiceAPI = new DefaultServiceAPIImpl(this.serviceStats, threadStats);
+        this.serviceStats.start();
+        this.publishService0(this.defaultServiceAPI);
+        this.remotingService().registerRequestProcessor(RpcRequestCode.CALL_SERVICE, this.rpcRequestProcessor, this.invokeServiceThreadPool);
+    }
+
+    public void stop() {
+        this.serviceStats.stop();
+        ThreadUtils.shutdownGracefully(this.invokeServiceThreadPool, 3000, TimeUnit.MILLISECONDS);
+    }
+
+    protected void publishService0(Object service) {
+        this.rpcRequestProcessor.putNewService(service);
+    }
+
+    protected void publishService0(Object service, ExecutorService executorService) {
+        this.rpcRequestProcessor.putNewService(service, executorService);
+    }
+
+    protected <T> T narrow0(Class<T> service, RpcJdkProxy rpcJdkProxy) {
+        return rpcJdkProxy.newInstance(service);
+    }
+
+    public abstract void registerServiceListener();
+
+    public ThreadPoolExecutor getInvokeServiceThreadPool() {
+        return invokeServiceThreadPool;
+    }
+
+    public void setInvokeServiceThreadPool(ThreadPoolExecutor invokeServiceThreadPool) {
+        this.invokeServiceThreadPool = invokeServiceThreadPool;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcJdkProxy.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcJdkProxy.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcJdkProxy.java
new file mode 100644
index 0000000..351c8b4
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcJdkProxy.java
@@ -0,0 +1,54 @@
+/*
+ * 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.rocketmq.rpc.impl.service;
+
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import org.apache.rocketmq.remoting.api.AsyncHandler;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.rpc.impl.exception.ServiceExceptionManager;
+import org.apache.rocketmq.rpc.internal.RpcErrorMapper;
+
+public abstract class RpcJdkProxy implements InvocationHandler {
+    private final Class<?> service;
+    private final RpcProxyCommon rpcProxyCommon;
+
+    public RpcJdkProxy(final Class<?> service, final RpcProxyCommon rpcProxyCommon) {
+        this.service = service;
+        this.rpcProxyCommon = rpcProxyCommon;
+    }
+
+    @Override
+    public Object invoke(final Object proxy, final Method method, final Object[] args) throws Throwable {
+        return rpcProxyCommon.invoke0(proxy, this, service, method, args);
+    }
+
+    @SuppressWarnings("unchecked")
+    public <T> T newInstance(Class<T> service) {
+        try {
+            return (T) Proxy.newProxyInstance(service.getClassLoader(), new Class[] {service}, this);
+        } catch (Exception e) {
+            throw ServiceExceptionManager.TranslateException(RpcErrorMapper.RpcErrorLatitude.LOCAL.getCode(), e);
+        }
+    }
+
+    public abstract void invokeOneWay(final RemotingCommand request);
+
+    public abstract void invokeAsync(final RemotingCommand request, final AsyncHandler handler);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcJdkProxyClient.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcJdkProxyClient.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcJdkProxyClient.java
new file mode 100644
index 0000000..4cff74f
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcJdkProxyClient.java
@@ -0,0 +1,50 @@
+/*
+ * 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.rocketmq.rpc.impl.service;
+
+import org.apache.rocketmq.remoting.api.AsyncHandler;
+import org.apache.rocketmq.remoting.api.RemotingClient;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.rpc.impl.config.RpcCommonConfig;
+
+public class RpcJdkProxyClient extends RpcJdkProxy {
+    private RemotingClient remotingClient;
+    private String remotingAddress;
+    private RpcCommonConfig rpcCommonConfig;
+
+    public RpcJdkProxyClient(final Class<?> service,
+        final RpcProxyCommon rpcProxyCommon,
+        final RemotingClient remotingClient,
+        final RpcCommonConfig rpcCommonConfig,
+        final String remotingAddress) {
+        super(service, rpcProxyCommon);
+        this.remotingClient = remotingClient;
+        this.rpcCommonConfig = rpcCommonConfig;
+        this.remotingAddress = remotingAddress;
+    }
+
+    @Override
+    public void invokeOneWay(final RemotingCommand request) {
+        this.remotingClient.invokeOneWay(remotingAddress, request, rpcCommonConfig.getServiceInvokeTimeout());
+    }
+
+    @Override
+    public void invokeAsync(final RemotingCommand request, final AsyncHandler handler) {
+        this.remotingClient.invokeAsync(remotingAddress, request, handler, rpcCommonConfig.getServiceInvokeTimeout());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq/blob/0b88e66f/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcJdkProxyServer.java
----------------------------------------------------------------------
diff --git a/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcJdkProxyServer.java b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcJdkProxyServer.java
new file mode 100644
index 0000000..c67c8cc
--- /dev/null
+++ b/remoting-core/rpc-impl/src/main/java/org/apache/rocketmq/rpc/impl/service/RpcJdkProxyServer.java
@@ -0,0 +1,51 @@
+/*
+ * 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.rocketmq.rpc.impl.service;
+
+import org.apache.rocketmq.remoting.api.AsyncHandler;
+import org.apache.rocketmq.remoting.api.RemotingServer;
+import org.apache.rocketmq.remoting.api.channel.RemotingChannel;
+import org.apache.rocketmq.remoting.api.command.RemotingCommand;
+import org.apache.rocketmq.rpc.impl.config.RpcCommonConfig;
+
+public class RpcJdkProxyServer extends RpcJdkProxy {
+    private final RemotingServer remotingServer;
+    private final RemotingChannel remotingChannel;
+    private final RpcCommonConfig rpcCommonConfig;
+
+    public RpcJdkProxyServer(final Class<?> service,
+        final RpcProxyCommon rpcProxyCommon,
+        final RemotingServer remotingServer,
+        final RpcCommonConfig rpcCommonConfig,
+        final RemotingChannel remotingChannel) {
+        super(service, rpcProxyCommon);
+        this.remotingServer = remotingServer;
+        this.remotingChannel = remotingChannel;
+        this.rpcCommonConfig = rpcCommonConfig;
+    }
+
+    @Override
+    public void invokeAsync(final RemotingCommand request, final AsyncHandler handler) {
+        this.remotingServer.invokeAsync(remotingChannel, request, handler, rpcCommonConfig.getServiceInvokeTimeout());
+    }
+
+    @Override
+    public void invokeOneWay(final RemotingCommand request) {
+        this.remotingServer.invokeOneWay(remotingChannel, request, rpcCommonConfig.getServiceInvokeTimeout());
+    }
+}