You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by GitBox <gi...@apache.org> on 2021/08/07 16:50:43 UTC

[GitHub] [skywalking-python] Superskyyy opened a new pull request #147: Feature: collect and report logs

Superskyyy opened a new pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147


   This is a OSPP Summer 2021 project supervised by @Humbertzhang | apache/skywalking#7118
   
   The feature implements optional log reporter functionalities in alignment with the SkyWalking Java agent. 
   
   - Intercepts logs from Python logging module.
   - Reports logs via a new temporary gRPC channel(to be removed in the future).
   - Supports unformatted/ formatted logs with custom layout.
   - Supports custom logging level threshold.
   - Bumps up submodule to support log collection protocols.
   - Bumps up skywalking-eyes and adds a config entry to ignore `.gitignore` during license checks.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] Humbertzhang commented on a change in pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
Humbertzhang commented on a change in pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#discussion_r684662955



##########
File path: skywalking/log/sw_logging.py
##########
@@ -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.
+#
+
+import logging
+
+from skywalking import config, agent
+from skywalking.protocol.common.Common_pb2 import KeyStringValuePair
+from skywalking.protocol.logging.Logging_pb2 import LogData, LogDataBody, TraceContext, LogTags, TextLog
+from skywalking.trace.context import get_context
+
+
+def install():
+    from logging import Logger
+
+    _handle = Logger.handle
+    log_reporter_level = logging.getLevelName(config.log_grpc_reporter_level)  # type: int
+
+    def _sw_handle(self, record):
+        if self.name == "skywalking":  # Ignore SkyWalking internal logger
+            return _handle(self, record)
+
+        if record.levelno < log_reporter_level:
+            return _handle(self, record)
+
+        def build_log_tags() -> LogTags:
+            core_tags = [
+                KeyStringValuePair(key='level', value=str(record.levelname)),
+                KeyStringValuePair(key='logger', value=str(record.name)),
+                KeyStringValuePair(key='thread', value=str(record.threadName))
+            ]
+            l_tags = LogTags()
+            l_tags.data.extend(core_tags)
+
+            if config.log_grpc_reporter_formatted:
+                return l_tags
+
+            for i, arg in enumerate(record.args):
+                l_tags.data.append(KeyStringValuePair(key='argument.' + str(i), value=str(arg)))
+
+            if record.exc_info:
+                l_tags.data.append(KeyStringValuePair(key='exception', value=str(record.exc_info)))
+
+            return l_tags
+
+        context = get_context()
+
+        log_data = LogData(
+            timestamp=round(record.created * 1000),
+            service=config.service_name,
+            serviceInstance=config.service_instance,
+            body=LogDataBody(
+                type='text',
+                text=TextLog(
+                    text=transform(record)
+                )
+            ),
+            traceContext=TraceContext(
+                traceId=str(context.segment.related_traces[0]),
+                traceSegmentId=str(context.segment.segment_id),
+                spanId=context.active_span().sid if context.active_span() else -1
+            ),
+            tags=build_log_tags(),
+        )
+
+        _handle(self=self, record=record)
+
+        agent.archive_log(log_data)
+
+    Logger.handle = _sw_handle
+
+    def transform(record) -> str:
+        if config.log_grpc_reporter_formatted:
+            layout = config.log_grpc_reporter_layout  # type: str
+            if layout:
+                from logging import Formatter
+                formatter = Formatter(fmt=layout)

Review comment:
       Is there a way not to init a new formatter object at every time?

##########
File path: skywalking/agent/protocol/grpc_log.py
##########
@@ -0,0 +1,103 @@
+#
+# 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.
+#
+
+import logging
+import traceback
+from queue import Queue, Empty
+from time import time
+
+import grpc
+from skywalking.protocol.logging.Logging_pb2 import LogData
+
+from skywalking import config
+from skywalking.agent import Protocol
+from skywalking.agent.protocol.interceptors import header_adder_interceptor
+from skywalking.client.grpc import GrpcServiceManagementClient, GrpcLogDataReportService
+from skywalking.loggings import logger
+
+
+class GrpcLogProtocol(Protocol):
+    def __init__(self):
+        self.properties_sent = False
+        self.state = None
+
+        if config.force_tls:
+            self.channel = grpc.secure_channel(config.log_grpc_collector_address, grpc.ssl_channel_credentials(),
+                                               options=(('grpc.max_send_message_length',
+                                                         config.log_grpc_reporter_max_message_size),))
+        else:
+            self.channel = grpc.insecure_channel(config.log_grpc_collector_address,
+                                                 options=(('grpc.max_send_message_length',
+                                                           config.log_grpc_reporter_max_message_size),))
+        if config.authentication:
+            self.channel = grpc.intercept_channel(
+                self.channel, header_adder_interceptor('authentication', config.authentication)
+            )
+
+        self.channel.subscribe(self._cb, try_to_connect=True)
+        self.service_management = GrpcServiceManagementClient(self.channel)
+        self.log_reporter = GrpcLogDataReportService(self.channel)
+
+    def _cb(self, state):
+        logger.debug('grpc log reporter channel connectivity changed, [%s -> %s]', self.state, state)
+        self.state = state
+
+    def heartbeat(self):
+        try:
+            if not self.properties_sent:
+                self.service_management.send_instance_props()
+                self.properties_sent = True
+
+            logger.debug(
+                'log reporter service heart beats, [%s], [%s]',
+                config.service_name,
+                config.service_instance,
+            )
+            self.service_management.send_heart_beat()
+
+        except grpc.RpcError:
+            self.on_error()
+
+    def on_error(self):
+        traceback.print_exc() if logger.isEnabledFor(logging.DEBUG) else None
+        self.channel.unsubscribe(self._cb)
+        self.channel.subscribe(self._cb, try_to_connect=True)
+
+    def report(self, queue: Queue, block: bool = True):
+        start = time()
+
+        def generator():
+            while True:
+                try:
+                    timeout = config.QUEUE_TIMEOUT - int(time() - start)  # type: int
+                    if timeout <= 0:  # this is to make sure we exit eventually instead of being fed continuously
+                        return
+                    log_data = queue.get(block=block, timeout=timeout)  # type: LogData
+                except Empty:
+                    return
+
+                queue.task_done()
+
+                logger.debug('Reporting Log %s', log_data)

Review comment:
       No need to print the `log_data` object out.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] wu-sheng commented on pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#issuecomment-894727783


   And after this change, a new section should be added into main repo's doc, https://github.com/apache/skywalking/blob/master/docs/en/setup/backend/log-analyzer.md#java-agents-toolkits, which should link to this required doc https://github.com/apache/skywalking-python/pull/147#discussion_r684698040


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] Humbertzhang commented on pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
Humbertzhang commented on pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#issuecomment-894758072


   Closes https://github.com/apache/skywalking/issues/7118


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] Humbertzhang commented on pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
Humbertzhang commented on pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#issuecomment-894694612


   Test it manually and it generally looks good to me. You should add test for it after this PR merged. 
   @kezhenxu94 Should this feature was tested by using [skywalking-infra-e2e](https://github.com/apache/skywalking-infra-e2e) ? 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] Superskyyy commented on pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
Superskyyy commented on pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#issuecomment-894732765


   > > Test it manually and it generally looks good to me. You should add test for it after this PR merged.
   > > @kezhenxu94, I am wondering if this feature should be tested by using [skywalking-infra-e2e](https://github.com/apache/skywalking-infra-e2e) ?
   > 
   > This is recommended, that mechanism makes sure we wouldn't break it in the future.
   
   Will do that next :)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] Humbertzhang commented on a change in pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
Humbertzhang commented on a change in pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#discussion_r684664142



##########
File path: skywalking/config.py
##########
@@ -67,6 +67,19 @@
                          os.getenv('SW_AGENT_PROFILE_ACTIVE') == 'True' else False  # type: bool
 profile_task_query_interval = int(os.getenv('SW_PROFILE_TASK_QUERY_INTERVAL') or '20')
 
+# NOTE - Log reporting requires a separate channel, will merge in the future.

Review comment:
       What is this comment means?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] Superskyyy commented on a change in pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
Superskyyy commented on a change in pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#discussion_r684720085



##########
File path: docs/EnvVars.md
##########
@@ -32,3 +32,10 @@ Environment Variable | Description | Default
 | `SW_CELERY_PARAMETERS_LENGTH`| The maximum length of `celery` functions parameters, longer than this will be truncated, 0 turns off  | `512` |
 | `SW_AGENT_PROFILE_ACTIVE` | If `True`, Python agent will enable profile when user create a new profile task. Otherwise disable profile. | `False` |
 | `SW_PROFILE_TASK_QUERY_INTERVAL` | The number of seconds between two profile task query. | `20` |
+| `SW_AGENT_LOG_REPORTER_ACTIVE` | If `True`, Python agent will report collected logs to the OAP or Satellite. Otherwise, it disables the feature. | `False` |

Review comment:
       @wu-sheng I just added a detailed guide, please see if that is sufficient.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] Humbertzhang commented on a change in pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
Humbertzhang commented on a change in pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#discussion_r684664142



##########
File path: skywalking/config.py
##########
@@ -67,6 +67,19 @@
                          os.getenv('SW_AGENT_PROFILE_ACTIVE') == 'True' else False  # type: bool
 profile_task_query_interval = int(os.getenv('SW_PROFILE_TASK_QUERY_INTERVAL') or '20')
 
+# NOTE - Log reporting requires a separate channel, will merge in the future.

Review comment:
       What is this comment means?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] tom-pytel commented on pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
tom-pytel commented on pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#issuecomment-894787230


   So this is grpc only? No http or kafka protocol?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] kezhenxu94 commented on pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#issuecomment-894722342


   > @kezhenxu94, I am wondering if this feature should be tested by using [skywalking-infra-e2e](https://github.com/apache/skywalking-infra-e2e) ? 
   
   I think it's not mandatory, though it would be a bonus if @Superskyyy can adopt skywalking-infra-e2e in the new test. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] Humbertzhang merged pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
Humbertzhang merged pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] Superskyyy commented on a change in pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
Superskyyy commented on a change in pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#discussion_r684700923



##########
File path: skywalking/agent/protocol/grpc_log.py
##########
@@ -0,0 +1,103 @@
+#
+# 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.
+#
+
+import logging
+import traceback
+from queue import Queue, Empty
+from time import time
+
+import grpc
+from skywalking.protocol.logging.Logging_pb2 import LogData
+
+from skywalking import config
+from skywalking.agent import Protocol
+from skywalking.agent.protocol.interceptors import header_adder_interceptor
+from skywalking.client.grpc import GrpcServiceManagementClient, GrpcLogDataReportService
+from skywalking.loggings import logger
+
+
+class GrpcLogProtocol(Protocol):
+    def __init__(self):
+        self.properties_sent = False
+        self.state = None
+
+        if config.force_tls:
+            self.channel = grpc.secure_channel(config.log_grpc_collector_address, grpc.ssl_channel_credentials(),
+                                               options=(('grpc.max_send_message_length',
+                                                         config.log_grpc_reporter_max_message_size),))
+        else:
+            self.channel = grpc.insecure_channel(config.log_grpc_collector_address,
+                                                 options=(('grpc.max_send_message_length',
+                                                           config.log_grpc_reporter_max_message_size),))
+        if config.authentication:
+            self.channel = grpc.intercept_channel(
+                self.channel, header_adder_interceptor('authentication', config.authentication)
+            )
+
+        self.channel.subscribe(self._cb, try_to_connect=True)
+        self.service_management = GrpcServiceManagementClient(self.channel)
+        self.log_reporter = GrpcLogDataReportService(self.channel)
+
+    def _cb(self, state):
+        logger.debug('grpc log reporter channel connectivity changed, [%s -> %s]', self.state, state)
+        self.state = state
+
+    def heartbeat(self):
+        try:
+            if not self.properties_sent:
+                self.service_management.send_instance_props()
+                self.properties_sent = True
+
+            logger.debug(
+                'log reporter service heart beats, [%s], [%s]',
+                config.service_name,
+                config.service_instance,
+            )
+            self.service_management.send_heart_beat()
+
+        except grpc.RpcError:
+            self.on_error()
+
+    def on_error(self):
+        traceback.print_exc() if logger.isEnabledFor(logging.DEBUG) else None
+        self.channel.unsubscribe(self._cb)
+        self.channel.subscribe(self._cb, try_to_connect=True)
+
+    def report(self, queue: Queue, block: bool = True):
+        start = time()
+
+        def generator():
+            while True:
+                try:
+                    timeout = config.QUEUE_TIMEOUT - int(time() - start)  # type: int
+                    if timeout <= 0:  # this is to make sure we exit eventually instead of being fed continuously
+                        return
+                    log_data = queue.get(block=block, timeout=timeout)  # type: LogData
+                except Empty:
+                    return
+
+                queue.task_done()
+
+                logger.debug('Reporting Log %s', log_data)

Review comment:
       Ok I will remove the argument.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] tom-pytel commented on pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
tom-pytel commented on pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#issuecomment-894791471


   > For this PR, yes, we need iterations to support http and kafka protocol
   
   This functionality should be pushed into the protocols themselves, which would also remove the separate channel for grpc as the author said.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] kezhenxu94 commented on pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#issuecomment-894795750


   > This functionality should be pushed into the protocols themselves, which would also remove the separate channel for grpc as the author said.
   
   @Humbertzhang will be working to remove the separate channel for logs by reusing the same gRPC protocol / channel.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] kezhenxu94 commented on pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#issuecomment-894791286


   > So this is grpc only? No http or kafka protocol?
   
   For this PR, yes, we need iterations to support http and kafka protocol


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] wu-sheng commented on pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#issuecomment-894727435


   > Test it manually and it generally looks good to me. You should add test for it after this PR merged.
   > @kezhenxu94, I am wondering if this feature should be tested by using [skywalking-infra-e2e](https://github.com/apache/skywalking-infra-e2e) ?
   
   This is recommended, that mechanism makes sure we wouldn't break it in the future.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] wu-sheng commented on a change in pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#discussion_r684698040



##########
File path: docs/EnvVars.md
##########
@@ -32,3 +32,10 @@ Environment Variable | Description | Default
 | `SW_CELERY_PARAMETERS_LENGTH`| The maximum length of `celery` functions parameters, longer than this will be truncated, 0 turns off  | `512` |
 | `SW_AGENT_PROFILE_ACTIVE` | If `True`, Python agent will enable profile when user create a new profile task. Otherwise disable profile. | `False` |
 | `SW_PROFILE_TASK_QUERY_INTERVAL` | The number of seconds between two profile task query. | `20` |
+| `SW_AGENT_LOG_REPORTER_ACTIVE` | If `True`, Python agent will report collected logs to the OAP or Satellite. Otherwise, it disables the feature. | `False` |

Review comment:
       Besides this doc change, let's add a specific doc to show users how to use it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] Superskyyy commented on a change in pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
Superskyyy commented on a change in pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#discussion_r684702624



##########
File path: skywalking/log/sw_logging.py
##########
@@ -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.
+#
+
+import logging
+
+from skywalking import config, agent
+from skywalking.protocol.common.Common_pb2 import KeyStringValuePair
+from skywalking.protocol.logging.Logging_pb2 import LogData, LogDataBody, TraceContext, LogTags, TextLog
+from skywalking.trace.context import get_context
+
+
+def install():
+    from logging import Logger
+
+    _handle = Logger.handle
+    log_reporter_level = logging.getLevelName(config.log_grpc_reporter_level)  # type: int
+
+    def _sw_handle(self, record):
+        if self.name == "skywalking":  # Ignore SkyWalking internal logger
+            return _handle(self, record)
+
+        if record.levelno < log_reporter_level:
+            return _handle(self, record)
+
+        def build_log_tags() -> LogTags:
+            core_tags = [
+                KeyStringValuePair(key='level', value=str(record.levelname)),
+                KeyStringValuePair(key='logger', value=str(record.name)),
+                KeyStringValuePair(key='thread', value=str(record.threadName))
+            ]
+            l_tags = LogTags()
+            l_tags.data.extend(core_tags)
+
+            if config.log_grpc_reporter_formatted:
+                return l_tags
+
+            for i, arg in enumerate(record.args):
+                l_tags.data.append(KeyStringValuePair(key='argument.' + str(i), value=str(arg)))
+
+            if record.exc_info:
+                l_tags.data.append(KeyStringValuePair(key='exception', value=str(record.exc_info)))
+
+            return l_tags
+
+        context = get_context()
+
+        log_data = LogData(
+            timestamp=round(record.created * 1000),
+            service=config.service_name,
+            serviceInstance=config.service_instance,
+            body=LogDataBody(
+                type='text',
+                text=TextLog(
+                    text=transform(record)
+                )
+            ),
+            traceContext=TraceContext(
+                traceId=str(context.segment.related_traces[0]),
+                traceSegmentId=str(context.segment.segment_id),
+                spanId=context.active_span().sid if context.active_span() else -1
+            ),
+            tags=build_log_tags(),
+        )
+
+        _handle(self=self, record=record)
+
+        agent.archive_log(log_data)
+
+    Logger.handle = _sw_handle
+
+    def transform(record) -> str:
+        if config.log_grpc_reporter_formatted:
+            layout = config.log_grpc_reporter_layout  # type: str
+            if layout:
+                from logging import Formatter
+                formatter = Formatter(fmt=layout)

Review comment:
       There certainly is! Thanks for pointing that out. I'm moving the initialization part to the outer scope.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] Superskyyy commented on pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
Superskyyy commented on pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#issuecomment-894756940


   > Thanks. Please update the main repo doc linking to this doc.
   
   Certainly.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [skywalking-python] Humbertzhang edited a comment on pull request #147: Feature: collect and report logs

Posted by GitBox <gi...@apache.org>.
Humbertzhang edited a comment on pull request #147:
URL: https://github.com/apache/skywalking-python/pull/147#issuecomment-894694612


   Test it manually and it generally looks good to me. You should add test for it after this PR merged. 
   @kezhenxu94, I am wondering if this feature should be tested by using [skywalking-infra-e2e](https://github.com/apache/skywalking-infra-e2e) ? 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@skywalking.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org