You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2016/08/19 11:48:55 UTC

[38/53] [abbrv] ignite git commit: IGNITE-1629 .NET: Introduced native logging facility. This closes #786.

IGNITE-1629 .NET: Introduced native logging facility. This closes #786.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/165c0cf9
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/165c0cf9
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/165c0cf9

Branch: refs/heads/ignite-3299
Commit: 165c0cf98ad788c5c0b9cd0210489a0cfd904d93
Parents: c154a08
Author: ptupitsyn <pt...@gridgain.com>
Authored: Mon Aug 15 16:40:24 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Aug 15 16:40:24 2016 +0300

----------------------------------------------------------------------
 .../logger/platform/PlatformLogger.java         | 223 ++++++++++
 .../platform/PlatformAbstractBootstrap.java     |  18 +-
 .../platform/PlatformConfigurationEx.java       |   6 +
 .../platform/PlatformNoopProcessor.java         |  10 +
 .../processors/platform/PlatformProcessor.java  |  16 +
 .../platform/PlatformProcessorImpl.java         |  56 +++
 .../callback/PlatformCallbackGateway.java       |  45 ++
 .../callback/PlatformCallbackUtils.java         |  20 +
 .../cpp/PlatformCppConfigurationEx.java         |   6 +
 .../dotnet/PlatformDotNetBootstrap.java         |   9 +
 .../PlatformDotNetConfigurationClosure.java     |  11 +-
 .../dotnet/PlatformDotNetConfigurationEx.java   |  12 +-
 .../cpp/jni/include/ignite/jni/exports.h        |   2 +
 .../platforms/cpp/jni/include/ignite/jni/java.h |  13 +
 modules/platforms/cpp/jni/project/vs/module.def |   3 +-
 modules/platforms/cpp/jni/src/exports.cpp       |  12 +-
 modules/platforms/cpp/jni/src/java.cpp          |  72 +++-
 .../Apache.Ignite.Core.Tests.csproj             |   8 +
 .../Binary/BinaryStringTest.cs                  |  26 +-
 .../Config/Log/custom-log.xml                   |  50 +++
 .../Config/Log/dotnet-log4j.xml                 | 143 ++++++
 .../IgniteConfigurationSerializerTest.cs        |  24 ++
 .../Log/CustomLoggerTest.cs                     | 430 +++++++++++++++++++
 .../Log/DefaultLoggerTest.cs                    | 114 +++++
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |  27 +-
 .../Apache.Ignite.Core.csproj                   |   6 +-
 .../Cache/Configuration/CacheConfiguration.cs   |  19 +-
 .../Cache/Configuration/QueryEntity.cs          |  25 +-
 .../Cache/Configuration/QueryField.cs           |  19 +-
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |   8 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  25 ++
 .../IgniteConfigurationSection.xsd              |  12 +
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |  39 +-
 .../Apache.Ignite.Core/Impl/Binary/JavaTypes.cs |  29 +-
 .../Apache.Ignite.Core/Impl/Common/Logger.cs    |  37 --
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |   9 +
 .../Apache.Ignite.Core/Impl/IgniteManager.cs    |   6 +-
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |   7 +
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |  26 +-
 .../Apache.Ignite.Core/Impl/Log/JavaLogger.cs   | 110 +++++
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |   7 +
 .../Impl/Unmanaged/UnmanagedCallbackHandlers.cs |   3 +
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |  76 +++-
 .../Impl/Unmanaged/UnmanagedUtils.cs            |  27 +-
 .../Apache.Ignite.Core/Log/CategoryLogger.cs    |  82 ++++
 .../dotnet/Apache.Ignite.Core/Log/ILogger.cs    |  51 +++
 .../dotnet/Apache.Ignite.Core/Log/LogLevel.cs   |  53 +++
 .../Apache.Ignite.Core/Log/LoggerExtensions.cs  | 320 ++++++++++++++
 48 files changed, 2241 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/core/src/main/java/org/apache/ignite/internal/logger/platform/PlatformLogger.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/logger/platform/PlatformLogger.java b/modules/core/src/main/java/org/apache/ignite/internal/logger/platform/PlatformLogger.java
new file mode 100644
index 0000000..0a0437e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/logger/platform/PlatformLogger.java
@@ -0,0 +1,223 @@
+/*
+ * 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.ignite.internal.logger.platform;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.binary.BinaryRawWriterEx;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformNativeException;
+import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
+import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+import org.apache.ignite.internal.util.typedef.X;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_QUIET;
+
+/**
+ * Logger that delegates to platform.
+ */
+public class PlatformLogger implements IgniteLogger {
+    /** */
+    public static final int LVL_TRACE = 0;
+
+    /** */
+    public static final int LVL_DEBUG = 1;
+
+    /** */
+    public static final int LVL_INFO = 2;
+
+    /** */
+    public static final int LVL_WARN = 3;
+
+    /** */
+    public static final int LVL_ERROR = 4;
+
+    /** Callbacks. */
+    private volatile PlatformCallbackGateway gate;
+
+    /** Context. */
+    private volatile PlatformContext ctx;
+
+    /** Category. */
+    private final String category;
+
+    /** Trace flag. */
+    private volatile boolean traceEnabled;
+
+    /** Debug flag. */
+    private volatile boolean debugEnabled;
+
+    /** Info flag. */
+    private volatile boolean infoEnabled;
+
+    /** Quiet flag. */
+    private static final boolean isQuiet = Boolean.valueOf(System.getProperty(IGNITE_QUIET, "true"));
+
+    /**
+     * Ctor.
+     *
+     */
+    public PlatformLogger() {
+        category = null;
+    }
+
+    /**
+     * Ctor.
+     */
+    private PlatformLogger(PlatformCallbackGateway gate, PlatformContext ctx, String category,
+        boolean traceEnabled, boolean debugEnabled, boolean infoEnabled) {
+        this.gate = gate;
+        this.ctx = ctx;
+        this.category = category;
+        this.traceEnabled = traceEnabled;
+        this.debugEnabled = debugEnabled;
+        this.infoEnabled = infoEnabled;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteLogger getLogger(Object ctgr) {
+        return new PlatformLogger(gate, ctx, getCategoryString(ctgr), traceEnabled, debugEnabled, infoEnabled);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void trace(String msg) {
+        log(LVL_TRACE, msg, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void debug(String msg) {
+        log(LVL_DEBUG, msg, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void info(String msg) {
+        log(LVL_INFO, msg, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void warning(String msg) {
+        log(LVL_WARN, msg, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void warning(String msg, @Nullable Throwable e) {
+        log(LVL_WARN, msg, e);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void error(String msg) {
+        log(LVL_ERROR, msg, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void error(String msg, @Nullable Throwable e) {
+        log(LVL_ERROR, msg, e);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isTraceEnabled() {
+        return traceEnabled;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isDebugEnabled() {
+        return debugEnabled;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isInfoEnabled() {
+        return infoEnabled;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isQuiet() {
+        return isQuiet;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String fileName() {
+        return null;
+    }
+
+    /**
+     * Sets the gateway.
+     *
+     * @param gate Callback gateway.
+     */
+    public void setGateway(PlatformCallbackGateway gate) {
+        assert gate != null;
+        this.gate = gate;
+
+        // Pre-calculate enabled levels (JNI calls are expensive)
+        traceEnabled = gate.loggerIsLevelEnabled(LVL_TRACE);
+        debugEnabled = gate.loggerIsLevelEnabled(LVL_DEBUG);
+        infoEnabled = gate.loggerIsLevelEnabled(LVL_INFO);
+    }
+
+    /**
+     * Sets the context.
+     *
+     * @param ctx Platform context.
+     */
+    public void setContext(PlatformContext ctx) {
+        assert ctx != null;
+        this.ctx = ctx;
+    }
+
+    /**
+     * Logs the message.
+     *
+     * @param level Log level.
+     * @param msg Message.
+     * @param e Exception.
+     */
+    private void log(int level, String msg, @Nullable Throwable e) {
+        String errorInfo = null;
+
+        if (e != null)
+            errorInfo = X.getFullStackTrace(e);
+
+        PlatformNativeException e0 = X.cause(e, PlatformNativeException.class);
+        if (ctx != null && e0 != null) {
+            try (PlatformMemory mem = ctx.memory().allocate()) {
+                PlatformOutputStream out = mem.output();
+                BinaryRawWriterEx writer = ctx.writer(out);
+                writer.writeObject(e0.cause());
+                out.synchronize();
+
+                gate.loggerLog(level, msg, category, errorInfo, mem.pointer());
+            }
+        }
+        else {
+            gate.loggerLog(level, msg, category, errorInfo, 0);
+        }
+    }
+
+    /**
+     * Gets the category string.
+     *
+     * @param ctgr Category object.
+     * @return Category string.
+     */
+    private static String getCategoryString(Object ctgr) {
+        return ctgr instanceof Class
+            ? ((Class)ctgr).getName()
+            : (ctgr == null ? null : String.valueOf(ctgr));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractBootstrap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractBootstrap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractBootstrap.java
index a28677f..8b4eb75 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractBootstrap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractBootstrap.java
@@ -22,7 +22,9 @@ import org.apache.ignite.Ignition;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgnitionEx;
+import org.apache.ignite.internal.logger.platform.PlatformLogger;
 import org.apache.ignite.internal.processors.platform.memory.PlatformExternalMemory;
+import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream;
 import org.apache.ignite.internal.processors.resource.GridSpringResourceContext;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteClosure;
@@ -35,7 +37,11 @@ public abstract class PlatformAbstractBootstrap implements PlatformBootstrap {
     /** {@inheritDoc} */
     @Override public PlatformProcessor start(IgniteConfiguration cfg, @Nullable GridSpringResourceContext springCtx,
         long envPtr, long dataPtr) {
-        Ignition.setClientMode(new PlatformExternalMemory(null, dataPtr).input().readBoolean());
+        final PlatformInputStream input = new PlatformExternalMemory(null, dataPtr).input();
+
+        Ignition.setClientMode(input.readBoolean());
+
+        processInput(input, cfg);
 
         IgniteConfiguration cfg0 = closure(envPtr).apply(cfg);
 
@@ -61,4 +67,14 @@ public abstract class PlatformAbstractBootstrap implements PlatformBootstrap {
      * @return Closure.
      */
     protected abstract IgniteClosure<IgniteConfiguration, IgniteConfiguration> closure(long envPtr);
+
+    /**
+     * Processes any additional input data.
+     *
+     * @param input Input stream.
+     * @param cfg Config.
+     */
+    protected void processInput(PlatformInputStream input, IgniteConfiguration cfg) {
+        // No-op.
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformConfigurationEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformConfigurationEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformConfigurationEx.java
index 66eff8b..96e4643 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformConfigurationEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformConfigurationEx.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform;
 
+import org.apache.ignite.internal.logger.platform.PlatformLogger;
 import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryManagerImpl;
 
@@ -45,4 +46,9 @@ public interface PlatformConfigurationEx {
      * @return Warnings to be displayed on grid start.
      */
     public Collection<String> warnings();
+
+    /**
+     * @return Platform logger.
+     */
+    public PlatformLogger logger();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
index cdf418f..a7b7a8d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
@@ -175,4 +175,14 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
     @Override public PlatformTarget getOrCreateNearCache(@Nullable String cacheName, long memPtr) {
         return null;
     }
+
+    /** {@inheritDoc} */
+    @Override public boolean loggerIsLevelEnabled(int level) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void loggerLog(int level, String message, String category, String errorInfo) {
+        // No-op.
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
index fc42b68..fdc66cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
@@ -259,4 +259,20 @@ public interface PlatformProcessor extends GridProcessor {
      */
     public PlatformTarget getOrCreateNearCache(@Nullable String cacheName, long memPtr);
 
+    /**
+     * Gets a value indicating whether Ignite logger has specified level enabled.
+     *
+     * @param level Log level.
+     */
+    public boolean loggerIsLevelEnabled(int level);
+
+    /**
+     * Logs to the Ignite logger.
+     *
+     * @param level Log level.
+     * @param message Message.
+     * @param category Category.
+     * @param errorInfo Error info.
+     */
+    public void loggerLog(int level, String message, String category, String errorInfo);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
index 8c9e205..e367768 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
+import org.apache.ignite.internal.logger.platform.PlatformLogger;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl;
@@ -118,6 +119,9 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
         }
 
         platformCtx = new PlatformContextImpl(ctx, interopCfg.gate(), interopCfg.memory(), interopCfg.platform());
+
+        if (interopCfg.logger() != null)
+            interopCfg.logger().setContext(platformCtx);
     }
 
     /** {@inheritDoc} */
@@ -416,6 +420,58 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
         return new PlatformCache(platformCtx, cache.keepBinary(), false);
     }
 
+    /** {@inheritDoc} */
+    @Override public boolean loggerIsLevelEnabled(int level) {
+        IgniteLogger log = ctx.grid().log();
+
+        switch (level) {
+            case PlatformLogger.LVL_TRACE:
+                return log.isTraceEnabled();
+            case PlatformLogger.LVL_DEBUG:
+                return log.isDebugEnabled();
+            case PlatformLogger.LVL_INFO:
+                return log.isInfoEnabled();
+            case PlatformLogger.LVL_WARN:
+                return true;
+            case PlatformLogger.LVL_ERROR:
+                return true;
+            default:
+                assert false;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void loggerLog(int level, String message, String category, String errorInfo) {
+        IgniteLogger log = ctx.grid().log();
+
+        if (category != null)
+            log = log.getLogger(category);
+
+        Throwable err = errorInfo == null ? null : new IgniteException("Platform error:" + errorInfo);
+
+        switch (level) {
+            case PlatformLogger.LVL_TRACE:
+                log.trace(message);
+                break;
+            case PlatformLogger.LVL_DEBUG:
+                log.debug(message);
+                break;
+            case PlatformLogger.LVL_INFO:
+                log.info(message);
+                break;
+            case PlatformLogger.LVL_WARN:
+                log.warning(message, err);
+                break;
+            case PlatformLogger.LVL_ERROR:
+                log.error(message, err);
+                break;
+            default:
+                assert false;
+        }
+    }
+
     /**
      * Gets the near cache config.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
index 41d3802..de23242 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
@@ -943,6 +943,44 @@ public class PlatformCallbackGateway {
     }
 
     /**
+     * Logs to the platform.
+     *
+     * @param level Log level.
+     * @param message Message.
+     * @param category Category.
+     * @param errorInfo Error info.
+     * @param memPtr Pointer to optional payload (serialized exception).
+     */
+    public void loggerLog(int level, String message, String category, String errorInfo, long memPtr) {
+        if (!tryEnter())
+            return;  // Do not lock for logger: this should work during shutdown
+
+        try {
+            PlatformCallbackUtils.loggerLog(envPtr, level, message, category, errorInfo, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Gets a value indicating whether native logger has specified level enabled.
+     *
+     * @param level Log level.
+     */
+    public boolean loggerIsLevelEnabled(int level) {
+        if (!tryEnter())
+            return false;  // Do not lock for logger: this should work during shutdown
+
+        try {
+            return PlatformCallbackUtils.loggerIsLevelEnabled(envPtr, level);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
      * Kernal stop callback.
      */
     public void onStop() {
@@ -1058,6 +1096,13 @@ public class PlatformCallbackGateway {
     }
 
     /**
+     * Enter gateway.
+     */
+    protected boolean tryEnter() {
+        return lock.enterBusy();
+    }
+
+    /**
      * Leave gateway.
      */
     protected void leave() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
index 63c6682..09a7f1b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
@@ -553,6 +553,26 @@ public class PlatformCallbackUtils {
     static native void consoleWrite(String str, boolean isErr);
 
     /**
+     * Logs to the native logger.
+     *
+     * @param envPtr Environment pointer.
+     * @param level Log level.
+     * @param message Message.
+     * @param category Category.
+     * @param errorInfo Error info.
+     * @param memPtr Pointer to optional payload (serialized exception).
+     */
+    static native void loggerLog(long envPtr, int level, String message, String category, String errorInfo, long memPtr);
+
+    /**
+     * Gets a value indicating whether native logger has specified level enabled.
+     *
+     * @param envPtr Environment pointer.
+     * @param level Log level.
+     */
+    static native boolean loggerIsLevelEnabled(long envPtr, int level);
+
+    /**
      * Private constructor.
      */
     private PlatformCallbackUtils() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationEx.java
index ea11ce9..75c48ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationEx.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform.cpp;
 
+import org.apache.ignite.internal.logger.platform.PlatformLogger;
 import org.apache.ignite.internal.processors.platform.PlatformConfigurationEx;
 import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryManagerImpl;
@@ -73,6 +74,11 @@ public class PlatformCppConfigurationEx extends PlatformCppConfiguration impleme
         return warns;
     }
 
+    /** {@inheritDoc} */
+    @Override public PlatformLogger logger() {
+        return null;
+    }
+
     /**
      * @param warnings Warnings.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetBootstrap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetBootstrap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetBootstrap.java
index 9278246..84a4577 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetBootstrap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetBootstrap.java
@@ -17,8 +17,11 @@
 
 package org.apache.ignite.internal.processors.platform.dotnet;
 
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.logger.platform.PlatformLogger;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractBootstrap;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurationClosure;
+import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream;
 
 import java.io.PrintStream;
 
@@ -40,4 +43,10 @@ public class PlatformDotNetBootstrap extends PlatformAbstractBootstrap {
     @Override protected PlatformAbstractConfigurationClosure closure(long envPtr) {
         return new PlatformDotNetConfigurationClosure(envPtr);
     }
+
+    /** {@inheritDoc} */
+    @Override protected void processInput(PlatformInputStream input, IgniteConfiguration cfg) {
+        if (input.readBoolean())
+            cfg.setGridLogger(new PlatformLogger());
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
index cb9696c..0c691af 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.GridBinaryMarshaller;
+import org.apache.ignite.internal.logger.platform.PlatformLogger;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurationClosure;
 import org.apache.ignite.internal.processors.platform.lifecycle.PlatformLifecycleBean;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
@@ -89,7 +90,15 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
 
         memMgr = new PlatformMemoryManagerImpl(gate, 1024);
 
-        PlatformDotNetConfigurationEx dotNetCfg0 = new PlatformDotNetConfigurationEx(dotNetCfg, gate, memMgr);
+        PlatformLogger userLogger = null;
+
+        if (igniteCfg.getGridLogger() instanceof PlatformLogger) {
+            userLogger = (PlatformLogger)igniteCfg.getGridLogger();
+            userLogger.setGateway(gate);
+        }
+
+        PlatformDotNetConfigurationEx dotNetCfg0 = new PlatformDotNetConfigurationEx(dotNetCfg, gate, memMgr,
+            userLogger);
 
         igniteCfg.setPlatformConfiguration(dotNetCfg0);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java
index eaf0997..906080e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform.dotnet;
 
+import org.apache.ignite.internal.logger.platform.PlatformLogger;
 import org.apache.ignite.internal.processors.platform.PlatformConfigurationEx;
 import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryManagerImpl;
@@ -35,6 +36,9 @@ public class PlatformDotNetConfigurationEx extends PlatformDotNetConfiguration i
     /** Memory manager. */
     private final PlatformMemoryManagerImpl memMgr;
 
+    /** Logger. */
+    private final PlatformLogger logger;
+
     /** Warnings */
     private Collection<String> warnings;
 
@@ -46,11 +50,12 @@ public class PlatformDotNetConfigurationEx extends PlatformDotNetConfiguration i
      * @param memMgr Memory manager.
      */
     public PlatformDotNetConfigurationEx(PlatformDotNetConfiguration cfg, PlatformCallbackGateway gate,
-        PlatformMemoryManagerImpl memMgr) {
+        PlatformMemoryManagerImpl memMgr, PlatformLogger logger) {
         super(cfg);
 
         this.gate = gate;
         this.memMgr = memMgr;
+        this.logger = logger;
     }
 
     /** {@inheritDoc} */
@@ -73,6 +78,11 @@ public class PlatformDotNetConfigurationEx extends PlatformDotNetConfiguration i
         return warnings;
     }
 
+    /** {@inheritDoc} */
+    @Override public PlatformLogger logger() {
+        return logger;
+    }
+
     /**
      * @param warnings Warnings.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/platforms/cpp/jni/include/ignite/jni/exports.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/include/ignite/jni/exports.h b/modules/platforms/cpp/jni/include/ignite/jni/exports.h
index 3f400fb..7fae0d2 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/exports.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/exports.h
@@ -54,6 +54,8 @@ extern "C" {
     void* IGNITE_CALL IgniteProcessorAtomicReference(gcj::JniContext* ctx, void* obj, char* name, long long memPtr, bool create);
     void IGNITE_CALL IgniteProcessorGetIgniteConfiguration(gcj::JniContext* ctx, void* obj, long long memPtr);
     void IGNITE_CALL IgniteProcessorGetCacheNames(gcj::JniContext* ctx, void* obj, long long memPtr);
+    bool IGNITE_CALL IgniteProcessorLoggerIsLevelEnabled(gcj::JniContext* ctx, void* obj, int level);
+    void IGNITE_CALL IgniteProcessorLoggerLog(gcj::JniContext* ctx, void* obj, int level, char* message, char* category, char* errorInfo);
     
     long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr);
     void IGNITE_CALL IgniteTargetInStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, long long inMemPtr, long long outMemPtr);

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/platforms/cpp/jni/include/ignite/jni/java.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/include/ignite/jni/java.h b/modules/platforms/cpp/jni/include/ignite/jni/java.h
index 8d79a7d..4cf00a3 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/java.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/java.h
@@ -108,6 +108,9 @@ namespace ignite
 
             typedef void(JNICALL *ConsoleWriteHandler)(const char* chars, int charsLen, unsigned char isErr);
 
+            typedef void(JNICALL *LoggerLogHandler)(void* target, int level, const char* messageChars, int messageCharsLen, const char* categoryChars, int categoryCharsLen, const char* errorInfoChars, int errorInfoCharsLen, long long memPtr);
+            typedef bool(JNICALL *LoggerIsLevelEnabledHandler)(void* target, int level);
+
             /**
              * JNI handlers holder.
              */
@@ -191,6 +194,9 @@ namespace ignite
                 AffinityFunctionAssignPartitionsHandler affinityFunctionAssignPartitions;
                 AffinityFunctionRemoveNodeHandler affinityFunctionRemoveNode;
                 AffinityFunctionDestroyHandler affinityFunctionDestroy;
+
+                LoggerLogHandler loggerLog;
+                LoggerIsLevelEnabledHandler loggerIsLevelEnabled;
             };
 
             /**
@@ -335,6 +341,8 @@ namespace ignite
                 jmethodID m_PlatformProcessor_getCacheNames;
                 jmethodID m_PlatformProcessor_atomicSequence;
                 jmethodID m_PlatformProcessor_atomicReference;
+                jmethodID m_PlatformProcessor_loggerIsLevelEnabled;
+                jmethodID m_PlatformProcessor_loggerLog;
 
                 jclass c_PlatformTarget;
                 jmethodID m_PlatformTarget_inStreamOutLong;
@@ -554,6 +562,8 @@ namespace ignite
                 jobject ProcessorAtomicReference(jobject obj, char* name, long long memPtr, bool create);
 				void ProcessorGetIgniteConfiguration(jobject obj, long long memPtr);
 				void ProcessorGetCacheNames(jobject obj, long long memPtr);
+				bool ProcessorLoggerIsLevelEnabled(jobject obj, int level);
+				void ProcessorLoggerLog(jobject obj, int level, char* message, char* category, char* errorInfo);
 
                 long long TargetInStreamOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);
                 void TargetInStreamOutStream(jobject obj, int opType, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL);
@@ -767,6 +777,9 @@ namespace ignite
             JNIEXPORT void JNICALL JniAffinityFunctionDestroy(JNIEnv *env, jclass cls, jlong envPtr, jlong ptr);
 
             JNIEXPORT void JNICALL JniConsoleWrite(JNIEnv *env, jclass cls, jstring str, jboolean isErr);
+
+            JNIEXPORT void JNICALL JniLoggerLog(JNIEnv *env, jclass cls, jlong envPtr, jint level, jstring message, jstring category, jstring errorInfo, jlong memPtr);
+            JNIEXPORT jboolean JNICALL JniLoggerIsLevelEnabled(JNIEnv *env, jclass cls, jlong envPtr, jint level);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/platforms/cpp/jni/project/vs/module.def
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/project/vs/module.def b/modules/platforms/cpp/jni/project/vs/module.def
index ddddace..c1582e6 100644
--- a/modules/platforms/cpp/jni/project/vs/module.def
+++ b/modules/platforms/cpp/jni/project/vs/module.def
@@ -136,4 +136,5 @@ IgniteProcessorGetCacheNames @133
 IgniteProjectionForServers @134
 IgniteSetConsoleHandler @135
 IgniteRemoveConsoleHandler @136
- 
\ No newline at end of file
+IgniteProcessorLoggerIsLevelEnabled @137
+IgniteProcessorLoggerLog @138
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/platforms/cpp/jni/src/exports.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/src/exports.cpp b/modules/platforms/cpp/jni/src/exports.cpp
index 2950d15..8ef8188 100644
--- a/modules/platforms/cpp/jni/src/exports.cpp
+++ b/modules/platforms/cpp/jni/src/exports.cpp
@@ -93,11 +93,11 @@ extern "C" {
     void*IGNITE_CALL IgniteProcessorDataStreamer(gcj::JniContext* ctx, void* obj, char* name, bool keepPortable) {
         return ctx->ProcessorDataStreamer(static_cast<jobject>(obj), name, keepPortable);
     }
-    
+
     void* IGNITE_CALL IgniteProcessorTransactions(gcj::JniContext* ctx, void* obj) {
         return ctx->ProcessorTransactions(static_cast<jobject>(obj));
     }
-        
+
     void* IGNITE_CALL IgniteProcessorCompute(gcj::JniContext* ctx, void* obj, void* prj) {
         return ctx->ProcessorCompute(static_cast<jobject>(obj), static_cast<jobject>(prj));
     }
@@ -138,6 +138,14 @@ extern "C" {
         return ctx->ProcessorGetCacheNames(static_cast<jobject>(obj), memPtr);
     }
 
+    bool IGNITE_CALL IgniteProcessorLoggerIsLevelEnabled(gcj::JniContext* ctx, void* obj, int level) {        
+        return ctx->ProcessorLoggerIsLevelEnabled(static_cast<jobject>(obj), level);
+    }
+
+    void IGNITE_CALL IgniteProcessorLoggerLog(gcj::JniContext* ctx, void* obj, int level, char* message, char* category, char* errorInfo) {
+        ctx->ProcessorLoggerLog(static_cast<jobject>(obj), level, message, category, errorInfo);
+    }
+
     long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr) {
         return ctx->TargetInStreamOutLong(static_cast<jobject>(obj), opType, memPtr);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/platforms/cpp/jni/src/java.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/src/java.cpp b/modules/platforms/cpp/jni/src/java.cpp
index 7bd6287..eb0d89a 100644
--- a/modules/platforms/cpp/jni/src/java.cpp
+++ b/modules/platforms/cpp/jni/src/java.cpp
@@ -216,6 +216,8 @@ namespace ignite
             JniMethod M_PLATFORM_PROCESSOR_ATOMIC_REFERENCE = JniMethod("atomicReference", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
             JniMethod M_PLATFORM_PROCESSOR_GET_IGNITE_CONFIGURATION = JniMethod("getIgniteConfiguration", "(J)V", false);
             JniMethod M_PLATFORM_PROCESSOR_GET_CACHE_NAMES = JniMethod("getCacheNames", "(J)V", false);
+            JniMethod M_PLATFORM_PROCESSOR_LOGGER_IS_LEVEL_ENABLED = JniMethod("loggerIsLevelEnabled", "(I)Z", false);
+            JniMethod M_PLATFORM_PROCESSOR_LOGGER_LOG = JniMethod("loggerLog", "(ILjava/lang/String;Ljava/lang/String;Ljava/lang/String;)V", false);
 
             const char* C_PLATFORM_TARGET = "org/apache/ignite/internal/processors/platform/PlatformTarget";
             JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_LONG = JniMethod("inStreamOutLong", "(IJ)J", false);
@@ -366,6 +368,9 @@ namespace ignite
             JniMethod M_PLATFORM_CALLBACK_UTILS_ON_CLIENT_DISCONNECTED = JniMethod("onClientDisconnected", "(J)V", true);
             JniMethod M_PLATFORM_CALLBACK_UTILS_ON_CLIENT_RECONNECTED = JniMethod("onClientReconnected", "(JZ)V", true);
 
+            JniMethod M_PLATFORM_CALLBACK_UTILS_LOGGER_LOG = JniMethod("loggerLog", "(JILjava/lang/String;Ljava/lang/String;Ljava/lang/String;J)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_LOGGER_IS_LEVEL_ENABLED = JniMethod("loggerIsLevelEnabled", "(JI)Z", true);
+
             JniMethod M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_INIT = JniMethod("affinityFunctionInit", "(JJLorg/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunctionTarget;)J", true);
             JniMethod M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_PARTITION = JniMethod("affinityFunctionPartition", "(JJJ)I", true);
             JniMethod M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_ASSIGN_PARTITIONS = JniMethod("affinityFunctionAssignPartitions", "(JJJJ)V", true);
@@ -716,6 +721,8 @@ namespace ignite
                 m_PlatformProcessor_atomicReference = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_ATOMIC_REFERENCE);
 				m_PlatformProcessor_getIgniteConfiguration = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_GET_IGNITE_CONFIGURATION);
 				m_PlatformProcessor_getCacheNames = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_GET_CACHE_NAMES);
+				m_PlatformProcessor_loggerIsLevelEnabled = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_LOGGER_IS_LEVEL_ENABLED);
+				m_PlatformProcessor_loggerLog = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_LOGGER_LOG);
 
                 c_PlatformTarget = FindClass(env, C_PLATFORM_TARGET);
                 m_PlatformTarget_inStreamOutLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_STREAM_OUT_LONG);
@@ -851,7 +858,7 @@ namespace ignite
 
             void RegisterNatives(JNIEnv* env) {
                 {
-					JNINativeMethod methods[60];
+					JNINativeMethod methods[62];
 
                     int idx = 0;
 
@@ -935,6 +942,9 @@ namespace ignite
                     AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_DESTROY, reinterpret_cast<void*>(JniAffinityFunctionDestroy));
                     AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_CONSOLE_WRITE, reinterpret_cast<void*>(JniConsoleWrite));
 
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_LOGGER_LOG, reinterpret_cast<void*>(JniLoggerLog));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_LOGGER_IS_LEVEL_ENABLED, reinterpret_cast<void*>(JniLoggerIsLevelEnabled));
+
                     jint res = env->RegisterNatives(FindClass(env, C_PLATFORM_CALLBACK_UTILS), methods, idx);
 
                     if (res != JNI_OK)
@@ -1526,6 +1536,40 @@ namespace ignite
                 ExceptionCheck(env);
             }
 
+            bool JniContext::ProcessorLoggerIsLevelEnabled(jobject obj, int level)
+            {
+                JNIEnv* env = Attach();
+
+                jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformProcessor_loggerIsLevelEnabled, level);
+
+                ExceptionCheck(env);
+
+                return res != 0;
+            }
+
+            void JniContext::ProcessorLoggerLog(jobject obj, int level, char* message, char* category, char* errorInfo)
+            {
+                JNIEnv* env = Attach();
+
+                jstring message0 = message != NULL ? env->NewStringUTF(message) : NULL;
+                jstring category0 = category != NULL ? env->NewStringUTF(category) : NULL;
+                jstring errorInfo0 = errorInfo != NULL ? env->NewStringUTF(errorInfo) : NULL;
+
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformProcessor_loggerLog, level, message0, category0, errorInfo0);
+
+                if (message0)
+                    env->DeleteLocalRef(message0);
+
+                if (category0)
+                    env->DeleteLocalRef(category0);
+
+                if (errorInfo0)
+                    env->DeleteLocalRef(errorInfo0);
+
+                ExceptionCheck(env);
+            }
+
             long long JniContext::TargetInStreamOutLong(jobject obj, int opType, long long memPtr, JniErrorInfo* err) {
                 JNIEnv* env = Attach();
 
@@ -2952,6 +2996,32 @@ namespace ignite
 
                 CONSOLE_LOCK.Leave();
             }
+
+            JNIEXPORT void JNICALL JniLoggerLog(JNIEnv *env, jclass cls, jlong envPtr, jint level, jstring message, jstring category, jstring errorInfo, jlong memPtr) {
+                int messageLen;
+                char* messageChars = StringToChars(env, message, &messageLen);
+
+                int categoryLen;
+                char* categoryChars = StringToChars(env, category, &categoryLen);
+
+                int errorInfoLen;
+                char* errorInfoChars = StringToChars(env, errorInfo, &errorInfoLen);
+
+                IGNITE_SAFE_PROC(env, envPtr, LoggerLogHandler, loggerLog, level, messageChars, messageLen, categoryChars, categoryLen, errorInfoChars, errorInfoLen, memPtr);
+
+                if (messageChars)
+                    delete[] messageChars;
+
+                if (categoryChars)
+                    delete[] categoryChars;
+
+                if (errorInfoChars)
+                    delete[] errorInfoChars;
+            }
+
+            JNIEXPORT jboolean JNICALL JniLoggerIsLevelEnabled(JNIEnv *env, jclass cls, jlong envPtr, jint level) {
+                IGNITE_SAFE_FUNC(env, envPtr, LoggerIsLevelEnabledHandler, loggerIsLevelEnabled, level);
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 7d21b34..29dc81b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -56,6 +56,7 @@
     <Reference Include="System.Xml.Linq" />
   </ItemGroup>
   <ItemGroup>
+    <Compile Include="Log\DefaultLoggerTest.cs" />
     <Compile Include="TestAppConfig.cs" />
     <Compile Include="AspNet\IgniteOutputCacheProviderTest.cs" />
     <Compile Include="Binary\BinaryBuilderSelfTestFullFooter.cs" />
@@ -139,6 +140,7 @@
     <Compile Include="LifecycleTest.cs" />
     <Compile Include="LoadDllTest.cs" />
     <Compile Include="IgniteManagerTest.cs" />
+    <Compile Include="Log\CustomLoggerTest.cs" />
     <Compile Include="MarshallerTest.cs" />
     <Compile Include="MessagingTest.cs" />
     <Compile Include="BinaryConfigurationTest.cs" />
@@ -249,6 +251,12 @@
     <Content Include="Config\Lifecycle\lifecycle-no-beans.xml">
       <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
     </Content>
+    <Content Include="Config\Log\dotnet-log4j.xml">
+      <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\Log\custom-log.xml">
+      <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
+    </Content>
     <Content Include="Config\spring-test.xml">
       <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
     </Content>

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryStringTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryStringTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryStringTest.cs
index 0c1f0f3..0593af7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryStringTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryStringTest.cs
@@ -18,7 +18,6 @@
 namespace Apache.Ignite.Core.Tests.Binary
 {
     using System;
-    using System.Diagnostics;
     using Apache.Ignite.Core.Impl.Binary;
     using NUnit.Framework;
 
@@ -83,29 +82,10 @@ namespace Apache.Ignite.Core.Tests.Binary
         [Test]
         public void TestNewMode()
         {
-            // Run "TestNewMode" in a separate process
-            var envVar = BinaryUtils.IgniteBinaryMarshallerUseStringSerializationVer2;
+            // Run "TestOldMode" in a separate process with changed setting.
+            Environment.SetEnvironmentVariable(BinaryUtils.IgniteBinaryMarshallerUseStringSerializationVer2, "true");
 
-            Environment.SetEnvironmentVariable(envVar, "true");
-
-            var procStart = new ProcessStartInfo
-            {
-                FileName = GetType().Assembly.Location,
-                Arguments = GetType().FullName + " TestOldMode",
-                CreateNoWindow = true,
-                UseShellExecute = false,
-                RedirectStandardOutput = true,
-                RedirectStandardError = true
-            };
-
-            var proc = Process.Start(procStart);
-
-            Assert.IsNotNull(proc);
-
-            Console.WriteLine(proc.StandardOutput.ReadToEnd());
-            Console.WriteLine(proc.StandardError.ReadToEnd());
-            Assert.IsTrue(proc.WaitForExit(15000));
-            Assert.AreEqual(0, proc.ExitCode);
+            TestUtils.RunTestInNewProcess(GetType().FullName, "TestOldMode");
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Log/custom-log.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Log/custom-log.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Log/custom-log.xml
new file mode 100644
index 0000000..5d7dc65
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Log/custom-log.xml
@@ -0,0 +1,50 @@
+<?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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="gridLogger">
+            <bean class="org.apache.ignite.logger.log4j.Log4JLogger">
+                <constructor-arg type="java.lang.String" value="config\log\dotnet-log4j.xml"/>
+            </bean>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+                <property name="socketTimeout" value="300" />
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Log/dotnet-log4j.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Log/dotnet-log4j.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Log/dotnet-log4j.xml
new file mode 100644
index 0000000..c8c62d7
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Log/dotnet-log4j.xml
@@ -0,0 +1,143 @@
+<?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 log4j:configuration PUBLIC "-//APACHE//DTD LOG4J 1.2//EN"
+    "http://logging.apache.org/log4j/1.2/apidocs/org/apache/log4j/xml/doc-files/log4j.dtd">
+<!--
+    Default log4j configuration for Ignite.
+-->
+<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/" debug="false">
+    <!--
+        Logs System.out messages to console.
+    -->
+    <appender name="CONSOLE" class="org.apache.log4j.ConsoleAppender">
+        <!-- Log to STDOUT. -->
+        <param name="Target" value="System.out"/>
+
+        <!-- Log from DEBUG and higher. -->
+        <param name="Threshold" value="DEBUG"/>
+
+        <!-- The default pattern: Date Priority [Category] Message\n -->
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="[%d{dd-MM-yyyy HH:mm:ss}][%-5p][%t][%c{1}] %m%n"/>
+        </layout>
+
+        <!-- Do not log beyond INFO level. -->
+        <filter class="org.apache.log4j.varia.LevelRangeFilter">
+            <param name="levelMin" value="DEBUG"/>
+            <param name="levelMax" value="INFO"/>
+        </filter>
+    </appender>
+
+    <!--
+        Logs all System.err messages to console.
+    -->
+    <appender name="CONSOLE_ERR" class="org.apache.log4j.ConsoleAppender">
+        <!-- Log to STDERR. -->
+        <param name="Target" value="System.err"/>
+
+        <!-- Log from WARN and higher. -->
+        <param name="Threshold" value="WARN"/>
+
+        <!-- The default pattern: Date Priority [Category] Message\n -->
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="[%d{dd-MMM-yyyy HH:mm:ss}][%-5p][%t][%c{1}] %m%n"/>
+        </layout>
+    </appender>
+
+    <!--
+        Logs all output to specified file.
+        By default, the logging goes to IGNITE_HOME/work/log folder
+
+        Note, this appender is disabled by default.
+        To enable, uncomment the section below and also FILE appender in the <root> element.
+    -->
+    <appender name="FILE" class="org.apache.log4j.RollingFileAppender">
+        <param name="Threshold" value="DEBUG"/>
+        <param name="File" value="${IGNITE_HOME}/work/log/dotnet-logger-test.log"/>
+        <param name="Append" value="true"/>
+        <param name="MaxFileSize" value="10MB"/>
+        <param name="MaxBackupIndex" value="10"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="[%d{ABSOLUTE}][%-5p][%t][%c{1}] %m%n"/>
+        </layout>
+    </appender>
+
+    <!--
+        Uncomment to enable Ignite debugging.
+    -->
+    <!--
+    <category name="org.apache.ignite">
+        <level value="DEBUG"/>
+    </category>
+    -->
+
+    <!--
+        Uncomment this category to enable cache
+        query execution tracing.
+    -->
+    <!--
+    <category name="org.apache.ignite.cache.queries">
+        <level value="DEBUG"/>
+    </category>
+    -->
+
+    <!--
+        Uncomment to enable DGC tracing.
+    -->
+    <!--
+    <category name="org.apache.ignite.grid.kernal.processors.cache.GridCacheDgcManager.trace">
+        <level value="DEBUG"/>
+    </category>
+    -->
+
+    <!--
+        Uncomment to disable courtesy notice.
+    -->
+    <!--
+    <category name="org.apache.ignite.CourtesyConfigNotice">
+        <level value="OFF"/>
+    </category>
+    -->
+
+    <category name="org.springframework">
+        <level value="WARN"/>
+    </category>
+
+    <category name="org.eclipse.jetty">
+        <level value="FATAL"/>
+    </category>
+
+    <category name="com.amazonaws">
+        <level value="WARN"/>
+    </category>
+
+    <!-- Default settings. -->
+    <root>
+        <!-- Print out all info by default. -->
+        <level value="DEBUG"/>
+
+        <!-- Append to console. -->
+        <appender-ref ref="CONSOLE"/>
+        <appender-ref ref="CONSOLE_ERR"/>
+
+        <!-- Uncomment to enable logging to a file. -->
+        <appender-ref ref="FILE"/>
+    </root>
+</log4j:configuration>

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
index b9e0236..7ba1897 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -45,6 +45,7 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.Events;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Lifecycle;
+    using Apache.Ignite.Core.Log;
     using Apache.Ignite.Core.Tests.Binary;
     using Apache.Ignite.Core.Transactions;
     using NUnit.Framework;
@@ -111,6 +112,7 @@ namespace Apache.Ignite.Core.Tests
                             <userAttributes><pair key='myNode' value='true' /></userAttributes>
                             <atomicConfiguration backups='2' cacheMode='Local' atomicSequenceReserveSize='250' />
                             <transactionConfiguration defaultTransactionConcurrency='Optimistic' defaultTransactionIsolation='RepeatableRead' defaultTimeout='0:1:2' pessimisticTransactionLogSize='15' pessimisticTransactionLogLinger='0:0:33' />
+                            <logger type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+TestLogger, Apache.Ignite.Core.Tests' />
                         </igniteConfig>";
 
             var cfg = IgniteConfiguration.FromXml(xml);
@@ -194,6 +196,8 @@ namespace Apache.Ignite.Core.Tests
             Assert.IsNotNull(comm);
             Assert.AreEqual(33, comm.AckSendThreshold);
             Assert.AreEqual(new TimeSpan(0, 1, 2), comm.IdleConnectionTimeout);
+
+            Assert.IsInstanceOf<TestLogger>(cfg.Logger);
         }
 
         /// <summary>
@@ -708,6 +712,7 @@ namespace Apache.Ignite.Core.Tests
                 },
                 IsLateAffinityAssignment = false,
                 SpringConfigUrl = "test",
+                Logger = new TestLogger()
             };
         }
 
@@ -845,5 +850,24 @@ namespace Apache.Ignite.Core.Tests
                 return null;
             }
         }
+
+        /// <summary>
+        /// Test logger.
+        /// </summary>
+        public class TestLogger : ILogger
+        {
+            /** <inheritdoc /> */
+            public void Log(LogLevel level, string message, object[] args, IFormatProvider formatProvider, string category,
+                string nativeErrorInfo, Exception ex)
+            {
+                throw new NotImplementedException();
+            }
+
+            /** <inheritdoc /> */
+            public bool IsEnabled(LogLevel level)
+            {
+                throw new NotImplementedException();
+            }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/CustomLoggerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/CustomLoggerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/CustomLoggerTest.cs
new file mode 100644
index 0000000..73134fe
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/CustomLoggerTest.cs
@@ -0,0 +1,430 @@
+\ufeff/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Log
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Globalization;
+    using System.Linq;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Communication.Tcp;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Lifecycle;
+    using Apache.Ignite.Core.Log;
+    using Apache.Ignite.Core.Resource;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests that user-defined logger receives Ignite events.
+    /// </summary>
+    public class CustomLoggerTest
+    {
+        /** */
+        private static readonly LogLevel[] AllLevels = Enum.GetValues(typeof (LogLevel)).OfType<LogLevel>().ToArray();
+
+        /// <summary>
+        /// Test setup.
+        /// </summary>
+        [SetUp]
+        public void TestSetUp()
+        {
+            TestLogger.Entries.Clear();
+        }
+
+        /// <summary>
+        /// Tests the startup output.
+        /// </summary>
+        [Test]
+        public void TestStartupOutput()
+        {
+            var cfg = GetConfigWithLogger(true);
+            using (var ignite = Ignition.Start(cfg))
+            {
+                // Check injection
+                Assert.AreEqual(((Ignite) ignite).Proxy, ((TestLogger) cfg.Logger).Ignite);
+
+                // Check initial message
+                Assert.IsTrue(TestLogger.Entries[0].Message.StartsWith("Starting Ignite.NET"));
+
+                // Check topology message
+                Assert.IsTrue(
+                    TestUtils.WaitForCondition(() =>
+                    {
+                        lock (TestLogger.Entries)
+                        {
+                            return TestLogger.Entries.Any(x => x.Message.Contains("Topology snapshot"));
+                        }
+                    }, 9000), "No topology snapshot");
+            }
+
+            // Test that all levels are present
+            foreach (var level in AllLevels.Where(x => x != LogLevel.Error))
+                Assert.IsTrue(TestLogger.Entries.Any(x => x.Level == level), "No messages with level " + level);
+        }
+
+
+        /// <summary>
+        /// Tests startup error in Java.
+        /// </summary>
+        [Test]
+        public void TestStartupJavaError()
+        {
+            // Invalid config
+            Assert.Throws<IgniteException>(() =>
+                Ignition.Start(new IgniteConfiguration(GetConfigWithLogger())
+                {
+                    CommunicationSpi = new TcpCommunicationSpi
+                    {
+                        IdleConnectionTimeout = TimeSpan.MinValue
+                    }
+                }));
+
+            var err = TestLogger.Entries.First(x => x.Level == LogLevel.Error);
+            Assert.IsTrue(err.NativeErrorInfo.Contains("SPI parameter failed condition check: idleConnTimeout > 0"));
+            Assert.AreEqual("org.apache.ignite.internal.IgniteKernal", err.Category);
+            Assert.IsNull(err.Exception);
+        }
+
+        /// <summary>
+        /// Tests startup error in .NET.
+        /// </summary>
+        [Test]
+        public void TestStartupDotNetError()
+        {
+            // Invalid bean
+            Assert.Throws<IgniteException>(() =>
+                Ignition.Start(new IgniteConfiguration(GetConfigWithLogger())
+                {
+                    LifecycleBeans = new[] {new FailBean()}
+                }));
+
+            var err = TestLogger.Entries.First(x => x.Level == LogLevel.Error);
+            Assert.IsInstanceOf<ArithmeticException>(err.Exception);
+        }
+
+        /// <summary>
+        /// Tests that .NET exception propagates through Java to the log.
+        /// </summary>
+        [Test]
+        public void TestDotNetErrorPropagation()
+        {
+            // Start 2 nodes: PlatformNativeException does not occur in local scenario
+            using (var ignite = Ignition.Start(GetConfigWithLogger()))
+            using (Ignition.Start(new IgniteConfiguration(TestUtils.GetTestConfiguration()) {GridName = "1"}))
+            {
+                var compute = ignite.GetCluster().ForRemotes().GetCompute();
+
+                Assert.Throws<ArithmeticException>(() => compute.Call(new FailFunc()));
+
+                // Log updates may not arrive immediately
+                TestUtils.WaitForCondition(() => TestLogger.Entries.Any(x => x.Exception != null), 3000);
+
+                var errFromJava = TestLogger.Entries.Single(x => x.Exception != null);
+                Assert.AreEqual("Error in func.", ((ArithmeticException) errFromJava.Exception.InnerException).Message);
+            }
+        }
+
+        /// <summary>
+        /// Tests the <see cref="QueryEntity"/> validation.
+        /// </summary>
+        [Test]
+        public void TestQueryEntityValidation()
+        {
+            var cacheCfg = new CacheConfiguration("cache1", new QueryEntity(typeof(uint), typeof(ulong))
+            {
+                Fields = new[]
+                {
+                    new QueryField("myField", typeof(ushort))
+                }
+            });
+
+            var cfg = new IgniteConfiguration(GetConfigWithLogger())
+            {
+                CacheConfiguration = new[]
+                {
+                    cacheCfg
+                }
+            };
+
+            using (var ignite = Ignition.Start(cfg))
+            {
+                // Check static and dynamic cache start
+                cacheCfg.Name = "cache2";
+                ignite.CreateCache<int, string>(cacheCfg);
+
+                var warns = TestLogger.Entries.Where(x => x.Level == LogLevel.Warn && x.Args != null)
+                    .Select(x => string.Format(x.Message, x.Args)).ToList();
+
+                Assert.AreEqual(6, warns.Count);
+
+                Assert.AreEqual("Validating cache configuration 'cache1', QueryEntity 'java.lang.Integer:java.lang." +
+                                "Long': Type 'System.UInt32' maps to Java type 'java.lang.Integer' using unchecked " +
+                                "conversion. This may cause issues in SQL queries. You can use 'System.Int32' " +
+                                "instead to achieve direct mapping.", warns[0]);
+
+                Assert.AreEqual("Validating cache configuration 'cache1', QueryEntity 'java.lang.Integer:java.lang." +
+                                "Long': Type 'System.UInt64' maps to Java type 'java.lang.Long' using unchecked " +
+                                "conversion. This may cause issues in SQL queries. You can use 'System.Int64' " +
+                                "instead to achieve direct mapping.", warns[1]);
+
+                Assert.AreEqual("Validating cache configuration 'cache1', QueryEntity 'java.lang.Integer:java.lang." +
+                                "Long', QueryField 'myField': Type 'System.UInt16' maps to Java type 'java.lang." +
+                                "Short' using unchecked conversion. This may cause issues in SQL queries. You " +
+                                "can use 'System.Int16' instead to achieve direct mapping.", warns[2]);
+
+                Assert.AreEqual("Validating cache configuration 'cache2', QueryEntity 'java.lang.Integer:java.lang." +
+                                "Long': Type 'System.UInt32' maps to Java type 'java.lang.Integer' using unchecked " +
+                                "conversion. This may cause issues in SQL queries. You can use 'System.Int32' " +
+                                "instead to achieve direct mapping.", warns[3]);
+
+                Assert.AreEqual("Validating cache configuration 'cache2', QueryEntity 'java.lang.Integer:java.lang." +
+                                "Long': Type 'System.UInt64' maps to Java type 'java.lang.Long' using unchecked " +
+                                "conversion. This may cause issues in SQL queries. You can use 'System.Int64' " +
+                                "instead to achieve direct mapping.", warns[4]);
+
+                Assert.AreEqual("Validating cache configuration 'cache2', QueryEntity 'java.lang.Integer:java.lang." +
+                                "Long', QueryField 'myField': Type 'System.UInt16' maps to Java type 'java.lang." +
+                                "Short' using unchecked conversion. This may cause issues in SQL queries. You " +
+                                "can use 'System.Int16' instead to achieve direct mapping.", warns[5]);
+            }
+        }
+
+        /// <summary>
+        /// Tests the <see cref="LoggerExtensions"/> methods.
+        /// </summary>
+        [Test]
+        public void TestExtensions()
+        {
+            var log = new TestLogger(LogLevel.Trace);
+            var ex = new FieldAccessException("abc");
+
+            // Log
+            log.Log(LogLevel.Trace, "trace");
+            CheckLastMessage(LogLevel.Trace, "trace");
+
+            log.Log(LogLevel.Debug, "msg {0} {1}", 1, "2");
+            CheckLastMessage(LogLevel.Debug, "msg {0} {1}", new object[] { 1, "2" }, CultureInfo.InvariantCulture);
+
+            log.Log(LogLevel.Info, ex, "msg");
+            CheckLastMessage(LogLevel.Info, "msg", e: ex);
+
+            log.Log(LogLevel.Warn, ex, "msg {0}", 1);
+            CheckLastMessage(LogLevel.Warn, "msg {0}", new object[] {1}, CultureInfo.InvariantCulture, e: ex);
+
+            // Trace
+            log.Trace("trace");
+            CheckLastMessage(LogLevel.Trace, "trace");
+
+            log.Trace("msg {0} {1}", 1, "2");
+            CheckLastMessage(LogLevel.Trace, "msg {0} {1}", new object[] { 1, "2" }, CultureInfo.InvariantCulture);
+
+            log.Trace(ex, "msg");
+            CheckLastMessage(LogLevel.Trace, "msg", e: ex);
+
+            log.Trace(ex, "msg {0}", 1);
+            CheckLastMessage(LogLevel.Trace, "msg {0}", new object[] { 1 }, CultureInfo.InvariantCulture, e: ex);
+
+            // Debug
+            log.Debug("test");
+            CheckLastMessage(LogLevel.Debug, "test");
+
+            log.Debug("msg {0} {1}", 1, "2");
+            CheckLastMessage(LogLevel.Debug, "msg {0} {1}", new object[] { 1, "2" }, CultureInfo.InvariantCulture);
+
+            log.Debug(ex, "msg");
+            CheckLastMessage(LogLevel.Debug, "msg", e: ex);
+
+            log.Debug(ex, "msg {0}", 1);
+            CheckLastMessage(LogLevel.Debug, "msg {0}", new object[] { 1 }, CultureInfo.InvariantCulture, e: ex);
+
+            // Info
+            log.Info("test");
+            CheckLastMessage(LogLevel.Info, "test");
+
+            log.Info("msg {0} {1}", 1, "2");
+            CheckLastMessage(LogLevel.Info, "msg {0} {1}", new object[] { 1, "2" }, CultureInfo.InvariantCulture);
+
+            log.Info(ex, "msg");
+            CheckLastMessage(LogLevel.Info, "msg", e: ex);
+
+            log.Info(ex, "msg {0}", 1);
+            CheckLastMessage(LogLevel.Info, "msg {0}", new object[] { 1 }, CultureInfo.InvariantCulture, e: ex);
+
+            // Warn
+            log.Warn("test");
+            CheckLastMessage(LogLevel.Warn, "test");
+
+            log.Warn("msg {0} {1}", 1, "2");
+            CheckLastMessage(LogLevel.Warn, "msg {0} {1}", new object[] { 1, "2" }, CultureInfo.InvariantCulture);
+
+            log.Warn(ex, "msg");
+            CheckLastMessage(LogLevel.Warn, "msg", e: ex);
+
+            log.Warn(ex, "msg {0}", 1);
+            CheckLastMessage(LogLevel.Warn, "msg {0}", new object[] { 1 }, CultureInfo.InvariantCulture, e: ex);
+
+            // Error
+            log.Error("test");
+            CheckLastMessage(LogLevel.Error, "test");
+
+            log.Error("msg {0} {1}", 1, "2");
+            CheckLastMessage(LogLevel.Error, "msg {0} {1}", new object[] { 1, "2" }, CultureInfo.InvariantCulture);
+
+            log.Error(ex, "msg");
+            CheckLastMessage(LogLevel.Error, "msg", e: ex);
+
+            log.Error(ex, "msg {0}", 1);
+            CheckLastMessage(LogLevel.Error, "msg {0}", new object[] { 1 }, CultureInfo.InvariantCulture, e: ex);
+
+            // GetLogger
+            var catLog = log.GetLogger("myCategory");
+            catLog.Info("info");
+            CheckLastMessage(LogLevel.Info, "info", category: "myCategory");
+
+            catLog.Log(LogLevel.Info, "info", null, null, "explicitCat", null, null);
+            CheckLastMessage(LogLevel.Info, "info", category: "explicitCat");
+
+            catLog = catLog.GetLogger("newCat");
+            catLog.Info("info");
+            CheckLastMessage(LogLevel.Info, "info", category: "newCat");
+
+            catLog.Log(LogLevel.Info, "info", null, null, "explicitCat", null, null);
+            CheckLastMessage(LogLevel.Info, "info", category: "explicitCat");
+        }
+
+        /// <summary>
+        /// Checks the last message.
+        /// </summary>
+        private static void CheckLastMessage(LogLevel level, string message, object[] args = null, 
+            IFormatProvider formatProvider = null, string category = null, string nativeErr = null, Exception e = null)
+        {
+            var msg = TestLogger.Entries.Last();
+
+            Assert.AreEqual(msg.Level, level);
+            Assert.AreEqual(msg.Message, message);
+            Assert.AreEqual(msg.Args, args);
+            Assert.AreEqual(msg.FormatProvider, formatProvider);
+            Assert.AreEqual(msg.Category, category);
+            Assert.AreEqual(msg.NativeErrorInfo, nativeErr);
+            Assert.AreEqual(msg.Exception, e);
+        }
+
+        /// <summary>
+        /// Gets the configuration with logger.
+        /// </summary>
+        private static IgniteConfiguration GetConfigWithLogger(bool verbose = false)
+        {
+            return new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                Logger = new TestLogger(verbose ? LogLevel.Trace : LogLevel.Info)
+            };
+        }
+
+        /// <summary>
+        /// Test log entry.
+        /// </summary>
+        private class LogEntry
+        {
+            public LogLevel Level;
+            public string Message;
+            public object[] Args;
+            public IFormatProvider FormatProvider;
+            public string Category;
+            public string NativeErrorInfo;
+            public Exception Exception;
+
+            public override string ToString()
+            {
+                return string.Format("Level: {0}, Message: {1}, Args: {2}, FormatProvider: {3}, Category: {4}, " +
+                                     "NativeErrorInfo: {5}, Exception: {6}", Level, Message, Args, FormatProvider, 
+                                     Category, NativeErrorInfo, Exception);
+            }
+        }
+
+        /// <summary>
+        /// Test logger.
+        /// </summary>
+        private class TestLogger : ILogger
+        {
+            public static readonly List<LogEntry> Entries = new List<LogEntry>(5000);
+
+            private readonly LogLevel _minLevel;
+
+            public TestLogger(LogLevel minLevel)
+            {
+                _minLevel = minLevel;
+            }
+
+            public void Log(LogLevel level, string message, object[] args, IFormatProvider formatProvider, 
+                string category, string nativeErrorInfo, Exception ex)
+            {
+                if (!IsEnabled(level))
+                    return;
+
+                lock (Entries)
+                {
+                    Entries.Add(new LogEntry
+                    {
+                        Level = level,
+                        Message = message,
+                        Args = args,
+                        FormatProvider = formatProvider,
+                        Category = category,
+                        NativeErrorInfo = nativeErrorInfo,
+                        Exception = ex
+                    });
+                }
+            }
+
+            public bool IsEnabled(LogLevel level)
+            {
+                return level >= _minLevel;
+            }
+
+            [InstanceResource]
+            // ReSharper disable once UnusedAutoPropertyAccessor.Local
+            public IIgnite Ignite { get; set; }
+        }
+
+
+        /// <summary>
+        /// Failing lifecycle bean.
+        /// </summary>
+        private class FailBean : ILifecycleBean
+        {
+            public void OnLifecycleEvent(LifecycleEventType evt)
+            {
+                throw new ArithmeticException("Failure in bean");
+            }
+        }
+
+        /// <summary>
+        /// Failing computation.
+        /// </summary>
+        [Serializable]
+        private class FailFunc : IComputeFunc<string>
+        {
+            public string Invoke()
+            {
+                throw new ArithmeticException("Error in func.");
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/DefaultLoggerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/DefaultLoggerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/DefaultLoggerTest.cs
new file mode 100644
index 0000000..6623416
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/DefaultLoggerTest.cs
@@ -0,0 +1,114 @@
+\ufeff/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Log
+{
+    using System;
+    using System.IO;
+    using System.Linq;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Impl.Common;
+    using NUnit.Framework;
+    using LogLevel = Apache.Ignite.Core.Log.LogLevel;
+
+    /// <summary>
+    /// Tests the default logger.
+    /// </summary>
+    public class DefaultLoggerTest
+    {
+        /// <summary>
+        /// Tests that default Java mechanism is used when there is no custom logger.
+        /// </summary>
+        [Test]
+        public void TestJavaLogger()
+        {
+            // Run the test in a separate process because log4jlogger has some static state,
+            // and after Ignite has been started once, it is not possible to start a new node 
+            // with a different logger config.
+            const string envVar = "DefaultLoggerTest.TestJavaLogger";
+
+            if (Environment.GetEnvironmentVariable(envVar) == "true")
+            {
+                // Delete all log files from the work dir
+                Func<string[]> getLogs = () =>
+                    Directory.GetFiles(IgniteHome.Resolve(null), "dotnet-logger-test.log", SearchOption.AllDirectories);
+
+                getLogs().ToList().ForEach(File.Delete);
+
+                var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration(false))
+                {
+                    SpringConfigUrl = @"config\log\custom-log.xml",
+                    CacheConfiguration = new[]
+                    {
+                        new CacheConfiguration("cache1", new QueryEntity(typeof(uint), typeof(ulong)))
+                    }
+                };
+
+                // Start Ignite and verify file log
+                using (var ignite = Ignition.Start(cfg))
+                {
+                    // Log with all levels
+                    var log = ignite.Logger;
+                    var levels = new[] {LogLevel.Trace, LogLevel.Info, LogLevel.Debug, LogLevel.Warn, LogLevel.Error};
+
+                    foreach (var level in levels)
+                    {
+                        var ex = new Exception("EXCEPTION_TEST_" + level);
+
+                        log.Log(level, "DOTNET-" + level, null, null, "=DOTNET=", null, ex);
+                    }
+                }
+
+                using (var fs = File.Open(getLogs().Single(), FileMode.Open, FileAccess.Read, FileShare.ReadWrite))
+                {
+                    var log = new StreamReader(fs).ReadToEnd();
+
+                    // Check output from Java:
+                    Assert.IsTrue(log.Contains(">>> Topology snapshot."));
+
+                    // Check output from .NET:
+                    Assert.IsTrue(log.Contains("Starting Ignite.NET " + typeof(Ignition).Assembly.GetName().Version));
+
+                    Assert.IsTrue(log.Contains(
+                        "Validating cache configuration 'cache1', QueryEntity 'java.lang.Integer:java.lang." +
+                        "Long': Type 'System.UInt32' maps to Java type 'java.lang.Integer' using unchecked " +
+                        "conversion. This may cause issues in SQL queries. You can use 'System.Int32' " +
+                        "instead to achieve direct mapping."));
+
+
+                    // Check custom log output (trace is disabled, errors are logged from Warn and up):
+                    Assert.IsTrue(log.Contains("[INFO ][main][=DOTNET=] DOTNET-Info"));
+
+                    Assert.IsTrue(log.Contains("[DEBUG][main][=DOTNET=] DOTNET-Debug"));
+
+                    Assert.IsTrue(log.Contains("[WARN ][main][=DOTNET=] DOTNET-Warn"));
+                    Assert.IsTrue(log.Contains("class org.apache.ignite.IgniteException: " +
+                                               "Platform error:System.Exception: EXCEPTION_TEST_Warn"));
+
+                    Assert.IsTrue(log.Contains("[ERROR][main][=DOTNET=] DOTNET-Error"));
+                    Assert.IsTrue(log.Contains("class org.apache.ignite.IgniteException: " +
+                                               "Platform error:System.Exception: EXCEPTION_TEST_Error"));
+                }
+            }
+            else
+            {
+                Environment.SetEnvironmentVariable(envVar, "true");
+                TestUtils.RunTestInNewProcess(GetType().FullName, "TestJavaLogger");
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
index c0b8599..d3851db 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
@@ -21,9 +21,9 @@ namespace Apache.Ignite.Core.Tests
     using System;
     using System.Collections.Concurrent;
     using System.Collections.Generic;
+    using System.Diagnostics;
     using System.Linq;
     using System.Threading;
-    using Apache.Ignite.Core.Discovery;
     using Apache.Ignite.Core.Discovery.Tcp;
     using Apache.Ignite.Core.Discovery.Tcp.Static;
     using Apache.Ignite.Core.Impl;
@@ -337,5 +337,30 @@ namespace Apache.Ignite.Core.Tests
                 JvmClasspath = CreateTestClasspath()
             };
         }
+
+        /// <summary>
+        /// Runs the test in new process.
+        /// </summary>
+        public static void RunTestInNewProcess(string fixtureName, string testName)
+        {
+            var procStart = new ProcessStartInfo
+            {
+                FileName = typeof(TestUtils).Assembly.Location,
+                Arguments = fixtureName + " " + testName,
+                CreateNoWindow = true,
+                UseShellExecute = false,
+                RedirectStandardOutput = true,
+                RedirectStandardError = true
+            };
+
+            var proc = System.Diagnostics.Process.Start(procStart);
+
+            Assert.IsNotNull(proc);
+
+            Console.WriteLine(proc.StandardOutput.ReadToEnd());
+            Console.WriteLine(proc.StandardError.ReadToEnd());
+            Assert.IsTrue(proc.WaitForExit(15000));
+            Assert.AreEqual(0, proc.ExitCode);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/165c0cf9/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 59bf090..db2a96b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -308,7 +308,6 @@
     <Compile Include="Impl\Common\IgniteConfigurationXmlSerializer.cs" />
     <Compile Include="Impl\Common\IgniteHome.cs" />
     <Compile Include="Impl\Common\LoadedAssembliesResolver.cs" />
-    <Compile Include="Impl\Common\Logger.cs" />
     <Compile Include="Impl\Common\ResizeableArray.cs" />
     <Compile Include="Impl\Common\TypeCaster.cs" />
     <Compile Include="Impl\Common\TypeStringConverter.cs" />
@@ -343,6 +342,7 @@
     <Compile Include="Impl\Ignite.cs" />
     <Compile Include="Impl\IgniteManager.cs" />
     <Compile Include="Impl\IgniteProxy.cs" />
+    <Compile Include="Impl\Log\JavaLogger.cs" />
     <Compile Include="Impl\PlatformTarget.cs" />
     <Compile Include="Impl\IgniteUtils.cs" />
     <Compile Include="Impl\Handle\Handle.cs" />
@@ -435,6 +435,10 @@
     <Compile Include="Interop\Package-Info.cs" />
     <Compile Include="Lifecycle\ClientReconnectEventArgs.cs" />
     <Compile Include="Lifecycle\Package-Info.cs" />
+    <Compile Include="Log\CategoryLogger.cs" />
+    <Compile Include="Log\ILogger.cs" />
+    <Compile Include="Log\LoggerExtensions.cs" />
+    <Compile Include="Log\LogLevel.cs" />
     <Compile Include="Messaging\Package-Info.cs" />
     <Compile Include="Package-Info.cs" />
     <Compile Include="Lifecycle\ILifecycleBean.cs" />